Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add multiple kerberos authentication environment adaptation #4817

Open
wants to merge 5 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 5 additions & 1 deletion src/rdkafka_broker.c
Original file line number Diff line number Diff line change
Expand Up @@ -4926,7 +4926,11 @@ rd_kafka_broker_t *rd_kafka_broker_add(rd_kafka_t *rk,
rkb->rkb_proto = proto;
rkb->rkb_port = port;
rkb->rkb_origname = rd_strdup(name);

if (rk->rk_conf.sasl.krb5_path) {
rkb->krb5_path = rd_strdup(rk->rk_conf.sasl.krb5_path);
} else {
rkb->krb5_path = NULL;
}
mtx_init(&rkb->rkb_lock, mtx_plain);
mtx_init(&rkb->rkb_logname_lock, mtx_plain);
rkb->rkb_logname = rd_strdup(rkb->rkb_name);
Expand Down
1 change: 1 addition & 0 deletions src/rdkafka_broker.h
Original file line number Diff line number Diff line change
Expand Up @@ -365,6 +365,7 @@ struct rd_kafka_broker_s { /* rd_kafka_broker_t */


rd_kafka_timer_t rkb_sasl_reauth_tmr;
char *krb5_path;
};

#define rd_kafka_broker_keep(rkb) rd_refcnt_add(&(rkb)->rkb_refcnt)
Expand Down
3 changes: 3 additions & 0 deletions src/rdkafka_conf.c
Original file line number Diff line number Diff line change
Expand Up @@ -944,6 +944,9 @@ static const struct rd_kafka_property rd_kafka_properties[] = {
"`sasl.kerberos.kinit.cmd` as "
"` ... -t \"%{sasl.kerberos.keytab}\"`.",
_UNSUPPORTED_WIN32_GSSAPI},
{_RK_GLOBAL, "sasl.kerberos.krb5path", _RK_C_STR, _RK(sasl.krb5_path),
"Path of krb5 conf file. ",
_UNSUPPORTED_WIN32_GSSAPI},
{_RK_GLOBAL, "sasl.kerberos.min.time.before.relogin", _RK_C_INT,
_RK(sasl.relogin_min_time),
"Minimum time in milliseconds between key refresh attempts. "
Expand Down
1 change: 1 addition & 0 deletions src/rdkafka_conf.h
Original file line number Diff line number Diff line change
Expand Up @@ -294,6 +294,7 @@ struct rd_kafka_conf_s {
mtx_t lock;
char *username;
char *password;
char *krb5_path;
#if WITH_SASL_SCRAM
/* SCRAM EVP-wrapped hash function
* (return value from EVP_shaX()) */
Expand Down
41 changes: 41 additions & 0 deletions src/rdkafka_sasl_cyrus.c
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@
* kinit cache corruption.
*/
static mtx_t rd_kafka_sasl_cyrus_kinit_lock;
static mtx_t rd_kafka_sasl_cyrus_ccache_env_lock;

/**
* @struct Per-client-instance handle
Expand All @@ -71,6 +72,9 @@ typedef struct rd_kafka_sasl_cyrus_state_s {
} rd_kafka_sasl_cyrus_state_t;


static inline void rd_kafka_sasl_cyrus_set_new_ccache_env(const char *new_value) {
setenv("KRB5_CONFIG", new_value, 1);
}

/**
* Handle received frame from broker.
Expand All @@ -93,8 +97,15 @@ static int rd_kafka_sasl_cyrus_recv(struct rd_kafka_transport_s *rktrans,
unsigned int outlen;

mtx_lock(&rktrans->rktrans_rkb->rkb_rk->rk_conf.sasl.lock);
if (rktrans->rktrans_rkb->krb5_path) {
mtx_lock(&rd_kafka_sasl_cyrus_ccache_env_lock);
rd_kafka_sasl_cyrus_set_new_ccache_env(rktrans->rktrans_rkb->krb5_path);
}
r = sasl_client_step(state->conn, size > 0 ? buf : NULL, size,
&interact, &out, &outlen);
if (rktrans->rktrans_rkb->krb5_path) {
mtx_unlock(&rd_kafka_sasl_cyrus_ccache_env_lock);
}
mtx_unlock(&rktrans->rktrans_rkb->rkb_rk->rk_conf.sasl.lock);

if (r >= 0) {
Expand Down Expand Up @@ -152,9 +163,16 @@ static int rd_kafka_sasl_cyrus_recv(struct rd_kafka_transport_s *rktrans,
const char *user, *mech, *authsrc;

mtx_lock(&rktrans->rktrans_rkb->rkb_rk->rk_conf.sasl.lock);
if (rktrans->rktrans_rkb->krb5_path) {
mtx_lock(&rd_kafka_sasl_cyrus_ccache_env_lock);
rd_kafka_sasl_cyrus_set_new_ccache_env(rktrans->rktrans_rkb->krb5_path);
}
if (sasl_getprop(state->conn, SASL_USERNAME,
(const void **)&user) != SASL_OK)
user = "(unknown)";
if (rktrans->rktrans_rkb->krb5_path) {
mtx_unlock(&rd_kafka_sasl_cyrus_ccache_env_lock);
}
mtx_unlock(&rktrans->rktrans_rkb->rkb_rk->rk_conf.sasl.lock);

if (sasl_getprop(state->conn, SASL_MECHNAME,
Expand Down Expand Up @@ -228,7 +246,14 @@ static int rd_kafka_sasl_cyrus_kinit_refresh(rd_kafka_t *rk) {
/* Prevent multiple simultaneous refreshes by the same process to
* avoid Kerberos credential cache corruption. */
mtx_lock(&rd_kafka_sasl_cyrus_kinit_lock);
if (rk->rk_conf.sasl.krb5_path) {
mtx_lock(&rd_kafka_sasl_cyrus_ccache_env_lock);
rd_kafka_sasl_cyrus_set_new_ccache_env(rk->rk_conf.sasl.krb5_path);
}
r = system(cmd);
if (rk->rk_conf.sasl.krb5_path) {
mtx_unlock(&rd_kafka_sasl_cyrus_ccache_env_lock);
}
mtx_unlock(&rd_kafka_sasl_cyrus_kinit_lock);

duration = (int)((rd_clock() - ts_start) / 1000);
Expand Down Expand Up @@ -545,9 +570,16 @@ static int rd_kafka_sasl_cyrus_client_new(rd_kafka_transport_t *rktrans,
memcpy(state->callbacks, callbacks, sizeof(callbacks));

mtx_lock(&rktrans->rktrans_rkb->rkb_rk->rk_conf.sasl.lock);
if (rktrans->rktrans_rkb->krb5_path) {
mtx_lock(&rd_kafka_sasl_cyrus_ccache_env_lock);
rd_kafka_sasl_cyrus_set_new_ccache_env(rktrans->rktrans_rkb->krb5_path);
}
r = sasl_client_new(rk->rk_conf.sasl.service_name, hostname, NULL,
NULL, /* no local & remote IP checks */
state->callbacks, 0, &state->conn);
if (rktrans->rktrans_rkb->krb5_path) {
mtx_unlock(&rd_kafka_sasl_cyrus_ccache_env_lock);
}
mtx_unlock(&rktrans->rktrans_rkb->rkb_rk->rk_conf.sasl.lock);
if (r != SASL_OK) {
rd_snprintf(errstr, errstr_size, "%s",
Expand All @@ -569,8 +601,15 @@ static int rd_kafka_sasl_cyrus_client_new(rd_kafka_transport_t *rktrans,
const char *mech = NULL;

mtx_lock(&rktrans->rktrans_rkb->rkb_rk->rk_conf.sasl.lock);
if (rktrans->rktrans_rkb->krb5_path) {
mtx_lock(&rd_kafka_sasl_cyrus_ccache_env_lock);
rd_kafka_sasl_cyrus_set_new_ccache_env(rktrans->rktrans_rkb->krb5_path);
}
r = sasl_client_start(state->conn, rk->rk_conf.sasl.mechanisms,
NULL, &out, &outlen, &mech);
if (rktrans->rktrans_rkb->krb5_path) {
mtx_unlock(&rd_kafka_sasl_cyrus_ccache_env_lock);
}
mtx_unlock(&rktrans->rktrans_rkb->rkb_rk->rk_conf.sasl.lock);

if (r >= 0)
Expand Down Expand Up @@ -689,6 +728,7 @@ void rd_kafka_sasl_cyrus_global_term(void) {
* too*/
/* sasl_done(); */
mtx_destroy(&rd_kafka_sasl_cyrus_kinit_lock);
mtx_destroy(&rd_kafka_sasl_cyrus_ccache_env_lock);
}


Expand All @@ -699,6 +739,7 @@ int rd_kafka_sasl_cyrus_global_init(void) {
int r;

mtx_init(&rd_kafka_sasl_cyrus_kinit_lock, mtx_plain);
mtx_init(&rd_kafka_sasl_cyrus_ccache_env_lock, mtx_plain);

r = sasl_client_init(NULL);
if (r != SASL_OK) {
Expand Down