-
Notifications
You must be signed in to change notification settings - Fork 0
/
kafkatee.c
352 lines (299 loc) · 8.59 KB
/
kafkatee.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
/*
* Copyright (c) 2014 Wikimedia Foundation
* Copyright (c) 2014 Magnus Edenhill <[email protected]>
*
* 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 "kafkatee.h"
#include <signal.h>
#include <string.h>
#include <getopt.h>
#include <errno.h>
#include <unistd.h>
#include "config.h"
#include "format.h"
#include "input.h"
#include "output.h"
#include "exec.h"
#include "ezd.h"
static void sighup (int sig) {
static int rotate_version = 0;
conf.rotate = ++rotate_version;
}
static void term (int sig) {
kt_log(LOG_NOTICE,
"Received signal %i: terminating", sig);
if (!conf.run) {
/* Force exit on second term signal */
kt_log(LOG_WARNING, "Forced termination");
exit(0);
}
atomic_set(&conf.run, 0);
}
/**
* Kafka error callback
*/
static void kafka_error_cb (rd_kafka_t *rk, int err,
const char *reason, void *opaque) {
kt_log(LOG_ERR,
"Kafka error (%i): %s", err, reason);
}
/**
* Kafka statistics callback.
*/
static int kafka_stats_cb (rd_kafka_t *rk, char *json, size_t json_len,
void *opaque) {
if (!conf.stats_fp)
return 0;
fprintf(conf.stats_fp, "{ \"kafka\": %s }\n", json);
/* flush stats_fp to make sure valid JSON data
(e.g. full lines with closing object brackets)
is written to disk */
if (fflush(conf.stats_fp)) {
kt_log(LOG_ERR,
"Failed to fflush log.statistics.file %s: %s",
conf.stats_file, strerror(errno));
}
return 0;
}
/**
* Output kafkatee specific stats to statsfile.
*/
static void stats_print (void) {
/* FIXME: Currently none */
}
static void stats_close (void) {
stats_print();
fclose(conf.stats_fp);
conf.stats_fp = NULL;
}
static int stats_open (void) {
/* Already open? close and then reopen */
if (conf.stats_fp)
stats_close();
if (!(conf.stats_fp = fopen(conf.stats_file, "a"))) {
kt_log(LOG_ERR,
"Failed to open statistics log file %s: %s",
conf.stats_file, strerror(errno));
return -1;
}
return 0;
}
static void usage (const char *argv0) {
fprintf(stderr,
"kafkatee version %s\n"
"Kafka consumer with multiple inputs and outputs\n"
"\n"
"Usage: %s [options]\n"
"\n"
"Options:\n"
" -c <path> Configuration file path (%s)\n"
" -p <path> Pid file path (%s)\n"
" -d Enable debugging\n"
" -D Do not daemonize\n"
" -e Exit on EOF:\n"
" Exit when all inputs have reached their\n"
" EOF and all output queues are empty.\n"
" -x Exit on input or output failure.\n"
" (exit code 2)\n"
"\n",
KAFKATEE_VERSION,
argv0,
KAFKATEE_CONF_PATH,
KAFKATEE_PID_FILE_PATH);
exit(1);
}
int main (int argc, char **argv) {
const char *conf_file_path = KAFKATEE_CONF_PATH;
char errstr[512];
char c;
int r;
static int our_rotate_version = 0;
/*
* Default configuration
*/
conf.pid_file_path = strdup(KAFKATEE_PID_FILE_PATH);
conf.run = 1;
conf.exit_code = 0;
conf.log_level = 6;
conf.daemonize = 1;
conf.stats_interval = 60;
conf.stats_file = strdup("/tmp/kafkatee.stats.json");
conf.input_buf_size = 1024 * 10;
conf.output_delimiter = strdup("\n");
conf.output_delimiter_len = strlen(conf.output_delimiter);
conf.output_queue_size = 100000;
/* Kafka main configuration */
conf.rk_conf = rd_kafka_conf_new();
rd_kafka_conf_set(conf.rk_conf, "client.id", "kafkatee", NULL, 0);
rd_kafka_conf_set_error_cb(conf.rk_conf, kafka_error_cb);
/* Kafka topic configuration template */
conf.rkt_conf = rd_kafka_topic_conf_new();
/* Parse command line arguments */
while ((c = getopt(argc, argv, "hc:p:dDex")) != -1) {
switch (c) {
case 'h':
usage(argv[0]);
break;
case 'c':
conf_file_path = optarg;
break;
case 'p':
conf.pid_file_path = strdup(optarg);
break;
case 'd':
conf.log_level = 7;
break;
case 'D':
conf.daemonize = 0;
break;
case 'e':
conf.flags |= CONF_F_EXIT_ON_EOF;
break;
case 'x':
conf.flags |= CONF_F_EXIT_ON_IO_TERM;
break;
default:
usage(argv[0]);
break;
}
}
openlog("kafkatee", LOG_PID|LOG_PERROR, LOG_DAEMON);
/* Read config file */
if (ezd_conf_file_read(conf_file_path, conf_set,
errstr, sizeof(errstr), NULL) == -1) {
kt_log(LOG_ERR, "%s", errstr);
exit(1);
}
/* Daemonize if desired */
if (conf.daemonize) {
if (ezd_daemon(10, errstr, sizeof(errstr)) == -1) {
kt_log(LOG_ERR, "%s", errstr);
exit(1);
}
if (ezd_pidfile_open(conf.pid_file_path,
errstr, sizeof(errstr)) == -1) {
kt_log(LOG_ERR, "%s", errstr);
exit(1);
}
}
/* Parse the format string */
if (conf.fconf.format) {
if (conf.fconf.encoding != ENC_STRING) {
kt_log(LOG_ERR, "Output formatting only supported for "
"output.encoding = string");
if (conf.daemonize)
ezd_pidfile_close();
exit(1);
}
if (format_parse(&conf.fconf, conf.fconf.format,
errstr, sizeof(errstr)) == -1) {
kt_log(LOG_ERR,
"Failed to parse format string: %s\n%s",
conf.fconf.format, errstr);
if (conf.daemonize)
ezd_pidfile_close();
exit(1);
}
}
/* Set up statistics gathering in librdkafka, if enabled. */
if (conf.stats_interval) {
char tmp[30];
if (stats_open() == -1) {
kt_log(LOG_ERR,
"Failed to open statistics log file %s: %s",
conf.stats_file, strerror(errno));
if (conf.daemonize)
ezd_pidfile_close();
exit(1);
}
snprintf(tmp, sizeof(tmp), "%i", conf.stats_interval*1000);
rd_kafka_conf_set_stats_cb(conf.rk_conf, kafka_stats_cb);
rd_kafka_conf_set(conf.rk_conf, "statistics.interval.ms", tmp,
NULL, 0);
}
/* Create Kafka handle */
if (!(conf.rk = rd_kafka_new(RD_KAFKA_CONSUMER, conf.rk_conf,
errstr, sizeof(errstr)))) {
kt_log(LOG_ERR,
"Failed to create kafka handle: %s", errstr);
if (conf.daemonize)
ezd_pidfile_close();
exit(1);
}
rd_kafka_set_log_level(conf.rk, conf.log_level);
/* Initialize subsystems */
exec_init();
/* Finalize daemonization */
if (conf.daemonize)
ezd_daemon_started();
/* Run init command, if any. */
if (conf.cmd_init) {
if ((r = system(conf.cmd_init) != 0))
kt_log(LOG_ERR,
"\"command.init\" execution of \"%s\" failed "
"with exit code %i", conf.cmd_init, r);
}
/* Block all signals in the main thread so new threads get the same
* procmask. */
ezd_thread_sigmask(SIG_BLOCK, 0/*ALL*/, -1/*end*/);
/* Start IO */
outputs_start();
inputs_start();
/* Set main thread sigmask */
ezd_thread_sigmask(SIG_UNBLOCK, SIGHUP, SIGINT, SIGTERM, -1);
signal(SIGHUP, sighup);
signal(SIGINT, term);
signal(SIGTERM, term);
kt_log(LOG_INFO, "kafkatee starting");
/* Main loop */
while (conf.run) {
rd_kafka_poll(conf.rk, 1000);
if (unlikely(conf.rotate != our_rotate_version)) {
our_rotate_version = conf.rotate;
if (conf.stats_interval)
stats_open();
}
}
inputs_term();
outputs_term();
exec_term();
rd_kafka_destroy(conf.rk);
rd_kafka_wait_destroyed(5000);
/* if stats_fp is set (i.e. open), close it. */
if (conf.stats_fp)
stats_close();
free(conf.stats_file);
/* Run termination command, if any. */
if (conf.cmd_term) {
if ((r = system(conf.cmd_term) != 0))
kt_log(LOG_ERR,
"\"command.term\" execution of \"%s\" failed "
"with exit code %i", conf.cmd_term, r);
}
if (conf.daemonize)
ezd_pidfile_close();
kt_log(LOG_INFO, "kafkatee exiting");
exit(conf.exit_code);
}