├── .dir-locals.el ├── .gitignore ├── Makefile ├── README.md ├── config.c ├── config.h ├── debian ├── changelog ├── compat ├── control ├── copyright ├── etc │ ├── kafkatee.conf │ └── kafkatee.d │ │ └── README ├── gbp.conf ├── kafkatee.dirs ├── kafkatee.install ├── kafkatee.postinst ├── kafkatee.postrm ├── kafkatee.service ├── rules └── source │ └── format ├── exec.c ├── exec.h ├── ezd.c ├── ezd.h ├── format.c ├── format.h ├── input.c ├── input.h ├── kafkatee.c ├── kafkatee.conf.example ├── kafkatee.h ├── output.c ├── output.h ├── queue.c ├── queue.h └── test ├── Makefile ├── README ├── base.conf ├── ingen.c ├── linecnt.c ├── local.conf.example ├── test-kafka.conf ├── test-multi_json.conf ├── test-multi_output.conf ├── test-performance.conf ├── test-single.conf └── testpipe.py /.dir-locals.el: -------------------------------------------------------------------------------- 1 | ( (c-mode . ((c-file-style . "linux"))) ) 2 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | kafkatee 2 | *~ 3 | .emacs* 4 | *.d 5 | *.o 6 | core 7 | *.pid 8 | *.offset -------------------------------------------------------------------------------- /Makefile: -------------------------------------------------------------------------------- 1 | 2 | NAME =kafkatee 3 | VER := `git describe --abbrev=6 --tags HEAD --always` 4 | CFLAGS +=-DKAFKATEE_VERSION="\"$(VER)\"" 5 | DESTDIR ?=/usr/local 6 | 7 | SRCS= kafkatee.c config.c queue.c input.c output.c exec.c format.c ezd.c 8 | 9 | OBJS= $(SRCS:.c=.o) 10 | DEPS= ${OBJS:%.o=%.d} 11 | 12 | LIBS= -lyajl -lrdkafka -lrt -lpthread -lz 13 | 14 | CFLAGS+=-O2 -Wall -Werror -Wfloat-equal -Wpointer-arith -g 15 | 16 | 17 | # Profiling 18 | #CFLAGS+=-O0 19 | #CFLAGS += -pg 20 | #LDFLAGS += -pg 21 | 22 | .PHONY: 23 | 24 | all: $(NAME) 25 | 26 | %.o: %.c 27 | $(CC) -MD -MP $(CFLAGS) -c $< 28 | 29 | $(NAME): $(OBJS) 30 | $(CC) $(LDFLAGS) $(OBJS) -o $(NAME) $(LIBS) 31 | 32 | install: 33 | if [ "$(DESTDIR)" != "/usr/local" ]; then \ 34 | DESTDIR="$(DESTDIR)/usr"; \ 35 | else \ 36 | DESTDIR="$(DESTDIR)" ; \ 37 | fi ; \ 38 | install -t $$DESTDIR/bin $(NAME) 39 | 40 | test: .PHONY 41 | make -C test 42 | 43 | clean: 44 | rm -f $(OBJS) $(DEPS) $(NAME) 45 | 46 | -include $(DEPS) 47 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | kafkatee - Apache Kafka consumer with multiple inputs and outputs 2 | ================================================================= 3 | 4 | Copyright (c) 2014 [Wikimedia Foundation](http://www.wikimedia.org) 5 | 6 | Copyright (c) 2014 [Magnus Edenhill](http://www.edenhill.se/) 7 | 8 | 9 | # Description 10 | 11 | kafkatee consumes messages from one or more Kafka topics and 12 | writes the messages to one or more outputs - either command pipes or files. 13 | 14 | It provides simple transformation from JSON to arbitrary string output 15 | controlled through configuration. 16 | 17 | Each output has a configurable sample rate. 18 | 19 | 20 | Features: 21 | * Supported input types: Kafka consumer or piped command. 22 | * Supported output types: Piped command or file. 23 | * Configuration file syntax is backwards compatible with Wikimedia's udp2log 24 | * Configurable output queue size 25 | * Memory frugal: message payload is shared by all output queues 26 | * Configured with configuration file 27 | * Operates as a daemon (daemonization, pidfile) 28 | * Closes/stops and reopens/restarts all inputs and outputs on SIGHUP. 29 | 30 | 31 | # Documentation 32 | 33 | See `kafkatee.conf.example` for an annotated configuration file explaining 34 | available configuration properties. 35 | 36 | 37 | # Build 38 | 39 | ## Dependencies 40 | 41 | * librdkafka-dev >= 0.8.3 42 | * libyajl-dev (yajl1 or yajl2) 43 | 44 | ## Compile 45 | 46 | make 47 | 48 | ## Install 49 | 50 | make install 51 | # or: 52 | DESTDIR=/alternate/buildroot make install 53 | 54 | 55 | ## Run tests 56 | 57 | make test 58 | 59 | 60 | 61 | -------------------------------------------------------------------------------- /config.c: -------------------------------------------------------------------------------- 1 | #define _ISOC99_SOURCE /* for strtoull() */ 2 | #define _GNU_SOURCE /* for strdupa() */ 3 | 4 | #include 5 | #include 6 | #include 7 | #include 8 | #include 9 | #include 10 | #include 11 | #include 12 | #include 13 | 14 | #include 15 | 16 | #include "kafkatee.h" 17 | #include "input.h" 18 | #include "output.h" 19 | #include "ezd.h" 20 | 21 | /** 22 | * kafkatee global configuration 23 | */ 24 | struct conf conf; 25 | 26 | 27 | 28 | 29 | /* Left trim string '*s' of white spaces and return the new position. 30 | * Does not modify the string. */ 31 | static char *ltrim (const char *s) { 32 | while (isspace(*s)) 33 | s++; 34 | 35 | return (char *)s; 36 | } 37 | 38 | 39 | 40 | /** 41 | * Parses the value as true or false. 42 | */ 43 | static int conf_tof (const char *val) { 44 | char *end; 45 | int i; 46 | 47 | i = strtoul(val, &end, 0); 48 | if (end > val) /* treat as integer value */ 49 | return !!i; 50 | 51 | if (!strcasecmp(val, "yes") || 52 | !strcasecmp(val, "true") || 53 | !strcasecmp(val, "on") || 54 | !*val /* empty value is true */) 55 | return 1; 56 | else 57 | return 0; 58 | } 59 | 60 | static encoding_t encoding_parse (const char *val) { 61 | if (!strcasecmp(val, "string")) 62 | return ENC_STRING; 63 | else if (!strcasecmp(val, "json")) 64 | return ENC_JSON; 65 | else 66 | return ENC_ERROR; 67 | } 68 | 69 | static char *str_unescape (char *str) { 70 | char *s = str; 71 | 72 | while (*s) { 73 | int esc = -1; 74 | if (!strncmp(s, "\\n", 2)) 75 | esc = '\n'; 76 | else if (!strncmp(s, "\\r", 2)) 77 | esc = '\r'; 78 | else if (!strncmp(s, "\\t", 2)) 79 | esc = '\t'; 80 | else if (!strncmp(s, "\\0", 2)) 81 | esc = '\0'; 82 | 83 | if (esc != -1) { 84 | *s = esc; 85 | memmove(s+1, s+2, strlen(s+2)+1); 86 | } 87 | 88 | s++; 89 | } 90 | 91 | return str; 92 | } 93 | 94 | /** 95 | * Parses a "k=v,k2=v2,.." string in 'str' (which is modified) 96 | * and returns 1 on match, else 0. 97 | * Key name and value are returned in 'namep' and 'valp'. 98 | * If no value if specified (and no '='), then value is set to "". 99 | * '*strp' is forwarded to the next token on return for a sub-sequent 100 | * call to kv_parse() (if return is 1). 101 | */ 102 | static int kv_parse (char **strp, const char **namep, const char **valp) { 103 | char *s; 104 | char *str = *strp; 105 | 106 | if (!str) 107 | return 0; 108 | 109 | while (isspace((int)*str)) 110 | str++; 111 | 112 | if (!*str) 113 | return 0; 114 | 115 | if (!(s = strchr(str, '='))) { 116 | *namep = str; 117 | *valp = " "; 118 | *strp = NULL; /* EOF */ 119 | return 1; 120 | } 121 | 122 | *s = '\0'; 123 | *namep = str; 124 | *valp = s+1; 125 | 126 | if ((s = strchr(*valp, ','))) { 127 | *s = '\0'; 128 | *strp = s+1; 129 | } else 130 | *strp = NULL; 131 | 132 | return 1; 133 | } 134 | 135 | 136 | /** 137 | * Parse input key-values. 138 | */ 139 | static int input_kvs_parse (char *str, encoding_t *encp, int *flagsp, 140 | char *errstr, size_t errstr_size) { 141 | 142 | const char *kv_n, *kv_v; 143 | 144 | while (kv_parse(&str, &kv_n, &kv_v)) { 145 | if (!strcmp(kv_n, "encoding")) { 146 | if ((*encp = encoding_parse(kv_v)) == ENC_ERROR) { 147 | snprintf(errstr, errstr_size, 148 | "Invalid encoding %s", kv_v); 149 | return -1; 150 | } 151 | } else if (!strcmp(kv_n, "stop.eof")) { 152 | if (conf_tof(kv_v)) 153 | *flagsp |= INPUT_F_STOP_EOF; 154 | else 155 | *flagsp &= ~INPUT_F_STOP_EOF; 156 | } else if (!strcmp(kv_n, "stop.error")) { 157 | if (conf_tof(kv_v)) 158 | *flagsp |= INPUT_F_STOP_ERROR; 159 | else 160 | *flagsp &= ~INPUT_F_STOP_ERROR; 161 | } else if (!strcmp(kv_n, "exit.on.exit")) { 162 | if (conf_tof(kv_v)) 163 | *flagsp |= INPUT_F_EXIT_ON_EXIT; 164 | else 165 | *flagsp &= ~INPUT_F_EXIT_ON_EXIT; 166 | } else { 167 | snprintf(errstr, errstr_size, 168 | "Unknown option: %s", kv_n); 169 | return -1; 170 | } 171 | } 172 | 173 | return 0; 174 | } 175 | 176 | 177 | /** 178 | * Set a single configuration property 'name' using value 'val'. 179 | * Returns 0 on success, and -1 on error in which case 'errstr' will 180 | * contain an error string. 181 | */ 182 | int conf_set (const char *name, const char *val, 183 | char *errstr, size_t errstr_size, 184 | void *opaque) { 185 | rd_kafka_conf_res_t res; 186 | 187 | /* Kafka configuration */ 188 | if (!strncmp(name, "kafka.", strlen("kafka."))) { 189 | name += strlen("kafka."); 190 | 191 | /* Kafka topic configuration. */ 192 | if (!strncmp(name, "topic.", strlen("topic."))) 193 | res = rd_kafka_topic_conf_set(conf.rkt_conf, 194 | name+strlen("topic."), 195 | val, 196 | errstr, errstr_size); 197 | else /* Kafka global configuration */ 198 | res = rd_kafka_conf_set(conf.rk_conf, name, 199 | val, errstr, errstr_size); 200 | 201 | if (res == RD_KAFKA_CONF_OK) 202 | return 0; 203 | else if (res != RD_KAFKA_CONF_UNKNOWN) 204 | return -1; 205 | 206 | /* Unknown configs: fall thru */ 207 | name -= strlen("kafka."); 208 | } 209 | 210 | /* Non "key=value" config properties */ 211 | if (!val) { 212 | struct iovec arg[8+1]; 213 | 214 | if (ezd_regmatch("^input +(\\[([^\\]+)\\] +)?pipe +(.+)$", 215 | name, arg, 3) == 3) { 216 | encoding_t enc = ENC_STRING; 217 | int flags = INPUT_F_DEFAULTS; 218 | 219 | /* Optional: [k=v,k2=v2,..] key-value pairs */ 220 | if (arg[1].iov_base) { 221 | if (input_kvs_parse(ezd_strndupa_iov(&arg[1]), 222 | &enc, &flags, 223 | errstr, errstr_size) == -1) 224 | return -1; 225 | } 226 | 227 | if (!input_add(INPUT_PIPE, enc, flags, 228 | ltrim(ezd_strndupa_iov(&arg[2])), 0, 0, 229 | errstr, errstr_size)) 230 | return -1; 231 | 232 | } else if (ezd_regmatch("^input +(\\[([^\\]+)\\] +)?kafka +" 233 | "topic +([^ ]+) +" 234 | "partition +([0-9]+)(-([0-9]+))?" 235 | "( +from " 236 | "+(beginning|end|stored|[0-9]+))" 237 | "$", 238 | name, arg, 8) >= 4) { 239 | int part_lo, part_hi; 240 | char *topic; 241 | int64_t offset = RD_KAFKA_OFFSET_STORED; 242 | encoding_t enc = ENC_STRING; 243 | int flags = INPUT_F_DEFAULTS; 244 | 245 | if (arg[1].iov_base) { 246 | if (input_kvs_parse(ezd_strndupa_iov(&arg[1]), 247 | &enc, &flags, 248 | errstr, errstr_size) == -1) 249 | return -1; 250 | } 251 | 252 | if (arg[7].iov_base) { 253 | if (!strncmp(arg[7].iov_base, "beginning", 9)) 254 | offset = RD_KAFKA_OFFSET_BEGINNING; 255 | else if (!strncmp(arg[7].iov_base, "end", 3)) 256 | offset = RD_KAFKA_OFFSET_END; 257 | else if (!strncmp(arg[7].iov_base, "stored", 6)) 258 | offset = RD_KAFKA_OFFSET_STORED; 259 | else 260 | offset = strtoull(ezd_strndupa_iov( 261 | &arg[7]), 262 | NULL, 10); 263 | } 264 | 265 | topic = ezd_strndupa_iov(&arg[2]); 266 | 267 | part_lo = atoi(ezd_strndupa_iov(&arg[3])); 268 | if (arg[5].iov_base) 269 | part_hi = atoi(ezd_strndupa_iov(&arg[5])); 270 | else 271 | part_hi = part_lo; 272 | 273 | for ( ; part_lo <= part_hi ; part_lo++) 274 | if (!input_add(INPUT_KAFKA, enc, flags, 275 | topic, part_lo, 276 | offset, errstr, errstr_size)) 277 | return -1; 278 | 279 | 280 | } else if (ezd_regmatch("^output +pipe +([0-9]+) +(.+)$", 281 | name, arg, 2) == 2) { 282 | output_add(OUTPUT_PIPE, atoi(ezd_strndupa_iov(&arg[0])), 283 | ezd_strndupa_iov(&arg[1])); 284 | 285 | } else if (ezd_regmatch("^output +file +([0-9]+) +(.+)$", 286 | name, arg, 2) == 2) { 287 | output_add(OUTPUT_FILE, atoi(ezd_strndupa_iov(&arg[0])), 288 | ezd_strndupa_iov(&arg[1])); 289 | 290 | } else { 291 | snprintf(errstr, errstr_size, 292 | "Unknown configuration directive \"%s\"", 293 | name); 294 | return -1; 295 | } 296 | 297 | return 0; 298 | } 299 | 300 | /* kafkatee configuration options */ 301 | if (!strcmp(name, "output.format")) { 302 | if (conf.fconf.format) 303 | free(conf.fconf.format); 304 | conf.fconf.format = strdup(val); 305 | } else if (!strcmp(name, "output.encoding")) { 306 | if ((conf.fconf.encoding = 307 | encoding_parse(val)) == ENC_ERROR) { 308 | snprintf(errstr, errstr_size, 309 | "Unknown %s value \"%s\"", name, val); 310 | return -1; 311 | } 312 | } else if (!strcmp(name, "output.delimiter")) { 313 | if (conf.output_delimiter) 314 | free(conf.output_delimiter); 315 | conf.output_delimiter = str_unescape(strdup(val)); 316 | conf.output_delimiter_len = strlen(conf.output_delimiter); 317 | } else if (!strcmp(name, "output.queue.size")) { 318 | conf.output_queue_size = atoi(val); 319 | } else if (!strcmp(name, "log.level")) 320 | conf.log_level = atoi(val); 321 | else if (!strcmp(name, "log.statistics.file")) { 322 | free(conf.stats_file); 323 | conf.stats_file = strdup(val); 324 | } else if (!strcmp(name, "log.statistics.interval")) 325 | conf.stats_interval = atoi(val); 326 | else if (!strcmp(name, "log.rate.max")) 327 | conf.log_rate = atoi(val); 328 | else if (!strcmp(name, "log.rate.period")) 329 | conf.log_rate_period = atoi(val); 330 | else if (!strcmp(name, "daemonize")) 331 | conf.daemonize = conf_tof(val); 332 | else if (!strcmp(name, "pid.file.path")) { 333 | free(conf.pid_file_path); 334 | conf.pid_file_path = strdup(val); 335 | } else if (!strcmp(name, "command.init")) { 336 | if (conf.cmd_init) 337 | free(conf.cmd_init); 338 | conf.cmd_init = strdup(val); 339 | } else if (!strcmp(name, "command.term")) { 340 | if (conf.cmd_term) 341 | free(conf.cmd_term); 342 | conf.cmd_term = strdup(val); 343 | } else if (!strncmp(name, "env.", strlen("env."))) { 344 | if (*val) 345 | setenv(name+strlen("env."), val, 1); 346 | else 347 | unsetenv(name+strlen("env.")); 348 | } else { 349 | snprintf(errstr, errstr_size, 350 | "Unknown configuration property \"%s\"\n", name); 351 | return -1; 352 | } 353 | 354 | 355 | return 0; 356 | } 357 | 358 | -------------------------------------------------------------------------------- /config.h: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2014 Wikimedia Foundation 3 | * Copyright (c) 2014 Magnus Edenhill 4 | * 5 | * All rights reserved. 6 | * 7 | * Redistribution and use in source and binary forms, with or without 8 | * modification, are permitted provided that the following conditions are met: 9 | * 10 | * 1. Redistributions of source code must retain the above copyright notice, 11 | * this list of conditions and the following disclaimer. 12 | * 2. Redistributions in binary form must reproduce the above copyright notice, 13 | * this list of conditions and the following disclaimer in the documentation 14 | * and/or other materials provided with the distribution. 15 | * 16 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" 17 | * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE 18 | * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE 19 | * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE 20 | * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR 21 | * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF 22 | * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS 23 | * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN 24 | * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) 25 | * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 26 | * POSSIBILITY OF SUCH DAMAGE. 27 | */ 28 | 29 | #pragma once 30 | 31 | int conf_set (const char *name, const char *val, 32 | char *errstr, size_t errstr_size, 33 | void *opaque); 34 | -------------------------------------------------------------------------------- /debian/changelog: -------------------------------------------------------------------------------- 1 | kafkatee (0.1.7-1) unstable; urgency=medium 2 | 3 | * Set Maintainer to ops, add myself to Uploaders. 4 | * Bump debhelper compatibility level to 11. 5 | * Bump Standards-Versions to 4.1.3. 6 | * Remove the kafkatee upstart script, Upstart is deprecated. 7 | * Don't create (or remove) /var/run/kafkatee anymore, not used by the 8 | package and wasn't set up properly anyway. 9 | * Drop versioned Build-Depends to pre-jessie versions. 10 | * Fix indentation of the extended description. 11 | 12 | -- Faidon Liambotis Tue, 16 Jan 2018 16:41:39 +0200 13 | 14 | kafkatee (0.1.6-1) unstable; urgency=medium 15 | 16 | * Remove logrotate and syslog configuration (will be handled by 17 | external config management like puppet). 18 | 19 | -- Luca Toscano Fri, 30 Jun 2017 12:13:00 +0200 20 | 21 | kafkatee (0.1.5-1) unstable; urgency=medium 22 | 23 | * Unblock signals in children processes, fixes cleanup of shell pipelines. 24 | 25 | -- Filippo Giunchedi Tue, 23 May 2017 12:13:00 +0200 26 | 27 | kafkatee (0.1.4-2) unstable; urgency=low 28 | 29 | * Add kafkatee.upstart for easier use with Ubuntu Trusty. 30 | 31 | -- Andrew Otto (WMF) Fri, 04 Sep 2015 18:39:10 +0000 32 | 33 | kafkatee (0.1.4-1) unstable; urgency=low 34 | 35 | * Remove kafkatee.service's syntactically incorrect and redundant 36 | ExecStopPost. 37 | 38 | * kafkatee will no longer write pid files when not daemonized. 39 | This works better with systemd. 40 | 41 | -- Faidon Liambotis Fri, 19 Jun 2015 17:25:42 +0300 42 | 43 | kafkatee (0.1.3-2) unstable; urgency=low 44 | 45 | * Fix for kafkatee.postinst in Jessie - use service command for restarting rsyslog. 46 | * kafkatee now uses systemd instead of upstart. 47 | 48 | -- Andrew Otto (WMF) Wed, 29 Apr 2015 17:00:00 +0000 49 | 50 | kafkatee (0.1.3-1) unstable; urgency=low 51 | 52 | * Assert lengths aren't negative 53 | 54 | -- Andrew Otto (WMF) Mon, 27 Apr 2015 17:00:00 +0000 55 | 56 | kafkatee (0.1.2-1) unstable; urgency=low 57 | 58 | * Fix for fflush of log.statistics.file 59 | 60 | -- Andrew Otto (WMF) Tue, 22 Apr 2014 17:50:09 +0000 61 | 62 | kafkatee (0.1.1-1) unstable; urgency=low 63 | 64 | * Changes to logging during kafkatee startup 65 | 66 | -- Andrew Otto (WMF) Thu, 20 Feb 2014 17:24:28 +0000 67 | 68 | kafkatee (0.1.0-1) unstable; urgency=low 69 | 70 | * Initial release 71 | 72 | -- Andrew Otto (WMF) Tue, 18 Feb 2014 20:24:14 +0000 73 | -------------------------------------------------------------------------------- /debian/compat: -------------------------------------------------------------------------------- 1 | 11 2 | -------------------------------------------------------------------------------- /debian/control: -------------------------------------------------------------------------------- 1 | Source: kafkatee 2 | Section: web 3 | Priority: optional 4 | Maintainer: Wikimedia Technical Operations 5 | Uploaders: Andrew Otto , Faidon Liambotis 6 | Build-Depends: debhelper (>= 11~), librdkafka-dev, libyajl-dev, zlib1g-dev 7 | Standards-Version: 4.1.3 8 | Vcs-Git: https://gerrit.wikimedia.org/r/analytics/kafkatee 9 | Vcs-Browser: https://phabricator.wikimedia.org/diffusion/ANKA/ 10 | 11 | Package: kafkatee 12 | Architecture: any 13 | Depends: ${shlibs:Depends}, ${misc:Depends}, adduser 14 | Description: Apache Kafka consumer with multiple inputs and outputs 15 | kafkatee consumes messages from one or more Kafka topics and writes 16 | the messages to one or more outputs - either command pipes or files. 17 | . 18 | It provides simple transformation from JSON to arbitrary string output 19 | controlled through configuration. Each output has a configurable sample 20 | rate. 21 | . 22 | Features: 23 | * Supported input types: Kafka consumer or piped command. 24 | * Supported output types: Piped command or file. 25 | * Configuration file syntax is backwards compatible with Wikimedia's udp2log 26 | * Configurable output queue size 27 | * Memory frugal: message payload is shared by all output queues 28 | * Configured with configuration file 29 | * Operates as a daemon (daemonization, pidfile) 30 | * Closes/stops and reopens/restarts all inputs and outputs on SIGHUP. 31 | -------------------------------------------------------------------------------- /debian/copyright: -------------------------------------------------------------------------------- 1 | Format: http://www.debian.org/doc/packaging-manuals/copyright-format/1.0/ 2 | Upstream-Name: kafkatee 3 | Source: https://github.com/wikimedia/analytics-kafkatee 4 | 5 | Files: * 6 | Copyright: 2014 Wikimedia Foundation, Inc. 7 | 2014 Magnus Edenhill 8 | License: BSD-2-clause 9 | 10 | Files: debian/* 11 | Copyright: 2014 Andrew Otto (WMF) 12 | 2014 Wikimedia Foundation, Inc. 13 | License: BSD-2-clause 14 | 15 | License: BSD-2-clause 16 | Redistribution and use in source and binary forms, with or without 17 | modification, are permitted provided that the following conditions are met: 18 | . 19 | 1. Redistributions of source code must retain the above copyright notice, 20 | this list of conditions and the following disclaimer. 21 | 2. Redistributions in binary form must reproduce the above copyright notice, 22 | this list of conditions and the following disclaimer in the documentation 23 | and/or other materials provided with the distribution. 24 | . 25 | THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" 26 | AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE 27 | IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE 28 | ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE 29 | LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR 30 | CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF 31 | SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS 32 | INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN 33 | CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) 34 | ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 35 | POSSIBILITY OF SUCH DAMAGE. 36 | 37 | 38 | -------------------------------------------------------------------------------- /debian/etc/kafkatee.conf: -------------------------------------------------------------------------------- 1 | ####################################################################### 2 | # # 3 | # kafkatee configuration file # 4 | # # 5 | # # 6 | ####################################################################### 7 | # # 8 | # Syntax: # 9 | # = # 10 | # input # 11 | # output # 12 | # # 13 | # Boolean property values: # 14 | # >0, "true", "yes", "on", "" - interpreted as true # 15 | # everything else - interpreted as false # 16 | # # 17 | # # 18 | # The configuration file consists of: # 19 | # - Configuration properties (key = value) to control various # 20 | # aspects of kafkatee. # 21 | # - Inputs # 22 | # - Outputs # 23 | # # 24 | ####################################################################### 25 | 26 | 27 | 28 | 29 | ####################################################################### 30 | # # 31 | # Misc configuration # 32 | # # 33 | ####################################################################### 34 | 35 | # Pid file location 36 | # Default: /var/run/kafkatee.pid 37 | pid.file.path = /var/run/kafkatee/kafkatee.pid 38 | 39 | # Daemonize (background) 40 | # Default: true 41 | daemonize = false 42 | 43 | # Logging output level 44 | # 1 = only emergencies .. 6 = info, 7 = debug 45 | # Default: 6 (info) 46 | #log.level = 7 47 | 48 | 49 | # 50 | # JSON Statistics 51 | # 52 | # Statistics is collected from kafkatee itself(*) as well as librdkafka 53 | # Each JSON object has a top level key of either 'kafkatee' or 54 | # 'kafka' to indicate which type of statistics the object contains. 55 | # Each line is a valid JSON object. 56 | # 57 | # *: kafkatee does not currently output any stats of its own, just from rdkafka. 58 | # 59 | 60 | # Statistics output interval 61 | # Defaults to 60 seconds, use 0 to disable. 62 | #log.statistics.interval = 60 63 | 64 | # Statistics output file 65 | # Defaults to /tmp/kafkatee.stats.json 66 | log.statistics.file = /var/cache/kafkatee/kafkatee.stats.json 67 | 68 | 69 | # Command to run on startup, before starting IO. 70 | # Default: none 71 | #command.init = ./my-script.sh 72 | 73 | # Command to run on termination after all IOs have been stopped. 74 | # Default: none 75 | #command.term = ./my-cleanup-script.sh 76 | 77 | # Set environment variable which will be available for all sub-sequent 78 | # command executions (command.*, input pipe, output pipe, ..) 79 | #setenv.NMSGS=12 80 | # clear: 81 | #setenv.LD_LIBRARY_PATH= 82 | 83 | 84 | 85 | 86 | ####################################################################### 87 | # # 88 | # Kafka configuration # 89 | # # 90 | # Kafka configuration properties are prefixed with "kafka." # 91 | # and topic properties are prefixed with "kafka.topic.". # 92 | # # 93 | # For the full range of Kafka handle and topic configuration # 94 | # properties, see: # 95 | # http://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md # 96 | # # 97 | # And the Apache Kafka configuration reference: # 98 | # http://kafka.apache.org/08/configuration.html # 99 | # # 100 | ####################################################################### 101 | 102 | # Initial list of kafka brokers 103 | # Default: none 104 | kafka.metadata.broker.list = localhost 105 | 106 | # Offset file directory. 107 | # Each topic + partition combination has its own offset file. 108 | # Default: current directory 109 | kafka.topic.offset.store.path = /var/cache/kafkatee/offsets 110 | 111 | # If the request offset was not found on broker, or there is no 112 | # initial offset known (no stored offset), then reset the offset according 113 | # to this configuration. 114 | # Values: smallest (oldest/beginning), largest (newest/end), error (fail) 115 | # Default: largest 116 | kafka.topic.auto.offset.reset = largest 117 | 118 | # Maximum message size. 119 | # Should be synchronized on all producers, brokers and consumers. 120 | # Default: 4000000 121 | #kafka.message.max.bytes = 10000000 122 | 123 | # Kafka debugging 124 | # Default: none 125 | #kafka.debug = msg,topic,broker 126 | 127 | 128 | 129 | 130 | ####################################################################### 131 | # # 132 | # Message transformation # 133 | # # 134 | # A message read from one of the inputs may be transformed before # 135 | # being enqueued on the output queues. # 136 | # # 137 | # Transformation requires that the input and output encoding differs, # 138 | # i.e., 'input [encoding=json] ..' and 'output.encoding=string' # 139 | # # 140 | # While the input encoding is configured per input, the output # 141 | # encoding is configured globally, all outputs will receive the same # 142 | # message. # 143 | # # 144 | # The currently supported transformation(s) are: # 145 | # JSON input -> string output: # 146 | # JSON data is formatted according to the output.format # 147 | # configuration. The %{field} syntax references the field in the # 148 | # original JSON object by the same name and outputs its value. # 149 | # # 150 | # If the input and output encoding matches then the message remains # 151 | # untouched. # 152 | # # 153 | # The output message delimiter (defaults to newline (\n)) is # 154 | # configurable (output.delimiter) and always appended to all output # 155 | # messages regardless of transformation. # 156 | # The input is always stripped of its delimiter (which is newline # 157 | # for pipe inputs). # 158 | # # 159 | ####################################################################### 160 | 161 | # Output encoding: string or json 162 | # Default: string 163 | #output.encoding = string 164 | 165 | 166 | 167 | 168 | ####################################################################### 169 | # Output formatting # 170 | # # 171 | # The format string is made up of %{..}-tokens and literals. # 172 | # # 173 | # Tokens: # 174 | # # 175 | # %{FIELD} # 176 | # Retrieves the value from the JSON object's field with the # 177 | # same name. # 178 | # # 179 | # %{FIELD?DEFAULT} # 180 | # 'DEFAULT' is the default string to use if no field was matched, # 181 | # the default default string is "-". # 182 | # # 183 | # Literals are copied verbatim to the output string. # 184 | # # 185 | # Example JSON: {"task":19, "name":"Mike"} # 186 | # Example format: Got task %{task} for user %{name?unknown} # 187 | # Example output: Got task 19 for user Mike # 188 | # # 189 | # Note: Multi-level JSON objects are flattened: # 190 | # JSON: {"a": {"b": 9}, "c": "hi"} # 191 | # Gives: { "b": 9, "c": "hi" } # 192 | # # 193 | ####################################################################### 194 | 195 | # Output format for JSON -> string transformation. 196 | # Default: none 197 | #output.format = %{hostname} %{sequence} %{dt} %{time_firstbyte} %{ip} %{handling}/%{http_status} %{bytes_sent} %{request_method} http://%{host}%{uri}%{query} - %{mime_type} %{referer} %{x_forwarded_for} %{user_agent} %{accept_language} %{x_analytics} 198 | 199 | # Output delimiter 200 | # The output message ends with this delimiter. 201 | # Supports \n, \r, \t, \0. 202 | # Default: newline (\n) 203 | #output.delimiter = ;END;\n 204 | 205 | # Maximum queue size for each output, in number of messages 206 | # Default: 100000 207 | #output.queue.size = 1000000 208 | 209 | 210 | 211 | 212 | ####################################################################### 213 | # # 214 | # Inputs # 215 | # # 216 | # The following types of inputs are supported: # 217 | # - Kafka consumer # 218 | # - Piped command # 219 | # # 220 | # Any number and mix of inputs can be configured. # 221 | # Each input may be configured with an optional list of # 222 | # input-specific configuration properties, called the key-values. # 223 | # # 224 | # Supported key-values: # 225 | # - encoding=string|json - message encoding from this input. # 226 | # Default: string # 227 | # # 228 | # - stop.eof=true|false - do not continue try reading from # 229 | # this input when EOF has been reached # 230 | # Default: false # 231 | # # 232 | # - stop.error=true|false - do not reopen/restart input on error. # 233 | # Default: false # 234 | # # 235 | # The key-values is CSV-separated and the list of key-values must be # 236 | # enveloped by brackets: [encoding=string,foo=bar] # 237 | # # 238 | ####################################################################### 239 | 240 | 241 | ####################################################################### 242 | # Kafka consumer input syntax: # 243 | # input [key-values] kafka topic partition from # 244 | # # 245 | # where: # 246 | # - [key-values] is an optional CSV-separated list of key-values. # 247 | # NOTE: the enveloping brackets are literal. # 248 | # - is the Kafka topic to consume from. # 249 | # - or - is the partition, or range of partitions, to # 250 | # consume from. # 251 | # - is the offset to start consuming from. # 252 | # supported values: beginning, end, stored, # 253 | # Where 'stored' means to use a local offset file to store and # 254 | # read the offset from, which allows a later run of kafkatee # 255 | # to pick up from where it left off. # 256 | # # 257 | ####################################################################### 258 | 259 | #input [encoding=json] kafka topic varnish partition 0-10 from stored 260 | #input [encoding=string] kafka topic test1 partition 0 from end 261 | 262 | 263 | ####################################################################### 264 | # Piped command input syntax: # 265 | # input [key-values] pipe # 266 | # # 267 | # where: # 268 | # - [key-values] is an optional CSV-separated list of key-values. # 269 | # NOTE: the enveloping brackets are literal. # 270 | # - is a command string that will be executed with: # 271 | # /bin/sh -c "", thus supporting pipes, etc. # 272 | # # 273 | ####################################################################### 274 | 275 | #input [encoding=string] pipe tail -f a.log | grep -v ^DEBUG: 276 | #input pipe wget -q -O- http://example.com/api/events 277 | 278 | 279 | 280 | 281 | ####################################################################### 282 | # # 283 | # Outputs # 284 | # # 285 | # The following types of outputs are supported: # 286 | # - Piped command # 287 | # - File # 288 | # # 289 | # Each output has its own queue where messages are enqueued prior to # 290 | # writing to the output, this queue is limited by output.queue.size. # 291 | # If the queue limit is reached no new messages are added to the # 292 | # queue (tail-drop). # 293 | # # 294 | # Outputs are configured with a sample rate, 1 means every message, # 295 | # 2 means every other message, 1000 means every 1000nd message, and # 296 | # so on. # 297 | # $ 298 | # If an output process terminates, or an output file fails writing, # 299 | # the output is closed and reopened/restarted. The messages in the # 300 | # output's queue remain in the queue while the output is unavailable. # 301 | # # 302 | ####################################################################### 303 | 304 | 305 | ####################################################################### 306 | # Piped command output syntax: # 307 | # output pipe # 308 | # # 309 | # where: # 310 | # - is the sample-rate: 1 for each message, 100 for # 311 | # every 100rd message, and so on. # 312 | # - is a command string that will be executed with: # 313 | # /bin/sh -c "", thus supporting pipes, etc. # 314 | # # 315 | # Output pipes are stopped and restarted if kafkatee receives a # 316 | # SIGHUP signal. # 317 | ####################################################################### 318 | 319 | #output pipe 1 grep ^Something >> somefile.log 320 | #output pipe 1000 do-some-stats.sh 321 | #output pipe 1 nc -u syslog.example.com 514 322 | 323 | 324 | ####################################################################### 325 | # File output syntax: # 326 | # output file # 327 | # # 328 | # where: # 329 | # - is the sample-rate: 1 for each message, 100 for # 330 | # every 100rd message, and so on. # 331 | # - is the file path to write. The file is appended. # 332 | # # 333 | # Output files are closed and reopened if kafkatee receives a # 334 | # SIGHUP signal, thus allowing log file rotation. # 335 | ####################################################################### 336 | 337 | #output file 100 /tmp/sampled-100.txt 338 | 339 | 340 | 341 | 342 | ####################################################################### 343 | # Include other config files using include clause. # 344 | # You may use file glob matching to include files, e.g. # 345 | # include /etc/kafkatee.d/*.conf # 346 | # # 347 | # These files will be include in alphabetic order and override # 348 | # any previously set configs. # 349 | ####################################################################### 350 | 351 | include /etc/kafkatee.d/*.conf 352 | -------------------------------------------------------------------------------- /debian/etc/kafkatee.d/README: -------------------------------------------------------------------------------- 1 | This directory should contain individual output and input configuration for 2 | kafkatee. Files named *.conf in this directory will be loaded by kafkatee 3 | in alphabetical order. 4 | -------------------------------------------------------------------------------- /debian/gbp.conf: -------------------------------------------------------------------------------- 1 | [git-buildpackage] 2 | upstream-tree=branch 3 | upstream-branch=master 4 | debian-branch=debian 5 | upstream-tag=%(version)s 6 | debian-tag=debian/%(version)s 7 | tarball-dir = ../tarballs 8 | export-dir = ../build-area 9 | -------------------------------------------------------------------------------- /debian/kafkatee.dirs: -------------------------------------------------------------------------------- 1 | etc/kafkatee.d 2 | var/cache/kafkatee 3 | var/cache/kafkatee/offsets 4 | -------------------------------------------------------------------------------- /debian/kafkatee.install: -------------------------------------------------------------------------------- 1 | kafkatee usr/bin 2 | debian/etc/kafkatee.conf etc 3 | debian/etc/kafkatee.d/README etc/kafkatee.d 4 | -------------------------------------------------------------------------------- /debian/kafkatee.postinst: -------------------------------------------------------------------------------- 1 | #!/bin/sh 2 | 3 | # postinst script for kafkatee 4 | # 5 | # see: dh_installdeb(1) 6 | 7 | set -e 8 | 9 | case "$1" in 10 | configure|reconfigure) 11 | if ! getent passwd kafkatee >/dev/null; then 12 | # Adding system user: kafkatee 13 | adduser \ 14 | --system \ 15 | --group \ 16 | --no-create-home \ 17 | --home /nonexistent \ 18 | --gecos "kafkatee" \ 19 | --shell /bin/false \ 20 | kafkatee >/dev/null 2>/dev/null || : 21 | fi 22 | 23 | # chown /var/cache/kafkatee for writing 24 | chown -R kafkatee:kafkatee /var/cache/kafkatee 25 | 26 | echo \ 27 | "NOTE: Default kafkatee config files have been installed at: 28 | /etc/kafkatee.conf 29 | /etc/kafkatee.d/*.conf 30 | You should edit these config files to suit your configuration. 31 | " 32 | ;; 33 | 34 | abort-upgrade|abort-remove|abort-deconfigure) 35 | ;; 36 | 37 | *) 38 | echo "postinst called with unknown argument \`$1'" >&2 39 | exit 1 40 | ;; 41 | esac 42 | 43 | # dh_installdeb will replace this with shell code automatically 44 | # generated by other debhelper scripts. 45 | 46 | #DEBHELPER# 47 | 48 | exit 0 49 | -------------------------------------------------------------------------------- /debian/kafkatee.postrm: -------------------------------------------------------------------------------- 1 | #!/bin/sh 2 | 3 | # postrm script for kafka 4 | # 5 | # see: dh_installdeb(1) 6 | 7 | set -e 8 | 9 | case "$1" in 10 | purge) 11 | rm -f /var/log/kafkatee.log* 12 | rm -rf /var/cache/kafkatee 13 | 14 | deluser --quiet --system kafkatee >/dev/null || true 15 | deluser --quiet --group --system --only-if-empty --remove-home kafkatee || true 16 | ;; 17 | remove|upgrade|failed-upgrade|abort-install|abort-upgrade|disappear) 18 | ;; 19 | *) 20 | echo "postrm called with unknown argument \`$1'" >&2 21 | exit 1 22 | ;; 23 | esac 24 | 25 | # dh_installdeb will replace this with shell code automatically 26 | # generated by other debhelper scripts. 27 | 28 | #DEBHELPER# 29 | 30 | exit 0 31 | -------------------------------------------------------------------------------- /debian/kafkatee.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=kafkatee 3 | After=local-fs.target 4 | After=network.target 5 | 6 | [Service] 7 | Type=simple 8 | User=kafkatee 9 | ExecStart=/usr/bin/kafkatee -D -c /etc/kafkatee.conf 10 | ExecReload=/bin/kill -HUP $MAINPID 11 | Restart=on-failure 12 | 13 | [Install] 14 | WantedBy=multi-user.target 15 | -------------------------------------------------------------------------------- /debian/rules: -------------------------------------------------------------------------------- 1 | #!/usr/bin/make -f 2 | 3 | # Uncomment this to turn on verbose mode. 4 | #export DH_VERBOSE=1 5 | 6 | %: 7 | dh $@ 8 | 9 | override_dh_auto_test: 10 | # don't run make tests, tests need Kafka set up to run 11 | 12 | override_dh_auto_install: 13 | # don't run "make install", use debian/*.install instead 14 | 15 | .PHONY: override_dh_auto_test 16 | .PHONY: override_dh_auto_install 17 | -------------------------------------------------------------------------------- /debian/source/format: -------------------------------------------------------------------------------- 1 | 3.0 (quilt) 2 | -------------------------------------------------------------------------------- /exec.c: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2014 Wikimedia Foundation 3 | * Copyright (c) 2014 Magnus Edenhill 4 | * 5 | * All rights reserved. 6 | * 7 | * Redistribution and use in source and binary forms, with or without 8 | * modification, are permitted provided that the following conditions are met: 9 | * 10 | * 1. Redistributions of source code must retain the above copyright notice, 11 | * this list of conditions and the following disclaimer. 12 | * 2. Redistributions in binary form must reproduce the above copyright notice, 13 | * this list of conditions and the following disclaimer in the documentation 14 | * and/or other materials provided with the distribution. 15 | * 16 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" 17 | * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE 18 | * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE 19 | * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE 20 | * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR 21 | * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF 22 | * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS 23 | * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN 24 | * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) 25 | * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 26 | * POSSIBILITY OF SUCH DAMAGE. 27 | */ 28 | 29 | #define _GNU_SOURCE /* for pipe2() */ 30 | 31 | #include "kafkatee.h" 32 | #include "exec.h" 33 | #include "ezd.h" 34 | 35 | #include 36 | #include 37 | #include 38 | #include 39 | #include 40 | #include 41 | #include 42 | 43 | 44 | 45 | /** 46 | * Keeps track of spawned child processes. 47 | */ 48 | struct child_proc { 49 | LIST_ENTRY(child_proc) link; 50 | pid_t pid; 51 | char *cmd; 52 | time_t t_start; 53 | int *statusp; 54 | }; 55 | 56 | static LIST_HEAD(, child_proc) child_procs; 57 | static pthread_mutex_t child_proc_lock = PTHREAD_MUTEX_INITIALIZER; 58 | static int child_proc_cnt = 0; /* atomic */ 59 | 60 | static int do_child_proc_reap = 0; /* atomic */ 61 | static pthread_t exec_thread; 62 | 63 | 64 | /** 65 | * Adds a child process to the list of known child processes. 66 | * NOTE: child_proc_lock must be held 67 | */ 68 | static void child_proc_add (const char *cmdstring, pid_t pid, int *statusp) { 69 | struct child_proc *cp; 70 | 71 | cp = calloc(1, sizeof(*cp)); 72 | cp->cmd = strdup(cmdstring); 73 | cp->pid = pid; 74 | cp->t_start = time(NULL); 75 | cp->statusp = statusp; 76 | *cp->statusp = -1; 77 | 78 | LIST_INSERT_HEAD(&child_procs, cp, link); 79 | atomic_add(&child_proc_cnt, 1); 80 | } 81 | 82 | 83 | /** 84 | * Removes a child proc from the list. 85 | * NOTE: child_proc_lock must be held 86 | */ 87 | static void child_proc_del (struct child_proc *cp) { 88 | LIST_REMOVE(cp, link); 89 | free(cp->cmd); 90 | free(cp); 91 | atomic_sub(&child_proc_cnt, 1); 92 | } 93 | 94 | /** 95 | * Finds a child proc based on its pid. 96 | * NOTE: child_proc_lock must be held 97 | */ 98 | static struct child_proc *child_proc_find (pid_t pid) { 99 | struct child_proc *cp; 100 | 101 | LIST_FOREACH(cp, &child_procs, link) 102 | if (cp->pid == pid) 103 | return cp; 104 | 105 | return NULL; 106 | } 107 | 108 | 109 | /** 110 | * Similar to popen() but uses file descriptors instead of streams and 111 | * returns the child process' pid in '*pidp'. 112 | * 'statusp' is a pointer to where the child's exit status will be stored 113 | * if the child terminates. The caller should set 'statusp' to -1 prior 114 | * to calling kt_open, -1 indicates that pipe is still running. 115 | * 116 | * Returns the file descriptor on success or -1 on error. 117 | */ 118 | int kt_popen (const char *cmdstring, const char *rw, 119 | pid_t *pidp, int *statusp, 120 | char *errstr, size_t errstr_size) { 121 | 122 | int pfd[2], i; 123 | pid_t pid; 124 | struct sigaction sig_action; 125 | 126 | /* Create pipe fds */ 127 | if (pipe2(pfd, O_CLOEXEC) == -1) { 128 | snprintf(errstr, errstr_size, 129 | "pipe failed: %s", strerror(errno)); 130 | return -1; 131 | } 132 | 133 | /* Lock child proc list prior to fork to avoid 134 | * race conditions from other threads should the child die instantly */ 135 | pthread_mutex_lock(&child_proc_lock); 136 | 137 | /* Fork new process */ 138 | if ((pid = fork()) == -1) { 139 | pthread_mutex_unlock(&child_proc_lock); 140 | snprintf(errstr, errstr_size, 141 | "fork failed: %s", strerror(errno)); 142 | close(pfd[0]); 143 | close(pfd[1]); 144 | return -1; 145 | } 146 | 147 | if (pid != 0) { 148 | /* Parent process */ 149 | *pidp = pid; 150 | 151 | /* Add child proc to list of known child procs */ 152 | child_proc_add(cmdstring, pid, statusp); 153 | pthread_mutex_unlock(&child_proc_lock); 154 | 155 | if (*rw == 'w') { 156 | close(pfd[0]); 157 | return pfd[1]; 158 | } else { 159 | close(pfd[1]); 160 | return pfd[0]; 161 | } 162 | } 163 | 164 | /* Child process */ 165 | 166 | pthread_mutex_unlock(&child_proc_lock); 167 | 168 | /* Move away from parent's process group to avoid signal propagation */ 169 | setpgid(0, getpid()); 170 | 171 | if (*rw == 'w') { /* write */ 172 | close(pfd[1]); 173 | if (pfd[0] != STDIN_FILENO) { 174 | dup2(pfd[0], STDIN_FILENO); 175 | close(pfd[0]); 176 | } 177 | 178 | } else { /* read */ 179 | close(pfd[0]); 180 | if (pfd[1] != STDOUT_FILENO) { 181 | dup2(pfd[1], STDOUT_FILENO); 182 | close(pfd[1]); 183 | } 184 | } 185 | 186 | 187 | /* Reset signal disposition and unblock signals for children */ 188 | memset(&sig_action, 0, sizeof(sig_action)); 189 | sig_action.sa_handler = SIG_DFL; 190 | sig_action.sa_flags = 0; 191 | sigemptyset(&sig_action.sa_mask); 192 | for (i = 1 ; i < NSIG ; i++) { 193 | sigaction(i, &sig_action, NULL); 194 | } 195 | 196 | ezd_thread_sigmask(SIG_UNBLOCK, 0, -1); 197 | 198 | /* Execute command */ 199 | execl("/bin/sh", "sh", "-c", cmdstring, NULL); 200 | _exit(127); 201 | } 202 | 203 | 204 | 205 | /** 206 | * Returns a human readable process exit reason based on the exit code. 207 | */ 208 | const char *exec_exitstatus (int status) { 209 | static __thread char ret[128]; 210 | 211 | if (WIFEXITED(status)) { 212 | if (WEXITSTATUS(status) == 127) 213 | snprintf(ret, sizeof(ret), 214 | "could not execute command"); 215 | else 216 | snprintf(ret, sizeof(ret), "exited with status %i", 217 | WEXITSTATUS(status)); 218 | 219 | } else if (WIFSIGNALED(status)) { 220 | #ifdef WCOREDUMP 221 | if (WCOREDUMP(status)) 222 | snprintf(ret, sizeof(ret), "core dumped"); 223 | else 224 | #endif 225 | snprintf(ret, sizeof(ret), "terminated by signal %i", 226 | WTERMSIG(status)); 227 | } else 228 | snprintf(ret, sizeof(ret), "exited with code %i", status); 229 | 230 | return ret; 231 | } 232 | 233 | 234 | /** 235 | * Called from non-signal context to reap any child processes that 236 | * might have died lately. 237 | */ 238 | static void child_proc_reap (void) { 239 | pid_t pid; 240 | int st; 241 | 242 | atomic_set(&do_child_proc_reap, 0); 243 | 244 | pthread_mutex_lock(&child_proc_lock); 245 | while ((pid = waitpid(-1, &st, WNOHANG)) > 0) { 246 | struct child_proc *cp; 247 | 248 | cp = child_proc_find(pid); 249 | 250 | if (likely(cp != NULL)) 251 | kt_log(LOG_INFO, 252 | "Child process \"%s\" (%i) %s: " 253 | "ran for %i seconds", 254 | cp->cmd, pid, exec_exitstatus(st), 255 | (int)(time(NULL) - cp->t_start)); 256 | else 257 | kt_log(LOG_WARNING, 258 | "Unknown child process (%i) %s", 259 | pid, exec_exitstatus(st)); 260 | if (!cp) 261 | continue; 262 | 263 | /* Store exit status for owner */ 264 | atomic_set(cp->statusp, st); 265 | child_proc_del(cp); 266 | } 267 | pthread_mutex_unlock(&child_proc_lock); 268 | 269 | } 270 | 271 | 272 | static void sigchld (int sig) { 273 | /* Simply indicate to non-signal context to reap the child proc */ 274 | atomic_set(&do_child_proc_reap, 1); 275 | } 276 | 277 | 278 | 279 | /** 280 | * Exec thread's main loop. 281 | * The exec thread is responsible for reaping spawned child processes. 282 | */ 283 | static void *exec_main (void *ignore) { 284 | time_t hard_timeout = 0; 285 | const int gracetime = 10; 286 | struct child_proc *cp; 287 | 288 | /* Install SIGCHLD handler to reap dying child processes */ 289 | ezd_thread_sigmask(SIG_BLOCK, 0/*ALL*/, -1/*end*/); 290 | ezd_thread_sigmask(SIG_UNBLOCK, SIGCHLD, -1/*end*/); 291 | signal(SIGCHLD, sigchld); 292 | 293 | while (conf.run || child_proc_cnt > 0) { 294 | if (!conf.run) { 295 | /* Allow for childs to terminate, then kill them. */ 296 | if (!hard_timeout) { 297 | hard_timeout = time(NULL) + gracetime; 298 | _DBG("Waiting %i seconds for " 299 | "%i child processes to terminate", 300 | gracetime, child_proc_cnt); 301 | } else if (hard_timeout < time(NULL)) 302 | break; 303 | } 304 | 305 | if (do_child_proc_reap) 306 | child_proc_reap(); 307 | usleep(500000); 308 | } 309 | 310 | /* Kill any remaining children */ 311 | pthread_mutex_lock(&child_proc_lock); 312 | LIST_FOREACH(cp, &child_procs, link) { 313 | if (cp->pid <= 0) 314 | continue; 315 | 316 | kt_log(LOG_WARNING, 317 | "Child process \"%s\" (%i) did not " 318 | "terminate in %is: killing", 319 | cp->cmd, (int)cp->pid, gracetime); 320 | kill(cp->pid, SIGTERM); 321 | sleep(1); 322 | kill(cp->pid, SIGKILL); 323 | } 324 | pthread_mutex_unlock(&child_proc_lock); 325 | 326 | child_proc_reap(); 327 | 328 | return NULL; 329 | } 330 | 331 | 332 | /** 333 | * Terminate child process exec layer 334 | */ 335 | void exec_term (void) { 336 | void *ignore; 337 | pthread_join(exec_thread, &ignore); 338 | } 339 | 340 | 341 | /** 342 | * Initialize child process exec layer 343 | */ 344 | void exec_init (void) { 345 | int err; 346 | 347 | if ((err = pthread_create(&exec_thread, NULL, exec_main, NULL))) { 348 | kt_log(LOG_ERR, 349 | "Failed to create exec_main thread: %s", strerror(err)); 350 | exit(1); 351 | } 352 | } 353 | -------------------------------------------------------------------------------- /exec.h: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2014 Wikimedia Foundation 3 | * Copyright (c) 2014 Magnus Edenhill 4 | * 5 | * All rights reserved. 6 | * 7 | * Redistribution and use in source and binary forms, with or without 8 | * modification, are permitted provided that the following conditions are met: 9 | * 10 | * 1. Redistributions of source code must retain the above copyright notice, 11 | * this list of conditions and the following disclaimer. 12 | * 2. Redistributions in binary form must reproduce the above copyright notice, 13 | * this list of conditions and the following disclaimer in the documentation 14 | * and/or other materials provided with the distribution. 15 | * 16 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" 17 | * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE 18 | * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE 19 | * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE 20 | * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR 21 | * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF 22 | * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS 23 | * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN 24 | * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) 25 | * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 26 | * POSSIBILITY OF SUCH DAMAGE. 27 | */ 28 | 29 | #pragma once 30 | 31 | int kt_popen (const char *cmdstring, const char *rwtype, 32 | pid_t *pidp, int *statusp, 33 | char *errstr, size_t errstr_size); 34 | 35 | const char *exec_exitstatus (int status); 36 | 37 | void exec_term (void); 38 | void exec_init (void); 39 | -------------------------------------------------------------------------------- /ezd.c: -------------------------------------------------------------------------------- 1 | /** 2 | * ezd - eazy daemon - makes your life as a daemon conceiver easy. 3 | * 4 | * Provides the most basic tools for daemonizing a program, such as: 5 | * - configuration file parsing and reading (name=value.. pairs) 6 | * - pidfile handling with locking 7 | * - daemon()ization with wait-for-child-to-fully-start support to 8 | * allow full initialization in the child process. 9 | * 10 | * Simply add ezd.c and ezd.h to your project and use as you like. 11 | */ 12 | 13 | /* 14 | * Copyright (c) 2013 Magnus Edenhill 15 | * 16 | * All rights reserved. 17 | * 18 | * Redistribution and use in source and binary forms, with or without 19 | * modification, are permitted provided that the following conditions are met: 20 | * 21 | * 1. Redistributions of source code must retain the above copyright notice, 22 | * this list of conditions and the following disclaimer. 23 | * 2. Redistributions in binary form must reproduce the above copyright notice, 24 | * this list of conditions and the following disclaimer in the documentation 25 | * and/or other materials provided with the distribution. 26 | * 27 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" 28 | * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE 29 | * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE 30 | * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE 31 | * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR 32 | * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF 33 | * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS 34 | * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN 35 | * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) 36 | * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 37 | * POSSIBILITY OF SUCH DAMAGE. 38 | */ 39 | 40 | 41 | #define _ISOC99_SOURCE /* for strtoull() */ 42 | #define _GNU_SOURCE /* for strdupa() */ 43 | 44 | #include 45 | #include 46 | #include 47 | #include 48 | #include 49 | #include 50 | #include 51 | #include 52 | #include 53 | #include 54 | #include 55 | #include 56 | #include 57 | #include 58 | #include 59 | #include 60 | #include 61 | #include 62 | 63 | static char ezd_pidfile_path[PATH_MAX]; 64 | static int ezd_pidfile_fd = -1; 65 | 66 | 67 | int ezd_str_tof (const char *val) { 68 | char *end; 69 | int i; 70 | 71 | i = strtoul(val, &end, 0); 72 | if (end > val) /* treat as integer value */ 73 | return !!i; 74 | 75 | if (!strcasecmp(val, "yes") || 76 | !strcasecmp(val, "true") || 77 | !strcasecmp(val, "on")) 78 | return 1; 79 | else 80 | return 0; 81 | } 82 | 83 | 84 | 85 | /* 86 | * Left and right trim string '*sp' of white spaces (incl newlines). 87 | */ 88 | static int ezd_trim (char **sp, char *end) { 89 | char *s = *sp; 90 | 91 | while (s < end && isspace(*s)) 92 | s++; 93 | 94 | end--; 95 | 96 | while (end > s && isspace(*end)) { 97 | *end = '\0'; 98 | end--; 99 | } 100 | 101 | *sp = s; 102 | 103 | return (int)(end - *sp); 104 | } 105 | 106 | 107 | int ezd_csv2array (char ***arrp, const char *valstr) { 108 | char *val = strdupa(valstr); 109 | int size = 0; 110 | int cnt = 0; 111 | 112 | *arrp = NULL; 113 | 114 | while (*val) { 115 | int len; 116 | char *t = NULL; 117 | char *s = val; 118 | 119 | do { 120 | if (!(t = strchr(s, ','))) { 121 | /* End of line */ 122 | t = val + strlen(val); 123 | } else if (t == val) 124 | break; 125 | else if (*(t-1) == '\\') { 126 | /* Escaped: remove the escaper and keep going */ 127 | memmove(t-1, t, strlen(t)+1); 128 | s = t; 129 | t = NULL; 130 | } else 131 | break; 132 | } while (!t); 133 | 134 | ezd_trim(&val, t); 135 | 136 | len = (int)(t-val); 137 | if (len > 0) { 138 | if (cnt == size) { 139 | size = (size + 4) * 2; 140 | *arrp = realloc(*arrp, sizeof(**arrp) * size); 141 | } 142 | 143 | (*arrp)[cnt++] = strndup(val, len); 144 | } else 145 | len++; 146 | 147 | val += len; 148 | } 149 | return cnt; 150 | } 151 | 152 | 153 | int ezd_regmatch (const char *regex, const char *str, 154 | struct iovec *match, int matchcnt) { 155 | regex_t preg; 156 | int i; 157 | regmatch_t pmatch[matchcnt+1]; 158 | 159 | memset(match, 0, sizeof(*match) * matchcnt); 160 | 161 | if (1) { 162 | int err; 163 | if ((err = regcomp(&preg, regex, REG_EXTENDED))) 164 | return -1; 165 | } 166 | 167 | if ((i = regexec(&preg, str, matchcnt+1, pmatch, 0)) == REG_NOMATCH) { 168 | regfree(&preg); 169 | return 0; 170 | } 171 | 172 | for (i = 1 ; i <= matchcnt ; i++) { 173 | if (pmatch[i].rm_so == -1) 174 | continue; 175 | 176 | match[i-1].iov_base = (char *)str + pmatch[i].rm_so; 177 | match[i-1].iov_len = pmatch[i].rm_eo - pmatch[i].rm_so; 178 | } 179 | 180 | regfree(&preg); 181 | 182 | return i-1; 183 | } 184 | 185 | 186 | 187 | 188 | 189 | int ezd_conf_file_read (const char *path, 190 | int (*conf_set_cb) (const char *name, 191 | const char *val, 192 | char *errstr, 193 | size_t errstr_size, 194 | void *opaque), 195 | char *errstr, size_t errstr_size, 196 | void *opaque) { 197 | FILE *fp; 198 | char buf[8192]; 199 | int line = 0; 200 | static int inc_depth = 0; 201 | glob_t glob_matches; 202 | int g = 0; /* iterator index in glob_matches.gl_pathv */ 203 | 204 | if (!(fp = fopen(path, "r"))) { 205 | snprintf(errstr, errstr_size, 206 | "Failed to open configuration file %s: %s", 207 | path, strerror(errno)); 208 | return -1; 209 | } 210 | 211 | if (inc_depth > 10) { 212 | snprintf(errstr, errstr_size, 213 | "Maximum include depth reached (%i)", inc_depth); 214 | return -1; 215 | } 216 | 217 | inc_depth++; 218 | 219 | while (fgets(buf, sizeof(buf), fp)) { 220 | char *s = buf; 221 | char *t; 222 | int errof; 223 | 224 | line++; 225 | 226 | while (isspace(*s)) 227 | s++; 228 | 229 | if (!*s || *s == '#') 230 | continue; 231 | 232 | /* Prepend error string with "file:line: " if enough room */ 233 | errof = snprintf(errstr, errstr_size, "%s:%i: ", 234 | path, line); 235 | if (errof + 50 > errstr_size) 236 | errof = 0; 237 | 238 | /* Supported formats: 239 | * key-value: "name[SPACES]=[SPACES]value[SPACES]": 240 | * whole-line: "anything..." 241 | */ 242 | 243 | /* "name=value" 244 | * find ^ */ 245 | if ((t = strchr(s, '='))) { 246 | char *t2; 247 | /* Check that '=' is following the first word, 248 | * if not this is a whole-line match. */ 249 | 250 | /* "name =value" 251 | * find ^ */ 252 | if ((t2 = strchr(s, ' '))) { 253 | while (*t2 == ' ') 254 | t2++; 255 | if (t2 != t) 256 | t = NULL; /* whole-line */ 257 | } 258 | } 259 | 260 | if (t) { 261 | /* trim "name"=.. */ 262 | if (!ezd_trim(&s, t)) { 263 | snprintf(errstr+errof, errstr_size-errof, 264 | "warning: empty left-hand-side\n"); 265 | continue; 266 | } 267 | 268 | /* terminate "name"=.. */ 269 | *t = '\0'; 270 | t++; 271 | 272 | /* ezd_trim ..="value" */ 273 | ezd_trim(&t, t + strlen(t)); 274 | 275 | if (!*t) 276 | t = NULL; /* empty value */ 277 | } else { 278 | ezd_trim(&s, s+strlen(s)); 279 | } 280 | 281 | 282 | /* Special config tokens handled by ezd: 283 | * include FILENAME 284 | */ 285 | if (!t && !strncmp(s, "include ", strlen("include "))) { 286 | t = s + strlen("include "); 287 | ezd_trim(&t, t + strlen(t)); 288 | if (!*t) { 289 | snprintf(errstr+errof, errstr_size-errof, 290 | "Syntax error, expected: " 291 | "include FILENAME"); 292 | inc_depth--; 293 | return -1; 294 | } 295 | 296 | /* Read in matching include files. */ 297 | if (glob(t, GLOB_ERR, NULL, &glob_matches) != 0) { 298 | snprintf(errstr+errof, errstr_size-errof, 299 | "Failed to read include file pattern '%s'", t); 300 | inc_depth--; 301 | return -1; 302 | } 303 | 304 | /* For each matched include file, call ezd_conf_file_read. */ 305 | for (g = 0; g < glob_matches.gl_pathc; g++) { 306 | if (ezd_conf_file_read(glob_matches.gl_pathv[g], conf_set_cb, 307 | errstr+errof, errstr_size-errof, 308 | opaque) == -1) { 309 | inc_depth--; 310 | globfree(&glob_matches); 311 | return -1; 312 | } 313 | } 314 | globfree(&glob_matches); 315 | 316 | continue; 317 | } 318 | 319 | /* set the configuration value. */ 320 | if (conf_set_cb(s, t, errstr+errof, errstr_size-errof, 321 | opaque) == -1) { 322 | fclose(fp); 323 | inc_depth--; 324 | return -1; 325 | } 326 | } 327 | 328 | inc_depth--; 329 | 330 | fclose(fp); 331 | return 0; 332 | } 333 | 334 | 335 | void ezd_pidfile_close (void) { 336 | if (!*ezd_pidfile_path) 337 | return; 338 | 339 | flock(ezd_pidfile_fd, LOCK_UN|LOCK_NB); 340 | unlink(ezd_pidfile_path); 341 | close(ezd_pidfile_fd); 342 | } 343 | 344 | 345 | 346 | int ezd_pidfile_open (const char *path, char *errstr, size_t errstr_size) { 347 | int fd; 348 | pid_t currpid = 0; 349 | char buf[64]; 350 | int r; 351 | 352 | fd = open(path, O_RDWR|O_CREAT, 0644); 353 | if (fd == -1) { 354 | snprintf(errstr, errstr_size, 355 | "Unable to open pidfile %s: %s", 356 | path, strerror(errno)); 357 | return -1; 358 | } 359 | 360 | /* Read current pid, if any. */ 361 | if ((r = read(fd, buf, sizeof(buf)-1)) > 0) { 362 | char *end; 363 | buf[r] = '\0'; 364 | currpid = strtoul(buf, &end, 10); 365 | if (end == buf) 366 | currpid = 0; 367 | } 368 | 369 | if (flock(fd, LOCK_EX|LOCK_NB) == -1) { 370 | if (errno == EWOULDBLOCK) 371 | snprintf(errstr, errstr_size, 372 | "Pidfile %s locked by other process (%i)", 373 | path, (int)currpid); 374 | else 375 | snprintf(errstr, errstr_size, 376 | "Failed to lock pidfile %s: %s", 377 | path, strerror(errno)); 378 | close(fd); 379 | return -1; 380 | } 381 | 382 | if (lseek(fd, 0, SEEK_SET) == -1) { 383 | snprintf(errstr, errstr_size, "Seek failed: %s", 384 | strerror(errno)); 385 | close(fd); 386 | return -1; 387 | } 388 | 389 | if (ftruncate(fd, 0) == -1) { 390 | snprintf(errstr, errstr_size, "ftruncate failed: %s", 391 | strerror(errno)); 392 | close(fd); 393 | return -1; 394 | } 395 | 396 | 397 | snprintf(buf, sizeof(buf), "%i\n", (int)getpid()); 398 | r = write(fd, buf, strlen(buf)); 399 | if (r == -1) { 400 | snprintf(errstr, errstr_size, 401 | "Failed to write pidfile %s: %s", 402 | path, strerror(errno)); 403 | close(fd); 404 | return -1; 405 | } else if (r < strlen(buf)) { 406 | snprintf(errstr, errstr_size, 407 | "Partial pidfile write %s: %i/%i bytes written", 408 | path, r, (int)strlen(buf)); 409 | close(fd); 410 | return -1; 411 | } 412 | 413 | strncpy(ezd_pidfile_path, path, sizeof(ezd_pidfile_path)-1); 414 | ezd_pidfile_fd = fd; 415 | 416 | return ezd_pidfile_fd; 417 | } 418 | 419 | 420 | 421 | enum { 422 | EZD_DAEMON_WAIT, 423 | EZD_DAEMON_FAILED, 424 | EZD_DAEMON_DIED, 425 | EZD_DAEMON_STARTED, 426 | } ezd_daemon_status = EZD_DAEMON_WAIT; 427 | 428 | static void ezd_daemon_sig_started_cb (int sig) { 429 | /* Child process is now fully started. */ 430 | ezd_daemon_status = EZD_DAEMON_STARTED; 431 | } 432 | 433 | static void ezd_daemon_sig_chld_cb (int sig) { 434 | int st; 435 | waitpid(-1, &st, 0); 436 | ezd_daemon_status = EZD_DAEMON_DIED; 437 | } 438 | 439 | int ezd_daemon (int timeout_sec, char *errstr, size_t errstr_size) { 440 | pid_t pid; 441 | sighandler_t sh_usr2_orig, sh_chld_orig; 442 | time_t timeout_abs; 443 | 444 | 445 | /* Parent process will wait for signal or termination of the 446 | * child thread. */ 447 | sh_usr2_orig = signal(SIGUSR2, ezd_daemon_sig_started_cb); 448 | sh_chld_orig = signal(SIGCHLD, ezd_daemon_sig_chld_cb); 449 | timeout_abs = time(NULL) + timeout_sec; 450 | 451 | if ((pid = fork()) == -1) { 452 | snprintf(errstr, errstr_size, 453 | "Daemon fork failed: %s", strerror(errno)); 454 | return -1; 455 | } 456 | 457 | if (pid == 0) { 458 | /* Child process. */ 459 | signal(SIGUSR2, sh_usr2_orig); 460 | signal(SIGCHLD, sh_chld_orig); 461 | return 0; 462 | } 463 | 464 | 465 | while (ezd_daemon_status == EZD_DAEMON_WAIT) { 466 | usleep(100000); 467 | if (time(NULL) >= timeout_abs) { 468 | snprintf(errstr, errstr_size, 469 | "Daemon child process (pid %i) did not " 470 | "start in %i seconds", 471 | (int)pid, timeout_sec); 472 | kill(pid, SIGTERM); 473 | ezd_daemon_status = EZD_DAEMON_FAILED; 474 | signal(SIGUSR2, sh_usr2_orig); 475 | signal(SIGCHLD, sh_chld_orig); 476 | return -1; 477 | } 478 | } 479 | 480 | signal(SIGUSR2, sh_usr2_orig); 481 | signal(SIGCHLD, sh_chld_orig); 482 | 483 | if (ezd_daemon_status == EZD_DAEMON_DIED) { 484 | snprintf(errstr, errstr_size, 485 | "Daemon child process (pid %i) terminated " 486 | "during startup", (int)pid); 487 | return -1; 488 | } else if (ezd_daemon_status == EZD_DAEMON_STARTED) 489 | exit(0); 490 | 491 | assert(!*"notreached"); 492 | exit(0); 493 | } 494 | 495 | 496 | void ezd_daemon_started (void) { 497 | int i; 498 | close(STDIN_FILENO); 499 | close(STDOUT_FILENO); 500 | close(STDERR_FILENO); 501 | for (i = 0 ; i < 3 ; i++) 502 | open("/dev/null", 0); 503 | kill(getppid(), SIGUSR2); 504 | setsid(); 505 | } 506 | 507 | 508 | 509 | int ezd_thread_sigmask (int how, ...) { 510 | va_list ap; 511 | sigset_t set; 512 | int sig; 513 | 514 | sigemptyset(&set); 515 | 516 | va_start(ap, how); 517 | while ((sig = va_arg(ap, int)) != -1) { 518 | if (sig == 0) 519 | sigfillset(&set); 520 | else 521 | sigaddset(&set, sig); 522 | } 523 | va_end(ap); 524 | 525 | return pthread_sigmask(how, &set, NULL); 526 | } 527 | 528 | -------------------------------------------------------------------------------- /ezd.h: -------------------------------------------------------------------------------- 1 | /** 2 | * ezd - eazy daemon - makes your life as a daemon conceiver easy. 3 | * 4 | * Provides the most basic tools for daemonizing a program, such as: 5 | * - configuration file parsing and reading (name=value.. pairs) 6 | * - pidfile handling with locking 7 | * - daemon()ization with wait-for-child-to-fully-start support to 8 | * allow full initialization in the child process. 9 | * 10 | * Simply add ezd.c and ezd.h to your project and use as you like. 11 | */ 12 | 13 | /* 14 | * Copyright (c) 2013 Magnus Edenhill 15 | * 16 | * All rights reserved. 17 | * 18 | * Redistribution and use in source and binary forms, with or without 19 | * modification, are permitted provided that the following conditions are met: 20 | * 21 | * 1. Redistributions of source code must retain the above copyright notice, 22 | * this list of conditions and the following disclaimer. 23 | * 2. Redistributions in binary form must reproduce the above copyright notice, 24 | * this list of conditions and the following disclaimer in the documentation 25 | * and/or other materials provided with the distribution. 26 | * 27 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" 28 | * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE 29 | * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE 30 | * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE 31 | * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR 32 | * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF 33 | * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS 34 | * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN 35 | * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) 36 | * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 37 | * POSSIBILITY OF SUCH DAMAGE. 38 | */ 39 | 40 | #pragma once 41 | 42 | #include 43 | 44 | /** 45 | * Check and create/open pidfile 'path'. 46 | * If the pidfile lock cant be acquired, or the pidfile cant be written, 47 | * this function fails and returns -1 (reason written in errstr). 48 | * On success 0 is returned and the pidfile remains opened and locked 49 | * until the application exits or calls 'ezd_pidfile_close()'. 50 | */ 51 | int ezd_pidfile_open (const char *path, char *errstr, size_t errstr_size); 52 | 53 | 54 | /** 55 | * Unlock and close the pidfile previously opened with 'ezd_pidfile_open()'. 56 | * This function should be called just prior to program termination. 57 | */ 58 | void ezd_pidfile_close (void); 59 | 60 | 61 | /** 62 | * Parses the CSV list in 'val' and allocates an array 'arrp' 63 | * with one malloced string in each slot. 64 | * Returns the number of slots in the array. 65 | */ 66 | int ezd_csv2array (char ***arrp, const char *val); 67 | 68 | 69 | /** 70 | * Read configuration file 'path'. 71 | * The configuration file format is assumed to be 'key=value' based 72 | * but if no '=' character is found the entire line is provided in 'val' to 73 | * 'conf_set_cb'. 74 | * 75 | * 'conf_set_cb' is an application provided callback to apply the 76 | * configuration lines. 77 | */ 78 | int ezd_conf_file_read (const char *path, 79 | int (*conf_set_cb) (const char *name, 80 | const char *val, 81 | char *errstr, 82 | size_t errstr_size, 83 | void *opaque), 84 | char *errstr, size_t errstr_size, 85 | void *opaque); 86 | 87 | 88 | /** 89 | * Parses the value as true or false. 90 | */ 91 | int ezd_str_tof (const char *val); 92 | 93 | 94 | /** 95 | * Regexp matcher with iovec matching output. 96 | * Compiles a regexp of 'regex', matches 'str' to it and puts the 97 | * sub-matches into the array of iovecs in 'match' of size 'matchcnt'. 98 | * Returns the number of sub-matches stored in 'match' or 0 on no-match 99 | * and -1 on regexp compilation failure. 100 | */ 101 | int ezd_regmatch (const char *regex, const char *str, 102 | struct iovec *match, int matchcnt); 103 | 104 | 105 | /** 106 | * Provides a null-terminated string in stack memory for the string in 'iov'. 107 | * For use with ezd_regmatch() 108 | */ 109 | #define ezd_strndupa_iov(iov) ({ \ 110 | struct iovec *_v = (iov); \ 111 | char *_s = alloca(_v->iov_len+1) ; \ 112 | memcpy(_s, _v->iov_base, _v->iov_len); \ 113 | _s[_v->iov_len] = '\0'; \ 114 | _s; }) 115 | 116 | 117 | /** 118 | * Start daemonization. 119 | * Parent process will linger for timeout_sec seconds waiting for 120 | * child process to call ezd_daemon_started(). 121 | * Finalize from child process with ezd_daemon_started(). 122 | * Parent process will exit(0) if child is properly started. 123 | */ 124 | int ezd_daemon (int timeout_sec, char *errstr, size_t errstr_size); 125 | 126 | /** 127 | * Tell parent process that child is now fully started. 128 | * This will finalize the ezd_daemon() call in the parent thread. 129 | */ 130 | void ezd_daemon_started (void); 131 | 132 | 133 | 134 | /** 135 | * SIG_BLOCKs or UNBLOCKs ('how') the given list of signals in 136 | * the current thread. 137 | * The list of signals must be terminated with -1. 138 | * The signal 0 means all signals. 139 | */ 140 | int ezd_thread_sigmask (int how, ...); 141 | -------------------------------------------------------------------------------- /format.c: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2014 Wikimedia Foundation 3 | * Copyright (c) 2014 Magnus Edenhill 4 | * 5 | * All rights reserved. 6 | * 7 | * Redistribution and use in source and binary forms, with or without 8 | * modification, are permitted provided that the following conditions are met: 9 | * 10 | * 1. Redistributions of source code must retain the above copyright notice, 11 | * this list of conditions and the following disclaimer. 12 | * 2. Redistributions in binary form must reproduce the above copyright notice, 13 | * this list of conditions and the following disclaimer in the documentation 14 | * and/or other materials provided with the distribution. 15 | * 16 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" 17 | * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE 18 | * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE 19 | * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE 20 | * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR 21 | * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF 22 | * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS 23 | * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN 24 | * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) 25 | * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 26 | * POSSIBILITY OF SUCH DAMAGE. 27 | */ 28 | 29 | 30 | #define _GNU_SOURCE /* for strndupa() */ 31 | #include 32 | 33 | 34 | #include 35 | #include 36 | #include 37 | 38 | #if YAJL_MAJOR >= 2 39 | #define YAJL_SIZE_TYPE size_t 40 | #else 41 | #define YAJL_SIZE_TYPE unsigned int 42 | #endif 43 | 44 | 45 | #include "kafkatee.h" 46 | 47 | 48 | 49 | /** 50 | * Temporary scratch buffer 51 | */ 52 | struct tmpbuf { 53 | struct tmpbuf *next; 54 | size_t size; 55 | int of; 56 | char buf[0]; /* Must be at end of struct: allocated to '.size' */ 57 | }; 58 | 59 | 60 | /** 61 | * Resize the scratch buffer. 62 | * This is a destructive operation, the original data will be lost. 63 | */ 64 | static void render_scratch_resize (struct render *render, int size) { 65 | assert( size >= 0 ); 66 | if (render->scratch_size < size) { 67 | if (render->scratch) 68 | free(render->scratch); 69 | 70 | render->scratch_size = size; 71 | render->scratch = malloc(size); 72 | } 73 | 74 | render->scratch_of = 0; 75 | } 76 | 77 | 78 | /** 79 | * Allocate space 'len' in scratch buffer. 80 | * The scratch buffer size is bound by its initial render_scratch_resize(). 81 | */ 82 | static char *render_scratch_alloc (struct render *render, int len) { 83 | char *p; 84 | 85 | assert(render->scratch_of + len <= render->scratch_size); 86 | 87 | p = render->scratch + render->scratch_of; 88 | render->scratch_of += len; 89 | 90 | return p; 91 | } 92 | 93 | /** 94 | * Find a %{field} formatter variable reference. 95 | */ 96 | static struct fmtvar *fmtvar_find (const struct fmt_conf *fconf, 97 | const char *var, ssize_t varlen) { 98 | 99 | int i; 100 | 101 | for (i = 0 ; i < fconf->fmtvar_cnt ; i++) { 102 | struct fmtvar *fmtvar = &fconf->fmtvar[i]; 103 | 104 | if (fmtvar->varlen == varlen && 105 | !strncmp(fmtvar->var, var, varlen)) 106 | return fmtvar; 107 | } 108 | 109 | return NULL; 110 | } 111 | 112 | /** 113 | * Write value for %{field} formatter 114 | */ 115 | static void render_fmtval_write (struct render *render, 116 | struct fmtval *fmtval, 117 | const char *val, int vallen) { 118 | if (unlikely(fmtval->val != NULL)) 119 | return; 120 | 121 | if (unlikely(!(fmtval->val = render_scratch_alloc(render, vallen)))) 122 | return; 123 | 124 | memcpy(fmtval->val, val, vallen); 125 | fmtval->vallen = vallen; 126 | 127 | } 128 | 129 | 130 | 131 | /** 132 | * One renderer per input thread. 133 | */ 134 | static __thread struct render *thread_render = NULL; 135 | 136 | /** 137 | * Destroys the thread-specific renderer. 138 | */ 139 | void render_destroy (void) { 140 | struct render *render = thread_render; 141 | 142 | if (!render) 143 | return; 144 | 145 | thread_render = NULL; 146 | 147 | if (render->scratch) 148 | free(render->scratch); 149 | free(render->fmtval); 150 | free(render); 151 | } 152 | 153 | 154 | /** 155 | * Create new thread-specific renderer. 156 | */ 157 | static struct render *render_new (const struct fmt_conf *fconf) { 158 | struct render *render; 159 | 160 | render = calloc(1, sizeof(*render)); 161 | render->fmtval = calloc(sizeof(*render->fmtval), fconf->fmtvar_cnt); 162 | render->fconf = fconf; 163 | 164 | assert(!thread_render); 165 | thread_render = render; 166 | 167 | return render; 168 | } 169 | 170 | static void render_reset (struct render *render) { 171 | memset(render->fmtval, 0, 172 | sizeof(*render->fmtval) * render->fconf->fmtvar_cnt); 173 | } 174 | 175 | /** 176 | * All constant strings in the format are placed in 'const_string' which 177 | * hopefully will be small enough to fit a single cache line. 178 | */ 179 | #define CONST_STRING_SIZE 4096 180 | static char const_string[CONST_STRING_SIZE]; 181 | static size_t const_string_len = 0; 182 | 183 | /** 184 | * Adds a constant string to the constant string area. 185 | * If the string is already found in the area, return it instead. 186 | */ 187 | static char *const_string_add (const char *in, int inlen) { 188 | char *ret; 189 | const char *instr = strndupa(in, inlen); 190 | 191 | assert( inlen >= 0 ); 192 | 193 | if (!(ret = strstr(const_string, instr))) { 194 | assert(const_string_len + inlen < CONST_STRING_SIZE); 195 | 196 | /* Append new string */ 197 | ret = const_string + const_string_len; 198 | memcpy(ret, in, inlen); 199 | ret[inlen] = '\0'; 200 | const_string_len += inlen; 201 | } 202 | 203 | return ret; 204 | } 205 | 206 | 207 | 208 | 209 | 210 | 211 | /** 212 | * Render output to '*dstp' according to 'output.format' config. 213 | */ 214 | static int render_out (struct render *render, size_t maxlen, 215 | void **dstp, size_t *dstlenp) { 216 | char *dst; 217 | int of = 0; 218 | int i; 219 | 220 | dst = malloc(maxlen); 221 | 222 | /* Traverse the formatter list */ 223 | for (i = 0 ; i < render->fconf->fmt_cnt ; i++) { 224 | struct fmt *fmt = &render->fconf->fmt[i]; 225 | struct fmtval *fmtval; 226 | const char *ptr; 227 | int len; 228 | 229 | if (fmt->varidx != -1 && 230 | (fmtval = &render->fmtval[fmt->varidx]) && 231 | fmtval->seen) { 232 | ptr = fmtval->val; 233 | len = fmtval->vallen; 234 | } else { 235 | ptr = fmt->def; 236 | len = fmt->deflen; 237 | } 238 | 239 | if (len == 0) 240 | continue; 241 | 242 | if (unlikely(of + len > maxlen)) { 243 | maxlen = (maxlen + 256) * 2; 244 | dst = realloc(dst, maxlen); 245 | } 246 | 247 | memcpy(dst+of, ptr, len); 248 | of += len; 249 | } 250 | 251 | *dstp = dst; 252 | *dstlenp = of; 253 | 254 | return 0; 255 | } 256 | 257 | 258 | 259 | /** 260 | * JSON parser callbacks (jp_..()) 261 | */ 262 | struct json_parse_ctx { 263 | enum { 264 | JP_PARSE, /* Parse next field */ 265 | JP_SKIP, /* Skip next value: key didnt match */ 266 | } state; 267 | struct fmtval *fmtval; /* write RHS value to this val. */ 268 | struct render *render; /* output rendering instance */ 269 | }; 270 | 271 | 272 | #define _JP_CHECK() do { \ 273 | if (jp->state == JP_SKIP) { \ 274 | jp->state = JP_PARSE; \ 275 | return 1; \ 276 | } \ 277 | } while (0) 278 | 279 | static int jp_write (struct json_parse_ctx *jp, const char *val, size_t len) { 280 | if (unlikely(!jp->fmtval)) 281 | return 1; 282 | 283 | render_fmtval_write(jp->render, jp->fmtval, val, len); 284 | jp->fmtval->seen++; 285 | jp->fmtval = NULL; 286 | 287 | return 1; 288 | } 289 | 290 | static int jp_bool (void *opaque, int val) { 291 | struct json_parse_ctx *jp = opaque; 292 | 293 | _JP_CHECK(); 294 | 295 | if (val) 296 | return jp_write(jp, "true", 4); 297 | else 298 | return jp_write(jp, "false", 5); 299 | } 300 | 301 | 302 | static int jp_number (void *opaque, const char *numval, 303 | YAJL_SIZE_TYPE numlen) { 304 | struct json_parse_ctx *jp = opaque; 305 | 306 | _JP_CHECK(); 307 | 308 | return jp_write(jp, numval, numlen); 309 | } 310 | 311 | 312 | static int jp_string (void *opaque, const unsigned char *val, 313 | YAJL_SIZE_TYPE len) { 314 | struct json_parse_ctx *jp = opaque; 315 | 316 | _JP_CHECK(); 317 | 318 | return jp_write(jp, (const char *)val, len); 319 | } 320 | 321 | 322 | static int jp_start_map (void *opaque) { 323 | /* Maps are flattened for now */ 324 | return 1; 325 | } 326 | 327 | 328 | static int jp_map_key (void *opaque, const unsigned char *key, 329 | YAJL_SIZE_TYPE len) { 330 | struct json_parse_ctx *jp = opaque; 331 | const struct fmtvar *fmtvar; 332 | 333 | /* Left hand side: find matching formatter variable */ 334 | if (!(fmtvar = fmtvar_find(jp->render->fconf, 335 | (const char *)key, len))) { 336 | /* Formatter key not found: skip right hand side */ 337 | jp->state = JP_SKIP; 338 | return 1; 339 | } 340 | 341 | jp->fmtval = &jp->render->fmtval[fmtvar->idx]; 342 | jp->state = JP_PARSE; 343 | 344 | return 1; 345 | } 346 | 347 | 348 | static int jp_end_map (void *opaque) { 349 | return 1; 350 | } 351 | 352 | 353 | /** 354 | * Transform payload to JSON. 355 | */ 356 | static int payload_transform_from_json (void **dstp, size_t *dstlenp, 357 | const void *src, size_t srclen) { 358 | static const yajl_callbacks callbacks = { 359 | .yajl_boolean = jp_bool, 360 | .yajl_number = jp_number, 361 | .yajl_string = jp_string, 362 | .yajl_start_map = jp_start_map, 363 | .yajl_map_key = jp_map_key, 364 | .yajl_end_map = jp_end_map, 365 | }; 366 | yajl_handle yh; 367 | struct json_parse_ctx jp = {}; 368 | yajl_status status; 369 | struct render *render; 370 | 371 | if (unlikely(!(render = thread_render))) 372 | render = render_new(&conf.fconf); 373 | else 374 | render_reset(render); 375 | 376 | /* JSON does not expand when transformed to string, 377 | * so we can use the JSON size for the scratch buffer. */ 378 | render_scratch_resize(render, srclen); 379 | 380 | jp.render = render; 381 | 382 | /* Set up JSON parser */ 383 | yh = yajl_alloc(&callbacks, NULL, 384 | #if YAJL_MAJOR < 2 385 | NULL, 386 | #endif 387 | &jp); 388 | #if YAJL_MAJOR >= 2 389 | yajl_config(yh, yajl_dont_validate_strings, 1); 390 | yajl_config(yh, yajl_allow_trailing_garbage, 1); /* e.g. newlines */ 391 | #endif 392 | 393 | /* Parse JSON */ 394 | status = yajl_parse(yh, src, srclen); 395 | if (likely(status == yajl_status_ok)) { 396 | #if YAJL_MAJOR >= 2 397 | status = yajl_complete_parse(yh); 398 | #else 399 | status = yajl_parse_complete(yh); 400 | #endif 401 | } 402 | 403 | /* Handle parsing errors, if any */ 404 | if (unlikely(status == yajl_status_error)) { 405 | char *errstr = (char *)yajl_get_error(yh, 0, NULL, 0); 406 | if (errstr[strlen(errstr)-1] == '\n') 407 | errstr[strlen(errstr)-1] = '\0'; 408 | kt_log(LOG_ERR, "JSON parse failed: %s: %.*s%s", errstr, 409 | (int)(srclen > 200 ? 200 : srclen), (const char *)src, 410 | srclen > 200 ? "..." : ""); 411 | 412 | 413 | yajl_free_error(yh, (unsigned char *)errstr); 414 | yajl_free(yh); 415 | return -1; 416 | } 417 | 418 | yajl_free(yh); 419 | 420 | 421 | return render_out(render, srclen, dstp, dstlenp); 422 | } 423 | 424 | 425 | /** 426 | * Transform payload from one encoding to another. 427 | * Returns -1 on error. 428 | */ 429 | int payload_transform (encoding_t in_enc, 430 | void **dstp, size_t *dstlenp, 431 | const void *src, size_t srclen) { 432 | 433 | assert(in_enc != conf.fconf.encoding); 434 | 435 | if (in_enc == ENC_JSON) 436 | return payload_transform_from_json(dstp, dstlenp, src, srclen); 437 | else 438 | return -1; 439 | } 440 | 441 | 442 | 443 | 444 | /** 445 | * Looks for any matching character from 'match' in 's' and returns 446 | * a pointer to the first match, or NULL if none of 'match' matched 's'. 447 | */ 448 | static char *strnchrs (const char *s, int len, const char *match) { 449 | const char *end = s + len; 450 | char map[256] = {}; 451 | while (*match) 452 | map[(int)*(match++)] = 1; 453 | 454 | while (s < end) { 455 | if (map[(int)*s]) 456 | return (char *)s; 457 | s++; 458 | } 459 | 460 | return NULL; 461 | } 462 | 463 | 464 | 465 | 466 | /** 467 | * Print parsed format string: formatters 468 | */ 469 | static __attribute__((unused)) void fmt_dump (const struct fmt_conf *fconf) { 470 | const struct fmtvar *fmtvar; 471 | int i; 472 | 473 | _DBG("%i/%i formats:", 474 | fconf->fmt_cnt, fconf->fmt_size); 475 | for (i = 0 ; i < fconf->fmt_cnt ; i++) { 476 | if (fconf->fmt[i].varidx != -1) 477 | fmtvar = &fconf->fmtvar[fconf->fmt[i].varidx]; 478 | else 479 | fmtvar = NULL; 480 | 481 | _DBG(" #%-3i \"%.*s\", varidx %i, def (%i)\"%.*s\"%s", 482 | i, 483 | fmtvar ? fmtvar->varlen : 0, 484 | fmtvar ? fmtvar->var : NULL, 485 | fconf->fmt[i].varidx, 486 | fconf->fmt[i].deflen, fconf->fmt[i].deflen, 487 | fconf->fmt[i].def, 488 | fconf->fmt[i].flags & FMT_F_ESCAPE ? ", escape" : ""); 489 | } 490 | 491 | _DBG("%i/%i fmtvars:", fconf->fmtvar_cnt, fconf->fmtvar_size); 492 | for (i = 0 ; i < fconf->fmtvar_cnt ; i++) { 493 | fmtvar = &fconf->fmtvar[i]; 494 | _DBG(" #%i/%i: \"%.*s\"", 495 | fmtvar->idx, i, fmtvar->varlen, fmtvar->var); 496 | } 497 | } 498 | 499 | 500 | /** 501 | * Adds a parsed formatter to the list of formatters 502 | */ 503 | static struct fmt *format_add (struct fmt_conf *fconf, 504 | const char *var, ssize_t varlen, 505 | const char *def, ssize_t deflen, 506 | int flags, 507 | char *errstr, size_t errstr_size) { 508 | struct fmt *fmt; 509 | 510 | if (fconf->fmt_cnt >= fconf->fmt_size) { 511 | fconf->fmt_size = (fconf->fmt_size ? : 32) * 2; 512 | fconf->fmt = realloc(fconf->fmt, 513 | fconf->fmt_size * sizeof(*fconf->fmt)); 514 | } 515 | 516 | fmt = &fconf->fmt[fconf->fmt_cnt]; 517 | memset(fmt, 0, sizeof(*fmt)); 518 | 519 | fmt->idx = fconf->fmt_cnt; 520 | fmt->flags = flags; 521 | fmt->varidx = -1; 522 | 523 | if (!def) 524 | def = "-"; 525 | 526 | if (deflen == -1) 527 | deflen = strlen(def); 528 | fmt->deflen = deflen; 529 | fmt->def = const_string_add(def, deflen); 530 | 531 | fconf->fmt_cnt++; 532 | 533 | return fmt; 534 | } 535 | 536 | 537 | /** 538 | * Adds a %{field} formatter reference. 539 | */ 540 | static struct fmtvar *fmtvar_add (struct fmt_conf *fconf, 541 | const char *var, ssize_t varlen) { 542 | struct fmtvar *fmtvar; 543 | 544 | if (fconf->fmtvar_cnt >= fconf->fmtvar_size) { 545 | fconf->fmtvar_size = (fconf->fmtvar_size ? : 32) * 2; 546 | fconf->fmtvar = realloc(fconf->fmtvar, 547 | fconf->fmtvar_size * 548 | sizeof(*fconf->fmtvar)); 549 | } 550 | 551 | fmtvar = &fconf->fmtvar[fconf->fmtvar_cnt]; 552 | fmtvar->var = const_string_add(var, varlen); 553 | fmtvar->varlen = varlen; 554 | fmtvar->idx = fconf->fmtvar_cnt++; 555 | 556 | 557 | return fmtvar; 558 | } 559 | 560 | 561 | /** 562 | * Parse the format string and build a parsing array. 563 | */ 564 | int format_parse (struct fmt_conf *fconf, const char *format_orig, 565 | char *errstr, size_t errstr_size) { 566 | const char *s, *t; 567 | const char *format; 568 | int cnt = 0; 569 | 570 | /* Perform legacy replacements. */ 571 | format = strdupa(format_orig); 572 | 573 | /* Parse the format string */ 574 | s = t = format; 575 | while (*s) { 576 | const char *begin; 577 | const char *var = NULL; 578 | int varlen = 0; 579 | const char *def = "-"; 580 | int deflen = 1; 581 | int flags = 0; 582 | const char *a; 583 | const char *b; 584 | const char *q; 585 | struct fmt *fmt; 586 | struct fmtvar *fmtvar; 587 | 588 | if (*s != '%') { 589 | s++; 590 | continue; 591 | } 592 | 593 | /* ".....%... " 594 | * ^---^ add this part as verbatim string */ 595 | if (s > t) 596 | if (!format_add(fconf, 597 | NULL, 0, 598 | t, (int)(s - t), 599 | 0, errstr, errstr_size)) 600 | return -1; 601 | 602 | begin = s; 603 | s++; 604 | 605 | if (*s != '{') { 606 | s++; 607 | continue; 608 | } 609 | 610 | 611 | /* Parse '{VAR}X': '*s' will be set to X, and 'var' to VAR. 612 | * Features: 613 | * 614 | * VAR?DEF where DEF is a default value. 615 | * 616 | */ 617 | 618 | a = s+1; 619 | b = strchr(a, '}'); 620 | 621 | if (!b) { 622 | snprintf(errstr, errstr_size, 623 | "Expecting '}' after \"%.*s...\"", 624 | 30, begin); 625 | return -1; 626 | } 627 | 628 | if (a == b) { 629 | snprintf(errstr, errstr_size, 630 | "Empty {} identifier at \"%.*s...\"", 631 | 30, begin); 632 | return -1; 633 | } 634 | 635 | s = b+1; 636 | 637 | var = a; 638 | 639 | /* Check for ?DEF and !OPTIONs */ 640 | if ((q = strnchrs(a, (int)(b-a), "?!"))) { 641 | const char *q2 = q; 642 | 643 | varlen = (int)(q - a); 644 | if (varlen == 0) 645 | var = NULL; 646 | 647 | /* Scan all ?DEF and !OPTIONs */ 648 | do { 649 | int qlen; 650 | 651 | q++; 652 | 653 | if ((q2 = strnchrs(q, (int)(b-q2-1), 654 | "@?!"))) 655 | qlen = (int)(q2-q); 656 | else 657 | qlen = (int)(b-q); 658 | 659 | switch (*(q-1)) 660 | { 661 | case '?': 662 | /* Default value */ 663 | def = q; 664 | deflen = qlen; 665 | break; 666 | case '!': 667 | /* Options */ 668 | if (0) { 669 | } else { 670 | snprintf(errstr, 671 | errstr_size, 672 | "Unknown " 673 | "formatter " 674 | "option " 675 | "\"%.*s\" at " 676 | "\"%.*s...\"", 677 | qlen, q, 678 | 30, a); 679 | return -1; 680 | } 681 | break; 682 | } 683 | 684 | } while ((q = q2)); 685 | 686 | } else 687 | varlen = (int)(b-a); 688 | 689 | /* Add formatter to ordered list of formatters */ 690 | if (!(fmt = format_add(fconf, var, varlen, 691 | def, deflen, flags, 692 | errstr, errstr_size))) 693 | return -1; 694 | 695 | cnt++; 696 | 697 | /* Now add a reference to the variable name, unless it is 698 | * already found. */ 699 | if (!(fmtvar = fmtvar_find(fconf, var, varlen))) 700 | fmtvar = fmtvar_add(fconf, var, varlen); 701 | 702 | fmt->varidx = fmtvar->idx; 703 | 704 | 705 | t = s = b+1; 706 | } 707 | 708 | /* "..%{..}....." 709 | * ^---^ add this part as verbatim string */ 710 | if (s > t) 711 | if (!format_add(fconf, NULL, 0, 712 | t, (int)(s - t), 0, 713 | errstr, errstr_size)) 714 | return -1; 715 | 716 | /* Add output delimiter to tail of format */ 717 | format_add(fconf, NULL, 0, 718 | conf.output_delimiter, strlen(conf.output_delimiter), 719 | 0, errstr, errstr_size); 720 | 721 | 722 | /* Dump parsed format string. */ 723 | if (conf.log_level >= 7) 724 | fmt_dump(fconf); 725 | 726 | 727 | if (fconf->fmt_cnt == 0) { 728 | snprintf(errstr, errstr_size, 729 | "format string is empty"); 730 | return -1; 731 | } else if (cnt == 0) { 732 | snprintf(errstr, errstr_size, 733 | "No %%.. formatters in format"); 734 | return -1; 735 | } 736 | 737 | return fconf->fmt_cnt; 738 | } 739 | 740 | 741 | 742 | 743 | 744 | 745 | 746 | 747 | -------------------------------------------------------------------------------- /format.h: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2014 Wikimedia Foundation 3 | * Copyright (c) 2014 Magnus Edenhill 4 | * 5 | * All rights reserved. 6 | * 7 | * Redistribution and use in source and binary forms, with or without 8 | * modification, are permitted provided that the following conditions are met: 9 | * 10 | * 1. Redistributions of source code must retain the above copyright notice, 11 | * this list of conditions and the following disclaimer. 12 | * 2. Redistributions in binary form must reproduce the above copyright notice, 13 | * this list of conditions and the following disclaimer in the documentation 14 | * and/or other materials provided with the distribution. 15 | * 16 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" 17 | * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE 18 | * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE 19 | * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE 20 | * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR 21 | * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF 22 | * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS 23 | * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN 24 | * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) 25 | * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 26 | * POSSIBILITY OF SUCH DAMAGE. 27 | */ 28 | 29 | #pragma once 30 | 31 | void render_destroy (void); 32 | 33 | int format_parse (struct fmt_conf *fconf, const char *format_orig, 34 | char *errstr, size_t errstr_size); 35 | 36 | int payload_transform (encoding_t in_enc, 37 | void **dstp, size_t *dstlenp, 38 | const void *src, size_t srclen); 39 | -------------------------------------------------------------------------------- /input.c: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2014 Wikimedia Foundation 3 | * Copyright (c) 2014 Magnus Edenhill 4 | * 5 | * All rights reserved. 6 | * 7 | * Redistribution and use in source and binary forms, with or without 8 | * modification, are permitted provided that the following conditions are met: 9 | * 10 | * 1. Redistributions of source code must retain the above copyright notice, 11 | * this list of conditions and the following disclaimer. 12 | * 2. Redistributions in binary form must reproduce the above copyright notice, 13 | * this list of conditions and the following disclaimer in the documentation 14 | * and/or other materials provided with the distribution. 15 | * 16 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" 17 | * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE 18 | * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE 19 | * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE 20 | * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR 21 | * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF 22 | * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS 23 | * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN 24 | * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) 25 | * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 26 | * POSSIBILITY OF SUCH DAMAGE. 27 | */ 28 | 29 | #include "kafkatee.h" 30 | #include "queue.h" 31 | #include "output.h" 32 | #include "format.h" 33 | #include "exec.h" 34 | #include "ezd.h" 35 | 36 | static LIST_HEAD(, input_s) inputs; 37 | 38 | /* Number of non-stopped inputs */ 39 | int inputs_cnt = 0; 40 | 41 | 42 | /** 43 | * Add an input. 44 | * This function must only be called during initial setup. 45 | */ 46 | input_t *input_add (input_type_t type, encoding_t enc, int flags, 47 | const char *arg1, int arg2, int64_t arg3, 48 | char *errstr, size_t errstr_size) { 49 | input_t *in; 50 | 51 | in = calloc(1, sizeof(*in)); 52 | 53 | in->in_type = type; 54 | in->in_enc = enc; 55 | in->in_flags = flags; 56 | 57 | switch (in->in_type) 58 | { 59 | case INPUT_PIPE: 60 | in->in_pipe.cmd = strdup(arg1); 61 | in->in_name = in->in_pipe.cmd; 62 | break; 63 | 64 | case INPUT_KAFKA: 65 | in->in_kafka.topic = strdup(arg1); 66 | in->in_kafka.partition = arg2; 67 | in->in_kafka.offset = arg3; 68 | in->in_name = in->in_kafka.topic; 69 | break; 70 | } 71 | 72 | LIST_INSERT_HEAD(&inputs, in, in_link); 73 | atomic_add(&inputs_cnt, 1); 74 | 75 | return in; 76 | } 77 | 78 | 79 | /** 80 | * Indicate that input 'in' has been stopped. 81 | */ 82 | static void input_stopped (input_t *in) { 83 | atomic_sub(&inputs_cnt, 1); 84 | } 85 | 86 | 87 | /** 88 | * Pipe input: main loop 89 | */ 90 | static void input_pipe_main (input_t *in) { 91 | FILE *fp = NULL; 92 | pid_t pid; 93 | int status; /* pipe exit status */ 94 | char errstr[512]; 95 | char *buf; 96 | static int our_rotate_version = 0; 97 | 98 | buf = malloc(conf.input_buf_size); 99 | 100 | while (conf.run) { 101 | char *ret; 102 | 103 | if (unlikely(!fp)) { 104 | int fd; 105 | _DBG("Starting input pipe \"%s\"", in->in_pipe.cmd); 106 | 107 | status = -1; 108 | 109 | if ((fd = kt_popen(in->in_pipe.cmd, "r", &pid, &status, 110 | errstr, sizeof(errstr))) == -1) { 111 | kt_log(LOG_ERR, 112 | "Failed to start input pipe \"%s\": %s", 113 | in->in_pipe.cmd, errstr); 114 | 115 | if (conf.flags & CONF_F_EXIT_ON_IO_TERM) { 116 | kt_log(LOG_NOTICE, 117 | "Exiting on input termination"); 118 | conf.exit_code = 2; 119 | conf.run = 0; 120 | break; 121 | } 122 | 123 | sleep(1); 124 | continue; 125 | } 126 | 127 | 128 | fp = fdopen(fd, "r"); 129 | } 130 | 131 | while (conf.run && 132 | status == -1 && 133 | ((ret = fgets(buf, conf.input_buf_size, fp)) || 134 | !ferror(fp))) { 135 | msgpayload_t *mp; 136 | 137 | _DBG("Input \"%s\" counters: rx %"PRIu64", " 138 | "fmterr %"PRIu64", empty %"PRIu64, 139 | in->in_name, 140 | in->in_c_rx, in->in_c_fmterr, in->in_c_empty); 141 | 142 | if (likely(ret != NULL)) { 143 | int len = strlen(buf); 144 | 145 | /* Remove input delimiter (newline) */ 146 | if (buf[len-1] == '\n') 147 | len--; 148 | 149 | mp = msgpayload_new(in, buf, len, NULL); 150 | if (likely(mp != NULL)) { 151 | outputs_msgpayload_enq(mp); 152 | 153 | /* Drop our reference */ 154 | msgpayload_destroy(mp); 155 | } 156 | } else { 157 | /* At EOF: clear EOF indicator, sleep, and 158 | * then try again. Unless we're not allowed 159 | * to restart on eof. */ 160 | _DBG("Input \"%s\" at EOF", in->in_name); 161 | 162 | if (in->in_flags & INPUT_F_STOP_EOF) 163 | break; 164 | 165 | clearerr(fp); 166 | usleep(100000); 167 | } 168 | 169 | if (unlikely(conf.rotate != our_rotate_version)) { 170 | our_rotate_version = conf.rotate; 171 | break; 172 | } 173 | } 174 | 175 | _DBG("Input \"%s\" Status=%i, EOF=%i, Error=%i", in->in_name, 176 | status, feof(fp), ferror(fp)); 177 | 178 | if (status != -1) { 179 | kt_log(status == 0 ? LOG_NOTICE : LOG_ERR, 180 | "Input \"%s\" (pid %i) %s", 181 | in->in_name, (int)pid, exec_exitstatus(status)); 182 | if (in->in_flags & INPUT_F_EXIT_ON_EXIT) { 183 | if (WIFEXITED(status)) 184 | conf.exit_code = WEXITSTATUS(status); 185 | else 186 | conf.exit_code = 126; 187 | conf.run = 0; 188 | break; 189 | } 190 | 191 | } else if (ferror(fp)) { 192 | kt_log(LOG_ERR, 193 | "Input \"%s\" error: %s", 194 | in->in_name, strerror(errno)); 195 | 196 | if (in->in_flags & INPUT_F_STOP_ERROR) 197 | break; 198 | } 199 | 200 | fclose(fp); 201 | fp = NULL; 202 | 203 | /* Hold off file-reopen to avoid busy-looping. */ 204 | sleep(1); 205 | } 206 | 207 | if (fp) 208 | fclose(fp); 209 | 210 | free(buf); 211 | } 212 | 213 | 214 | /** 215 | * Kafka consumer message callback. 216 | */ 217 | static void input_kafka_msg_cb (rd_kafka_message_t *rkmessage, void *opaque) { 218 | input_t *in = opaque; 219 | msgpayload_t *mp; 220 | 221 | if (rkmessage->err) { 222 | _DBG("%s: %s[%"PRId32"]: %s", 223 | rkmessage->err != RD_KAFKA_RESP_ERR__PARTITION_EOF ? 224 | "err" : "EOF", 225 | rd_kafka_topic_name(rkmessage->rkt), 226 | rkmessage->partition, 227 | rd_kafka_message_errstr(rkmessage)); 228 | 229 | if (rkmessage->err == RD_KAFKA_RESP_ERR__PARTITION_EOF) { 230 | if (in->in_flags & INPUT_F_STOP_EOF) 231 | in->in_kafka.terminate = 1; 232 | 233 | } else { 234 | if (in->in_flags & INPUT_F_STOP_ERROR) 235 | in->in_kafka.terminate = 1; 236 | } 237 | 238 | return; 239 | } 240 | 241 | /* Create payload handle. 242 | * This will also perform any configured transformations. */ 243 | mp = msgpayload_new(in, 244 | rkmessage->payload, rkmessage->len, NULL); 245 | if (unlikely(!mp)) 246 | return; 247 | 248 | /* Enqueue payload on output queue(s) */ 249 | outputs_msgpayload_enq(mp); 250 | 251 | /* Drop our reference */ 252 | msgpayload_destroy(mp); 253 | } 254 | 255 | 256 | /** 257 | * Kafka input: main loop 258 | */ 259 | static void input_kafka_main (input_t *in) { 260 | rd_kafka_topic_t *rkt; 261 | 262 | rkt = rd_kafka_topic_new(conf.rk, in->in_kafka.topic, 263 | rd_kafka_topic_conf_dup(conf.rkt_conf)); 264 | 265 | _DBG("Starting consumer for kafka topic %s partition %i", 266 | rd_kafka_topic_name(rkt), in->in_kafka.partition); 267 | 268 | if (rd_kafka_consume_start(rkt, in->in_kafka.partition, 269 | in->in_kafka.offset) == -1) { 270 | kt_log(LOG_ERR, 271 | "Failed to start Kafka consumer for topic %s " 272 | "partition %i (input \"%s\"): %s", 273 | rd_kafka_topic_name(rkt), in->in_kafka.partition, 274 | in->in_name, strerror(errno)); 275 | rd_kafka_topic_destroy(rkt); 276 | return; 277 | } 278 | 279 | while (conf.run && !in->in_kafka.terminate) { 280 | rd_kafka_consume_callback(rkt, in->in_kafka.partition, 1000, 281 | input_kafka_msg_cb, in); 282 | } 283 | 284 | rd_kafka_consume_stop(rkt, in->in_kafka.partition); 285 | 286 | rd_kafka_topic_destroy(rkt); 287 | } 288 | 289 | 290 | /** 291 | * Per-input thread's main loop trampoline. 292 | */ 293 | static void *input_main (void *arg) { 294 | input_t *in = arg; 295 | 296 | ezd_thread_sigmask(SIG_BLOCK, 0/*ALL*/, -1/*end*/); 297 | 298 | switch (in->in_type) 299 | { 300 | case INPUT_PIPE: 301 | input_pipe_main(in); 302 | break; 303 | 304 | case INPUT_KAFKA: 305 | input_kafka_main(in); 306 | break; 307 | } 308 | 309 | /* Destroy thread-specific renderer */ 310 | render_destroy(); 311 | 312 | input_stopped(in); 313 | 314 | return NULL; 315 | } 316 | 317 | 318 | /** 319 | * Wait for input 'in' to stop 320 | */ 321 | void input_wait_stopped (input_t *in) { 322 | void *ignore; 323 | pthread_join(in->in_thread, &ignore); 324 | } 325 | 326 | /** 327 | * Start input. 328 | */ 329 | int input_start (input_t *in) { 330 | 331 | if (pthread_create(&in->in_thread, NULL, input_main, in) == 1) { 332 | kt_log(LOG_ERR, 333 | "Failed to start input thread: %s", strerror(errno)); 334 | input_stopped(in); 335 | return -1; 336 | } 337 | 338 | return 0; 339 | } 340 | 341 | 342 | /** 343 | * Stop all inputs. 344 | */ 345 | void inputs_term (void) { 346 | input_t *in; 347 | 348 | LIST_FOREACH(in, &inputs, in_link) 349 | input_wait_stopped(in); 350 | } 351 | 352 | 353 | /** 354 | * Start all inputs. 355 | * Called after initial setup. 356 | */ 357 | void inputs_start (void) { 358 | input_t *in; 359 | 360 | if (LIST_EMPTY(&inputs)) { 361 | kt_log(LOG_ERR, 362 | "No inputs configured: terminating"); 363 | exit(1); 364 | } 365 | 366 | LIST_FOREACH(in, &inputs, in_link) { 367 | if (input_start(in) == -1) 368 | exit(1); 369 | } 370 | 371 | } 372 | -------------------------------------------------------------------------------- /input.h: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2014 Wikimedia Foundation 3 | * Copyright (c) 2014 Magnus Edenhill 4 | * 5 | * All rights reserved. 6 | * 7 | * Redistribution and use in source and binary forms, with or without 8 | * modification, are permitted provided that the following conditions are met: 9 | * 10 | * 1. Redistributions of source code must retain the above copyright notice, 11 | * this list of conditions and the following disclaimer. 12 | * 2. Redistributions in binary form must reproduce the above copyright notice, 13 | * this list of conditions and the following disclaimer in the documentation 14 | * and/or other materials provided with the distribution. 15 | * 16 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" 17 | * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE 18 | * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE 19 | * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE 20 | * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR 21 | * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF 22 | * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS 23 | * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN 24 | * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) 25 | * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 26 | * POSSIBILITY OF SUCH DAMAGE. 27 | */ 28 | 29 | 30 | #pragma once 31 | 32 | extern int inputs_cnt; 33 | 34 | input_t *input_add (input_type_t type, encoding_t enc, int flags, 35 | const char *arg1, int arg2, int64_t arg3, 36 | char *errstr, size_t errstr_size); 37 | 38 | void inputs_term (void); 39 | void inputs_start (void); 40 | -------------------------------------------------------------------------------- /kafkatee.c: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2014 Wikimedia Foundation 3 | * Copyright (c) 2014 Magnus Edenhill 4 | * 5 | * All rights reserved. 6 | * 7 | * Redistribution and use in source and binary forms, with or without 8 | * modification, are permitted provided that the following conditions are met: 9 | * 10 | * 1. Redistributions of source code must retain the above copyright notice, 11 | * this list of conditions and the following disclaimer. 12 | * 2. Redistributions in binary form must reproduce the above copyright notice, 13 | * this list of conditions and the following disclaimer in the documentation 14 | * and/or other materials provided with the distribution. 15 | * 16 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" 17 | * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE 18 | * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE 19 | * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE 20 | * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR 21 | * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF 22 | * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS 23 | * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN 24 | * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) 25 | * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 26 | * POSSIBILITY OF SUCH DAMAGE. 27 | */ 28 | 29 | #include "kafkatee.h" 30 | 31 | #include 32 | #include 33 | #include 34 | #include 35 | #include 36 | 37 | #include "config.h" 38 | #include "format.h" 39 | #include "input.h" 40 | #include "output.h" 41 | #include "exec.h" 42 | #include "ezd.h" 43 | 44 | static void sighup (int sig) { 45 | static int rotate_version = 0; 46 | conf.rotate = ++rotate_version; 47 | } 48 | 49 | static void term (int sig) { 50 | kt_log(LOG_NOTICE, 51 | "Received signal %i: terminating", sig); 52 | 53 | if (!conf.run) { 54 | /* Force exit on second term signal */ 55 | kt_log(LOG_WARNING, "Forced termination"); 56 | exit(0); 57 | } 58 | 59 | atomic_set(&conf.run, 0); 60 | } 61 | 62 | /** 63 | * Kafka error callback 64 | */ 65 | static void kafka_error_cb (rd_kafka_t *rk, int err, 66 | const char *reason, void *opaque) { 67 | kt_log(LOG_ERR, 68 | "Kafka error (%i): %s", err, reason); 69 | } 70 | 71 | 72 | 73 | /** 74 | * Kafka statistics callback. 75 | */ 76 | static int kafka_stats_cb (rd_kafka_t *rk, char *json, size_t json_len, 77 | void *opaque) { 78 | if (!conf.stats_fp) 79 | return 0; 80 | 81 | fprintf(conf.stats_fp, "{ \"kafka\": %s }\n", json); 82 | /* flush stats_fp to make sure valid JSON data 83 | (e.g. full lines with closing object brackets) 84 | is written to disk */ 85 | if (fflush(conf.stats_fp)) { 86 | kt_log(LOG_ERR, 87 | "Failed to fflush log.statistics.file %s: %s", 88 | conf.stats_file, strerror(errno)); 89 | } 90 | return 0; 91 | } 92 | 93 | /** 94 | * Output kafkatee specific stats to statsfile. 95 | */ 96 | static void stats_print (void) { 97 | /* FIXME: Currently none */ 98 | } 99 | 100 | static void stats_close (void) { 101 | stats_print(); 102 | fclose(conf.stats_fp); 103 | conf.stats_fp = NULL; 104 | } 105 | 106 | static int stats_open (void) { 107 | /* Already open? close and then reopen */ 108 | if (conf.stats_fp) 109 | stats_close(); 110 | 111 | if (!(conf.stats_fp = fopen(conf.stats_file, "a"))) { 112 | kt_log(LOG_ERR, 113 | "Failed to open statistics log file %s: %s", 114 | conf.stats_file, strerror(errno)); 115 | return -1; 116 | } 117 | 118 | return 0; 119 | } 120 | 121 | 122 | static void usage (const char *argv0) { 123 | fprintf(stderr, 124 | "kafkatee version %s\n" 125 | "Kafka consumer with multiple inputs and outputs\n" 126 | "\n" 127 | "Usage: %s [options]\n" 128 | "\n" 129 | "Options:\n" 130 | " -c Configuration file path (%s)\n" 131 | " -p Pid file path (%s)\n" 132 | " -d Enable debugging\n" 133 | " -D Do not daemonize\n" 134 | " -e Exit on EOF:\n" 135 | " Exit when all inputs have reached their\n" 136 | " EOF and all output queues are empty.\n" 137 | " -x Exit on input or output failure.\n" 138 | " (exit code 2)\n" 139 | "\n", 140 | KAFKATEE_VERSION, 141 | argv0, 142 | KAFKATEE_CONF_PATH, 143 | KAFKATEE_PID_FILE_PATH); 144 | exit(1); 145 | } 146 | 147 | 148 | int main (int argc, char **argv) { 149 | const char *conf_file_path = KAFKATEE_CONF_PATH; 150 | char errstr[512]; 151 | char c; 152 | int r; 153 | static int our_rotate_version = 0; 154 | 155 | /* 156 | * Default configuration 157 | */ 158 | conf.pid_file_path = strdup(KAFKATEE_PID_FILE_PATH); 159 | conf.run = 1; 160 | conf.exit_code = 0; 161 | conf.log_level = 6; 162 | conf.daemonize = 1; 163 | conf.stats_interval = 60; 164 | conf.stats_file = strdup("/tmp/kafkatee.stats.json"); 165 | conf.input_buf_size = 1024 * 10; 166 | conf.output_delimiter = strdup("\n"); 167 | conf.output_delimiter_len = strlen(conf.output_delimiter); 168 | conf.output_queue_size = 100000; 169 | 170 | /* Kafka main configuration */ 171 | conf.rk_conf = rd_kafka_conf_new(); 172 | rd_kafka_conf_set(conf.rk_conf, "client.id", "kafkatee", NULL, 0); 173 | rd_kafka_conf_set_error_cb(conf.rk_conf, kafka_error_cb); 174 | 175 | /* Kafka topic configuration template */ 176 | conf.rkt_conf = rd_kafka_topic_conf_new(); 177 | 178 | /* Parse command line arguments */ 179 | while ((c = getopt(argc, argv, "hc:p:dDex")) != -1) { 180 | switch (c) { 181 | case 'h': 182 | usage(argv[0]); 183 | break; 184 | case 'c': 185 | conf_file_path = optarg; 186 | break; 187 | case 'p': 188 | conf.pid_file_path = strdup(optarg); 189 | break; 190 | case 'd': 191 | conf.log_level = 7; 192 | break; 193 | case 'D': 194 | conf.daemonize = 0; 195 | break; 196 | case 'e': 197 | conf.flags |= CONF_F_EXIT_ON_EOF; 198 | break; 199 | case 'x': 200 | conf.flags |= CONF_F_EXIT_ON_IO_TERM; 201 | break; 202 | default: 203 | usage(argv[0]); 204 | break; 205 | } 206 | } 207 | 208 | openlog("kafkatee", LOG_PID|LOG_PERROR, LOG_DAEMON); 209 | 210 | /* Read config file */ 211 | if (ezd_conf_file_read(conf_file_path, conf_set, 212 | errstr, sizeof(errstr), NULL) == -1) { 213 | kt_log(LOG_ERR, "%s", errstr); 214 | exit(1); 215 | } 216 | 217 | /* Daemonize if desired */ 218 | if (conf.daemonize) { 219 | if (ezd_daemon(10, errstr, sizeof(errstr)) == -1) { 220 | kt_log(LOG_ERR, "%s", errstr); 221 | exit(1); 222 | } 223 | 224 | if (ezd_pidfile_open(conf.pid_file_path, 225 | errstr, sizeof(errstr)) == -1) { 226 | kt_log(LOG_ERR, "%s", errstr); 227 | exit(1); 228 | } 229 | } 230 | 231 | 232 | /* Parse the format string */ 233 | if (conf.fconf.format) { 234 | if (conf.fconf.encoding != ENC_STRING) { 235 | kt_log(LOG_ERR, "Output formatting only supported for " 236 | "output.encoding = string"); 237 | if (conf.daemonize) 238 | ezd_pidfile_close(); 239 | exit(1); 240 | } 241 | 242 | if (format_parse(&conf.fconf, conf.fconf.format, 243 | errstr, sizeof(errstr)) == -1) { 244 | kt_log(LOG_ERR, 245 | "Failed to parse format string: %s\n%s", 246 | conf.fconf.format, errstr); 247 | if (conf.daemonize) 248 | ezd_pidfile_close(); 249 | exit(1); 250 | } 251 | } 252 | 253 | /* Set up statistics gathering in librdkafka, if enabled. */ 254 | if (conf.stats_interval) { 255 | char tmp[30]; 256 | 257 | if (stats_open() == -1) { 258 | kt_log(LOG_ERR, 259 | "Failed to open statistics log file %s: %s", 260 | conf.stats_file, strerror(errno)); 261 | if (conf.daemonize) 262 | ezd_pidfile_close(); 263 | exit(1); 264 | } 265 | 266 | snprintf(tmp, sizeof(tmp), "%i", conf.stats_interval*1000); 267 | rd_kafka_conf_set_stats_cb(conf.rk_conf, kafka_stats_cb); 268 | rd_kafka_conf_set(conf.rk_conf, "statistics.interval.ms", tmp, 269 | NULL, 0); 270 | } 271 | 272 | 273 | /* Create Kafka handle */ 274 | if (!(conf.rk = rd_kafka_new(RD_KAFKA_CONSUMER, conf.rk_conf, 275 | errstr, sizeof(errstr)))) { 276 | kt_log(LOG_ERR, 277 | "Failed to create kafka handle: %s", errstr); 278 | if (conf.daemonize) 279 | ezd_pidfile_close(); 280 | exit(1); 281 | } 282 | 283 | rd_kafka_set_log_level(conf.rk, conf.log_level); 284 | 285 | 286 | /* Initialize subsystems */ 287 | exec_init(); 288 | 289 | /* Finalize daemonization */ 290 | if (conf.daemonize) 291 | ezd_daemon_started(); 292 | 293 | /* Run init command, if any. */ 294 | if (conf.cmd_init) { 295 | if ((r = system(conf.cmd_init) != 0)) 296 | kt_log(LOG_ERR, 297 | "\"command.init\" execution of \"%s\" failed " 298 | "with exit code %i", conf.cmd_init, r); 299 | } 300 | 301 | /* Block all signals in the main thread so new threads get the same 302 | * procmask. */ 303 | ezd_thread_sigmask(SIG_BLOCK, 0/*ALL*/, -1/*end*/); 304 | 305 | /* Start IO */ 306 | outputs_start(); 307 | inputs_start(); 308 | 309 | 310 | /* Set main thread sigmask */ 311 | ezd_thread_sigmask(SIG_UNBLOCK, SIGHUP, SIGINT, SIGTERM, -1); 312 | signal(SIGHUP, sighup); 313 | signal(SIGINT, term); 314 | signal(SIGTERM, term); 315 | 316 | kt_log(LOG_INFO, "kafkatee starting"); 317 | /* Main loop */ 318 | while (conf.run) { 319 | rd_kafka_poll(conf.rk, 1000); 320 | if (unlikely(conf.rotate != our_rotate_version)) { 321 | our_rotate_version = conf.rotate; 322 | if (conf.stats_interval) 323 | stats_open(); 324 | } 325 | } 326 | 327 | inputs_term(); 328 | outputs_term(); 329 | exec_term(); 330 | 331 | rd_kafka_destroy(conf.rk); 332 | rd_kafka_wait_destroyed(5000); 333 | 334 | /* if stats_fp is set (i.e. open), close it. */ 335 | if (conf.stats_fp) 336 | stats_close(); 337 | free(conf.stats_file); 338 | 339 | /* Run termination command, if any. */ 340 | if (conf.cmd_term) { 341 | if ((r = system(conf.cmd_term) != 0)) 342 | kt_log(LOG_ERR, 343 | "\"command.term\" execution of \"%s\" failed " 344 | "with exit code %i", conf.cmd_term, r); 345 | } 346 | 347 | if (conf.daemonize) 348 | ezd_pidfile_close(); 349 | 350 | kt_log(LOG_INFO, "kafkatee exiting"); 351 | exit(conf.exit_code); 352 | } 353 | -------------------------------------------------------------------------------- /kafkatee.conf.example: -------------------------------------------------------------------------------- 1 | ####################################################################### 2 | # # 3 | # kafkatee configuration file # 4 | # # 5 | # # 6 | ####################################################################### 7 | # # 8 | # Syntax: # 9 | # = # 10 | # input # 11 | # output # 12 | # # 13 | # Boolean property values: # 14 | # >0, "true", "yes", "on", "" - interpreted as true # 15 | # everything else - interpreted as false # 16 | # # 17 | # # 18 | # The configuration file consists of: # 19 | # - Configuration properties (key = value) to control various # 20 | # aspects of kafkatee. # 21 | # - Inputs # 22 | # - Outputs # 23 | # # 24 | ####################################################################### 25 | 26 | 27 | 28 | 29 | ####################################################################### 30 | # # 31 | # Misc configuration # 32 | # # 33 | ####################################################################### 34 | 35 | # Pid file location 36 | # Default: /run/kafkatee.pid 37 | #pid.file.path = kafkatee.pid 38 | 39 | # Daemonize (background) 40 | # Default: true 41 | #daemonize = false 42 | 43 | # Logging output level 44 | # 1 = only emergencies .. 6 = info, 7 = debug 45 | # Default: 6 (info) 46 | #log.level = 7 47 | 48 | 49 | # 50 | # JSON Statistics 51 | # 52 | # Statistics is collected from kafkatee itself(*) as well as librdkafka 53 | # Each JSON object has a top level key of either 'kafkatee' or 54 | # 'kafka' to indicate which type of statistics the object contains. 55 | # Each line is a valid JSON object. 56 | # 57 | # *: kafkatee does not currently output any stats of its own, just from rdkafka. 58 | # 59 | 60 | # Statistics output interval 61 | # Defaults to 60 seconds, use 0 to disable. 62 | #log.statistics.interval = 60 63 | 64 | # Statistics output file 65 | # Defaults to /tmp/kafkatee.stats.json 66 | #log.statistics.file = /tmp/kafkatee.stats.json 67 | 68 | 69 | # Command to run on startup, before starting IO. 70 | # Default: none 71 | #command.init = ./my-script.sh 72 | 73 | # Command to run on termination after all IOs have been stopped. 74 | # Default: none 75 | #command.term = ./my-cleanup-script.sh 76 | 77 | # Set environment variable which will be available for all sub-sequent 78 | # command executions (command.*, input pipe, output pipe, ..) 79 | #setenv.NMSGS=12 80 | # clear: 81 | #setenv.LD_LIBRARY_PATH= 82 | 83 | 84 | 85 | 86 | ####################################################################### 87 | # # 88 | # Kafka configuration # 89 | # # 90 | # Kafka configuration properties are prefixed with "kafka." # 91 | # and topic properties are prefixed with "kafka.topic.". # 92 | # # 93 | # For the full range of Kafka handle and topic configuration # 94 | # properties, see: # 95 | # http://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md # 96 | # # 97 | # And the Apache Kafka configuration reference: # 98 | # http://kafka.apache.org/08/configuration.html # 99 | # # 100 | ####################################################################### 101 | 102 | # Initial list of kafka brokers 103 | # Default: none 104 | #kafka.metadata.broker.list = localhost 105 | 106 | # Offset file directory. 107 | # Each topic + partition combination has its own offset file. 108 | # Default: current directory 109 | #kafka.topic.offset.store.path = /var/cache/kafkatee/offsets/ 110 | 111 | # If the request offset was not found on broker, or there is no 112 | # initial offset known (no stored offset), then reset the offset according 113 | # to this configuration. 114 | # Values: smallest (oldest/beginning), largest (newest/end), error (fail) 115 | # Default: largest 116 | #kafka.topic.auto.offset.reset = smallest 117 | 118 | # Maximum message size. 119 | # Should be synchronized on all producers, brokers and consumers. 120 | # Default: 4000000 121 | #kafka.message.max.bytes = 10000000 122 | 123 | # Kafka debugging 124 | # Default: none 125 | #kafka.debug = msg,topic,broker 126 | 127 | 128 | 129 | 130 | ####################################################################### 131 | # # 132 | # Message transformation # 133 | # # 134 | # A message read from one of the inputs may be transformed before # 135 | # being enqueued on the output queues. # 136 | # # 137 | # Transformation requires that the input and output encoding differs, # 138 | # i.e., 'input [encoding=json] ..' and 'output.encoding=string' # 139 | # # 140 | # While the input encoding is configured per input, the output # 141 | # encoding is configured globally, all outputs will receive the same # 142 | # message. # 143 | # # 144 | # The currently supported transformation(s) are: # 145 | # JSON input -> string output: # 146 | # JSON data is formatted according to the output.format # 147 | # configuration. The %{field} syntax references the field in the # 148 | # original JSON object by the same name and outputs its value. # 149 | # # 150 | # If the input and output encoding matches then the message remains # 151 | # untouched. # 152 | # # 153 | # The output message delimiter (defaults to newline (\n)) is # 154 | # configurable (output.delimiter) and always appended to all output # 155 | # messages regardless of transformation. # 156 | # The input is always stripped of its delimiter (which is newline # 157 | # for pipe inputs). # 158 | # # 159 | ####################################################################### 160 | 161 | # Output encoding: string or json 162 | # Default: string 163 | #output.encoding = string 164 | 165 | 166 | 167 | 168 | ####################################################################### 169 | # Output formatting # 170 | # # 171 | # The format string is made up of %{..}-tokens and literals. # 172 | # # 173 | # Tokens: # 174 | # # 175 | # %{FIELD} # 176 | # Retrieves the value from the JSON object's field with the # 177 | # same name. # 178 | # # 179 | # %{FIELD?DEFAULT} # 180 | # 'DEFAULT' is the default string to use if no field was matched, # 181 | # the default default string is "-". # 182 | # # 183 | # Literals are copied verbatim to the output string. # 184 | # # 185 | # Example JSON: {"task":19, "name":"Mike"} # 186 | # Example format: Got task %{task} for user %{name?unknown} # 187 | # Example output: Got task 19 for user Mike # 188 | # # 189 | # Note: Multi-level JSON objects are flattened: # 190 | # JSON: {"a": {"b": 9}, "c": "hi"} # 191 | # Gives: { "b": 9, "c": "hi" } # 192 | # # 193 | ####################################################################### 194 | 195 | # Output format for JSON -> string transformation. 196 | # Default: none 197 | #output.format = %{hostname} %{sequence} %{dt} %{time_firstbyte} %{ip} %{handling}/%{http_status} %{bytes_sent} %{request_method} http://%{host}%{uri}%{query} - %{mime_type} %{referer} %{x_forwarded_for} %{user_agent} %{accept_language} %{x_analytics} 198 | 199 | # Output delimiter 200 | # The output message ends with this delimiter. 201 | # Supports \n, \r, \t, \0. 202 | # Default: newline (\n) 203 | #output.delimiter = ;END;\n 204 | 205 | # Maximum queue size for each output, in number of messages 206 | # Default: 100000 207 | #output.queue.size = 1000000 208 | 209 | 210 | 211 | 212 | ####################################################################### 213 | # # 214 | # Inputs # 215 | # # 216 | # The following types of inputs are supported: # 217 | # - Kafka consumer # 218 | # - Piped command # 219 | # # 220 | # Any number and mix of inputs can be configured. # 221 | # Each input may be configured with an optional list of # 222 | # input-specific configuration properties, called the key-values. # 223 | # # 224 | # Supported key-values: # 225 | # - encoding=string|json - message encoding from this input. # 226 | # Default: string # 227 | # # 228 | # - stop.eof=true|false - do not continue try reading from # 229 | # this input when EOF has been reached # 230 | # Default: false # 231 | # # 232 | # - stop.error=true|false - do not reopen/restart input on error. # 233 | # Default: false # 234 | # # 235 | # The key-values is CSV-separated and the list of key-values must be # 236 | # enveloped by brackets: [encoding=string,foo=bar] # 237 | # # 238 | ####################################################################### 239 | 240 | 241 | ####################################################################### 242 | # Kafka consumer input syntax: # 243 | # input [key-values] kafka topic partition from # 244 | # # 245 | # where: # 246 | # - [key-values] is an optional CSV-separated list of key-values. # 247 | # NOTE: the enveloping brackets are literal. # 248 | # - is the Kafka topic to consume from. # 249 | # - or - is the partition, or range of partitions, to # 250 | # consume from. # 251 | # - is the offset to start consuming from. # 252 | # supported values: beginning, end, stored, # 253 | # Where 'stored' means to use a local offset file to store and # 254 | # read the offset from, which allows a later run of kafkatee # 255 | # to pick up from where it left off. # 256 | # # 257 | ####################################################################### 258 | 259 | #input [encoding=json] kafka topic varnish partition 0-10 from stored 260 | #input [encoding=string] kafka topic test1 partition 0 from end 261 | 262 | 263 | ####################################################################### 264 | # Piped command input syntax: # 265 | # input [key-values] pipe # 266 | # # 267 | # where: # 268 | # - [key-values] is an optional CSV-separated list of key-values. # 269 | # NOTE: the enveloping brackets are literal. # 270 | # - is a command string that will be executed with: # 271 | # /bin/sh -c "", thus supporting pipes, etc. # 272 | # # 273 | ####################################################################### 274 | 275 | #input [encoding=string] pipe tail -f a.log | grep -v ^DEBUG: 276 | #input pipe wget -q -O- http://example.com/api/events 277 | 278 | 279 | 280 | 281 | ####################################################################### 282 | # # 283 | # Outputs # 284 | # # 285 | # The following types of outputs are supported: # 286 | # - Piped command # 287 | # - File # 288 | # # 289 | # Each output has its own queue where messages are enqueued prior to # 290 | # writing to the output, this queue is limited by output.queue.size. # 291 | # If the queue limit is reached no new messages are added to the # 292 | # queue (tail-drop). # 293 | # # 294 | # Outputs are configured with a sample rate, 1 means every message, # 295 | # 2 means every other message, 1000 means every 1000nd message, and # 296 | # so on. # 297 | # $ 298 | # If an output process terminates, or an output file fails writing, # 299 | # the output is closed and reopened/restarted. The messages in the # 300 | # output's queue remain in the queue while the output is unavailable. # 301 | # # 302 | ####################################################################### 303 | 304 | 305 | ####################################################################### 306 | # Piped command output syntax: # 307 | # output pipe # 308 | # # 309 | # where: # 310 | # - is the sample-rate: 1 for each message, 100 for # 311 | # every 100rd message, and so on. # 312 | # - is a command string that will be executed with: # 313 | # /bin/sh -c "", thus supporting pipes, etc. # 314 | # # 315 | # Output pipes are stopped and restarted if kafkatee receives a # 316 | # SIGHUP signal. # 317 | ####################################################################### 318 | 319 | #output pipe 1 grep ^Something >> somefile.log 320 | #output pipe 1000 do-some-stats.sh 321 | #output pipe 1 nc -u syslog.example.com 514 322 | 323 | 324 | ####################################################################### 325 | # File output syntax: # 326 | # output file # 327 | # # 328 | # where: # 329 | # - is the sample-rate: 1 for each message, 100 for # 330 | # every 100rd message, and so on. # 331 | # - is the file path to write. The file is appended. # 332 | # # 333 | # Output files are closed and reopened if kafkatee receives a # 334 | # SIGHUP signal, thus allowing log file rotation. # 335 | ####################################################################### 336 | 337 | #output file 100 /tmp/sampled-100.txt 338 | 339 | 340 | 341 | 342 | ####################################################################### 343 | # Include other config files using include clause. # 344 | # You may use file glob matching to include files, e.g. # 345 | # include /etc/kafkatee.d/*.conf # 346 | # # 347 | # These files will be include in alphabetic order and override # 348 | # any previously set configs. # 349 | ####################################################################### 350 | 351 | # Include other config file 352 | #include local.conf 353 | -------------------------------------------------------------------------------- /kafkatee.h: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2014 Wikimedia Foundation 3 | * Copyright (c) 2014 Magnus Edenhill 4 | * 5 | * All rights reserved. 6 | * 7 | * Redistribution and use in source and binary forms, with or without 8 | * modification, are permitted provided that the following conditions are met: 9 | * 10 | * 1. Redistributions of source code must retain the above copyright notice, 11 | * this list of conditions and the following disclaimer. 12 | * 2. Redistributions in binary form must reproduce the above copyright notice, 13 | * this list of conditions and the following disclaimer in the documentation 14 | * and/or other materials provided with the distribution. 15 | * 16 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" 17 | * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE 18 | * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE 19 | * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE 20 | * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR 21 | * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF 22 | * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS 23 | * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN 24 | * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) 25 | * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 26 | * POSSIBILITY OF SUCH DAMAGE. 27 | */ 28 | 29 | #pragma once 30 | 31 | #define __GNU_SOURCE /* For strdup() */ 32 | #include 33 | #include 34 | #include 35 | #include 36 | #include 37 | #include 38 | #include 39 | #include 40 | #include 41 | #include 42 | #include 43 | #include 44 | 45 | #include 46 | 47 | #define KAFKATEE_CONF_PATH "/etc/kafkatee.conf" 48 | #define KAFKATEE_PID_FILE_PATH "/run/kafkatee.pid" 49 | 50 | #define _DBG(fmt...) do { \ 51 | if (conf.log_level > 6) { \ 52 | printf(fmt); \ 53 | printf("\n"); \ 54 | } \ 55 | } while (0) 56 | 57 | #define atomic_add(ptr,incr) __sync_add_and_fetch(ptr,incr) 58 | #define atomic_sub(ptr,decr) __sync_sub_and_fetch(ptr,decr) 59 | #define atomic_set(ptr,val) __sync_lock_test_and_set(ptr,val) 60 | 61 | #ifndef likely 62 | #define likely(x) __builtin_expect((x),1) 63 | #endif 64 | 65 | #ifndef unlikely 66 | #define unlikely(x) __builtin_expect((x),0) 67 | #endif 68 | 69 | typedef struct msgpayload_s { 70 | struct iovec mp_iov[2]; 71 | int mp_iovcnt; 72 | int mp_refcnt; 73 | rd_kafka_message_t *mp_rkm; 74 | } msgpayload_t; 75 | 76 | typedef struct msg_s { 77 | TAILQ_ENTRY(msg_s) m_link; 78 | msgpayload_t *m_mp; 79 | struct iovec m_iov[2]; 80 | int m_iovcnt; 81 | } msg_t; 82 | 83 | typedef struct msgq_s { 84 | TAILQ_HEAD(, msg_s) mq_msgs; 85 | int mq_msgcnt; 86 | pthread_mutex_t mq_lock; 87 | pthread_cond_t mq_cond; 88 | time_t mq_t_write; /* Time of last fd write */ 89 | } msgq_t; 90 | 91 | #define msgq_lock(mq) pthread_mutex_lock(&(mq)->mq_lock) 92 | #define msgq_unlock(mq) pthread_mutex_unlock(&(mq)->mq_lock) 93 | 94 | typedef enum { 95 | INPUT_PIPE, 96 | INPUT_KAFKA, 97 | } input_type_t; 98 | 99 | 100 | typedef enum { 101 | ENC_ERROR = -1, 102 | ENC_STRING, 103 | ENC_JSON, 104 | } encoding_t; 105 | 106 | typedef struct input_s { 107 | LIST_ENTRY(input_s) in_link; 108 | 109 | input_type_t in_type; 110 | encoding_t in_enc; 111 | char *in_name; 112 | 113 | int in_flags; 114 | #define INPUT_F_STOP_EOF 0x1 /* Stop on EOF */ 115 | #define INPUT_F_STOP_ERROR 0x2 /* Stop on exit/error */ 116 | #define INPUT_F_EXIT_ON_EXIT 0x4 /* Exit kafkatee if this input exits */ 117 | #define INPUT_F_DEFAULTS 0 118 | 119 | union { 120 | struct { 121 | char *cmd; 122 | } pipe; 123 | struct { 124 | char *topic; 125 | int partition; 126 | int64_t offset; 127 | rd_kafka_topic_t *rkt; 128 | int terminate; 129 | } kafka; 130 | } in_u; 131 | #define in_kafka in_u.kafka 132 | #define in_pipe in_u.pipe 133 | 134 | uint64_t in_c_rx; /* Read messages */ 135 | uint64_t in_c_fmterr; /* Messages dropped due to formatting error */ 136 | uint64_t in_c_empty; /* Empty messages dropped */ 137 | 138 | pthread_t in_thread; 139 | } input_t; 140 | 141 | 142 | typedef enum { 143 | OUTPUT_PIPE, 144 | OUTPUT_FILE, 145 | } output_type_t; 146 | 147 | typedef struct output_s { 148 | LIST_ENTRY(output_s) o_link; 149 | int o_id; 150 | char *o_name; 151 | output_type_t o_type; 152 | int o_sample_rate; 153 | uint64_t o_c_rx; /* number of received messages */ 154 | uint64_t o_c_tx; /* number of transmitted messages */ 155 | uint64_t o_c_tx_qdrop; /* number of output queue drops */ 156 | 157 | int o_fd; /* Output file descriptor */ 158 | time_t o_t_open; /* Time of file creation/cmd start */ 159 | time_t o_t_close; /* Time of last close/termination */ 160 | time_t o_t_open_fail; /* Time of last open failure */ 161 | int o_open_fails; /* Number of consecutive 162 | * open failures */ 163 | 164 | msgq_t o_outq; 165 | 166 | union { 167 | struct { 168 | char *cmd; 169 | pid_t pid; 170 | int status; /* Exit status */ 171 | } pipe; 172 | 173 | struct { 174 | char *path; 175 | } file; 176 | } o_u; 177 | 178 | #define o_pipe o_u.pipe 179 | #define o_file o_u.file 180 | 181 | } output_t; 182 | 183 | 184 | 185 | /** 186 | * Formatting variable (%{VAR}) 187 | */ 188 | struct fmtvar { 189 | const char *var; 190 | int varlen; 191 | int idx; 192 | }; 193 | 194 | /** 195 | * Formatting value 196 | */ 197 | struct fmtval { 198 | char *val; 199 | int vallen; 200 | int seen; 201 | }; 202 | 203 | 204 | /** 205 | * Formatting from format 206 | */ 207 | struct fmt { 208 | int idx; /* fmt[] array index */ 209 | int varidx; /* fmtvar index (non-zero if formatter is a var) */ 210 | const char *def; /* default string, typically "-" */ 211 | int deflen; /* default string's length */ 212 | int flags; 213 | #define FMT_F_ESCAPE 0x1 /* Escape the value string */ 214 | }; 215 | 216 | 217 | /** 218 | * Formatter configuration 219 | */ 220 | struct fmt_conf { 221 | encoding_t encoding; 222 | 223 | /* Array of formatters in output order. */ 224 | struct fmt *fmt; 225 | int fmt_cnt; 226 | int fmt_size; 227 | 228 | /* Array of fmtvars */ 229 | struct fmtvar *fmtvar; 230 | int fmtvar_cnt; 231 | int fmtvar_size; 232 | 233 | char *format; 234 | }; 235 | 236 | 237 | 238 | /** 239 | * Formatter renderer 240 | */ 241 | struct render { 242 | const struct fmt_conf *fconf; 243 | struct fmtval *fmtval; /* Variables' values */ 244 | 245 | /* Value scratch buffer */ 246 | char *scratch; 247 | int scratch_of; 248 | int scratch_size; 249 | }; 250 | 251 | 252 | 253 | /** 254 | * kafkatee configuration container 255 | */ 256 | struct conf { 257 | char *output_delimiter; /* Output delimiter */ 258 | int output_delimiter_len; 259 | 260 | struct fmt_conf fconf; /* Output format */ 261 | 262 | int output_queue_size; /* Per-output queue size */ 263 | 264 | int input_queue_size_min; 265 | int input_buf_size; 266 | 267 | int run; 268 | int output_id_next; 269 | 270 | int rotate; 271 | int log_level; 272 | int log_rate; 273 | int log_rate_period; 274 | int daemonize; 275 | 276 | int stats_interval; 277 | char *stats_file; 278 | FILE *stats_fp; 279 | 280 | char *pid_file_path; 281 | 282 | char *cmd_init; /* Command run prior to starting IO */ 283 | char *cmd_term; /* Command run at termination */ 284 | 285 | int flags; 286 | #define CONF_F_EXIT_ON_EOF 0x1 /* Exit when all inputs have reached 287 | * their end-of-file and all 288 | * output queues are empty. */ 289 | #define CONF_F_EXIT_ON_IO_TERM 0x2 /* Exit on input/output termination. */ 290 | 291 | int exit_code; 292 | 293 | rd_kafka_t *rk; 294 | rd_kafka_conf_t *rk_conf; 295 | rd_kafka_topic_conf_t *rkt_conf; 296 | 297 | }; 298 | 299 | extern struct conf conf; 300 | 301 | 302 | #define kt_log(level,fmt...) syslog(level, fmt) 303 | 304 | 305 | 306 | 307 | #ifndef TAILQ_FOREACH_SAFE 308 | /* 309 | * TAILQ_FOREACH_SAFE() provides a traversal where the current iterated element 310 | * may be freed or unlinked. 311 | * It does not allow freeing or modifying any other element in the list, 312 | * at least not the next element. 313 | */ 314 | #define TAILQ_FOREACH_SAFE(elm,head,field,tmpelm) \ 315 | for ((elm) = TAILQ_FIRST(head) ; \ 316 | (elm) && ((tmpelm) = TAILQ_NEXT((elm), field), 1) ; \ 317 | (elm) = (tmpelm)) 318 | #endif 319 | 320 | 321 | #ifndef LIST_INSERT_SORTED 322 | #define LIST_INSERT_SORTED(head, elm, field, cmpfunc) do { \ 323 | if(LIST_EMPTY(head)) { \ 324 | LIST_INSERT_HEAD(head, elm, field); \ 325 | } else { \ 326 | typeof(elm) _tmp; \ 327 | LIST_FOREACH(_tmp,head,field) { \ 328 | if(cmpfunc(elm,_tmp) <= 0) { \ 329 | LIST_INSERT_BEFORE(_tmp,elm,field); \ 330 | break; \ 331 | } \ 332 | if(!LIST_NEXT(_tmp,field)) { \ 333 | LIST_INSERT_AFTER(_tmp,elm,field); \ 334 | break; \ 335 | } \ 336 | } \ 337 | } \ 338 | } while(0) 339 | #endif 340 | -------------------------------------------------------------------------------- /output.c: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2014 Wikimedia Foundation 3 | * Copyright (c) 2014 Magnus Edenhill 4 | * 5 | * All rights reserved. 6 | * 7 | * Redistribution and use in source and binary forms, with or without 8 | * modification, are permitted provided that the following conditions are met: 9 | * 10 | * 1. Redistributions of source code must retain the above copyright notice, 11 | * this list of conditions and the following disclaimer. 12 | * 2. Redistributions in binary form must reproduce the above copyright notice, 13 | * this list of conditions and the following disclaimer in the documentation 14 | * and/or other materials provided with the distribution. 15 | * 16 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" 17 | * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE 18 | * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE 19 | * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE 20 | * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR 21 | * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF 22 | * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS 23 | * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN 24 | * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) 25 | * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 26 | * POSSIBILITY OF SUCH DAMAGE. 27 | */ 28 | 29 | #define _GNU_SOURCE 30 | 31 | #include "kafkatee.h" 32 | #include "queue.h" 33 | #include "exec.h" 34 | #include "input.h" 35 | #include "ezd.h" 36 | 37 | #include 38 | #include 39 | #include 40 | #include 41 | #include 42 | #include 43 | #include 44 | #include 45 | #include 46 | #include 47 | 48 | 49 | static LIST_HEAD(, output_s) outputs; /* Sorted in sample-rate order */ 50 | static int outputs_cnt; 51 | static int outputs_epoll_fd = -1; 52 | static pthread_t outputs_thread; 53 | 54 | 55 | /** 56 | * If output is eligible for a message (according to queue pressure and 57 | * its configured sample-rate) then create a new message shadowing the 58 | * msgpayload and enqueue it on the output's output queue. 59 | */ 60 | static int output_msgpayload_enq (output_t *o, msgpayload_t *mp) { 61 | 62 | msgq_lock(&o->o_outq); 63 | if (o->o_outq.mq_msgcnt < conf.output_queue_size) { 64 | if (!(o->o_c_rx++ % o->o_sample_rate)) 65 | msgq_enq(&o->o_outq, msg_new(mp)); 66 | } else 67 | o->o_c_tx_qdrop++; 68 | msgq_unlock(&o->o_outq); 69 | return 0; 70 | } 71 | 72 | 73 | /** 74 | * Attempt to enqueue msgs based on the msgpayload on all outputs. 75 | */ 76 | int outputs_msgpayload_enq (msgpayload_t *mp) { 77 | output_t *o; 78 | int cnt = 0; 79 | 80 | LIST_FOREACH(o, &outputs, o_link) { 81 | if (output_msgpayload_enq(o, mp) != -1) 82 | cnt++; 83 | } 84 | 85 | return cnt; 86 | } 87 | 88 | 89 | /** 90 | * Output comparator for sorting: sort ascendingly on sample_rate 91 | */ 92 | static int output_cmp (void *_a, void *_b) { 93 | output_t *a = _a, *b = _b; 94 | 95 | return a->o_sample_rate - b->o_sample_rate; 96 | } 97 | 98 | 99 | /** 100 | * Add a new output. 101 | * This must only be done during initial setup and is typically done 102 | * by the config file reader. 103 | */ 104 | output_t *output_add (output_type_t type, int sample_rate, const char *cmd) { 105 | output_t *o; 106 | 107 | o = calloc(1, sizeof(*o)); 108 | o->o_id = conf.output_id_next++; 109 | o->o_type = type; 110 | o->o_sample_rate = sample_rate; 111 | o->o_fd = -1; 112 | 113 | assert(*cmd); 114 | 115 | switch (o->o_type) 116 | { 117 | case OUTPUT_PIPE: 118 | o->o_pipe.cmd = strdup(cmd); 119 | o->o_name = o->o_pipe.cmd; 120 | break; 121 | 122 | case OUTPUT_FILE: 123 | o->o_file.path = strdup(cmd); 124 | o->o_name = o->o_file.path; 125 | break; 126 | } 127 | 128 | msgq_init(&o->o_outq); 129 | 130 | LIST_INSERT_SORTED(&outputs, o, o_link, output_cmp); 131 | outputs_cnt++; 132 | 133 | 134 | 135 | return o; 136 | } 137 | 138 | 139 | 140 | /** 141 | * Open/start output. 142 | */ 143 | int output_open (output_t *o) { 144 | int fd = -1; 145 | struct epoll_event ev = {}; 146 | int r; 147 | char errstr[512]; 148 | 149 | _DBG("Opening output \"%s\"", o->o_name); 150 | 151 | assert(o->o_fd == -1); 152 | 153 | switch (o->o_type) 154 | { 155 | case OUTPUT_PIPE: 156 | o->o_pipe.status = 0; 157 | if ((fd = kt_popen(o->o_pipe.cmd, "w", 158 | &o->o_pipe.pid, &o->o_pipe.status, 159 | errstr, sizeof(errstr))) == -1) { 160 | kt_log(LOG_ERR, 161 | "Failed to start output pipe #%i \"%s\": %s", 162 | o->o_id, o->o_pipe.cmd, errstr); 163 | goto failed; 164 | } 165 | _DBG("Started output \"%s\" pid %i", 166 | o->o_name, o->o_pipe.pid); 167 | break; 168 | 169 | case OUTPUT_FILE: 170 | if ((fd = open(o->o_file.path, 171 | O_WRONLY|O_APPEND|O_CREAT, 0644)) == -1) { 172 | kt_log(LOG_ERR, 173 | "Failed to open output file #%i \"%s\": %s", 174 | o->o_id, o->o_file.path, strerror(errno)); 175 | goto failed; 176 | } 177 | _DBG("Output file \"%s\" opened", o->o_name); 178 | 179 | break; 180 | 181 | default: 182 | goto failed; 183 | } 184 | 185 | /* Set non-blocking */ 186 | if ((r = fcntl(fd, F_GETFL)) == -1 || 187 | fcntl(fd, F_SETFL, r|O_NONBLOCK) == -1) { 188 | kt_log(LOG_ERR, 189 | "Failed to set non-blocking output for \"%s\": %s", 190 | o->o_name, strerror(errno)); 191 | goto failed; 192 | } 193 | 194 | o->o_fd = fd; 195 | o->o_t_open = time(NULL); 196 | o->o_open_fails = 0; 197 | o->o_t_open_fail = 0; 198 | 199 | /* Start polling for writability (edge-triggered) */ 200 | ev.events = EPOLLOUT | EPOLLET; 201 | ev.data.ptr = o; 202 | epoll_ctl(outputs_epoll_fd, EPOLL_CTL_ADD, o->o_fd, &ev); 203 | 204 | 205 | return 0; 206 | 207 | failed: 208 | if (fd != -1) 209 | close(o->o_fd); 210 | o->o_open_fails++; 211 | o->o_t_open_fail = time(NULL); 212 | return -1; 213 | } 214 | 215 | 216 | 217 | /** 218 | * Close/stop an output. 219 | */ 220 | void output_close (output_t *o, int level, const char *reason) { 221 | 222 | if (o->o_fd == -1) 223 | return; 224 | 225 | kt_log(level, "Closing output \"%s\": %s (%i messages in queue)", 226 | o->o_name, reason, o->o_outq.mq_msgcnt); 227 | 228 | epoll_ctl(outputs_epoll_fd, EPOLL_CTL_DEL, o->o_fd, NULL); 229 | 230 | switch (o->o_type) 231 | { 232 | case OUTPUT_PIPE: 233 | case OUTPUT_FILE: 234 | close(o->o_fd); 235 | break; 236 | } 237 | 238 | o->o_fd = -1; 239 | o->o_t_close = time(NULL); 240 | 241 | if (conf.run && conf.flags & CONF_F_EXIT_ON_IO_TERM) { 242 | kt_log(LOG_NOTICE, "Exiting on output \"%s\" termination", 243 | o->o_name); 244 | conf.exit_code = 2; 245 | conf.run = 0; 246 | } 247 | } 248 | 249 | 250 | 251 | /** 252 | * Attempts to write as many messages as possible from the output's out queue 253 | * to the output fd. 254 | * Returns 1 if more messages can be written, 0 if not, or -1 on error. 255 | */ 256 | static int output_write0 (output_t *o) { 257 | struct iovec iov[IOV_MAX]; 258 | int i = 0; 259 | ssize_t r; 260 | msg_t *m, *mtmp; 261 | 262 | /* Collect messages into iovec */ 263 | msgq_lock(&o->o_outq); 264 | TAILQ_FOREACH(m, &o->o_outq.mq_msgs, m_link) { 265 | memcpy(&iov[i], &m->m_iov[0], sizeof(iov[i]) * m->m_iovcnt); 266 | i += m->m_iovcnt; 267 | if (unlikely(i+2 == IOV_MAX)) 268 | break; 269 | } 270 | msgq_unlock(&o->o_outq); 271 | 272 | if (i == 0) 273 | return 0; 274 | 275 | /* Write iovec */ 276 | r = writev(o->o_fd, iov, i); 277 | if (r == -1) { 278 | if (likely(errno == EAGAIN)) 279 | return 0; 280 | _DBG("Write failure on fd %i at %"PRIu64" msgs sent: %s", 281 | o->o_fd, o->o_c_tx, strerror(errno)); 282 | return -1; 283 | } 284 | 285 | /* Unlink written messages and adjust partially written messages 286 | * for the next write attempt. */ 287 | msgq_lock(&o->o_outq); 288 | TAILQ_FOREACH_SAFE(m, &o->o_outq.mq_msgs, m_link, mtmp) { 289 | if (r == 0) 290 | break; 291 | 292 | for (i = 0 ; i < m->m_iovcnt ; i++) { 293 | if (m->m_iov[i].iov_len <= r) { 294 | /* Wrote full iov element */ 295 | r -= m->m_iov[i].iov_len; 296 | if (i == m->m_iovcnt - 1) { 297 | /* Wrote full message */ 298 | o->o_c_tx++; 299 | msgq_deq(&o->o_outq, m); 300 | msg_destroy(m); 301 | break; 302 | } 303 | } else { 304 | /* Wrote partial iov element */ 305 | m->m_iov[i].iov_base = 306 | (char *)m->m_iov[i].iov_base + r; 307 | m->m_iov[i].iov_len -= r; 308 | goto done; 309 | } 310 | } 311 | } 312 | done: 313 | msgq_unlock(&o->o_outq); 314 | 315 | /* All messages written? If so we may be called again */ 316 | return !m ? 1 : 0; 317 | } 318 | 319 | 320 | /** 321 | * Write as many messages to file descriptor from outq as possible. 322 | * Propogate errors to caller. 323 | */ 324 | static int output_write (output_t *o) { 325 | int r; 326 | 327 | if (unlikely(o->o_fd == -1)) 328 | return -1; 329 | 330 | while ((r = output_write0(o)) > 0) 331 | ; 332 | 333 | return r; 334 | } 335 | 336 | 337 | 338 | /** 339 | * Check if any outputs needs starting/opening. 340 | */ 341 | static void outputs_check (void) { 342 | output_t *o; 343 | time_t now = time(NULL); 344 | int totqcnt = 0; 345 | 346 | LIST_FOREACH(o, &outputs, o_link) { 347 | #if 0 /* Output queue pressure debugging */ 348 | _DBG("\"%s\" queue pressure: %i (rx %"PRIu64", tx %"PRIu64 349 | ", tx qdrop %"PRIu64", fd %i)", 350 | o->o_name, o->o_outq.mq_msgcnt, 351 | o->o_c_rx, o->o_c_tx, o->o_c_tx_qdrop, o->o_fd); 352 | #endif 353 | 354 | msgq_lock(&o->o_outq); 355 | totqcnt = o->o_outq.mq_msgcnt; 356 | msgq_unlock(&o->o_outq); 357 | 358 | /* Already open */ 359 | if (o->o_fd != -1) { 360 | if (o->o_type == OUTPUT_PIPE && 361 | o->o_pipe.status != -1) 362 | output_close(o, LOG_WARNING, 363 | exec_exitstatus(o->o_pipe.status)); 364 | continue; 365 | } 366 | 367 | /* Last open failed: back off */ 368 | if (o->o_t_open_fail + (2 * (o->o_open_fails % 10)) > now) 369 | continue; 370 | 371 | /* Open output */ 372 | output_open(o); 373 | } 374 | 375 | /* exit_on_eof check, terminate application if: 376 | * - all inputs are stopped 377 | * - all output queues are drained. */ 378 | if (unlikely(inputs_cnt == 0 && 379 | (conf.flags & CONF_F_EXIT_ON_EOF) && 380 | totqcnt == 0)) { 381 | kt_log(LOG_NOTICE, "Exiting on EOF"); 382 | conf.run = 0; 383 | } 384 | } 385 | 386 | 387 | 388 | /** 389 | * Output rotation: 390 | * Close all outputs and open them again. 391 | */ 392 | static void outputs_rotate (void) { 393 | output_t *o; 394 | 395 | kt_log(LOG_INFO, "Rotating %i outputs", outputs_cnt); 396 | 397 | LIST_FOREACH(o, &outputs, o_link) { 398 | /* Close output */ 399 | if (o->o_fd != -1) 400 | output_close(o, LOG_DEBUG, "rotate"); 401 | 402 | /* Reset open_fail to force a quick retry */ 403 | o->o_t_open_fail = 0; 404 | } 405 | 406 | /* Checker will re-open outputs */ 407 | outputs_check(); 408 | } 409 | 410 | 411 | /** 412 | * The outputs thread main loop. 413 | */ 414 | static void *outputs_main (void *ignore) { 415 | struct epoll_event *ev; 416 | time_t t_last_check = 0; 417 | output_t *o; 418 | static int our_rotate_version = 0; 419 | 420 | /* Block all signals in this thread */ 421 | ezd_thread_sigmask(SIG_BLOCK, 0/*ALL*/, -1/*end*/); 422 | 423 | ev = malloc(sizeof(*ev) * outputs_cnt); 424 | 425 | while (conf.run) { 426 | int r; 427 | time_t now; 428 | int i; 429 | 430 | if (unlikely(conf.rotate != our_rotate_version)) { 431 | /* Outputs rotation */ 432 | our_rotate_version = conf.rotate; 433 | outputs_rotate(); 434 | } else { 435 | /* Periodic outputs checker */ 436 | now = time(NULL); 437 | if (unlikely(t_last_check + 1 < now)) { 438 | t_last_check = now; 439 | outputs_check(); 440 | } 441 | } 442 | 443 | 444 | /* Poll output fds for writeability */ 445 | r = epoll_wait(outputs_epoll_fd, ev, outputs_cnt, 100); 446 | 447 | /* Handle events */ 448 | for (i = 0 ; i < r ; i++) { 449 | o = ev[i].data.ptr; 450 | 451 | if (unlikely(ev[i].events & EPOLLHUP)) 452 | output_close(o, LOG_WARNING, "hung up"); 453 | } 454 | 455 | /* Try to write as many messages as possible to all outputs 456 | * regardless of poll status or not, the poll status is 457 | * probably outdated by the time we come here anyway. */ 458 | LIST_FOREACH(o, &outputs, o_link) 459 | if (unlikely(output_write(o) == -1)) 460 | output_close(o, LOG_WARNING, "write failure"); 461 | } 462 | 463 | free(ev); 464 | 465 | /* Close all outputs */ 466 | LIST_FOREACH(o, &outputs, o_link) 467 | output_close(o, LOG_INFO, "terminating"); 468 | 469 | return NULL; 470 | } 471 | 472 | 473 | /** 474 | * Terminate outputs sub-system 475 | */ 476 | void outputs_term (void) { 477 | void *ignore; 478 | pthread_join(outputs_thread, &ignore); 479 | } 480 | 481 | /** 482 | * Initialize outputs sub-system 483 | */ 484 | void outputs_start (void) { 485 | 486 | /* Create epoll handle */ 487 | outputs_epoll_fd = epoll_create1(EPOLL_CLOEXEC); 488 | if (outputs_epoll_fd == -1) { 489 | kt_log(LOG_CRIT, "Failed to create epoll fd: %s", 490 | strerror(errno)); 491 | exit(1); 492 | } 493 | 494 | /* Start outputs thread */ 495 | if (pthread_create(&outputs_thread, NULL, outputs_main, NULL) == -1) { 496 | kt_log(LOG_CRIT, "Failed to create outputs thread: %s", 497 | strerror(errno)); 498 | exit(1); 499 | } 500 | } 501 | -------------------------------------------------------------------------------- /output.h: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2014 Wikimedia Foundation 3 | * Copyright (c) 2014 Magnus Edenhill 4 | * 5 | * All rights reserved. 6 | * 7 | * Redistribution and use in source and binary forms, with or without 8 | * modification, are permitted provided that the following conditions are met: 9 | * 10 | * 1. Redistributions of source code must retain the above copyright notice, 11 | * this list of conditions and the following disclaimer. 12 | * 2. Redistributions in binary form must reproduce the above copyright notice, 13 | * this list of conditions and the following disclaimer in the documentation 14 | * and/or other materials provided with the distribution. 15 | * 16 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" 17 | * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE 18 | * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE 19 | * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE 20 | * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR 21 | * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF 22 | * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS 23 | * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN 24 | * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) 25 | * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 26 | * POSSIBILITY OF SUCH DAMAGE. 27 | */ 28 | 29 | #pragma once 30 | 31 | 32 | int outputs_msgpayload_enq (msgpayload_t *m); 33 | output_t *output_add (output_type_t type, int sample_rate, const char *cmd); 34 | 35 | void outputs_term (void); 36 | void outputs_start (void); 37 | -------------------------------------------------------------------------------- /queue.c: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2014 Wikimedia Foundation 3 | * Copyright (c) 2014 Magnus Edenhill 4 | * 5 | * All rights reserved. 6 | * 7 | * Redistribution and use in source and binary forms, with or without 8 | * modification, are permitted provided that the following conditions are met: 9 | * 10 | * 1. Redistributions of source code must retain the above copyright notice, 11 | * this list of conditions and the following disclaimer. 12 | * 2. Redistributions in binary form must reproduce the above copyright notice, 13 | * this list of conditions and the following disclaimer in the documentation 14 | * and/or other materials provided with the distribution. 15 | * 16 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" 17 | * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE 18 | * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE 19 | * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE 20 | * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR 21 | * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF 22 | * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS 23 | * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN 24 | * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) 25 | * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 26 | * POSSIBILITY OF SUCH DAMAGE. 27 | */ 28 | 29 | #include "kafkatee.h" 30 | #include "format.h" 31 | 32 | #include 33 | 34 | 35 | /** 36 | * Number of original messages (i.e., msgpayloads) currently in existence. 37 | */ 38 | int msgs_cnt = 0; 39 | 40 | 41 | /** 42 | * Unref msgpayload, free it if refcnt reaches 0. 43 | */ 44 | void msgpayload_destroy (msgpayload_t *mp) { 45 | if (atomic_sub(&mp->mp_refcnt, 1) > 0) 46 | return; 47 | 48 | if (mp->mp_rkm) 49 | rd_kafka_message_destroy(mp->mp_rkm); 50 | else 51 | free(mp->mp_iov[0].iov_base); 52 | 53 | free(mp); 54 | 55 | atomic_sub(&msgs_cnt, 1); 56 | } 57 | 58 | /** 59 | * Unref message's msgpayload and free 'm'. 60 | */ 61 | void msg_destroy (msg_t *m) { 62 | msgpayload_destroy(m->m_mp); 63 | free(m); 64 | } 65 | 66 | 67 | /** 68 | * Create new sharable msgpayload. 69 | * Performs payload transformation according to configuration. 70 | */ 71 | msgpayload_t *msgpayload_new (input_t *in, void *payload, size_t len, 72 | rd_kafka_message_t *rkm) { 73 | msgpayload_t *mp; 74 | void *final_payload; 75 | size_t final_len; 76 | int extra_delim = 0; 77 | 78 | atomic_add(&in->in_c_rx, 1); 79 | 80 | /* Perform payload transformation if necessary. */ 81 | if (in->in_enc != conf.fconf.encoding) { 82 | if (unlikely(payload_transform(in->in_enc, 83 | &final_payload, &final_len, 84 | payload, len) == -1)) { 85 | atomic_add(&in->in_c_fmterr, 1); 86 | return NULL; 87 | } 88 | } else { 89 | /* No transformation */ 90 | if (rkm) { 91 | /* Kafka message: dont copy, point to message payload*/ 92 | final_payload = payload; 93 | final_len = len; 94 | /* As we cant modify the payload for non-copied messages 95 | * we add the delimiter as an extra iovec instead. */ 96 | extra_delim = 1; 97 | } else { 98 | /* Copy data */ 99 | final_payload = malloc(len+conf.output_delimiter_len); 100 | memcpy(final_payload, payload, len); 101 | memcpy((char *)final_payload+len, conf.output_delimiter, 102 | conf.output_delimiter_len); 103 | final_len = len+conf.output_delimiter_len; 104 | } 105 | } 106 | 107 | if (unlikely(final_len == 0)) { 108 | atomic_add(&in->in_c_empty, 1); 109 | return NULL; 110 | } 111 | 112 | mp = malloc(sizeof(*mp)); 113 | mp->mp_iov[0].iov_base = final_payload; 114 | mp->mp_iov[0].iov_len = final_len; 115 | if (extra_delim) { 116 | mp->mp_iov[1].iov_base = conf.output_delimiter; 117 | mp->mp_iov[1].iov_len = conf.output_delimiter_len; 118 | mp->mp_iovcnt = 2; 119 | } else 120 | mp->mp_iovcnt = 1; 121 | mp->mp_rkm = rkm; 122 | mp->mp_refcnt = 1; 123 | 124 | atomic_add(&msgs_cnt, 1); 125 | 126 | return mp; 127 | } 128 | 129 | 130 | /** 131 | * Create a message pointing to the provided payload 132 | */ 133 | msg_t *msg_new (msgpayload_t *mp) { 134 | msg_t *m; 135 | 136 | m = malloc(sizeof(*m)); 137 | m->m_mp = mp; 138 | memcpy(m->m_iov, mp->mp_iov, sizeof(*mp->mp_iov) * mp->mp_iovcnt); 139 | m->m_iovcnt = mp->mp_iovcnt; 140 | atomic_add(&mp->mp_refcnt, 1); 141 | 142 | return m; 143 | } 144 | 145 | 146 | /** 147 | * Enqueue message on queue. 148 | * NOTE: msgq must be locked 149 | */ 150 | void msgq_enq (msgq_t *mq, msg_t *m) { 151 | TAILQ_INSERT_TAIL(&mq->mq_msgs, m, m_link); 152 | mq->mq_msgcnt++; 153 | } 154 | 155 | /** 156 | * Dequeue message from queue. 157 | * NOTE: msgq must be locked 158 | */ 159 | void msgq_deq (msgq_t *mq, msg_t *m) { 160 | assert(mq->mq_msgcnt > 0); 161 | TAILQ_REMOVE(&mq->mq_msgs, m, m_link); 162 | mq->mq_msgcnt--; 163 | } 164 | 165 | 166 | /** 167 | * Initialize message queue. 168 | */ 169 | void msgq_init (msgq_t *mq) { 170 | TAILQ_INIT(&mq->mq_msgs); 171 | mq->mq_msgcnt = 0; 172 | } 173 | 174 | -------------------------------------------------------------------------------- /queue.h: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2014 Wikimedia Foundation 3 | * Copyright (c) 2014 Magnus Edenhill 4 | * 5 | * All rights reserved. 6 | * 7 | * Redistribution and use in source and binary forms, with or without 8 | * modification, are permitted provided that the following conditions are met: 9 | * 10 | * 1. Redistributions of source code must retain the above copyright notice, 11 | * this list of conditions and the following disclaimer. 12 | * 2. Redistributions in binary form must reproduce the above copyright notice, 13 | * this list of conditions and the following disclaimer in the documentation 14 | * and/or other materials provided with the distribution. 15 | * 16 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" 17 | * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE 18 | * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE 19 | * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE 20 | * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR 21 | * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF 22 | * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS 23 | * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN 24 | * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) 25 | * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 26 | * POSSIBILITY OF SUCH DAMAGE. 27 | */ 28 | 29 | #pragma once 30 | 31 | void msgq_enq (msgq_t *mq, msg_t *m); 32 | void msgq_deq (msgq_t *mq, msg_t *m); 33 | void msgq_init (msgq_t *mq); 34 | 35 | void msgpayload_destroy (msgpayload_t *mp); 36 | msgpayload_t *msgpayload_new (input_t *in, void *payload, size_t len, 37 | rd_kafka_message_t *rkm); 38 | 39 | void msg_destroy (msg_t *m); 40 | msg_t *msg_new (msgpayload_t *mp); 41 | 42 | -------------------------------------------------------------------------------- /test/Makefile: -------------------------------------------------------------------------------- 1 | TESTS ?= test-*.conf 2 | 3 | .PHONY: 4 | .PRECIOUS: %.conf 5 | 6 | %.conf: .PHONY 7 | @(echo "\033[34m######## Test $@ START #########\033[0m" ; \ 8 | ../kafkatee -c $@ -p kafkatee.test.pid -D -x ; \ 9 | if [ $$? -eq 0 ]; then \ 10 | echo "\033[32m######## Test $@ PASSED #########\033[0m" ; \ 11 | true ; \ 12 | else \ 13 | echo "\033[31m######## Test $@ FAILED #########\033[0m" ; \ 14 | false ; \ 15 | fi) 16 | 17 | all: linecnt ingen $(TESTS) 18 | 19 | linecnt: linecnt.c 20 | $(CC) -O2 -Wall -Werror $^ -o $@ 21 | 22 | ingen: ingen.c 23 | $(CC) -O2 -Wall -Werror $^ -o $@ 24 | 25 | -------------------------------------------------------------------------------- /test/README: -------------------------------------------------------------------------------- 1 | Automated regression tests for kafkatee 2 | ======================================= 3 | 4 | Each test is implemented as a standard kafkatee configuration file named 5 | "test-*.conf". 6 | 7 | A local configuration file must be written, use local.conf.example as template. 8 | 9 | 10 | Run all tests: 11 | 12 | make 13 | 14 | 15 | Run specific tests: 16 | 17 | TESTS="first.conf second.conf" make 18 | 19 | 20 | 21 | 22 | Prerequisits: 23 | * Kafka broker where the topic "kafkatee_test" exists with 10 partitions. 24 | * A configuration file for local configuration: local.conf 25 | * The rdkafka_example tool from librdkafka must be in $PATH 26 | 27 | 28 | Tools: 29 | * testpipe.py generates input for kafkatee and checks the output from it, 30 | verifying the number of messages, contents and order. 31 | * linecnt simply counts the number of lines received and performs better 32 | than testpipe.py 33 | * ingen generates simple input and performs better than testpipe.py 34 | -------------------------------------------------------------------------------- /test/base.conf: -------------------------------------------------------------------------------- 1 | # Test base config, should be included from all tests. 2 | 3 | # Commands to run on init and termination. 4 | # We use them to create and purge FIFOs. 5 | command.init = for N in `seq 1 $NOUTS`; do rm -f test$N.fifo ; mkfifo test$N.fifo ; done 6 | command.term = for N in `seq 1 $NOUTS`; do rm -f test$N.fifo ; done 7 | 8 | 9 | # Include local configuration file. 10 | include local.conf -------------------------------------------------------------------------------- /test/ingen.c: -------------------------------------------------------------------------------- 1 | /** 2 | * ingen - simple input generator for kafkatee tests 3 | */ 4 | /* 5 | * Copyright (c) 2014 Wikimedia Foundation 6 | * Copyright (c) 2014 Magnus Edenhill 7 | * 8 | * All rights reserved. 9 | * 10 | * Redistribution and use in source and binary forms, with or without 11 | * modification, are permitted provided that the following conditions are met: 12 | * 13 | * 1. Redistributions of source code must retain the above copyright notice, 14 | * this list of conditions and the following disclaimer. 15 | * 2. Redistributions in binary form must reproduce the above copyright notice, 16 | * this list of conditions and the following disclaimer in the documentation 17 | * and/or other materials provided with the distribution. 18 | * 19 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" 20 | * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE 21 | * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE 22 | * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE 23 | * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR 24 | * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF 25 | * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS 26 | * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN 27 | * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) 28 | * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 29 | * POSSIBILITY OF SUCH DAMAGE. 30 | */ 31 | 32 | 33 | #include 34 | #include 35 | #include 36 | #include 37 | #include 38 | #include 39 | #include 40 | 41 | 42 | int main (int argc, char **argv) { 43 | int cnt; 44 | char buf[] = "This is just a test line\n"; 45 | int len = strlen(buf); 46 | 47 | if (argc < 2) { 48 | printf("Usage: %s \n", argv[0]); 49 | exit(1); 50 | } 51 | 52 | cnt = atoi(argv[1]); 53 | 54 | 55 | while (cnt-- > 0) 56 | if (fwrite(buf, 1, len, stdout) == -1) { 57 | perror("write error"); 58 | exit(1); 59 | } 60 | 61 | exit(0); 62 | } 63 | -------------------------------------------------------------------------------- /test/linecnt.c: -------------------------------------------------------------------------------- 1 | /** 2 | * linecnt - simple output line counter for kafkatee testing 3 | */ 4 | /* 5 | * Copyright (c) 2014 Wikimedia Foundation 6 | * Copyright (c) 2014 Magnus Edenhill 7 | * 8 | * All rights reserved. 9 | * 10 | * Redistribution and use in source and binary forms, with or without 11 | * modification, are permitted provided that the following conditions are met: 12 | * 13 | * 1. Redistributions of source code must retain the above copyright notice, 14 | * this list of conditions and the following disclaimer. 15 | * 2. Redistributions in binary form must reproduce the above copyright notice, 16 | * this list of conditions and the following disclaimer in the documentation 17 | * and/or other materials provided with the distribution. 18 | * 19 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" 20 | * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE 21 | * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE 22 | * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE 23 | * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR 24 | * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF 25 | * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS 26 | * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN 27 | * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) 28 | * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 29 | * POSSIBILITY OF SUCH DAMAGE. 30 | */ 31 | 32 | 33 | #include 34 | #include 35 | #include 36 | #include 37 | #include 38 | 39 | static int run = 1; 40 | 41 | static void do_timeout (int sig) { 42 | run = 0; 43 | fclose(stdin); /* trigger fgets() to quit */ 44 | } 45 | 46 | int main (int argc, char **argv) { 47 | int cnt = 0; 48 | int expected; 49 | int sleeptime = 5; 50 | int endtime = 0; 51 | char buf[4096]; 52 | 53 | if (argc < 3) { 54 | printf("Usage: %s " 55 | " " 56 | " []\n", argv[0]); 57 | exit(1); 58 | } 59 | 60 | expected = atoi(argv[1]); 61 | alarm(atoi(argv[2])); 62 | signal(SIGALRM, do_timeout); 63 | 64 | if (argc == 4) 65 | sleeptime = atoi(argv[3]); 66 | 67 | 68 | while (run && fgets(buf, sizeof(buf)-1, stdin)) { 69 | cnt++; 70 | 71 | if (cnt >= expected) { 72 | if (!endtime) 73 | endtime = time(NULL) + sleeptime; 74 | else if (time(NULL) >= endtime) 75 | break; 76 | } 77 | } 78 | 79 | printf("%i lines read, %i expected\n", cnt, expected); 80 | 81 | if (cnt == expected) 82 | exit(0); 83 | else 84 | exit(2); 85 | } 86 | -------------------------------------------------------------------------------- /test/local.conf.example: -------------------------------------------------------------------------------- 1 | # Local configuration for tests 2 | 3 | # List of Kafka brokers 4 | kafka.metadata.broker.list = localhost 5 | 6 | -------------------------------------------------------------------------------- /test/test-kafka.conf: -------------------------------------------------------------------------------- 1 | # Tests JSON -> string transformation with Kafka 2 | # * one Kafka input 3 | # * 3 outputs 4 | # * JSON -> string transformation 5 | # * sample-rates 1,10,100 6 | 7 | 8 | include base.conf 9 | 10 | # Number of outputs (you need to create the same number of 11 | # 'output pipe ..' lines below. 12 | env.NOUTS=10 13 | 14 | # Number of messages to send 15 | env.NMSGS=10000 16 | 17 | 18 | # Output encoding 19 | output.encoding = string 20 | 21 | # Output formatting 22 | output.format = %{seq},%{field1},%{field2},%{field3},%{seq} 23 | 24 | 25 | # Execute testpipe.py as an input but pipe its output to a kafka producer. 26 | # This means this input will not produce any messages. 27 | # The sleep allows the kafkatee kafka consumer to read from the end of the 28 | # queue before we start producing, to make sure we dont read any old messages 29 | # and dont miss any new ones. 30 | input [encoding=json,exit.on.exit=true] pipe ./testpipe.py --cnt $NMSGS --genenc json --checkenc string --timeout 100 --config '{"outputs": [{"fifo":"test1.fifo","rate":1},{"fifo":"test2.fifo","rate":10},{"fifo":"test3.fifo","rate":100},{"fifo":"test4.fifo","rate":1},{"fifo":"test5.fifo","rate":1000},{"fifo":"test6.fifo","rate":500},{"fifo":"test7.fifo","rate":3},{"fifo":"test8.fifo","rate":1},{"fifo":"test9.fifo","rate":1},{"fifo":"test10.fifo","rate":1}]}' | (sleep 3 ; rdkafka_example -P -t kafkatee_test -p 0 -b localhost) > /dev/null 31 | 32 | # Instead specify a kafka input for the same topic. 33 | input [encoding=json] kafka topic kafkatee_test partition 0 from end 34 | 35 | # Output pipe simply writes to FIFOs that testpipe.py reads. 36 | # The number of outputs must be the sa,e as NMSGS above. 37 | output pipe 1 cat >> test1.fifo 38 | output pipe 10 cat >> test2.fifo 39 | output pipe 100 cat >> test3.fifo 40 | output pipe 1 cat >> test4.fifo 41 | output pipe 1000 cat >> test5.fifo 42 | output pipe 500 cat >> test6.fifo 43 | output pipe 3 cat >> test7.fifo 44 | output pipe 1 cat >> test8.fifo 45 | output pipe 1 cat >> test9.fifo 46 | output pipe 1 cat >> test10.fifo 47 | 48 | -------------------------------------------------------------------------------- /test/test-multi_json.conf: -------------------------------------------------------------------------------- 1 | # Tests JSON -> string transformation 2 | # * one input 3 | # * 4 outputs 4 | # * JSON -> string transformation 5 | # * sample-rates 1,10,100,1000 6 | 7 | # Commands to run on init and termination. 8 | # We use them to create and purge FIFOs. 9 | env.NOUTS=4 10 | command.init = for N in `seq 1 $NOUTS`; do rm -f test$N.fifo ; mkfifo test$N.fifo ; done 11 | command.term = for N in `seq 1 $NOUTS`; do rm -f test$N.fifo ; done 12 | 13 | # Output encoding 14 | output.encoding = string 15 | 16 | # Output formatting 17 | output.format = %{seq},%{field1},%{field2},%{field3},%{seq} 18 | 19 | 20 | # Input from testpipe.py script that will 21 | # output to stdout and read from the FIFOs 22 | input [encoding=json,exit.on.exit=true] pipe ./testpipe.py --cnt 1000 --genenc json --checkenc string --config '{"outputs": [{"fifo":"test1.fifo","rate":1},{"fifo":"test2.fifo","rate":10},{"fifo":"test3.fifo","rate":100},{"fifo":"test4.fifo","rate":1000}]}' 23 | 24 | # Output pipe simply writes to FIFOs that testpipe.py reads. 25 | # Outputs with sample-rate 1, 10 and 1000 26 | output pipe 1 cat >> test1.fifo 27 | output pipe 10 cat >> test2.fifo 28 | output pipe 100 cat >> test3.fifo 29 | output pipe 1000 cat >> test4.fifo 30 | 31 | 32 | #log.level = 7 33 | -------------------------------------------------------------------------------- /test/test-multi_output.conf: -------------------------------------------------------------------------------- 1 | # Tests plain message forwarding: 2 | # * one input 3 | # * three outputs 4 | # * string -> string (no transformation) 5 | # * sample-rates 1,10,100 6 | 7 | # Commands to run on init and termination. 8 | # We use them to create and purge FIFOs. 9 | env.NOUTS=3 10 | command.init = for N in `seq 1 $NOUTS`; do rm -f test$N.fifo ; mkfifo test$N.fifo ; done 11 | command.term = for N in `seq 1 $NOUTS`; do rm -f test$N.fifo ; done 12 | 13 | # Output encoding 14 | output.encoding = string 15 | 16 | # Input from testpipe.py script that will 17 | # output to stdout and read from the FIFOs 18 | input [encoding=string,exit.on.exit=true] pipe ./testpipe.py --cnt 1000 --genenc string --checkenc string --config '{"outputs": [{"fifo":"test1.fifo","rate":1},{"fifo":"test2.fifo","rate":10},{"fifo":"test3.fifo","rate":100}]}' 19 | 20 | # Output pipe simply writes to FIFOs that testpipe.py reads. 21 | # Outputs with sample-rate 1, 10 and 1000 22 | output pipe 1 cat >> test1.fifo 23 | output pipe 10 cat >> test2.fifo 24 | output pipe 100 cat >> test3.fifo 25 | 26 | 27 | #log.level = 7 28 | -------------------------------------------------------------------------------- /test/test-performance.conf: -------------------------------------------------------------------------------- 1 | # Tests kafkatee throughput performance 2 | # * String input and output: no transformation 3 | # * Read from pipe 4 | # * Write to pipes 5 | 6 | include base.conf 7 | 8 | # Number of messages to send 9 | env.NMSGS=1000000 10 | 11 | 12 | # Output encoding 13 | output.encoding = string 14 | 15 | input [encoding=string,exit.on.exit=true] pipe ./ingen $NMSGS 16 | 17 | 18 | env.FAST_TIMEOUT=20 19 | output pipe 1 ./linecnt $NMSGS $FAST_TIMEOUT 5 20 | output pipe 1 ./linecnt $NMSGS $FAST_TIMEOUT 5 21 | output pipe 1 ./linecnt $NMSGS $FAST_TIMEOUT 5 22 | output pipe 1 ./linecnt $NMSGS $FAST_TIMEOUT 5 23 | output pipe 1 ./linecnt $NMSGS $FAST_TIMEOUT 5 24 | output pipe 1 ./linecnt $NMSGS $FAST_TIMEOUT 5 25 | output pipe 1 ./linecnt $NMSGS $FAST_TIMEOUT 5 26 | output pipe 1 ./linecnt $NMSGS $FAST_TIMEOUT 5 27 | output pipe 100 ./linecnt $(expr $NMSGS / 100) $FAST_TIMEOUT 5 28 | output pipe 100 ./linecnt $(expr $NMSGS / 100) $FAST_TIMEOUT 5 29 | output pipe 100 ./linecnt $(expr $NMSGS / 100) $FAST_TIMEOUT 5 30 | output pipe 100 ./linecnt $(expr $NMSGS / 100) $FAST_TIMEOUT 5 31 | output pipe 100 ./linecnt $(expr $NMSGS / 100) $FAST_TIMEOUT 5 32 | output pipe 100 ./linecnt $(expr $NMSGS / 100) $FAST_TIMEOUT 5 33 | output pipe 100 ./linecnt $(expr $NMSGS / 100) $FAST_TIMEOUT 5 34 | output pipe 100 ./linecnt $(expr $NMSGS / 100) $FAST_TIMEOUT 5 35 | output pipe 100 ./linecnt $(expr $NMSGS / 100) $FAST_TIMEOUT 5 36 | output pipe 100 ./linecnt $(expr $NMSGS / 100) $FAST_TIMEOUT 5 37 | -------------------------------------------------------------------------------- /test/test-single.conf: -------------------------------------------------------------------------------- 1 | # Tests plain message forwarding: 2 | # * one input 3 | # * one output 4 | # * string -> string (no transformation) 5 | # * sample-rate 1 6 | 7 | # Commands to run on init and termination. 8 | # We use them to create and purge FIFOs. 9 | env.NOUTS=1 10 | command.init = for N in `seq 1 $NOUTS`; do rm -f test$N.fifo ; mkfifo test$N.fifo ; done 11 | command.term = for N in `seq 1 $NOUTS`; do rm -f test$N.fifo ; done 12 | 13 | # Output encoding 14 | output.encoding = string 15 | 16 | # Input from testpipe.py script that will 17 | # output to stdout and read from the FIFO 18 | input [encoding=string,exit.on.exit=true] pipe ./testpipe.py --cnt 10 --genenc string --checkenc string --config '{"outputs":[{"fifo":"test1.fifo","rate":1}]}' 19 | 20 | # Output pipe simply writes to FIFO that testpipe.py reads. 21 | output pipe 1 cat >> test1.fifo 22 | 23 | 24 | #log.level = 7 25 | -------------------------------------------------------------------------------- /test/testpipe.py: -------------------------------------------------------------------------------- 1 | #!/usr/bin/python 2 | # 3 | 4 | """ 5 | The testpipe is configured in kafkatee configuration file as an input pipe. 6 | testpipe creates a fifo file which it reads from, and configures kafkatee 7 | to write the output to this fifo. 8 | This way testpipe can generate input for kafkatee as well as check its output, 9 | all in the same process. 10 | 11 | 'input' referred to in this file corresponds to the kafkatee input, 12 | while 'output' refers to the kafkatee output. testpipe writes to the 'input' 13 | and reads from the 'output'. 14 | 15 | The input generator runs in its own thread while the output checker 16 | runs in the main thread. 17 | 18 | The FIFO must have been previously created (by command.init=mkfifo in 19 | the kafkatee configuration file). 20 | 21 | """ 22 | 23 | import argparse 24 | import os 25 | import errno 26 | import time 27 | import threading 28 | import sys 29 | import json 30 | import signal 31 | import uuid 32 | 33 | 34 | conf = {} 35 | 36 | # Sorted list of json obj fields. 37 | json_obj_fields = ('seq','field1','field2','field3','field4') 38 | json_obj = {'seq': 0, 39 | 'field1': 'testid', 40 | 'field2': 'FIELD-TWO', 41 | 'field3': 'FIELD-THREE', 42 | 'field4': 'also seq'} 43 | 44 | test_id = '' 45 | 46 | failures = 0 47 | outputs_started = 0 48 | outputs_good = 0 49 | 50 | def log (str): 51 | sys.stderr.write('\033[33mTEST:%s: %s\033[0m\n' % \ 52 | (threading.current_thread().name, str)) 53 | 54 | def fatal (str): 55 | global failures 56 | failures += 1 57 | raise Exception('\033[31mTEST:%s: FAILED: %s\033[0m' % \ 58 | (threading.current_thread().name, str)) 59 | 60 | 61 | def input_generator_main (): 62 | global test_id 63 | 64 | log('started input generator, will generate seqs %i .. %i' % 65 | (args.seq, args.seq_hi)) 66 | 67 | cnt = 0 68 | for seq in range(args.seq, args.seq_hi): 69 | obj = json_obj 70 | obj['seq'] = seq 71 | obj['field1'] = test_id 72 | obj['field4'] = seq 73 | 74 | if args.genenc == 'json': 75 | print json.dumps(obj) 76 | else: 77 | print '%d,%s,%s,%s,%s' % (obj['seq'], 78 | obj['field1'], 79 | obj['field2'], 80 | obj['field3'], 81 | obj['field4']) 82 | 83 | cnt += 1 84 | if (cnt % args.chunk) == 0: 85 | sys.stdout.flush() 86 | time.sleep(1) 87 | 88 | sys.stdout.flush() 89 | 90 | log('input generator done, %i messages written' % (cnt)) 91 | time.sleep(args.idle) 92 | 93 | 94 | def input_generator_start (): 95 | thr = threading.Thread(None, input_generator_main, 'inputgen') 96 | thr.daemon = True 97 | thr.start() 98 | 99 | 100 | def check_line (o, line): 101 | global last_seq 102 | global json_obj 103 | global test_id 104 | 105 | if args.checkenc == 'json': 106 | obj = json.loads(line) 107 | else: 108 | arr = line.split(',') 109 | obj = {} 110 | i = 0 111 | for f in json_obj_fields: 112 | obj[f] = arr[i] 113 | i += 1 114 | 115 | if not 'seq' in obj: 116 | fatal('No seq in: %s' % line) 117 | 118 | if not 'field1' in obj: 119 | fatal('Missing field1 (test_id) in: %s' % line) 120 | 121 | # Ignore messages from wrong test ids 122 | if str(obj['field1']) != test_id: 123 | log('Ignoring message for wrong test_id %s, wanted %s' % 124 | (str(obj['field1']), test_id)) 125 | return 126 | 127 | # Check correct sequence number 128 | try: 129 | seq = int(obj['seq']) 130 | except ValueError: 131 | fatal('seq %s is not an int' % obj['seq']) 132 | 133 | # First time we see a seq, set up where we expect this to end. 134 | if o['last_seq'] == 0: 135 | o['seq_hi'] = seq + (((args.cnt / o['rate'])-seq)*o['rate']) 136 | log('first seq seen %d, end seq will be %d, rate %d (test_id %s)' % \ 137 | (seq, o['seq_hi'], o['rate'], obj['field1'])) 138 | 139 | if o['last_seq'] > 0 and o['last_seq'] + int(o['rate']) != seq: 140 | fatal('Received seq %d, last received seq %d, expected %d' % \ 141 | (seq, o['last_seq'], o['last_seq'] + int(o['rate']))) 142 | o['last_seq'] = seq 143 | 144 | if obj['seq'] != obj['field4']: 145 | fatal('"seq" and "field4" differs, should be same: %s' % obj) 146 | 147 | # Verify received content 148 | for f in json_obj: 149 | if f not in obj: 150 | fatal('Expected field %s not received: %s' % \ 151 | (f, obj)) 152 | 153 | if f in ('seq', 'field4'): 154 | continue 155 | 156 | if str(json_obj[f]) != str(obj[f]): 157 | fatal('Expected field %s value "%s", ' 158 | 'but got "%s" in message: %s' % \ 159 | (f, json_obj[cf], str(obj[f]), obj)) 160 | 161 | if o['last_seq'] == o['seq_hi']: 162 | log('All seqs seen. We are now good') 163 | return 1 164 | elif o['last_seq'] > o['seq_hi']: 165 | fatal('Overshoot: seq %(last_seq)i > hi %(seq_h)i' % o) 166 | 167 | return 0 168 | 169 | 170 | """ 171 | Reads from the created fifo and checks that the output matches what 172 | is expected in terms of encoding, sample rate, order, etc. 173 | 'o' is the output configuration. 174 | """ 175 | def output_checker_main (o): 176 | global outputs_started 177 | outputs_started += 1 178 | 179 | log('opening output checker on ' + o['fifo']) 180 | 181 | f = open(o['fifo'], 'r') 182 | 183 | log('output checker opened fifo %s, sample-rate %d' % \ 184 | (o['fifo'], o['rate'])) 185 | 186 | o['last_seq'] = 0; 187 | 188 | line = f.readline() 189 | cnt = 0 190 | while line: 191 | line = line.rstrip() 192 | cnt += 1 193 | if (cnt % 1000) == 0: 194 | log('%i messages read' % cnt) 195 | #log('READ LINE: ' + o['fifo'] + ': ' + line + ';') 196 | if check_line(o, line) == 1: 197 | break 198 | line = f.readline() 199 | 200 | f.close() 201 | os.unlink(o['fifo']) 202 | 203 | global outputs_good 204 | outputs_good += 1 205 | return 206 | 207 | 208 | def output_checker_start (o): 209 | thr = threading.Thread(None, output_checker_main, 'output ' + o['fifo'], 210 | (o,)) 211 | thr.daemon = False 212 | thr.start() 213 | return thr 214 | 215 | 216 | def sig_timeout (sig, frame): 217 | log('\033[31mFATAL: Test timed out\033[0m') 218 | os.kill(os.getpid(), 9) 219 | # NOTREACHED 220 | fatal("Test timed out") 221 | 222 | 223 | if __name__ == '__main__': 224 | parser = argparse.ArgumentParser(description='kafkatee test tool') 225 | parser.add_argument('--cnt', default=1, type=int, help="Message count") 226 | parser.add_argument('--genenc', default='string', type=str, 227 | help="Input generator encoding") 228 | parser.add_argument('--checkenc', default='string', type=str, 229 | help="Output checker encoding") 230 | parser.add_argument('--seq', default=1, type=int, 231 | help='Start sequence number') 232 | parser.add_argument('--idle', default=0, type=int, 233 | help='Idle time after finishing write (-i)') 234 | parser.add_argument('--chunk', default=1000, type=int, 235 | help='Write chunk size expressed in messages (-i)') 236 | parser.add_argument('--config', required=True, type=str, 237 | help="Configuration in JSON format") 238 | parser.add_argument('--timeout', default=30, type=int, 239 | help='Test timeout') 240 | 241 | # Parse command line arguments 242 | global args 243 | args = parser.parse_args() 244 | conf = json.loads(args.config) 245 | 246 | if 'outputs' not in conf or len(conf['outputs']) == 0: 247 | fatal('Invalid configuration JSON object: missing outputs') 248 | 249 | args.seq_hi = args.seq + args.cnt 250 | 251 | # Generate unique test_id 252 | test_id = uuid.uuid1().hex[0:7] 253 | log('Running test id %s with timeout %ds' % (test_id, args.timeout)) 254 | 255 | # Install test timeout alarm 256 | signal.signal(signal.SIGALRM, sig_timeout) 257 | signal.alarm(args.timeout) 258 | 259 | 260 | # Start input generator thread 261 | input_generator_start() 262 | 263 | # Start output checker threads 264 | for o in conf['outputs']: 265 | o['thread'] = output_checker_start(o) 266 | 267 | # Monitor output checker threads and wait for them to finish. 268 | while True: 269 | time.sleep(0.1) 270 | cnt = 0 271 | for o in conf['outputs']: 272 | if o['thread'].is_alive(): 273 | cnt += 1 274 | o['thread'].join(0.1) 275 | # If the thread was joined we'll see it on the next run 276 | 277 | if cnt == 0: 278 | break 279 | 280 | if failures > 0: 281 | fatal("%i test failures" % failures) 282 | if outputs_good != outputs_started: 283 | fatal("%i/%i output checkers failed" % (outputs_started-outputs_good, 284 | outputs_started)) 285 | else: 286 | log('all %i output checkers finished' % outputs_started) 287 | sys.exit(0) 288 | 289 | --------------------------------------------------------------------------------