|
@@ -163,6 +163,10 @@ public class ShuffleHandler extends AuxiliaryService {
|
|
|
|
|
|
public static final String MAX_SHUFFLE_CONNECTIONS = "mapreduce.shuffle.max.connections";
|
|
|
public static final int DEFAULT_MAX_SHUFFLE_CONNECTIONS = 0; // 0 implies no limit
|
|
|
+
|
|
|
+ public static final String MAX_SHUFFLE_THREADS = "mapreduce.shuffle.max.threads";
|
|
|
+ // 0 implies Netty default of 2 * number of available processors
|
|
|
+ public static final int DEFAULT_MAX_SHUFFLE_THREADS = 0;
|
|
|
|
|
|
@Metrics(about="Shuffle output metrics", context="mapred")
|
|
|
static class ShuffleMetrics implements ChannelFutureListener {
|
|
@@ -282,6 +286,11 @@ public class ShuffleHandler extends AuxiliaryService {
|
|
|
|
|
|
maxShuffleConnections = conf.getInt(MAX_SHUFFLE_CONNECTIONS,
|
|
|
DEFAULT_MAX_SHUFFLE_CONNECTIONS);
|
|
|
+ int maxShuffleThreads = conf.getInt(MAX_SHUFFLE_THREADS,
|
|
|
+ DEFAULT_MAX_SHUFFLE_THREADS);
|
|
|
+ if (maxShuffleThreads == 0) {
|
|
|
+ maxShuffleThreads = 2 * Runtime.getRuntime().availableProcessors();
|
|
|
+ }
|
|
|
|
|
|
ThreadFactory bossFactory = new ThreadFactoryBuilder()
|
|
|
.setNameFormat("ShuffleHandler Netty Boss #%d")
|
|
@@ -292,7 +301,8 @@ public class ShuffleHandler extends AuxiliaryService {
|
|
|
|
|
|
selector = new NioServerSocketChannelFactory(
|
|
|
Executors.newCachedThreadPool(bossFactory),
|
|
|
- Executors.newCachedThreadPool(workerFactory));
|
|
|
+ Executors.newCachedThreadPool(workerFactory),
|
|
|
+ maxShuffleThreads);
|
|
|
super.serviceInit(new Configuration(conf));
|
|
|
}
|
|
|
|