forked from apache/druid
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Opentsdb emitter extension (apache#5380)
* opentsdb emitter extension * doc for opentsdb emitter extension * update opentsdb emitter doc * add the ms unit to the constant name * add a configurable event limit * fix version to 0.13.0-SNAPSHOT * using a thread to consume metric event * rename method and parameter
- Loading branch information
Showing
17 changed files
with
1,178 additions
and
0 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
42 changes: 42 additions & 0 deletions
42
docs/content/development/extensions-contrib/opentsdb-emitter.md
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,42 @@ | ||
--- | ||
layout: doc_page | ||
--- | ||
|
||
# Opentsdb Emitter | ||
|
||
To use this extension, make sure to [include](../../operations/including-extensions.html) `opentsdb-emitter` extension. | ||
|
||
## Introduction | ||
|
||
This extension emits druid metrics to [OpenTSDB](https://github.com/OpenTSDB/opentsdb) over HTTP. And this emitter only emits service metric events to OpenTSDB (See http://druid.io/docs/latest/operations/metrics.html for a list of metrics). | ||
|
||
## Configuration | ||
|
||
All the configuration parameters for the opentsdb emitter are under `druid.emitter.opentsdb`. | ||
|
||
|property|description|required?|default| | ||
|--------|-----------|---------|-------| | ||
|`druid.emitter.opentsdb.host`|The host of the OpenTSDB server.|yes|none| | ||
|`druid.emitter.opentsdb.port`|The port of the OpenTSDB server.|yes|none| | ||
|`druid.emitter.opentsdb.connectionTimeout`|Connection timeout(in milliseconds).|no|2000| | ||
|`druid.emitter.opentsdb.readTimeout`|Read timeout(in milliseconds).|no|2000| | ||
|`druid.emitter.opentsdb.flushThreshold`|Queue flushing threshold.(Events will be sent as one batch)|no|100| | ||
|`druid.emitter.opentsdb.maxQueueSize`|Maximum size of the queue used to buffer events.|no|1000| | ||
|`druid.emitter.opentsdb.metricMapPath`|JSON file defining the desired metrics and dimensions for every Druid metric|no|./src/main/resources/defaultMetrics.json| | ||
|
||
### Druid to OpenTSDB Event Converter | ||
|
||
The opentsdb emitter will send only the desired metrics and dimensions which is defined in a JSON file. | ||
If the user does not specify their own JSON file, a default file is used. All metrics are expected to be configured in the JSON file. Metrics which are not configured will be logged. | ||
Desired metrics and dimensions is organized using the following schema:`<druid metric name> : [ <dimension list> ]`<br /> | ||
e.g. | ||
|
||
```json | ||
"query/time": [ | ||
"dataSource", | ||
"type" | ||
] | ||
``` | ||
|
||
For most use-cases, the default configuration is sufficient. | ||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,64 @@ | ||
<?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/xsd/maven-4.0.0.xsd"> | ||
<modelVersion>4.0.0</modelVersion> | ||
|
||
<groupId>io.druid.extensions.contrib</groupId> | ||
<artifactId>druid-opentsdb-emitter</artifactId> | ||
<name>druid-opentsdb-emitter</name> | ||
|
||
<parent> | ||
<groupId>io.druid</groupId> | ||
<artifactId>druid</artifactId> | ||
<version>0.13.0-SNAPSHOT</version> | ||
<relativePath>../../pom.xml</relativePath> | ||
</parent> | ||
|
||
<dependencies> | ||
<dependency> | ||
<groupId>io.druid</groupId> | ||
<artifactId>druid-api</artifactId> | ||
<version>${project.parent.version}</version> | ||
<scope>provided</scope> | ||
</dependency> | ||
<dependency> | ||
<groupId>io.druid</groupId> | ||
<artifactId>druid-server</artifactId> | ||
<version>${project.parent.version}</version> | ||
<scope>provided</scope> | ||
</dependency> | ||
<dependency> | ||
<groupId>com.sun.jersey</groupId> | ||
<artifactId>jersey-client</artifactId> | ||
<version>${jersey.version}</version> | ||
</dependency> | ||
|
||
<!-- Tests --> | ||
<dependency> | ||
<groupId>junit</groupId> | ||
<artifactId>junit</artifactId> | ||
<scope>test</scope> | ||
</dependency> | ||
</dependencies> | ||
</project> |
109 changes: 109 additions & 0 deletions
109
...ions-contrib/opentsdb-emitter/src/main/java/io/druid/emitter/opentsdb/EventConverter.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,109 @@ | ||
/* | ||
* 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.opentsdb; | ||
|
||
import com.fasterxml.jackson.core.type.TypeReference; | ||
import com.fasterxml.jackson.databind.ObjectMapper; | ||
import com.google.common.base.Strings; | ||
import io.druid.java.util.emitter.service.ServiceMetricEvent; | ||
import io.druid.java.util.common.ISE; | ||
import io.druid.java.util.common.logger.Logger; | ||
|
||
import java.io.File; | ||
import java.io.FileInputStream; | ||
import java.io.IOException; | ||
import java.io.InputStream; | ||
import java.util.HashMap; | ||
import java.util.Map; | ||
import java.util.Set; | ||
import java.util.regex.Pattern; | ||
|
||
public class EventConverter | ||
{ | ||
private static final Logger log = new Logger(EventConverter.class); | ||
private static final Pattern WHITESPACE = Pattern.compile("[\\s]+"); | ||
|
||
private final Map<String, Set<String>> metricMap; | ||
|
||
public EventConverter(ObjectMapper mapper, String metricMapPath) | ||
{ | ||
metricMap = readMap(mapper, metricMapPath); | ||
} | ||
|
||
protected String sanitize(String metric) | ||
{ | ||
return WHITESPACE.matcher(metric.trim()).replaceAll("_").replaceAll("/", "."); | ||
} | ||
|
||
/** | ||
* This function will convert a druid event to a opentsdb event. | ||
* Also this function acts as a filter. It returns <tt>null</tt> if the event is not suppose to be emitted to Opentsdb. | ||
* And it will filter out dimensions which is not suppose to be emitted. | ||
* | ||
* @param serviceMetricEvent Druid event ot type {@link ServiceMetricEvent} | ||
* | ||
* @return {@link OpentsdbEvent} or <tt>null</tt> | ||
*/ | ||
public OpentsdbEvent convert(ServiceMetricEvent serviceMetricEvent) | ||
{ | ||
String metric = serviceMetricEvent.getMetric(); | ||
if (!metricMap.containsKey(metric)) { | ||
return null; | ||
} | ||
|
||
long timestamp = serviceMetricEvent.getCreatedTime().getMillis() / 1000L; | ||
Number value = serviceMetricEvent.getValue(); | ||
|
||
Map<String, Object> tags = new HashMap<>(); | ||
String service = serviceMetricEvent.getService(); | ||
String host = serviceMetricEvent.getHost(); | ||
tags.put("service", service); | ||
tags.put("host", host); | ||
|
||
Map<String, Object> userDims = serviceMetricEvent.getUserDims(); | ||
for (String dim : metricMap.get(metric)) { | ||
if (userDims.containsKey(dim)) { | ||
tags.put(dim, userDims.get(dim)); | ||
} | ||
} | ||
|
||
return new OpentsdbEvent(sanitize(metric), timestamp, value, tags); | ||
} | ||
|
||
private Map<String, Set<String>> readMap(ObjectMapper mapper, String metricMapPath) | ||
{ | ||
try { | ||
InputStream is; | ||
if (Strings.isNullOrEmpty(metricMapPath)) { | ||
log.info("Using default metric map"); | ||
is = this.getClass().getClassLoader().getResourceAsStream("defaultMetrics.json"); | ||
} else { | ||
log.info("Using default metric map located at [%s]", metricMapPath); | ||
is = new FileInputStream(new File(metricMapPath)); | ||
} | ||
return mapper.reader(new TypeReference<Map<String, Set<String>>>() | ||
{ | ||
}).readValue(is); | ||
} | ||
catch (IOException e) { | ||
throw new ISE(e, "Failed to parse metrics and dimensions"); | ||
} | ||
} | ||
} |
82 changes: 82 additions & 0 deletions
82
...ons-contrib/opentsdb-emitter/src/main/java/io/druid/emitter/opentsdb/OpentsdbEmitter.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,82 @@ | ||
/* | ||
* 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.opentsdb; | ||
|
||
import com.fasterxml.jackson.databind.ObjectMapper; | ||
import io.druid.java.util.common.logger.Logger; | ||
import io.druid.java.util.emitter.core.Emitter; | ||
import io.druid.java.util.emitter.core.Event; | ||
import io.druid.java.util.emitter.service.ServiceMetricEvent; | ||
|
||
import java.io.IOException; | ||
|
||
public class OpentsdbEmitter implements Emitter | ||
{ | ||
private static final Logger log = new Logger(OpentsdbEmitter.class); | ||
|
||
private final OpentsdbSender sender; | ||
private final EventConverter converter; | ||
|
||
public OpentsdbEmitter(OpentsdbEmitterConfig config, ObjectMapper mapper) | ||
{ | ||
this.sender = new OpentsdbSender( | ||
config.getHost(), | ||
config.getPort(), | ||
config.getConnectionTimeout(), | ||
config.getReadTimeout(), | ||
config.getFlushThreshold(), | ||
config.getMaxQueueSize() | ||
); | ||
this.converter = new EventConverter(mapper, config.getMetricMapPath()); | ||
} | ||
|
||
@Override | ||
public void start() | ||
{ | ||
} | ||
|
||
@Override | ||
public void emit(Event event) | ||
{ | ||
if (event instanceof ServiceMetricEvent) { | ||
OpentsdbEvent opentsdbEvent = converter.convert((ServiceMetricEvent) event); | ||
if (opentsdbEvent != null) { | ||
sender.enqueue(opentsdbEvent); | ||
} else { | ||
log.debug( | ||
"Metric=[%s] has not been configured to be emitted to opentsdb", | ||
((ServiceMetricEvent) event).getMetric() | ||
); | ||
} | ||
} | ||
} | ||
|
||
@Override | ||
public void flush() throws IOException | ||
{ | ||
sender.flush(); | ||
} | ||
|
||
@Override | ||
public void close() throws IOException | ||
{ | ||
sender.close(); | ||
} | ||
} |
Oops, something went wrong.