diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index d13e03d..52aca9e 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -3170,6 +3170,9 @@ private static void populateLlapDaemonVarsSet(Set llapDaemonVarsSetLocal new TimeValidator(TimeUnit.MILLISECONDS), "Connection timeout (in milliseconds) before a failure to an LLAP daemon from Tez AM.", "llap.task.communicator.connection.timeout-millis"), + LLAP_TASK_COMMUNICATOR_LISTENER_THREAD_COUNT( + "hive.llap.task.communicator.listener.thread-count", 30, + "The number of task communicator listener threads."), LLAP_TASK_COMMUNICATOR_CONNECTION_SLEEP_BETWEEN_RETRIES_MS( "hive.llap.task.communicator.connection.sleep.between.retries.ms", "2000ms", new TimeValidator(TimeUnit.MILLISECONDS), diff --git a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java index e593b33..c700913 100644 --- a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java +++ b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java @@ -219,8 +219,8 @@ protected void startRpcServer() { new JobTokenSecretManager(); jobTokenSecretManager.addTokenForJob(tokenIdentifier, sessionToken); - int numHandlers = conf.getInt(TezConfiguration.TEZ_AM_TASK_LISTENER_THREAD_COUNT, - TezConfiguration.TEZ_AM_TASK_LISTENER_THREAD_COUNT_DEFAULT); + int numHandlers = + HiveConf.getIntVar(conf, ConfVars.LLAP_TASK_COMMUNICATOR_LISTENER_THREAD_COUNT); server = new RPC.Builder(conf) .setProtocol(LlapTaskUmbilicalProtocol.class) .setBindAddress("0.0.0.0")