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
9 changes: 9 additions & 0 deletions docs/operations/metrics.md
Original file line number Diff line number Diff line change
Expand Up @@ -361,6 +361,15 @@ These metrics are only available if the `JVMMonitor` module is included.
|`jvm/gc/count`|Garbage collection count|`gcName` (cms/g1/parallel/etc.), `gcGen` (old/young)|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`|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)). |

### ZooKeeper

These metrics are available unless `druid.zk.service.enabled = false`.

|Metric|Description|Dimensions|Normal Value|
|------|-----------|----------|------------|
|`zk/connected`|Indicator of connection status. `1` for connected, `0` for disconnected. Emitted once per monitor period.|None|1|
|`zk/reconnect/time`|Amount of time, in milliseconds, that a server was disconnected from ZooKeeper before reconnecting. Emitted on reconnection. Not emitted if connection to ZooKeeper is permanently lost, because in this case, there is no reconnection.|None|Not present|

### EventReceiverFirehose

The following metric is only available if the `EventReceiverFirehoseMonitor` module is included.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3440,7 +3440,7 @@ public void testCheckpointForUnknownTaskGroup()
"Cannot find taskGroup [0] among all activelyReadingTaskGroups [{}]",
serviceEmitter.getExceptionMessage()
);
Assert.assertEquals(ISE.class, serviceEmitter.getExceptionClass());
Assert.assertEquals(ISE.class.getName(), serviceEmitter.getExceptionClass());
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3461,7 +3461,7 @@ public void testCheckpointForUnknownTaskGroup()
"Cannot find taskGroup [0] among all activelyReadingTaskGroups [{}]",
serviceEmitter.getExceptionMessage()
);
Assert.assertEquals(ISE.class, serviceEmitter.getExceptionClass());
Assert.assertEquals(ISE.class.getName(), serviceEmitter.getExceptionClass());
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -168,7 +168,11 @@ public void configure(Binder binder)
JsonConfigProvider.bind(binder, MetadataStorageTablesConfig.PROPERTY_BASE, MetadataStorageTablesConfig.class);

// Build from properties provided in the config
JsonConfigProvider.bind(binder, MetadataStorageConnectorConfig.PROPERTY_BASE, MetadataStorageConnectorConfig.class);
JsonConfigProvider.bind(
binder,
MetadataStorageConnectorConfig.PROPERTY_BASE,
MetadataStorageConnectorConfig.class
);
}

@Provides
Expand Down Expand Up @@ -327,7 +331,7 @@ public Builder test(Object test)
* <p>
* The builder registers {@code DruidNodeDiscoveryProvider} by default: add any
* test-specific instances as needed.
*/
*/
public Builder eagerInstance(Class<?> theClass)
{
this.eagerCreation.add(theClass);
Expand All @@ -343,7 +347,7 @@ public Builder modules(List<Module> modules)
return this;
}

public Builder modules(Module...modules)
public Builder modules(Module... modules)
{
return modules(Arrays.asList(modules));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,17 +29,11 @@
import org.apache.druid.java.util.emitter.service.ServiceEmitter;

import javax.annotation.Nullable;
import java.io.PrintWriter;
import java.io.StringWriter;

/**
*/
public class EmittingLogger extends Logger
{
public static final String EXCEPTION_TYPE_KEY = "exceptionType";
public static final String EXCEPTION_MESSAGE_KEY = "exceptionMessage";
public static final String EXCEPTION_STACK_TRACE_KEY = "exceptionStackTrace";

private static volatile ServiceEmitter emitter = null;

private final String className;
Expand Down Expand Up @@ -93,19 +87,8 @@ public AlertBuilder makeAlert(@Nullable Throwable t, String message, Object... o
throw e;
}

final AlertBuilder retVal = new EmittingAlertBuilder(t, StringUtils.format(message, objects), emitter)
return new EmittingAlertBuilder(t, StringUtils.format(message, objects), emitter)
.addData("class", className);

if (t != null) {
final StringWriter trace = new StringWriter();
final PrintWriter pw = new PrintWriter(trace);
t.printStackTrace(pw);
retVal.addData("exceptionType", t.getClass());
retVal.addData("exceptionMessage", t.getMessage());
retVal.addData("exceptionStackTrace", trace.toString());
}

return retVal;
}

public class EmittingAlertBuilder extends AlertBuilder
Expand All @@ -118,6 +101,7 @@ private EmittingAlertBuilder(Throwable t, String description, ServiceEmitter emi
{
super(description, emitter);
this.t = t;
addThrowable(t);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,12 +24,20 @@
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.StringUtils;

import javax.annotation.Nullable;
import java.io.PrintWriter;
import java.io.StringWriter;
import java.util.Map;

/**
*/
*
*/
public class AlertBuilder extends ServiceEventBuilder<AlertEvent>
{
public static final String EXCEPTION_TYPE_KEY = "exceptionType";
public static final String EXCEPTION_MESSAGE_KEY = "exceptionMessage";
public static final String EXCEPTION_STACK_TRACE_KEY = "exceptionStackTrace";

protected final Map<String, Object> dataMap = Maps.newLinkedHashMap();
protected final String description;
protected final ServiceEmitter emitter;
Expand Down Expand Up @@ -67,6 +75,20 @@ public AlertBuilder addData(Map<String, Object> data)
return this;
}

public AlertBuilder addThrowable(@Nullable final Throwable t)
{
if (t != null) {
final StringWriter trace = new StringWriter();
final PrintWriter pw = new PrintWriter(trace);
t.printStackTrace(pw);
addData(EXCEPTION_TYPE_KEY, t.getClass().getName());
addData(EXCEPTION_MESSAGE_KEY, t.getMessage());
addData(EXCEPTION_STACK_TRACE_KEY, trace.toString());
}

return this;
}

public AlertBuilder severity(AlertEvent.Severity severity)
{
this.severity = severity;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,54 @@
/*
* 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.emitter.service;

import com.google.common.collect.ImmutableMap;
import org.apache.druid.java.util.emitter.core.EventMap;
import org.junit.Assert;
import org.junit.Test;

import java.util.Map;

public class AlertBuilderTest
{
@Test
public void testAlertBuilder()
{
final AlertEvent alertEvent =
AlertBuilder.create("alert[%s]", "oops")
.addData(ImmutableMap.of("foo", "bar"))
.addData(ImmutableMap.of("baz", "qux"))
.addThrowable(new RuntimeException("an exception!"))
.build("druid/test", "example.com");

final EventMap alertMap = alertEvent.toMap();

Assert.assertEquals("alerts", alertMap.get("feed"));
Assert.assertEquals("alert[oops]", alertMap.get("description"));
Assert.assertEquals("druid/test", alertMap.get("service"));
Assert.assertEquals("example.com", alertMap.get("host"));

final Map<String, Object> dataMap = (Map<String, Object>) alertMap.get("data");
Assert.assertEquals("java.lang.RuntimeException", dataMap.get("exceptionType"));
Assert.assertEquals("an exception!", dataMap.get("exceptionMessage"));
Assert.assertEquals("bar", dataMap.get("foo"));
Assert.assertEquals("qux", dataMap.get("baz"));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@

package org.apache.druid.testing.junit;

import com.google.common.collect.ImmutableList;
import com.google.errorprone.annotations.concurrent.GuardedBy;
import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.core.LogEvent;
Expand All @@ -28,8 +30,8 @@
import org.apache.logging.log4j.core.config.LoggerConfig;
import org.junit.rules.ExternalResource;

import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.CopyOnWriteArrayList;

/**
* JUnit rule to capture a class's logger output to an in-memory buffer to allow verification of log messages in tests.
Expand Down Expand Up @@ -73,39 +75,66 @@ public void clearLogEvents()
inMemoryAppender.clearLogEvents();
}

/**
* Wait for the captured
*/
public void awaitLogEvents() throws InterruptedException
{
inMemoryAppender.awaitLogEvents();
}

private static class InMemoryAppender extends AbstractAppender
{
static final String NAME = InMemoryAppender.class.getName();

private final String targetLoggerName;

// logEvents has concurrent iteration and modification in CuratorModuleTest::exitsJvmWhenMaxRetriesExceeded(), needs to be thread safe
private final CopyOnWriteArrayList<LogEvent> logEvents;
@GuardedBy("logEvents")
private final List<LogEvent> logEvents;

InMemoryAppender(Class<?> targetClass)
{
super(NAME, null, null);
targetLoggerName = targetClass.getName();
logEvents = new CopyOnWriteArrayList<>();
logEvents = new ArrayList<>();
}

@Override
public void append(LogEvent logEvent)
{
if (logEvent.getLoggerName().equals(targetLoggerName)) {
logEvents.add(logEvent);
synchronized (logEvents) {
if (logEvent.getLoggerName().equals(targetLoggerName)) {
logEvents.add(logEvent);
logEvents.notifyAll();
}
}
}

List<LogEvent> getLogEvents()
{
return logEvents;
synchronized (logEvents) {
return ImmutableList.copyOf(logEvents);
}
}

void clearLogEvents()
{
logEvents.clear();
synchronized (logEvents) {
logEvents.clear();
}
}

/**
* Wait for "logEvents" to be nonempty. If it is already nonempty, return immediately.
*/
void awaitLogEvents() throws InterruptedException
{
synchronized (logEvents) {
while (logEvents.isEmpty()) {
logEvents.wait();
}
}
}
}
}

Loading