forked from confluentinc/librdkafka
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathrdkafka_offset.c
441 lines (366 loc) · 12.5 KB
/
rdkafka_offset.c
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
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
/*
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2012,2013 Magnus Edenhill
* 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.
*/
/**
* This file implements the consumer offset storage.
* It currently only supports local file storage, not zookeeper.
*
* Regardless of commit method (file, zookeeper, ..) this is how it works:
* - When rdkafka, or the application, depending on if auto.offset.commit
* is enabled or not, calls rd_kafka_offset_store() with an offset to store,
* all it does is set rktp->rktp_stored_offset to this value.
* This can happen from any thread and is locked by the rktp lock.
* - The actual commit/write of the offset to its backing store (filesystem)
* is performed by the main rdkafka thread and scheduled at the configured
* auto.commit.interval.ms interval.
* - The write is performed in the main rdkafka thread in a blocking manner
* and once the write has succeeded rktp->rktp_commited_offset is updated
* to the new value.
* - If offset.store.sync.interval.ms is configured the main rdkafka thread
* will also make sure to fsync() each offset file accordingly.
*/
#include <stdio.h>
#include <sys/types.h>
#include <sys/stat.h>
#include <fcntl.h>
#include "rdkafka.h"
#include "rdkafka_int.h"
#include "rdkafka_offset.h"
#include "rdkafka_topic.h"
/**
* NOTE: toppar_lock(rktp) must be held
*/
static void rd_kafka_offset_file_close (rd_kafka_toppar_t *rktp) {
if (rktp->rktp_offset_fd == -1)
return;
close(rktp->rktp_offset_fd);
rktp->rktp_offset_fd = -1;
}
/**
* NOTE: toppar_lock(rktp) must be held
*/
static int rd_kafka_offset_file_open (rd_kafka_toppar_t *rktp) {
int fd;
if ((fd = open(rktp->rktp_offset_path, O_CREAT|O_RDWR, 0644)) == -1) {
rd_kafka_op_err(rktp->rktp_rkt->rkt_rk,
RD_KAFKA_RESP_ERR__FS,
"%s [%"PRId32"]: "
"Failed to open offset file %s: %s",
rktp->rktp_rkt->rkt_topic->str,
rktp->rktp_partition,
rktp->rktp_offset_path, strerror(errno));
return -1;
}
rktp->rktp_offset_fd = fd;
return 0;
}
/**
* NOTE: toppar_lock(rktp) must be held
*/
static int64_t rd_kafka_offset_file_read (rd_kafka_toppar_t *rktp) {
char buf[22];
char *end;
int64_t offset;
int r;
if (lseek(rktp->rktp_offset_fd, SEEK_SET, 0) == -1) {
rd_kafka_op_err(rktp->rktp_rkt->rkt_rk,
RD_KAFKA_RESP_ERR__FS,
"%s [%"PRId32"]: "
"Seek (for read) failed on offset file %s: %s",
rktp->rktp_rkt->rkt_topic->str,
rktp->rktp_partition,
rktp->rktp_offset_path,
strerror(errno));
rd_kafka_offset_file_close(rktp);
return -1;
}
if ((r = read(rktp->rktp_offset_fd, buf, sizeof(buf)-1)) == -1) {
rd_kafka_op_err(rktp->rktp_rkt->rkt_rk,
RD_KAFKA_RESP_ERR__FS,
"%s [%"PRId32"]: "
"Failed to read offset file %s: %s",
rktp->rktp_rkt->rkt_topic->str,
rktp->rktp_partition,
rktp->rktp_offset_path, strerror(errno));
rd_kafka_offset_file_close(rktp);
return -1;
}
if (r == 0) {
rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSET",
"%s [%"PRId32"]: offset file (%s) is empty",
rktp->rktp_rkt->rkt_topic->str,
rktp->rktp_partition,
rktp->rktp_offset_path);
return -1;
}
buf[r] = '\0';
offset = strtoull(buf, &end, 10);
if (buf == end) {
rd_kafka_op_err(rktp->rktp_rkt->rkt_rk,
RD_KAFKA_RESP_ERR__FS,
"%s [%"PRId32"]: "
"Unable to parse offset in %s",
rktp->rktp_rkt->rkt_topic->str,
rktp->rktp_partition,
rktp->rktp_offset_path);
return -1;
}
rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSET",
"%s [%"PRId32"]: Read offset %"PRId64" from offset "
"file (%s)",
rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition,
offset, rktp->rktp_offset_path);
return offset;
}
/**
* NOTE: rktp lock is not required.
* Locality: rdkafka main thread
*/
static int rd_kafka_offset_file_commit (rd_kafka_toppar_t *rktp,
int64_t offset) {
rd_kafka_topic_t *rkt = rktp->rktp_rkt;
int attempt;
for (attempt = 0 ; attempt < 2 ; attempt++) {
char buf[22];
int len;
if (rktp->rktp_offset_fd == -1)
if (rd_kafka_offset_file_open(rktp) == -1)
continue;
if (lseek(rktp->rktp_offset_fd, 0, SEEK_SET) == -1) {
rd_kafka_op_err(rktp->rktp_rkt->rkt_rk,
RD_KAFKA_RESP_ERR__FS,
"%s [%"PRId32"]: "
"Seek failed on offset file %s: %s",
rktp->rktp_rkt->rkt_topic->str,
rktp->rktp_partition,
rktp->rktp_offset_path,
strerror(errno));
rd_kafka_offset_file_close(rktp);
continue;
}
len = snprintf(buf, sizeof(buf), "%"PRId64"\n", offset);
if (write(rktp->rktp_offset_fd, buf, len) == -1) {
rd_kafka_op_err(rktp->rktp_rkt->rkt_rk,
RD_KAFKA_RESP_ERR__FS,
"%s [%"PRId32"]: "
"Failed to write offset %"PRId64" to "
"offset file %s (fd %i): %s",
rktp->rktp_rkt->rkt_topic->str,
rktp->rktp_partition,
offset,
rktp->rktp_offset_path,
rktp->rktp_offset_fd,
strerror(errno));
rd_kafka_offset_file_close(rktp);
continue;
}
if (ftruncate(rktp->rktp_offset_fd, len) == -1)
; /* Ignore truncate failures */
rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSET",
"%s [%"PRId32"]: wrote offset %"PRId64" to "
"file %s",
rktp->rktp_rkt->rkt_topic->str,
rktp->rktp_partition, offset,
rktp->rktp_offset_path);
rktp->rktp_commited_offset = offset;
/* If sync interval is set to immediate we sync right away. */
if (rkt->rkt_conf.offset_store_sync_interval_ms == 0)
fsync(rktp->rktp_offset_fd);
return 0;
}
return -1;
}
/**
* Store offset.
* Typically called from application code.
*
* NOTE: No lucks must be held.
*/
rd_kafka_resp_err_t rd_kafka_offset_store (rd_kafka_topic_t *rkt,
int32_t partition, int64_t offset) {
rd_kafka_toppar_t *rktp;
/* Find toppar */
rd_kafka_topic_rdlock(rkt);
if (!(rktp = rd_kafka_toppar_get(rkt, partition, 0/*!ua_on_miss*/))) {
rd_kafka_topic_unlock(rkt);
return RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION;
}
rd_kafka_topic_unlock(rkt);
rd_kafka_offset_store0(rktp, offset, 1/*lock*/);
rd_kafka_toppar_destroy(rktp);
return RD_KAFKA_RESP_ERR_NO_ERROR;
}
/**
* Offset file commit timer callback.
*/
static void rd_kafka_offset_file_commit_tmr_cb (rd_kafka_t *rk, void *arg) {
rd_kafka_toppar_t *rktp = arg;
rd_kafka_toppar_lock(rktp);
rd_kafka_dbg(rk, TOPIC, "OFFSET",
"%s [%"PRId32"]: periodic commit: "
"stored offset %"PRId64" > commited offset %"PRId64" ?",
rktp->rktp_rkt->rkt_topic->str,
rktp->rktp_partition,
rktp->rktp_stored_offset, rktp->rktp_commited_offset);
if (rktp->rktp_stored_offset > rktp->rktp_commited_offset)
rd_kafka_offset_file_commit(rktp, rktp->rktp_stored_offset);
rd_kafka_toppar_unlock(rktp);
}
/**
* Offset file sync timer callback
*/
static void rd_kafka_offset_file_sync_tmr_cb (rd_kafka_t *rk, void *arg) {
rd_kafka_toppar_t *rktp = arg;
rd_kafka_toppar_lock(rktp);
if (rktp->rktp_offset_fd != -1) {
rd_kafka_dbg(rk, TOPIC, "SYNC",
"%s [%"PRId32"]: offset file sync",
rktp->rktp_rkt->rkt_topic->str,
rktp->rktp_partition);
fsync(rktp->rktp_offset_fd);
}
rd_kafka_toppar_unlock(rktp);
}
/**
* Decommissions the use of an offset file for a toppar.
* The file content will not be touched and the file will not be removed.
*
* NOTE: toppar_lock(rktp) must be held.
*/
static void rd_kafka_offset_file_term (rd_kafka_toppar_t *rktp) {
if (rktp->rktp_rkt->rkt_conf.offset_store_sync_interval_ms > 0)
rd_kafka_timer_stop(rktp->rktp_rkt->rkt_rk,
&rktp->rktp_offset_sync_tmr, 1/*lock*/);
rd_kafka_timer_stop(rktp->rktp_rkt->rkt_rk,
&rktp->rktp_offset_commit_tmr, 1/*lock*/);
if (rktp->rktp_stored_offset > rktp->rktp_commited_offset)
rd_kafka_offset_file_commit(rktp, rktp->rktp_stored_offset);
rd_kafka_offset_file_close(rktp);
free(rktp->rktp_offset_path);
rktp->rktp_offset_path = NULL;
}
/**
* Take action when the offset for a toppar becomes unusable.
* NOTE: toppar_lock(rktp) must be held
*/
void rd_kafka_offset_reset (rd_kafka_toppar_t *rktp, int64_t err_offset,
rd_kafka_resp_err_t err, const char *reason) {
int64_t offset = RD_KAFKA_OFFSET_ERROR;
rd_kafka_op_t *rko;
switch (rktp->rktp_rkt->rkt_conf.auto_offset_reset)
{
case RD_KAFKA_OFFSET_END:
case RD_KAFKA_OFFSET_BEGINNING:
offset = rktp->rktp_rkt->rkt_conf.auto_offset_reset;
rktp->rktp_query_offset = offset;
rktp->rktp_fetch_state = RD_KAFKA_TOPPAR_FETCH_OFFSET_QUERY;
break;
case RD_KAFKA_OFFSET_ERROR:
rko = rd_kafka_op_new(RD_KAFKA_OP_ERR);
rko->rko_err = err;
rko->rko_rkmessage.offset = err_offset;
rko->rko_rkmessage.rkt = rktp->rktp_rkt;
rko->rko_rkmessage.partition = rktp->rktp_partition;
rko->rko_payload = strdup(reason);
rko->rko_len = strlen(rko->rko_payload);
rko->rko_flags |= RD_KAFKA_OP_F_FREE;
rd_kafka_q_enq(&rktp->rktp_fetchq, rko);
rktp->rktp_fetch_state = RD_KAFKA_TOPPAR_FETCH_NONE;
break;
}
rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSET",
"%s [%"PRId32"]: offset reset (at offset %"PRId64") "
"to %"PRId64": %s: %s",
rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition,
err_offset, offset, reason, rd_kafka_err2str(err));
}
/**
* Prepare a toppar for using an offset file.
*
* NOTE: toppar_lock(rktp) must be held.
*/
static void rd_kafka_offset_file_init (rd_kafka_toppar_t *rktp) {
struct stat st;
char spath[4096];
const char *path = rktp->rktp_rkt->rkt_conf.offset_store_path;
int64_t offset = -1;
if (stat(path, &st) == 0 && S_ISDIR(st.st_mode)) {
snprintf(spath, sizeof(spath),
"%s%s%s-%"PRId32".offset",
path, path[strlen(path)-1] == '/' ? "" : "/",
rktp->rktp_rkt->rkt_topic->str,
rktp->rktp_partition);
path = spath;
}
rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSET",
"%s [%"PRId32"] using offset file %s",
rktp->rktp_rkt->rkt_topic->str,
rktp->rktp_partition,
path);
rktp->rktp_offset_path = strdup(path);
rd_kafka_timer_start(rktp->rktp_rkt->rkt_rk,
&rktp->rktp_offset_commit_tmr,
rktp->rktp_rkt->rkt_conf.auto_commit_interval_ms *
1000,
rd_kafka_offset_file_commit_tmr_cb, rktp);
if (rktp->rktp_rkt->rkt_conf.offset_store_sync_interval_ms > 0)
rd_kafka_timer_start(rktp->rktp_rkt->rkt_rk,
&rktp->rktp_offset_sync_tmr,
rktp->rktp_rkt->rkt_conf.
offset_store_sync_interval_ms * 1000,
rd_kafka_offset_file_sync_tmr_cb, rktp);
if (rd_kafka_offset_file_open(rktp) != -1) {
/* Read offset from offset file. */
offset = rd_kafka_offset_file_read(rktp);
}
if (offset != -1) {
/* Start fetching from offset */
rktp->rktp_commited_offset = offset;
rktp->rktp_next_offset = offset;
rktp->rktp_fetch_state = RD_KAFKA_TOPPAR_FETCH_ACTIVE;
} else {
/* Offset was not usable: perform offset reset logic */
rktp->rktp_commited_offset = 0;
rd_kafka_offset_reset(rktp, RD_KAFKA_OFFSET_ERROR,
RD_KAFKA_RESP_ERR__FS,
"non-readable offset file");
}
}
/**
* Terminates toppar's offset store.
* NOTE: toppar_lock(rktp) must be held.
*/
void rd_kafka_offset_store_term (rd_kafka_toppar_t *rktp) {
rd_kafka_offset_file_term(rktp);
}
/**
* Initialize toppar's offset store.
* NOTE: toppar_lock(rktp) must be held.
*/
void rd_kafka_offset_store_init (rd_kafka_toppar_t *rktp) {
rd_kafka_offset_file_init(rktp);
}