@@ -75,19 +75,34 @@ public synchronized void init(HiveConf hiveConf) {
7575 initOperationLogRootDir ();
7676 }
7777 createBackgroundOperationPool ();
78+ addService (operationManager );
79+ }
80+
81+ private void createBackgroundOperationPool () {
82+ int poolSize = hiveConf .getIntVar (ConfVars .HIVE_SERVER2_ASYNC_EXEC_THREADS );
83+ LOG .info ("HiveServer2: Background operation thread pool size: " + poolSize );
84+ int poolQueueSize = hiveConf .getIntVar (ConfVars .HIVE_SERVER2_ASYNC_EXEC_WAIT_QUEUE_SIZE );
85+ LOG .info ("HiveServer2: Background operation thread wait queue size: " + poolQueueSize );
86+ long keepAliveTime = HiveConf .getTimeVar (
87+ hiveConf , ConfVars .HIVE_SERVER2_ASYNC_EXEC_KEEPALIVE_TIME , TimeUnit .SECONDS );
88+ LOG .info (
89+ "HiveServer2: Background operation thread keepalive time: " + keepAliveTime + " seconds" );
90+
91+ // Create a thread pool with #poolSize threads
92+ // Threads terminate when they are idle for more than the keepAliveTime
93+ // A bounded blocking queue is used to queue incoming operations, if #operations > poolSize
94+ String threadPoolName = "HiveServer2-Background-Pool" ;
95+ backgroundOperationPool = new ThreadPoolExecutor (poolSize , poolSize ,
96+ keepAliveTime , TimeUnit .SECONDS , new LinkedBlockingQueue <Runnable >(poolQueueSize ),
97+ new ThreadFactoryWithGarbageCleanup (threadPoolName ));
98+ backgroundOperationPool .allowCoreThreadTimeOut (true );
99+
78100 checkInterval = HiveConf .getTimeVar (
79101 hiveConf , ConfVars .HIVE_SERVER2_SESSION_CHECK_INTERVAL , TimeUnit .MILLISECONDS );
80102 sessionTimeout = HiveConf .getTimeVar (
81103 hiveConf , ConfVars .HIVE_SERVER2_IDLE_SESSION_TIMEOUT , TimeUnit .MILLISECONDS );
82104 checkOperation = HiveConf .getBoolVar (hiveConf ,
83105 ConfVars .HIVE_SERVER2_IDLE_SESSION_CHECK_OPERATION );
84- addService (operationManager );
85- }
86-
87- private void createBackgroundOperationPool () {
88- int backgroundPoolSize = hiveConf .getIntVar (ConfVars .HIVE_SERVER2_ASYNC_EXEC_THREADS );
89- backgroundOperationPool = (ThreadPoolExecutor ) Executors .newFixedThreadPool (backgroundPoolSize );
90- LOG .info ("HiveServer2: Async execution pool size: " + backgroundPoolSize );
91106 }
92107
93108 private void initOperationLogRootDir () {
0 commit comments