-
Notifications
You must be signed in to change notification settings - Fork 3.2k
Fix SSL verification fail bug when Kafka endpoint is a FQDN with a do… #4991
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
Conversation
|
🎉 All Contributor License Agreements have been signed. Ready to merge. |
Co-authored-by: Joe Julian <me@joejulian.name>
|
Hi team based on the discussion in openssl/openssl#11560 (comment) would it be reasonable to accept the change in behavior for ignoring the trailing . in the FQDN? Also see this here: #4348. Although folks have worked around this already it does seem like the proper behavior to follow. |
|
@emasab Any chance we could get your take on this PR? This seems like valid behavior to have and check for in the client and doesn't sound like would break existing behavior. |
| char* dot = strrchr(rktrans->rktrans_rkb->rkb_nodename, '.'); | ||
| if (dot != NULL && dot[1] == ':') | ||
| memmove(dot, &dot[1], strlen(&dot[1]) + 1); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It shouldn't modify the rkb_nodename but the change should be done in rd_kafka_transport_ssl_set_endpoint_id after copying the string as the name can be resolved again on rd_kafka_broker_connect
|
@david-yu yes the change seems valid following RFC 6066 but only when using it as SNI, |
|
Hi
I tried to move the dot removal in rd_kafka_transport_ssl_set_endpoint_id
function.
Using this patch it seems that everything work fine but I need someone else
test it:
diff --git a/src/rdkafka_ssl.c b/src/rdkafka_ssl.c
index 417da818..ae546af2 100644
--- a/src/rdkafka_ssl.c
+++ b/src/rdkafka_ssl.c
@@ -461,6 +461,9 @@ static int
rd_kafka_transport_ssl_set_endpoint_id(rd_kafka_transport_t *rktrans,
/* Remove ":9092" port suffix from nodename */
if ((t = strrchr(name, ':')))
*t = '\0';
+ if (name[strlen(name) - 1] == '.') {
+ name[strlen(name) - 1] = '\0';
+ }
#if (OPENSSL_VERSION_NUMBER >= 0x0090806fL) && !defined(OPENSSL_NO_TLSEXT)
/* If non-numerical hostname, send it for SNI */
…On Fri, May 30, 2025 at 11:01 AM Emanuele Sabellico < ***@***.***> wrote:
*emasab* left a comment (confluentinc/librdkafka#4991)
<#4991 (comment)>
@david-yu <https://github.com/david-yu> yes the change seems valid
following RFC 6066 but only when using it as SNI, rkb_nodename shouldn't
be changed in other places.
—
Reply to this email directly, view it on GitHub
<#4991 (comment)>,
or unsubscribe
<https://github.com/notifications/unsubscribe-auth/AACTF2FCXCQW3Y7LM4PEQQT3BAM5ZAVCNFSM6AAAAABYZGHXBSVHI2DSMVQWIX3LMV43OSLTON2WKQ3PNVWWK3TUHMZDSMRRGY4TQMRZGM>
.
You are receiving this because you modified the open/close state.Message
ID: ***@***.***>
|
|
Hi,
pull request updated.
…On Fri, May 30, 2025 at 3:36 PM Torello Querci ***@***.***> wrote:
Hi
I tried to move the dot removal in rd_kafka_transport_ssl_set_endpoint_id
function.
Using this patch it seems that everything work fine but I need someone
else test it:
diff --git a/src/rdkafka_ssl.c b/src/rdkafka_ssl.c
index 417da818..ae546af2 100644
--- a/src/rdkafka_ssl.c
+++ b/src/rdkafka_ssl.c
@@ -461,6 +461,9 @@ static int
rd_kafka_transport_ssl_set_endpoint_id(rd_kafka_transport_t *rktrans,
/* Remove ":9092" port suffix from nodename */
if ((t = strrchr(name, ':')))
*t = '\0';
+ if (name[strlen(name) - 1] == '.') {
+ name[strlen(name) - 1] = '\0';
+ }
#if (OPENSSL_VERSION_NUMBER >= 0x0090806fL) && !defined(OPENSSL_NO_TLSEXT)
/* If non-numerical hostname, send it for SNI */
On Fri, May 30, 2025 at 11:01 AM Emanuele Sabellico <
***@***.***> wrote:
> *emasab* left a comment (confluentinc/librdkafka#4991)
> <#4991 (comment)>
>
> @david-yu <https://github.com/david-yu> yes the change seems valid
> following RFC 6066 but only when using it as SNI, rkb_nodename shouldn't
> be changed in other places.
>
> —
> Reply to this email directly, view it on GitHub
> <#4991 (comment)>,
> or unsubscribe
> <https://github.com/notifications/unsubscribe-auth/AACTF2FCXCQW3Y7LM4PEQQT3BAM5ZAVCNFSM6AAAAABYZGHXBSVHI2DSMVQWIX3LMV43OSLTON2WKQ3PNVWWK3TUHMZDSMRRGY4TQMRZGM>
> .
> You are receiving this because you modified the open/close state.Message
> ID: ***@***.***>
>
|
I'm not able to connect to Kafka server when the BOOTSTRAP_BROKER is a FQDN with a dot at the end.
Different Kafka implementation provide endpoints with a dot at the end so you can use a BOOTSTRAP_BROKER without dot at the end but you ad not able to connect to the topic anyway.