mirror of https://github.com/apache/druid.git
Add per-event JVM GC duration tracking
This commit is contained in:
parent
1dbd005df6
commit
a3b7593528
|
@ -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
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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)
|
||||
);
|
||||
|
|
|
@ -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")));
|
||||
}
|
||||
}
|
|
@ -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");
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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
|
||||
|
|
Loading…
Reference in New Issue