|
@@ -42,6 +42,8 @@ import java.util.Map;
|
|
|
import java.util.Set;
|
|
|
|
|
|
import org.apache.commons.io.FileUtils;
|
|
|
+import org.apache.commons.logging.Log;
|
|
|
+import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
|
|
import org.apache.hadoop.io.DataOutputBuffer;
|
|
@@ -104,6 +106,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
|
|
|
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
|
|
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
|
import org.apache.hadoop.yarn.util.ConverterUtils;
|
|
@@ -121,6 +125,7 @@ import com.google.common.collect.ImmutableMap;
|
|
|
import com.google.common.collect.Sets;
|
|
|
|
|
|
public class TestRMRestart extends ParameterizedSchedulerTestBase {
|
|
|
+ private static final Log LOG = LogFactory.getLog(TestRMRestart.class);
|
|
|
private final static File TEMP_DIR = new File(System.getProperty(
|
|
|
"test.build.data", "/tmp"), "decommision");
|
|
|
private File hostFile = new File(TEMP_DIR + File.separator + "hostFile.txt");
|
|
@@ -2321,4 +2326,57 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
|
|
|
rm2.stop();
|
|
|
}
|
|
|
|
|
|
+ @Test(timeout = 120000)
|
|
|
+ public void testRMRestartAfterPreemption() throws Exception {
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
|
|
|
+ if (!getSchedulerType().equals(SchedulerType.CAPACITY)) {
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ MemoryRMStateStore memStore = new MemoryRMStateStore();
|
|
|
+ memStore.init(conf);
|
|
|
+ // start RM
|
|
|
+ MockRM rm1 = new MockRM(conf, memStore);
|
|
|
+ rm1.start();
|
|
|
+ MockNM nm1 =
|
|
|
+ new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
|
|
|
+ nm1.registerNode();
|
|
|
+ int CONTAINER_MEMORY = 1024;
|
|
|
+ // create app and launch the AM
|
|
|
+ RMApp app0 = rm1.submitApp(CONTAINER_MEMORY);
|
|
|
+ MockAM am0 = MockRM.launchAM(app0, rm1, nm1);
|
|
|
+ nm1.nodeHeartbeat(am0.getApplicationAttemptId(), 1,
|
|
|
+ ContainerState.COMPLETE);
|
|
|
+ am0.waitForState(RMAppAttemptState.FAILED);
|
|
|
+ for (int i = 0; i < 4; i++) {
|
|
|
+ am0 = MockRM.launchAM(app0, rm1, nm1);
|
|
|
+ am0.registerAppAttempt();
|
|
|
+ CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
|
|
|
+ // get scheduler app
|
|
|
+ FiCaSchedulerApp schedulerAppAttempt = cs.getSchedulerApplications()
|
|
|
+ .get(app0.getApplicationId()).getCurrentAppAttempt();
|
|
|
+ // kill app0-attempt
|
|
|
+ cs.killPreemptedContainer(schedulerAppAttempt.getRMContainer(
|
|
|
+ app0.getCurrentAppAttempt().getMasterContainer().getId()));
|
|
|
+ }
|
|
|
+ am0 = MockRM.launchAM(app0, rm1, nm1);
|
|
|
+ am0.registerAppAttempt();
|
|
|
+ rm1.killApp(app0.getApplicationId());
|
|
|
+ rm1.waitForState(app0.getCurrentAppAttempt().getAppAttemptId(),
|
|
|
+ RMAppAttemptState.KILLED);
|
|
|
+
|
|
|
+ MockRM rm2 = null;
|
|
|
+ // start RM2
|
|
|
+ try {
|
|
|
+ rm2 = new MockRM(conf, memStore);
|
|
|
+ rm2.start();
|
|
|
+ Assert.assertTrue("RM start successfully", true);
|
|
|
+ } catch (Exception e) {
|
|
|
+ LOG.debug("Exception on start", e);
|
|
|
+ Assert.fail("RM should start with out any issue");
|
|
|
+ } finally {
|
|
|
+ rm1.stop();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
}
|