zeroflag commented on a change in pull request #75:
URL: https://github.com/apache/tez/pull/75#discussion_r469814436
##########
File path: tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
##########
@@ -621,8 +622,13 @@ public synchronized void serviceInit(final Configuration
conf) throws Exception
}
}
- rawExecutor = Executors.newCachedThreadPool(new
ThreadFactoryBuilder().setDaemon(true)
- .setNameFormat("App Shared Pool - " + "#%d").build());
+ int thread_count =
conf.getInt(TezConfiguration.TEZ_AM_DAG_APPCONTEXT_THREAD_COUNT_LIMIT,
+ TezConfiguration.TEZ_AM_DAG_APPCONTEXT_THREAD_COUNT_LIMIT_DEFAULT);
+ // NOTE: LinkedBlockingQueue does not have a capacity Limit and can thus
+ // occupy large memory chunks when numerous Runables are pending for
execution
+ rawExecutor = new ThreadPoolExecutor(1, thread_count,
Review comment:
I think this will use only 1 thread all time time. If corePoolSize is
set to one, then the executor will prefer queueing the requests instead of
adding threads. It'll only add a new thread if the request cannot be queued,
but that will never happen because of the unbounded LinkedBlockingQueue.
Here is the relevant part from the ThreadPoolExecutor's javadoc:
* If fewer than corePoolSize threads are running, the Executor always
prefers adding a new thread rather than queuing.
* If corePoolSize or more threads are running, the Executor always prefers
queuing a request rather than adding a new thread.
* If a request cannot be queued, a new thread is created unless this would
exceed maximumPoolSize, in which case, the task will be rejected.
I think both corePoolSize and maxPoolSize should be set to thread_count
(like Executors.newFixedThreadPool).
Just a nitpick, thread_count should be called threadCount to match the
naming convention.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]