forked from splunk/kafka-connect-splunk
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathSplunkSinkTask.java
More file actions
594 lines (514 loc) · 22.6 KB
/
SplunkSinkTask.java
File metadata and controls
594 lines (514 loc) · 22.6 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
/*
* Copyright 2017 Splunk, Inc..
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.splunk.kafka.connect;
import com.splunk.hecclient.*;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.lang3.SystemUtils;
import org.apache.kafka.clients.admin.Config;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.connect.errors.RetriableException;
import org.apache.kafka.connect.header.Headers;
import org.apache.kafka.connect.sink.SinkRecord;
import org.apache.kafka.connect.sink.SinkTask;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.kafka.connect.header.Header;
import java.io.IOException;
import java.net.InetAddress;
import java.net.UnknownHostException;
import java.text.ParseException;
import java.text.SimpleDateFormat;
import java.util.*;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public final class SplunkSinkTask extends SinkTask implements PollerCallback {
private static final Logger log = LoggerFactory.getLogger(SplunkSinkTask.class);
private static long flushWindow = 30 * 1000; // 30 seconds
private static final String HEADERTOKEN = "$$$";
private HecInf hec;
private KafkaRecordTracker tracker;
private SplunkSinkConnectorConfig connectorConfig;
private List<SinkRecord> bufferedRecords;
private long lastFlushed = System.currentTimeMillis();
private long threadId = Thread.currentThread().getId();
private static final String HOSTNAME;
static {
String h = null;
try {
h = InetAddress.getLocalHost().getHostName();
} catch (UnknownHostException e) {
}
HOSTNAME = h;
}
@Override
public void start(Map<String, String> taskConfig) {
connectorConfig = new SplunkSinkConnectorConfig(taskConfig);
if (hec == null) {
hec = createHec();
}
tracker = new KafkaRecordTracker();
bufferedRecords = new ArrayList<>();
if(connectorConfig.flushWindow > 0) {
flushWindow = connectorConfig.flushWindow * 1000; // Flush window set to user configured value (Multiply by 1000 as all the calculations are done in milliseconds)
}
log.info("kafka-connect-splunk task starts with config={}", connectorConfig);
}
@Override
public void put(Collection<SinkRecord> records) {
long startTime = System.currentTimeMillis();
log.debug("tid={} received {} records with total {} outstanding events tracked", threadId, records.size(), tracker.totalEvents());
handleFailedBatches();
preventTooManyOutstandingEvents();
bufferedRecords.addAll(records);
if (bufferedRecords.size() < connectorConfig.maxBatchSize) {
if (System.currentTimeMillis() - lastFlushed < flushWindow) {
logDuration(startTime);
// still in flush window, buffer the records and return
return;
}
if (bufferedRecords.isEmpty()) {
lastFlushed = System.currentTimeMillis();
logDuration(startTime);
return;
}
}
// either flush window reached or max batch size reached
records = bufferedRecords;
bufferedRecords = new ArrayList<>();
lastFlushed = System.currentTimeMillis();
if (connectorConfig.raw) {
/* /raw endpoint */
handleRaw(records);
} else {
/* /event endpoint */
handleEvent(records);
}
logDuration(startTime);
}
private void logDuration(long startTime) {
long endTime = System.currentTimeMillis();
log.debug("tid={} cost={} ms", threadId, endTime - startTime);
}
// for testing hook
SplunkSinkTask setHec(final HecInf hec) {
this.hec = hec;
return this;
}
KafkaRecordTracker getTracker() {
return tracker;
}
private void handleFailedBatches() {
Collection<EventBatch> failed = tracker.getAndRemoveFailedRecords();
if (failed.isEmpty()) {
return;
}
log.debug("handling {} failed batches", failed.size());
long failedEvents = 0;
// if there are failed ones, first deal with them
for (final EventBatch batch: failed) {
failedEvents += batch.size();
if (connectorConfig.maxRetries > 0 && batch.getFailureCount() > connectorConfig.maxRetries) {
log.error("dropping EventBatch {} with {} events after reaching maximum retries {}",
batch.getUUID(), batch.size(), connectorConfig.maxRetries);
continue;
}
log.warn("attempting to resend batch {} with {} events, this is attempt {} out of {} for this batch ",
batch.getUUID(), batch.size(), batch.getFailureCount(), connectorConfig.maxRetries);
send(batch);
}
log.info("handled {} failed batches with {} events", failed.size(), failedEvents);
if (failedEvents * 10 > connectorConfig.maxOutstandingEvents) {
String msg = String.format("failed events have reached 10 %% of max outstanding events %d, pausing the pull of events for a while", connectorConfig.maxOutstandingEvents);
throw new RetriableException(new HecException(msg));
}
}
private void preventTooManyOutstandingEvents() {
if (tracker.totalEvents() >= connectorConfig.maxOutstandingEvents) {
String msg = String.format("max outstanding events %d have reached, pause the pull for a while", connectorConfig.maxOutstandingEvents);
throw new RetriableException(new HecException(msg));
}
}
private void handleRaw(final Collection<SinkRecord> records) {
if(connectorConfig.headerSupport) {
if(records != null) { handleRecordsWithHeader(records); }
} else if (connectorConfig.hasMetaDataConfigured()) {
// when setup metadata - index, source, sourcetype, we need partition records for /raw
Map<TopicPartition, Collection<SinkRecord>> partitionedRecords = partitionRecords(records);
for (Map.Entry<TopicPartition, Collection<SinkRecord>> entry: partitionedRecords.entrySet()) {
EventBatch batch = createRawEventBatch(entry.getKey());
batch.setEnableCompression(connectorConfig.enableCompression);
sendEvents(entry.getValue(), batch);
}
} else {
EventBatch batch = createRawEventBatch(null);
batch.setEnableCompression(connectorConfig.enableCompression);
sendEvents(records, batch);
}
}
private void handleRecordsWithHeader(final Collection<SinkRecord> records) {
HashMap<String, ArrayList<SinkRecord>> recordsWithSameHeaders = new HashMap<>();
for (SinkRecord record : records) {
String key = headerId(record);
if (!recordsWithSameHeaders.containsKey(key)) {
ArrayList<SinkRecord> recordList = new ArrayList<SinkRecord>();
recordsWithSameHeaders.put(key, recordList);
}
ArrayList<SinkRecord> recordList = recordsWithSameHeaders.get(key);
recordList.add(record);
}
Iterator<Map.Entry<String, ArrayList<SinkRecord>>> itr = recordsWithSameHeaders.entrySet().iterator();
while(itr.hasNext()) {
Map.Entry<String, ArrayList<SinkRecord>> set = itr.next();
String splunkSinkRecordKey = set.getKey();
ArrayList<SinkRecord> recordArrayList = set.getValue();
EventBatch batch = createRawHeaderEventBatch(splunkSinkRecordKey);
batch.setEnableCompression(connectorConfig.enableCompression);
sendEvents(recordArrayList, batch);
}
log.debug("{} records have been bucketed in to {} batches", records.size(), recordsWithSameHeaders.size());
}
public String headerId(SinkRecord sinkRecord) {
Headers headers = sinkRecord.headers();
Header indexHeader = headers.lastWithName(connectorConfig.headerIndex);
Header hostHeader = headers.lastWithName(connectorConfig.headerHost);
Header sourceHeader = headers.lastWithName(connectorConfig.headerSource);
Header sourcetypeHeader = headers.lastWithName(connectorConfig.headerSourcetype);
Map<String, String> metas = connectorConfig.topicMetas.get(sinkRecord.topic());
StringBuilder headerString = new StringBuilder();
if(indexHeader != null) {
headerString.append(indexHeader.value().toString());
} else {
if(metas != null) {
headerString.append(metas.get("index"));
}
}
headerString.append(insertHeaderToken());
if(hostHeader != null) {
headerString.append(hostHeader.value().toString());
} else {
if(metas != null) {
headerString.append("default-host");
}
}
headerString.append(insertHeaderToken());
if(sourceHeader != null) {
headerString.append(sourceHeader.value().toString());
} else {
if(metas != null) {
headerString.append(metas.get("source"));
}
}
headerString.append(insertHeaderToken());
if(sourcetypeHeader != null) {
headerString.append(sourcetypeHeader.value().toString());
} else {
if(metas != null) {
headerString.append(metas.get("sourcetype"));
}
}
headerString.append(insertHeaderToken());
return headerString.toString();
}
public String insertHeaderToken() {
return HEADERTOKEN;
}
private void handleEvent(final Collection<SinkRecord> records) {
EventBatch batch = new JsonEventBatch();
batch.setEnableCompression(connectorConfig.enableCompression);
sendEvents(records, batch);
}
private void sendEvents(final Collection<SinkRecord> records, EventBatch batch) {
for (final SinkRecord record: records) {
Event event;
try {
event = createHecEventFrom(record);
} catch (HecEmptyEventException | HecNullEventException ex) {
log.warn("Ignoring Null/Empty event for topicPartitionOffset=({}, {}, {})",
record.topic(), record.kafkaPartition(), record.kafkaOffset(), ex);
continue;
} catch (HecException ex) {
log.error("ignore malformed event for topicPartitionOffset=({}, {}, {})",
record.topic(), record.kafkaPartition(), record.kafkaOffset(), ex);
event = createHecEventFromMalformed(record);
}
batch.add(event);
if (batch.size() >= connectorConfig.maxBatchSize) {
send(batch);
// start a new batch after send
batch = batch.createFromThis();
batch.setEnableCompression(connectorConfig.enableCompression);
}
}
// Last batch
if (!batch.isEmpty()) {
send(batch);
}
}
private void send(final EventBatch batch) {
batch.resetSendTimestamp();
tracker.addEventBatch(batch);
try {
hec.send(batch);
} catch (Exception ex) {
batch.fail();
onEventFailure(Arrays.asList(batch), ex);
log.error(String.format("failed to send batch %s", batch.getUUID()), ex);
}
}
private EventBatch createRawHeaderEventBatch(String splunkSinkRecord) {
String[] split = splunkSinkRecord.split("[$]{3}", -1);
return RawEventBatch.factory()
.setIndex(split[0])
.setHost(split[1])
.setSource(split[2])
.setSourcetype(split[3])
.build();
}
// setup metadata on RawEventBatch
private EventBatch createRawEventBatch(final TopicPartition tp) {
if (tp == null) {
return RawEventBatch.factory().build();
}
Map<String, String> metas = connectorConfig.topicMetas.get(tp.topic());
if (metas == null || metas.isEmpty()) {
return RawEventBatch.factory().build();
}
return RawEventBatch.factory()
.setIndex(metas.get(SplunkSinkConnectorConfig.INDEX))
.setSourcetype(metas.get(SplunkSinkConnectorConfig.SOURCETYPE))
.setSource(metas.get(SplunkSinkConnectorConfig.SOURCE))
.build();
}
@Override
public void open(Collection<TopicPartition> partitions) {
tracker.open(partitions);
}
@Override
public void close(Collection<TopicPartition> partitions) {
/* Purge buffered events tied to closed partitions because this task
* won't be able to commit their offsets. */
bufferedRecords.removeIf(r -> partitions.contains(
new TopicPartition(r.topic(), r.kafkaPartition())));
/* Tell tracker about now closed partitions so to clean up. */
tracker.close(partitions);
}
@Override
public Map<TopicPartition, OffsetAndMetadata> preCommit(Map<TopicPartition, OffsetAndMetadata> meta) {
// tell Kafka Connect framework what are offsets we can safely commit to Kafka now
Map<TopicPartition, OffsetAndMetadata> offsets = tracker.computeOffsets();
log.debug("commits offsets offered={}, pushed={}", offsets, meta);
return offsets;
}
@Override
public void stop() {
if (hec != null) {
hec.close();
}
log.info("kafka-connect-splunk task ends with config={}", connectorConfig);
}
@Override
public String version() {
return VersionUtils.getVersionString();
}
public void onEventCommitted(final List<EventBatch> batches) {
tracker.removeAckedEventBatches(batches);
}
public void onEventFailure(final List<EventBatch> batches, Exception ex) {
log.info("add {} failed batches", batches.size());
for (EventBatch batch: batches) {
tracker.addFailedEventBatch(batch);
}
}
private Event createHecEventFrom(final SinkRecord record) {
if (connectorConfig.raw) {
RawEvent event = new RawEvent(record.value(), record);
event.setLineBreaker(connectorConfig.lineBreaker);
if(connectorConfig.headerSupport) {
event = (RawEvent)addHeaders(event, record);
}
return event;
}
JsonEvent event;
ObjectMapper objectMapper = new ObjectMapper();
if(connectorConfig.hecEventFormatted) {
try {
event = objectMapper.readValue(record.value().toString(), JsonEvent.class);
event.setTied(record);
event.addFields(connectorConfig.enrichments);
} catch(Exception e) {
log.error("event does not follow correct HEC pre-formatted format: {}", record.value().toString());
event = createHECEventNonFormatted(record);
}
} else {
event = createHECEventNonFormatted(record);
}
if(connectorConfig.headerSupport) {
addHeaders(event, record);
}
if (connectorConfig.trackData) {
Map<String, String> trackMetas = new HashMap<>();
trackMetas.put("kafka_offset", String.valueOf(record.kafkaOffset()));
trackMetas.put("kafka_timestamp", String.valueOf(record.timestamp()));
trackMetas.put("kafka_topic", record.topic());
trackMetas.put("kafka_partition", String.valueOf(record.kafkaPartition()));
trackMetas.put("kafka_record_key", String.valueOf(record.key()));
if (HOSTNAME != null)
trackMetas.put("kafka_connect_host", HOSTNAME);
event.addFields(trackMetas);
}
event.validate();
return event;
}
private Event addHeaders(Event event, SinkRecord record) {
Headers headers = record.headers();
if(headers.isEmpty() && connectorConfig.headerCustom.isEmpty()) {
return event;
}
Header headerIndex = headers.lastWithName(connectorConfig.headerIndex);
Header headerHost = headers.lastWithName(connectorConfig.headerHost);
Header headerSource = headers.lastWithName(connectorConfig.headerSource);
Header headerSourcetype = headers.lastWithName(connectorConfig.headerSourcetype);
if (headerIndex != null) {
event.setIndex(headerIndex.value().toString());
}
if (headerHost != null) {
event.setHost(headerHost.value().toString());
}
if (headerSource != null) {
event.setSource(headerSource.value().toString());
}
if (headerSourcetype != null) {
event.setSourcetype(headerSourcetype.value().toString());
}
// Custom headers are configured with a comma separated list passed in configuration
// "custom_header_1,custom_header_2,custom_header_3"
if (!connectorConfig.headerCustom.isEmpty()) {
String[] customHeaders = connectorConfig.headerCustom.split(",\\s?");
Map<String, String> headerMap = new HashMap<>();
for (String header : customHeaders) {
Header customHeader = headers.lastWithName(header);
if (customHeader != null) {
if (customHeader.value() == null) {
headerMap.put(header, null);
} else {
headerMap.put(header, customHeader.value().toString());
}
}
}
event.addFields(headerMap);
}
return event;
}
private JsonEvent createHECEventNonFormatted(final SinkRecord record) {
JsonEvent event = new JsonEvent(record.value(), record);
if (connectorConfig.useRecordTimestamp && record.timestamp() != null) {
event.setTime(record.timestamp() / 1000.0); // record timestamp is in milliseconds
}
if(connectorConfig.enableTimestampExtraction) {
timestampExtraction(event);
}
Map<String, String> metas = connectorConfig.topicMetas.get(record.topic());
if (metas != null) {
event.setIndex(metas.get(SplunkSinkConnectorConfig.INDEX));
event.setSourcetype(metas.get(SplunkSinkConnectorConfig.SOURCETYPE));
event.setSource(metas.get(SplunkSinkConnectorConfig.SOURCE));
event.addFields(connectorConfig.enrichments);
}
return event;
}
private Event createHecEventFromMalformed(final SinkRecord record) {
Object data;
if (connectorConfig.raw) {
data = "timestamp=" + record.timestamp() + ", topic='" + record.topic() + '\'' +
", partition=" + record.kafkaPartition() +
", offset=" + record.kafkaOffset() + ", type=malformed";
} else {
Map<String, Object> v = new HashMap<>();
v.put("timestamp", record.timestamp());
v.put("topic", record.topic());
v.put("partition", record.kafkaPartition());
v.put("offset", record.kafkaOffset());
v.put("type", "malformed");
data = v;
}
final SinkRecord r = record.newRecord("malformed", 0, null, null, null, data, record.timestamp());
return createHecEventFrom(r);
}
// partition records according to topic-partition key
private Map<TopicPartition, Collection<SinkRecord>> partitionRecords(Collection<SinkRecord> records) {
Map<TopicPartition, Collection<SinkRecord>> partitionedRecords = new HashMap<>();
for (SinkRecord record: records) {
TopicPartition key = new TopicPartition(record.topic(), record.kafkaPartition());
Collection<SinkRecord> partitioned = partitionedRecords.get(key);
if (partitioned == null) {
partitioned = new ArrayList<>();
partitionedRecords.put(key, partitioned);
}
partitioned.add(record);
}
return partitionedRecords;
}
private HecInf createHec() {
if (connectorConfig.numberOfThreads > 1) {
return new ConcurrentHec(connectorConfig.numberOfThreads, connectorConfig.ack,
connectorConfig.getHecConfig(), this);
} else {
if (connectorConfig.ack) {
return Hec.newHecWithAck(connectorConfig.getHecConfig(), this);
} else {
return Hec.newHecWithoutAck(connectorConfig.getHecConfig(), this);
}
}
}
private void timestampExtraction(Event event) {
String jsonStr = event.getEvent().toString();
String string = jsonStr.replaceAll("\\\"", "\"");
String timestamp = "";
final Pattern pattern = Pattern.compile(connectorConfig.regex);
final Matcher matcher = pattern.matcher(string);
try {
if (matcher.find()) {
timestamp = (matcher.group("time"));
}
} catch (Exception e) {
log.warn("Couldn't extract timestamp", e);
return;
}
if (connectorConfig.timestampFormat.equalsIgnoreCase("epoch")) {
try {
double epoch;
epoch = ((Double.parseDouble(timestamp)));
long long_epoch = (new Double(epoch)).longValue();
event.setTime(epoch / (Math.pow(10, Long.toString(long_epoch).length()-10)));
} catch (Exception e) {
log.warn("Could not set the time", e);
}
} else {
SimpleDateFormat df = new SimpleDateFormat(connectorConfig.timestampFormat);
Date date;
try {
if(!connectorConfig.timeZone.isEmpty())
df.setTimeZone(TimeZone.getTimeZone(connectorConfig.timeZone));
date = df.parse(timestamp);
event.setTime(date.getTime() / 1000.0);
} catch (ParseException e) {
log.warn("Couldn't parse the timestamp", e);
}
}
}
}