YARN-9615. Add dispatcher metrics to RM. Contributed by Qi Zhu.
This commit is contained in:
parent
c3aa413ee3
commit
369f75b7a7
|
@ -26,6 +26,9 @@ import java.util.Map;
|
|||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
|
||||
import org.apache.hadoop.yarn.metrics.EventTypeMetrics;
|
||||
import org.apache.hadoop.yarn.util.Clock;
|
||||
import org.apache.hadoop.yarn.util.MonotonicClock;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.slf4j.Marker;
|
||||
|
@ -85,6 +88,11 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
|
|||
protected final Map<Class<? extends Enum>, EventHandler> eventDispatchers;
|
||||
private boolean exitOnDispatchException = true;
|
||||
|
||||
private Map<Class<? extends Enum>,
|
||||
EventTypeMetrics> eventTypeMetricsMap;
|
||||
|
||||
private Clock clock = new MonotonicClock();
|
||||
|
||||
/**
|
||||
* The thread name for dispatcher.
|
||||
*/
|
||||
|
@ -98,6 +106,8 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
|
|||
super("Dispatcher");
|
||||
this.eventQueue = eventQueue;
|
||||
this.eventDispatchers = new HashMap<Class<? extends Enum>, EventHandler>();
|
||||
this.eventTypeMetricsMap = new HashMap<Class<? extends Enum>,
|
||||
EventTypeMetrics>();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -135,7 +145,16 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
|
|||
return;
|
||||
}
|
||||
if (event != null) {
|
||||
dispatch(event);
|
||||
if (eventTypeMetricsMap.
|
||||
get(event.getType().getDeclaringClass()) != null) {
|
||||
long startTime = clock.getTime();
|
||||
dispatch(event);
|
||||
eventTypeMetricsMap.get(event.getType().getDeclaringClass())
|
||||
.increment(event.getType(),
|
||||
clock.getTime() - startTime);
|
||||
} else {
|
||||
dispatch(event);
|
||||
}
|
||||
if (printTrigger) {
|
||||
//Log the latest dispatch event type
|
||||
// may cause the too many events queued
|
||||
|
@ -369,4 +388,9 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
|
|||
protected boolean isStopped() {
|
||||
return stopped;
|
||||
}
|
||||
|
||||
public void addMetrics(EventTypeMetrics metrics,
|
||||
Class<? extends Enum> eventClass) {
|
||||
eventTypeMetricsMap.put(eventClass, metrics);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,6 +19,9 @@
|
|||
package org.apache.hadoop.yarn.event;
|
||||
|
||||
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hadoop.yarn.metrics.EventTypeMetrics;
|
||||
import org.apache.hadoop.yarn.util.Clock;
|
||||
import org.apache.hadoop.yarn.util.MonotonicClock;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.slf4j.Marker;
|
||||
|
@ -47,12 +50,15 @@ public class EventDispatcher<T extends Event> extends
|
|||
private final Thread eventProcessor;
|
||||
private volatile boolean stopped = false;
|
||||
private boolean shouldExitOnError = true;
|
||||
private EventTypeMetrics metrics;
|
||||
|
||||
private static final Logger LOG =
|
||||
LoggerFactory.getLogger(EventDispatcher.class);
|
||||
private static final Marker FATAL =
|
||||
MarkerFactory.getMarker("FATAL");
|
||||
|
||||
private Clock clock = new MonotonicClock();
|
||||
|
||||
private final class EventProcessor implements Runnable {
|
||||
@Override
|
||||
public void run() {
|
||||
|
@ -68,7 +74,14 @@ public class EventDispatcher<T extends Event> extends
|
|||
}
|
||||
|
||||
try {
|
||||
handler.handle(event);
|
||||
if (metrics != null) {
|
||||
long startTime = clock.getTime();
|
||||
handler.handle(event);
|
||||
metrics.increment(event.getType(),
|
||||
clock.getTime() - startTime);
|
||||
} else {
|
||||
handler.handle(event);
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
// An error occurred, but we are shutting down anyway.
|
||||
// If it was an InterruptedException, the very act of
|
||||
|
@ -136,4 +149,9 @@ public class EventDispatcher<T extends Event> extends
|
|||
public void disableExitOnError() {
|
||||
shouldExitOnError = false;
|
||||
}
|
||||
|
||||
public void setMetrics(EventTypeMetrics metrics) {
|
||||
this.metrics = metrics;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,42 @@
|
|||
/**
|
||||
* 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.hadoop.yarn.metrics;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.metrics2.MetricsCollector;
|
||||
import org.apache.hadoop.metrics2.annotation.Metrics;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
@Metrics(context="yarn")
|
||||
public class DisableEventTypeMetrics implements EventTypeMetrics {
|
||||
@Override
|
||||
public void increment(Enum type, long processingTimeUs) {
|
||||
//nop
|
||||
return;
|
||||
}
|
||||
@Override
|
||||
public void getMetrics(MetricsCollector collector, boolean all) {
|
||||
//nop
|
||||
return;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long get(Enum type) {
|
||||
return 0;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,32 @@
|
|||
/**
|
||||
* 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.hadoop.yarn.metrics;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.metrics2.MetricsSource;
|
||||
import org.apache.hadoop.metrics2.annotation.Metrics;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
@Metrics(context="yarn")
|
||||
public interface EventTypeMetrics<T extends Enum<T>>
|
||||
extends MetricsSource {
|
||||
|
||||
void increment(T type, long processingTimeUs);
|
||||
|
||||
long get(T type);
|
||||
}
|
|
@ -0,0 +1,160 @@
|
|||
/**
|
||||
* 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.hadoop.yarn.metrics;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.metrics2.MetricsCollector;
|
||||
import org.apache.hadoop.metrics2.MetricsInfo;
|
||||
import org.apache.hadoop.metrics2.MetricsSystem;
|
||||
import org.apache.hadoop.metrics2.annotation.Metrics;
|
||||
import org.apache.hadoop.metrics2.lib.MetricsRegistry;
|
||||
import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.util.EnumMap;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
@Metrics(context="yarn")
|
||||
public class GenericEventTypeMetrics<T extends Enum<T>>
|
||||
implements EventTypeMetrics<T> {
|
||||
|
||||
static final Logger LOG =
|
||||
LoggerFactory.getLogger(GenericEventTypeMetrics.class);
|
||||
|
||||
private final EnumMap<T, MutableGaugeLong> eventCountMetrics;
|
||||
private final EnumMap<T, MutableGaugeLong> processingTimeMetrics;
|
||||
private final MetricsRegistry registry;
|
||||
private final MetricsSystem ms;
|
||||
private final MetricsInfo info;
|
||||
private final Class<T> enumClass;
|
||||
|
||||
private boolean isInitialized = false;
|
||||
|
||||
public GenericEventTypeMetrics(MetricsInfo info, MetricsSystem ms,
|
||||
final T[] enums, Class<T> enumClass) {
|
||||
this.enumClass = enumClass;
|
||||
this.eventCountMetrics = new EnumMap<>(this.enumClass);
|
||||
this.processingTimeMetrics = new EnumMap<>(this.enumClass);
|
||||
this.ms = ms;
|
||||
this.info = info;
|
||||
this.registry = new MetricsRegistry(this.info);
|
||||
|
||||
//Initialize enum
|
||||
for (final T type : enums) {
|
||||
String eventCountMetricsName =
|
||||
type.toString() + "_" + "event_count";
|
||||
String processingTimeMetricsName =
|
||||
type.toString() + "_" + "processing_time";
|
||||
eventCountMetrics.put(type, this.registry.
|
||||
newGauge(eventCountMetricsName, eventCountMetricsName, 0L));
|
||||
processingTimeMetrics.put(type, this.registry.
|
||||
newGauge(processingTimeMetricsName, processingTimeMetricsName, 0L));
|
||||
}
|
||||
}
|
||||
|
||||
public synchronized GenericEventTypeMetrics registerMetrics() {
|
||||
if (!isInitialized) {
|
||||
// Register with the MetricsSystems
|
||||
if (this.ms != null) {
|
||||
LOG.info("Registering GenericEventTypeMetrics");
|
||||
ms.register(info.name(),
|
||||
info.description(), this);
|
||||
isInitialized = true;
|
||||
}
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void increment(T type, long processingTimeUs) {
|
||||
if (eventCountMetrics.get(type) != null) {
|
||||
eventCountMetrics.get(type).incr();
|
||||
processingTimeMetrics.get(type).incr(processingTimeUs);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long get(T type) {
|
||||
return eventCountMetrics.get(type).value();
|
||||
}
|
||||
|
||||
public long getTotalProcessingTime(T type) {
|
||||
return processingTimeMetrics.get(type).value();
|
||||
}
|
||||
|
||||
public EnumMap<T, MutableGaugeLong> getEventCountMetrics() {
|
||||
return eventCountMetrics;
|
||||
}
|
||||
|
||||
public EnumMap<T, MutableGaugeLong> getProcessingTimeMetrics() {
|
||||
return processingTimeMetrics;
|
||||
}
|
||||
|
||||
public MetricsRegistry getRegistry() {
|
||||
return registry;
|
||||
}
|
||||
|
||||
public MetricsInfo getInfo() {
|
||||
return info;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getMetrics(MetricsCollector collector, boolean all) {
|
||||
registry.snapshot(collector.addRecord(registry.info()), all);
|
||||
}
|
||||
|
||||
public Class<T> getEnumClass() {
|
||||
return enumClass;
|
||||
}
|
||||
|
||||
/** Builder class for GenericEventTypeMetrics. */
|
||||
public static class EventTypeMetricsBuilder<T extends Enum<T>>{
|
||||
public EventTypeMetricsBuilder() {
|
||||
}
|
||||
|
||||
public EventTypeMetricsBuilder setEnumClass(Class<T> enumClassValue) {
|
||||
this.enumClass = enumClassValue;
|
||||
return this;
|
||||
}
|
||||
|
||||
public EventTypeMetricsBuilder setEnums(T[] enumsValue) {
|
||||
this.enums = enumsValue.clone();
|
||||
return this;
|
||||
}
|
||||
|
||||
public EventTypeMetricsBuilder setInfo(MetricsInfo infoValue) {
|
||||
this.info = infoValue;
|
||||
return this;
|
||||
}
|
||||
|
||||
public EventTypeMetricsBuilder setMs(MetricsSystem msValue) {
|
||||
this.ms = msValue;
|
||||
return this;
|
||||
}
|
||||
|
||||
public GenericEventTypeMetrics build() {
|
||||
return new GenericEventTypeMetrics(info, ms, enums, enumClass);
|
||||
}
|
||||
|
||||
private MetricsSystem ms;
|
||||
private MetricsInfo info;
|
||||
private Class<T> enumClass;
|
||||
private T[] enums;
|
||||
}
|
||||
}
|
|
@ -20,9 +20,20 @@ package org.apache.hadoop.yarn.event;
|
|||
|
||||
import java.lang.reflect.Field;
|
||||
import java.lang.reflect.Modifier;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
|
||||
|
||||
import org.apache.hadoop.metrics2.AbstractMetric;
|
||||
import org.apache.hadoop.metrics2.MetricsRecord;
|
||||
import org.apache.hadoop.metrics2.impl.MetricsCollectorImpl;
|
||||
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
||||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
import org.apache.hadoop.yarn.metrics.GenericEventTypeMetrics;
|
||||
import org.slf4j.Logger;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
|
@ -30,6 +41,7 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import static org.apache.hadoop.metrics2.lib.Interns.info;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.mockito.Mockito.*;
|
||||
|
||||
|
@ -118,7 +130,7 @@ public class TestAsyncDispatcher {
|
|||
}
|
||||
|
||||
private enum TestEnum {
|
||||
TestEventType
|
||||
TestEventType, TestEventType2
|
||||
}
|
||||
|
||||
@SuppressWarnings({ "rawtypes", "unchecked" })
|
||||
|
@ -230,5 +242,171 @@ public class TestAsyncDispatcher {
|
|||
}
|
||||
}
|
||||
|
||||
}
|
||||
@Test
|
||||
public void testMetricsForDispatcher() throws Exception {
|
||||
YarnConfiguration conf = new YarnConfiguration();
|
||||
AsyncDispatcher dispatcher = null;
|
||||
|
||||
try {
|
||||
dispatcher = new AsyncDispatcher("RM Event dispatcher");
|
||||
|
||||
GenericEventTypeMetrics genericEventTypeMetrics =
|
||||
new GenericEventTypeMetrics.EventTypeMetricsBuilder()
|
||||
.setMs(DefaultMetricsSystem.instance())
|
||||
.setInfo(info("GenericEventTypeMetrics for "
|
||||
+ TestEnum.class.getName(),
|
||||
"Metrics for " + dispatcher.getName()))
|
||||
.setEnumClass(TestEnum.class)
|
||||
.setEnums(TestEnum.class.getEnumConstants())
|
||||
.build().registerMetrics();
|
||||
|
||||
// We can the metrics enabled for TestEnum
|
||||
dispatcher.addMetrics(genericEventTypeMetrics,
|
||||
genericEventTypeMetrics.getEnumClass());
|
||||
dispatcher.init(conf);
|
||||
|
||||
// Register handler
|
||||
dispatcher.register(TestEnum.class, new TestHandler());
|
||||
dispatcher.start();
|
||||
|
||||
for (int i = 0; i < 3; ++i) {
|
||||
Event event = mock(Event.class);
|
||||
when(event.getType()).thenReturn(TestEnum.TestEventType);
|
||||
dispatcher.getEventHandler().handle(event);
|
||||
}
|
||||
|
||||
for (int i = 0; i < 2; ++i) {
|
||||
Event event = mock(Event.class);
|
||||
when(event.getType()).thenReturn(TestEnum.TestEventType2);
|
||||
dispatcher.getEventHandler().handle(event);
|
||||
}
|
||||
|
||||
// Check event type count.
|
||||
GenericTestUtils.waitFor(() -> genericEventTypeMetrics.
|
||||
get(TestEnum.TestEventType) == 3, 1000, 10000);
|
||||
|
||||
GenericTestUtils.waitFor(() -> genericEventTypeMetrics.
|
||||
get(TestEnum.TestEventType2) == 2, 1000, 10000);
|
||||
|
||||
// Check time spend.
|
||||
Assert.assertTrue(genericEventTypeMetrics.
|
||||
getTotalProcessingTime(TestEnum.TestEventType)
|
||||
>= 1500*3);
|
||||
Assert.assertTrue(genericEventTypeMetrics.
|
||||
getTotalProcessingTime(TestEnum.TestEventType)
|
||||
< 1500*4);
|
||||
|
||||
Assert.assertTrue(genericEventTypeMetrics.
|
||||
getTotalProcessingTime(TestEnum.TestEventType2)
|
||||
>= 1500*2);
|
||||
Assert.assertTrue(genericEventTypeMetrics.
|
||||
getTotalProcessingTime(TestEnum.TestEventType2)
|
||||
< 1500*3);
|
||||
|
||||
// Make sure metrics consistent.
|
||||
Assert.assertEquals(Long.toString(genericEventTypeMetrics.
|
||||
get(TestEnum.TestEventType)),
|
||||
genericEventTypeMetrics.
|
||||
getRegistry().get("TestEventType_event_count").toString());
|
||||
Assert.assertEquals(Long.toString(genericEventTypeMetrics.
|
||||
get(TestEnum.TestEventType2)),
|
||||
genericEventTypeMetrics.
|
||||
getRegistry().get("TestEventType2_event_count").toString());
|
||||
Assert.assertEquals(Long.toString(genericEventTypeMetrics.
|
||||
getTotalProcessingTime(TestEnum.TestEventType)),
|
||||
genericEventTypeMetrics.
|
||||
getRegistry().get("TestEventType_processing_time").toString());
|
||||
Assert.assertEquals(Long.toString(genericEventTypeMetrics.
|
||||
getTotalProcessingTime(TestEnum.TestEventType2)),
|
||||
genericEventTypeMetrics.
|
||||
getRegistry().get("TestEventType2_processing_time").toString());
|
||||
|
||||
} finally {
|
||||
dispatcher.close();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDispatcherMetricsHistogram() throws Exception {
|
||||
YarnConfiguration conf = new YarnConfiguration();
|
||||
AsyncDispatcher dispatcher = null;
|
||||
|
||||
try {
|
||||
dispatcher = new AsyncDispatcher("RM Event dispatcher");
|
||||
|
||||
GenericEventTypeMetrics genericEventTypeMetrics =
|
||||
new GenericEventTypeMetrics.EventTypeMetricsBuilder()
|
||||
.setMs(DefaultMetricsSystem.instance())
|
||||
.setInfo(info("GenericEventTypeMetrics for "
|
||||
+ TestEnum.class.getName(),
|
||||
"Metrics for " + dispatcher.getName()))
|
||||
.setEnumClass(TestEnum.class)
|
||||
.setEnums(TestEnum.class.getEnumConstants())
|
||||
.build().registerMetrics();
|
||||
|
||||
// We can the metrics enabled for TestEnum
|
||||
dispatcher.addMetrics(genericEventTypeMetrics,
|
||||
genericEventTypeMetrics.getEnumClass());
|
||||
dispatcher.init(conf);
|
||||
|
||||
// Register handler
|
||||
dispatcher.register(TestEnum.class, new TestHandler());
|
||||
dispatcher.start();
|
||||
|
||||
for (int i = 0; i < 3; ++i) {
|
||||
Event event = mock(Event.class);
|
||||
when(event.getType()).thenReturn(TestEnum.TestEventType);
|
||||
dispatcher.getEventHandler().handle(event);
|
||||
}
|
||||
|
||||
for (int i = 0; i < 2; ++i) {
|
||||
Event event = mock(Event.class);
|
||||
when(event.getType()).thenReturn(TestEnum.TestEventType2);
|
||||
dispatcher.getEventHandler().handle(event);
|
||||
}
|
||||
|
||||
// Check event type count.
|
||||
GenericTestUtils.waitFor(() -> genericEventTypeMetrics.
|
||||
get(TestEnum.TestEventType) == 3, 1000, 10000);
|
||||
|
||||
GenericTestUtils.waitFor(() -> genericEventTypeMetrics.
|
||||
get(TestEnum.TestEventType2) == 2, 1000, 10000);
|
||||
|
||||
// submit actual values
|
||||
Map<String, Long> expectedValues = new HashMap<>();
|
||||
expectedValues.put("TestEventType_event_count",
|
||||
genericEventTypeMetrics.get(TestEnum.TestEventType));
|
||||
expectedValues.put("TestEventType_processing_time",
|
||||
genericEventTypeMetrics.
|
||||
getTotalProcessingTime(TestEnum.TestEventType));
|
||||
expectedValues.put("TestEventType2_event_count",
|
||||
genericEventTypeMetrics.get(TestEnum.TestEventType2));
|
||||
expectedValues.put("TestEventType2_processing_time",
|
||||
genericEventTypeMetrics.
|
||||
getTotalProcessingTime(TestEnum.TestEventType2));
|
||||
Set<String> testResults = new HashSet<>();
|
||||
|
||||
MetricsCollectorImpl collector = new MetricsCollectorImpl();
|
||||
genericEventTypeMetrics.getMetrics(collector, true);
|
||||
|
||||
for (MetricsRecord record : collector.getRecords()) {
|
||||
for (AbstractMetric metric : record.metrics()) {
|
||||
String metricName = metric.name();
|
||||
if (expectedValues.containsKey(metricName)) {
|
||||
Long expectedValue = expectedValues.get(metricName);
|
||||
Assert.assertEquals(
|
||||
"Metric " + metricName + " doesn't have expected value",
|
||||
expectedValue, metric.value());
|
||||
testResults.add(metricName);
|
||||
}
|
||||
}
|
||||
}
|
||||
Assert.assertEquals(expectedValues.keySet(), testResults);
|
||||
|
||||
} finally {
|
||||
dispatcher.close();
|
||||
}
|
||||
|
||||
}
|
||||
}
|
|
@ -0,0 +1,42 @@
|
|||
/**
|
||||
* 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.hadoop.yarn.server.resourcemanager;
|
||||
|
||||
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
||||
import org.apache.hadoop.yarn.metrics.GenericEventTypeMetrics;
|
||||
|
||||
import static org.apache.hadoop.metrics2.lib.Interns.info;
|
||||
|
||||
public final class GenericEventTypeMetricsManager {
|
||||
|
||||
private GenericEventTypeMetricsManager() {
|
||||
// nothing to do
|
||||
}
|
||||
|
||||
// Construct a GenericEventTypeMetrics for dispatcher
|
||||
public static <T extends Enum<T>> GenericEventTypeMetrics
|
||||
create(String dispatcherName, Class<T> eventTypeClass) {
|
||||
return new GenericEventTypeMetrics.EventTypeMetricsBuilder<T>()
|
||||
.setMs(DefaultMetricsSystem.instance())
|
||||
.setInfo(info("GenericEventTypeMetrics for " + eventTypeClass.getName(),
|
||||
"Metrics for " + dispatcherName))
|
||||
.setEnumClass(eventTypeClass)
|
||||
.setEnums(eventTypeClass.getEnumConstants())
|
||||
.build().registerMetrics();
|
||||
}
|
||||
}
|
|
@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager;
|
|||
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
|
||||
import com.sun.jersey.spi.container.servlet.ServletContainer;
|
||||
|
||||
import org.apache.hadoop.yarn.metrics.GenericEventTypeMetrics;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.slf4j.Marker;
|
||||
|
@ -62,6 +63,7 @@ import org.apache.hadoop.yarn.event.AsyncDispatcher;
|
|||
import org.apache.hadoop.yarn.event.Dispatcher;
|
||||
import org.apache.hadoop.yarn.event.EventDispatcher;
|
||||
import org.apache.hadoop.yarn.event.EventHandler;
|
||||
|
||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
||||
import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager;
|
||||
|
@ -449,11 +451,23 @@ public class ResourceManager extends CompositeService
|
|||
}
|
||||
|
||||
protected EventHandler<SchedulerEvent> createSchedulerEventDispatcher() {
|
||||
return new EventDispatcher(this.scheduler, "SchedulerEventDispatcher");
|
||||
EventDispatcher dispatcher = new
|
||||
EventDispatcher(this.scheduler, "SchedulerEventDispatcher");
|
||||
dispatcher.
|
||||
setMetrics(GenericEventTypeMetricsManager.
|
||||
create(dispatcher.getName(), SchedulerEventType.class));
|
||||
return dispatcher;
|
||||
}
|
||||
|
||||
protected Dispatcher createDispatcher() {
|
||||
return new AsyncDispatcher("RM Event dispatcher");
|
||||
AsyncDispatcher dispatcher = new AsyncDispatcher("RM Event dispatcher");
|
||||
GenericEventTypeMetrics genericEventTypeMetrics =
|
||||
GenericEventTypeMetricsManager.
|
||||
create(dispatcher.getName(), NodesListManagerEventType.class);
|
||||
// We can add more
|
||||
dispatcher.addMetrics(genericEventTypeMetrics,
|
||||
genericEventTypeMetrics.getEnumClass());
|
||||
return dispatcher;
|
||||
}
|
||||
|
||||
protected ResourceScheduler createScheduler() {
|
||||
|
|
|
@ -29,6 +29,7 @@ import java.util.concurrent.TimeoutException;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
||||
import org.apache.hadoop.http.lib.StaticUserWebFilter;
|
||||
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
||||
import org.apache.hadoop.net.NetworkTopology;
|
||||
import org.apache.hadoop.security.AuthenticationFilterInitializer;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
|
@ -73,6 +74,7 @@ public class TestResourceManager {
|
|||
public void setUp() throws Exception {
|
||||
YarnConfiguration conf = new YarnConfiguration();
|
||||
UserGroupInformation.setConfiguration(conf);
|
||||
DefaultMetricsSystem.setMiniClusterMode(true);
|
||||
resourceManager = new ResourceManager();
|
||||
resourceManager.init(conf);
|
||||
resourceManager.getRMContext().getContainerTokenSecretManager().rollMasterKey();
|
||||
|
|
|
@ -18,6 +18,7 @@
|
|||
|
||||
package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
|
||||
|
||||
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
||||
import org.apache.hadoop.net.NetworkTopology;
|
||||
import org.apache.hadoop.util.Time;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
|
@ -52,6 +53,7 @@ public class TestSchedulerHealth {
|
|||
private ResourceManager resourceManager;
|
||||
|
||||
public void setup() {
|
||||
DefaultMetricsSystem.setMiniClusterMode(true);
|
||||
resourceManager = new ResourceManager() {
|
||||
@Override
|
||||
protected RMNodeLabelsManager createNodeLabelManager() {
|
||||
|
|
|
@ -212,6 +212,7 @@ public class TestCapacityScheduler extends CapacitySchedulerTestBase {
|
|||
@Before
|
||||
public void setUp() throws Exception {
|
||||
ResourceUtils.resetResourceTypes(new Configuration());
|
||||
DefaultMetricsSystem.setMiniClusterMode(true);
|
||||
resourceManager = new ResourceManager() {
|
||||
@Override
|
||||
protected RMNodeLabelsManager createNodeLabelManager() {
|
||||
|
|
Loading…
Reference in New Issue