diff --git a/distribution/pom.xml b/distribution/pom.xml
index 1059f5ecd6d..e93dbdb1628 100644
--- a/distribution/pom.xml
+++ b/distribution/pom.xml
@@ -224,6 +224,8 @@
-c
io.druid.extensions.contrib:graphite-emitter
-c
+ io.druid.extensions.contrib:druid-opentsdb-emitter
+ -c
io.druid.extensions.contrib:druid-orc-extensions
-c
io.druid.extensions.contrib:druid-parquet-extensions
diff --git a/docs/content/development/extensions-contrib/opentsdb-emitter.md b/docs/content/development/extensions-contrib/opentsdb-emitter.md
new file mode 100644
index 00000000000..dafdfd07552
--- /dev/null
+++ b/docs/content/development/extensions-contrib/opentsdb-emitter.md
@@ -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:` : [ ]`
+e.g.
+
+```json
+"query/time": [
+ "dataSource",
+ "type"
+]
+```
+
+For most use-cases, the default configuration is sufficient.
+
diff --git a/docs/content/development/extensions.md b/docs/content/development/extensions.md
index a9e1d1ee2f0..4e2bea613fa 100644
--- a/docs/content/development/extensions.md
+++ b/docs/content/development/extensions.md
@@ -71,6 +71,7 @@ All of these community extensions can be downloaded using *pull-deps* with the c
|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)|
+|druid-opentsdb-emitter|OpenTSDB metrics emitter |[link](../development/extensions-contrib/opentsdb-emitter.html)|
## Promoting Community Extension to Core Extension
diff --git a/extensions-contrib/opentsdb-emitter/pom.xml b/extensions-contrib/opentsdb-emitter/pom.xml
new file mode 100644
index 00000000000..22f475dd5fc
--- /dev/null
+++ b/extensions-contrib/opentsdb-emitter/pom.xml
@@ -0,0 +1,64 @@
+
+
+
+
+
+ 4.0.0
+
+ io.druid.extensions.contrib
+ druid-opentsdb-emitter
+ druid-opentsdb-emitter
+
+
+ io.druid
+ druid
+ 0.13.0-SNAPSHOT
+ ../../pom.xml
+
+
+
+
+ io.druid
+ druid-api
+ ${project.parent.version}
+ provided
+
+
+ io.druid
+ druid-server
+ ${project.parent.version}
+ provided
+
+
+ com.sun.jersey
+ jersey-client
+ ${jersey.version}
+
+
+
+
+ junit
+ junit
+ test
+
+
+
diff --git a/extensions-contrib/opentsdb-emitter/src/main/java/io/druid/emitter/opentsdb/EventConverter.java b/extensions-contrib/opentsdb-emitter/src/main/java/io/druid/emitter/opentsdb/EventConverter.java
new file mode 100644
index 00000000000..83f8986ef52
--- /dev/null
+++ b/extensions-contrib/opentsdb-emitter/src/main/java/io/druid/emitter/opentsdb/EventConverter.java
@@ -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> 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 null 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 null
+ */
+ 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 tags = new HashMap<>();
+ String service = serviceMetricEvent.getService();
+ String host = serviceMetricEvent.getHost();
+ tags.put("service", service);
+ tags.put("host", host);
+
+ Map 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> 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