Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Adding Kafka-emitter #3860

Merged
merged 28 commits into from
Apr 4, 2017
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
28 commits
Select commit Hold shift + click to select a range
5dd63e2
Initial commit
dkhwangbo Jan 10, 2017
a303a1f
Apply another config: clustername
dkhwangbo Jan 19, 2017
e911b35
Rename variable
dkhwangbo Jan 20, 2017
18126b9
Fix bug
dkhwangbo Jan 20, 2017
5ddbdb7
Add retry logic
dkhwangbo Feb 2, 2017
5e634d3
Edit retry logic
dkhwangbo Feb 4, 2017
efc938d
Upgrade kafka-clients version to the most recent release
dkhwangbo Feb 4, 2017
86fcb78
Make callback single object
dkhwangbo Feb 4, 2017
887b293
Write documentation
dkhwangbo Feb 4, 2017
e6fa868
Rewrite error message and emit logic
dkhwangbo Feb 4, 2017
80ccfb8
Handling AlertEvent
dkhwangbo Feb 6, 2017
4bf0a69
Override toString()
dkhwangbo Feb 7, 2017
e635bbf
make clusterName more optional
dkhwangbo Feb 7, 2017
01800f2
bump up druid version
dkhwangbo Feb 7, 2017
ea12d92
add producer.config option which make user can apply another optional…
dkhwangbo Feb 8, 2017
8d1e80c
remove potential blocking in emit()
dkhwangbo Feb 12, 2017
2d9178c
using MemoryBoundLinkedBlockingQueue
dkhwangbo Feb 19, 2017
e4fb6eb
Fixing coding convention
dkhwangbo Feb 19, 2017
4d316ae
Remove logging every exception and just increment counting
dkhwangbo Feb 27, 2017
c7e8fce
refactoring
dkhwangbo Feb 27, 2017
e1706f7
trivial modification
dkhwangbo Feb 27, 2017
6b383ea
logging when callback has exception
dkhwangbo Mar 1, 2017
1408124
Replace kafka-clients 0.10.1.1 with 0.10.2.0
dkhwangbo Mar 1, 2017
d2058d5
Resolve the problem related of classloader
dkhwangbo Mar 16, 2017
ecb5c69
adopt try statement
dkhwangbo Mar 17, 2017
3a4ca9f
code reformatting
dkhwangbo Mar 27, 2017
8be4c4c
make variables final
dkhwangbo Mar 27, 2017
f02218a
rewrite toString
dkhwangbo Mar 27, 2017
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
35 changes: 35 additions & 0 deletions docs/content/development/extensions-contrib/kafka-emitter.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,35 @@
---
layout: doc_page
---

# Kafka Emitter

To use this extension, make sure to [include](../../operations/including-extensions.html) `kafka-emitter` extension.

## Introduction

This extension emits Druid metrics to a [Kafka](https://kafka.apache.org) directly with JSON format.<br>
Currently, Kafka has not only their nice ecosystem but also consumer API readily available.
So, If you currently use Kafka, It's easy to integrate various tool or UI
to monitor the status of your Druid cluster with this extension.

## Configuration

All the configuration parameters for the Kafka emitter are under `druid.emitter.kafka`.

|property|description|required?|default|
|--------|-----------|---------|-------|
|`druid.emitter.kafka.bootstrap.servers`|Comma-separated Kafka broker. (`[hostname:port],[hostname:port]...`)|yes|none|
|`druid.emitter.kafka.metric.topic`|Kafka topic name for emitter's target to emit service metric.|yes|none|
|`druid.emitter.kafka.alert.topic`|Kafka topic name for emitter's target to emit alert.|yes|none|
|`druid.emitter.kafka.producer.config`|JSON formatted configuration which user want to set additional properties to Kafka producer.|no|none|
|`druid.emitter.kafka.clusterName`|Optional value to specify name of your druid cluster. It can help make groups in your monitoring environment. |no|none|

### Example

```
druid.emitter.kafka.bootstrap.servers=hostname1:9092,hostname2:9092
druid.emitter.kafka.metric.topic=druid-metric
druid.emitter.kafka.alert.topic=druid-alert
druid.emitter.kafka.producer.config={"max.block.ms":10000}
```
1 change: 1 addition & 0 deletions docs/content/development/extensions.md
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,7 @@ All of these community extensions can be downloaded using *pull-deps* with the c
|sqlserver-metadata-storage|Microsoft SqlServer deep storage.|[link](../development/extensions-contrib/sqlserver.html)|
|graphite-emitter|Graphite metrics emitter|[link](../development/extensions-contrib/graphite.html)|
|statsd-emitter|StatsD metrics emitter|[link](../development/extensions-contrib/statsd.html)|
|kafka-emitter|Kafka metrics emitter|[link](../development/extensions-contrib/kafka-emitter.html)|
|druid-thrift-extensions|Support thrift ingestion |[link](../development/extensions-contrib/thrift.html)|
|scan-query|Scan query|[link](../development/extensions-contrib/scan-query.html)|

Expand Down
91 changes: 91 additions & 0 deletions extensions-contrib/kafka-emitter/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,91 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
~ Licensed to Metamarkets Group Inc. (Metamarkets) under one
~ or more contributor license agreements. See the NOTICE file
~ distributed with this work for additional information
~ regarding copyright ownership. Metamarkets licenses this file
~ to you 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.
-->

<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>

<parent>
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<version>0.10.0-SNAPSHOT</version>
<relativePath>../../pom.xml</relativePath>
</parent>

<groupId>io.druid.extensions.contrib</groupId>
<artifactId>kafka-emitter</artifactId>
<name>kafka-emitter</name>
<description>Druid emitter extension to support kafka</description>

<dependencies>
<dependency>
<groupId>org.apache.kafka</groupId>
<artifactId>kafka-clients</artifactId>
<version>0.10.2.0</version>
</dependency>
<dependency>
<groupId>io.druid</groupId>
<artifactId>druid-common</artifactId>
<version>${project.parent.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>io.druid</groupId>
<artifactId>druid-api</artifactId>
<version>${project.parent.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>com.metamx</groupId>
<artifactId>emitter</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.easymock</groupId>
<artifactId>easymock</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>pl.pragmatists</groupId>
<artifactId>JUnitParams</artifactId>
<version>1.0.4</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>io.druid</groupId>
<artifactId>druid-server</artifactId>
<version>${project.parent.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>io.druid</groupId>
<artifactId>druid-processing</artifactId>
<version>${project.parent.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
</dependencies>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,199 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you 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 io.druid.emitter.kafka;

import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap;
import com.metamx.emitter.core.Emitter;
import com.metamx.emitter.core.Event;
import com.metamx.emitter.service.AlertEvent;
import com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.emitter.kafka.MemoryBoundLinkedBlockingQueue.ObjectContainer;
import io.druid.java.util.common.lifecycle.LifecycleStart;
import io.druid.java.util.common.lifecycle.LifecycleStop;
import io.druid.java.util.common.logger.Logger;
import org.apache.kafka.clients.producer.Callback;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.serialization.StringSerializer;

import java.io.IOException;
import java.util.Map;
import java.util.Properties;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;

public class KafkaEmitter implements Emitter
{
private static Logger log = new Logger(KafkaEmitter.class);

private final static int DEFAULT_RETRIES = 3;
private final AtomicLong metricLost;
private final AtomicLong alertLost;
private final AtomicLong invalidLost;

private final KafkaEmitterConfig config;
private final Producer<String, String> producer;
private final Callback producerCallback;
private final ObjectMapper jsonMapper;
private final MemoryBoundLinkedBlockingQueue<String> metricQueue;
private final MemoryBoundLinkedBlockingQueue<String> alertQueue;
private final ScheduledExecutorService scheduler;

public KafkaEmitter(KafkaEmitterConfig config, ObjectMapper jsonMapper)
{
this.config = config;
this.jsonMapper = jsonMapper;
this.producer = setKafkaProducer();
this.producerCallback = setProducerCallback();
// same with kafka producer's buffer.memory
long queueMemoryBound = Long.parseLong(this.config.getKafkaProducerConfig()
.getOrDefault(ProducerConfig.BUFFER_MEMORY_CONFIG, "33554432"));
this.metricQueue = new MemoryBoundLinkedBlockingQueue<>(queueMemoryBound);
this.alertQueue = new MemoryBoundLinkedBlockingQueue<>(queueMemoryBound);
this.scheduler = Executors.newScheduledThreadPool(3);
this.metricLost = new AtomicLong(0L);
this.alertLost = new AtomicLong(0L);
this.invalidLost = new AtomicLong(0L);
}

private Callback setProducerCallback()
{
return (recordMetadata, e) -> {
if (e != null) {
log.debug("Event send failed [%s]", e.getMessage());
if (recordMetadata.topic().equals(config.getMetricTopic())) {
metricLost.incrementAndGet();
} else if (recordMetadata.topic().equals(config.getAlertTopic())) {
alertLost.incrementAndGet();
} else {
invalidLost.incrementAndGet();
}
}
};
}

private Producer<String, String> setKafkaProducer()
{
ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader();
try {
Thread.currentThread().setContextClassLoader(getClass().getClassLoader());

Properties props = new Properties();
props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, config.getBootstrapServers());
props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
props.put(ProducerConfig.RETRIES_CONFIG, DEFAULT_RETRIES);
props.putAll(config.getKafkaProducerConfig());

return new KafkaProducer<>(props);
}
finally {
Thread.currentThread().setContextClassLoader(currCtxCl);
}
}

@Override
@LifecycleStart
public void start()
{
scheduler.scheduleWithFixedDelay(this::sendMetricToKafka, 10, 10, TimeUnit.SECONDS);
scheduler.scheduleWithFixedDelay(this::sendAlertToKafka, 10, 10, TimeUnit.SECONDS);
scheduler.scheduleWithFixedDelay(() -> {
log.info("Message lost counter: metricLost=[%d], alertLost=[%d], invalidLost=[%d]",
metricLost.get(), alertLost.get(), invalidLost.get());
}, 5, 5, TimeUnit.MINUTES);
log.info("Starting Kafka Emitter.");
}

private void sendMetricToKafka()
{
sendToKafka(config.getMetricTopic(), metricQueue);
}

private void sendAlertToKafka()
{
sendToKafka(config.getAlertTopic(), alertQueue);
}

private void sendToKafka(final String topic, MemoryBoundLinkedBlockingQueue<String> recordQueue)
{
ObjectContainer<String> objectToSend;
try {
while (true) {
objectToSend = recordQueue.take();
producer.send(new ProducerRecord<>(topic, objectToSend.getData()), producerCallback);
}
}
catch (InterruptedException e) {
log.warn(e, "Failed to take record from queue!");
}
}

@Override
public void emit(final Event event)
{
if (event != null) {
ImmutableMap.Builder<String, Object> resultBuilder = ImmutableMap.<String, Object>builder().putAll(event.toMap());
if (config.getClusterName() != null) {
resultBuilder.put("clusterName", config.getClusterName());
}
Map<String, Object> result = resultBuilder.build();

try {
String resultJson = jsonMapper.writeValueAsString(result);
ObjectContainer<String> objectContainer = new ObjectContainer<>(resultJson, resultJson.getBytes().length);
if (event instanceof ServiceMetricEvent) {
if (!metricQueue.offer(objectContainer)) {
metricLost.incrementAndGet();
}
} else if (event instanceof AlertEvent) {
if (!alertQueue.offer(objectContainer)) {
alertLost.incrementAndGet();
}
} else {
invalidLost.incrementAndGet();
}
}
catch (JsonProcessingException e) {
invalidLost.incrementAndGet();
}
}
}

@Override
public void flush() throws IOException
{
producer.flush();
}

@Override
@LifecycleStop
public void close() throws IOException
{
scheduler.shutdownNow();
producer.close();
}
}
Loading