YARN-1821. NPE on registerNodeManager if the request has containers for UnmanagedAMs (kasha)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1576527 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Karthik Kambatla 2014-03-11 22:19:30 +00:00
parent 237a2054b4
commit 6f4d705a63
3 changed files with 44 additions and 8 deletions

View File

@ -428,6 +428,9 @@ Release 2.4.0 - UNRELEASED
apps-killed metrics correctly for killed applications. (Varun Vasudev via apps-killed metrics correctly for killed applications. (Varun Vasudev via
vinodkv) vinodkv)
YARN-1821. NPE on registerNodeManager if the request has containers for
UnmanagedAMs. (kasha)
Release 2.3.1 - UNRELEASED Release 2.3.1 - UNRELEASED
INCOMPATIBLE CHANGES INCOMPATIBLE CHANGES

View File

@ -210,14 +210,16 @@ public class ResourceTrackerService extends AbstractService implements
rmContext.getRMApps().get(appAttemptId.getApplicationId()); rmContext.getRMApps().get(appAttemptId.getApplicationId());
if (rmApp != null) { if (rmApp != null) {
RMAppAttempt rmAppAttempt = rmApp.getRMAppAttempt(appAttemptId); RMAppAttempt rmAppAttempt = rmApp.getRMAppAttempt(appAttemptId);
if (rmAppAttempt.getMasterContainer().getId() if (rmAppAttempt != null) {
.equals(containerStatus.getContainerId()) if (rmAppAttempt.getMasterContainer().getId()
&& containerStatus.getState() == ContainerState.COMPLETE) { .equals(containerStatus.getContainerId())
// sending master container finished event. && containerStatus.getState() == ContainerState.COMPLETE) {
RMAppAttemptContainerFinishedEvent evt = // sending master container finished event.
new RMAppAttemptContainerFinishedEvent(appAttemptId, RMAppAttemptContainerFinishedEvent evt =
containerStatus); new RMAppAttemptContainerFinishedEvent(appAttemptId,
rmContext.getDispatcher().getEventHandler().handle(evt); containerStatus);
rmContext.getDispatcher().getEventHandler().handle(evt);
}
} }
} else { } else {
LOG.error("Received finished container :" LOG.error("Received finished container :"

View File

@ -21,6 +21,8 @@ package org.apache.hadoop.yarn.server.resourcemanager;
import java.io.File; import java.io.File;
import java.io.FileOutputStream; import java.io.FileOutputStream;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
@ -29,7 +31,11 @@ import junit.framework.Assert;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerState;
import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.NodeState; import org.apache.hadoop.yarn.api.records.NodeState;
@ -42,6 +48,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
import org.apache.hadoop.yarn.server.api.records.NodeAction; import org.apache.hadoop.yarn.server.api.records.NodeAction;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.server.utils.BuilderUtils;
@ -50,6 +57,8 @@ import org.apache.hadoop.yarn.util.YarnVersionInfo;
import org.junit.After; import org.junit.After;
import org.junit.Test; import org.junit.Test;
import static org.junit.Assert.assertEquals;
public class TestResourceTrackerService { public class TestResourceTrackerService {
private final static File TEMP_DIR = new File(System.getProperty( private final static File TEMP_DIR = new File(System.getProperty(
@ -457,6 +466,28 @@ public class TestResourceTrackerService {
ClusterMetrics.getMetrics().getUnhealthyNMs()); ClusterMetrics.getMetrics().getUnhealthyNMs());
} }
@Test
public void testNodeRegistrationWithContainers() throws Exception {
MockRM rm = new MockRM();
rm.init(new YarnConfiguration());
rm.start();
RMApp app = rm.submitApp(1024);
MockNM nm = rm.registerNode("host1:1234", 8192);
nm.nodeHeartbeat(true);
// Register node with some container statuses
ContainerStatus status = ContainerStatus.newInstance(
ContainerId.newInstance(ApplicationAttemptId.newInstance(
app.getApplicationId(), 2), 1),
ContainerState.COMPLETE, "Dummy Completed", 0);
// The following shouldn't throw NPE
nm.registerNode(Collections.singletonList(status));
assertEquals("Incorrect number of nodes", 1,
rm.getRMContext().getRMNodes().size());
}
@Test @Test
public void testReconnectNode() throws Exception { public void testReconnectNode() throws Exception {
final DrainDispatcher dispatcher = new DrainDispatcher(); final DrainDispatcher dispatcher = new DrainDispatcher();