1/*
2 * librdkafka - The Apache Kafka C/C++ library
3 *
4 * Copyright (c) 2015 Magnus Edenhill
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 "rdkafka_int.h"
30#include "rdkafka_pattern.h"
31
32void rd_kafka_pattern_destroy (rd_kafka_pattern_list_t *plist,
33 rd_kafka_pattern_t *rkpat) {
34 TAILQ_REMOVE(&plist->rkpl_head, rkpat, rkpat_link);
35 rd_regex_destroy(rkpat->rkpat_re);
36 rd_free(rkpat->rkpat_orig);
37 rd_free(rkpat);
38}
39
40void rd_kafka_pattern_add (rd_kafka_pattern_list_t *plist,
41 rd_kafka_pattern_t *rkpat) {
42 TAILQ_INSERT_TAIL(&plist->rkpl_head, rkpat, rkpat_link);
43}
44
45rd_kafka_pattern_t *rd_kafka_pattern_new (const char *pattern,
46 char *errstr, int errstr_size) {
47 rd_kafka_pattern_t *rkpat;
48
49 rkpat = rd_calloc(1, sizeof(*rkpat));
50
51 /* Verify and precompile pattern */
52 if (!(rkpat->rkpat_re = rd_regex_comp(pattern, errstr, errstr_size))) {
53 rd_free(rkpat);
54 return NULL;
55 }
56
57 rkpat->rkpat_orig = rd_strdup(pattern);
58
59 return rkpat;
60}
61
62
63
64int rd_kafka_pattern_match (rd_kafka_pattern_list_t *plist, const char *str) {
65 rd_kafka_pattern_t *rkpat;
66
67 TAILQ_FOREACH(rkpat, &plist->rkpl_head, rkpat_link) {
68 if (rd_regex_exec(rkpat->rkpat_re, str))
69 return 1;
70 }
71
72 return 0;
73}
74
75
76/**
77 * Append pattern to list.
78 */
79int rd_kafka_pattern_list_append (rd_kafka_pattern_list_t *plist,
80 const char *pattern,
81 char *errstr, int errstr_size) {
82 rd_kafka_pattern_t *rkpat;
83 rkpat = rd_kafka_pattern_new(pattern, errstr, errstr_size);
84 if (!rkpat)
85 return -1;
86
87 rd_kafka_pattern_add(plist, rkpat);
88 return 0;
89}
90
91/**
92 * Remove matching patterns.
93 * Returns the number of removed patterns.
94 */
95int rd_kafka_pattern_list_remove (rd_kafka_pattern_list_t *plist,
96 const char *pattern) {
97 rd_kafka_pattern_t *rkpat, *rkpat_tmp;
98 int cnt = 0;
99
100 TAILQ_FOREACH_SAFE(rkpat, &plist->rkpl_head, rkpat_link, rkpat_tmp) {
101 if (!strcmp(rkpat->rkpat_orig, pattern)) {
102 rd_kafka_pattern_destroy(plist, rkpat);
103 cnt++;
104 }
105 }
106 return cnt;
107}
108
109/**
110 * Parse a patternlist and populate a list with it.
111 */
112static int rd_kafka_pattern_list_parse (rd_kafka_pattern_list_t *plist,
113 const char *patternlist,
114 char *errstr, size_t errstr_size) {
115 char *s;
116 rd_strdupa(&s, patternlist);
117
118 while (s && *s) {
119 char *t = s;
120 char re_errstr[256];
121
122 /* Find separator */
123 while ((t = strchr(t, ','))) {
124 if (t > s && *(t-1) == ',') {
125 /* separator was escaped,
126 remove escape and scan again. */
127 memmove(t-1, t, strlen(t)+1);
128 t++;
129 } else {
130 *t = '\0';
131 t++;
132 break;
133 }
134 }
135
136 if (rd_kafka_pattern_list_append(plist, s, re_errstr,
137 sizeof(re_errstr)) == -1) {
138 rd_snprintf(errstr, errstr_size,
139 "Failed to parse pattern \"%s\": "
140 "%s", s, re_errstr);
141 rd_kafka_pattern_list_clear(plist);
142 return -1;
143 }
144
145 s = t;
146 }
147
148 return 0;
149}
150
151
152/**
153 * Clear a pattern list.
154 */
155void rd_kafka_pattern_list_clear (rd_kafka_pattern_list_t *plist) {
156 rd_kafka_pattern_t *rkpat;
157
158 while ((rkpat = TAILQ_FIRST(&plist->rkpl_head)))
159 rd_kafka_pattern_destroy(plist, rkpat);
160
161 if (plist->rkpl_orig) {
162 rd_free(plist->rkpl_orig);
163 plist->rkpl_orig = NULL;
164 }
165}
166
167
168/**
169 * Free a pattern list previously created with list_new()
170 */
171void rd_kafka_pattern_list_destroy (rd_kafka_pattern_list_t *plist) {
172 rd_kafka_pattern_list_clear(plist);
173 rd_free(plist);
174}
175
176/**
177 * Initialize a pattern list, optionally populating it with the
178 * comma-separated patterns in 'patternlist'.
179 */
180int rd_kafka_pattern_list_init (rd_kafka_pattern_list_t *plist,
181 const char *patternlist,
182 char *errstr, size_t errstr_size) {
183 TAILQ_INIT(&plist->rkpl_head);
184 if (patternlist) {
185 if (rd_kafka_pattern_list_parse(plist, patternlist,
186 errstr, errstr_size) == -1)
187 return -1;
188 plist->rkpl_orig = rd_strdup(patternlist);
189 } else
190 plist->rkpl_orig = NULL;
191
192 return 0;
193}
194
195
196/**
197 * Allocate and initialize a new list.
198 */
199rd_kafka_pattern_list_t *rd_kafka_pattern_list_new (const char *patternlist,
200 char *errstr,
201 int errstr_size) {
202 rd_kafka_pattern_list_t *plist;
203
204 plist = rd_calloc(1, sizeof(*plist));
205
206 if (rd_kafka_pattern_list_init(plist, patternlist,
207 errstr, errstr_size) == -1) {
208 rd_free(plist);
209 return NULL;
210 }
211
212 return plist;
213}
214
215
216/**
217 * Make a copy of a pattern list.
218 */
219rd_kafka_pattern_list_t *
220rd_kafka_pattern_list_copy (rd_kafka_pattern_list_t *src) {
221 char errstr[16];
222 return rd_kafka_pattern_list_new(src->rkpl_orig,
223 errstr, sizeof(errstr));
224}
225