|
@@ -34,24 +34,26 @@ import org.apache.hadoop.yarn.server.resourcemanager.placement.PrimaryGroupPlace
|
|
import org.apache.hadoop.yarn.server.resourcemanager.placement.SpecifiedPlacementRule;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.placement.SpecifiedPlacementRule;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.placement.UserPlacementRule;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.placement.UserPlacementRule;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSchedulerConfiguration;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSchedulerConfiguration;
|
|
|
|
+
|
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.allocationfile.AllocationFileQueue;
|
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.allocationfile.AllocationFileQueuePlacementPolicy;
|
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.allocationfile.AllocationFileQueuePlacementRule;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.allocationfile.AllocationFileWriter;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.allocationfile.AllocationFileWriter;
|
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.allocationfile.UserSettings;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.DominantResourceFairnessPolicy;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.DominantResourceFairnessPolicy;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FairSharePolicy;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FairSharePolicy;
|
|
import org.apache.hadoop.yarn.util.ControlledClock;
|
|
import org.apache.hadoop.yarn.util.ControlledClock;
|
|
import org.apache.hadoop.yarn.util.SystemClock;
|
|
import org.apache.hadoop.yarn.util.SystemClock;
|
|
import org.apache.hadoop.yarn.util.resource.CustomResourceTypesConfigurationProvider;
|
|
import org.apache.hadoop.yarn.util.resource.CustomResourceTypesConfigurationProvider;
|
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
|
|
|
+import org.junit.After;
|
|
import org.junit.Before;
|
|
import org.junit.Before;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
|
|
+
|
|
import java.io.File;
|
|
import java.io.File;
|
|
-import java.io.FileOutputStream;
|
|
|
|
-import java.io.FileWriter;
|
|
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
-import java.io.OutputStreamWriter;
|
|
|
|
-import java.io.PrintWriter;
|
|
|
|
import java.net.URISyntaxException;
|
|
import java.net.URISyntaxException;
|
|
import java.net.URL;
|
|
import java.net.URL;
|
|
-import java.nio.charset.StandardCharsets;
|
|
|
|
import java.util.List;
|
|
import java.util.List;
|
|
import java.util.Map;
|
|
import java.util.Map;
|
|
import java.util.Set;
|
|
import java.util.Set;
|
|
@@ -97,6 +99,11 @@ public class TestAllocationFileLoaderService {
|
|
when(scheduler.getRMContext()).thenReturn(rmContext);
|
|
when(scheduler.getRMContext()).thenReturn(rmContext);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @After
|
|
|
|
+ public void teardown() {
|
|
|
|
+ new File(ALLOC_FILE).delete();
|
|
|
|
+ }
|
|
|
|
+
|
|
@Test
|
|
@Test
|
|
public void testGetAllocationFileFromFileSystem()
|
|
public void testGetAllocationFileFromFileSystem()
|
|
throws IOException, URISyntaxException {
|
|
throws IOException, URISyntaxException {
|
|
@@ -152,18 +159,14 @@ public class TestAllocationFileLoaderService {
|
|
|
|
|
|
@Test (timeout = 10000)
|
|
@Test (timeout = 10000)
|
|
public void testReload() throws Exception {
|
|
public void testReload() throws Exception {
|
|
- PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
|
|
|
|
- out.println("<?xml version=\"1.0\"?>");
|
|
|
|
- out.println("<allocations>");
|
|
|
|
- out.println(" <queue name=\"queueA\">");
|
|
|
|
- out.println(" <maxRunningApps>1</maxRunningApps>");
|
|
|
|
- out.println(" </queue>");
|
|
|
|
- out.println(" <queue name=\"queueB\" />");
|
|
|
|
- out.println(" <queuePlacementPolicy>");
|
|
|
|
- out.println(" <rule name='default' />");
|
|
|
|
- out.println(" </queuePlacementPolicy>");
|
|
|
|
- out.println("</allocations>");
|
|
|
|
- out.close();
|
|
|
|
|
|
+ AllocationFileWriter.create()
|
|
|
|
+ .addQueue(new AllocationFileQueue.Builder("queueA")
|
|
|
|
+ .maxRunningApps(1).build())
|
|
|
|
+ .addQueue(new AllocationFileQueue.Builder("queueB").build())
|
|
|
|
+ .queuePlacementPolicy(new AllocationFileQueuePlacementPolicy()
|
|
|
|
+ .addRule(new AllocationFileQueuePlacementRule(
|
|
|
|
+ AllocationFileQueuePlacementRule.RuleName.DEFAULT)))
|
|
|
|
+ .writeToFile(ALLOC_FILE);
|
|
|
|
|
|
ControlledClock clock = new ControlledClock();
|
|
ControlledClock clock = new ControlledClock();
|
|
clock.setTime(0);
|
|
clock.setTime(0);
|
|
@@ -195,20 +198,17 @@ public class TestAllocationFileLoaderService {
|
|
confHolder.allocConf = null;
|
|
confHolder.allocConf = null;
|
|
|
|
|
|
// Modify file and advance the clock
|
|
// Modify file and advance the clock
|
|
- out = new PrintWriter(new FileWriter(ALLOC_FILE));
|
|
|
|
- out.println("<?xml version=\"1.0\"?>");
|
|
|
|
- out.println("<allocations>");
|
|
|
|
- out.println(" <queue name=\"queueB\">");
|
|
|
|
- out.println(" <maxRunningApps>3</maxRunningApps>");
|
|
|
|
- out.println(" </queue>");
|
|
|
|
- out.println(" <queuePlacementPolicy>");
|
|
|
|
- out.println(" <rule name='specified' />");
|
|
|
|
- out.println(" <rule name='nestedUserQueue' >");
|
|
|
|
- out.println(" <rule name='primaryGroup' />");
|
|
|
|
- out.println(" </rule>");
|
|
|
|
- out.println(" </queuePlacementPolicy>");
|
|
|
|
- out.println("</allocations>");
|
|
|
|
- out.close();
|
|
|
|
|
|
+ AllocationFileWriter.create()
|
|
|
|
+ .addQueue(new AllocationFileQueue.Builder("queueB")
|
|
|
|
+ .maxRunningApps(3).build())
|
|
|
|
+ .queuePlacementPolicy(new AllocationFileQueuePlacementPolicy()
|
|
|
|
+ .addRule(new AllocationFileQueuePlacementRule(
|
|
|
|
+ AllocationFileQueuePlacementRule.RuleName.SPECIFIED))
|
|
|
|
+ .addRule(new AllocationFileQueuePlacementRule(
|
|
|
|
+ AllocationFileQueuePlacementRule.RuleName.NESTED)
|
|
|
|
+ .addNestedRule(new AllocationFileQueuePlacementRule(
|
|
|
|
+ AllocationFileQueuePlacementRule.RuleName.PRIMARY_GROUP))))
|
|
|
|
+ .writeToFile(ALLOC_FILE);
|
|
|
|
|
|
clock.tickMsec(System.currentTimeMillis()
|
|
clock.tickMsec(System.currentTimeMillis()
|
|
+ AllocationFileLoaderService.ALLOC_RELOAD_WAIT_MS + 10000);
|
|
+ AllocationFileLoaderService.ALLOC_RELOAD_WAIT_MS + 10000);
|
|
@@ -242,57 +242,56 @@ public class TestAllocationFileLoaderService {
|
|
AllocationFileLoaderService allocLoader =
|
|
AllocationFileLoaderService allocLoader =
|
|
new AllocationFileLoaderService(scheduler);
|
|
new AllocationFileLoaderService(scheduler);
|
|
|
|
|
|
- AllocationFileWriter
|
|
|
|
- .create()
|
|
|
|
|
|
+ AllocationFileWriter.create()
|
|
// Give queue A a minimum of 1024 M
|
|
// Give queue A a minimum of 1024 M
|
|
- .queue("queueA")
|
|
|
|
- .minResources("1024mb,0vcores")
|
|
|
|
- .maxResources("2048mb,10vcores")
|
|
|
|
- .buildQueue()
|
|
|
|
|
|
+ .addQueue(new AllocationFileQueue.Builder("queueA")
|
|
|
|
+ .minResources("1024mb,0vcores")
|
|
|
|
+ .maxResources("2048mb,10vcores")
|
|
|
|
+ .build())
|
|
// Give queue B a minimum of 2048 M
|
|
// Give queue B a minimum of 2048 M
|
|
- .queue("queueB")
|
|
|
|
|
|
+ .addQueue(new AllocationFileQueue.Builder("queueB")
|
|
.minResources("2048mb,0vcores")
|
|
.minResources("2048mb,0vcores")
|
|
.maxResources("5120mb,110vcores")
|
|
.maxResources("5120mb,110vcores")
|
|
.aclAdministerApps("alice,bob admins")
|
|
.aclAdministerApps("alice,bob admins")
|
|
.schedulingPolicy("fair")
|
|
.schedulingPolicy("fair")
|
|
- .buildQueue()
|
|
|
|
|
|
+ .build())
|
|
// Give queue C no minimum
|
|
// Give queue C no minimum
|
|
- .queue("queueC")
|
|
|
|
- .minResources("5120mb,0vcores")
|
|
|
|
- .aclSubmitApps("alice,bob admins")
|
|
|
|
- .buildQueue()
|
|
|
|
|
|
+ .addQueue(new AllocationFileQueue.Builder("queueC")
|
|
|
|
+ .minResources("5120mb,0vcores")
|
|
|
|
+ .aclSubmitApps("alice,bob admins")
|
|
|
|
+ .build())
|
|
// Give queue D a limit of 3 running apps and 0.4f maxAMShare
|
|
// Give queue D a limit of 3 running apps and 0.4f maxAMShare
|
|
- .queue("queueD")
|
|
|
|
- .maxRunningApps(3)
|
|
|
|
- .maxAMShare(0.4)
|
|
|
|
- .buildQueue()
|
|
|
|
|
|
+ .addQueue(new AllocationFileQueue.Builder("queueD")
|
|
|
|
+ .maxRunningApps(3)
|
|
|
|
+ .maxAMShare(0.4)
|
|
|
|
+ .build())
|
|
// Give queue E a preemption timeout of one minute
|
|
// Give queue E a preemption timeout of one minute
|
|
- .queue("queueE")
|
|
|
|
- .minSharePreemptionTimeout(60)
|
|
|
|
- .buildQueue()
|
|
|
|
|
|
+ .addQueue(new AllocationFileQueue.Builder("queueE")
|
|
|
|
+ .minSharePreemptionTimeout(60)
|
|
|
|
+ .build())
|
|
// Make queue F a parent queue without configured leaf queues
|
|
// Make queue F a parent queue without configured leaf queues
|
|
// using the 'type' attribute
|
|
// using the 'type' attribute
|
|
- .queue("queueF")
|
|
|
|
- .parent(true)
|
|
|
|
- .maxChildResources("2048mb,64vcores")
|
|
|
|
- .buildQueue()
|
|
|
|
- .queue("queueG")
|
|
|
|
- .maxChildResources("2048mb,64vcores")
|
|
|
|
- .fairSharePreemptionTimeout(120)
|
|
|
|
- .minSharePreemptionTimeout(50)
|
|
|
|
- .fairSharePreemptionThreshold(0.6)
|
|
|
|
- .maxContainerAllocation(
|
|
|
|
|
|
+ .addQueue(new AllocationFileQueue.Builder("queueF")
|
|
|
|
+ .parent(true)
|
|
|
|
+ .maxChildResources("2048mb,64vcores")
|
|
|
|
+ .build())
|
|
|
|
+ .addQueue(new AllocationFileQueue.Builder("queueG")
|
|
|
|
+ .maxChildResources("2048mb,64vcores")
|
|
|
|
+ .fairSharePreemptionTimeout(120)
|
|
|
|
+ .minSharePreemptionTimeout(50)
|
|
|
|
+ .fairSharePreemptionThreshold(0.6)
|
|
|
|
+ .maxContainerAllocation(
|
|
"vcores=16, memory-mb=512, " + A_CUSTOM_RESOURCE + "=10")
|
|
"vcores=16, memory-mb=512, " + A_CUSTOM_RESOURCE + "=10")
|
|
// Create hierarchical queues G,H, with different min/fair
|
|
// Create hierarchical queues G,H, with different min/fair
|
|
// share preemption timeouts and preemption thresholds.
|
|
// share preemption timeouts and preemption thresholds.
|
|
// Also add a child default to make sure it doesn't impact queue H.
|
|
// Also add a child default to make sure it doesn't impact queue H.
|
|
- .subQueue("queueH")
|
|
|
|
- .fairSharePreemptionTimeout(180)
|
|
|
|
- .minSharePreemptionTimeout(40)
|
|
|
|
- .fairSharePreemptionThreshold(0.7)
|
|
|
|
- .maxContainerAllocation("1024mb,8vcores")
|
|
|
|
- .buildSubQueue()
|
|
|
|
- .buildQueue()
|
|
|
|
|
|
+ .subQueue(new AllocationFileQueue.Builder("queueH")
|
|
|
|
+ .fairSharePreemptionTimeout(180)
|
|
|
|
+ .minSharePreemptionTimeout(40)
|
|
|
|
+ .fairSharePreemptionThreshold(0.7)
|
|
|
|
+ .maxContainerAllocation("1024mb,8vcores")
|
|
|
|
+ .build())
|
|
|
|
+ .build())
|
|
// Set default limit of apps per queue to 15
|
|
// Set default limit of apps per queue to 15
|
|
.queueMaxAppsDefault(15)
|
|
.queueMaxAppsDefault(15)
|
|
// Set default limit of max resource per queue to 4G and 100 cores
|
|
// Set default limit of max resource per queue to 4G and 100 cores
|
|
@@ -308,11 +307,11 @@ public class TestAllocationFileLoaderService {
|
|
// Set default fair share preemption threshold to 0.4
|
|
// Set default fair share preemption threshold to 0.4
|
|
.defaultFairSharePreemptionThreshold(0.4)
|
|
.defaultFairSharePreemptionThreshold(0.4)
|
|
// Set default scheduling policy to DRF
|
|
// Set default scheduling policy to DRF
|
|
- .defaultQueueSchedulingPolicy("drf")
|
|
|
|
|
|
+ .drfDefaultQueueSchedulingPolicy()
|
|
// Give user1 a limit of 10 jobs
|
|
// Give user1 a limit of 10 jobs
|
|
- .userSettings("user1")
|
|
|
|
|
|
+ .userSettings(new UserSettings.Builder("user1")
|
|
.maxRunningApps(10)
|
|
.maxRunningApps(10)
|
|
- .build()
|
|
|
|
|
|
+ .build())
|
|
.writeToFile(ALLOC_FILE);
|
|
.writeToFile(ALLOC_FILE);
|
|
|
|
|
|
allocLoader.init(conf);
|
|
allocLoader.init(conf);
|
|
@@ -488,48 +487,51 @@ public class TestAllocationFileLoaderService {
|
|
AllocationFileLoaderService allocLoader =
|
|
AllocationFileLoaderService allocLoader =
|
|
new AllocationFileLoaderService(scheduler);
|
|
new AllocationFileLoaderService(scheduler);
|
|
|
|
|
|
- PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
|
|
|
|
- out.println("<?xml version=\"1.0\"?>");
|
|
|
|
- out.println("<allocations>");
|
|
|
|
- // Give queue A a minimum of 1024 M
|
|
|
|
- out.println("<pool name=\"queueA\">");
|
|
|
|
- out.println("<minResources>1024mb,0vcores</minResources>");
|
|
|
|
- out.println("</pool>");
|
|
|
|
- // Give queue B a minimum of 2048 M
|
|
|
|
- out.println("<pool name=\"queueB\">");
|
|
|
|
- out.println("<minResources>2048mb,0vcores</minResources>");
|
|
|
|
- out.println("<aclAdministerApps>alice,bob admins</aclAdministerApps>");
|
|
|
|
- out.println("</pool>");
|
|
|
|
- // Give queue C no minimum
|
|
|
|
- out.println("<pool name=\"queueC\">");
|
|
|
|
- out.println("<aclSubmitApps>alice,bob admins</aclSubmitApps>");
|
|
|
|
- out.println("</pool>");
|
|
|
|
- // Give queue D a limit of 3 running apps
|
|
|
|
- out.println("<pool name=\"queueD\">");
|
|
|
|
- out.println("<maxRunningApps>3</maxRunningApps>");
|
|
|
|
- out.println("</pool>");
|
|
|
|
- // Give queue E a preemption timeout of one minute and 0.3f threshold
|
|
|
|
- out.println("<pool name=\"queueE\">");
|
|
|
|
- out.println("<minSharePreemptionTimeout>60</minSharePreemptionTimeout>");
|
|
|
|
- out.println("<fairSharePreemptionThreshold>0.3</fairSharePreemptionThreshold>");
|
|
|
|
- out.println("</pool>");
|
|
|
|
- // Set default limit of apps per queue to 15
|
|
|
|
- out.println("<queueMaxAppsDefault>15</queueMaxAppsDefault>");
|
|
|
|
- // Set default limit of apps per user to 5
|
|
|
|
- out.println("<userMaxAppsDefault>5</userMaxAppsDefault>");
|
|
|
|
- // Give user1 a limit of 10 jobs
|
|
|
|
- out.println("<user name=\"user1\">");
|
|
|
|
- out.println("<maxRunningApps>10</maxRunningApps>");
|
|
|
|
- out.println("</user>");
|
|
|
|
- // Set default min share preemption timeout to 2 minutes
|
|
|
|
- out.println("<defaultMinSharePreemptionTimeout>120"
|
|
|
|
- + "</defaultMinSharePreemptionTimeout>");
|
|
|
|
- // Set fair share preemption timeout to 5 minutes
|
|
|
|
- out.println("<fairSharePreemptionTimeout>300</fairSharePreemptionTimeout>");
|
|
|
|
- // Set default fair share preemption threshold to 0.6f
|
|
|
|
- out.println("<defaultFairSharePreemptionThreshold>0.6</defaultFairSharePreemptionThreshold>");
|
|
|
|
- out.println("</allocations>");
|
|
|
|
- out.close();
|
|
|
|
|
|
+ AllocationFileWriter.create()
|
|
|
|
+ .useLegacyTagNameForQueues()
|
|
|
|
+ // Give queue A a minimum of 1024 M
|
|
|
|
+ .addQueue(new AllocationFileQueue.Builder("queueA")
|
|
|
|
+ .minResources("1024mb,0vcores")
|
|
|
|
+ .build())
|
|
|
|
+ // Give queue B a minimum of 2048 M
|
|
|
|
+ .addQueue(new AllocationFileQueue.Builder("queueB")
|
|
|
|
+ .minResources("2048mb,0vcores")
|
|
|
|
+ .aclAdministerApps("alice,bob admins")
|
|
|
|
+ .build())
|
|
|
|
+ // Give queue C no minimum
|
|
|
|
+ .addQueue(new AllocationFileQueue.Builder("queueC")
|
|
|
|
+ .aclAdministerApps("alice,bob admins")
|
|
|
|
+ .build())
|
|
|
|
+ // Give queue D a limit of 3 running apps
|
|
|
|
+ .addQueue(new AllocationFileQueue.Builder("queueD")
|
|
|
|
+ .maxRunningApps(3)
|
|
|
|
+ .build())
|
|
|
|
+ // Give queue E a preemption timeout of one minute and 0.3f threshold
|
|
|
|
+ .addQueue(new AllocationFileQueue.Builder("queueE")
|
|
|
|
+ .minSharePreemptionTimeout(60)
|
|
|
|
+ .fairSharePreemptionThreshold(0.3)
|
|
|
|
+ .build())
|
|
|
|
+ // Set default limit of apps per queue to 15
|
|
|
|
+ .queueMaxAppsDefault(15)
|
|
|
|
+ // Set default limit of apps per user to 5
|
|
|
|
+ .userMaxAppsDefault(5)
|
|
|
|
+ // Set default limit of max resource per queue to 4G and 100 cores
|
|
|
|
+ .queueMaxResourcesDefault("4096mb,100vcores")
|
|
|
|
+ // Set default limit of AMResourceShare to 0.5f
|
|
|
|
+ .queueMaxAMShareDefault(0.5)
|
|
|
|
+ // Set default min share preemption timeout to 2 minutes
|
|
|
|
+ .defaultMinSharePreemptionTimeout(120)
|
|
|
|
+ // Set default fair share preemption timeout to 5 minutes
|
|
|
|
+ .defaultFairSharePreemptionTimeout(300)
|
|
|
|
+ // Set default fair share preemption threshold to 0.6
|
|
|
|
+ .defaultFairSharePreemptionThreshold(0.6)
|
|
|
|
+ // Set default scheduling policy to DRF
|
|
|
|
+ .drfDefaultQueueSchedulingPolicy()
|
|
|
|
+ // Give user1 a limit of 10 jobs
|
|
|
|
+ .userSettings(new UserSettings.Builder("user1")
|
|
|
|
+ .maxRunningApps(10)
|
|
|
|
+ .build())
|
|
|
|
+ .writeToFile(ALLOC_FILE);
|
|
|
|
|
|
allocLoader.init(conf);
|
|
allocLoader.init(conf);
|
|
ReloadListener confHolder = new ReloadListener();
|
|
ReloadListener confHolder = new ReloadListener();
|
|
@@ -602,11 +604,7 @@ public class TestAllocationFileLoaderService {
|
|
conf.setBoolean(FairSchedulerConfiguration.ALLOW_UNDECLARED_POOLS, false);
|
|
conf.setBoolean(FairSchedulerConfiguration.ALLOW_UNDECLARED_POOLS, false);
|
|
conf.setBoolean(FairSchedulerConfiguration.USER_AS_DEFAULT_QUEUE, false);
|
|
conf.setBoolean(FairSchedulerConfiguration.USER_AS_DEFAULT_QUEUE, false);
|
|
|
|
|
|
- PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
|
|
|
|
- out.println("<?xml version=\"1.0\"?>");
|
|
|
|
- out.println("<allocations>");
|
|
|
|
- out.println("</allocations>");
|
|
|
|
- out.close();
|
|
|
|
|
|
+ AllocationFileWriter.create().writeToFile(ALLOC_FILE);
|
|
|
|
|
|
AllocationFileLoaderService allocLoader =
|
|
AllocationFileLoaderService allocLoader =
|
|
new AllocationFileLoaderService(scheduler);
|
|
new AllocationFileLoaderService(scheduler);
|
|
@@ -632,15 +630,10 @@ public class TestAllocationFileLoaderService {
|
|
public void testQueueAlongsideRoot() throws Exception {
|
|
public void testQueueAlongsideRoot() throws Exception {
|
|
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
|
|
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
|
|
|
|
|
|
- PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
|
|
|
|
- out.println("<?xml version=\"1.0\"?>");
|
|
|
|
- out.println("<allocations>");
|
|
|
|
- out.println("<queue name=\"root\">");
|
|
|
|
- out.println("</queue>");
|
|
|
|
- out.println("<queue name=\"other\">");
|
|
|
|
- out.println("</queue>");
|
|
|
|
- out.println("</allocations>");
|
|
|
|
- out.close();
|
|
|
|
|
|
+ AllocationFileWriter.create()
|
|
|
|
+ .addQueue(new AllocationFileQueue.Builder("root").build())
|
|
|
|
+ .addQueue(new AllocationFileQueue.Builder("other").build())
|
|
|
|
+ .writeToFile(ALLOC_FILE);
|
|
|
|
|
|
AllocationFileLoaderService allocLoader =
|
|
AllocationFileLoaderService allocLoader =
|
|
new AllocationFileLoaderService(scheduler);
|
|
new AllocationFileLoaderService(scheduler);
|
|
@@ -658,13 +651,9 @@ public class TestAllocationFileLoaderService {
|
|
public void testQueueNameContainingPeriods() throws Exception {
|
|
public void testQueueNameContainingPeriods() throws Exception {
|
|
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
|
|
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
|
|
|
|
|
|
- PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
|
|
|
|
- out.println("<?xml version=\"1.0\"?>");
|
|
|
|
- out.println("<allocations>");
|
|
|
|
- out.println("<queue name=\"parent1.child1\">");
|
|
|
|
- out.println("</queue>");
|
|
|
|
- out.println("</allocations>");
|
|
|
|
- out.close();
|
|
|
|
|
|
+ AllocationFileWriter.create()
|
|
|
|
+ .addQueue(new AllocationFileQueue.Builder("parent1.child").build())
|
|
|
|
+ .writeToFile(ALLOC_FILE);
|
|
|
|
|
|
AllocationFileLoaderService allocLoader =
|
|
AllocationFileLoaderService allocLoader =
|
|
new AllocationFileLoaderService(scheduler);
|
|
new AllocationFileLoaderService(scheduler);
|
|
@@ -682,13 +671,9 @@ public class TestAllocationFileLoaderService {
|
|
public void testQueueNameContainingOnlyWhitespace() throws Exception {
|
|
public void testQueueNameContainingOnlyWhitespace() throws Exception {
|
|
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
|
|
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
|
|
|
|
|
|
- PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
|
|
|
|
- out.println("<?xml version=\"1.0\"?>");
|
|
|
|
- out.println("<allocations>");
|
|
|
|
- out.println("<queue name=\" \">");
|
|
|
|
- out.println("</queue>");
|
|
|
|
- out.println("</allocations>");
|
|
|
|
- out.close();
|
|
|
|
|
|
+ AllocationFileWriter.create()
|
|
|
|
+ .addQueue(new AllocationFileQueue.Builder(" ").build())
|
|
|
|
+ .writeToFile(ALLOC_FILE);
|
|
|
|
|
|
AllocationFileLoaderService allocLoader =
|
|
AllocationFileLoaderService allocLoader =
|
|
new AllocationFileLoaderService(scheduler);
|
|
new AllocationFileLoaderService(scheduler);
|
|
@@ -702,15 +687,12 @@ public class TestAllocationFileLoaderService {
|
|
public void testParentTagWithReservation() throws Exception {
|
|
public void testParentTagWithReservation() throws Exception {
|
|
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
|
|
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
|
|
|
|
|
|
- PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
|
|
|
|
- out.println("<?xml version=\"1.0\"?>");
|
|
|
|
- out.println("<allocations>");
|
|
|
|
- out.println("<queue name=\"parent\" type=\"parent\">");
|
|
|
|
- out.println("<reservation>");
|
|
|
|
- out.println("</reservation>");
|
|
|
|
- out.println("</queue>");
|
|
|
|
- out.println("</allocations>");
|
|
|
|
- out.close();
|
|
|
|
|
|
+ AllocationFileWriter.create()
|
|
|
|
+ .addQueue(new AllocationFileQueue.Builder("parent")
|
|
|
|
+ .parent(true)
|
|
|
|
+ .reservation()
|
|
|
|
+ .build())
|
|
|
|
+ .writeToFile(ALLOC_FILE);
|
|
|
|
|
|
AllocationFileLoaderService allocLoader =
|
|
AllocationFileLoaderService allocLoader =
|
|
new AllocationFileLoaderService(scheduler);
|
|
new AllocationFileLoaderService(scheduler);
|
|
@@ -731,17 +713,13 @@ public class TestAllocationFileLoaderService {
|
|
public void testParentWithReservation() throws Exception {
|
|
public void testParentWithReservation() throws Exception {
|
|
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
|
|
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
|
|
|
|
|
|
- PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
|
|
|
|
- out.println("<?xml version=\"1.0\"?>");
|
|
|
|
- out.println("<allocations>");
|
|
|
|
- out.println("<queue name=\"parent\">");
|
|
|
|
- out.println("<reservation>");
|
|
|
|
- out.println("</reservation>");
|
|
|
|
- out.println(" <queue name=\"child\">");
|
|
|
|
- out.println(" </queue>");
|
|
|
|
- out.println("</queue>");
|
|
|
|
- out.println("</allocations>");
|
|
|
|
- out.close();
|
|
|
|
|
|
+ AllocationFileWriter.create()
|
|
|
|
+ .addQueue(new AllocationFileQueue.Builder("parent")
|
|
|
|
+ .parent(true)
|
|
|
|
+ .subQueue(new AllocationFileQueue.Builder("child").build())
|
|
|
|
+ .reservation()
|
|
|
|
+ .build())
|
|
|
|
+ .writeToFile(ALLOC_FILE);
|
|
|
|
|
|
AllocationFileLoaderService allocLoader =
|
|
AllocationFileLoaderService allocLoader =
|
|
new AllocationFileLoaderService(scheduler);
|
|
new AllocationFileLoaderService(scheduler);
|
|
@@ -762,15 +740,12 @@ public class TestAllocationFileLoaderService {
|
|
public void testParentTagWithChild() throws Exception {
|
|
public void testParentTagWithChild() throws Exception {
|
|
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
|
|
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
|
|
|
|
|
|
- PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
|
|
|
|
- out.println("<?xml version=\"1.0\"?>");
|
|
|
|
- out.println("<allocations>");
|
|
|
|
- out.println("<queue name=\"parent\" type=\"parent\">");
|
|
|
|
- out.println(" <queue name=\"child\">");
|
|
|
|
- out.println(" </queue>");
|
|
|
|
- out.println("</queue>");
|
|
|
|
- out.println("</allocations>");
|
|
|
|
- out.close();
|
|
|
|
|
|
+ AllocationFileWriter.create()
|
|
|
|
+ .addQueue(new AllocationFileQueue.Builder("parent")
|
|
|
|
+ .parent(true)
|
|
|
|
+ .subQueue(new AllocationFileQueue.Builder("child").build())
|
|
|
|
+ .build())
|
|
|
|
+ .writeToFile(ALLOC_FILE);
|
|
|
|
|
|
AllocationFileLoaderService allocLoader =
|
|
AllocationFileLoaderService allocLoader =
|
|
new AllocationFileLoaderService(scheduler);
|
|
new AllocationFileLoaderService(scheduler);
|
|
@@ -794,14 +769,9 @@ public class TestAllocationFileLoaderService {
|
|
public void testQueueNameContainingNBWhitespace() throws Exception {
|
|
public void testQueueNameContainingNBWhitespace() throws Exception {
|
|
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
|
|
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
|
|
|
|
|
|
- PrintWriter out = new PrintWriter(new OutputStreamWriter(
|
|
|
|
- new FileOutputStream(ALLOC_FILE), StandardCharsets.UTF_8));
|
|
|
|
- out.println("<?xml version=\"1.0\" encoding=\"UTF-8\"?>");
|
|
|
|
- out.println("<allocations>");
|
|
|
|
- out.println("<queue name=\"\u00a0\">");
|
|
|
|
- out.println("</queue>");
|
|
|
|
- out.println("</allocations>");
|
|
|
|
- out.close();
|
|
|
|
|
|
+ AllocationFileWriter.create()
|
|
|
|
+ .addQueue(new AllocationFileQueue.Builder("\u00a0").build())
|
|
|
|
+ .writeToFile(ALLOC_FILE);
|
|
|
|
|
|
AllocationFileLoaderService allocLoader =
|
|
AllocationFileLoaderService allocLoader =
|
|
new AllocationFileLoaderService(scheduler);
|
|
new AllocationFileLoaderService(scheduler);
|
|
@@ -818,13 +788,9 @@ public class TestAllocationFileLoaderService {
|
|
public void testDefaultQueueSchedulingModeIsFIFO() throws Exception {
|
|
public void testDefaultQueueSchedulingModeIsFIFO() throws Exception {
|
|
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
|
|
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
|
|
|
|
|
|
- PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
|
|
|
|
- out.println("<?xml version=\"1.0\"?>");
|
|
|
|
- out.println("<allocations>");
|
|
|
|
- out.println("<defaultQueueSchedulingPolicy>fifo" +
|
|
|
|
- "</defaultQueueSchedulingPolicy>");
|
|
|
|
- out.println("</allocations>");
|
|
|
|
- out.close();
|
|
|
|
|
|
+ AllocationFileWriter.create()
|
|
|
|
+ .fifoDefaultQueueSchedulingPolicy()
|
|
|
|
+ .writeToFile(ALLOC_FILE);
|
|
|
|
|
|
AllocationFileLoaderService allocLoader =
|
|
AllocationFileLoaderService allocLoader =
|
|
new AllocationFileLoaderService(scheduler);
|
|
new AllocationFileLoaderService(scheduler);
|
|
@@ -838,19 +804,14 @@ public class TestAllocationFileLoaderService {
|
|
public void testReservableQueue() throws Exception {
|
|
public void testReservableQueue() throws Exception {
|
|
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
|
|
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
|
|
|
|
|
|
- PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
|
|
|
|
- out.println("<?xml version=\"1.0\"?>");
|
|
|
|
- out.println("<allocations>");
|
|
|
|
- out.println("<queue name=\"reservable\">");
|
|
|
|
- out.println("<reservation>");
|
|
|
|
- out.println("</reservation>");
|
|
|
|
- out.println("</queue>");
|
|
|
|
- out.println("<queue name=\"other\">");
|
|
|
|
- out.println("</queue>");
|
|
|
|
- out.println("<reservation-agent>DummyAgentName</reservation-agent>");
|
|
|
|
- out.println("<reservation-policy>AnyAdmissionPolicy</reservation-policy>");
|
|
|
|
- out.println("</allocations>");
|
|
|
|
- out.close();
|
|
|
|
|
|
+ AllocationFileWriter.create()
|
|
|
|
+ .addQueue(new AllocationFileQueue.Builder("reservable")
|
|
|
|
+ .reservation()
|
|
|
|
+ .build())
|
|
|
|
+ .addQueue(new AllocationFileQueue.Builder("other").build())
|
|
|
|
+ .reservationAgent("DummyAgentName")
|
|
|
|
+ .reservationPolicy("AnyAdmissionPolicy")
|
|
|
|
+ .writeToFile(ALLOC_FILE);
|
|
|
|
|
|
AllocationFileLoaderService allocLoader =
|
|
AllocationFileLoaderService allocLoader =
|
|
new AllocationFileLoaderService(scheduler);
|
|
new AllocationFileLoaderService(scheduler);
|
|
@@ -900,15 +861,12 @@ public class TestAllocationFileLoaderService {
|
|
throws Exception {
|
|
throws Exception {
|
|
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
|
|
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
|
|
|
|
|
|
- PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
|
|
|
|
- out.println("<?xml version=\"1.0\"?>");
|
|
|
|
- out.println("<allocations>");
|
|
|
|
- out.println("<queue name=\"notboth\" type=\"parent\" >");
|
|
|
|
- out.println("<reservation>");
|
|
|
|
- out.println("</reservation>");
|
|
|
|
- out.println("</queue>");
|
|
|
|
- out.println("</allocations>");
|
|
|
|
- out.close();
|
|
|
|
|
|
+ AllocationFileWriter.create()
|
|
|
|
+ .addQueue(new AllocationFileQueue.Builder("notboth")
|
|
|
|
+ .parent(true)
|
|
|
|
+ .reservation()
|
|
|
|
+ .build())
|
|
|
|
+ .writeToFile(ALLOC_FILE);
|
|
|
|
|
|
AllocationFileLoaderService allocLoader =
|
|
AllocationFileLoaderService allocLoader =
|
|
new AllocationFileLoaderService(scheduler);
|
|
new AllocationFileLoaderService(scheduler);
|