summaryrefslogtreecommitdiff
blob: cc6b57ea72d22d6b7555eab680a1e1509eb36464 (plain)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
https://bugs.gentoo.org/915433
https://github.com/confluentinc/librdkafka/pull/4449

From 8b311b8a850805f4ec9bb068c0edb31492ad03fe Mon Sep 17 00:00:00 2001
From: Emanuele Sabellico <esabellico@confluent.io>
Date: Wed, 27 Sep 2023 11:08:33 +0200
Subject: [PATCH 1/3] tmpabuf refactor and fix for insufficient buffer
 allocation

---
 CHANGELOG.md                 | 10 ++++++
 src/rdkafka_buf.h            | 25 ++++++++++++---
 src/rdkafka_metadata.c       | 59 +++++++++++++++++++-----------------
 src/rdkafka_metadata_cache.c | 38 +++++++++++------------
 src/rdkafka_topic.c          | 36 +++++++++++++---------
 5 files changed, 100 insertions(+), 68 deletions(-)

diff --git a/src/rdkafka_buf.h b/src/rdkafka_buf.h
index ccd563cc6..623ec49ae 100644
--- a/src/rdkafka_buf.h
+++ b/src/rdkafka_buf.h
@@ -49,21 +49,36 @@ typedef struct rd_tmpabuf_s {
         size_t of;
         char *buf;
         int failed;
-        int assert_on_fail;
+        rd_bool_t assert_on_fail;
 } rd_tmpabuf_t;
 
 /**
- * @brief Allocate new tmpabuf with \p size bytes pre-allocated.
+ * @brief Initialize new tmpabuf of non-final \p size bytes.
  */
 static RD_UNUSED void
-rd_tmpabuf_new(rd_tmpabuf_t *tab, size_t size, int assert_on_fail) {
-        tab->buf            = rd_malloc(size);
-        tab->size           = size;
+rd_tmpabuf_new(rd_tmpabuf_t *tab, size_t size, rd_bool_t assert_on_fail) {
+        tab->buf            = NULL;
+        tab->size           = RD_ROUNDUP(size, 8);
         tab->of             = 0;
         tab->failed         = 0;
         tab->assert_on_fail = assert_on_fail;
 }
 
+/**
+ * @brief Add a new allocation of \p _size bytes,
+ *        rounded up to maximum word size,
+ *        for \p _times times.
+ */
+#define rd_tmpabuf_add_alloc_times(_tab, _size, _times)                        \
+        (_tab)->size += RD_ROUNDUP(_size, 8) * _times
+
+#define rd_tmpabuf_add_alloc(_tab, _size)                                      \
+        rd_tmpabuf_add_alloc_times(_tab, _size, 1)
+/**
+ * @brief Finalize tmpabuf pre-allocating tab->size bytes.
+ */
+#define rd_tmpabuf_finalize(_tab) (_tab)->buf = rd_malloc((_tab)->size)
+
 /**
  * @brief Free memory allocated by tmpabuf
  */
diff --git a/src/rdkafka_metadata.c b/src/rdkafka_metadata.c
index f96edf658..6c2f60ae3 100644
--- a/src/rdkafka_metadata.c
+++ b/src/rdkafka_metadata.c
@@ -164,7 +164,8 @@ static rd_kafka_metadata_internal_t *rd_kafka_metadata_copy_internal(
          * Because of this we copy all the structs verbatim but
          * any pointer fields needs to be copied explicitly to update
          * the pointer address. */
-        rd_tmpabuf_new(&tbuf, size, 1 /*assert on fail*/);
+        rd_tmpabuf_new(&tbuf, size, rd_true /*assert on fail*/);
+        rd_tmpabuf_finalize(&tbuf);
         mdi = rd_tmpabuf_write(&tbuf, src, sizeof(*mdi));
         md  = &mdi->metadata;
 
@@ -506,11 +507,13 @@ rd_kafka_parse_Metadata(rd_kafka_broker_t *rkb,
          * no more than 4 times larger than the wire representation.
          * This is increased to 5 times in case if we want to compute partition
          * to rack mapping. */
-        rd_tmpabuf_new(&tbuf,
-                       sizeof(*mdi) + rkb_namelen +
-                           (rkbuf->rkbuf_totlen * 4 +
-                            (compute_racks ? rkbuf->rkbuf_totlen : 0)),
-                       0 /*dont assert on fail*/);
+        rd_tmpabuf_new(&tbuf, 0, rd_false /*dont assert on fail*/);
+        rd_tmpabuf_add_alloc(&tbuf, sizeof(*mdi));
+        rd_tmpabuf_add_alloc(&tbuf, rkb_namelen);
+        rd_tmpabuf_add_alloc(&tbuf, rkbuf->rkbuf_totlen *
+                                        (4 + (compute_racks ? 1 : 0)));
+
+        rd_tmpabuf_finalize(&tbuf);
 
         if (!(mdi = rd_tmpabuf_alloc(&tbuf, sizeof(*mdi)))) {
                 rd_kafka_broker_unlock(rkb);
@@ -1603,35 +1606,37 @@ rd_kafka_metadata_new_topic_mock(const rd_kafka_metadata_topic_t *topics,
         rd_kafka_metadata_internal_t *mdi;
         rd_kafka_metadata_t *md;
         rd_tmpabuf_t tbuf;
-        size_t topic_names_size = 0;
-        int total_partition_cnt = 0;
         size_t i;
         int curr_broker = 0;
 
-        /* Calculate total partition count and topic names size before
-         * allocating memory. */
-        for (i = 0; i < topic_cnt; i++) {
-                topic_names_size += 1 + strlen(topics[i].topic);
-                total_partition_cnt += topics[i].partition_cnt;
-        }
-
         /* If the replication factor is given, num_brokers must also be given */
         rd_assert(replication_factor <= 0 || num_brokers > 0);
 
         /* Allocate contiguous buffer which will back all the memory
          * needed by the final metadata_t object */
-        rd_tmpabuf_new(
-            &tbuf,
-            sizeof(*mdi) + (sizeof(*md->topics) * topic_cnt) +
-                topic_names_size + (64 /*topic name size..*/ * topic_cnt) +
-                (sizeof(*md->topics[0].partitions) * total_partition_cnt) +
-                (sizeof(*mdi->topics) * topic_cnt) +
-                (sizeof(*mdi->topics[0].partitions) * total_partition_cnt) +
-                (sizeof(*mdi->brokers) * RD_ROUNDUP(num_brokers, 8)) +
-                (replication_factor > 0 ? RD_ROUNDUP(replication_factor, 8) *
-                                              total_partition_cnt * sizeof(int)
-                                        : 0),
-            1 /*assert on fail*/);
+        rd_tmpabuf_new(&tbuf, sizeof(*mdi), rd_true /*assert on fail*/);
+
+        rd_tmpabuf_add_alloc(&tbuf, topic_cnt * sizeof(*md->topics));
+        rd_tmpabuf_add_alloc(&tbuf, topic_cnt * sizeof(*mdi->topics));
+        rd_tmpabuf_add_alloc(&tbuf, num_brokers * sizeof(*md->brokers));
+
+        /* Calculate total partition count and topic names size before
+         * allocating memory. */
+        for (i = 0; i < topic_cnt; i++) {
+                rd_tmpabuf_add_alloc(&tbuf, 1 + strlen(topics[i].topic));
+                rd_tmpabuf_add_alloc(&tbuf,
+                                     topics[i].partition_cnt *
+                                         sizeof(*md->topics[i].partitions));
+                rd_tmpabuf_add_alloc(&tbuf,
+                                     topics[i].partition_cnt *
+                                         sizeof(*mdi->topics[i].partitions));
+                if (replication_factor > 0)
+                        rd_tmpabuf_add_alloc_times(
+                            &tbuf, replication_factor * sizeof(int),
+                            topics[i].partition_cnt);
+        }
+
+        rd_tmpabuf_finalize(&tbuf);
 
         mdi = rd_tmpabuf_alloc(&tbuf, sizeof(*mdi));
         memset(mdi, 0, sizeof(*mdi));
diff --git a/src/rdkafka_metadata_cache.c b/src/rdkafka_metadata_cache.c
index 18f19a4d0..1530e699e 100644
--- a/src/rdkafka_metadata_cache.c
+++ b/src/rdkafka_metadata_cache.c
@@ -249,8 +249,6 @@ static struct rd_kafka_metadata_cache_entry *rd_kafka_metadata_cache_insert(
     rd_kafka_metadata_broker_internal_t *brokers_internal,
     size_t broker_cnt) {
         struct rd_kafka_metadata_cache_entry *rkmce, *old;
-        size_t topic_len;
-        size_t racks_size = 0;
         rd_tmpabuf_t tbuf;
         int i;
 
@@ -261,34 +259,32 @@ static struct rd_kafka_metadata_cache_entry *rd_kafka_metadata_cache_insert(
          * any pointer fields needs to be copied explicitly to update
          * the pointer address.
          * See also rd_kafka_metadata_cache_delete which frees this. */
-        topic_len = strlen(mtopic->topic) + 1;
+        rd_tmpabuf_new(&tbuf, 0, rd_true /*assert on fail*/);
+
+        rd_tmpabuf_add_alloc(&tbuf, sizeof(*rkmce));
+        rd_tmpabuf_add_alloc(&tbuf, strlen(mtopic->topic) + 1);
+        rd_tmpabuf_add_alloc(&tbuf, mtopic->partition_cnt *
+                                        sizeof(*mtopic->partitions));
+        rd_tmpabuf_add_alloc(&tbuf,
+                             mtopic->partition_cnt *
+                                 sizeof(*metadata_internal_topic->partitions));
 
         for (i = 0; include_racks && i < mtopic->partition_cnt; i++) {
                 size_t j;
-                racks_size += RD_ROUNDUP(
-                    metadata_internal_topic->partitions[i].racks_cnt *
-                        sizeof(char *),
-                    8);
+                rd_tmpabuf_add_alloc(
+                    &tbuf, metadata_internal_topic->partitions[i].racks_cnt *
+                               sizeof(char *));
                 for (j = 0;
                      j < metadata_internal_topic->partitions[i].racks_cnt;
                      j++) {
-                        racks_size += RD_ROUNDUP(
-                            strlen(metadata_internal_topic->partitions[i]
-                                       .racks[j]) +
-                                1,
-                            8);
+                        rd_tmpabuf_add_alloc(
+                            &tbuf, strlen(metadata_internal_topic->partitions[i]
+                                              .racks[j]) +
+                                       1);
                 }
         }
 
-        rd_tmpabuf_new(
-            &tbuf,
-            RD_ROUNDUP(sizeof(*rkmce), 8) + RD_ROUNDUP(topic_len, 8) +
-                (mtopic->partition_cnt *
-                 RD_ROUNDUP(sizeof(*mtopic->partitions), 8)) +
-                (mtopic->partition_cnt *
-                 RD_ROUNDUP(sizeof(*metadata_internal_topic->partitions), 8)) +
-                racks_size,
-            1 /*assert on fail*/);
+        rd_tmpabuf_finalize(&tbuf);
 
         rkmce = rd_tmpabuf_alloc(&tbuf, sizeof(*rkmce));
 
diff --git a/src/rdkafka_topic.c b/src/rdkafka_topic.c
index 3b3986d43..b63a0bbea 100644
--- a/src/rdkafka_topic.c
+++ b/src/rdkafka_topic.c
@@ -1831,38 +1831,44 @@ rd_kafka_topic_info_t *rd_kafka_topic_info_new_with_rack(
     const rd_kafka_metadata_partition_internal_t *mdpi) {
         rd_kafka_topic_info_t *ti;
         rd_tmpabuf_t tbuf;
-        size_t tlen             = RD_ROUNDUP(strlen(topic) + 1, 8);
-        size_t total_racks_size = 0;
         int i;
+        rd_bool_t has_racks = rd_false;
 
+        rd_tmpabuf_new(&tbuf, 0, rd_true /* assert on fail */);
+
+        rd_tmpabuf_add_alloc(&tbuf, sizeof(*ti));
+        rd_tmpabuf_add_alloc(&tbuf, strlen(topic) + 1);
         for (i = 0; i < partition_cnt; i++) {
                 size_t j;
                 if (!mdpi[i].racks)
                         continue;
 
+                if (unlikely(!has_racks))
+                        has_racks = rd_true;
+
                 for (j = 0; j < mdpi[i].racks_cnt; j++) {
-                        total_racks_size +=
-                            RD_ROUNDUP(strlen(mdpi[i].racks[j]) + 1, 8);
+                        rd_tmpabuf_add_alloc(&tbuf,
+                                             strlen(mdpi[i].racks[j]) + 1);
                 }
-                total_racks_size +=
-                    RD_ROUNDUP(sizeof(char *) * mdpi[i].racks_cnt, 8);
+                rd_tmpabuf_add_alloc(&tbuf, sizeof(char *) * mdpi[i].racks_cnt);
+        }
+
+        /* Only bother allocating this if at least one
+         * rack is there. */
+        if (has_racks) {
+                rd_tmpabuf_add_alloc(
+                    &tbuf, sizeof(rd_kafka_metadata_partition_internal_t) *
+                               partition_cnt);
         }
 
-        if (total_racks_size) /* Only bother allocating this if at least one
-                                 rack is there. */
-                total_racks_size +=
-                    RD_ROUNDUP(sizeof(rd_kafka_metadata_partition_internal_t) *
-                                   partition_cnt,
-                               8);
+        rd_tmpabuf_finalize(&tbuf);
 
-        rd_tmpabuf_new(&tbuf, sizeof(*ti) + tlen + total_racks_size,
-                       1 /* assert on fail */);
         ti                      = rd_tmpabuf_alloc(&tbuf, sizeof(*ti));
         ti->topic               = rd_tmpabuf_write_str(&tbuf, topic);
         ti->partition_cnt       = partition_cnt;
         ti->partitions_internal = NULL;
 
-        if (total_racks_size) {
+        if (has_racks) {
                 ti->partitions_internal = rd_tmpabuf_alloc(
                     &tbuf, sizeof(*ti->partitions_internal) * partition_cnt);