YARN-1287. Consolidate MockClocks. (Sebastian Wong and Anubhav Dhoot via kasha)
This commit is contained in:
parent
df36ad0a08
commit
70fb37cd79
|
@ -69,13 +69,13 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.NodeId;
|
||||
import org.apache.hadoop.yarn.api.records.Token;
|
||||
import org.apache.hadoop.yarn.event.AsyncDispatcher;
|
||||
import org.apache.hadoop.yarn.event.EventHandler;
|
||||
import org.apache.hadoop.yarn.factories.RecordFactory;
|
||||
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
||||
import org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager;
|
||||
import org.apache.hadoop.yarn.util.Clock;
|
||||
import org.apache.hadoop.yarn.util.ControlledClock;
|
||||
import org.apache.hadoop.yarn.util.SystemClock;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
@ -90,7 +90,7 @@ public class TestRuntimeEstimators {
|
|||
private static int MAP_TASKS = 200;
|
||||
private static int REDUCE_TASKS = 150;
|
||||
|
||||
MockClock clock;
|
||||
ControlledClock clock;
|
||||
|
||||
Job myJob;
|
||||
|
||||
|
@ -120,7 +120,7 @@ public class TestRuntimeEstimators {
|
|||
private void coreTestEstimator
|
||||
(TaskRuntimeEstimator testedEstimator, int expectedSpeculations) {
|
||||
estimator = testedEstimator;
|
||||
clock = new MockClock();
|
||||
clock = new ControlledClock();
|
||||
dispatcher = new AsyncDispatcher();
|
||||
myJob = null;
|
||||
slotsInUse.set(0);
|
||||
|
@ -129,7 +129,7 @@ public class TestRuntimeEstimators {
|
|||
successfulSpeculations.set(0);
|
||||
taskTimeSavedBySpeculation.set(0);
|
||||
|
||||
clock.advanceTime(1000);
|
||||
clock.tickMsec(1000);
|
||||
|
||||
Configuration conf = new Configuration();
|
||||
|
||||
|
@ -230,7 +230,7 @@ public class TestRuntimeEstimators {
|
|||
}
|
||||
}
|
||||
|
||||
clock.advanceTime(1000L);
|
||||
clock.tickMsec(1000L);
|
||||
|
||||
if (clock.getTime() % 10000L == 0L) {
|
||||
speculator.scanForSpeculations();
|
||||
|
@ -777,22 +777,6 @@ public class TestRuntimeEstimators {
|
|||
}
|
||||
}
|
||||
|
||||
static class MockClock implements Clock {
|
||||
private long currentTime = 0;
|
||||
|
||||
public long getTime() {
|
||||
return currentTime;
|
||||
}
|
||||
|
||||
void setMeasuredTime(long newTime) {
|
||||
currentTime = newTime;
|
||||
}
|
||||
|
||||
void advanceTime(long increment) {
|
||||
currentTime += increment;
|
||||
}
|
||||
}
|
||||
|
||||
class MyAppMaster extends CompositeService {
|
||||
final Clock clock;
|
||||
public MyAppMaster(Clock clock) {
|
||||
|
|
|
@ -218,6 +218,9 @@ Release 2.8.0 - UNRELEASED
|
|||
YARN-1912. ResourceLocalizer started without any jvm memory control.
|
||||
(Masatake Iwasaki via xgong)
|
||||
|
||||
YARN-1287. Consolidate MockClocks.
|
||||
(Sebastian Wong and Anubhav Dhoot via kasha)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
YARN-3339. TestDockerContainerExecutor should pull a single image and not
|
||||
|
|
|
@ -21,6 +21,11 @@ package org.apache.hadoop.yarn.util;
|
|||
public class ControlledClock implements Clock {
|
||||
private long time = -1;
|
||||
private final Clock actualClock;
|
||||
// Convenience for getting a controlled clock with overridden time
|
||||
public ControlledClock() {
|
||||
this(new SystemClock());
|
||||
setTime(0);
|
||||
}
|
||||
public ControlledClock(Clock actualClock) {
|
||||
this.actualClock = actualClock;
|
||||
}
|
||||
|
@ -30,6 +35,16 @@ public class ControlledClock implements Clock {
|
|||
public synchronized void reset() {
|
||||
time = -1;
|
||||
}
|
||||
public synchronized void tickSec(int seconds) {
|
||||
tickMsec(seconds * 1000L);
|
||||
}
|
||||
public synchronized void tickMsec(long millisec) {
|
||||
if (time == -1) {
|
||||
throw new IllegalStateException("ControlledClock setTime should be " +
|
||||
"called before incrementing time");
|
||||
}
|
||||
time = time + millisec;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized long getTime() {
|
||||
|
|
|
@ -22,10 +22,11 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
|
|||
import org.apache.hadoop.yarn.api.records.Resource;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.LinuxContainerExecutor;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.TestCGroupsHandlerImpl;
|
||||
|
||||
import org.apache.hadoop.yarn.util.ControlledClock;
|
||||
import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin;
|
||||
import org.junit.Assert;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.util.Clock;
|
||||
import org.junit.Test;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
|
@ -39,14 +40,6 @@ import java.util.concurrent.CountDownLatch;
|
|||
public class TestCgroupsLCEResourcesHandler {
|
||||
static File cgroupDir = null;
|
||||
|
||||
static class MockClock implements Clock {
|
||||
long time;
|
||||
@Override
|
||||
public long getTime() {
|
||||
return time;
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
cgroupDir =
|
||||
|
@ -93,8 +86,7 @@ public class TestCgroupsLCEResourcesHandler {
|
|||
// Verify DeleteCgroup times out if "tasks" file contains data
|
||||
@Test
|
||||
public void testDeleteCgroup() throws Exception {
|
||||
final MockClock clock = new MockClock();
|
||||
clock.time = System.currentTimeMillis();
|
||||
final ControlledClock clock = new ControlledClock();
|
||||
CgroupsLCEResourcesHandler handler = new CgroupsLCEResourcesHandler();
|
||||
handler.setConf(new YarnConfiguration());
|
||||
handler.initConfig();
|
||||
|
@ -118,8 +110,8 @@ public class TestCgroupsLCEResourcesHandler {
|
|||
} catch (InterruptedException ex) {
|
||||
//NOP
|
||||
}
|
||||
clock.time += YarnConfiguration.
|
||||
DEFAULT_NM_LINUX_CONTAINER_CGROUPS_DELETE_TIMEOUT;
|
||||
clock.tickMsec(YarnConfiguration.
|
||||
DEFAULT_NM_LINUX_CONTAINER_CGROUPS_DELETE_TIMEOUT);
|
||||
}
|
||||
}.start();
|
||||
latch.await();
|
||||
|
|
|
@ -49,18 +49,6 @@ import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|||
import org.apache.hadoop.yarn.util.Clock;
|
||||
|
||||
public class FairSchedulerTestBase {
|
||||
protected static class MockClock implements Clock {
|
||||
private long time = 0;
|
||||
@Override
|
||||
public long getTime() {
|
||||
return time;
|
||||
}
|
||||
|
||||
public void tick(int seconds) {
|
||||
time = time + seconds * 1000;
|
||||
}
|
||||
}
|
||||
|
||||
public final static String TEST_DIR =
|
||||
new File(System.getProperty("test.build.data", "/tmp")).getAbsolutePath();
|
||||
|
||||
|
|
|
@ -31,7 +31,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSche
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.QueuePlacementRule.NestedUserQueue;
|
||||
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.util.Clock;
|
||||
import org.apache.hadoop.yarn.util.ControlledClock;
|
||||
import org.apache.hadoop.yarn.util.resource.Resources;
|
||||
import org.junit.Test;
|
||||
|
||||
|
@ -43,18 +43,6 @@ public class TestAllocationFileLoaderService {
|
|||
final static String ALLOC_FILE = new File(TEST_DIR,
|
||||
"test-queues").getAbsolutePath();
|
||||
|
||||
private class MockClock implements Clock {
|
||||
private long time = 0;
|
||||
@Override
|
||||
public long getTime() {
|
||||
return time;
|
||||
}
|
||||
|
||||
public void tick(long ms) {
|
||||
time += ms;
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetAllocationFileFromClasspath() {
|
||||
Configuration conf = new Configuration();
|
||||
|
@ -81,7 +69,8 @@ public class TestAllocationFileLoaderService {
|
|||
out.println("</allocations>");
|
||||
out.close();
|
||||
|
||||
MockClock clock = new MockClock();
|
||||
ControlledClock clock = new ControlledClock();
|
||||
clock.setTime(0);
|
||||
Configuration conf = new Configuration();
|
||||
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
|
||||
|
||||
|
@ -126,7 +115,7 @@ public class TestAllocationFileLoaderService {
|
|||
out.println("</allocations>");
|
||||
out.close();
|
||||
|
||||
clock.tick(System.currentTimeMillis()
|
||||
clock.tickMsec(System.currentTimeMillis()
|
||||
+ AllocationFileLoaderService.ALLOC_RELOAD_WAIT_MS + 10000);
|
||||
allocLoader.start();
|
||||
|
||||
|
|
|
@ -19,20 +19,16 @@
|
|||
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.metrics2.lib.MutableRate;
|
||||
import org.apache.hadoop.metrics2.util.SampleStat;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.Resource;
|
||||
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
|
||||
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event
|
||||
.NodeUpdateSchedulerEvent;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
|
||||
import org.apache.hadoop.yarn.util.ControlledClock;
|
||||
import org.apache.hadoop.yarn.util.resource.Resources;
|
||||
import org.junit.After;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
@ -41,11 +37,10 @@ import org.junit.Before;
|
|||
import org.junit.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
public class TestContinuousScheduling extends FairSchedulerTestBase {
|
||||
private MockClock mockClock;
|
||||
private ControlledClock mockClock;
|
||||
|
||||
@Override
|
||||
public Configuration createConfiguration() {
|
||||
|
@ -59,7 +54,7 @@ public class TestContinuousScheduling extends FairSchedulerTestBase {
|
|||
|
||||
@Before
|
||||
public void setup() {
|
||||
mockClock = new MockClock();
|
||||
mockClock = new ControlledClock();
|
||||
conf = createConfiguration();
|
||||
resourceManager = new MockRM(conf);
|
||||
resourceManager.start();
|
||||
|
@ -108,7 +103,7 @@ public class TestContinuousScheduling extends FairSchedulerTestBase {
|
|||
FSAppAttempt app = scheduler.getSchedulerApp(appAttemptId);
|
||||
|
||||
// Advance time and let continuous scheduling kick in
|
||||
mockClock.tick(1);
|
||||
mockClock.tickSec(1);
|
||||
while (1024 != app.getCurrentConsumption().getMemory()) {
|
||||
Thread.sleep(100);
|
||||
}
|
||||
|
|
|
@ -31,7 +31,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
|
|||
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.FifoPolicy;
|
||||
import org.apache.hadoop.yarn.util.Clock;
|
||||
|
||||
import org.apache.hadoop.yarn.util.ControlledClock;
|
||||
import org.apache.hadoop.yarn.util.resource.Resources;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
@ -39,19 +40,6 @@ import org.mockito.Mockito;
|
|||
|
||||
public class TestFSAppAttempt extends FairSchedulerTestBase {
|
||||
|
||||
private class MockClock implements Clock {
|
||||
private long time = 0;
|
||||
@Override
|
||||
public long getTime() {
|
||||
return time;
|
||||
}
|
||||
|
||||
public void tick(int seconds) {
|
||||
time = time + seconds * 1000;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setup() {
|
||||
Configuration conf = createConfiguration();
|
||||
|
@ -125,7 +113,7 @@ public class TestFSAppAttempt extends FairSchedulerTestBase {
|
|||
Priority prio = Mockito.mock(Priority.class);
|
||||
Mockito.when(prio.getPriority()).thenReturn(1);
|
||||
|
||||
MockClock clock = new MockClock();
|
||||
ControlledClock clock = new ControlledClock();
|
||||
scheduler.setClock(clock);
|
||||
|
||||
long nodeLocalityDelayMs = 5 * 1000L; // 5 seconds
|
||||
|
@ -143,13 +131,13 @@ public class TestFSAppAttempt extends FairSchedulerTestBase {
|
|||
nodeLocalityDelayMs, rackLocalityDelayMs, clock.getTime()));
|
||||
|
||||
// after 4 seconds should remain node local
|
||||
clock.tick(4);
|
||||
clock.tickSec(4);
|
||||
assertEquals(NodeType.NODE_LOCAL,
|
||||
schedulerApp.getAllowedLocalityLevelByTime(prio,
|
||||
nodeLocalityDelayMs, rackLocalityDelayMs, clock.getTime()));
|
||||
|
||||
// after 6 seconds should switch to rack local
|
||||
clock.tick(2);
|
||||
clock.tickSec(2);
|
||||
assertEquals(NodeType.RACK_LOCAL,
|
||||
schedulerApp.getAllowedLocalityLevelByTime(prio,
|
||||
nodeLocalityDelayMs, rackLocalityDelayMs, clock.getTime()));
|
||||
|
@ -162,12 +150,12 @@ public class TestFSAppAttempt extends FairSchedulerTestBase {
|
|||
nodeLocalityDelayMs, rackLocalityDelayMs, clock.getTime()));
|
||||
|
||||
// Now escalate again to rack-local, then to off-switch
|
||||
clock.tick(6);
|
||||
clock.tickSec(6);
|
||||
assertEquals(NodeType.RACK_LOCAL,
|
||||
schedulerApp.getAllowedLocalityLevelByTime(prio,
|
||||
nodeLocalityDelayMs, rackLocalityDelayMs, clock.getTime()));
|
||||
|
||||
clock.tick(7);
|
||||
clock.tickSec(7);
|
||||
assertEquals(NodeType.OFF_SWITCH,
|
||||
schedulerApp.getAllowedLocalityLevelByTime(prio,
|
||||
nodeLocalityDelayMs, rackLocalityDelayMs, clock.getTime()));
|
||||
|
|
|
@ -72,7 +72,6 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceType;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.MockRMApp;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
||||
|
@ -100,6 +99,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.QueuePlaceme
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.DominantResourceFairnessPolicy;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FifoPolicy;
|
||||
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
||||
import org.apache.hadoop.yarn.util.ControlledClock;
|
||||
import org.apache.hadoop.yarn.util.SystemClock;
|
||||
import org.apache.hadoop.yarn.util.resource.Resources;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
|
@ -1489,7 +1490,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE + ".allocation.file", ALLOC_FILE);
|
||||
conf.set(FairSchedulerConfiguration.USER_AS_DEFAULT_QUEUE, "false");
|
||||
|
||||
MockClock clock = new MockClock();
|
||||
ControlledClock clock = new ControlledClock();
|
||||
scheduler.setClock(clock);
|
||||
|
||||
PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
|
||||
|
@ -1587,7 +1588,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
scheduler.getSchedulerApp(app2).getPreemptionContainers()));
|
||||
|
||||
// Pretend 15 seconds have passed
|
||||
clock.tick(15);
|
||||
clock.tickSec(15);
|
||||
|
||||
// Trigger a kill by insisting we want containers back
|
||||
scheduler.preemptResources(Resources.createResource(2 * 1024));
|
||||
|
@ -1617,7 +1618,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
scheduler.preemptResources(Resources.createResource(2 * 1024));
|
||||
|
||||
// Pretend 15 seconds have passed
|
||||
clock.tick(15);
|
||||
clock.tickSec(15);
|
||||
|
||||
// We should be able to claw back another container from A and B each.
|
||||
// For queueA (fifo), continue preempting from app2.
|
||||
|
@ -1649,7 +1650,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
|
||||
conf.set(FairSchedulerConfiguration.USER_AS_DEFAULT_QUEUE, "false");
|
||||
|
||||
MockClock clock = new MockClock();
|
||||
ControlledClock clock = new ControlledClock();
|
||||
scheduler.setClock(clock);
|
||||
|
||||
PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
|
||||
|
@ -1702,7 +1703,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
scheduler.update();
|
||||
|
||||
// Let 11 sec pass
|
||||
clock.tick(11);
|
||||
clock.tickSec(11);
|
||||
|
||||
scheduler.update();
|
||||
Resource toPreempt = scheduler.resToPreempt(scheduler.getQueueManager()
|
||||
|
@ -1722,7 +1723,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
*/
|
||||
public void testPreemptionDecision() throws Exception {
|
||||
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
|
||||
MockClock clock = new MockClock();
|
||||
ControlledClock clock = new ControlledClock();
|
||||
scheduler.setClock(clock);
|
||||
|
||||
PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
|
||||
|
@ -1833,7 +1834,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
Resources.none(), scheduler.resToPreempt(schedD, clock.getTime())));
|
||||
// After minSharePreemptionTime has passed, they should want to preempt min
|
||||
// share.
|
||||
clock.tick(6);
|
||||
clock.tickSec(6);
|
||||
assertEquals(
|
||||
1024, scheduler.resToPreempt(schedC, clock.getTime()).getMemory());
|
||||
assertEquals(
|
||||
|
@ -1842,7 +1843,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
// After fairSharePreemptionTime has passed, they should want to preempt
|
||||
// fair share.
|
||||
scheduler.update();
|
||||
clock.tick(6);
|
||||
clock.tickSec(6);
|
||||
assertEquals(
|
||||
1536 , scheduler.resToPreempt(schedC, clock.getTime()).getMemory());
|
||||
assertEquals(
|
||||
|
@ -1855,7 +1856,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
*/
|
||||
public void testPreemptionDecisionWithVariousTimeout() throws Exception {
|
||||
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
|
||||
MockClock clock = new MockClock();
|
||||
ControlledClock clock = new ControlledClock();
|
||||
scheduler.setClock(clock);
|
||||
|
||||
PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
|
||||
|
@ -1971,7 +1972,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
|
||||
// After 5 seconds, queueB1 wants to preempt min share
|
||||
scheduler.update();
|
||||
clock.tick(6);
|
||||
clock.tickSec(6);
|
||||
assertEquals(
|
||||
1024, scheduler.resToPreempt(queueB1, clock.getTime()).getMemory());
|
||||
assertEquals(
|
||||
|
@ -1981,7 +1982,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
|
||||
// After 10 seconds, queueB2 wants to preempt min share
|
||||
scheduler.update();
|
||||
clock.tick(5);
|
||||
clock.tickSec(5);
|
||||
assertEquals(
|
||||
1024, scheduler.resToPreempt(queueB1, clock.getTime()).getMemory());
|
||||
assertEquals(
|
||||
|
@ -1991,7 +1992,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
|
||||
// After 15 seconds, queueC wants to preempt min share
|
||||
scheduler.update();
|
||||
clock.tick(5);
|
||||
clock.tickSec(5);
|
||||
assertEquals(
|
||||
1024, scheduler.resToPreempt(queueB1, clock.getTime()).getMemory());
|
||||
assertEquals(
|
||||
|
@ -2001,7 +2002,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
|
||||
// After 20 seconds, queueB2 should want to preempt fair share
|
||||
scheduler.update();
|
||||
clock.tick(5);
|
||||
clock.tickSec(5);
|
||||
assertEquals(
|
||||
1024, scheduler.resToPreempt(queueB1, clock.getTime()).getMemory());
|
||||
assertEquals(
|
||||
|
@ -2011,7 +2012,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
|
||||
// After 25 seconds, queueB1 should want to preempt fair share
|
||||
scheduler.update();
|
||||
clock.tick(5);
|
||||
clock.tickSec(5);
|
||||
assertEquals(
|
||||
1536, scheduler.resToPreempt(queueB1, clock.getTime()).getMemory());
|
||||
assertEquals(
|
||||
|
@ -2021,7 +2022,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
|
||||
// After 30 seconds, queueC should want to preempt fair share
|
||||
scheduler.update();
|
||||
clock.tick(5);
|
||||
clock.tickSec(5);
|
||||
assertEquals(
|
||||
1536, scheduler.resToPreempt(queueB1, clock.getTime()).getMemory());
|
||||
assertEquals(
|
||||
|
@ -3703,7 +3704,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
@Test
|
||||
public void testMaxRunningAppsHierarchicalQueues() throws Exception {
|
||||
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
|
||||
MockClock clock = new MockClock();
|
||||
ControlledClock clock = new ControlledClock();
|
||||
scheduler.setClock(clock);
|
||||
|
||||
PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
|
||||
|
@ -3728,28 +3729,28 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
ApplicationAttemptId attId1 = createSchedulingRequest(1024, "queue1.sub1", "user1");
|
||||
verifyAppRunnable(attId1, true);
|
||||
verifyQueueNumRunnable("queue1.sub1", 1, 0);
|
||||
clock.tick(10);
|
||||
clock.tickSec(10);
|
||||
// exceeds no limits
|
||||
ApplicationAttemptId attId2 = createSchedulingRequest(1024, "queue1.sub3", "user1");
|
||||
verifyAppRunnable(attId2, true);
|
||||
verifyQueueNumRunnable("queue1.sub3", 1, 0);
|
||||
clock.tick(10);
|
||||
clock.tickSec(10);
|
||||
// exceeds no limits
|
||||
ApplicationAttemptId attId3 = createSchedulingRequest(1024, "queue1.sub2", "user1");
|
||||
verifyAppRunnable(attId3, true);
|
||||
verifyQueueNumRunnable("queue1.sub2", 1, 0);
|
||||
clock.tick(10);
|
||||
clock.tickSec(10);
|
||||
// exceeds queue1 limit
|
||||
ApplicationAttemptId attId4 = createSchedulingRequest(1024, "queue1.sub2", "user1");
|
||||
verifyAppRunnable(attId4, false);
|
||||
verifyQueueNumRunnable("queue1.sub2", 1, 1);
|
||||
clock.tick(10);
|
||||
clock.tickSec(10);
|
||||
// exceeds sub3 limit
|
||||
ApplicationAttemptId attId5 = createSchedulingRequest(1024, "queue1.sub3", "user1");
|
||||
verifyAppRunnable(attId5, false);
|
||||
verifyQueueNumRunnable("queue1.sub3", 1, 1);
|
||||
clock.tick(10);
|
||||
|
||||
clock.tickSec(10);
|
||||
|
||||
// Even though the app was removed from sub3, the app from sub2 gets to go
|
||||
// because it came in first
|
||||
AppAttemptRemovedSchedulerEvent appRemovedEvent1 =
|
||||
|
@ -3923,7 +3924,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
public void testRecoverRequestAfterPreemption() throws Exception {
|
||||
conf.setLong(FairSchedulerConfiguration.WAIT_TIME_BEFORE_KILL, 10);
|
||||
|
||||
MockClock clock = new MockClock();
|
||||
ControlledClock clock = new ControlledClock();
|
||||
scheduler.setClock(clock);
|
||||
scheduler.init(conf);
|
||||
scheduler.start();
|
||||
|
@ -3974,8 +3975,8 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
scheduler.warnOrKillContainer(rmContainer);
|
||||
|
||||
// Wait for few clock ticks
|
||||
clock.tick(5);
|
||||
|
||||
clock.tickSec(5);
|
||||
|
||||
// preempt now
|
||||
scheduler.warnOrKillContainer(rmContainer);
|
||||
|
||||
|
|
|
@ -26,6 +26,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
|
||||
|
||||
import org.apache.hadoop.yarn.util.ControlledClock;
|
||||
import org.apache.hadoop.yarn.util.resource.Resources;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
|
@ -43,7 +45,7 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
|
|||
private final static String ALLOC_FILE = new File(TEST_DIR,
|
||||
TestFairSchedulerPreemption.class.getName() + ".xml").getAbsolutePath();
|
||||
|
||||
private MockClock clock;
|
||||
private ControlledClock clock;
|
||||
|
||||
private static class StubbedFairScheduler extends FairScheduler {
|
||||
public int lastPreemptMemory = -1;
|
||||
|
@ -70,7 +72,7 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
|
|||
@Before
|
||||
public void setup() throws IOException {
|
||||
conf = createConfiguration();
|
||||
clock = new MockClock();
|
||||
clock = new ControlledClock();
|
||||
}
|
||||
|
||||
@After
|
||||
|
@ -148,7 +150,7 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
|
|||
// Verify submitting another request triggers preemption
|
||||
createSchedulingRequest(1024, "queueB", "user1", 1, 1);
|
||||
scheduler.update();
|
||||
clock.tick(6);
|
||||
clock.tickSec(6);
|
||||
|
||||
((StubbedFairScheduler) scheduler).resetLastPreemptResources();
|
||||
scheduler.preemptTasksIfNecessary();
|
||||
|
@ -164,7 +166,7 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
|
|||
// Verify submitting another request doesn't trigger preemption
|
||||
createSchedulingRequest(1024, "queueB", "user1", 1, 1);
|
||||
scheduler.update();
|
||||
clock.tick(6);
|
||||
clock.tickSec(6);
|
||||
|
||||
((StubbedFairScheduler) scheduler).resetLastPreemptResources();
|
||||
scheduler.preemptTasksIfNecessary();
|
||||
|
@ -180,7 +182,7 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
|
|||
// Verify submitting another request triggers preemption
|
||||
createSchedulingRequest(1024, "queueB", "user1", 1, 1);
|
||||
scheduler.update();
|
||||
clock.tick(6);
|
||||
clock.tickSec(6);
|
||||
|
||||
((StubbedFairScheduler) scheduler).resetLastPreemptResources();
|
||||
scheduler.preemptTasksIfNecessary();
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.util.ControlledClock;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
|
@ -40,14 +41,14 @@ public class TestMaxRunningAppsEnforcer {
|
|||
private Map<String, Integer> userMaxApps;
|
||||
private MaxRunningAppsEnforcer maxAppsEnforcer;
|
||||
private int appNum;
|
||||
private TestFairScheduler.MockClock clock;
|
||||
private ControlledClock clock;
|
||||
private RMContext rmContext;
|
||||
private FairScheduler scheduler;
|
||||
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
Configuration conf = new Configuration();
|
||||
clock = new TestFairScheduler.MockClock();
|
||||
clock = new ControlledClock();
|
||||
scheduler = mock(FairScheduler.class);
|
||||
when(scheduler.getConf()).thenReturn(
|
||||
new FairSchedulerConfiguration(conf));
|
||||
|
@ -151,7 +152,7 @@ public class TestMaxRunningAppsEnforcer {
|
|||
FSAppAttempt app1 = addApp(leaf1, "user");
|
||||
addApp(leaf2, "user");
|
||||
addApp(leaf2, "user");
|
||||
clock.tick(20);
|
||||
clock.tickSec(20);
|
||||
addApp(leaf1, "user");
|
||||
assertEquals(1, leaf1.getNumRunnableApps());
|
||||
assertEquals(1, leaf2.getNumRunnableApps());
|
||||
|
|
Loading…
Reference in New Issue