Skip to content

Commit

Permalink
Avoid treating an OpenSSL error as a permanent error
Browse files Browse the repository at this point in the history
and treat unclean SSL  closes as normal ones

fixes #4293
  • Loading branch information
emasab committed May 26, 2023
1 parent bb62aad commit cd32771
Show file tree
Hide file tree
Showing 4 changed files with 21 additions and 3 deletions.
11 changes: 11 additions & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,8 @@ librdkafka v2.2.0 is a feature release:
* Store offset commit metadata in `rd_kafka_offsets_store` (@mathispesch, #4084).
* Fix a bug that happens when skipping tags, causing buffer underflow in
MetadataResponse (#4278).
* Avoid treating an OpenSSL error as a permanent error and treat unclean SSL
closes as normal ones (#4294).

## Fixes

Expand All @@ -17,6 +19,15 @@ librdkafka v2.2.0 is a feature release:
when using Confluent Platform, only when racks are set,
observers are activated and there is more than one partition.
Fixed by skipping the correct amount of bytes when tags are received.
* Avoid treating an OpenSSL error as a permanent error and treat unclean SSL
closes as normal ones. When SSL connections are closed without `close_notify`,
in OpenSSL 3.x a new type of error is set and it was interpreted as permanent
in librdkafka. It can cause a different issue depending on the RPC.
If received when waiting for OffsetForLeaderEpoch response, it triggers
an offset reset following the configured policy.
Solved by treating SSL errors as transport errors and
by setting an OpenSSL flag that allows to treat unclean SSL closes as normal
ones. These types of errors can happen it the other side doesn't support `close_notify` or if there's a TCP connection reset.


# librdkafka v2.1.1
Expand Down
4 changes: 1 addition & 3 deletions src/rdkafka_offset.c
Original file line number Diff line number Diff line change
Expand Up @@ -998,10 +998,8 @@ static void rd_kafka_toppar_handle_OffsetForLeaderEpoch(rd_kafka_t *rk,
&rk->rk_timers, &rktp->rktp_validate_tmr, rd_false,
500 * 1000 /* 500ms */,
rd_kafka_offset_validate_tmr_cb, rktp);
goto done;
}

if (!(actions & RD_KAFKA_ERR_ACTION_REFRESH)) {
} else if (actions & RD_KAFKA_ERR_ACTION_PERMANENT) {
/* Permanent error */
rd_kafka_offset_reset(
rktp, rd_kafka_broker_id(rkb),
Expand Down
1 change: 1 addition & 0 deletions src/rdkafka_request.c
Original file line number Diff line number Diff line change
Expand Up @@ -134,6 +134,7 @@ int rd_kafka_err_action(rd_kafka_broker_t *rkb,
break;

case RD_KAFKA_RESP_ERR__TRANSPORT:
case RD_KAFKA_RESP_ERR__SSL:
case RD_KAFKA_RESP_ERR__TIMED_OUT:
case RD_KAFKA_RESP_ERR_REQUEST_TIMED_OUT:
case RD_KAFKA_RESP_ERR_NOT_ENOUGH_REPLICAS_AFTER_APPEND:
Expand Down
8 changes: 8 additions & 0 deletions src/rdkafka_ssl.c
Original file line number Diff line number Diff line change
Expand Up @@ -1722,6 +1722,14 @@ int rd_kafka_ssl_ctx_init(rd_kafka_t *rk, char *errstr, size_t errstr_size) {
goto fail;


#ifdef SSL_OP_IGNORE_UNEXPECTED_EOF
/* Ignore unexpected EOF error in OpenSSL 3.x, treating
* it like a normal connection close even if
* close_notify wasn't received.
* see issue #4293 */
SSL_CTX_set_options(ctx, SSL_OP_IGNORE_UNEXPECTED_EOF);
#endif

SSL_CTX_set_mode(ctx, SSL_MODE_ENABLE_PARTIAL_WRITE);

rk->rk_conf.ssl.ctx = ctx;
Expand Down

0 comments on commit cd32771

Please sign in to comment.