/** * Set transport IO event polling based on SSL error. * * Returns -1 on permanent errors. * * Locality: broker thread */ static RD_INLINE int rd_kafka_transport_ssl_io_update (rd_kafka_transport_t *rktrans, int ret, char *errstr, size_t errstr_size) { int serr = SSL_get_error(rktrans->rktrans_ssl, ret); int serr2; switch (serr) { case SSL_ERROR_WANT_READ: rd_kafka_transport_poll_set(rktrans, POLLIN); break; case SSL_ERROR_WANT_WRITE: case SSL_ERROR_WANT_CONNECT: rd_kafka_transport_poll_set(rktrans, POLLOUT); break; case SSL_ERROR_SYSCALL: if (!(serr2 = SSL_get_error(rktrans->rktrans_ssl, ret))) { if (ret == 0) errno = ECONNRESET; rd_snprintf(errstr, errstr_size, "SSL syscall error: %s", rd_strerror(errno)); } else rd_snprintf(errstr, errstr_size, "SSL syscall error number: %d: %s", serr2, rd_strerror(errno)); return -1; default: rd_kafka_ssl_error(NULL, rktrans->rktrans_rkb, errstr, errstr_size); return -1; } return 0; }
/** * Poll and serve IOs * * Locality: broker thread */ void rd_kafka_transport_io_serve (rd_kafka_transport_t *rktrans, int timeout_ms) { rd_kafka_broker_t *rkb = rktrans->rktrans_rkb; int events; if (rd_kafka_bufq_cnt(&rkb->rkb_waitresps) < rkb->rkb_max_inflight && rd_kafka_bufq_cnt(&rkb->rkb_outbufs) > 0) rd_kafka_transport_poll_set(rkb->rkb_transport, POLLOUT); if ((events = rd_kafka_transport_poll(rktrans, timeout_ms)) <= 0) return; rd_kafka_transport_poll_clear(rktrans, POLLOUT); rd_kafka_transport_io_event(rktrans, events); }
/** * Initialize and start SASL authentication. * * Returns 0 on successful init and -1 on error. * * Locality: broker thread */ int rd_kafka_sasl_client_new (rd_kafka_transport_t *rktrans, char *errstr, int errstr_size) { int r; rd_kafka_broker_t *rkb = rktrans->rktrans_rkb; rd_kafka_t *rk = rkb->rkb_rk; char *hostname, *t; sasl_callback_t callbacks[16] = { // { SASL_CB_GETOPT, (void *)rd_kafka_sasl_cb_getopt, rktrans }, { SASL_CB_LOG, (void *)rd_kafka_sasl_cb_log, rktrans }, { SASL_CB_AUTHNAME, (void *)rd_kafka_sasl_cb_getsimple, rktrans }, { SASL_CB_PASS, (void *)rd_kafka_sasl_cb_getsecret, rktrans }, { SASL_CB_ECHOPROMPT, (void *)rd_kafka_sasl_cb_chalprompt, rktrans }, { SASL_CB_GETREALM, (void *)rd_kafka_sasl_cb_getrealm, rktrans }, { SASL_CB_CANON_USER, (void *)rd_kafka_sasl_cb_canon, rktrans }, { SASL_CB_LIST_END } }; /* SASL_CB_USER is needed for PLAIN but breaks GSSAPI */ if (!strcmp(rk->rk_conf.sasl.service_name, "PLAIN")) { int endidx; /* Find end of callbacks array */ for (endidx = 0 ; callbacks[endidx].id != SASL_CB_LIST_END ; endidx++) ; callbacks[endidx].id = SASL_CB_USER; callbacks[endidx].proc = (void *)rd_kafka_sasl_cb_getsimple; endidx++; callbacks[endidx].id = SASL_CB_LIST_END; } rd_strdupa(&hostname, rktrans->rktrans_rkb->rkb_nodename); if ((t = strchr(hostname, ':'))) *t = '\0'; /* remove ":port" */ rd_rkb_dbg(rkb, SECURITY, "SASL", "Initializing SASL client: service name %s, " "hostname %s, mechanisms %s", rk->rk_conf.sasl.service_name, hostname, rk->rk_conf.sasl.mechanisms); /* Acquire or refresh ticket if kinit is configured */ rd_kafka_sasl_kinit_refresh(rkb); r = sasl_client_new(rk->rk_conf.sasl.service_name, hostname, NULL, NULL, /* no local & remote IP checks */ callbacks, 0, &rktrans->rktrans_sasl.conn); if (r != SASL_OK) { rd_snprintf(errstr, errstr_size, "%s", sasl_errstring(r, NULL, NULL)); return -1; } if (rk->rk_conf.debug & RD_KAFKA_DBG_SECURITY) { const char *avail_mechs; sasl_listmech(rktrans->rktrans_sasl.conn, NULL, NULL, " ", NULL, &avail_mechs, NULL, NULL); rd_rkb_dbg(rkb, SECURITY, "SASL", "My supported SASL mechanisms: %s", avail_mechs); } rd_kafka_transport_poll_set(rktrans, POLLIN); do { const char *out; unsigned int outlen; const char *mech = NULL; r = sasl_client_start(rktrans->rktrans_sasl.conn, rk->rk_conf.sasl.mechanisms, NULL, &out, &outlen, &mech); if (r >= 0) if (rd_kafka_sasl_send(rktrans, out, outlen, errstr, errstr_size)) return -1; } while (r == SASL_INTERACT); if (r == SASL_OK) { /* PLAIN is appearantly done here, but we still need to make sure * the PLAIN frame is sent and we get a response back (but we must * not pass the response to libsasl or it will fail). */ rktrans->rktrans_sasl.complete = 1; return 0; } else if (r != SASL_CONTINUE) { rd_snprintf(errstr, errstr_size, "SASL handshake failed (start (%d)): %s", r, sasl_errdetail(rktrans->rktrans_sasl.conn)); return -1; } return 0; }
/** * Initiate asynchronous connection attempt. * * Locality: broker thread */ rd_kafka_transport_t *rd_kafka_transport_connect (rd_kafka_broker_t *rkb, const rd_sockaddr_inx_t *sinx, char *errstr, int errstr_size) { rd_kafka_transport_t *rktrans; int s = -1; int on = 1; rkb->rkb_addr_last = sinx; s = rkb->rkb_rk->rk_conf.socket_cb(sinx->in.sin_family, SOCK_STREAM, IPPROTO_TCP, rkb->rkb_rk->rk_conf.opaque); if (s == -1) { rd_snprintf(errstr, errstr_size, "Failed to create socket: %s", socket_strerror(socket_errno)); return NULL; } #ifdef SO_NOSIGPIPE /* Disable SIGPIPE signalling for this socket on OSX */ if (setsockopt(s, SOL_SOCKET, SO_NOSIGPIPE, &on, sizeof(on)) == -1) rd_rkb_dbg(rkb, BROKER, "SOCKET", "Failed to set SO_NOSIGPIPE: %s", socket_strerror(socket_errno)); #endif /* Enable TCP keep-alives, if configured. */ if (rkb->rkb_rk->rk_conf.socket_keepalive) { #ifdef SO_KEEPALIVE if (setsockopt(s, SOL_SOCKET, SO_KEEPALIVE, (void *)&on, sizeof(on)) == SOCKET_ERROR) rd_rkb_dbg(rkb, BROKER, "SOCKET", "Failed to set SO_KEEPALIVE: %s", socket_strerror(socket_errno)); #else rd_rkb_dbg(rkb, BROKER, "SOCKET", "System does not support " "socket.keepalive.enable (SO_KEEPALIVE)"); #endif } /* Set the socket to non-blocking */ #ifdef _MSC_VER if (ioctlsocket(s, FIONBIO, &on) == SOCKET_ERROR) { rd_snprintf(errstr, errstr_size, "Failed to set socket non-blocking: %s", socket_strerror(socket_errno)); goto err; } #else { int fl = fcntl(s, F_GETFL, 0); if (fl == -1 || fcntl(s, F_SETFL, fl | O_NONBLOCK) == -1) { rd_snprintf(errstr, errstr_size, "Failed to set socket non-blocking: %s", socket_strerror(socket_errno)); goto err; } } #endif rd_rkb_dbg(rkb, BROKER, "CONNECT", "Connecting to %s (%s) " "with socket %i", rd_sockaddr2str(sinx, RD_SOCKADDR2STR_F_FAMILY | RD_SOCKADDR2STR_F_PORT), rd_kafka_secproto_names[rkb->rkb_proto], s); /* Connect to broker */ if (connect(s, (struct sockaddr *)sinx, RD_SOCKADDR_INX_LEN(sinx)) == SOCKET_ERROR && (socket_errno != EINPROGRESS #ifdef _MSC_VER && socket_errno != WSAEWOULDBLOCK #endif )) { rd_rkb_dbg(rkb, BROKER, "CONNECT", "couldn't connect to %s: %s (%i)", rd_sockaddr2str(sinx, RD_SOCKADDR2STR_F_PORT | RD_SOCKADDR2STR_F_FAMILY), socket_strerror(socket_errno), socket_errno); rd_snprintf(errstr, errstr_size, "Failed to connect to broker at %s: %s", rd_sockaddr2str(sinx, RD_SOCKADDR2STR_F_NICE), socket_strerror(socket_errno)); goto err; } /* Create transport handle */ rktrans = rd_calloc(1, sizeof(*rktrans)); rktrans->rktrans_rkb = rkb; rktrans->rktrans_s = s; rktrans->rktrans_pfd.fd = s; /* Poll writability to trigger on connection success/failure. */ rd_kafka_transport_poll_set(rktrans, POLLOUT); return rktrans; err: if (s != -1) { #ifndef _MSC_VER close(s); #else closesocket(s); #endif } return NULL; }