Revert "YARN-9615. Add dispatcher metrics to RM. Contributed by Qi Zhu."

This reverts commit 369f75b7a7.
This commit is contained in:
Peter Bacsko 2021-03-09 14:32:02 +01:00
parent 369f75b7a7
commit 3851994cd6
11 changed files with 6 additions and 521 deletions

View File

@ -26,9 +26,6 @@ import java.util.Map;
import java.util.concurrent.BlockingQueue; import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue; 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.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import org.slf4j.Marker; import org.slf4j.Marker;
@ -88,11 +85,6 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
protected final Map<Class<? extends Enum>, EventHandler> eventDispatchers; protected final Map<Class<? extends Enum>, EventHandler> eventDispatchers;
private boolean exitOnDispatchException = true; private boolean exitOnDispatchException = true;
private Map<Class<? extends Enum>,
EventTypeMetrics> eventTypeMetricsMap;
private Clock clock = new MonotonicClock();
/** /**
* The thread name for dispatcher. * The thread name for dispatcher.
*/ */
@ -106,8 +98,6 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
super("Dispatcher"); super("Dispatcher");
this.eventQueue = eventQueue; this.eventQueue = eventQueue;
this.eventDispatchers = new HashMap<Class<? extends Enum>, EventHandler>(); this.eventDispatchers = new HashMap<Class<? extends Enum>, EventHandler>();
this.eventTypeMetricsMap = new HashMap<Class<? extends Enum>,
EventTypeMetrics>();
} }
/** /**
@ -145,16 +135,7 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
return; return;
} }
if (event != null) { if (event != null) {
if (eventTypeMetricsMap. dispatch(event);
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) { if (printTrigger) {
//Log the latest dispatch event type //Log the latest dispatch event type
// may cause the too many events queued // may cause the too many events queued
@ -388,9 +369,4 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
protected boolean isStopped() { protected boolean isStopped() {
return stopped; return stopped;
} }
public void addMetrics(EventTypeMetrics metrics,
Class<? extends Enum> eventClass) {
eventTypeMetricsMap.put(eventClass, metrics);
}
} }

View File

@ -19,9 +19,6 @@
package org.apache.hadoop.yarn.event; package org.apache.hadoop.yarn.event;
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; 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.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import org.slf4j.Marker; import org.slf4j.Marker;
@ -50,15 +47,12 @@ public class EventDispatcher<T extends Event> extends
private final Thread eventProcessor; private final Thread eventProcessor;
private volatile boolean stopped = false; private volatile boolean stopped = false;
private boolean shouldExitOnError = true; private boolean shouldExitOnError = true;
private EventTypeMetrics metrics;
private static final Logger LOG = private static final Logger LOG =
LoggerFactory.getLogger(EventDispatcher.class); LoggerFactory.getLogger(EventDispatcher.class);
private static final Marker FATAL = private static final Marker FATAL =
MarkerFactory.getMarker("FATAL"); MarkerFactory.getMarker("FATAL");
private Clock clock = new MonotonicClock();
private final class EventProcessor implements Runnable { private final class EventProcessor implements Runnable {
@Override @Override
public void run() { public void run() {
@ -74,14 +68,7 @@ public class EventDispatcher<T extends Event> extends
} }
try { try {
if (metrics != null) { handler.handle(event);
long startTime = clock.getTime();
handler.handle(event);
metrics.increment(event.getType(),
clock.getTime() - startTime);
} else {
handler.handle(event);
}
} catch (Throwable t) { } catch (Throwable t) {
// An error occurred, but we are shutting down anyway. // An error occurred, but we are shutting down anyway.
// If it was an InterruptedException, the very act of // If it was an InterruptedException, the very act of
@ -149,9 +136,4 @@ public class EventDispatcher<T extends Event> extends
public void disableExitOnError() { public void disableExitOnError() {
shouldExitOnError = false; shouldExitOnError = false;
} }
public void setMetrics(EventTypeMetrics metrics) {
this.metrics = metrics;
}
} }

View File

@ -1,42 +0,0 @@
/**
* 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;
}
}

View File

@ -1,32 +0,0 @@
/**
* 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);
}

View File

@ -1,160 +0,0 @@
/**
* 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;
}
}

View File

@ -20,20 +20,9 @@ package org.apache.hadoop.yarn.event;
import java.lang.reflect.Field; import java.lang.reflect.Field;
import java.lang.reflect.Modifier; 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.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue; 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.slf4j.Logger;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
@ -41,7 +30,6 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
import static org.apache.hadoop.metrics2.lib.Interns.info;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.mockito.Mockito.*; import static org.mockito.Mockito.*;
@ -130,7 +118,7 @@ public class TestAsyncDispatcher {
} }
private enum TestEnum { private enum TestEnum {
TestEventType, TestEventType2 TestEventType
} }
@SuppressWarnings({ "rawtypes", "unchecked" }) @SuppressWarnings({ "rawtypes", "unchecked" })
@ -242,171 +230,5 @@ 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();
}
}
}

View File

@ -1,42 +0,0 @@
/**
* 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();
}
}

View File

@ -21,7 +21,6 @@ package org.apache.hadoop.yarn.server.resourcemanager;
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
import com.sun.jersey.spi.container.servlet.ServletContainer; import com.sun.jersey.spi.container.servlet.ServletContainer;
import org.apache.hadoop.yarn.metrics.GenericEventTypeMetrics;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import org.slf4j.Marker; import org.slf4j.Marker;
@ -63,7 +62,6 @@ import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.event.EventDispatcher; import org.apache.hadoop.yarn.event.EventDispatcher;
import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager; import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager;
@ -451,23 +449,11 @@ public class ResourceManager extends CompositeService
} }
protected EventHandler<SchedulerEvent> createSchedulerEventDispatcher() { protected EventHandler<SchedulerEvent> createSchedulerEventDispatcher() {
EventDispatcher dispatcher = new return new EventDispatcher(this.scheduler, "SchedulerEventDispatcher");
EventDispatcher(this.scheduler, "SchedulerEventDispatcher");
dispatcher.
setMetrics(GenericEventTypeMetricsManager.
create(dispatcher.getName(), SchedulerEventType.class));
return dispatcher;
} }
protected Dispatcher createDispatcher() { protected Dispatcher createDispatcher() {
AsyncDispatcher dispatcher = new AsyncDispatcher("RM Event dispatcher"); return 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() { protected ResourceScheduler createScheduler() {

View File

@ -29,7 +29,6 @@ import java.util.concurrent.TimeoutException;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.http.lib.StaticUserWebFilter; import org.apache.hadoop.http.lib.StaticUserWebFilter;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.net.NetworkTopology; import org.apache.hadoop.net.NetworkTopology;
import org.apache.hadoop.security.AuthenticationFilterInitializer; import org.apache.hadoop.security.AuthenticationFilterInitializer;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
@ -74,7 +73,6 @@ public class TestResourceManager {
public void setUp() throws Exception { public void setUp() throws Exception {
YarnConfiguration conf = new YarnConfiguration(); YarnConfiguration conf = new YarnConfiguration();
UserGroupInformation.setConfiguration(conf); UserGroupInformation.setConfiguration(conf);
DefaultMetricsSystem.setMiniClusterMode(true);
resourceManager = new ResourceManager(); resourceManager = new ResourceManager();
resourceManager.init(conf); resourceManager.init(conf);
resourceManager.getRMContext().getContainerTokenSecretManager().rollMasterKey(); resourceManager.getRMContext().getContainerTokenSecretManager().rollMasterKey();

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler; 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.net.NetworkTopology;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerId;
@ -53,7 +52,6 @@ public class TestSchedulerHealth {
private ResourceManager resourceManager; private ResourceManager resourceManager;
public void setup() { public void setup() {
DefaultMetricsSystem.setMiniClusterMode(true);
resourceManager = new ResourceManager() { resourceManager = new ResourceManager() {
@Override @Override
protected RMNodeLabelsManager createNodeLabelManager() { protected RMNodeLabelsManager createNodeLabelManager() {

View File

@ -212,7 +212,6 @@ public class TestCapacityScheduler extends CapacitySchedulerTestBase {
@Before @Before
public void setUp() throws Exception { public void setUp() throws Exception {
ResourceUtils.resetResourceTypes(new Configuration()); ResourceUtils.resetResourceTypes(new Configuration());
DefaultMetricsSystem.setMiniClusterMode(true);
resourceManager = new ResourceManager() { resourceManager = new ResourceManager() {
@Override @Override
protected RMNodeLabelsManager createNodeLabelManager() { protected RMNodeLabelsManager createNodeLabelManager() {