From 09fcb7558309fb300a61ff00aeecb0fffda5cfbc Mon Sep 17 00:00:00 2001 From: Jonathan Wei Date: Fri, 22 Sep 2017 06:14:32 -0700 Subject: [PATCH] Add RequestLogEvent emitters config to graphite-emitter (#4678) * Add RequestLogEvent emitters config to graphite-emitter * eagerly compute emitter list * use lambdas * checkstyle --- .../extensions-contrib/graphite.md | 11 ++++++- extensions-contrib/graphite-emitter/pom.xml | 13 ++++---- .../emitter/graphite/GraphiteEmitter.java | 18 +++++++---- .../graphite/GraphiteEmitterConfig.java | 17 +++++++++++ .../graphite/GraphiteEmitterModule.java | 30 +++++++++++-------- .../graphite/GraphiteEmitterConfigTest.java | 1 + .../server/log/EmittingRequestLogger.java | 2 +- 7 files changed, 66 insertions(+), 26 deletions(-) diff --git a/docs/content/development/extensions-contrib/graphite.md b/docs/content/development/extensions-contrib/graphite.md index 6692d35a693..da25cb34023 100644 --- a/docs/content/development/extensions-contrib/graphite.md +++ b/docs/content/development/extensions-contrib/graphite.md @@ -25,10 +25,19 @@ All the configuration parameters for graphite emitter are under `druid.emitter.g |`druid.emitter.graphite.eventConverter`| Filter and converter of druid events to graphite event (please see next section).|yes|none| |`druid.emitter.graphite.flushPeriod` | Queue flushing period in milliseconds. |no|1 minute| |`druid.emitter.graphite.maxQueueSize`| Maximum size of the queue used to buffer events. |no|`MAX_INT`| -|`druid.emitter.graphite.alertEmitters`| List of emitters where alerts will be forwarded to. |no| empty list (no forwarding)| +|`druid.emitter.graphite.alertEmitters`| List of emitters where alerts will be forwarded to. This is a JSON list of emitter names, e.g. `["logging", "http"]`|no| empty list (no forwarding)| +|`druid.emitter.graphite.requestLogEmitters`| List of emitters where request logs (i.e., query logging events sent to emitters when `druid.request.logging.type` is set to `emitter`) will be forwarded to. This is a JSON list of emitter names, e.g. `["logging", "http"]`|no| empty list (no forwarding)| |`druid.emitter.graphite.emitWaitTime` | wait time in milliseconds to try to send the event otherwise emitter will throwing event. |no|0| |`druid.emitter.graphite.waitForEventTime` | waiting time in milliseconds if necessary for an event to become available. |no|1000 (1 sec)| +### Supported event types + +The graphite emitter only emits service metric events to graphite (See http://druid.io/docs/latest/operations/metrics.html for a list of metrics). + +Alerts and request logs are not sent to graphite. These event types are not well represented in Graphite, which is more suited for timeseries views on numeric metrics, vs. storing non-numeric log events. + +Instead, alerts and request logs are optionally forwarded to other emitter implementations, specified by `druid.emitter.graphite.alertEmitters` and `druid.emitter.graphite.requestLogEmitters` respectively. + ### Druid to Graphite Event Converter Graphite Event Converter defines a mapping between druid metrics name plus dimensions to a Graphite metric path. diff --git a/extensions-contrib/graphite-emitter/pom.xml b/extensions-contrib/graphite-emitter/pom.xml index d25e9159c62..d862faea7f3 100644 --- a/extensions-contrib/graphite-emitter/pom.xml +++ b/extensions-contrib/graphite-emitter/pom.xml @@ -47,6 +47,12 @@ ${project.parent.version} provided + + io.druid + druid-server + ${project.parent.version} + provided + com.metamx emitter @@ -73,13 +79,6 @@ 1.0.4 test - - io.druid - druid-server - ${project.parent.version} - test-jar - test - io.druid druid-processing diff --git a/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/GraphiteEmitter.java b/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/GraphiteEmitter.java index 156718f02fb..cb0fb30d05b 100644 --- a/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/GraphiteEmitter.java +++ b/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/GraphiteEmitter.java @@ -30,6 +30,7 @@ import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.java.util.common.ISE; import io.druid.java.util.common.logger.Logger; +import io.druid.server.log.EmittingRequestLogger; import java.io.IOException; import java.net.SocketException; @@ -52,7 +53,8 @@ public class GraphiteEmitter implements Emitter private final DruidToGraphiteEventConverter graphiteEventConverter; private final GraphiteEmitterConfig graphiteEmitterConfig; - private final List emitterList; + private final List alertEmitters; + private final List requestLogEmitters; private final AtomicBoolean started = new AtomicBoolean(false); private final LinkedBlockingQueue eventsQueue; private static final long FLUSH_TIMEOUT = 60000; // default flush wait 1 min @@ -64,10 +66,12 @@ public class GraphiteEmitter implements Emitter public GraphiteEmitter( GraphiteEmitterConfig graphiteEmitterConfig, - List emitterList + List alertEmitters, + List requestLogEmitters ) { - this.emitterList = emitterList; + this.alertEmitters = alertEmitters; + this.requestLogEmitters = requestLogEmitters; this.graphiteEmitterConfig = graphiteEmitterConfig; this.graphiteEventConverter = graphiteEmitterConfig.getDruidToGraphiteEventConverter(); this.eventsQueue = new LinkedBlockingQueue(graphiteEmitterConfig.getMaxQueueSize()); @@ -121,8 +125,12 @@ public class GraphiteEmitter implements Emitter log.error(e, "got interrupted with message [%s]", e.getMessage()); Thread.currentThread().interrupt(); } - } else if (!emitterList.isEmpty() && event instanceof AlertEvent) { - for (Emitter emitter : emitterList) { + } else if (event instanceof EmittingRequestLogger.RequestLogEvent) { + for (Emitter emitter : requestLogEmitters) { + emitter.emit(event); + } + } else if (!alertEmitters.isEmpty() && event instanceof AlertEvent) { + for (Emitter emitter : alertEmitters) { emitter.emit(event); } } else if (event instanceof AlertEvent) { diff --git a/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/GraphiteEmitterConfig.java b/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/GraphiteEmitterConfig.java index 3758279964b..5fc11d62c0c 100644 --- a/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/GraphiteEmitterConfig.java +++ b/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/GraphiteEmitterConfig.java @@ -51,6 +51,9 @@ public class GraphiteEmitterConfig final private DruidToGraphiteEventConverter druidToGraphiteEventConverter; @JsonProperty final private List alertEmitters; + @JsonProperty + final private List requestLogEmitters; + @JsonProperty final private Long emitWaitTime; //waiting up to the specified wait time if necessary for an event to become available. @@ -95,6 +98,11 @@ public class GraphiteEmitterConfig : that.getAlertEmitters() != null) { return false; } + if (getRequestLogEmitters() != null + ? !getRequestLogEmitters().equals(that.getRequestLogEmitters()) + : that.getRequestLogEmitters() != null) { + return false; + } if (!getEmitWaitTime().equals(that.getEmitWaitTime())) { return false; } @@ -113,6 +121,7 @@ public class GraphiteEmitterConfig result = 31 * result + getMaxQueueSize().hashCode(); result = 31 * result + getDruidToGraphiteEventConverter().hashCode(); result = 31 * result + (getAlertEmitters() != null ? getAlertEmitters().hashCode() : 0); + result = 31 * result + (getRequestLogEmitters() != null ? getRequestLogEmitters().hashCode() : 0); result = 31 * result + getEmitWaitTime().hashCode(); result = 31 * result + getWaitForEventTime().hashCode(); return result; @@ -128,6 +137,7 @@ public class GraphiteEmitterConfig @JsonProperty("maxQueueSize") Integer maxQueueSize, @JsonProperty("eventConverter") DruidToGraphiteEventConverter druidToGraphiteEventConverter, @JsonProperty("alertEmitters") List alertEmitters, + @JsonProperty("requestLogEmitters") List requestLogEmitters, @JsonProperty("emitWaitTime") Long emitWaitTime, @JsonProperty("waitForEventTime") Long waitForEventTime ) @@ -135,6 +145,7 @@ public class GraphiteEmitterConfig this.waitForEventTime = waitForEventTime == null ? DEFAULT_GET_TIMEOUT : waitForEventTime; this.emitWaitTime = emitWaitTime == null ? 0 : emitWaitTime; this.alertEmitters = alertEmitters == null ? Collections.emptyList() : alertEmitters; + this.requestLogEmitters = requestLogEmitters == null ? Collections.emptyList() : requestLogEmitters; this.druidToGraphiteEventConverter = Preconditions.checkNotNull( druidToGraphiteEventConverter, "Event converter can not ne null dude" @@ -195,6 +206,12 @@ public class GraphiteEmitterConfig return alertEmitters; } + @JsonProperty + public List getRequestLogEmitters() + { + return requestLogEmitters; + } + @JsonProperty public Long getEmitWaitTime() { diff --git a/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/GraphiteEmitterModule.java b/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/GraphiteEmitterModule.java index 651ee186911..b7fd38b3681 100644 --- a/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/GraphiteEmitterModule.java +++ b/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/GraphiteEmitterModule.java @@ -21,7 +21,7 @@ package io.druid.emitter.graphite; import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Function; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.inject.Binder; import com.google.inject.Injector; @@ -57,17 +57,23 @@ public class GraphiteEmitterModule implements DruidModule @Named(EMITTER_TYPE) public Emitter getEmitter(GraphiteEmitterConfig graphiteEmitterConfig, ObjectMapper mapper, final Injector injector) { - List emitters = Lists.transform( - graphiteEmitterConfig.getAlertEmitters(), - new Function() - { - @Override - public Emitter apply(String s) - { - return injector.getInstance(Key.get(Emitter.class, Names.named(s))); - } - } + List emitters = ImmutableList.copyOf( + Lists.transform( + graphiteEmitterConfig.getAlertEmitters(), + alertEmitterName -> { + return injector.getInstance(Key.get(Emitter.class, Names.named(alertEmitterName))); + } + ) ); - return new GraphiteEmitter(graphiteEmitterConfig, emitters); + + List requestLogEmitters = ImmutableList.copyOf( + Lists.transform( + graphiteEmitterConfig.getRequestLogEmitters(), + requestLogEmitterName -> { + return injector.getInstance(Key.get(Emitter.class, Names.named(requestLogEmitterName))); + } + ) + ); + return new GraphiteEmitter(graphiteEmitterConfig, emitters, requestLogEmitters); } } diff --git a/extensions-contrib/graphite-emitter/src/test/java/io/druid/emitter/graphite/GraphiteEmitterConfigTest.java b/extensions-contrib/graphite-emitter/src/test/java/io/druid/emitter/graphite/GraphiteEmitterConfigTest.java index 43d36f6d427..196a3177220 100644 --- a/extensions-contrib/graphite-emitter/src/test/java/io/druid/emitter/graphite/GraphiteEmitterConfigTest.java +++ b/extensions-contrib/graphite-emitter/src/test/java/io/druid/emitter/graphite/GraphiteEmitterConfigTest.java @@ -54,6 +54,7 @@ public class GraphiteEmitterConfigTest 100, new SendAllGraphiteEventConverter("prefix", true, true, false), Collections.EMPTY_LIST, + Collections.EMPTY_LIST, null, null ); diff --git a/server/src/main/java/io/druid/server/log/EmittingRequestLogger.java b/server/src/main/java/io/druid/server/log/EmittingRequestLogger.java index 2cb5e5ad6b6..2e57f747f46 100644 --- a/server/src/main/java/io/druid/server/log/EmittingRequestLogger.java +++ b/server/src/main/java/io/druid/server/log/EmittingRequestLogger.java @@ -59,7 +59,7 @@ public class EmittingRequestLogger implements RequestLogger '}'; } - private static class RequestLogEvent implements Event + public static class RequestLogEvent implements Event { final ImmutableMap serviceDimensions; final String feed;