mirror of https://github.com/apache/druid.git
Add RequestLogEvent emitters config to graphite-emitter (#4678)
* Add RequestLogEvent emitters config to graphite-emitter * eagerly compute emitter list * use lambdas * checkstyle
This commit is contained in:
parent
e267f3901b
commit
09fcb75583
|
@ -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.
|
||||
|
|
|
@ -47,6 +47,12 @@
|
|||
<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.metamx</groupId>
|
||||
<artifactId>emitter</artifactId>
|
||||
|
@ -73,13 +79,6 @@
|
|||
<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>
|
||||
|
|
|
@ -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<Emitter> emitterList;
|
||||
private final List<Emitter> alertEmitters;
|
||||
private final List<Emitter> requestLogEmitters;
|
||||
private final AtomicBoolean started = new AtomicBoolean(false);
|
||||
private final LinkedBlockingQueue<GraphiteEvent> 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<Emitter> emitterList
|
||||
List<Emitter> alertEmitters,
|
||||
List<Emitter> 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) {
|
||||
|
|
|
@ -51,6 +51,9 @@ public class GraphiteEmitterConfig
|
|||
final private DruidToGraphiteEventConverter druidToGraphiteEventConverter;
|
||||
@JsonProperty
|
||||
final private List<String> alertEmitters;
|
||||
@JsonProperty
|
||||
final private List<String> 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<String> alertEmitters,
|
||||
@JsonProperty("requestLogEmitters") List<String> 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.<String>emptyList() : alertEmitters;
|
||||
this.requestLogEmitters = requestLogEmitters == null ? Collections.<String>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<String> getRequestLogEmitters()
|
||||
{
|
||||
return requestLogEmitters;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Long getEmitWaitTime()
|
||||
{
|
||||
|
|
|
@ -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<Emitter> emitters = Lists.transform(
|
||||
graphiteEmitterConfig.getAlertEmitters(),
|
||||
new Function<String, Emitter>()
|
||||
{
|
||||
@Override
|
||||
public Emitter apply(String s)
|
||||
{
|
||||
return injector.getInstance(Key.get(Emitter.class, Names.named(s)));
|
||||
}
|
||||
}
|
||||
List<Emitter> emitters = ImmutableList.copyOf(
|
||||
Lists.transform(
|
||||
graphiteEmitterConfig.getAlertEmitters(),
|
||||
alertEmitterName -> {
|
||||
return injector.getInstance(Key.get(Emitter.class, Names.named(alertEmitterName)));
|
||||
}
|
||||
)
|
||||
);
|
||||
return new GraphiteEmitter(graphiteEmitterConfig, emitters);
|
||||
|
||||
List<Emitter> requestLogEmitters = ImmutableList.copyOf(
|
||||
Lists.transform(
|
||||
graphiteEmitterConfig.getRequestLogEmitters(),
|
||||
requestLogEmitterName -> {
|
||||
return injector.getInstance(Key.get(Emitter.class, Names.named(requestLogEmitterName)));
|
||||
}
|
||||
)
|
||||
);
|
||||
return new GraphiteEmitter(graphiteEmitterConfig, emitters, requestLogEmitters);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -54,6 +54,7 @@ public class GraphiteEmitterConfigTest
|
|||
100,
|
||||
new SendAllGraphiteEventConverter("prefix", true, true, false),
|
||||
Collections.EMPTY_LIST,
|
||||
Collections.EMPTY_LIST,
|
||||
null,
|
||||
null
|
||||
);
|
||||
|
|
|
@ -59,7 +59,7 @@ public class EmittingRequestLogger implements RequestLogger
|
|||
'}';
|
||||
}
|
||||
|
||||
private static class RequestLogEvent implements Event
|
||||
public static class RequestLogEvent implements Event
|
||||
{
|
||||
final ImmutableMap<String, String> serviceDimensions;
|
||||
final String feed;
|
||||
|
|
Loading…
Reference in New Issue