Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions docs/content/configuration/index.md
Original file line number Diff line number Diff line change
Expand Up @@ -118,6 +118,7 @@ The following monitors are available:
|`io.druid.server.metrics.HistoricalMetricsMonitor`|Reports statistics on Historical nodes.|
|`com.metamx.metrics.JvmMonitor`|Reports JVM-related statistics.|
|`io.druid.segment.realtime.RealtimeMetricsMonitor`|Reports statistics on Realtime nodes.|
|`io.druid.server.metrics.EventReceiverFirehoseMonitor`|Reports how many events have been queued in the EventReceiverFirehose.|

### Emitting Metrics

Expand Down
8 changes: 8 additions & 0 deletions docs/content/operations/metrics.md
Original file line number Diff line number Diff line change
Expand Up @@ -156,6 +156,14 @@ These metrics are only available if the JVMMonitor module is included.
|`jvm/gc/count`|Garbage collection count.|gcName.|< 100|
|`jvm/gc/time`|Garbage collection time.|gcName.|< 1s|

### EventReceiverFirehose

The following metric is only available if the EventReceiverFirehoseMonitor module is included.

|Metric|Description|Dimensions|Normal Value|
|------|-----------|----------|------------|
|`ingest/events/buffered`|Number of events queued in the EventReceiverFirehose's buffer|serviceName, bufferCapacity.|Equal to current # of events in the buffer queue.|

## Sys

These metrics are only available if the SysMonitor module is included.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,12 +33,12 @@
import io.druid.data.input.Firehose;
import io.druid.data.input.FirehoseFactory;
import io.druid.data.input.InputRow;
import io.druid.data.input.Rows;
import io.druid.data.input.impl.MapInputRowParser;

import io.druid.guice.annotations.Json;
import io.druid.guice.annotations.Smile;
import java.io.InputStream;
import io.druid.server.metrics.EventReceiverFirehoseMetric;
import io.druid.server.metrics.EventReceiverFirehoseRegister;

import javax.servlet.http.HttpServletRequest;
import javax.ws.rs.Consumes;
import javax.ws.rs.POST;
Expand All @@ -48,6 +48,7 @@
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import java.io.IOException;
import java.io.InputStream;
import java.util.Collection;
import java.util.List;
import java.util.Map;
Expand All @@ -70,14 +71,16 @@ public class EventReceiverFirehoseFactory implements FirehoseFactory<MapInputRow
private final Optional<ChatHandlerProvider> chatHandlerProvider;
private final ObjectMapper jsonMapper;
private final ObjectMapper smileMapper;
private final EventReceiverFirehoseRegister eventReceiverFirehoseRegister;

@JsonCreator
public EventReceiverFirehoseFactory(
@JsonProperty("serviceName") String serviceName,
@JsonProperty("bufferSize") Integer bufferSize,
@JacksonInject ChatHandlerProvider chatHandlerProvider,
@JacksonInject @Json ObjectMapper jsonMapper,
@JacksonInject @Smile ObjectMapper smileMapper
@JacksonInject @Smile ObjectMapper smileMapper,
@JacksonInject EventReceiverFirehoseRegister eventReceiverFirehoseRegister
)
{
Preconditions.checkNotNull(serviceName, "serviceName");
Expand All @@ -87,13 +90,13 @@ public EventReceiverFirehoseFactory(
this.chatHandlerProvider = Optional.fromNullable(chatHandlerProvider);
this.jsonMapper = jsonMapper;
this.smileMapper = smileMapper;
this.eventReceiverFirehoseRegister = eventReceiverFirehoseRegister;
}

@Override
public Firehose connect(MapInputRowParser firehoseParser) throws IOException
{
log.info("Connecting firehose: %s", serviceName);

final EventReceiverFirehose firehose = new EventReceiverFirehose(firehoseParser);

if (chatHandlerProvider.isPresent()) {
Expand All @@ -106,6 +109,8 @@ public Firehose connect(MapInputRowParser firehoseParser) throws IOException
log.info("No chathandler detected");
}

eventReceiverFirehoseRegister.register(serviceName, firehose);

return firehose;
}

Expand All @@ -121,7 +126,7 @@ public int getBufferSize()
return bufferSize;
}

public class EventReceiverFirehose implements ChatHandler, Firehose
public class EventReceiverFirehose implements ChatHandler, Firehose, EventReceiverFirehoseMetric
{
private final BlockingQueue<InputRow> buffer;
private final MapInputRowParser parser;
Expand Down Expand Up @@ -243,12 +248,25 @@ public void run()
};
}

@Override
public int getCurrentBufferSize()
{
// ArrayBlockingQueue's implementation of size() is thread-safe, so we can use that
return buffer.size();
}

@Override
public int getCapacity()
{
return bufferSize;
}

@Override
public void close() throws IOException
{
log.info("Firehose closing.");
closed = true;

eventReceiverFirehoseRegister.unregister(serviceName);
if (chatHandlerProvider.isPresent()) {
chatHandlerProvider.get().unregister(serviceName);
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,41 @@
/*
* 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.server.metrics;

/**
* An EventReceiverFirehoseMetric is an object with metrics about EventReceiverFirehose objects.
* It is not likely that anything other than an EventReceiverFirehose actually implements this.
* This interface is not part of the public API and backwards incompatible changes can occur without
* requiring a major (or even minor) version change.
* The interface's primary purpose is to be able to share metrics via the EventReceiverFirehoseRegister
* without exposing the entire EventReceiverFirehose
*/
public interface EventReceiverFirehoseMetric
{
/**
* Return the current number of {@link io.druid.data.input.InputRow} that are stored in the buffer.
*/
int getCurrentBufferSize();

/**
* Return the capacity of the buffer.
*/
int getCapacity();
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
/*
* 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.server.metrics;

import com.google.inject.Inject;
import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.emitter.service.ServiceMetricEvent;
import com.metamx.metrics.AbstractMonitor;

import java.util.Map;

public class EventReceiverFirehoseMonitor extends AbstractMonitor
{

private final EventReceiverFirehoseRegister register;

@Inject
public EventReceiverFirehoseMonitor(
EventReceiverFirehoseRegister eventReceiverFirehoseRegister
)
{
this.register = eventReceiverFirehoseRegister;
}

@Override
public boolean doMonitor(ServiceEmitter emitter)
{
for (Map.Entry<String, EventReceiverFirehoseMetric> entry : register.getMetrics()) {
final String serviceName = entry.getKey();
final EventReceiverFirehoseMetric metric = entry.getValue();

final ServiceMetricEvent.Builder builder = ServiceMetricEvent.builder()
.setDimension("serviceName", serviceName)
.setDimension(
"bufferCapacity",
String.valueOf(metric.getCapacity())
);

emitter.emit(builder.build("ingest/events/buffered", metric.getCurrentBufferSize()));
}

return true;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,56 @@
/*
* 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.server.metrics;

import com.metamx.common.ISE;
import com.metamx.common.logger.Logger;

import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;

public class EventReceiverFirehoseRegister
{

private static final Logger log = new Logger(EventReceiverFirehoseRegister.class);

private final ConcurrentMap<String, EventReceiverFirehoseMetric> metrics = new ConcurrentHashMap<>();

public void register(String serviceName, EventReceiverFirehoseMetric metric)
{
log.info("Registering EventReceiverFirehoseMetric for service [%s]", serviceName);
if (metrics.putIfAbsent(serviceName, metric) != null) {
throw new ISE("Service [%s] is already registered!", serviceName);
}
}

public Iterable<Map.Entry<String, EventReceiverFirehoseMetric>> getMetrics()
{
return metrics.entrySet();
}

public void unregister(String serviceName)
{
log.info("Unregistering EventReceiverFirehoseMetric for service [%s]", serviceName);
if (metrics.remove(serviceName) == null) {
log.warn("Unregistering a non-exist service. Service [%s] never exists.");
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import io.druid.concurrent.Execs;
import io.druid.guice.DruidBinders;
import io.druid.guice.JsonConfigProvider;
import io.druid.guice.LazySingleton;
import io.druid.guice.ManageLifecycle;

import java.util.List;
Expand All @@ -59,6 +60,8 @@ public void configure(Binder binder)

DruidBinders.metricMonitorBinder(binder); // get the binder so that it will inject the empty set at a minimum.

binder.bind(EventReceiverFirehoseRegister.class).in(LazySingleton.class);

// Instantiate eagerly so that we get everything registered and put into the Lifecycle
binder.bind(Key.get(MonitorScheduler.class, Names.named("ForTheEagerness")))
.to(MonitorScheduler.class)
Expand Down
Loading