Skip to content

Commit

Permalink
MAPREDUCE-6265. Make ContainerLauncherImpl.INITIAL_POOL_SIZE configur…
Browse files Browse the repository at this point in the history
…able to better control to launch/kill containers. Contributed by Zhihai Xu
  • Loading branch information
oza committed Mar 14, 2015
1 parent 32741cf commit 9d38520
Show file tree
Hide file tree
Showing 5 changed files with 45 additions and 9 deletions.
3 changes: 3 additions & 0 deletions hadoop-mapreduce-project/CHANGES.txt
Expand Up @@ -340,6 +340,9 @@ Release 2.7.0 - UNRELEASED
MAPREDUCE-6263. Configurable timeout between YARNRunner terminate the
application and forcefully kill. (Eric Payne via junping_du)

MAPREDUCE-6265. Make ContainerLauncherImpl.INITIAL_POOL_SIZE configurable
to better control to launch/kill containers. (Zhihai Xu via ozawa)

OPTIMIZATIONS

MAPREDUCE-6169. MergeQueue should release reference to the current item
Expand Down
Expand Up @@ -70,7 +70,7 @@ public class ContainerLauncherImpl extends AbstractService implements
new ConcurrentHashMap<ContainerId, Container>();
private final AppContext context;
protected ThreadPoolExecutor launcherPool;
protected static final int INITIAL_POOL_SIZE = 10;
protected int initialPoolSize;
private int limitOnPoolSize;
private Thread eventHandlingThread;
protected BlockingQueue<ContainerLauncherEvent> eventQueue =
Expand Down Expand Up @@ -246,6 +246,12 @@ protected void serviceInit(Configuration conf) throws Exception {
MRJobConfig.MR_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT,
MRJobConfig.DEFAULT_MR_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT);
LOG.info("Upper limit on the thread pool size is " + this.limitOnPoolSize);

this.initialPoolSize = conf.getInt(
MRJobConfig.MR_AM_CONTAINERLAUNCHER_THREADPOOL_INITIAL_SIZE,
MRJobConfig.DEFAULT_MR_AM_CONTAINERLAUNCHER_THREADPOOL_INITIAL_SIZE);
LOG.info("The thread pool initial size is " + this.initialPoolSize);

super.serviceInit(conf);
cmProxy = new ContainerManagementProtocolProxy(conf);
}
Expand All @@ -256,7 +262,7 @@ protected void serviceStart() throws Exception {
"ContainerLauncher #%d").setDaemon(true).build();

// Start with a default core-pool size of 10 and change it dynamically.
launcherPool = new ThreadPoolExecutor(INITIAL_POOL_SIZE,
launcherPool = new ThreadPoolExecutor(initialPoolSize,
Integer.MAX_VALUE, 1, TimeUnit.HOURS,
new LinkedBlockingQueue<Runnable>(),
tf);
Expand Down Expand Up @@ -289,11 +295,11 @@ public void run() {
int idealPoolSize = Math.min(limitOnPoolSize, numNodes);

if (poolSize < idealPoolSize) {
// Bump up the pool size to idealPoolSize+INITIAL_POOL_SIZE, the
// Bump up the pool size to idealPoolSize+initialPoolSize, the
// later is just a buffer so we are not always increasing the
// pool-size
int newPoolSize = Math.min(limitOnPoolSize, idealPoolSize
+ INITIAL_POOL_SIZE);
+ initialPoolSize);
LOG.info("Setting ContainerLauncher pool size to " + newPoolSize
+ " as number-of-nodes to talk to is " + numNodes);
launcherPool.setCorePoolSize(newPoolSize);
Expand Down
Expand Up @@ -90,7 +90,7 @@ public class TestContainerLauncher {

static final Log LOG = LogFactory.getLog(TestContainerLauncher.class);

@Test (timeout = 5000)
@Test (timeout = 10000)
public void testPoolSize() throws InterruptedException {

ApplicationId appId = ApplicationId.newInstance(12345, 67);
Expand All @@ -108,12 +108,14 @@ public void testPoolSize() throws InterruptedException {
ThreadPoolExecutor threadPool = containerLauncher.getThreadPool();

// No events yet
Assert.assertEquals(containerLauncher.initialPoolSize,
MRJobConfig.DEFAULT_MR_AM_CONTAINERLAUNCHER_THREADPOOL_INITIAL_SIZE);
Assert.assertEquals(0, threadPool.getPoolSize());
Assert.assertEquals(ContainerLauncherImpl.INITIAL_POOL_SIZE,
Assert.assertEquals(containerLauncher.initialPoolSize,
threadPool.getCorePoolSize());
Assert.assertNull(containerLauncher.foundErrors);

containerLauncher.expectedCorePoolSize = ContainerLauncherImpl.INITIAL_POOL_SIZE;
containerLauncher.expectedCorePoolSize = containerLauncher.initialPoolSize;
for (int i = 0; i < 10; i++) {
ContainerId containerId = ContainerId.newContainerId(appAttemptId, i);
TaskAttemptId taskAttemptId = MRBuilderUtils.newTaskAttemptId(taskId, i);
Expand Down Expand Up @@ -152,7 +154,7 @@ public void testPoolSize() throws InterruptedException {
// Different hosts, there should be an increase in core-thread-pool size to
// 21(11hosts+10buffer)
// Core pool size should be 21 but the live pool size should be only 11.
containerLauncher.expectedCorePoolSize = 11 + ContainerLauncherImpl.INITIAL_POOL_SIZE;
containerLauncher.expectedCorePoolSize = 11 + containerLauncher.initialPoolSize;
containerLauncher.finishEventHandling = false;
ContainerId containerId = ContainerId.newContainerId(appAttemptId, 21);
TaskAttemptId taskAttemptId = MRBuilderUtils.newTaskAttemptId(taskId, 21);
Expand All @@ -164,6 +166,15 @@ public void testPoolSize() throws InterruptedException {
Assert.assertNull(containerLauncher.foundErrors);

containerLauncher.stop();

// change configuration MR_AM_CONTAINERLAUNCHER_THREADPOOL_INITIAL_SIZE
// and verify initialPoolSize value.
Configuration conf = new Configuration();
conf.setInt(MRJobConfig.MR_AM_CONTAINERLAUNCHER_THREADPOOL_INITIAL_SIZE,
20);
containerLauncher = new CustomContainerLauncher(context);
containerLauncher.init(conf);
Assert.assertEquals(containerLauncher.initialPoolSize, 20);
}

@Test(timeout = 5000)
Expand All @@ -187,7 +198,7 @@ public void testPoolLimits() throws InterruptedException {
ThreadPoolExecutor threadPool = containerLauncher.getThreadPool();

// 10 different hosts
containerLauncher.expectedCorePoolSize = ContainerLauncherImpl.INITIAL_POOL_SIZE;
containerLauncher.expectedCorePoolSize = containerLauncher.initialPoolSize;
for (int i = 0; i < 10; i++) {
containerLauncher.handle(new ContainerLauncherEvent(taskAttemptId,
containerId, "host" + i + ":1234", null,
Expand Down
Expand Up @@ -504,6 +504,14 @@ public interface MRJobConfig {
public static final int DEFAULT_MR_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT =
500;

/**
* The initial size of thread pool to launch containers in the app master
*/
public static final String MR_AM_CONTAINERLAUNCHER_THREADPOOL_INITIAL_SIZE =
MR_AM_PREFIX+"containerlauncher.threadpool-initial-size";
public static final int DEFAULT_MR_AM_CONTAINERLAUNCHER_THREADPOOL_INITIAL_SIZE =
10;

/** Number of threads to handle job client RPC requests.*/
public static final String MR_AM_JOB_CLIENT_THREAD_COUNT =
MR_AM_PREFIX + "job.client.thread-count";
Expand Down
Expand Up @@ -1694,4 +1694,12 @@
calculated as (heapSize / mapreduce.heap.memory-mb.ratio).
</description>
</property>

<property>
<name>yarn.app.mapreduce.am.containerlauncher.threadpool-initial-size</name>
<value>10</value>
<description>The initial size of thread pool to launch containers in the
app master.
</description>
</property>
</configuration>

0 comments on commit 9d38520

Please sign in to comment.