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 6fb31272ec0b48a7772a4a9165fa546bbf9a1264..99d067393b0358ec5940bc5323cd0fde3328f2d8 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -1661,6 +1661,13 @@ "Minimum number of Thrift worker threads"), HIVE_SERVER2_THRIFT_MAX_WORKER_THREADS("hive.server2.thrift.max.worker.threads", 500, "Maximum number of Thrift worker threads"), + HIVE_SERVER2_THRIFT_LOGIN_BEBACKOFF_SLOT_LENGTH( + "hive.server2.thrift.exponential.backoff.slot.length", "100ms", + new TimeValidator(TimeUnit.MILLISECONDS), + "Binary exponential backoff slot time for thrift clients during login to HiveServer2,\n" + + "for retries until hitting thrift client timeout"), + HIVE_SERVER2_THRIFT_LOGIN_TIMEOUT("hive.server2.thrift.login.timeout", "20s", + new TimeValidator(TimeUnit.SECONDS), "Timeout for thrift clients during login to HiveServer2"), HIVE_SERVER2_THRIFT_WORKER_KEEPALIVE_TIME("hive.server2.thrift.worker.keepalive.time", "60s", new TimeValidator(TimeUnit.SECONDS), "Keepalive time (in seconds) for an idle worker thread. When the number of workers exceeds min workers, " + diff --git a/service/src/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java b/service/src/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java index 3c58453434c9a68cdac52ab20c72f63cdf20a785..b6e851a83c31cd5b7bdd09f0fe979ed13dcc5b20 100644 --- a/service/src/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java +++ b/service/src/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java @@ -78,10 +78,16 @@ public void run() { // Server args int maxMessageSize = hiveConf.getIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_MAX_MESSAGE_SIZE); + int requestTimeout = (int) hiveConf.getTimeVar( + HiveConf.ConfVars.HIVE_SERVER2_THRIFT_LOGIN_TIMEOUT, TimeUnit.SECONDS); + int beBackoffSlotLength = (int) hiveConf.getTimeVar( + HiveConf.ConfVars.HIVE_SERVER2_THRIFT_LOGIN_BEBACKOFF_SLOT_LENGTH, TimeUnit.MILLISECONDS); TThreadPoolServer.Args sargs = new TThreadPoolServer.Args(serverSocket) .processorFactory(processorFactory).transportFactory(transportFactory) .protocolFactory(new TBinaryProtocol.Factory()) .inputProtocolFactory(new TBinaryProtocol.Factory(true, true, maxMessageSize, maxMessageSize)) + .requestTimeout(requestTimeout).requestTimeoutUnit(TimeUnit.SECONDS) + .beBackoffSlotLength(beBackoffSlotLength).beBackoffSlotLengthUnit(TimeUnit.MILLISECONDS) .executorService(executorService); // TCP Server