Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
23 commits
Select commit Hold shift + click to select a range
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
Original file line number Diff line number Diff line change
Expand Up @@ -3070,6 +3070,10 @@ public static boolean isAclEnabled(Configuration conf) {
+ "amrmproxy.ha.enable";
public static final boolean DEFAULT_AMRM_PROXY_HA_ENABLED = false;

// Enable NM Dispatcher Metric default False.
public static final String NM_DISPATCHER_METRIC_ENABLED = NM_PREFIX + "dispatcher.metric.enable";
public static final boolean DEFAULT_NM_DISPATCHER_METRIC_ENABLED = false;

/**
* Default platform-agnostic CLASSPATH for YARN applications. A
* comma-separated list of CLASSPATH entries. The parameter expansion marker
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,10 +57,8 @@ public GenericEventTypeMetrics(MetricsInfo info, MetricsSystem ms,

//Initialize enum
for (final T type : enums) {
String eventCountMetricsName =
type.toString() + "_" + "event_count";
String processingTimeMetricsName =
type.toString() + "_" + "processing_time";
String eventCountMetricsName = type + "_" + "event_count";
String processingTimeMetricsName = type + "_" + "processing_time";
eventCountMetrics.put(type, this.registry.
newGauge(eventCountMetricsName, eventCountMetricsName, 0L));
processingTimeMetrics.put(type, this.registry.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5065,6 +5065,16 @@
</description>
</property>

<property>
<name>yarn.nodemanager.dispatcher.metric.enable</name>
<value>false</value>
<description>
Yarn NodeManager enables Dispatcher Metric.
if true, will enable dispatcher metric; if false, will not enable dispatcher metric;
Default is false.
</description>
</property>

<property>
<name>yarn.router.interceptor.user-thread-pool.minimum-pool-size</name>
<value>5</value>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,43 @@
/**
* 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.nodemanager;

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
@SuppressWarnings("unchecked")
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();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.metrics.GenericEventTypeMetrics;
import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
Expand Down Expand Up @@ -144,8 +145,10 @@ public int getExitCode() {
private AtomicBoolean isStopping = new AtomicBoolean(false);
private boolean rmWorkPreservingRestartEnabled;
private boolean shouldExitOnShutdownEvent = false;
private boolean nmDispatherMetricEnabled;

private NMLogAggregationStatusTracker nmLogAggregationStatusTracker;

/**
* Default Container State transition listener.
*/
Expand Down Expand Up @@ -366,6 +369,10 @@ protected void serviceInit(Configuration conf) throws Exception {
.RM_WORK_PRESERVING_RECOVERY_ENABLED,
YarnConfiguration.DEFAULT_RM_WORK_PRESERVING_RECOVERY_ENABLED);

nmDispatherMetricEnabled = conf.getBoolean(
YarnConfiguration.NM_DISPATCHER_METRIC_ENABLED,
YarnConfiguration.DEFAULT_NM_DISPATCHER_METRIC_ENABLED);

try {
initAndStartRecoveryStore(conf);
} catch (IOException e) {
Expand Down Expand Up @@ -1006,8 +1013,17 @@ ContainerManagerImpl getContainerManager() {
/**
* Unit test friendly.
*/
@SuppressWarnings("unchecked")
protected AsyncDispatcher createNMDispatcher() {
return new AsyncDispatcher("NM Event dispatcher");
dispatcher = new AsyncDispatcher("NM Event dispatcher");
if (nmDispatherMetricEnabled) {
GenericEventTypeMetrics<ContainerManagerEventType> eventTypeMetrics =
GenericEventTypeMetricsManager.create(dispatcher.getName(),
ContainerManagerEventType.class);
dispatcher.addMetrics(eventTypeMetrics, eventTypeMetrics.getEnumClass());
LOG.info("NM Event dispatcher Metric Initialization Completed.");
}
return dispatcher;
}

//For testing
Expand Down Expand Up @@ -1052,4 +1068,10 @@ private NMLogAggregationStatusTracker createNMLogAggregationStatusTracker(
Context ctxt) {
return new NMLogAggregationStatusTracker(ctxt);
}

@VisibleForTesting
@Private
public AsyncDispatcher getDispatcher() {
return dispatcher;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,9 @@
import org.apache.hadoop.yarn.api.protocolrecords.GetLocalizationStatusesRequest;
import org.apache.hadoop.yarn.api.protocolrecords.GetLocalizationStatusesResponse;
import org.apache.hadoop.yarn.api.records.LocalizationStatus;
import org.apache.hadoop.yarn.metrics.GenericEventTypeMetrics;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.UpdateContainerTokenEvent;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizerEventType;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerTokenUpdatedEvent;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler.ContainerSchedulerEvent;
import org.apache.hadoop.yarn.server.nodemanager.recovery.RecoveryIterator;
Expand Down Expand Up @@ -105,6 +107,7 @@
import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent;
import org.apache.hadoop.yarn.server.nodemanager.GenericEventTypeMetricsManager;
import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedContainersEvent;
import org.apache.hadoop.yarn.server.nodemanager.CMgrUpdateContainersEvent;
import org.apache.hadoop.yarn.server.nodemanager.CMgrSignalContainersEvent;
Expand All @@ -120,6 +123,7 @@
import org.apache.hadoop.yarn.server.nodemanager.amrmproxy.AMRMProxyService;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationContainerInitEvent;

import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEvent;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEventType;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationFinishEvent;
Expand Down Expand Up @@ -217,7 +221,7 @@ private enum ReInitOp {
protected final NodeStatusUpdater nodeStatusUpdater;

protected LocalDirsHandlerService dirsHandler;
protected final AsyncDispatcher dispatcher;
private AsyncDispatcher dispatcher;

private final DeletionService deletionService;
private LogHandler logHandler;
Expand All @@ -233,6 +237,7 @@ private enum ReInitOp {
// NM metrics publisher is set only if the timeline service v.2 is enabled
private NMTimelinePublisher nmMetricsPublisher;
private boolean timelineServiceV2Enabled;
private boolean nmDispatherMetricEnabled;

public ContainerManagerImpl(Context context, ContainerExecutor exec,
DeletionService deletionContext, NodeStatusUpdater nodeStatusUpdater,
Expand All @@ -242,7 +247,7 @@ public ContainerManagerImpl(Context context, ContainerExecutor exec,
this.dirsHandler = dirsHandler;

// ContainerManager level dispatcher.
dispatcher = new AsyncDispatcher("NM ContainerManager dispatcher");
dispatcher = createContainerManagerDispatcher();
this.deletionService = deletionContext;
this.metrics = metrics;

Expand Down Expand Up @@ -324,10 +329,67 @@ public void serviceInit(Configuration conf) throws Exception {
YarnConfiguration.DEFAULT_NM_PROCESS_KILL_WAIT_MS) +
SHUTDOWN_CLEANUP_SLOP_MS;

nmDispatherMetricEnabled = conf.getBoolean(
YarnConfiguration.NM_DISPATCHER_METRIC_ENABLED,
YarnConfiguration.DEFAULT_NM_DISPATCHER_METRIC_ENABLED);

super.serviceInit(conf);
recover();
}

@SuppressWarnings("unchecked")
protected AsyncDispatcher createContainerManagerDispatcher() {
dispatcher = new AsyncDispatcher("NM ContainerManager dispatcher");

if (!nmDispatherMetricEnabled) {
return dispatcher;
}

GenericEventTypeMetrics<ContainerEventType> containerEventTypeMetrics =
GenericEventTypeMetricsManager.create(dispatcher.getName(), ContainerEventType.class);
dispatcher.addMetrics(containerEventTypeMetrics, containerEventTypeMetrics.getEnumClass());

GenericEventTypeMetrics<LocalizationEventType> localizationEventTypeMetrics =
GenericEventTypeMetricsManager.create(dispatcher.getName(), LocalizationEventType.class);
dispatcher.addMetrics(localizationEventTypeMetrics,
localizationEventTypeMetrics.getEnumClass());

GenericEventTypeMetrics<ApplicationEventType> applicationEventTypeMetrics =
GenericEventTypeMetricsManager.create(dispatcher.getName(), ApplicationEventType.class);
dispatcher.addMetrics(applicationEventTypeMetrics,
applicationEventTypeMetrics.getEnumClass());

GenericEventTypeMetrics<ContainersLauncherEventType> containersLauncherEventTypeMetrics =
GenericEventTypeMetricsManager.create(dispatcher.getName(),
ContainersLauncherEventType.class);
dispatcher.addMetrics(containersLauncherEventTypeMetrics,
containersLauncherEventTypeMetrics.getEnumClass());

GenericEventTypeMetrics<ContainerSchedulerEventType> containerSchedulerEventTypeMetrics =
GenericEventTypeMetricsManager.create(dispatcher.getName(),
ContainerSchedulerEventType.class);
dispatcher.addMetrics(containerSchedulerEventTypeMetrics,
containerSchedulerEventTypeMetrics.getEnumClass());

GenericEventTypeMetrics<ContainersMonitorEventType> containersMonitorEventTypeMetrics =
GenericEventTypeMetricsManager.create(dispatcher.getName(),
ContainersMonitorEventType.class);
dispatcher.addMetrics(containersMonitorEventTypeMetrics,
containersMonitorEventTypeMetrics.getEnumClass());

GenericEventTypeMetrics<AuxServicesEventType> auxServicesEventTypeTypeMetrics =
GenericEventTypeMetricsManager.create(dispatcher.getName(), AuxServicesEventType.class);
dispatcher.addMetrics(auxServicesEventTypeTypeMetrics,
auxServicesEventTypeTypeMetrics.getEnumClass());

GenericEventTypeMetrics<LocalizerEventType> localizerEventTypeMetrics =
GenericEventTypeMetricsManager.create(dispatcher.getName(), LocalizerEventType.class);
dispatcher.addMetrics(localizerEventTypeMetrics, localizerEventTypeMetrics.getEnumClass());
LOG.info("NM ContainerManager dispatcher Metric Initialization Completed.");

return dispatcher;
}

protected void createAMRMProxyService(Configuration conf) {
this.amrmProxyEnabled =
conf.getBoolean(YarnConfiguration.AMRM_PROXY_ENABLED,
Expand Down Expand Up @@ -2034,4 +2096,8 @@ private List<LocalizationStatus> getLocalizationStatusesInternal(
public ResourceLocalizationService getResourceLocalizationService() {
return rsrcLocalizationSrvc;
}

public AsyncDispatcher getDispatcher() {
return dispatcher;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -70,15 +70,15 @@ public DummyContainerManager(Context context, ContainerExecutor exec,
NodeManagerMetrics metrics, LocalDirsHandlerService dirsHandler) {
super(context, exec, deletionContext, nodeStatusUpdater, metrics,
dirsHandler);
dispatcher.disableExitOnDispatchException();
getDispatcher().disableExitOnDispatchException();
}

@Override
@SuppressWarnings("unchecked")
protected ResourceLocalizationService createResourceLocalizationService(
ContainerExecutor exec, DeletionService deletionContext, Context context,
NodeManagerMetrics metrics) {
return new ResourceLocalizationService(super.dispatcher, exec,
return new ResourceLocalizationService(getDispatcher(), exec,
deletionContext, super.dirsHandler, context, metrics) {
@Override
public void handle(LocalizationEvent event) {
Expand Down Expand Up @@ -148,20 +148,20 @@ protected UserGroupInformation getRemoteUgi() throws YarnException {
@SuppressWarnings("unchecked")
protected ContainersLauncher createContainersLauncher(Context context,
ContainerExecutor exec) {
return new ContainersLauncher(context, super.dispatcher, exec,
return new ContainersLauncher(context, getDispatcher(), exec,
super.dirsHandler, this) {
@Override
public void handle(ContainersLauncherEvent event) {
Container container = event.getContainer();
ContainerId containerId = container.getContainerId();
switch (event.getType()) {
case LAUNCH_CONTAINER:
dispatcher.getEventHandler().handle(
getDispatcher().getEventHandler().handle(
new ContainerEvent(containerId,
ContainerEventType.CONTAINER_LAUNCHED));
break;
case CLEANUP_CONTAINER:
dispatcher.getEventHandler().handle(
getDispatcher().getEventHandler().handle(
new ContainerExitEvent(containerId,
ContainerEventType.CONTAINER_KILLED_ON_REQUEST, 0,
"Container exited with exit code 0."));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -135,7 +135,7 @@ protected UserGroupInformation getRemoteUgi() throws YarnException {
@Override
protected ContainersMonitor createContainersMonitor(
ContainerExecutor exec) {
return new ContainersMonitorImpl(exec, dispatcher, this.context) {
return new ContainersMonitorImpl(exec, getDispatcher(), this.context) {
// Define resources available for containers to be executed.
@Override
public long getPmemAllocatedForContainers() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -797,7 +797,7 @@ protected void authorizeGetAndStopContainerRequest(
}
@Override
protected ContainerScheduler createContainerScheduler(Context context) {
return new ContainerScheduler(context, dispatcher, metrics){
return new ContainerScheduler(context, getDispatcher(), metrics){
@Override
public ContainersMonitor getContainersMonitor() {
return new ContainersMonitorImpl(null, null, null) {
Expand Down Expand Up @@ -1001,7 +1001,7 @@ protected ContainersLauncher createContainersLauncher(
return null;
}
};
containerManager.dispatcher.disableExitOnDispatchException();
containerManager.getDispatcher().disableExitOnDispatchException();
return containerManager;
}

Expand Down
Loading