Skip to content

Commit

Permalink
Proper log message on SSL connection close
Browse files Browse the repository at this point in the history
Includes honouring log.connection.close=true for SSL disconnects.
  • Loading branch information
edenhill committed May 29, 2018
1 parent 6c2b339 commit f26b9b2
Show file tree
Hide file tree
Showing 4 changed files with 67 additions and 21 deletions.
9 changes: 3 additions & 6 deletions src/rdkafka_int.h
Original file line number Diff line number Diff line change
Expand Up @@ -413,17 +413,14 @@ static RD_UNUSED RD_INLINE
rd_kafka_resp_err_t rd_kafka_set_last_error (rd_kafka_resp_err_t err,
int errnox) {
if (errnox) {
#ifdef _MSC_VER
/* This is the correct way to set errno on Windows,
/* MSVC:
* This is the correct way to set errno on Windows,
* but it is still pointless due to different errnos in
* in different runtimes:
* https://social.msdn.microsoft.com/Forums/vstudio/en-US/b4500c0d-1b69-40c7-9ef5-08da1025b5bf/setting-errno-from-within-a-dll?forum=vclanguage/
* errno is thus highly deprecated, and buggy, on Windows
* when using librdkafka as a dynamically loaded DLL. */
_set_errno(errnox);
#else
errno = errnox;
#endif
rd_set_errno(errnox);
}
rd_kafka_last_error_code = err;
return err;
Expand Down
65 changes: 50 additions & 15 deletions src/rdkafka_transport.c
Original file line number Diff line number Diff line change
Expand Up @@ -387,6 +387,21 @@ void rd_kafka_transport_connect_done (rd_kafka_transport_t *rktrans,
#if WITH_SSL


/**
* @brief Clear OpenSSL error queue to get a proper error reporting in case
* the next SSL_*() operation fails.
*/
static RD_INLINE void
rd_kafka_transport_ssl_clear_error (rd_kafka_transport_t *rktrans) {
ERR_clear_error();
#ifdef _MSC_VER
WSASetLastError(0);
#else
rd_set_errno(0);
#endif
}


/**
* Serves the entire OpenSSL error queue and logs each error.
* The last error is not logged but returned in 'errstr'.
Expand Down Expand Up @@ -426,15 +441,15 @@ static char *rd_kafka_ssl_error (rd_kafka_t *rk, rd_kafka_broker_t *rkb,
}


static void rd_kafka_transport_ssl_lock_cb (int mode, int i,
const char *file, int line) {
static RD_UNUSED void
rd_kafka_transport_ssl_lock_cb (int mode, int i, const char *file, int line) {
if (mode & CRYPTO_LOCK)
mtx_lock(&rd_kafka_ssl_locks[i]);
else
mtx_unlock(&rd_kafka_ssl_locks[i]);
}

static unsigned long rd_kafka_transport_ssl_threadid_cb (void) {
static RD_UNUSED unsigned long rd_kafka_transport_ssl_threadid_cb (void) {
#ifdef _MSC_VER
/* Windows makes a distinction between thread handle
* and thread id, which means we can't use the
Expand Down Expand Up @@ -496,7 +511,7 @@ 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;
int serr2;

switch (serr)
{
Expand All @@ -509,17 +524,18 @@ rd_kafka_transport_ssl_io_update (rd_kafka_transport_t *rktrans, int ret,
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;
case SSL_ERROR_SYSCALL:
serr2 = ERR_peek_error();
if (!serr2 && !socket_errno)
rd_snprintf(errstr, errstr_size, "Disconnected");
else if (serr2)
rd_kafka_ssl_error(NULL, rktrans->rktrans_rkb,
errstr, errstr_size);
else
rd_snprintf(errstr, errstr_size,
"SSL transport error: %s",
rd_strerror(socket_errno));
return -1;

case SSL_ERROR_ZERO_RETURN:
rd_snprintf(errstr, errstr_size, "Disconnected");
Expand All @@ -542,6 +558,8 @@ rd_kafka_transport_ssl_send (rd_kafka_transport_t *rktrans,
const void *p;
size_t rlen;

rd_kafka_transport_ssl_clear_error(rktrans);

while ((rlen = rd_slice_peeker(slice, &p))) {
int r;

Expand Down Expand Up @@ -579,6 +597,8 @@ rd_kafka_transport_ssl_recv (rd_kafka_transport_t *rktrans,
while ((len = rd_buf_get_writable(rbuf, &p))) {
int r;

rd_kafka_transport_ssl_clear_error(rktrans);

r = SSL_read(rktrans->rktrans_ssl, p, (int)len);

if (unlikely(r <= 0)) {
Expand Down Expand Up @@ -668,6 +688,8 @@ static int rd_kafka_transport_ssl_connect (rd_kafka_broker_t *rkb,
goto fail;
#endif

rd_kafka_transport_ssl_clear_error(rktrans);

r = SSL_connect(rktrans->rktrans_ssl);
if (r == 1) {
/* Connected, highly unlikely since this is a
Expand Down Expand Up @@ -695,6 +717,8 @@ rd_kafka_transport_ssl_io_event (rd_kafka_transport_t *rktrans, int events) {
char errstr[512];

if (events & POLLOUT) {
rd_kafka_transport_ssl_clear_error(rktrans);

r = SSL_write(rktrans->rktrans_ssl, NULL, 0);
if (rd_kafka_transport_ssl_io_update(rktrans, r,
errstr,
Expand Down Expand Up @@ -801,6 +825,17 @@ int rd_kafka_transport_ssl_ctx_init (rd_kafka_t *rk,
int r;
SSL_CTX *ctx;

#if OPENSSL_VERSION_NUMBER >= 0x10100000
rd_kafka_dbg(rk, SECURITY, "OPENSSL", "Using OpenSSL version %s "
"(0x%lx, librdkafka built with 0x%lx)",
OpenSSL_version(OPENSSL_VERSION),
OpenSSL_version_num(),
OPENSSL_VERSION_NUMBER);
#else
rd_kafka_dbg(rk, SECURITY, "OPENSSL", "librdkafka built with OpenSSL "
"version 0x%lx", OPENSSL_VERSION_NUMBER);
#endif

if (errstr_size > 0)
errstr[0] = '\0';

Expand Down
4 changes: 4 additions & 0 deletions src/rdposix.h
Original file line number Diff line number Diff line change
Expand Up @@ -90,6 +90,10 @@
/**
* Errors
*/


#define rd_set_errno(err) (errno = (err))

#if HAVE_STRERROR_R
static RD_INLINE RD_UNUSED const char *rd_strerror(int err) {
static RD_TLS char ret[128];
Expand Down
10 changes: 10 additions & 0 deletions src/rdwin32.h
Original file line number Diff line number Diff line change
Expand Up @@ -137,6 +137,16 @@ int rd_snprintf (char *str, size_t size, const char *format, ...) {
/**
* Errors
*/

/* MSVC:
* This is the correct way to set errno on Windows,
* but it is still pointless due to different errnos in
* in different runtimes:
* https://social.msdn.microsoft.com/Forums/vstudio/en-US/b4500c0d-1b69-40c7-9ef5-08da1025b5bf/setting-errno-from-within-a-dll?forum=vclanguage/
* errno is thus highly deprecated, and buggy, on Windows
* when using librdkafka as a dynamically loaded DLL. */
#define rd_set_errno(err) _set_errno((err))

static RD_INLINE RD_UNUSED const char *rd_strerror(int err) {
static RD_TLS char ret[128];

Expand Down

0 comments on commit f26b9b2

Please sign in to comment.