Add per-event JVM GC duration tracking

This commit is contained in:
Jesse Tuglu 2024-11-13 01:11:15 -08:00
parent 1dbd005df6
commit a3b7593528
10 changed files with 445 additions and 76 deletions

View File

@ -427,6 +427,8 @@ For more information, see [Enabling Metrics](../configuration/index.md#enabling-
|`jvm/mem/committed`|Committed memory|`memKind`, `jvmVersion`|Close to max memory|
|`jvm/gc/count`|Garbage collection count|`gcName` (cms/g1/parallel/etc.), `gcGen` (old/young), `jvmVersion`|Varies|
|`jvm/gc/cpu`|Count of CPU time in Nanoseconds spent on garbage collection. Note: `jvm/gc/cpu` represents the total time over multiple GC cycles; divide by `jvm/gc/count` to get the mean GC time per cycle.|`gcName`, `gcGen`, `jvmVersion`|Sum of `jvm/gc/cpu` should be within 10-30% of sum of `jvm/cpu/total`, depending on the GC algorithm used (reported by [`JvmCpuMonitor`](../configuration/index.md#enabling-metrics)). |
|`jvm/gc/pause`| Stop-the-world garbage collection JVM-reported pause time (ms). Only emitted if `druid.monitoring.jvm.duration=true`.| `gcName` (cms/g1/parallel/etc.), `gcGen` (old/young), `jvmVersion` | Varies|
|`jvm/gc/concurrentTime`| JVM-reported time spent in concurrent phases of CMS pauses (ms). Only emitted if `druid.monitoring.jvm.duration=true`.| `gcName`(cms/g1/etc.), `gcGen` (old/young), `jvmVersion` | Varies|
### ZooKeeper

View File

@ -0,0 +1,70 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.java.util.metrics;
import java.lang.reflect.Array;
// A fixed-size, thread-safe, append-only ring buffer for buffering events prior to emission.
// Events will overflow and wrap-around if buffer size is exceeded within a single emission window.
public class EventBuffer<Event>
{
private final Event[] buffer;
private final Class<Event> clazz;
private final int capacity;
private int back;
private int size;
public EventBuffer(Class<Event> clazz, int capacity)
{
this.clazz = clazz;
this.buffer = (Event[]) Array.newInstance(clazz, capacity);
this.capacity = capacity;
this.back = 0;
}
public synchronized int getCapacity()
{
return capacity;
}
public synchronized int getSize()
{
return size;
}
public synchronized void push(Event event)
{
buffer[back] = event;
back = (back + 1) % capacity;
if (size < capacity) {
++size;
}
}
public synchronized Event[] extract()
{
final Event[] finalEvents = (Event[]) Array.newInstance(clazz, size);
System.arraycopy(buffer, 0, finalEvents, 0, size);
size = back = 0;
return finalEvents;
}
}

View File

@ -22,11 +22,16 @@ package org.apache.druid.java.util.metrics;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.java.util.common.Pair;
import com.sun.management.GarbageCollectionNotificationInfo;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
import org.apache.druid.java.util.metrics.jvm.gc.GcEvent;
import javax.annotation.Nullable;
import javax.management.Notification;
import javax.management.NotificationEmitter;
import javax.management.NotificationListener;
import javax.management.openmbean.CompositeData;
import java.lang.management.BufferPoolMXBean;
import java.lang.management.GarbageCollectorMXBean;
import java.lang.management.ManagementFactory;
@ -42,30 +47,37 @@ public class JvmMonitor extends FeedDefiningMonitor
private static final String JVM_VERSION = "jvmVersion";
private static final String JAVA_VERSION = System.getProperty("java.version");
private final JvmMonitorConfig config;
@VisibleForTesting
@Nullable
final GcCollectors gcCollectors;
private final Map<String, String[]> dimensions;
@Nullable
private final AllocationMetricCollector collector;
@Nullable
final GcGranularEventCollector gcEventCollector;
public JvmMonitor()
public JvmMonitor(JvmMonitorConfig config)
{
this(ImmutableMap.of());
this(ImmutableMap.of(), config);
}
public JvmMonitor(Map<String, String[]> dimensions)
public JvmMonitor(Map<String, String[]> dimensions, JvmMonitorConfig config)
{
this(dimensions, DEFAULT_METRICS_FEED);
this(dimensions, DEFAULT_METRICS_FEED, config);
}
public JvmMonitor(Map<String, String[]> dimensions, String feed)
public JvmMonitor(Map<String, String[]> dimensions, String feed, JvmMonitorConfig config)
{
super(feed);
Preconditions.checkNotNull(dimensions);
this.config = config;
this.dimensions = ImmutableMap.copyOf(dimensions);
this.collector = AllocationMetricCollectors.getAllocationMetricCollector();
this.gcCollectors = new GcCollectors();
this.gcEventCollector = config.recordDuration() ? new GcGranularEventCollector() : null;
}
@Override
@ -150,6 +162,9 @@ public class JvmMonitor extends FeedDefiningMonitor
private void emitGcMetrics(ServiceEmitter emitter)
{
gcCollectors.emit(emitter, dimensions);
if (gcEventCollector != null) {
gcEventCollector.emit(emitter, dimensions);
}
}
private class GcCollectors
@ -188,79 +203,25 @@ public class JvmMonitor extends FeedDefiningMonitor
private class GcGenerationCollector
{
private static final String GC_YOUNG_GENERATION_NAME = "young";
private static final String GC_OLD_GENERATION_NAME = "old";
private static final String GC_ZGC_GENERATION_NAME = "zgc";
private static final String CMS_COLLECTOR_NAME = "cms";
private static final String G1_COLLECTOR_NAME = "g1";
private static final String PARALLEL_COLLECTOR_NAME = "parallel";
private static final String SERIAL_COLLECTOR_NAME = "serial";
private static final String ZGC_COLLECTOR_NAME = "zgc";
private static final String SHENANDOAN_COLLECTOR_NAME = "shenandoah";
private final String generation;
private final String collectorName;
private final GcEvent event;
private final GarbageCollectorMXBean gcBean;
private long lastInvocations = 0;
private long lastCpuMillis = 0;
GcGenerationCollector(GarbageCollectorMXBean gcBean)
{
Pair<String, String> gcNamePair = getReadableName(gcBean.getName());
this.generation = gcNamePair.lhs;
this.collectorName = gcNamePair.rhs;
this.event = new GcEvent(gcBean.getName());
this.gcBean = gcBean;
}
private Pair<String, String> getReadableName(String name)
{
switch (name) {
//CMS
case "ParNew":
return new Pair<>(GC_YOUNG_GENERATION_NAME, CMS_COLLECTOR_NAME);
case "ConcurrentMarkSweep":
return new Pair<>(GC_OLD_GENERATION_NAME, CMS_COLLECTOR_NAME);
// G1
case "G1 Young Generation":
return new Pair<>(GC_YOUNG_GENERATION_NAME, G1_COLLECTOR_NAME);
case "G1 Old Generation":
return new Pair<>(GC_OLD_GENERATION_NAME, G1_COLLECTOR_NAME);
// Parallel
case "PS Scavenge":
return new Pair<>(GC_YOUNG_GENERATION_NAME, PARALLEL_COLLECTOR_NAME);
case "PS MarkSweep":
return new Pair<>(GC_OLD_GENERATION_NAME, PARALLEL_COLLECTOR_NAME);
// Serial
case "Copy":
return new Pair<>(GC_YOUNG_GENERATION_NAME, SERIAL_COLLECTOR_NAME);
case "MarkSweepCompact":
return new Pair<>(GC_OLD_GENERATION_NAME, SERIAL_COLLECTOR_NAME);
//zgc
case "ZGC":
return new Pair<>(GC_ZGC_GENERATION_NAME, ZGC_COLLECTOR_NAME);
//Shenandoah
case "Shenandoah Cycles":
return new Pair<>(GC_YOUNG_GENERATION_NAME, SHENANDOAN_COLLECTOR_NAME);
case "Shenandoah Pauses":
return new Pair<>(GC_OLD_GENERATION_NAME, SHENANDOAN_COLLECTOR_NAME);
default:
return new Pair<>(name, name);
}
}
void emit(ServiceEmitter emitter, Map<String, String[]> dimensions)
{
ImmutableMap.Builder<String, String[]> dimensionsCopyBuilder = ImmutableMap
.<String, String[]>builder()
.putAll(dimensions)
.put("gcGen", new String[]{generation});
.put("gcGen", new String[]{event.druidGenerationName});
dimensionsCopyBuilder.put("gcName", new String[]{collectorName});
dimensionsCopyBuilder.put("gcName", new String[]{event.druidCollectorName});
Map<String, String[]> dimensionsCopy = dimensionsCopyBuilder.build();
@ -323,4 +284,63 @@ public class JvmMonitor extends FeedDefiningMonitor
emitter.emit(builder.setMetric("jvm/gc/mem/init", memoryUsage.getInit()));
}
}
private class GcGranularEventCollector
{
// From: https://github.com/Netflix/spectator/blob/main/spectator-ext-gc/src/main/java/com/netflix/spectator/gc/GcLogger.java#L56
private static final int BUFFER_SIZE = 256;
private final EventBuffer<ServiceMetricEvent.Builder> buffer;
private final GcNotificationListener listener;
public GcGranularEventCollector()
{
this.buffer = new EventBuffer<>(ServiceMetricEvent.Builder.class, BUFFER_SIZE);
this.listener = new GcNotificationListener();
for (GarbageCollectorMXBean mbean : ManagementFactory.getGarbageCollectorMXBeans()) {
if (mbean instanceof NotificationEmitter) {
final NotificationEmitter emitter = (NotificationEmitter) mbean;
emitter.addNotificationListener(this.listener, null, null);
}
}
}
void emit(ServiceEmitter emitter, Map<String, String[]> dimensions)
{
final ServiceMetricEvent.Builder[] events = buffer.extract();
for (ServiceMetricEvent.Builder builder : events) {
MonitorUtils.addDimensionsToBuilder(builder, dimensions);
emitter.emit(builder);
}
}
private void processGcEvent(GarbageCollectionNotificationInfo info)
{
final ServiceMetricEvent.Builder builder = builder();
final GcEvent event = new GcEvent(info.getGcName(), info.getGcCause());
builder.setDimension("gcName", new String[]{event.druidCollectorName});
builder.setDimension("gcGen", new String[]{event.druidGenerationName});
builder.setDimension(JVM_VERSION, JAVA_VERSION);
// record pause time or concurrent time
final String durationMetricName = event.isConcurrent() ? "jvm/gc/concurrentTime" : "jvm/gc/pause";
builder.setMetric(durationMetricName, info.getGcInfo().getDuration());
buffer.push(builder);
}
private class GcNotificationListener implements NotificationListener
{
@Override
public void handleNotification(Notification notification, Object ref)
{
final String type = notification.getType();
if (GarbageCollectionNotificationInfo.GARBAGE_COLLECTION_NOTIFICATION.equals(type)) {
CompositeData cd = (CompositeData) notification.getUserData();
GarbageCollectionNotificationInfo info = GarbageCollectionNotificationInfo.from(cd);
processGcEvent(info);
}
}
}
}
}

View File

@ -0,0 +1,21 @@
package org.apache.druid.java.util.metrics;
import com.fasterxml.jackson.annotation.JsonProperty;
public class JvmMonitorConfig
{
// The JVMMonitor is really more like a JVM memory + GC monitor
public static final String PREFIX = "druid.monitoring.jvm";
@JsonProperty("duration")
private boolean duration = false;
public boolean recordDuration() {
return duration;
}
public JvmMonitorConfig(@JsonProperty("duration") final boolean duration)
{
this.duration = duration;
}
}

View File

@ -25,36 +25,38 @@ import java.util.Map;
public class Monitors
{
/**
* Creates a JVM monitor, configured with the given dimensions, that gathers all currently available JVM-wide
* Creates a JVM monitor, configured with the given dimensions and config that gathers all currently available JVM-wide
* monitors. Emitted events have default feed {@link FeedDefiningMonitor#DEFAULT_METRICS_FEED}
* See: {@link Monitors#createCompoundJvmMonitor(Map, String)}
* See: {@link Monitors#createCompoundJvmMonitor(Map, String, JvmMonitorConfig)}
*
* @param dimensions common dimensions to configure the JVM monitor with
* @param config JVM-wide monitor config
*
* @return a universally useful JVM-wide monitor
*/
public static Monitor createCompoundJvmMonitor(Map<String, String[]> dimensions)
public static Monitor createCompoundJvmMonitor(Map<String, String[]> dimensions, JvmMonitorConfig config)
{
return createCompoundJvmMonitor(dimensions, FeedDefiningMonitor.DEFAULT_METRICS_FEED);
return createCompoundJvmMonitor(dimensions, FeedDefiningMonitor.DEFAULT_METRICS_FEED, config);
}
/**
* Creates a JVM monitor, configured with the given dimensions, that gathers all currently available JVM-wide
* Creates a JVM monitor, configured with the given dimensions and config that gathers all currently available JVM-wide
* monitors: {@link JvmMonitor}, {@link JvmCpuMonitor} and {@link JvmThreadsMonitor} (this list may
* change in any future release of this library, including a minor release).
*
* @param dimensions common dimensions to configure the JVM monitor with
* @param feed feed for all emitted events
* @param config JVM-wide monitor config
*
* @return a universally useful JVM-wide monitor
*/
public static Monitor createCompoundJvmMonitor(Map<String, String[]> dimensions, String feed)
public static Monitor createCompoundJvmMonitor(Map<String, String[]> dimensions, String feed, JvmMonitorConfig config)
{
// This list doesn't include SysMonitor because it should probably be run only in one JVM, if several JVMs are
// running on the same instance, so most of the time SysMonitor should be configured/set up differently than
// "simple" JVM monitors, created below.
return and(// Could equally be or(), because all member monitors always return true from their monitor() methods.
new JvmMonitor(dimensions, feed),
new JvmMonitor(dimensions, feed, config),
new JvmCpuMonitor(dimensions, feed),
new JvmThreadsMonitor(dimensions, feed)
);

View File

@ -0,0 +1,115 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.java.util.metrics.jvm.gc;
import org.apache.druid.java.util.common.Pair;
import java.util.Optional;
public class GcEvent
{
private static final String GC_YOUNG_GENERATION_NAME = "young";
private static final String GC_OLD_GENERATION_NAME = "old";
private static final String GC_ZGC_GENERATION_NAME = "zgc";
private static final String GC_G1_CONCURRENT_GENERATION_NAME = "g1_concurrent";
private static final String CMS_COLLECTOR_NAME = "cms";
private static final String G1_COLLECTOR_NAME = "g1";
private static final String PARALLEL_COLLECTOR_NAME = "parallel";
private static final String SERIAL_COLLECTOR_NAME = "serial";
private static final String ZGC_COLLECTOR_NAME = "zgc";
private static final String SHENANDOAN_COLLECTOR_NAME = "shenandoah";
public final String druidGenerationName;
public final String druidCollectorName;
public final String jvmGcName;
public final Optional<String> jvmGcCause;
public GcEvent(final String jvmGcName)
{
this.jvmGcName = jvmGcName;
this.jvmGcCause = Optional.empty();
final Pair<String, String> druidNames = convertEventName(jvmGcName);
this.druidGenerationName = druidNames.lhs;
this.druidCollectorName = druidNames.rhs;
}
public GcEvent(final String jvmGcName, final String jvmGcCause)
{
this.jvmGcName = jvmGcName;
this.jvmGcCause = Optional.of(jvmGcCause);
final Pair<String, String> druidNames = convertEventName(jvmGcName);
this.druidGenerationName = druidNames.lhs;
this.druidCollectorName = druidNames.rhs;
}
private static Pair<String, String> convertEventName(final String jvmGcName)
{
switch (jvmGcName) {
// CMS
case "ParNew":
return new Pair<>(GC_YOUNG_GENERATION_NAME, CMS_COLLECTOR_NAME);
case "ConcurrentMarkSweep":
return new Pair<>(GC_OLD_GENERATION_NAME, CMS_COLLECTOR_NAME);
// G1
case "G1 Young Generation":
return new Pair<>(GC_YOUNG_GENERATION_NAME, G1_COLLECTOR_NAME);
case "G1 Old Generation":
return new Pair<>(GC_OLD_GENERATION_NAME, G1_COLLECTOR_NAME);
case "G1 Concurrent GC":
return new Pair<>(GC_G1_CONCURRENT_GENERATION_NAME, G1_COLLECTOR_NAME);
// Parallel
case "PS Scavenge":
return new Pair<>(GC_YOUNG_GENERATION_NAME, PARALLEL_COLLECTOR_NAME);
case "PS MarkSweep":
return new Pair<>(GC_OLD_GENERATION_NAME, PARALLEL_COLLECTOR_NAME);
// Serial
case "Copy":
return new Pair<>(GC_YOUNG_GENERATION_NAME, SERIAL_COLLECTOR_NAME);
case "MarkSweepCompact":
return new Pair<>(GC_OLD_GENERATION_NAME, SERIAL_COLLECTOR_NAME);
// zgc
case "ZGC":
return new Pair<>(GC_ZGC_GENERATION_NAME, ZGC_COLLECTOR_NAME);
// Shenandoah
case "Shenandoah Cycles":
return new Pair<>(GC_YOUNG_GENERATION_NAME, SHENANDOAN_COLLECTOR_NAME);
case "Shenandoah Pauses":
return new Pair<>(GC_OLD_GENERATION_NAME, SHENANDOAN_COLLECTOR_NAME);
default:
return new Pair<>(jvmGcName, jvmGcName);
}
}
public boolean isConcurrent()
{
return (
jvmGcName.endsWith(" Cycles") ||
druidGenerationName.equals(GC_G1_CONCURRENT_GENERATION_NAME) ||
(jvmGcCause.isPresent() && jvmGcCause.get().equals("No GC")));
}
}

View File

@ -0,0 +1,73 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.java.util.metrics;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
public class EventBufferTest
{
private EventBuffer<ServiceMetricEvent> buffer;
private final int capacity = 5;
@BeforeEach
public void setUp()
{
buffer = new EventBuffer<>(ServiceMetricEvent.class, capacity);
}
@Test
public void testInsertAndSize()
{
fill(capacity);
buffer.push(ServiceMetricEvent.builder().setMetric("my/other/test/metric", capacity).build(ImmutableMap.of()));
Assertions.assertEquals(capacity, buffer.getSize(), "Size should not exceed capacity");
}
@Test
public void testExtractAndSize()
{
int numPush = capacity + 1;
fill(numPush);
ServiceMetricEvent[] extractedEvents = buffer.extract();
for (int i = numPush - 1; i >= numPush - capacity; --i) {
ServiceMetricEvent event = extractedEvents[i % capacity];
Assertions.assertEquals("my/test/metric", event.getMetric(), "Metric name incorrect");
Assertions.assertEquals(i, event.getValue(), "Metric value should equal index");
}
Assertions.assertEquals(0, buffer.getSize(), "EventBuffer::extract() should clear size");
}
private void fill(int n)
{
for (int i = 0; i < n; ++i) {
ServiceMetricEvent.Builder builder = ServiceMetricEvent.builder();
builder.setMetric("my/test/metric", i);
buffer.push(builder.build(ImmutableMap.of()));
Assertions.assertEquals(Math.min(i + 1, capacity), buffer.getSize(), "Size should not exceed capacity");
}
}
}

View File

@ -28,6 +28,7 @@ import org.junit.Assume;
import org.junit.Test;
import java.util.List;
import java.util.concurrent.atomic.AtomicLong;
public class JvmMonitorTest
{
@ -39,7 +40,7 @@ public class JvmMonitorTest
final ServiceEmitter serviceEmitter = new ServiceEmitter("test", "localhost", emitter);
serviceEmitter.start();
final JvmMonitor jvmMonitor = new JvmMonitor();
final JvmMonitor jvmMonitor = new JvmMonitor(new JvmMonitorConfig(false));
// skip tests if gc counters fail to initialize with this JDK
Assume.assumeNotNull(jvmMonitor.gcCollectors);
@ -56,6 +57,30 @@ public class JvmMonitorTest
}
}
@Test(timeout = 60_000L)
public void testGcEventDurationMetrics() throws InterruptedException
{
GcEventDurationTrackingEmitter emitter = new GcEventDurationTrackingEmitter();
final ServiceEmitter serviceEmitter = new ServiceEmitter("test", "localhost", emitter);
serviceEmitter.start();
final JvmMonitor jvmMonitor = new JvmMonitor(new JvmMonitorConfig(true));
// skip tests if gc counters fail to initialize with this JDK
Assume.assumeNotNull(jvmMonitor.gcEventCollector);
while (true) {
// generate some garbage to see gc events occur
@SuppressWarnings("unused")
byte[] b = new byte[1024 * 1024 * 50];
emitter.reset();
jvmMonitor.doMonitor(serviceEmitter);
if (emitter.eventDurationMetricSeen()) {
return;
}
Thread.sleep(10);
}
}
private static class GcTrackingEmitter implements Emitter
{
private Number oldGcCount;
@ -147,4 +172,42 @@ public class JvmMonitorTest
}
}
private static class GcEventDurationTrackingEmitter implements Emitter
{
private final AtomicLong numGcEvents = new AtomicLong(0);
@Override
public void start() {}
@Override
public void emit(Event e)
{
ServiceMetricEvent event = (ServiceMetricEvent) e;
switch (event.getMetric()) {
case "jvm/gc/pause":
case "jvm/gc/concurrentTime":
numGcEvents.incrementAndGet();
default:
break;
}
}
@Override
public void flush() {}
@Override
public void close() {}
void reset()
{
numGcEvents.set(0);
}
boolean eventDurationMetricSeen()
{
return numGcEvents.get() > 0;
}
}
}

View File

@ -45,7 +45,7 @@ public class MonitorsTest
{
String feed = "testFeed";
StubServiceEmitter emitter = new StubServiceEmitter("dev/monitor-test", "localhost:0000");
Monitor m = Monitors.createCompoundJvmMonitor(ImmutableMap.of(), feed);
Monitor m = Monitors.createCompoundJvmMonitor(ImmutableMap.of(), feed, new JvmMonitorConfig(false));
m.start();
m.monitor(emitter);
m.stop();
@ -56,7 +56,7 @@ public class MonitorsTest
public void testDefaultFeed()
{
StubServiceEmitter emitter = new StubServiceEmitter("dev/monitor-test", "localhost:0000");
Monitor m = Monitors.createCompoundJvmMonitor(ImmutableMap.of());
Monitor m = Monitors.createCompoundJvmMonitor(ImmutableMap.of(), new JvmMonitorConfig(false));
m.start();
m.monitor(emitter);
m.stop();

View File

@ -44,6 +44,7 @@ import org.apache.druid.java.util.metrics.ClockDriftSafeMonitorScheduler;
import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig;
import org.apache.druid.java.util.metrics.JvmCpuMonitor;
import org.apache.druid.java.util.metrics.JvmMonitor;
import org.apache.druid.java.util.metrics.JvmMonitorConfig;
import org.apache.druid.java.util.metrics.JvmThreadsMonitor;
import org.apache.druid.java.util.metrics.Monitor;
import org.apache.druid.java.util.metrics.MonitorScheduler;
@ -88,6 +89,7 @@ public class MetricsModule implements Module
JsonConfigProvider.bind(binder, MONITORING_PROPERTY_PREFIX, DruidMonitorSchedulerConfig.class);
JsonConfigProvider.bind(binder, MONITORING_PROPERTY_PREFIX, MonitorsConfig.class);
JsonConfigProvider.bind(binder, OshiSysMonitorConfig.PREFIX, OshiSysMonitorConfig.class);
JsonConfigProvider.bind(binder, JvmMonitorConfig.PREFIX, JvmMonitorConfig.class);
DruidBinders.metricMonitorBinder(binder); // get the binder so that it will inject the empty set at a minimum.
@ -151,14 +153,15 @@ public class MetricsModule implements Module
@Provides
@ManageLifecycle
public JvmMonitor getJvmMonitor(
DataSourceTaskIdHolder dataSourceTaskIdHolder
DataSourceTaskIdHolder dataSourceTaskIdHolder,
JvmMonitorConfig config
)
{
Map<String, String[]> dimensions = MonitorsConfig.mapOfDatasourceAndTaskID(
dataSourceTaskIdHolder.getDataSource(),
dataSourceTaskIdHolder.getTaskId()
);
return new JvmMonitor(dimensions);
return new JvmMonitor(dimensions, config);
}
@Provides