Skip to content

Commit

Permalink
Fix for SCRAM client final message, the 'r' parameter (#4895)
Browse files Browse the repository at this point in the history
must be equal to the server sent nonce, that already contains the client side nonce. librdkafka was incorrectly concatenating the client side nonce again, leading to this fix being made on AK side, released in 3.8.1, with endsWith instead of equals.
apache/kafka@0a00456
  • Loading branch information
emasab authored Nov 7, 2024
1 parent 82a8ebd commit 816df5e
Show file tree
Hide file tree
Showing 2 changed files with 17 additions and 10 deletions.
10 changes: 10 additions & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -14,10 +14,20 @@ librdkafka v2.6.1 is a maintenance release:
* Upgrade Linux dependencies: OpenSSL 3.0.15, CURL 8.10.1 (#4875).
* Upgrade Windows dependencies: MSVC runtime to 14.40.338160.0,
zstd 1.5.6, zlib 1.3.1, OpenSSL 3.3.2, CURL 8.10.1 (#4872).
* SASL/SCRAM authentication fix: avoid concatenating
client side nonce once more, as it's already prepended in server sent nonce (#4895).


## Fixes

### General fixes

* SASL/SCRAM authentication fix: avoid concatenating
client side nonce once more, as it's already prepended in
server sent nonce.
librdkafka was incorrectly concatenating the client side nonce again, leading to [this fix](https://github.com/apache/kafka/commit/0a004562b8475d48a9961d6dab3a6aa24021c47f) being made on AK side, released with 3.8.1, with `endsWith` instead of `equals`.
Happening since v0.0.99 (#4895).

### Consumer fixes

* Issues: #4870
Expand Down
17 changes: 7 additions & 10 deletions src/rdkafka_sasl_scram.c
Original file line number Diff line number Diff line change
Expand Up @@ -253,22 +253,19 @@ static char *rd_kafka_sasl_safe_string(const char *str) {
* @brief Build client-final-message-without-proof
* @remark out->ptr will be allocated and must be freed.
*/
static void rd_kafka_sasl_scram_build_client_final_message_wo_proof(
struct rd_kafka_sasl_scram_state *state,
const char *snonce,
rd_chariov_t *out) {
static void
rd_kafka_sasl_scram_build_client_final_message_wo_proof(const char *snonce,
rd_chariov_t *out) {
const char *attr_c = "biws"; /* base64 encode of "n,," */

/*
* client-final-message-without-proof =
* channel-binding "," nonce [","
* extensions]
*/
out->size = strlen("c=,r=") + strlen(attr_c) + state->cnonce.size +
strlen(snonce);
out->ptr = rd_malloc(out->size + 1);
rd_snprintf(out->ptr, out->size + 1, "c=%s,r=%.*s%s", attr_c,
(int)state->cnonce.size, state->cnonce.ptr, snonce);
out->size = strlen("c=,r=") + strlen(attr_c) + strlen(snonce);
out->ptr = rd_malloc(out->size + 1);
rd_snprintf(out->ptr, out->size + 1, "c=%s,r=%s", attr_c, snonce);
}


Expand Down Expand Up @@ -338,7 +335,7 @@ static int rd_kafka_sasl_scram_build_client_final_message(

/* client-final-message-without-proof */
rd_kafka_sasl_scram_build_client_final_message_wo_proof(
state, server_nonce, &client_final_msg_wo_proof);
server_nonce, &client_final_msg_wo_proof);

/* AuthMessage := client-first-message-bare + "," +
* server-first-message + "," +
Expand Down

0 comments on commit 816df5e

Please sign in to comment.