|
@@ -33,7 +33,9 @@ import java.util.Map;
|
|
|
import java.util.Set;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.service.Service;
|
|
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
|
|
+import org.apache.hadoop.yarn.api.records.NodeLabel;
|
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.event.Dispatcher;
|
|
@@ -44,6 +46,7 @@ import org.apache.hadoop.yarn.nodelabels.NodeLabelTestBase;
|
|
|
import org.apache.hadoop.yarn.nodelabels.RMNodeLabel;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeLabelsUpdateSchedulerEvent;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
|
|
@@ -606,6 +609,23 @@ public class TestRMNodeLabelsManager extends NodeLabelTestBase {
|
|
|
|
|
|
@Test(timeout = 60000)
|
|
|
public void testcheckRemoveFromClusterNodeLabelsOfQueue() throws Exception {
|
|
|
+ lmgr = new RMNodeLabelsManager();
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ File tempDir = File.createTempFile("nlb", ".tmp");
|
|
|
+ tempDir.delete();
|
|
|
+ tempDir.mkdirs();
|
|
|
+ tempDir.deleteOnExit();
|
|
|
+ conf.set(YarnConfiguration.FS_NODE_LABELS_STORE_ROOT_DIR,
|
|
|
+ tempDir.getAbsolutePath());
|
|
|
+ conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true);
|
|
|
+ conf.set(YarnConfiguration.RM_SCHEDULER,
|
|
|
+ "org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler");
|
|
|
+ Configuration withQueueLabels = getConfigurationWithQueueLabels(conf);
|
|
|
+ MockRM rm = initRM(conf);
|
|
|
+ lmgr.addToCluserNodeLabels(toSet(NodeLabel.newInstance("x", false)));
|
|
|
+ lmgr.removeFromClusterNodeLabels(Arrays.asList(new String[] { "x" }));
|
|
|
+ lmgr.addToCluserNodeLabelsWithDefaultExclusivity(toSet("x"));
|
|
|
+ rm.stop();
|
|
|
class TestRMLabelManger extends RMNodeLabelsManager {
|
|
|
@Override
|
|
|
protected void checkRemoveFromClusterNodeLabelsOfQueue(
|
|
@@ -613,34 +633,45 @@ public class TestRMNodeLabelsManager extends NodeLabelTestBase {
|
|
|
checkQueueCall = true;
|
|
|
// Do nothing
|
|
|
}
|
|
|
-
|
|
|
}
|
|
|
lmgr = new TestRMLabelManger();
|
|
|
- Configuration conf = new Configuration();
|
|
|
- File tempDir = File.createTempFile("nlb", ".tmp");
|
|
|
- tempDir.delete();
|
|
|
- tempDir.mkdirs();
|
|
|
- tempDir.deleteOnExit();
|
|
|
- conf.set(YarnConfiguration.FS_NODE_LABELS_STORE_ROOT_DIR,
|
|
|
- tempDir.getAbsolutePath());
|
|
|
- conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true);
|
|
|
+ MockRM rm2 = initRM(withQueueLabels);
|
|
|
+ Assert.assertFalse(
|
|
|
+ "checkRemoveFromClusterNodeLabelsOfQueue should not be called"
|
|
|
+ + "on recovery",
|
|
|
+ checkQueueCall);
|
|
|
+ lmgr.removeFromClusterNodeLabels(Arrays.asList(new String[] { "x" }));
|
|
|
+ Assert
|
|
|
+ .assertTrue("checkRemoveFromClusterNodeLabelsOfQueue should be called "
|
|
|
+ + "since its not recovery", checkQueueCall);
|
|
|
+ rm2.stop();
|
|
|
+ }
|
|
|
+
|
|
|
+ private MockRM initRM(Configuration conf) {
|
|
|
MockRM rm = new MockRM(conf) {
|
|
|
@Override
|
|
|
public RMNodeLabelsManager createNodeLabelManager() {
|
|
|
return lmgr;
|
|
|
}
|
|
|
};
|
|
|
- lmgr.addToCluserNodeLabelsWithDefaultExclusivity(toSet("a"));
|
|
|
- lmgr.removeFromClusterNodeLabels(Arrays.asList(new String[] { "a" }));
|
|
|
rm.getRMContext().setNodeLabelManager(lmgr);
|
|
|
rm.start();
|
|
|
- lmgr.addToCluserNodeLabelsWithDefaultExclusivity(toSet("a"));
|
|
|
- Assert.assertEquals(false, checkQueueCall);
|
|
|
- lmgr.removeFromClusterNodeLabels(Arrays.asList(new String[] { "a" }));
|
|
|
- Assert.assertEquals(true, checkQueueCall);
|
|
|
- lmgr.stop();
|
|
|
- lmgr.close();
|
|
|
- rm.stop();
|
|
|
+ Assert.assertEquals(Service.STATE.STARTED, rm.getServiceState());
|
|
|
+ return rm;
|
|
|
+ }
|
|
|
+
|
|
|
+ private Configuration getConfigurationWithQueueLabels(Configuration config) {
|
|
|
+ CapacitySchedulerConfiguration conf =
|
|
|
+ new CapacitySchedulerConfiguration(config);
|
|
|
+ // Define top-level queues
|
|
|
+ conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { "a" });
|
|
|
+ conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100);
|
|
|
+
|
|
|
+ final String A = CapacitySchedulerConfiguration.ROOT + ".a";
|
|
|
+ conf.setCapacity(A, 100);
|
|
|
+ conf.setAccessibleNodeLabels(A, ImmutableSet.of("x"));
|
|
|
+ conf.setCapacityByLabel(A, "x", 100);
|
|
|
+ return conf;
|
|
|
}
|
|
|
|
|
|
@Test(timeout = 5000)
|