|
@@ -41,6 +41,7 @@ import org.apache.hadoop.util.Shell;
|
|
|
import org.apache.hadoop.util.Shell.ShellCommandExecutor;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
+import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
import org.apache.hadoop.yarn.conf.HAUtil;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.event.Dispatcher;
|
|
@@ -67,6 +68,7 @@ import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdater;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdaterImpl;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.ResourceTrackerService;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystemTestUtil;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptRegistrationEvent;
|
|
@@ -76,6 +78,7 @@ import org.apache.hadoop.yarn.server.timeline.TimelineStore;
|
|
|
import org.apache.hadoop.yarn.server.timeline.recovery.MemoryTimelineStateStore;
|
|
|
import org.apache.hadoop.yarn.server.timeline.recovery.TimelineStateStore;
|
|
|
import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
|
|
|
+import org.apache.hadoop.yarn.util.resource.Resources;
|
|
|
import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
@@ -275,6 +278,12 @@ public class MiniYARNCluster extends CompositeService {
|
|
|
conf instanceof YarnConfiguration ? conf : new YarnConfiguration(conf));
|
|
|
}
|
|
|
|
|
|
+ @Override
|
|
|
+ protected synchronized void serviceStart() throws Exception {
|
|
|
+ super.serviceStart();
|
|
|
+ this.waitForNodeManagersToConnect(5000);
|
|
|
+ }
|
|
|
+
|
|
|
private void setNonHARMConfigurationWithEphemeralPorts(Configuration conf) {
|
|
|
String hostname = MiniYARNCluster.getHostname();
|
|
|
conf.set(YarnConfiguration.RM_ADDRESS, hostname + ":0");
|
|
@@ -314,19 +323,7 @@ public class MiniYARNCluster extends CompositeService {
|
|
|
|
|
|
private synchronized void startResourceManager(final int index) {
|
|
|
try {
|
|
|
- Thread rmThread = new Thread() {
|
|
|
- public void run() {
|
|
|
- resourceManagers[index].start();
|
|
|
- }
|
|
|
- };
|
|
|
- rmThread.setName("RM-" + index);
|
|
|
- rmThread.start();
|
|
|
- int waitCount = 0;
|
|
|
- while (resourceManagers[index].getServiceState() == STATE.INITED
|
|
|
- && waitCount++ < 60) {
|
|
|
- LOG.info("Waiting for RM to start...");
|
|
|
- Thread.sleep(1500);
|
|
|
- }
|
|
|
+ resourceManagers[index].start();
|
|
|
if (resourceManagers[index].getServiceState() != STATE.STARTED) {
|
|
|
// RM could have failed.
|
|
|
throw new IOException(
|
|
@@ -456,6 +453,11 @@ public class MiniYARNCluster extends CompositeService {
|
|
|
@Override
|
|
|
protected synchronized void serviceStart() throws Exception {
|
|
|
startResourceManager(index);
|
|
|
+ if(index == 0) {
|
|
|
+ resourceManagers[index].getRMContext().getRMAdminService()
|
|
|
+ .transitionToActive(new HAServiceProtocol.StateChangeRequestInfo(
|
|
|
+ HAServiceProtocol.RequestSource.REQUEST_BY_USER_FORCED));
|
|
|
+ }
|
|
|
Configuration conf = resourceManagers[index].getConfig();
|
|
|
LOG.info("Starting resourcemanager " + index);
|
|
|
LOG.info("MiniYARN ResourceManager address: " +
|
|
@@ -565,26 +567,12 @@ public class MiniYARNCluster extends CompositeService {
|
|
|
}
|
|
|
|
|
|
protected synchronized void serviceStart() throws Exception {
|
|
|
- try {
|
|
|
- new Thread() {
|
|
|
- public void run() {
|
|
|
- nodeManagers[index].start();
|
|
|
- }
|
|
|
- }.start();
|
|
|
- int waitCount = 0;
|
|
|
- while (nodeManagers[index].getServiceState() == STATE.INITED
|
|
|
- && waitCount++ < 60) {
|
|
|
- LOG.info("Waiting for NM " + index + " to start...");
|
|
|
- Thread.sleep(1000);
|
|
|
- }
|
|
|
- if (nodeManagers[index].getServiceState() != STATE.STARTED) {
|
|
|
- // RM could have failed.
|
|
|
- throw new IOException("NodeManager " + index + " failed to start");
|
|
|
- }
|
|
|
- super.serviceStart();
|
|
|
- } catch (Throwable t) {
|
|
|
- throw new YarnRuntimeException(t);
|
|
|
+ nodeManagers[index].start();
|
|
|
+ if (nodeManagers[index].getServiceState() != STATE.STARTED) {
|
|
|
+ // NM could have failed.
|
|
|
+ throw new IOException("NodeManager " + index + " failed to start");
|
|
|
}
|
|
|
+ super.serviceStart();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -715,7 +703,7 @@ public class MiniYARNCluster extends CompositeService {
|
|
|
/**
|
|
|
* Wait for all the NodeManagers to connect to the ResourceManager.
|
|
|
*
|
|
|
- * @param timeout Time to wait (sleeps in 100 ms intervals) in milliseconds.
|
|
|
+ * @param timeout Time to wait (sleeps in 10 ms intervals) in milliseconds.
|
|
|
* @return true if all NodeManagers connect to the (Active)
|
|
|
* ResourceManager, false otherwise.
|
|
|
* @throws YarnException
|
|
@@ -724,17 +712,19 @@ public class MiniYARNCluster extends CompositeService {
|
|
|
public boolean waitForNodeManagersToConnect(long timeout)
|
|
|
throws YarnException, InterruptedException {
|
|
|
GetClusterMetricsRequest req = GetClusterMetricsRequest.newInstance();
|
|
|
- for (int i = 0; i < timeout / 100; i++) {
|
|
|
+ for (int i = 0; i < timeout / 10; i++) {
|
|
|
ResourceManager rm = getResourceManager();
|
|
|
if (rm == null) {
|
|
|
throw new YarnException("Can not find the active RM.");
|
|
|
}
|
|
|
else if (nodeManagers.length == rm.getClientRMService()
|
|
|
- .getClusterMetrics(req).getClusterMetrics().getNumNodeManagers()) {
|
|
|
+ .getClusterMetrics(req).getClusterMetrics().getNumNodeManagers()) {
|
|
|
+ LOG.info("All Node Managers connected in MiniYARNCluster");
|
|
|
return true;
|
|
|
}
|
|
|
- Thread.sleep(100);
|
|
|
+ Thread.sleep(10);
|
|
|
}
|
|
|
+ LOG.info("Node Managers did not connect within 5000ms");
|
|
|
return false;
|
|
|
}
|
|
|
|
|
@@ -769,18 +759,7 @@ public class MiniYARNCluster extends CompositeService {
|
|
|
|
|
|
@Override
|
|
|
protected synchronized void serviceStart() throws Exception {
|
|
|
-
|
|
|
- new Thread() {
|
|
|
- public void run() {
|
|
|
- appHistoryServer.start();
|
|
|
- };
|
|
|
- }.start();
|
|
|
- int waitCount = 0;
|
|
|
- while (appHistoryServer.getServiceState() == STATE.INITED
|
|
|
- && waitCount++ < 60) {
|
|
|
- LOG.info("Waiting for Timeline Server to start...");
|
|
|
- Thread.sleep(1500);
|
|
|
- }
|
|
|
+ appHistoryServer.start();
|
|
|
if (appHistoryServer.getServiceState() != STATE.STARTED) {
|
|
|
// AHS could have failed.
|
|
|
IOException ioe = new IOException(
|