From 7c780dc6086eb31fc64bf7c09295e73f71669355 Mon Sep 17 00:00:00 2001 From: Michael Schiff Date: Fri, 5 Feb 2016 13:16:21 -0800 Subject: [PATCH] statsd-emitter --- .../io.druid.initialization.DruidModule | 1 + .../development/extensions-contrib/statsd.md | 46 ++++++ extensions-contrib/statsd-emitter/pom.xml | 90 ++++++++++++ .../emitter/statsd/DimensionConverter.java | 91 ++++++++++++ .../druid/emitter/statsd/StatsDEmitter.java | 128 ++++++++++++++++ .../emitter/statsd/StatsDEmitterConfig.java | 139 ++++++++++++++++++ .../emitter/statsd/StatsDEmitterModule.java | 57 +++++++ .../io/druid/emitter/statsd/StatsDMetric.java | 44 ++++++ .../io.druid.initialization.DruidModule | 1 + .../resources/defaultMetricDimensions.json | 105 +++++++++++++ .../src/test/java/DimensionConverterTest.java | 64 ++++++++ pom.xml | 1 + 12 files changed, 767 insertions(+) create mode 100644 META-INF/services/io.druid.initialization.DruidModule create mode 100644 docs/content/development/extensions-contrib/statsd.md create mode 100644 extensions-contrib/statsd-emitter/pom.xml create mode 100644 extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/DimensionConverter.java create mode 100644 extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitter.java create mode 100644 extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitterConfig.java create mode 100644 extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitterModule.java create mode 100644 extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDMetric.java create mode 100644 extensions-contrib/statsd-emitter/src/main/resources/META-INF/services/io.druid.initialization.DruidModule create mode 100644 extensions-contrib/statsd-emitter/src/main/resources/defaultMetricDimensions.json create mode 100644 extensions-contrib/statsd-emitter/src/test/java/DimensionConverterTest.java diff --git a/META-INF/services/io.druid.initialization.DruidModule b/META-INF/services/io.druid.initialization.DruidModule new file mode 100644 index 000000000000..141bbc405d7a --- /dev/null +++ b/META-INF/services/io.druid.initialization.DruidModule @@ -0,0 +1 @@ +io.druid.emitter.statsd.StatsDEmitterModule diff --git a/docs/content/development/extensions-contrib/statsd.md b/docs/content/development/extensions-contrib/statsd.md new file mode 100644 index 000000000000..25d41b38afb9 --- /dev/null +++ b/docs/content/development/extensions-contrib/statsd.md @@ -0,0 +1,46 @@ +--- +layout: doc_page +--- + +# StatsD Emitter + +To use this extension, make sure to [include](../../operations/including-extensions.html) `statsd-emitter` extension. + +## Introduction + +This extension emits druid metrics to a StatsD server. +(https://github.com/etsy/statsd) +(https://github.com/armon/statsite) + +## Configuration + +All the configuration parameters for the StatsD emitter are under `druid.emitter.statsd`. + +|property|description|required?|default| +|--------|-----------|---------|-------| +|`druid.emitter.statsd.hostname`|The hostname of the StatsD server.|yes|none| +|`druid.emitter.statsd.port`|The port of the StatsD server.|yes|none| +|`druid.emitter.statsd.prefix`|Optional metric name prefix.|no|""| +|`druid.emitter.statsd.separator`|Metric name separator|no|.| +|`druid.emitter.statsd.includeHost`|Flag to include the hostname as part of the metric name.|no|false| +|`druid.emitter.statsd.dimensionMapPath`|JSON file defining the StatsD type, and desired dimensions for every Druid metric|no|Default mapping provided. See below.| + +### Druid to StatsD Event Converter + +Each metric sent to StatsD must specify a type, one of `[timer, counter, guage]`. StatsD Emitter expects this mapping to +be provided as a JSON file. Additionally, this mapping specifies which dimensions should be included for each metric. +If the user does not specify their own JSON file, a default mapping is used. All +metrics are expected to be mapped. Metrics which are not mapped will log an error. +StatsD metric path is organized using the following schema: +` : { "dimensions" : , "type" : }` +e.g. +`query/time" : { "dimensions" : ["dataSource", "type"], "type" : "timer"}` + +For metrics which are emitted from multiple services with different dimensions, the metric name is prefixed with +the service name. +e.g. +`"coordinator-segment/count" : { "dimensions" : ["dataSource"], "type" : "gauge" }, + "historical-segment/count" : { "dimensions" : ["dataSource", "tier", "priority"], "type" : "gauge" }` + +For most use-cases, the default mapping is sufficient. + diff --git a/extensions-contrib/statsd-emitter/pom.xml b/extensions-contrib/statsd-emitter/pom.xml new file mode 100644 index 000000000000..7ebd56711234 --- /dev/null +++ b/extensions-contrib/statsd-emitter/pom.xml @@ -0,0 +1,90 @@ + + + + + druid + io.druid + 0.9.1-SNAPSHOT + ../../pom.xml + + 4.0.0 + + io.druid.extensions.contrib + statsd-emitter + statsd-emitter + Extension support for emitting Druid metrics to StatsD + + + + io.druid + druid-common + ${project.parent.version} + provided + + + io.druid + druid-api + ${project.parent.version} + provided + + + com.metamx + emitter + provided + + + com.timgroup + java-statsd-client + 3.0.1 + + + junit + junit + test + + + org.easymock + easymock + test + + + pl.pragmatists + JUnitParams + 1.0.4 + test + + + io.druid + druid-server + ${project.parent.version} + test-jar + test + + + io.druid + druid-processing + ${project.parent.version} + test-jar + test + + + diff --git a/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/DimensionConverter.java b/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/DimensionConverter.java new file mode 100644 index 000000000000..2de1290b88f4 --- /dev/null +++ b/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/DimensionConverter.java @@ -0,0 +1,91 @@ +/* + * 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.statsd; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; +import com.metamx.common.ISE; +import com.metamx.common.logger.Logger; + +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.Map; + +/** + */ +public class DimensionConverter +{ + + private final static Logger log = new Logger(DimensionConverter.class); + private Map metricMap; + + public DimensionConverter(ObjectMapper mapper, String dimensionMapPath) + { + metricMap = readMap(mapper, dimensionMapPath); + } + + public StatsDMetric.Type addFilteredUserDims(String service, String metric, Map userDims, ImmutableList.Builder builder) + { + /* + Find the metric in the map. If we cant find it try to look it up prefixed by the service name. + This is because some metrics are reported differently, but with the same name, from different services. + */ + StatsDMetric statsDMetric = null; + if (metricMap.containsKey(metric)) { + statsDMetric = metricMap.get(metric); + } else if (metricMap.containsKey(service + "-" + metric)) { + statsDMetric = metricMap.get(service + "-" + metric); + } + if (statsDMetric != null) { + for (String dim : statsDMetric.dimensions) { + if (userDims.containsKey(dim)) { + builder.add(userDims.get(dim).toString()); + } + } + return statsDMetric.type; + } else { + return null; + } + } + + private Map readMap(ObjectMapper mapper, String dimensionMapPath) + { + try { + InputStream is; + if (Strings.isNullOrEmpty(dimensionMapPath)) { + log.info("Using default metric dimension and types"); + is = this.getClass().getClassLoader().getResourceAsStream("defaultMetricDimensions.json"); + } else { + log.info("Using metric dimensions at types at [%s]", dimensionMapPath); + is = new FileInputStream(new File(dimensionMapPath)); + } + return mapper.reader(new TypeReference>() + { + }).readValue(is); + } + catch (IOException e) { + throw new ISE(e, "Failed to parse metric dimensions and types"); + } + } +} diff --git a/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitter.java b/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitter.java new file mode 100644 index 000000000000..1d11f08f0e57 --- /dev/null +++ b/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitter.java @@ -0,0 +1,128 @@ +/* + * 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.statsd; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Joiner; +import com.google.common.collect.ImmutableList; +import com.metamx.common.logger.Logger; +import com.metamx.emitter.core.Emitter; +import com.metamx.emitter.core.Event; +import com.metamx.emitter.service.ServiceMetricEvent; +import com.timgroup.statsd.NonBlockingStatsDClient; +import com.timgroup.statsd.StatsDClient; +import com.timgroup.statsd.StatsDClientErrorHandler; + +import java.io.IOException; +import java.util.Map; + +/** + */ +public class StatsDEmitter implements Emitter +{ + + private final static Logger log = new Logger(StatsDEmitter.class); + private final static String DRUID_METRIC_SEPARATOR = "\\/"; + private final static String STATSD_SEPARATOR = ":|\\|"; + + private final StatsDClient statsd; + private final StatsDEmitterConfig config; + private final DimensionConverter converter; + + public StatsDEmitter(StatsDEmitterConfig config, ObjectMapper mapper) { + this.config = config; + this.converter = new DimensionConverter(mapper, config.getDimensionMapPath()); + statsd = new NonBlockingStatsDClient( + config.getPrefix(), + config.getHostname(), + config.getPort(), + new StatsDClientErrorHandler() + { + private int exceptionCount = 0; + @Override + public void handle(Exception exception) + { + if (exceptionCount % 1000 == 0) { + log.error(exception, "Error sending metric to StatsD."); + } + exceptionCount += 1; + } + } + ); + } + + + @Override + public void start() {} + + @Override + public void emit(Event event) + { + if (event instanceof ServiceMetricEvent) { + ServiceMetricEvent metricEvent = (ServiceMetricEvent) event; + String host = metricEvent.getHost(); + String service = metricEvent.getService(); + String metric = metricEvent.getMetric(); + Map userDims = metricEvent.getUserDims(); + Number value = metricEvent.getValue(); + + ImmutableList.Builder nameBuilder = new ImmutableList.Builder<>(); + if (config.getIncludeHost()) { + nameBuilder.add(host); + } + nameBuilder.add(service); + nameBuilder.add(metric); + + StatsDMetric.Type metricType = converter.addFilteredUserDims(service, metric, userDims, nameBuilder); + + if (metricType != null) { + + String fullName = Joiner.on(config.getSeparator()) + .join(nameBuilder.build()) + .replaceAll(DRUID_METRIC_SEPARATOR, config.getSeparator()) + .replaceAll(STATSD_SEPARATOR, config.getSeparator()); + + switch (metricType) { + case count: + statsd.count(fullName, value.longValue()); + break; + case timer: + statsd.time(fullName, value.longValue()); + break; + case gauge: + statsd.gauge(fullName, value.longValue()); + break; + } + } else { + log.error("Metric=[%s] has no StatsD type mapping", metric); + } + } + } + + @Override + public void flush() throws IOException {} + + @Override + public void close() throws IOException + { + statsd.stop(); + } + +} diff --git a/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitterConfig.java b/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitterConfig.java new file mode 100644 index 000000000000..548bc42f3170 --- /dev/null +++ b/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitterConfig.java @@ -0,0 +1,139 @@ +/* + * 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.statsd; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; + +/** + */ +public class StatsDEmitterConfig +{ + + @JsonProperty + final private String hostname; + @JsonProperty + final private Integer port; + @JsonProperty + final private String prefix; + @JsonProperty + final private String separator; + @JsonProperty + final private Boolean includeHost; + @JsonProperty + final private String dimensionMapPath; + + @JsonCreator + public StatsDEmitterConfig( + @JsonProperty("hostname") String hostname, + @JsonProperty("port") Integer port, + @JsonProperty("prefix") String prefix, + @JsonProperty("separator") String separator, + @JsonProperty("includeHost") Boolean includeHost, + @JsonProperty("dimensionMapPath") String dimensionMapPath) + { + this.hostname = Preconditions.checkNotNull(hostname, "StatsD hostname cannot be null."); + this.port = Preconditions.checkNotNull(port, "StatsD port cannot be null."); + this.prefix = prefix != null ? prefix : ""; + this.separator = separator != null ? separator : "."; + this.includeHost = includeHost != null ? includeHost : false; + this.dimensionMapPath = dimensionMapPath; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + StatsDEmitterConfig that = (StatsDEmitterConfig) o; + + if (hostname != null ? !hostname.equals(that.hostname) : that.hostname != null) { + return false; + } + if (port != null ? !port.equals(that.port) : that.port != null) { + return false; + } + if (prefix != null ? !prefix.equals(that.prefix) : that.prefix != null) { + return false; + } + if (separator != null ? !separator.equals(that.separator) : that.separator != null) { + return false; + } + if (includeHost != null ? !includeHost.equals(that.includeHost) : that.includeHost != null) { + return false; + } + return dimensionMapPath != null ? dimensionMapPath.equals(that.dimensionMapPath) : that.dimensionMapPath == null; + + } + + @Override + public int hashCode() + { + int result = hostname != null ? hostname.hashCode() : 0; + result = 31 * result + (port != null ? port.hashCode() : 0); + result = 31 * result + (prefix != null ? prefix.hashCode() : 0); + result = 31 * result + (separator != null ? separator.hashCode() : 0); + result = 31 * result + (includeHost != null ? includeHost.hashCode() : 0); + result = 31 * result + (dimensionMapPath != null ? dimensionMapPath.hashCode() : 0); + return result; + } + + @JsonProperty + public String getHostname() + { + return hostname; + } + + @JsonProperty + public int getPort() + { + return port; + } + + @JsonProperty + public String getPrefix() + { + return prefix; + } + + @JsonProperty + public String getSeparator() + { + return separator; + } + + @JsonProperty + public Boolean getIncludeHost() + { + return includeHost; + } + + @JsonProperty + public String getDimensionMapPath() + { + return dimensionMapPath; + } +} diff --git a/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitterModule.java b/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitterModule.java new file mode 100644 index 000000000000..7d8b037f04de --- /dev/null +++ b/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitterModule.java @@ -0,0 +1,57 @@ +/* + * 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.statsd; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Binder; +import com.google.inject.Provides; +import com.google.inject.name.Named; +import com.metamx.emitter.core.Emitter; +import io.druid.guice.JsonConfigProvider; +import io.druid.guice.ManageLifecycle; +import io.druid.initialization.DruidModule; + +import java.util.Collections; +import java.util.List; + +/** + */ +public class StatsDEmitterModule implements DruidModule +{ + private static final String EMITTER_TYPE = "statsd"; + @Override + public List getJacksonModules() { + return Collections.EMPTY_LIST; + } + + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.emitter." + EMITTER_TYPE, StatsDEmitterConfig.class); + } + + @Provides + @ManageLifecycle + @Named(EMITTER_TYPE) + public Emitter getEmitter(StatsDEmitterConfig config, ObjectMapper mapper){ + return new StatsDEmitter(config, mapper); + } +} diff --git a/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDMetric.java b/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDMetric.java new file mode 100644 index 000000000000..8af557689302 --- /dev/null +++ b/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDMetric.java @@ -0,0 +1,44 @@ +/* + * 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.statsd; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.SortedSet; + +/** +*/ +public class StatsDMetric { + public final SortedSet dimensions; + public final Type type; + @JsonCreator + public StatsDMetric( + @JsonProperty("dimensions") SortedSet dimensions, + @JsonProperty("type") Type type) + { + this.dimensions = dimensions; + this.type = type; + } + + public enum Type { + count, gauge, timer + } +} diff --git a/extensions-contrib/statsd-emitter/src/main/resources/META-INF/services/io.druid.initialization.DruidModule b/extensions-contrib/statsd-emitter/src/main/resources/META-INF/services/io.druid.initialization.DruidModule new file mode 100644 index 000000000000..141bbc405d7a --- /dev/null +++ b/extensions-contrib/statsd-emitter/src/main/resources/META-INF/services/io.druid.initialization.DruidModule @@ -0,0 +1 @@ +io.druid.emitter.statsd.StatsDEmitterModule diff --git a/extensions-contrib/statsd-emitter/src/main/resources/defaultMetricDimensions.json b/extensions-contrib/statsd-emitter/src/main/resources/defaultMetricDimensions.json new file mode 100644 index 000000000000..a452ec6ea308 --- /dev/null +++ b/extensions-contrib/statsd-emitter/src/main/resources/defaultMetricDimensions.json @@ -0,0 +1,105 @@ +{ + "query/time" : { "dimensions" : ["dataSource", "type"], "type" : "timer"}, + "query/node/time" : { "dimensions" : ["server"], "type" : "timer"}, + "query/node/ttfb" : { "dimensions" : ["server"], "type" : "timer"}, + "query/intervalChunk/time" : { "dimensions" : [], "type" : "timer"}, + + "query/segment/time" : { "dimensions" : [], "type" : "timer"}, + "query/wait/time" : { "dimensions" : [], "type" : "timer"}, + "segment/scan/pending" : { "dimensions" : [], "type" : "gauge"}, + "query/segmentAndCache/time" : { "dimensions" : [], "type" : "timer" }, + "query/cpu/time" : { "dimensions" : ["dataSource", "type"], "type" : "timer" }, + + "query/cache/delta/numEntries" : { "dimensions" : [], "type" : "count" }, + "query/cache/delta/sizeBytes" : { "dimensions" : [], "type" : "count" }, + "query/cache/delta/hits" : { "dimensions" : [], "type" : "count" }, + "query/cache/delta/misses" : { "dimensions" : [], "type" : "count" }, + "query/cache/delta/evictions" : { "dimensions" : [], "type" : "count" }, + "query/cache/delta/hitRate" : { "dimensions" : [], "type" : "count" }, + "query/cache/delta/averageBytes" : { "dimensions" : [], "type" : "count" }, + "query/cache/delta/timeouts" : { "dimensions" : [], "type" : "count" }, + "query/cache/delta/errors" : { "dimensions" : [], "type" : "count" }, + + "query/cache/total/numEntries" : { "dimensions" : [], "type" : "gauge" }, + "query/cache/total/sizeBytes" : { "dimensions" : [], "type" : "gauge" }, + "query/cache/total/hits" : { "dimensions" : [], "type" : "gauge" }, + "query/cache/total/misses" : { "dimensions" : [], "type" : "gauge" }, + "query/cache/total/evictions" : { "dimensions" : [], "type" : "gauge" }, + "query/cache/total/hitRate" : { "dimensions" : [], "type" : "gauge" }, + "query/cache/total/averageBytes" : { "dimensions" : [], "type" : "gauge" }, + "query/cache/total/timeouts" : { "dimensions" : [], "type" : "gauge" }, + "query/cache/total/errors" : { "dimensions" : [], "type" : "gauge" }, + + "ingest/events/thrownAway" : { "dimensions" : ["dataSource"], "type" : "count" }, + "ingest/events/unparseable" : { "dimensions" : ["dataSource"], "type" : "count" }, + "ingest/events/processed" : { "dimensions" : ["dataSource"], "type" : "count" }, + "ingest/rows/output" : { "dimensions" : ["dataSource"], "type" : "count" }, + "ingest/persist/count" : { "dimensions" : ["dataSource"], "type" : "count" }, + "ingest/persist/time" : { "dimensions" : ["dataSource"], "type" : "timer" }, + "ingest/persist/cpu" : { "dimensions" : ["dataSource"], "type" : "timer" }, + "ingest/persist/backPressure" : { "dimensions" : ["dataSource"], "type" : "gauge" }, + "ingest/persist/failed" : { "dimensions" : ["dataSource"], "type" : "count" }, + "ingest/handoff/failed" : { "dimensions" : ["dataSource"], "type" : "count" }, + "ingest/merge/time" : { "dimensions" : ["dataSource"], "type" : "timer" }, + "ingest/merge/cpu" : { "dimensions" : ["dataSource"], "type" : "timer" }, + + "task/run/time" : { "dimensions" : ["dataSource", "taskType"], "type" : "timer" }, + "segment/added/bytes" : { "dimensions" : ["dataSource", "taskType"], "type" : "count" }, + "segment/moved/bytes" : { "dimensions" : ["dataSource", "taskType"], "type" : "count" }, + "segment/nuked/bytes" : { "dimensions" : ["dataSource", "taskType"], "type" : "count" }, + + "segment/assigned/count" : { "dimensions" : ["tier"], "type" : "count" }, + "segment/moved/count" : { "dimensions" : ["tier"], "type" : "count" }, + "segment/dropped/count" : { "dimensions" : ["tier"], "type" : "count" }, + "segment/deleted/count" : { "dimensions" : ["tier"], "type" : "count" }, + "segment/unneeded/count" : { "dimensions" : ["tier"], "type" : "count" }, + "segment/cost/raw" : { "dimensions" : ["tier"], "type" : "count" }, + "segment/cost/normalization" : { "dimensions" : ["tier"], "type" : "count" }, + "segment/cost/normalized" : { "dimensions" : ["tier"], "type" : "count" }, + "segment/loadQueue/size" : { "dimensions" : ["server"], "type" : "gauge" }, + "segment/loadQueue/failed" : { "dimensions" : ["server"], "type" : "gauge" }, + "segment/loadQueue/count" : { "dimensions" : ["server"], "type" : "gauge" }, + "segment/dropQueue/count" : { "dimensions" : ["server"], "type" : "gauge" }, + "segment/size" : { "dimensions" : ["dataSource"], "type" : "gauge" }, + "segment/overShadowed/count" : { "dimensions" : [], "type" : "gauge" }, + + "segment/max" : { "dimensions" : [], "type" : "gauge"}, + "segment/used" : { "dimensions" : ["dataSource", "tier", "priority"], "type" : "gauge" }, + "segment/usedPercent" : { "dimensions" : ["dataSource", "tier", "priority"], "type" : "gauge" }, + + "jvm/pool/committed" : { "dimensions" : ["poolKind", "poolName"], "type" : "gauge" }, + "jvm/pool/init" : { "dimensions" : ["poolKind", "poolName"], "type" : "gauge" }, + "jvm/pool/max" : { "dimensions" : ["poolKind", "poolName"], "type" : "gauge" }, + "jvm/pool/used" : { "dimensions" : ["poolKind", "poolName"], "type" : "gauge" }, + "jvm/bufferpool/count" : { "dimensions" : ["bufferPoolName"], "type" : "gauge" }, + "jvm/bufferpool/used" : { "dimensions" : ["bufferPoolName"], "type" : "gauge" }, + "jvm/bufferpool/capacity" : { "dimensions" : ["bufferPoolName"], "type" : "gauge" }, + "jvm/mem/init" : { "dimensions" : ["memKind"], "type" : "gauge" }, + "jvm/mem/max" : { "dimensions" : ["memKind"], "type" : "gauge" }, + "jvm/mem/used" : { "dimensions" : ["memKind"], "type" : "gauge" }, + "jvm/mem/committed" : { "dimensions" : ["memKind"], "type" : "gauge" }, + "jvm/gc/count" : { "dimensions" : ["gcName"], "type" : "count" }, + "jvm/gc/time" : { "dimensions" : ["gcName"], "type" : "timer" }, + + "ingest/events/buffered" : { "dimensions" : ["serviceName, bufferCapacity"], "type" : "gauge"}, + + "sys/swap/free" : { "dimensions" : [], "type" : "gauge"}, + "sys/swap/max" : { "dimensions" : [], "type" : "gauge"}, + "sys/swap/pageIn" : { "dimensions" : [], "type" : "gauge"}, + "sys/swap/pageOut" : { "dimensions" : [], "type" : "gauge"}, + "sys/disk/write/count" : { "dimensions" : ["fsDevName"], "type" : "count"}, + "sys/disk/read/count" : { "dimensions" : ["fsDevName"], "type" : "count"}, + "sys/disk/write/size" : { "dimensions" : ["fsDevName"], "type" : "count"}, + "sys/disk/read/size" : { "dimensions" : ["fsDevName"], "type" : "count"}, + "sys/net/write/size" : { "dimensions" : [], "type" : "count"}, + "sys/net/read/size" : { "dimensions" : [], "type" : "count"}, + "sys/fs/used" : { "dimensions" : ["fsDevName", "fsDirName", "fsTypeName", "fsSysTypeName", "fsOptions"], "type" : "gauge"}, + "sys/fs/max" : { "dimensions" : ["fsDevName", "fsDirName", "fsTypeName", "fsSysTypeName", "fsOptions"], "type" : "gauge"}, + "sys/mem/used" : { "dimensions" : [], "type" : "gauge"}, + "sys/mem/max" : { "dimensions" : [], "type" : "gauge"}, + "sys/storage/used" : { "dimensions" : ["fsDirName"], "type" : "gauge"}, + "sys/cpu" : { "dimensions" : ["cpuName", "cpuTime"], "type" : "gauge"}, + + "coordinator-segment/count" : { "dimensions" : ["dataSource"], "type" : "gauge" }, + "historical-segment/count" : { "dimensions" : ["dataSource", "tier", "priority"], "type" : "gauge" } +} \ No newline at end of file diff --git a/extensions-contrib/statsd-emitter/src/test/java/DimensionConverterTest.java b/extensions-contrib/statsd-emitter/src/test/java/DimensionConverterTest.java new file mode 100644 index 000000000000..581dcf1df684 --- /dev/null +++ b/extensions-contrib/statsd-emitter/src/test/java/DimensionConverterTest.java @@ -0,0 +1,64 @@ +/* + * 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. + */ + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.metamx.emitter.service.ServiceMetricEvent; +import io.druid.emitter.statsd.DimensionConverter; +import io.druid.emitter.statsd.StatsDMetric; +import org.joda.time.DateTime; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +/** + */ +public class DimensionConverterTest +{ + @Test + public void testConvert() throws Exception + { + DimensionConverter dimensionConverter = new DimensionConverter(new ObjectMapper(), null); + ServiceMetricEvent event = new ServiceMetricEvent.Builder().setDimension("dataSource", "data-source") + .setDimension("type", "groupBy") + .setDimension("interval", "2013/2015") + .setDimension("some_random_dim1", "random_dim_value1") + .setDimension("some_random_dim2", "random_dim_value2") + .setDimension("hasFilters", "no") + .setDimension("duration", "P1D") + .setDimension("remoteAddress", "194.0.90.2") + .setDimension("id", "ID") + .setDimension("context", "{context}") + .build(new DateTime(), "query/time", 10) + .build("broker", "brokerHost1"); + + ImmutableList.Builder actual = new ImmutableList.Builder<>(); + StatsDMetric.Type type = dimensionConverter.addFilteredUserDims( + event.getService(), + event.getMetric(), + event.getUserDims(), + actual + ); + assertEquals("correct StatsDMetric.Type", StatsDMetric.Type.timer, type); + ImmutableList.Builder expected = new ImmutableList.Builder<>(); + expected.add("data-source"); + expected.add("groupBy"); + assertEquals("correct Dimensions", expected.build(), actual.build()); + } +} diff --git a/pom.xml b/pom.xml index e011e0762ce6..13fc667a4247 100644 --- a/pom.xml +++ b/pom.xml @@ -103,6 +103,7 @@ extensions-contrib/rabbitmq extensions-contrib/distinctcount extensions-contrib/parquet-extensions + extensions-contrib/statsd-emitter distribution