diff --git a/CHANGELOG.md b/CHANGELOG.md index d01f80dbe9..82d1781a52 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -12,6 +12,8 @@ librdkafka v2.2.0 is a feature release: (#4184, #4291, #4252). * Fix several bugs with sticky assignor in case of partition ownership changing between members of the consumer group (#4252). + * Avoid treating an OpenSSL error as a permanent error and treat unclean SSL + closes as normal ones (#4294). ## Fixes @@ -23,6 +25,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. ### Consumer fixes diff --git a/src/rdkafka_offset.c b/src/rdkafka_offset.c index 2cdcd7aa3c..fd76f138d2 100644 --- a/src/rdkafka_offset.c +++ b/src/rdkafka_offset.c @@ -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), diff --git a/src/rdkafka_request.c b/src/rdkafka_request.c index 1302e74d41..132f2c01f1 100644 --- a/src/rdkafka_request.c +++ b/src/rdkafka_request.c @@ -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: diff --git a/src/rdkafka_ssl.c b/src/rdkafka_ssl.c index 9961a240f7..f9c9aee469 100644 --- a/src/rdkafka_ssl.c +++ b/src/rdkafka_ssl.c @@ -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; diff --git a/tests/0139-offset_validation_mock.c b/tests/0139-offset_validation_mock.c index 3fff5277a4..d1619634b1 100644 --- a/tests/0139-offset_validation_mock.c +++ b/tests/0139-offset_validation_mock.c @@ -138,6 +138,80 @@ static void do_test_no_duplicates_during_offset_validation(void) { SUB_TEST_PASS(); } + +/** + * @brief Test that an SSL error doesn't cause an offset reset. + * See issue #4293. + */ +static void do_test_ssl_error_retried(void) { + rd_kafka_mock_cluster_t *mcluster; + rd_kafka_conf_t *conf; + const char *bootstraps; + const char *topic = test_mk_topic_name(__FUNCTION__, 1); + const char *c1_groupid = topic; + rd_kafka_t *c1; + rd_kafka_topic_partition_list_t *rktpars; + rd_kafka_topic_partition_t *rktpar; + int msg_count = 5; + uint64_t testid = test_id_generate(); + + SUB_TEST_QUICK(); + + mcluster = test_mock_cluster_new(3, &bootstraps); + rd_kafka_mock_topic_create(mcluster, topic, 1, 1); + + /* Seed the topic with messages */ + test_produce_msgs_easy_v(topic, testid, 0, 0, msg_count, 10, + "bootstrap.servers", bootstraps, + "batch.num.messages", "1", NULL); + + /* Make OffsetForLeaderEpoch fail with the _SSL error */ + rd_kafka_mock_push_request_errors(mcluster, + RD_KAFKAP_OffsetForLeaderEpoch, 1, + RD_KAFKA_RESP_ERR__SSL); + + test_conf_init(&conf, NULL, 60); + + test_conf_set(conf, "bootstrap.servers", bootstraps); + test_conf_set(conf, "topic.metadata.refresh.interval.ms", "5000"); + test_conf_set(conf, "auto.offset.reset", "latest"); + test_conf_set(conf, "enable.auto.commit", "false"); + test_conf_set(conf, "enable.auto.offset.store", "false"); + test_conf_set(conf, "enable.partition.eof", "true"); + + c1 = test_create_consumer(c1_groupid, NULL, conf, NULL); + test_consumer_subscribe(c1, topic); + + /* EOF because of reset to latest */ + test_consumer_poll("MSG_EOF", c1, testid, 1, 0, 0, NULL); + + rd_kafka_mock_partition_set_leader(mcluster, topic, 0, 2); + + /* Seek to 0 for validating the offset. */ + rktpars = rd_kafka_topic_partition_list_new(1); + rktpar = rd_kafka_topic_partition_list_add(rktpars, topic, 0); + rktpar->offset = 0; + + /* Will validate the offset at start fetching again + * from offset 0. */ + rd_kafka_topic_partition_set_leader_epoch(rktpar, 0); + rd_kafka_seek_partitions(c1, rktpars, -1); + rd_kafka_topic_partition_list_destroy(rktpars); + + /* Read all messages after seek to zero. + * In case of permanent error instead it reset to latest and + * gets an EOF. */ + test_consumer_poll("MSG_ALL", c1, testid, 0, 0, 5, NULL); + + rd_kafka_destroy(c1); + + test_mock_cluster_destroy(mcluster); + + TEST_LATER_CHECK(); + SUB_TEST_PASS(); +} + + int main_0139_offset_validation_mock(int argc, char **argv) { if (test_needs_auth()) { @@ -147,5 +221,7 @@ int main_0139_offset_validation_mock(int argc, char **argv) { do_test_no_duplicates_during_offset_validation(); + do_test_ssl_error_retried(); + return 0; }