From 895358af231af79dda32cdb9e7b73114223c3613 Mon Sep 17 00:00:00 2001
From: Pranav Rathi <pranav.rth@gmail.com>
Date: Wed, 2 Aug 2023 21:47:08 +0530
Subject: [PATCH 1/7] Added new data type UUID for topic id. Added topic id
 support in Metadata Response.

---
 INTRODUCTION.md                     |   2 +-
 src/CMakeLists.txt                  |   1 +
 src/Makefile                        |   2 +-
 src/rdbase64.c                      | 130 ++++++++++++++++++++++++++++
 src/rdbase64.h                      |  41 +++++++++
 src/rdkafka_buf.h                   |  18 +++-
 src/rdkafka_metadata.c              |   5 ++
 src/rdkafka_metadata.h              |   1 +
 src/rdkafka_proto.h                 |  69 +++++++++++++++
 src/rdkafka_request.c               |  11 ++-
 src/rdkafka_sasl_oauthbearer_oidc.c |  21 +----
 src/rdkafka_sasl_scram.c            |  76 +---------------
 win32/librdkafka.vcxproj            |   2 +
 13 files changed, 282 insertions(+), 97 deletions(-)
 create mode 100644 src/rdbase64.c
 create mode 100644 src/rdbase64.h

diff --git a/INTRODUCTION.md b/INTRODUCTION.md
index 7d96f8b8fd..7fbe9adbcf 100644
--- a/INTRODUCTION.md
+++ b/INTRODUCTION.md
@@ -1975,7 +1975,7 @@ release of librdkafka.
 | 0       | Produce                       | 9           | 7                       |
 | 1       | Fetch                         | 15          | 11                      |
 | 2       | ListOffsets                   | 8           | 5                       |
-| 3       | Metadata                      | 12          | 9                       |
+| 3       | Metadata                      | 12          | 12                      |
 | 8       | OffsetCommit                  | 8           | 7                       |
 | 9       | OffsetFetch                   | 8           | 7                       |
 | 10      | FindCoordinator               | 4           | 2                       |
diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index 33481ba1ac..cbcff5000a 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -8,6 +8,7 @@ set(
     rdbuf.c
     rdcrc32.c
     rdfnv1a.c
+    rdbase64.c
     rdkafka.c
     rdkafka_assignor.c
     rdkafka_broker.c
diff --git a/src/Makefile b/src/Makefile
index 26df5723b8..1c43f0b017 100644
--- a/src/Makefile
+++ b/src/Makefile
@@ -52,7 +52,7 @@ SRCS=		rdkafka.c rdkafka_broker.c rdkafka_msg.c rdkafka_topic.c \
 		rdkafka_msgset_writer.c rdkafka_msgset_reader.c \
 		rdkafka_header.c rdkafka_admin.c rdkafka_aux.c \
 		rdkafka_background.c rdkafka_idempotence.c rdkafka_cert.c \
-		rdkafka_txnmgr.c rdkafka_coord.c \
+		rdkafka_txnmgr.c rdkafka_coord.c rdbase64.c \
 		rdvarint.c rdbuf.c rdmap.c rdunittest.c \
 		rdkafka_mock.c rdkafka_mock_handlers.c rdkafka_mock_cgrp.c \
 		rdkafka_error.c rdkafka_fetcher.c \
diff --git a/src/rdbase64.c b/src/rdbase64.c
new file mode 100644
index 0000000000..81c386ed80
--- /dev/null
+++ b/src/rdbase64.c
@@ -0,0 +1,130 @@
+/*
+ * librdkafka - The Apache Kafka C/C++ library
+ *
+ * Copyright (c) 2023 Confluent Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are met:
+ *
+ * 1. Redistributions of source code must retain the above copyright notice,
+ *    this list of conditions and the following disclaimer.
+ * 2. Redistributions in binary form must reproduce the above copyright notice,
+ *    this list of conditions and the following disclaimer in the documentation
+ *    and/or other materials provided with the distribution.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+ * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+ * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
+ * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+ * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+ * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+ * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+ * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+ * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
+ * POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include "rdbase64.h"
+
+#if WITH_SSL
+#include <openssl/ssl.h>
+#endif
+
+/**
+ * @brief Base64 encode binary input \p in, and write base64-encoded string
+ *        and it's size to \p out. out->ptr will be NULL in case of some issue
+ *        with the conversion or the conversion is not supported.
+ *
+ * @post out->ptr must be freed after use.
+ */
+void rd_base64_encode(const rd_chariov_t *in, rd_chariov_t *out) {
+
+#if !WITH_SSL
+        out->ptr = NULL;
+        return;
+#endif
+
+        size_t max_len;
+
+        /* OpenSSL takes an |int| argument so the input cannot exceed that. */
+        if (in->size > INT_MAX) {
+                out->ptr = NULL;
+                return;
+        }
+
+        max_len  = (((in->size + 2) / 3) * 4) + 1;
+        out->ptr = rd_malloc(max_len);
+        if (out->ptr == NULL)
+                return;
+
+        out->size = EVP_EncodeBlock((uint8_t *)out->ptr, (uint8_t *)in->ptr,
+                                    (int)in->size);
+
+        rd_assert(out->size < max_len);
+        out->ptr[out->size] = 0;
+}
+
+
+/**
+ * @brief Base64 encode binary input \p in.
+ * @returns a newly allocated, base64-encoded string or NULL in case of some
+ * issue with the conversion or the conversion is not supported.
+ *
+ * @post Returned string must be freed after use.
+ */
+char *rd_base64_encode_str(const rd_chariov_t *in) {
+        rd_chariov_t out;
+        rd_base64_encode(in, &out);
+        return out.ptr;
+}
+
+
+/**
+ * @brief Base64 decode input string \p in. Ignores leading and trailing
+ *         whitespace.
+ * @returns * 0 on successes in which case a newly allocated binary string is
+ * set in out (and size).
+ *          * -1 on invalid Base64.
+ *          * -2 on conversion not supported.
+ */
+int rd_base64_decode(const rd_chariov_t *in, rd_chariov_t *out) {
+
+#if !WITH_SSL
+        return -2;
+#endif
+
+        size_t ret_len;
+
+        /* OpenSSL takes an |int| argument, so |in->size| must not exceed
+         * that. */
+        if (in->size % 4 != 0 || in->size > INT_MAX) {
+                return -1;
+        }
+
+        ret_len  = ((in->size / 4) * 3);
+        out->ptr = rd_malloc(ret_len + 1);
+
+        if (EVP_DecodeBlock((uint8_t *)out->ptr, (uint8_t *)in->ptr,
+                            (int)in->size) == -1) {
+                rd_free(out->ptr);
+                out->ptr = NULL;
+                return -1;
+        }
+
+        /* EVP_DecodeBlock will pad the output with trailing NULs and count
+         * them in the return value. */
+        if (in->size > 1 && in->ptr[in->size - 1] == '=') {
+                if (in->size > 2 && in->ptr[in->size - 2] == '=') {
+                        ret_len -= 2;
+                } else {
+                        ret_len -= 1;
+                }
+        }
+
+        out->ptr[ret_len] = 0;
+        out->size         = ret_len;
+
+        return 0;
+}
\ No newline at end of file
diff --git a/src/rdbase64.h b/src/rdbase64.h
new file mode 100644
index 0000000000..fd9e7a209f
--- /dev/null
+++ b/src/rdbase64.h
@@ -0,0 +1,41 @@
+/*
+ * librdkafka - The Apache Kafka C/C++ library
+ *
+ * Copyright (c) 2023 Confluent Inc.
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions are met:
+ *
+ * 1. Redistributions of source code must retain the above copyright notice,
+ *    this list of conditions and the following disclaimer.
+ * 2. Redistributions in binary form must reproduce the above copyright notice,
+ *    this list of conditions and the following disclaimer in the documentation
+ *    and/or other materials provided with the distribution.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+ * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+ * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
+ * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+ * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+ * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+ * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+ * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+ * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
+ * POSSIBILITY OF SUCH DAMAGE.
+ */
+
+
+#ifndef _RDBASE64_H_
+#define _RDBASE64_H_
+
+#include "rd.h"
+
+void rd_base64_encode(const rd_chariov_t *in, rd_chariov_t *out);
+
+char *rd_base64_encode_str(const rd_chariov_t *in);
+
+int rd_base64_decode(const rd_chariov_t *in, rd_chariov_t *out);
+
+#endif /* _RDBASE64_H_ */
\ No newline at end of file
diff --git a/src/rdkafka_buf.h b/src/rdkafka_buf.h
index 7845beff90..ccd563cc6c 100644
--- a/src/rdkafka_buf.h
+++ b/src/rdkafka_buf.h
@@ -1206,7 +1206,6 @@ rd_kafka_buf_update_i64(rd_kafka_buf_t *rkbuf, size_t of, int64_t v) {
         rd_kafka_buf_update(rkbuf, of, &v, sizeof(v));
 }
 
-
 /**
  * @brief Write standard (2-byte header) or KIP-482 COMPACT_STRING to buffer.
  *
@@ -1428,4 +1427,21 @@ void rd_kafka_buf_set_maker(rd_kafka_buf_t *rkbuf,
                             rd_kafka_make_req_cb_t *make_cb,
                             void *make_opaque,
                             void (*free_make_opaque_cb)(void *make_opaque));
+
+
+#define rd_kafka_buf_read_uuid(rkbuf, uuid)                                    \
+        do {                                                                   \
+                rd_kafka_buf_read_i64(rkbuf,                                   \
+                                      &((uuid)->most_significant_bits));       \
+                rd_kafka_buf_read_i64(rkbuf,                                   \
+                                      &((uuid)->least_significant_bits));      \
+                (uuid)->base64str[0] = '\0';                                   \
+        } while (0)
+
+static RD_UNUSED void rd_kafka_buf_write_uuid(rd_kafka_buf_t *rkbuf,
+                                              rd_kafka_uuid_t *uuid) {
+        rd_kafka_buf_write_i64(rkbuf, uuid->most_significant_bits);
+        rd_kafka_buf_write_i64(rkbuf, uuid->least_significant_bits);
+}
+
 #endif /* _RDKAFKA_BUF_H_ */
diff --git a/src/rdkafka_metadata.c b/src/rdkafka_metadata.c
index f6d2bfbb49..f96edf6583 100644
--- a/src/rdkafka_metadata.c
+++ b/src/rdkafka_metadata.c
@@ -593,6 +593,11 @@ rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb,
                 rd_kafka_buf_read_i16a(rkbuf, md->topics[i].err);
                 rd_kafka_buf_read_str_tmpabuf(rkbuf, &tbuf,
                                               md->topics[i].topic);
+
+                if (ApiVersion >= 10) {
+                        rd_kafka_buf_read_uuid(rkbuf, &mdi->topics[i].topic_id);
+                }
+
                 if (ApiVersion >= 1) {
                         int8_t is_internal;
                         rd_kafka_buf_read_i8(rkbuf, &is_internal);
diff --git a/src/rdkafka_metadata.h b/src/rdkafka_metadata.h
index 03586618db..8a8f16dbfa 100644
--- a/src/rdkafka_metadata.h
+++ b/src/rdkafka_metadata.h
@@ -54,6 +54,7 @@ typedef struct rd_kafka_metadata_topic_internal_s {
          *  same count as metadata.topics[i].partition_cnt.
          *  Sorted by Partition Id. */
         rd_kafka_metadata_partition_internal_t *partitions;
+        rd_kafka_uuid_t topic_id;
 } rd_kafka_metadata_topic_internal_t;
 
 
diff --git a/src/rdkafka_proto.h b/src/rdkafka_proto.h
index 24fce04106..9e0efa79f0 100644
--- a/src/rdkafka_proto.h
+++ b/src/rdkafka_proto.h
@@ -30,8 +30,10 @@
 #define _RDKAFKA_PROTO_H_
 
 
+#include "rdstring.h"
 #include "rdendian.h"
 #include "rdvarint.h"
+#include "rdbase64.h"
 
 /* Protocol defines */
 #include "rdkafka_protocol.h"
@@ -565,6 +567,73 @@ typedef struct rd_kafka_buf_s rd_kafka_buf_t;
         (8 + 4 + 4 + 1 + 4 + 2 + 4 + 8 + 8 + 8 + 2 + 4)
 
 
+/**
+ * @brief UUID
+ *
+ * @param most_significant_bits - most significant 64 bits for the UUID
+ * @param least_significant_bits - least significant 64 bits for the UUID
+ * @param base64str - base64 encoding for the uuid. By default, it is lazy
+ * loaded. Use function `rd_kafka_uuid_base64str()` as a getter for this field.
+ */
+typedef struct rd_kafka_uuid_s {
+        int64_t most_significant_bits;
+        int64_t least_significant_bits;
+        char base64str[23];
+} rd_kafka_uuid_t;
+
+#define RD_KAFKA_ZERO_UUID                                                     \
+        { 0, 0, "" }
+
+#define RD_KAFKA_METADATA_TOPIC_ID                                             \
+        { 0, 1, "" }
+
+/**
+ * Initialize given UUID to zero UUID.
+ *
+ * @param uuid - UUID to initialize.
+ */
+static RD_INLINE RD_UNUSED void rd_kafka_uuid_init(rd_kafka_uuid_t *uuid) {
+        memset(uuid, 0, sizeof(*uuid));
+}
+
+/**
+ * @brief Computes base64 encoding for the given uuid string.
+ * @param uuid UUID for which base64 encoding is required.
+ *
+ * @return base64 encoded string for the given UUID or NULL in case of some
+ *         issue with the conversion or the conversion is not supported.
+ */
+static RD_INLINE RD_UNUSED char *
+rd_kafka_uuid_base64str(rd_kafka_uuid_t *uuid) {
+        if (strlen(uuid->base64str))
+                return uuid->base64str;
+
+        rd_chariov_t in_base64;
+        char *out_base64_str;
+        char *uuid_bytes;
+        uint64_t input_uuid[2];
+
+        input_uuid[0]  = htobe64(uuid->most_significant_bits);
+        input_uuid[1]  = htobe64(uuid->least_significant_bits);
+        uuid_bytes     = (char *)input_uuid;
+        in_base64.ptr  = uuid_bytes;
+        in_base64.size = sizeof(uuid->most_significant_bits) +
+                         sizeof(uuid->least_significant_bits);
+
+        out_base64_str = rd_base64_encode_str(&in_base64);
+        if (!out_base64_str)
+                return NULL;
+
+        rd_strlcpy(uuid->base64str, out_base64_str,
+                   23 /* Removing extra ('=') padding */);
+        rd_free(out_base64_str);
+        return uuid->base64str;
+}
+
+static RD_INLINE RD_UNUSED void rd_kafka_uuid_destroy(rd_kafka_uuid_t *uuid) {
+        rd_free(uuid);
+}
+
 
 /**
  * @name Producer ID and Epoch for the Idempotent Producer
diff --git a/src/rdkafka_request.c b/src/rdkafka_request.c
index 8d0789cfc7..b01973a2cf 100644
--- a/src/rdkafka_request.c
+++ b/src/rdkafka_request.c
@@ -2216,7 +2216,7 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb,
         int *full_incr = NULL;
 
         ApiVersion = rd_kafka_broker_ApiVersion_supported(
-            rkb, RD_KAFKAP_Metadata, 0, 9, &features);
+            rkb, RD_KAFKAP_Metadata, 0, 12, &features);
 
         rkbuf = rd_kafka_buf_new_flexver_request(rkb, RD_KAFKAP_Metadata, 1,
                                                  4 + (50 * topic_cnt) + 1,
@@ -2305,6 +2305,7 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb,
         if (topic_cnt > 0) {
                 char *topic;
                 int i;
+                rd_kafka_uuid_t zero_uuid = RD_KAFKA_ZERO_UUID;
 
                 /* Maintain a copy of the topics list so we can purge
                  * hints from the metadata cache on error. */
@@ -2312,6 +2313,12 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb,
                     rd_list_copy(topics, rd_list_string_copy, NULL);
 
                 RD_LIST_FOREACH(topic, topics, i) {
+                        if (ApiVersion >= 10) {
+                                /* FIXME: Not supporting topic id in the request
+                                 * right now. Update this to correct topic
+                                 * id once KIP-516 is fully implemented. */
+                                rd_kafka_buf_write_uuid(rkbuf, &zero_uuid);
+                        }
                         rd_kafka_buf_write_str(rkbuf, topic, -1);
                         /* Tags for previous topic */
                         rd_kafka_buf_write_tags(rkbuf);
@@ -2337,7 +2344,7 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb,
                            "on broker auto.create.topics.enable configuration");
         }
 
-        if (ApiVersion >= 8 && ApiVersion < 10) {
+        if (ApiVersion >= 8 && ApiVersion <= 10) {
                 /* TODO: implement KIP-430 */
                 /* IncludeClusterAuthorizedOperations */
                 rd_kafka_buf_write_bool(rkbuf, rd_false);
diff --git a/src/rdkafka_sasl_oauthbearer_oidc.c b/src/rdkafka_sasl_oauthbearer_oidc.c
index 9fa0972a39..57a6d2e30e 100644
--- a/src/rdkafka_sasl_oauthbearer_oidc.c
+++ b/src/rdkafka_sasl_oauthbearer_oidc.c
@@ -37,25 +37,7 @@
 #include <curl/curl.h>
 #include "rdhttp.h"
 #include "rdkafka_sasl_oauthbearer_oidc.h"
-
-
-/**
- * @brief Base64 encode binary input \p in, and write base64-encoded string
- *        and it's size to \p out
- */
-static void rd_base64_encode(const rd_chariov_t *in, rd_chariov_t *out) {
-        size_t max_len;
-
-        max_len  = (((in->size + 2) / 3) * 4) + 1;
-        out->ptr = rd_malloc(max_len);
-        rd_assert(out->ptr);
-
-        out->size = EVP_EncodeBlock((uint8_t *)out->ptr, (uint8_t *)in->ptr,
-                                    (int)in->size);
-
-        rd_assert(out->size <= max_len);
-        out->ptr[out->size] = 0;
-}
+#include "rdbase64.h"
 
 
 /**
@@ -84,6 +66,7 @@ static char *rd_kafka_oidc_build_auth_header(const char *client_id,
 
         client_authorization_in.size--;
         rd_base64_encode(&client_authorization_in, &client_authorization_out);
+        rd_assert(client_authorization_out.ptr);
 
         authorization_base64_header_size =
             strlen("Authorization: Basic ") + client_authorization_out.size + 1;
diff --git a/src/rdkafka_sasl_scram.c b/src/rdkafka_sasl_scram.c
index 32f13a4c04..01a6cd75e4 100644
--- a/src/rdkafka_sasl_scram.c
+++ b/src/rdkafka_sasl_scram.c
@@ -38,6 +38,7 @@
 #include "rdkafka_sasl_int.h"
 #include "rdrand.h"
 #include "rdunittest.h"
+#include "rdbase64.h"
 
 
 #if WITH_SSL
@@ -142,77 +143,6 @@ static char *rd_kafka_sasl_scram_get_attr(const rd_chariov_t *inbuf,
 }
 
 
-/**
- * @brief Base64 encode binary input \p in
- * @returns a newly allocated, base64-encoded string or NULL on error.
- */
-static char *rd_base64_encode(const rd_chariov_t *in) {
-        char *ret;
-        size_t ret_len, max_len;
-
-        /* OpenSSL takes an |int| argument so the input cannot exceed that. */
-        if (in->size > INT_MAX) {
-                return NULL;
-        }
-
-        /* This does not overflow given the |INT_MAX| bound, above. */
-        max_len = (((in->size + 2) / 3) * 4) + 1;
-        ret     = rd_malloc(max_len);
-        if (ret == NULL) {
-                return NULL;
-        }
-
-        ret_len =
-            EVP_EncodeBlock((uint8_t *)ret, (uint8_t *)in->ptr, (int)in->size);
-        assert(ret_len < max_len);
-        ret[ret_len] = 0;
-
-        return ret;
-}
-
-
-/**
- * @brief Base64 decode input string \p in. Ignores leading and trailing
- *         whitespace.
- * @returns -1 on invalid Base64, or 0 on successes in which case a
- *         newly allocated binary string is set in out (and size).
- */
-static int rd_base64_decode(const rd_chariov_t *in, rd_chariov_t *out) {
-        size_t ret_len;
-
-        /* OpenSSL takes an |int| argument, so |in->size| must not exceed
-         * that. */
-        if (in->size % 4 != 0 || in->size > INT_MAX) {
-                return -1;
-        }
-
-        ret_len  = ((in->size / 4) * 3);
-        out->ptr = rd_malloc(ret_len + 1);
-
-        if (EVP_DecodeBlock((uint8_t *)out->ptr, (uint8_t *)in->ptr,
-                            (int)in->size) == -1) {
-                rd_free(out->ptr);
-                out->ptr = NULL;
-                return -1;
-        }
-
-        /* EVP_DecodeBlock will pad the output with trailing NULs and count
-         * them in the return value. */
-        if (in->size > 1 && in->ptr[in->size - 1] == '=') {
-                if (in->size > 2 && in->ptr[in->size - 2] == '=') {
-                        ret_len -= 2;
-                } else {
-                        ret_len -= 1;
-                }
-        }
-
-        out->ptr[ret_len] = 0;
-        out->size         = ret_len;
-
-        return 0;
-}
-
-
 /**
  * @brief Perform H(str) hash function and stores the result in \p out
  *        which must be at least EVP_MAX_MD_SIZE.
@@ -443,7 +373,7 @@ static int rd_kafka_sasl_scram_build_client_final_message(
         }
 
         /* Store the Base64 encoded ServerSignature for quick comparison */
-        state->ServerSignatureB64 = rd_base64_encode(&ServerSignature);
+        state->ServerSignatureB64 = rd_base64_encode_str(&ServerSignature);
         if (state->ServerSignatureB64 == NULL) {
                 rd_free(client_final_msg_wo_proof.ptr);
                 return -1;
@@ -468,7 +398,7 @@ static int rd_kafka_sasl_scram_build_client_final_message(
 
 
         /* Base64 encoded ClientProof */
-        ClientProofB64 = rd_base64_encode(&ClientProof);
+        ClientProofB64 = rd_base64_encode_str(&ClientProof);
         if (ClientProofB64 == NULL) {
                 rd_free(client_final_msg_wo_proof.ptr);
                 return -1;
diff --git a/win32/librdkafka.vcxproj b/win32/librdkafka.vcxproj
index 2735fca9ca..a7f267e89e 100644
--- a/win32/librdkafka.vcxproj
+++ b/win32/librdkafka.vcxproj
@@ -102,6 +102,7 @@
     <ClInclude Include="..\src\rdatomic.h" />
     <ClInclude Include="..\src\rdavg.h" />
     <ClInclude Include="..\src\rdbuf.h" />
+    <ClInclude Include="..\src\rdbase64.h" />
     <ClInclude Include="..\src\rdendian.h" />
     <ClInclude Include="..\src\rdfloat.h" />
     <ClInclude Include="..\src\rdgz.h" />
@@ -174,6 +175,7 @@
     <ClCompile Include="..\src\crc32c.c" />
     <ClCompile Include="..\src\rdaddr.c" />
     <ClCompile Include="..\src\rdbuf.c" />
+    <ClCompile Include="..\src\rdbase64.c" />
     <ClCompile Include="..\src\rdcrc32.c" />
     <ClCompile Include="..\src\rdfnv1a.c" />
     <ClCompile Include="..\src\rdgz.c" />

From 15384321582999a2810c72b9a4f26410a1e34539 Mon Sep 17 00:00:00 2001
From: Pranav Rathi <pranav.rth@gmail.com>
Date: Thu, 3 Aug 2023 19:48:07 +0530
Subject: [PATCH 2/7] PR comment

---
 src/rdbase64.c | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/src/rdbase64.c b/src/rdbase64.c
index 81c386ed80..6d0e0404fb 100644
--- a/src/rdbase64.c
+++ b/src/rdbase64.c
@@ -59,7 +59,7 @@ void rd_base64_encode(const rd_chariov_t *in, rd_chariov_t *out) {
         if (out->ptr == NULL)
                 return;
 
-        out->size = EVP_EncodeBlock((uint8_t *)out->ptr, (uint8_t *)in->ptr,
+        out->size = EVP_EncodeBlock((unsigned char *)out->ptr, (unsigned char *)in->ptr,
                                     (int)in->size);
 
         rd_assert(out->size < max_len);
@@ -106,7 +106,7 @@ int rd_base64_decode(const rd_chariov_t *in, rd_chariov_t *out) {
         ret_len  = ((in->size / 4) * 3);
         out->ptr = rd_malloc(ret_len + 1);
 
-        if (EVP_DecodeBlock((uint8_t *)out->ptr, (uint8_t *)in->ptr,
+        if (EVP_DecodeBlock((unsigned char *)out->ptr, (unsigned char *)in->ptr,
                             (int)in->size) == -1) {
                 rd_free(out->ptr);
                 out->ptr = NULL;

From 7490da4e188f0ad732f0a500a990aaad47eb022c Mon Sep 17 00:00:00 2001
From: Pranav Rathi <pranav.rth@gmail.com>
Date: Thu, 3 Aug 2023 23:31:17 +0530
Subject: [PATCH 3/7] Style fix

---
 src/rdbase64.c | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/src/rdbase64.c b/src/rdbase64.c
index 6d0e0404fb..1a1dc31e46 100644
--- a/src/rdbase64.c
+++ b/src/rdbase64.c
@@ -59,8 +59,8 @@ void rd_base64_encode(const rd_chariov_t *in, rd_chariov_t *out) {
         if (out->ptr == NULL)
                 return;
 
-        out->size = EVP_EncodeBlock((unsigned char *)out->ptr, (unsigned char *)in->ptr,
-                                    (int)in->size);
+        out->size = EVP_EncodeBlock((unsigned char *)out->ptr,
+                                    (unsigned char *)in->ptr, (int)in->size);
 
         rd_assert(out->size < max_len);
         out->ptr[out->size] = 0;

From e7b1e0defb69f01b5640d48257fb1006e27d1923 Mon Sep 17 00:00:00 2001
From: Pranav Rathi <pranav.rth@gmail.com>
Date: Fri, 4 Aug 2023 12:35:58 +0530
Subject: [PATCH 4/7] Updated changelog and added kip-516 in supported kips
 section

---
 CHANGELOG.md    |   6 ++
 INTRODUCTION.md | 223 ++++++++++++++++++++++++------------------------
 2 files changed, 118 insertions(+), 111 deletions(-)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index e7577a989a..54b4c293e7 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,5 +1,11 @@
 # librdkafka v2.2.0
 
+librdkafka v2.2.0 is a feature release:
+
+ * Added Topic id to the metadata response which is part of the [KIP-516](https://cwiki.apache.org/confluence/display/KAFKA/KIP-516%3A+Topic+Identifiers)
+
+# librdkafka v2.2.0
+
 librdkafka v2.2.0 is a feature release:
 
  * Fix a segmentation fault when subscribing to non-existent topics and
diff --git a/INTRODUCTION.md b/INTRODUCTION.md
index 7fbe9adbcf..19d4e58f24 100644
--- a/INTRODUCTION.md
+++ b/INTRODUCTION.md
@@ -1847,118 +1847,119 @@ librdkafka v0.11.0.
 The [Apache Kafka Implementation Proposals (KIPs)](https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals) supported by librdkafka.
 
 
-| KIP                                                                      | Kafka release               | Status                                                                                        |
-|--------------------------------------------------------------------------|-----------------------------|-----------------------------------------------------------------------------------------------|
-| KIP-1 - Stop accepting request.required.acks > 1                         | 0.9.0.0                     | Not enforced on client (due to backwards compat with brokers  <0.8.3)                         |
-| KIP-4 - Metadata protocol changes                                        | 0.9.0.0, 0.10.0.0, 0.10.1.0 | Supported                                                                                     |
-| KIP-8 - Producer flush()                                                 | 0.9.0.0                     | Supported                                                                                     |
-| KIP-12 - SASL Kerberos                                                   | 0.9.0.0                     | Supported (uses SSPI/logged-on-user on Windows, full KRB5 keytabs on Unix)                    |
-| KIP-13 - Protocol request throttling (enforced on broker)                | 0.9.0.0                     | Supported                                                                                     |
-| KIP-15 - Producer close with timeout                                     | 0.9.0.0                     | Supported (through flush() + destroy())                                                       |
-| KIP-19 - Request timeouts                                                | 0.9.0.0                     | Supported                                                                                     |
-| KIP-22 - Producer pluggable partitioner                                  | 0.9.0.0                     | Supported (not supported by Go, .NET and Python)                                              |
-| KIP-31 - Relative offsets in messagesets                                 | 0.10.0.0                    | Supported                                                                                     |
-| KIP-35 - ApiVersionRequest                                               | 0.10.0.0                    | Supported                                                                                     |
-| KIP-40 - ListGroups and DescribeGroups                                   | 0.9.0.0                     | Supported                                                                                     |
-| KIP-41 - max.poll.records                                                | 0.10.0.0                    | Supported through batch consumption interface (not supported by .NET and Go)                  |
-| KIP-42 - Producer and Consumer interceptors                              | 0.10.0.0                    | Supported (not supported by Go, .NET and Python)                                              |
-| KIP-43 - SASL PLAIN and handshake                                        | 0.10.0.0                    | Supported                                                                                     |
-| KIP-48 - Delegation tokens                                               | 1.1.0                       | Not supported                                                                                 |
-| KIP-54 - Sticky partition assignment strategy                            | 0.11.0.0                    | Supported but not available, use KIP-429 instead.                                             |
-| KIP-57 - Interoperable LZ4 framing                                       | 0.10.0.0                    | Supported                                                                                     |
-| KIP-62 - max.poll.interval and background heartbeats                     | 0.10.1.0                    | Supported                                                                                     |
-| KIP-70 - Proper client rebalance event on unsubscribe/subscribe          | 0.10.1.0                    | Supported                                                                                     |
-| KIP-74 - max.partition.fetch.bytes                                       | 0.10.1.0                    | Supported                                                                                     |
-| KIP-78 - Retrieve Cluster Id                                             | 0.10.1.0                    | Supported (not supported by .NET)                                                             |
-| KIP-79 - OffsetsForTimes                                                 | 0.10.1.0                    | Supported                                                                                     |
-| KIP-81 - Consumer pre-fetch buffer size                                  | 2.4.0 (WIP)                 | Supported                                                                                     |
-| KIP-82 - Record Headers                                                  | 0.11.0.0                    | Supported                                                                                     |
-| KIP-84 - SASL SCRAM                                                      | 0.10.2.0                    | Supported                                                                                     |
-| KIP-85 - SASL config properties                                          | 0.10.2.0                    | Supported                                                                                     |
-| KIP-86 - Configurable SASL callbacks                                     | 2.0.0                       | Not supported                                                                                 |
-| KIP-88 - AdminAPI: ListGroupOffsets                                      | 0.10.2.0                    | Supported                                                                                 |
-| KIP-91 - Intuitive timeouts in Producer                                  | 2.1.0                       | Supported                                                                                     |
-| KIP-92 - Per-partition lag metrics in Consumer                           | 0.10.2.0                    | Supported                                                                                     |
-| KIP-97 - Backwards compatibility with older brokers                      | 0.10.2.0                    | Supported                                                                                     |
-| KIP-98 - EOS                                                             | 0.11.0.0                    | Supported                                                                                     |
-| KIP-102 - Close with timeout in consumer                                 | 0.10.2.0                    | Not supported                                                                                 |
-| KIP-107 - AdminAPI: DeleteRecordsBefore                                  | 0.11.0.0                    | Supported                                                                                     |
-| KIP-110 - ZStd compression                                               | 2.1.0                       | Supported                                                                                     |
-| KIP-117 - AdminClient                                                    | 0.11.0.0                    | Supported                                                                                     |
-| KIP-124 - Request rate quotas                                            | 0.11.0.0                    | Partially supported (depending on protocol request)                                           |
-| KIP-126 - Producer ensure proper batch size after compression            | 0.11.0.0                    | Supported                                                                                     |
-| KIP-133 - AdminAPI: DescribeConfigs and AlterConfigs                     | 0.11.0.0                    | Supported                                                                                     |
-| KIP-140 - AdminAPI: ACLs                                                 | 0.11.0.0                    | Supported                                                                                     |
-| KIP-144 - Broker reconnect backoff                                       | 0.11.0.0                    | Supported                                                                                     |
-| KIP-152 - Improved SASL auth error messages                              | 1.0.0                       | Supported                                                                                     |
-| KIP-192 - Cleaner idempotence semantics                                  | 1.0.0                       | Not supported (superceeded by KIP-360)                                                        |
-| KIP-195 - AdminAPI: CreatePartitions                                     | 1.0.0                       | Supported                                                                                     |
-| KIP-204 - AdminAPI: DeleteRecords                                        | 1.1.0                       | Supported                                                                                     |
-| KIP-219 - Client-side throttling                                         | 2.0.0                       | Not supported                                                                                 |
-| KIP-222 - AdminAPI: Consumer group operations                            | 2.0.0                       | Supported                                                                                     |
-| KIP-223 - Consumer partition lead metric                                 | 2.0.0                       | Not supported                                                                                 |
-| KIP-226 - AdminAPI: Dynamic broker config                                | 1.1.0                       | Supported                                                                                     |
-| KIP-227 - Consumer Incremental Fetch                                     | 1.1.0                       | Not supported                                                                                 |
-| KIP-229 - AdminAPI: DeleteGroups                                         | 1.1.0                       | Supported                                                                                     |
-| KIP-235 - DNS alias for secure connections                               | 2.1.0                       | Supported                                                                                 |
-| KIP-249 - AdminAPI: Deletegation Tokens                                  | 2.0.0                       | Not supported                                                                                 |
-| KIP-255 - SASL OAUTHBEARER                                               | 2.0.0                       | Supported                                                                                     |
-| KIP-266 - Fix indefinite consumer timeouts                               | 2.0.0                       | Supported (bound by session.timeout.ms and max.poll.interval.ms)                              |
-| KIP-289 - Consumer group.id default to NULL                              | 2.2.0                       | Supported                                                                                     |
-| KIP-294 - SSL endpoint verification                                      | 2.0.0                       | Supported                                                                                     |
-| KIP-302 - Use all addresses for resolved broker hostname                 | 2.1.0                       | Supported                                                                                     |
-| KIP-320 - Consumer: handle log truncation                                | 2.1.0, 2.2.0                | Supported                                                                                     |
-| KIP-322 - DeleteTopics disabled error code                               | 2.1.0                       | Supported                                                                                     |
-| KIP-339 - AdminAPI: incrementalAlterConfigs                              | 2.3.0                       | Supported                                                                                     |
-| KIP-341 - Update Sticky partition assignment data                        | 2.3.0                       | Not supported (superceeded by KIP-429)                                                        |
-| KIP-342 - Custom SASL OAUTHBEARER extensions                             | 2.1.0                       | Supported                                                                                     |
-| KIP-345 - Consumer: Static membership                                    | 2.4.0                       | Supported                                                                                     |
-| KIP-357 - AdminAPI: list ACLs per principal                              | 2.1.0                       | Not supported                                                                                 |
-| KIP-359 - Producer: use EpochLeaderId                                    | 2.4.0                       | Not supported                                                                                 |
-| KIP-360 - Improve handling of unknown Idempotent Producer                | 2.5.0                       | Supported                                                                                     |
-| KIP-361 - Consumer: add config to disable auto topic creation            | 2.3.0                       | Supported                                                                                     |
-| KIP-368 - SASL periodic reauth                                           | 2.2.0                       | Supported                                                                                     |
-| KIP-369 - Always roundRobin partitioner                                  | 2.4.0                       | Not supported                                                                                 |
+| KIP                                                                      | Kafka release               | Status                                                                           |
+|--------------------------------------------------------------------------|-----------------------------|----------------------------------------------------------------------------------|
+| KIP-1 - Stop accepting request.required.acks > 1                         | 0.9.0.0                     | Not enforced on client (due to backwards compat with brokers  <0.8.3)            |
+| KIP-4 - Metadata protocol changes                                        | 0.9.0.0, 0.10.0.0, 0.10.1.0 | Supported                                                                        |
+| KIP-8 - Producer flush()                                                 | 0.9.0.0                     | Supported                                                                        |
+| KIP-12 - SASL Kerberos                                                   | 0.9.0.0                     | Supported (uses SSPI/logged-on-user on Windows, full KRB5 keytabs on Unix)       |
+| KIP-13 - Protocol request throttling (enforced on broker)                | 0.9.0.0                     | Supported                                                                        |
+| KIP-15 - Producer close with timeout                                     | 0.9.0.0                     | Supported (through flush() + destroy())                                          |
+| KIP-19 - Request timeouts                                                | 0.9.0.0                     | Supported                                                                        |
+| KIP-22 - Producer pluggable partitioner                                  | 0.9.0.0                     | Supported (not supported by Go, .NET and Python)                                 |
+| KIP-31 - Relative offsets in messagesets                                 | 0.10.0.0                    | Supported                                                                        |
+| KIP-35 - ApiVersionRequest                                               | 0.10.0.0                    | Supported                                                                        |
+| KIP-40 - ListGroups and DescribeGroups                                   | 0.9.0.0                     | Supported                                                                        |
+| KIP-41 - max.poll.records                                                | 0.10.0.0                    | Supported through batch consumption interface (not supported by .NET and Go)     |
+| KIP-42 - Producer and Consumer interceptors                              | 0.10.0.0                    | Supported (not supported by Go, .NET and Python)                                 |
+| KIP-43 - SASL PLAIN and handshake                                        | 0.10.0.0                    | Supported                                                                        |
+| KIP-48 - Delegation tokens                                               | 1.1.0                       | Not supported                                                                    |
+| KIP-54 - Sticky partition assignment strategy                            | 0.11.0.0                    | Supported but not available, use KIP-429 instead.                                |
+| KIP-57 - Interoperable LZ4 framing                                       | 0.10.0.0                    | Supported                                                                        |
+| KIP-62 - max.poll.interval and background heartbeats                     | 0.10.1.0                    | Supported                                                                        |
+| KIP-70 - Proper client rebalance event on unsubscribe/subscribe          | 0.10.1.0                    | Supported                                                                        |
+| KIP-74 - max.partition.fetch.bytes                                       | 0.10.1.0                    | Supported                                                                        |
+| KIP-78 - Retrieve Cluster Id                                             | 0.10.1.0                    | Supported (not supported by .NET)                                                |
+| KIP-79 - OffsetsForTimes                                                 | 0.10.1.0                    | Supported                                                                        |
+| KIP-81 - Consumer pre-fetch buffer size                                  | 2.4.0 (WIP)                 | Supported                                                                        |
+| KIP-82 - Record Headers                                                  | 0.11.0.0                    | Supported                                                                        |
+| KIP-84 - SASL SCRAM                                                      | 0.10.2.0                    | Supported                                                                        |
+| KIP-85 - SASL config properties                                          | 0.10.2.0                    | Supported                                                                        |
+| KIP-86 - Configurable SASL callbacks                                     | 2.0.0                       | Not supported                                                                    |
+| KIP-88 - AdminAPI: ListGroupOffsets                                      | 0.10.2.0                    | Supported                                                                        |
+| KIP-91 - Intuitive timeouts in Producer                                  | 2.1.0                       | Supported                                                                        |
+| KIP-92 - Per-partition lag metrics in Consumer                           | 0.10.2.0                    | Supported                                                                        |
+| KIP-97 - Backwards compatibility with older brokers                      | 0.10.2.0                    | Supported                                                                        |
+| KIP-98 - EOS                                                             | 0.11.0.0                    | Supported                                                                        |
+| KIP-102 - Close with timeout in consumer                                 | 0.10.2.0                    | Not supported                                                                    |
+| KIP-107 - AdminAPI: DeleteRecordsBefore                                  | 0.11.0.0                    | Supported                                                                        |
+| KIP-110 - ZStd compression                                               | 2.1.0                       | Supported                                                                        |
+| KIP-117 - AdminClient                                                    | 0.11.0.0                    | Supported                                                                        |
+| KIP-124 - Request rate quotas                                            | 0.11.0.0                    | Partially supported (depending on protocol request)                              |
+| KIP-126 - Producer ensure proper batch size after compression            | 0.11.0.0                    | Supported                                                                        |
+| KIP-133 - AdminAPI: DescribeConfigs and AlterConfigs                     | 0.11.0.0                    | Supported                                                                        |
+| KIP-140 - AdminAPI: ACLs                                                 | 0.11.0.0                    | Supported                                                                        |
+| KIP-144 - Broker reconnect backoff                                       | 0.11.0.0                    | Supported                                                                        |
+| KIP-152 - Improved SASL auth error messages                              | 1.0.0                       | Supported                                                                        |
+| KIP-192 - Cleaner idempotence semantics                                  | 1.0.0                       | Not supported (superceeded by KIP-360)                                           |
+| KIP-195 - AdminAPI: CreatePartitions                                     | 1.0.0                       | Supported                                                                        |
+| KIP-204 - AdminAPI: DeleteRecords                                        | 1.1.0                       | Supported                                                                        |
+| KIP-219 - Client-side throttling                                         | 2.0.0                       | Not supported                                                                    |
+| KIP-222 - AdminAPI: Consumer group operations                            | 2.0.0                       | Supported                                                                        |
+| KIP-223 - Consumer partition lead metric                                 | 2.0.0                       | Not supported                                                                    |
+| KIP-226 - AdminAPI: Dynamic broker config                                | 1.1.0                       | Supported                                                                        |
+| KIP-227 - Consumer Incremental Fetch                                     | 1.1.0                       | Not supported                                                                    |
+| KIP-229 - AdminAPI: DeleteGroups                                         | 1.1.0                       | Supported                                                                        |
+| KIP-235 - DNS alias for secure connections                               | 2.1.0                       | Supported                                                                        |
+| KIP-249 - AdminAPI: Deletegation Tokens                                  | 2.0.0                       | Not supported                                                                    |
+| KIP-255 - SASL OAUTHBEARER                                               | 2.0.0                       | Supported                                                                        |
+| KIP-266 - Fix indefinite consumer timeouts                               | 2.0.0                       | Supported (bound by session.timeout.ms and max.poll.interval.ms)                 |
+| KIP-289 - Consumer group.id default to NULL                              | 2.2.0                       | Supported                                                                        |
+| KIP-294 - SSL endpoint verification                                      | 2.0.0                       | Supported                                                                        |
+| KIP-302 - Use all addresses for resolved broker hostname                 | 2.1.0                       | Supported                                                                        |
+| KIP-320 - Consumer: handle log truncation                                | 2.1.0, 2.2.0                | Supported                                                                        |
+| KIP-322 - DeleteTopics disabled error code                               | 2.1.0                       | Supported                                                                        |
+| KIP-339 - AdminAPI: incrementalAlterConfigs                              | 2.3.0                       | Supported                                                                        |
+| KIP-341 - Update Sticky partition assignment data                        | 2.3.0                       | Not supported (superceeded by KIP-429)                                           |
+| KIP-342 - Custom SASL OAUTHBEARER extensions                             | 2.1.0                       | Supported                                                                        |
+| KIP-345 - Consumer: Static membership                                    | 2.4.0                       | Supported                                                                        |
+| KIP-357 - AdminAPI: list ACLs per principal                              | 2.1.0                       | Not supported                                                                    |
+| KIP-359 - Producer: use EpochLeaderId                                    | 2.4.0                       | Not supported                                                                    |
+| KIP-360 - Improve handling of unknown Idempotent Producer                | 2.5.0                       | Supported                                                                        |
+| KIP-361 - Consumer: add config to disable auto topic creation            | 2.3.0                       | Supported                                                                        |
+| KIP-368 - SASL periodic reauth                                           | 2.2.0                       | Supported                                                                        |
+| KIP-369 - Always roundRobin partitioner                                  | 2.4.0                       | Not supported                                                                    |
 | KIP-389 - Consumer group max size                                        | 2.2.0                       | Supported (error is propagated to application, but the consumer does not raise a fatal error) |
-| KIP-392 - Allow consumers to fetch from closest replica                  | 2.4.0                       | Supported                                                                                     |
-| KIP-394 - Consumer: require member.id in JoinGroupRequest                | 2.2.0                       | Supported                                                                                     |
-| KIP-396 - AdminAPI: commit/list offsets                                  | 2.4.0                       | Partially supported (remaining APIs available outside Admin client)                           |
-| KIP-412 - AdminAPI: adjust log levels                                    | 2.4.0                       | Not supported                                                                                 |
-| KIP-421 - Variables in client config files                               | 2.3.0                       | Not applicable (librdkafka, et.al, does not provide a config file interface, and shouldn't)   |
-| KIP-429 - Consumer: incremental rebalance protocol                       | 2.4.0                       | Supported                                                                                     |
-| KIP-430 - AdminAPI: return authorized operations in Describe.. responses | 2.3.0                       | Not supported                                                                                 |
-| KIP-436 - Start time in stats                                            | 2.3.0                       | Supported                                                                                     |
-| KIP-447 - Producer scalability for EOS                                   | 2.5.0                       | Supported                                                                                     |
-| KIP-455 - AdminAPI: Replica assignment                                   | 2.4.0 (WIP)                 | Not supported                                                                                 |
-| KIP-460 - AdminAPI: electPreferredLeader                                 | 2.4.0                       | Not supported                                                                                 |
-| KIP-464 - AdminAPI: defaults for createTopics                            | 2.4.0                       | Supported                                                                                     |
-| KIP-467 - Per-message (sort of) error codes in ProduceResponse           | 2.4.0 (WIP)                 | Not supported                                                                                 |
-| KIP-480 - Sticky partitioner                                             | 2.4.0                       | Supported                                                                                     |
-| KIP-482 - Optional fields in Kafka protocol                              | 2.4.0                       | Partially supported (ApiVersionRequest)                                                       |
-| KIP-496 - AdminAPI: delete offsets                                       | 2.4.0                       | Supported                                                                                     |
-| KIP-511 - Collect Client's Name and Version                              | 2.4.0                       | Supported                                                                                     |
-| KIP-514 - Bounded flush()                                                | 2.4.0                       | Supported                                                                                     |
-| KIP-517 - Consumer poll() metrics                                        | 2.4.0                       | Not supported                                                                                 |
-| KIP-518 - Allow listing consumer groups per state                        | 2.6.0                       | Supported                                                                                     |
-| KIP-519 - Make SSL engine configurable                                   | 2.6.0                       | Supported                                                                                     |
-| KIP-525 - Return topic metadata and configs in CreateTopics response     | 2.4.0                       | Not supported                                                                                 |
-| KIP-526 - Reduce Producer Metadata Lookups for Large Number of Topics    | 2.5.0                       | Not supported                                                                                 |
-| KIP-533 - Add default API timeout to AdminClient                         | 2.5.0                       | Not supported                                                                                 |
-| KIP-546 - Add Client Quota APIs to AdminClient                           | 2.6.0                       | Not supported                                                                                 |
-| KIP-554 - Add Broker-side SCRAM Config API                               | 2.7.0                       | Supported                                                                                     |
-| KIP-559 - Make the Kafka Protocol Friendlier with L7 Proxies             | 2.5.0                       | Not supported                                                                                 |
-| KIP-568 - Explicit rebalance triggering on the Consumer                  | 2.6.0                       | Not supported                                                                                 |
-| KIP-659 - Add metadata to DescribeConfigsResponse                        | 2.6.0                       | Not supported                                                                                 |
-| KIP-580 - Exponential backoff for Kafka clients                          | WIP                         | Partially supported                                                                           |
-| KIP-584 - Versioning scheme for features                                 | WIP                         | Not supported                                                                                 |
-| KIP-588 - Allow producers to recover gracefully from txn timeouts        | 2.8.0 (WIP)                 | Not supported                                                                                 |
-| KIP-601 - Configurable socket connection timeout                         | 2.7.0                       | Supported                                                                                     |
-| KIP-602 - Use all resolved addresses by default                          | 2.6.0                       | Supported                                                                                     |
-| KIP-651 - Support PEM format for SSL certs and keys                      | 2.7.0                       | Supported                                                                                     |
-| KIP-654 - Aborted txns with non-flushed msgs should not be fatal         | 2.7.0                       | Supported                                                                                     |
-| KIP-735 - Increase default consumer session timeout                      | 3.0.0                       | Supported                                                                                     |
-| KIP-768 - SASL/OAUTHBEARER OIDC support                                  | 3.0                         | Supported                                                                                     |
-| KIP-881 - Rack-aware Partition Assignment for Kafka Consumers            | 3.5.0 (WIP)                 | Supported                                                                                     |
+| KIP-392 - Allow consumers to fetch from closest replica                  | 2.4.0                       | Supported                                                                        |
+| KIP-394 - Consumer: require member.id in JoinGroupRequest                | 2.2.0                       | Supported                                                                        |
+| KIP-396 - AdminAPI: commit/list offsets                                  | 2.4.0                       | Partially supported (remaining APIs available outside Admin client)              |
+| KIP-412 - AdminAPI: adjust log levels                                    | 2.4.0                       | Not supported                                                                    |
+| KIP-421 - Variables in client config files                               | 2.3.0                       | Not applicable (librdkafka, et.al, does not provide a config file interface, and shouldn't) |
+| KIP-429 - Consumer: incremental rebalance protocol                       | 2.4.0                       | Supported                                                                        |
+| KIP-430 - AdminAPI: return authorized operations in Describe.. responses | 2.3.0                       | Not supported                                                                    |
+| KIP-436 - Start time in stats                                            | 2.3.0                       | Supported                                                                        |
+| KIP-447 - Producer scalability for EOS                                   | 2.5.0                       | Supported                                                                        |
+| KIP-455 - AdminAPI: Replica assignment                                   | 2.4.0 (WIP)                 | Not supported                                                                    |
+| KIP-460 - AdminAPI: electPreferredLeader                                 | 2.4.0                       | Not supported                                                                    |
+| KIP-464 - AdminAPI: defaults for createTopics                            | 2.4.0                       | Supported                                                                        |
+| KIP-467 - Per-message (sort of) error codes in ProduceResponse           | 2.4.0 (WIP)                 | Not supported                                                                    |
+| KIP-480 - Sticky partitioner                                             | 2.4.0                       | Supported                                                                        |
+| KIP-482 - Optional fields in Kafka protocol                              | 2.4.0                       | Partially supported (ApiVersionRequest)                                          |
+| KIP-496 - AdminAPI: delete offsets                                       | 2.4.0                       | Supported                                                                        |
+| KIP-511 - Collect Client's Name and Version                              | 2.4.0                       | Supported                                                                        |
+| KIP-514 - Bounded flush()                                                | 2.4.0                       | Supported                                                                        |
+| KIP-516 - Topic Identifiers                                              | 2.8.0 (WIP)                 | Partially Supported                                                              |
+| KIP-517 - Consumer poll() metrics                                        | 2.4.0                       | Not supported                                                                    |
+| KIP-518 - Allow listing consumer groups per state                        | 2.6.0                       | Supported                                                                        |
+| KIP-519 - Make SSL engine configurable                                   | 2.6.0                       | Supported                                                                        |
+| KIP-525 - Return topic metadata and configs in CreateTopics response     | 2.4.0                       | Not supported                                                                    |
+| KIP-526 - Reduce Producer Metadata Lookups for Large Number of Topics    | 2.5.0                       | Not supported                                                                    |
+| KIP-533 - Add default API timeout to AdminClient                         | 2.5.0                       | Not supported                                                                    |
+| KIP-546 - Add Client Quota APIs to AdminClient                           | 2.6.0                       | Not supported                                                                    |
+| KIP-554 - Add Broker-side SCRAM Config API                               | 2.7.0                       | Supported                                                                        |
+| KIP-559 - Make the Kafka Protocol Friendlier with L7 Proxies             | 2.5.0                       | Not supported                                                                    |
+| KIP-568 - Explicit rebalance triggering on the Consumer                  | 2.6.0                       | Not supported                                                                    |
+| KIP-659 - Add metadata to DescribeConfigsResponse                        | 2.6.0                       | Not supported                                                                    |
+| KIP-580 - Exponential backoff for Kafka clients                          | WIP                         | Partially supported                                                              |
+| KIP-584 - Versioning scheme for features                                 | WIP                         | Not supported                                                                    |
+| KIP-588 - Allow producers to recover gracefully from txn timeouts        | 2.8.0 (WIP)                 | Not supported                                                                    |
+| KIP-601 - Configurable socket connection timeout                         | 2.7.0                       | Supported                                                                        |
+| KIP-602 - Use all resolved addresses by default                          | 2.6.0                       | Supported                                                                        |
+| KIP-651 - Support PEM format for SSL certs and keys                      | 2.7.0                       | Supported                                                                        |
+| KIP-654 - Aborted txns with non-flushed msgs should not be fatal         | 2.7.0                       | Supported                                                                        |
+| KIP-735 - Increase default consumer session timeout                      | 3.0.0                       | Supported                                                                        |
+| KIP-768 - SASL/OAUTHBEARER OIDC support                                  | 3.0                         | Supported                                                                        |
+| KIP-881 - Rack-aware Partition Assignment for Kafka Consumers            | 3.5.0 (WIP)                 | Supported                                                                        |
 
 
 

From 47bb41b3eb4fbee678bfdd8e6e8de23be4d9018b Mon Sep 17 00:00:00 2001
From: Pranav Rathi <pranav.rth@gmail.com>
Date: Fri, 4 Aug 2023 17:32:19 +0530
Subject: [PATCH 5/7] Fixed warning when configuring without SSL

---
 src/rdbase64.c | 18 +++++++++---------
 1 file changed, 9 insertions(+), 9 deletions(-)

diff --git a/src/rdbase64.c b/src/rdbase64.c
index 1a1dc31e46..6b81ee42bc 100644
--- a/src/rdbase64.c
+++ b/src/rdbase64.c
@@ -41,11 +41,7 @@
  */
 void rd_base64_encode(const rd_chariov_t *in, rd_chariov_t *out) {
 
-#if !WITH_SSL
-        out->ptr = NULL;
-        return;
-#endif
-
+#if WITH_SSL
         size_t max_len;
 
         /* OpenSSL takes an |int| argument so the input cannot exceed that. */
@@ -64,6 +60,10 @@ void rd_base64_encode(const rd_chariov_t *in, rd_chariov_t *out) {
 
         rd_assert(out->size < max_len);
         out->ptr[out->size] = 0;
+#else
+        out->ptr = NULL;
+        return;
+#endif
 }
 
 
@@ -91,10 +91,7 @@ char *rd_base64_encode_str(const rd_chariov_t *in) {
  */
 int rd_base64_decode(const rd_chariov_t *in, rd_chariov_t *out) {
 
-#if !WITH_SSL
-        return -2;
-#endif
-
+#if WITH_SSL
         size_t ret_len;
 
         /* OpenSSL takes an |int| argument, so |in->size| must not exceed
@@ -127,4 +124,7 @@ int rd_base64_decode(const rd_chariov_t *in, rd_chariov_t *out) {
         out->size         = ret_len;
 
         return 0;
+#else
+        return -2;
+#endif
 }
\ No newline at end of file

From 73be0f82b39d99f18526e9f94c0bb1f0ddb7d3c8 Mon Sep 17 00:00:00 2001
From: Pranav Rathi <pranav.rth@gmail.com>
Date: Mon, 7 Aug 2023 14:42:14 +0530
Subject: [PATCH 6/7] PR Review changes

---
 CHANGELOG.md                        |   6 +-
 INTRODUCTION.md                     | 224 ++++++++++++++--------------
 src/rdbase64.c                      |  11 +-
 src/rdkafka_proto.h                 |  39 +++--
 src/rdkafka_request.c               |   2 +-
 src/rdkafka_sasl_oauthbearer_oidc.c |   2 +
 6 files changed, 149 insertions(+), 135 deletions(-)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index 54b4c293e7..22beb99799 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,9 +1,11 @@
-# librdkafka v2.2.0
+# librdkafka v2.3.0
 
-librdkafka v2.2.0 is a feature release:
+librdkafka v2.3.0 is a feature release:
 
  * Added Topic id to the metadata response which is part of the [KIP-516](https://cwiki.apache.org/confluence/display/KAFKA/KIP-516%3A+Topic+Identifiers)
 
+
+
 # librdkafka v2.2.0
 
 librdkafka v2.2.0 is a feature release:
diff --git a/INTRODUCTION.md b/INTRODUCTION.md
index 19d4e58f24..c360719d26 100644
--- a/INTRODUCTION.md
+++ b/INTRODUCTION.md
@@ -1847,119 +1847,119 @@ librdkafka v0.11.0.
 The [Apache Kafka Implementation Proposals (KIPs)](https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals) supported by librdkafka.
 
 
-| KIP                                                                      | Kafka release               | Status                                                                           |
-|--------------------------------------------------------------------------|-----------------------------|----------------------------------------------------------------------------------|
-| KIP-1 - Stop accepting request.required.acks > 1                         | 0.9.0.0                     | Not enforced on client (due to backwards compat with brokers  <0.8.3)            |
-| KIP-4 - Metadata protocol changes                                        | 0.9.0.0, 0.10.0.0, 0.10.1.0 | Supported                                                                        |
-| KIP-8 - Producer flush()                                                 | 0.9.0.0                     | Supported                                                                        |
-| KIP-12 - SASL Kerberos                                                   | 0.9.0.0                     | Supported (uses SSPI/logged-on-user on Windows, full KRB5 keytabs on Unix)       |
-| KIP-13 - Protocol request throttling (enforced on broker)                | 0.9.0.0                     | Supported                                                                        |
-| KIP-15 - Producer close with timeout                                     | 0.9.0.0                     | Supported (through flush() + destroy())                                          |
-| KIP-19 - Request timeouts                                                | 0.9.0.0                     | Supported                                                                        |
-| KIP-22 - Producer pluggable partitioner                                  | 0.9.0.0                     | Supported (not supported by Go, .NET and Python)                                 |
-| KIP-31 - Relative offsets in messagesets                                 | 0.10.0.0                    | Supported                                                                        |
-| KIP-35 - ApiVersionRequest                                               | 0.10.0.0                    | Supported                                                                        |
-| KIP-40 - ListGroups and DescribeGroups                                   | 0.9.0.0                     | Supported                                                                        |
-| KIP-41 - max.poll.records                                                | 0.10.0.0                    | Supported through batch consumption interface (not supported by .NET and Go)     |
-| KIP-42 - Producer and Consumer interceptors                              | 0.10.0.0                    | Supported (not supported by Go, .NET and Python)                                 |
-| KIP-43 - SASL PLAIN and handshake                                        | 0.10.0.0                    | Supported                                                                        |
-| KIP-48 - Delegation tokens                                               | 1.1.0                       | Not supported                                                                    |
-| KIP-54 - Sticky partition assignment strategy                            | 0.11.0.0                    | Supported but not available, use KIP-429 instead.                                |
-| KIP-57 - Interoperable LZ4 framing                                       | 0.10.0.0                    | Supported                                                                        |
-| KIP-62 - max.poll.interval and background heartbeats                     | 0.10.1.0                    | Supported                                                                        |
-| KIP-70 - Proper client rebalance event on unsubscribe/subscribe          | 0.10.1.0                    | Supported                                                                        |
-| KIP-74 - max.partition.fetch.bytes                                       | 0.10.1.0                    | Supported                                                                        |
-| KIP-78 - Retrieve Cluster Id                                             | 0.10.1.0                    | Supported (not supported by .NET)                                                |
-| KIP-79 - OffsetsForTimes                                                 | 0.10.1.0                    | Supported                                                                        |
-| KIP-81 - Consumer pre-fetch buffer size                                  | 2.4.0 (WIP)                 | Supported                                                                        |
-| KIP-82 - Record Headers                                                  | 0.11.0.0                    | Supported                                                                        |
-| KIP-84 - SASL SCRAM                                                      | 0.10.2.0                    | Supported                                                                        |
-| KIP-85 - SASL config properties                                          | 0.10.2.0                    | Supported                                                                        |
-| KIP-86 - Configurable SASL callbacks                                     | 2.0.0                       | Not supported                                                                    |
-| KIP-88 - AdminAPI: ListGroupOffsets                                      | 0.10.2.0                    | Supported                                                                        |
-| KIP-91 - Intuitive timeouts in Producer                                  | 2.1.0                       | Supported                                                                        |
-| KIP-92 - Per-partition lag metrics in Consumer                           | 0.10.2.0                    | Supported                                                                        |
-| KIP-97 - Backwards compatibility with older brokers                      | 0.10.2.0                    | Supported                                                                        |
-| KIP-98 - EOS                                                             | 0.11.0.0                    | Supported                                                                        |
-| KIP-102 - Close with timeout in consumer                                 | 0.10.2.0                    | Not supported                                                                    |
-| KIP-107 - AdminAPI: DeleteRecordsBefore                                  | 0.11.0.0                    | Supported                                                                        |
-| KIP-110 - ZStd compression                                               | 2.1.0                       | Supported                                                                        |
-| KIP-117 - AdminClient                                                    | 0.11.0.0                    | Supported                                                                        |
-| KIP-124 - Request rate quotas                                            | 0.11.0.0                    | Partially supported (depending on protocol request)                              |
-| KIP-126 - Producer ensure proper batch size after compression            | 0.11.0.0                    | Supported                                                                        |
-| KIP-133 - AdminAPI: DescribeConfigs and AlterConfigs                     | 0.11.0.0                    | Supported                                                                        |
-| KIP-140 - AdminAPI: ACLs                                                 | 0.11.0.0                    | Supported                                                                        |
-| KIP-144 - Broker reconnect backoff                                       | 0.11.0.0                    | Supported                                                                        |
-| KIP-152 - Improved SASL auth error messages                              | 1.0.0                       | Supported                                                                        |
-| KIP-192 - Cleaner idempotence semantics                                  | 1.0.0                       | Not supported (superceeded by KIP-360)                                           |
-| KIP-195 - AdminAPI: CreatePartitions                                     | 1.0.0                       | Supported                                                                        |
-| KIP-204 - AdminAPI: DeleteRecords                                        | 1.1.0                       | Supported                                                                        |
-| KIP-219 - Client-side throttling                                         | 2.0.0                       | Not supported                                                                    |
-| KIP-222 - AdminAPI: Consumer group operations                            | 2.0.0                       | Supported                                                                        |
-| KIP-223 - Consumer partition lead metric                                 | 2.0.0                       | Not supported                                                                    |
-| KIP-226 - AdminAPI: Dynamic broker config                                | 1.1.0                       | Supported                                                                        |
-| KIP-227 - Consumer Incremental Fetch                                     | 1.1.0                       | Not supported                                                                    |
-| KIP-229 - AdminAPI: DeleteGroups                                         | 1.1.0                       | Supported                                                                        |
-| KIP-235 - DNS alias for secure connections                               | 2.1.0                       | Supported                                                                        |
-| KIP-249 - AdminAPI: Deletegation Tokens                                  | 2.0.0                       | Not supported                                                                    |
-| KIP-255 - SASL OAUTHBEARER                                               | 2.0.0                       | Supported                                                                        |
-| KIP-266 - Fix indefinite consumer timeouts                               | 2.0.0                       | Supported (bound by session.timeout.ms and max.poll.interval.ms)                 |
-| KIP-289 - Consumer group.id default to NULL                              | 2.2.0                       | Supported                                                                        |
-| KIP-294 - SSL endpoint verification                                      | 2.0.0                       | Supported                                                                        |
-| KIP-302 - Use all addresses for resolved broker hostname                 | 2.1.0                       | Supported                                                                        |
-| KIP-320 - Consumer: handle log truncation                                | 2.1.0, 2.2.0                | Supported                                                                        |
-| KIP-322 - DeleteTopics disabled error code                               | 2.1.0                       | Supported                                                                        |
-| KIP-339 - AdminAPI: incrementalAlterConfigs                              | 2.3.0                       | Supported                                                                        |
-| KIP-341 - Update Sticky partition assignment data                        | 2.3.0                       | Not supported (superceeded by KIP-429)                                           |
-| KIP-342 - Custom SASL OAUTHBEARER extensions                             | 2.1.0                       | Supported                                                                        |
-| KIP-345 - Consumer: Static membership                                    | 2.4.0                       | Supported                                                                        |
-| KIP-357 - AdminAPI: list ACLs per principal                              | 2.1.0                       | Not supported                                                                    |
-| KIP-359 - Producer: use EpochLeaderId                                    | 2.4.0                       | Not supported                                                                    |
-| KIP-360 - Improve handling of unknown Idempotent Producer                | 2.5.0                       | Supported                                                                        |
-| KIP-361 - Consumer: add config to disable auto topic creation            | 2.3.0                       | Supported                                                                        |
-| KIP-368 - SASL periodic reauth                                           | 2.2.0                       | Supported                                                                        |
-| KIP-369 - Always roundRobin partitioner                                  | 2.4.0                       | Not supported                                                                    |
+| KIP                                                                      | Kafka release               | Status                                                                                        |
+|--------------------------------------------------------------------------|-----------------------------|-----------------------------------------------------------------------------------------------|
+| KIP-1 - Stop accepting request.required.acks > 1                         | 0.9.0.0                     | Not enforced on client (due to backwards compat with brokers  <0.8.3)                         |
+| KIP-4 - Metadata protocol changes                                        | 0.9.0.0, 0.10.0.0, 0.10.1.0 | Supported                                                                                     |
+| KIP-8 - Producer flush()                                                 | 0.9.0.0                     | Supported                                                                                     |
+| KIP-12 - SASL Kerberos                                                   | 0.9.0.0                     | Supported (uses SSPI/logged-on-user on Windows, full KRB5 keytabs on Unix)                    |
+| KIP-13 - Protocol request throttling (enforced on broker)                | 0.9.0.0                     | Supported                                                                                     |
+| KIP-15 - Producer close with timeout                                     | 0.9.0.0                     | Supported (through flush() + destroy())                                                       |
+| KIP-19 - Request timeouts                                                | 0.9.0.0                     | Supported                                                                                     |
+| KIP-22 - Producer pluggable partitioner                                  | 0.9.0.0                     | Supported (not supported by Go, .NET and Python)                                              |
+| KIP-31 - Relative offsets in messagesets                                 | 0.10.0.0                    | Supported                                                                                     |
+| KIP-35 - ApiVersionRequest                                               | 0.10.0.0                    | Supported                                                                                     |
+| KIP-40 - ListGroups and DescribeGroups                                   | 0.9.0.0                     | Supported                                                                                     |
+| KIP-41 - max.poll.records                                                | 0.10.0.0                    | Supported through batch consumption interface (not supported by .NET and Go)                  |
+| KIP-42 - Producer and Consumer interceptors                              | 0.10.0.0                    | Supported (not supported by Go, .NET and Python)                                              |
+| KIP-43 - SASL PLAIN and handshake                                        | 0.10.0.0                    | Supported                                                                                     |
+| KIP-48 - Delegation tokens                                               | 1.1.0                       | Not supported                                                                                 |
+| KIP-54 - Sticky partition assignment strategy                            | 0.11.0.0                    | Supported but not available, use KIP-429 instead.                                             |
+| KIP-57 - Interoperable LZ4 framing                                       | 0.10.0.0                    | Supported                                                                                     |
+| KIP-62 - max.poll.interval and background heartbeats                     | 0.10.1.0                    | Supported                                                                                     |
+| KIP-70 - Proper client rebalance event on unsubscribe/subscribe          | 0.10.1.0                    | Supported                                                                                     |
+| KIP-74 - max.partition.fetch.bytes                                       | 0.10.1.0                    | Supported                                                                                     |
+| KIP-78 - Retrieve Cluster Id                                             | 0.10.1.0                    | Supported (not supported by .NET)                                                             |
+| KIP-79 - OffsetsForTimes                                                 | 0.10.1.0                    | Supported                                                                                     |
+| KIP-81 - Consumer pre-fetch buffer size                                  | 2.4.0 (WIP)                 | Supported                                                                                     |
+| KIP-82 - Record Headers                                                  | 0.11.0.0                    | Supported                                                                                     |
+| KIP-84 - SASL SCRAM                                                      | 0.10.2.0                    | Supported                                                                                     |
+| KIP-85 - SASL config properties                                          | 0.10.2.0                    | Supported                                                                                     |
+| KIP-86 - Configurable SASL callbacks                                     | 2.0.0                       | Not supported                                                                                 |
+| KIP-88 - AdminAPI: ListGroupOffsets                                      | 0.10.2.0                    | Supported                                                                                     |
+| KIP-91 - Intuitive timeouts in Producer                                  | 2.1.0                       | Supported                                                                                     |
+| KIP-92 - Per-partition lag metrics in Consumer                           | 0.10.2.0                    | Supported                                                                                     |
+| KIP-97 - Backwards compatibility with older brokers                      | 0.10.2.0                    | Supported                                                                                     |
+| KIP-98 - EOS                                                             | 0.11.0.0                    | Supported                                                                                     |
+| KIP-102 - Close with timeout in consumer                                 | 0.10.2.0                    | Not supported                                                                                 |
+| KIP-107 - AdminAPI: DeleteRecordsBefore                                  | 0.11.0.0                    | Supported                                                                                     |
+| KIP-110 - ZStd compression                                               | 2.1.0                       | Supported                                                                                     |
+| KIP-117 - AdminClient                                                    | 0.11.0.0                    | Supported                                                                                     |
+| KIP-124 - Request rate quotas                                            | 0.11.0.0                    | Partially supported (depending on protocol request)                                           |
+| KIP-126 - Producer ensure proper batch size after compression            | 0.11.0.0                    | Supported                                                                                     |
+| KIP-133 - AdminAPI: DescribeConfigs and AlterConfigs                     | 0.11.0.0                    | Supported                                                                                     |
+| KIP-140 - AdminAPI: ACLs                                                 | 0.11.0.0                    | Supported                                                                                     |
+| KIP-144 - Broker reconnect backoff                                       | 0.11.0.0                    | Supported                                                                                     |
+| KIP-152 - Improved SASL auth error messages                              | 1.0.0                       | Supported                                                                                     |
+| KIP-192 - Cleaner idempotence semantics                                  | 1.0.0                       | Not supported (superceeded by KIP-360)                                                        |
+| KIP-195 - AdminAPI: CreatePartitions                                     | 1.0.0                       | Supported                                                                                     |
+| KIP-204 - AdminAPI: DeleteRecords                                        | 1.1.0                       | Supported                                                                                     |
+| KIP-219 - Client-side throttling                                         | 2.0.0                       | Not supported                                                                                 |
+| KIP-222 - AdminAPI: Consumer group operations                            | 2.0.0                       | Supported                                                                                     |
+| KIP-223 - Consumer partition lead metric                                 | 2.0.0                       | Not supported                                                                                 |
+| KIP-226 - AdminAPI: Dynamic broker config                                | 1.1.0                       | Supported                                                                                     |
+| KIP-227 - Consumer Incremental Fetch                                     | 1.1.0                       | Not supported                                                                                 |
+| KIP-229 - AdminAPI: DeleteGroups                                         | 1.1.0                       | Supported                                                                                     |
+| KIP-235 - DNS alias for secure connections                               | 2.1.0                       | Supported                                                                                     |
+| KIP-249 - AdminAPI: Deletegation Tokens                                  | 2.0.0                       | Not supported                                                                                 |
+| KIP-255 - SASL OAUTHBEARER                                               | 2.0.0                       | Supported                                                                                     |
+| KIP-266 - Fix indefinite consumer timeouts                               | 2.0.0                       | Supported (bound by session.timeout.ms and max.poll.interval.ms)                              |
+| KIP-289 - Consumer group.id default to NULL                              | 2.2.0                       | Supported                                                                                     |
+| KIP-294 - SSL endpoint verification                                      | 2.0.0                       | Supported                                                                                     |
+| KIP-302 - Use all addresses for resolved broker hostname                 | 2.1.0                       | Supported                                                                                     |
+| KIP-320 - Consumer: handle log truncation                                | 2.1.0, 2.2.0                | Supported                                                                                     |
+| KIP-322 - DeleteTopics disabled error code                               | 2.1.0                       | Supported                                                                                     |
+| KIP-339 - AdminAPI: incrementalAlterConfigs                              | 2.3.0                       | Supported                                                                                     |
+| KIP-341 - Update Sticky partition assignment data                        | 2.3.0                       | Not supported (superceeded by KIP-429)                                                        |
+| KIP-342 - Custom SASL OAUTHBEARER extensions                             | 2.1.0                       | Supported                                                                                     |
+| KIP-345 - Consumer: Static membership                                    | 2.4.0                       | Supported                                                                                     |
+| KIP-357 - AdminAPI: list ACLs per principal                              | 2.1.0                       | Not supported                                                                                 |
+| KIP-359 - Producer: use EpochLeaderId                                    | 2.4.0                       | Not supported                                                                                 |
+| KIP-360 - Improve handling of unknown Idempotent Producer                | 2.5.0                       | Supported                                                                                     |
+| KIP-361 - Consumer: add config to disable auto topic creation            | 2.3.0                       | Supported                                                                                     |
+| KIP-368 - SASL periodic reauth                                           | 2.2.0                       | Supported                                                                                     |
+| KIP-369 - Always roundRobin partitioner                                  | 2.4.0                       | Not supported                                                                                 |
 | KIP-389 - Consumer group max size                                        | 2.2.0                       | Supported (error is propagated to application, but the consumer does not raise a fatal error) |
-| KIP-392 - Allow consumers to fetch from closest replica                  | 2.4.0                       | Supported                                                                        |
-| KIP-394 - Consumer: require member.id in JoinGroupRequest                | 2.2.0                       | Supported                                                                        |
-| KIP-396 - AdminAPI: commit/list offsets                                  | 2.4.0                       | Partially supported (remaining APIs available outside Admin client)              |
-| KIP-412 - AdminAPI: adjust log levels                                    | 2.4.0                       | Not supported                                                                    |
-| KIP-421 - Variables in client config files                               | 2.3.0                       | Not applicable (librdkafka, et.al, does not provide a config file interface, and shouldn't) |
-| KIP-429 - Consumer: incremental rebalance protocol                       | 2.4.0                       | Supported                                                                        |
-| KIP-430 - AdminAPI: return authorized operations in Describe.. responses | 2.3.0                       | Not supported                                                                    |
-| KIP-436 - Start time in stats                                            | 2.3.0                       | Supported                                                                        |
-| KIP-447 - Producer scalability for EOS                                   | 2.5.0                       | Supported                                                                        |
-| KIP-455 - AdminAPI: Replica assignment                                   | 2.4.0 (WIP)                 | Not supported                                                                    |
-| KIP-460 - AdminAPI: electPreferredLeader                                 | 2.4.0                       | Not supported                                                                    |
-| KIP-464 - AdminAPI: defaults for createTopics                            | 2.4.0                       | Supported                                                                        |
-| KIP-467 - Per-message (sort of) error codes in ProduceResponse           | 2.4.0 (WIP)                 | Not supported                                                                    |
-| KIP-480 - Sticky partitioner                                             | 2.4.0                       | Supported                                                                        |
-| KIP-482 - Optional fields in Kafka protocol                              | 2.4.0                       | Partially supported (ApiVersionRequest)                                          |
-| KIP-496 - AdminAPI: delete offsets                                       | 2.4.0                       | Supported                                                                        |
-| KIP-511 - Collect Client's Name and Version                              | 2.4.0                       | Supported                                                                        |
-| KIP-514 - Bounded flush()                                                | 2.4.0                       | Supported                                                                        |
-| KIP-516 - Topic Identifiers                                              | 2.8.0 (WIP)                 | Partially Supported                                                              |
-| KIP-517 - Consumer poll() metrics                                        | 2.4.0                       | Not supported                                                                    |
-| KIP-518 - Allow listing consumer groups per state                        | 2.6.0                       | Supported                                                                        |
-| KIP-519 - Make SSL engine configurable                                   | 2.6.0                       | Supported                                                                        |
-| KIP-525 - Return topic metadata and configs in CreateTopics response     | 2.4.0                       | Not supported                                                                    |
-| KIP-526 - Reduce Producer Metadata Lookups for Large Number of Topics    | 2.5.0                       | Not supported                                                                    |
-| KIP-533 - Add default API timeout to AdminClient                         | 2.5.0                       | Not supported                                                                    |
-| KIP-546 - Add Client Quota APIs to AdminClient                           | 2.6.0                       | Not supported                                                                    |
-| KIP-554 - Add Broker-side SCRAM Config API                               | 2.7.0                       | Supported                                                                        |
-| KIP-559 - Make the Kafka Protocol Friendlier with L7 Proxies             | 2.5.0                       | Not supported                                                                    |
-| KIP-568 - Explicit rebalance triggering on the Consumer                  | 2.6.0                       | Not supported                                                                    |
-| KIP-659 - Add metadata to DescribeConfigsResponse                        | 2.6.0                       | Not supported                                                                    |
-| KIP-580 - Exponential backoff for Kafka clients                          | WIP                         | Partially supported                                                              |
-| KIP-584 - Versioning scheme for features                                 | WIP                         | Not supported                                                                    |
-| KIP-588 - Allow producers to recover gracefully from txn timeouts        | 2.8.0 (WIP)                 | Not supported                                                                    |
-| KIP-601 - Configurable socket connection timeout                         | 2.7.0                       | Supported                                                                        |
-| KIP-602 - Use all resolved addresses by default                          | 2.6.0                       | Supported                                                                        |
-| KIP-651 - Support PEM format for SSL certs and keys                      | 2.7.0                       | Supported                                                                        |
-| KIP-654 - Aborted txns with non-flushed msgs should not be fatal         | 2.7.0                       | Supported                                                                        |
-| KIP-735 - Increase default consumer session timeout                      | 3.0.0                       | Supported                                                                        |
-| KIP-768 - SASL/OAUTHBEARER OIDC support                                  | 3.0                         | Supported                                                                        |
-| KIP-881 - Rack-aware Partition Assignment for Kafka Consumers            | 3.5.0 (WIP)                 | Supported                                                                        |
+| KIP-392 - Allow consumers to fetch from closest replica                  | 2.4.0                       | Supported                                                                                     |
+| KIP-394 - Consumer: require member.id in JoinGroupRequest                | 2.2.0                       | Supported                                                                                     |
+| KIP-396 - AdminAPI: commit/list offsets                                  | 2.4.0                       | Partially supported (remaining APIs available outside Admin client)                           |
+| KIP-412 - AdminAPI: adjust log levels                                    | 2.4.0                       | Not supported                                                                                 |
+| KIP-421 - Variables in client config files                               | 2.3.0                       | Not applicable (librdkafka, et.al, does not provide a config file interface, and shouldn't)   |
+| KIP-429 - Consumer: incremental rebalance protocol                       | 2.4.0                       | Supported                                                                                     |
+| KIP-430 - AdminAPI: return authorized operations in Describe.. responses | 2.3.0                       | Not supported                                                                                 |
+| KIP-436 - Start time in stats                                            | 2.3.0                       | Supported                                                                                     |
+| KIP-447 - Producer scalability for EOS                                   | 2.5.0                       | Supported                                                                                     |
+| KIP-455 - AdminAPI: Replica assignment                                   | 2.4.0 (WIP)                 | Not supported                                                                                 |
+| KIP-460 - AdminAPI: electPreferredLeader                                 | 2.4.0                       | Not supported                                                                                 |
+| KIP-464 - AdminAPI: defaults for createTopics                            | 2.4.0                       | Supported                                                                                     |
+| KIP-467 - Per-message (sort of) error codes in ProduceResponse           | 2.4.0 (WIP)                 | Not supported                                                                                 |
+| KIP-480 - Sticky partitioner                                             | 2.4.0                       | Supported                                                                                     |
+| KIP-482 - Optional fields in Kafka protocol                              | 2.4.0                       | Partially supported (ApiVersionRequest)                                                       |
+| KIP-496 - AdminAPI: delete offsets                                       | 2.4.0                       | Supported                                                                                     |
+| KIP-511 - Collect Client's Name and Version                              | 2.4.0                       | Supported                                                                                     |
+| KIP-514 - Bounded flush()                                                | 2.4.0                       | Supported                                                                                     |
+| KIP-516 - Topic Identifiers                                              | 2.8.0 (WIP)                 | Partially Supported                                                                           |
+| KIP-517 - Consumer poll() metrics                                        | 2.4.0                       | Not supported                                                                                 |
+| KIP-518 - Allow listing consumer groups per state                        | 2.6.0                       | Supported                                                                                     |
+| KIP-519 - Make SSL engine configurable                                   | 2.6.0                       | Supported                                                                                     |
+| KIP-525 - Return topic metadata and configs in CreateTopics response     | 2.4.0                       | Not supported                                                                                 |
+| KIP-526 - Reduce Producer Metadata Lookups for Large Number of Topics    | 2.5.0                       | Not supported                                                                                 |
+| KIP-533 - Add default API timeout to AdminClient                         | 2.5.0                       | Not supported                                                                                 |
+| KIP-546 - Add Client Quota APIs to AdminClient                           | 2.6.0                       | Not supported                                                                                 |
+| KIP-554 - Add Broker-side SCRAM Config API                               | 2.7.0                       | Supported                                                                                     |
+| KIP-559 - Make the Kafka Protocol Friendlier with L7 Proxies             | 2.5.0                       | Not supported                                                                                 |
+| KIP-568 - Explicit rebalance triggering on the Consumer                  | 2.6.0                       | Not supported                                                                                 |
+| KIP-659 - Add metadata to DescribeConfigsResponse                        | 2.6.0                       | Not supported                                                                                 |
+| KIP-580 - Exponential backoff for Kafka clients                          | WIP                         | Partially supported                                                                           |
+| KIP-584 - Versioning scheme for features                                 | WIP                         | Not supported                                                                                 |
+| KIP-588 - Allow producers to recover gracefully from txn timeouts        | 2.8.0 (WIP)                 | Not supported                                                                                 |
+| KIP-601 - Configurable socket connection timeout                         | 2.7.0                       | Supported                                                                                     |
+| KIP-602 - Use all resolved addresses by default                          | 2.6.0                       | Supported                                                                                     |
+| KIP-651 - Support PEM format for SSL certs and keys                      | 2.7.0                       | Supported                                                                                     |
+| KIP-654 - Aborted txns with non-flushed msgs should not be fatal         | 2.7.0                       | Supported                                                                                     |
+| KIP-735 - Increase default consumer session timeout                      | 3.0.0                       | Supported                                                                                     |
+| KIP-768 - SASL/OAUTHBEARER OIDC support                                  | 3.0                         | Supported                                                                                     |
+| KIP-881 - Rack-aware Partition Assignment for Kafka Consumers            | 3.5.0 (WIP)                 | Supported                                                                                     |
 
 
 
diff --git a/src/rdbase64.c b/src/rdbase64.c
index 6b81ee42bc..d81858418f 100644
--- a/src/rdbase64.c
+++ b/src/rdbase64.c
@@ -37,7 +37,7 @@
  *        and it's size to \p out. out->ptr will be NULL in case of some issue
  *        with the conversion or the conversion is not supported.
  *
- * @post out->ptr must be freed after use.
+ * @remark out->ptr must be freed after use.
  */
 void rd_base64_encode(const rd_chariov_t *in, rd_chariov_t *out) {
 
@@ -52,8 +52,6 @@ void rd_base64_encode(const rd_chariov_t *in, rd_chariov_t *out) {
 
         max_len  = (((in->size + 2) / 3) * 4) + 1;
         out->ptr = rd_malloc(max_len);
-        if (out->ptr == NULL)
-                return;
 
         out->size = EVP_EncodeBlock((unsigned char *)out->ptr,
                                     (unsigned char *)in->ptr, (int)in->size);
@@ -62,7 +60,6 @@ void rd_base64_encode(const rd_chariov_t *in, rd_chariov_t *out) {
         out->ptr[out->size] = 0;
 #else
         out->ptr = NULL;
-        return;
 #endif
 }
 
@@ -70,9 +67,9 @@ void rd_base64_encode(const rd_chariov_t *in, rd_chariov_t *out) {
 /**
  * @brief Base64 encode binary input \p in.
  * @returns a newly allocated, base64-encoded string or NULL in case of some
- * issue with the conversion or the conversion is not supported.
+ *          issue with the conversion or the conversion is not supported.
  *
- * @post Returned string must be freed after use.
+ * @remark Returned string must be freed after use.
  */
 char *rd_base64_encode_str(const rd_chariov_t *in) {
         rd_chariov_t out;
@@ -85,7 +82,7 @@ char *rd_base64_encode_str(const rd_chariov_t *in) {
  * @brief Base64 decode input string \p in. Ignores leading and trailing
  *         whitespace.
  * @returns * 0 on successes in which case a newly allocated binary string is
- * set in out (and size).
+ *            set in \p out (and size).
  *          * -1 on invalid Base64.
  *          * -2 on conversion not supported.
  */
diff --git a/src/rdkafka_proto.h b/src/rdkafka_proto.h
index 9e0efa79f0..6ee948f2cb 100644
--- a/src/rdkafka_proto.h
+++ b/src/rdkafka_proto.h
@@ -2,6 +2,8 @@
  * librdkafka - Apache Kafka C library
  *
  * Copyright (c) 2012-2022, Magnus Edenhill
+ *               2023, Confluent Inc.
+
  * All rights reserved.
  *
  * Redistribution and use in source and binary forms, with or without
@@ -568,29 +570,40 @@ typedef struct rd_kafka_buf_s rd_kafka_buf_t;
 
 
 /**
- * @brief UUID
- *
- * @param most_significant_bits - most significant 64 bits for the UUID
- * @param least_significant_bits - least significant 64 bits for the UUID
- * @param base64str - base64 encoding for the uuid. By default, it is lazy
- * loaded. Use function `rd_kafka_uuid_base64str()` as a getter for this field.
+ * @struct Struct representing UUID protocol primitive type.
  */
 typedef struct rd_kafka_uuid_s {
-        int64_t most_significant_bits;
-        int64_t least_significant_bits;
-        char base64str[23];
+        int64_t
+            most_significant_bits; /**< Most significant 64 bits for the UUID */
+        int64_t least_significant_bits; /**< Least significant 64 bits for the
+                                           UUID */
+        char base64str[23]; /**< base64 encoding for the uuid. By default, it is
+                               lazy loaded. Use function
+                               `rd_kafka_uuid_base64str()` as a getter for this
+                               field. */
 } rd_kafka_uuid_t;
 
-#define RD_KAFKA_ZERO_UUID                                                     \
+#define RD_KAFKA_UUID_ZERO                                                     \
         { 0, 0, "" }
 
-#define RD_KAFKA_METADATA_TOPIC_ID                                             \
+#define RD_KAFKA_UUID_METADATA_TOPIC_ID                                        \
         { 0, 1, "" }
 
+
+/**
+ * Creates a new UUID.
+ *
+ * @return A newly allocated UUID.
+ */
+static RD_INLINE RD_UNUSED rd_kafka_uuid_t *rd_kafka_uuid_new() {
+        rd_kafka_uuid_t *uuid = rd_calloc(1, sizeof(rd_kafka_uuid_t *));
+        return uuid;
+}
+
 /**
  * Initialize given UUID to zero UUID.
  *
- * @param uuid - UUID to initialize.
+ * @param uuid UUID to initialize.
  */
 static RD_INLINE RD_UNUSED void rd_kafka_uuid_init(rd_kafka_uuid_t *uuid) {
         memset(uuid, 0, sizeof(*uuid));
@@ -605,7 +618,7 @@ static RD_INLINE RD_UNUSED void rd_kafka_uuid_init(rd_kafka_uuid_t *uuid) {
  */
 static RD_INLINE RD_UNUSED char *
 rd_kafka_uuid_base64str(rd_kafka_uuid_t *uuid) {
-        if (strlen(uuid->base64str))
+        if (*uuid->base64str)
                 return uuid->base64str;
 
         rd_chariov_t in_base64;
diff --git a/src/rdkafka_request.c b/src/rdkafka_request.c
index b01973a2cf..e96b0f7c78 100644
--- a/src/rdkafka_request.c
+++ b/src/rdkafka_request.c
@@ -2305,7 +2305,7 @@ rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb,
         if (topic_cnt > 0) {
                 char *topic;
                 int i;
-                rd_kafka_uuid_t zero_uuid = RD_KAFKA_ZERO_UUID;
+                rd_kafka_uuid_t zero_uuid = RD_KAFKA_UUID_ZERO;
 
                 /* Maintain a copy of the topics list so we can purge
                  * hints from the metadata cache on error. */
diff --git a/src/rdkafka_sasl_oauthbearer_oidc.c b/src/rdkafka_sasl_oauthbearer_oidc.c
index 57a6d2e30e..d56efbf355 100644
--- a/src/rdkafka_sasl_oauthbearer_oidc.c
+++ b/src/rdkafka_sasl_oauthbearer_oidc.c
@@ -2,6 +2,8 @@
  * librdkafka - The Apache Kafka C/C++ library
  *
  * Copyright (c) 2021-2022, Magnus Edenhill
+ *               2023, Confluent Inc.
+
  * All rights reserved.
  *
  * Redistribution and use in source and binary forms, with or without

From 66cd81183beac7cbcc3e50502ac66e6cdd7ef157 Mon Sep 17 00:00:00 2001
From: Pranav Rathi <4427674+pranavrth@users.noreply.github.com>
Date: Mon, 7 Aug 2023 18:53:37 +0530
Subject: [PATCH 7/7] Updated changelog version to 2.2.1

---
 CHANGELOG.md | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index 22beb99799..11b1be60ae 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,6 +1,6 @@
-# librdkafka v2.3.0
+# librdkafka v2.2.1
 
-librdkafka v2.3.0 is a feature release:
+librdkafka v2.2.1 is a maintenance release:
 
  * Added Topic id to the metadata response which is part of the [KIP-516](https://cwiki.apache.org/confluence/display/KAFKA/KIP-516%3A+Topic+Identifiers)