You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by vg...@apache.org on 2015/01/07 00:44:59 UTC

svn commit: r1649974 - in /hive/branches/branch-0.14: common/src/java/org/apache/hadoop/hive/conf/ service/src/java/org/apache/hive/service/auth/ service/src/java/org/apache/hive/service/cli/thrift/

Author: vgumashta
Date: Tue Jan  6 23:44:59 2015
New Revision: 1649974

URL: http://svn.apache.org/r1649974
Log:
HIVE-6679: HiveServer2 should support configurable the server side socket timeout and keepalive for various transports types where applicable (Navis Ryu, Vaibhav Gumashta reviewed by Thejas Nair)

Modified:
    hive/branches/branch-0.14/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
    hive/branches/branch-0.14/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java
    hive/branches/branch-0.14/service/src/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java

Modified: hive/branches/branch-0.14/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java?rev=1649974&r1=1649973&r2=1649974&view=diff
==============================================================================
--- hive/branches/branch-0.14/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (original)
+++ hive/branches/branch-0.14/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java Tue Jan  6 23:44:59 2015
@@ -1617,7 +1617,7 @@ public class HiveConf extends Configurat
     HIVE_SERVER2_THRIFT_BIND_HOST("hive.server2.thrift.bind.host", "",
         "Bind host on which to run the HiveServer2 Thrift service."),
 
-    // http (over thrift) transport settings
+    // Http (over thrift) transport settings
     HIVE_SERVER2_THRIFT_HTTP_PORT("hive.server2.thrift.http.port", 10001,
         "Port number of HiveServer2 Thrift interface when hive.server2.transport.mode is 'http'."),
     HIVE_SERVER2_THRIFT_HTTP_PATH("hive.server2.thrift.http.path", "cliservice",
@@ -1634,7 +1634,12 @@ public class HiveConf extends Configurat
         "Keepalive time for an idle http worker thread. When the number of workers exceeds min workers, " +
         "excessive threads are killed after this time interval."),
 
-    // binary transport settings
+    // Binary (TCP) transport settings
+    HIVE_SERVER2_TCP_SOCKET_BLOCKING_TIMEOUT("hive.server2.tcp.socket.blocking.timeout", "0s",
+        new TimeValidator(TimeUnit.SECONDS),"Timeout (in seconds) on blocking socket operations (accept, read). " +
+        "0 means infinite timeout."),
+    HIVE_SERVER2_TCP_SOCKET_KEEPALIVE("hive.server2.tcp.socket.keepalive", true,
+        "Whether to send tcp socket keepalive probe"),
     HIVE_SERVER2_THRIFT_PORT("hive.server2.thrift.port", 10000,
         "Port number of HiveServer2 Thrift interface when hive.server2.transport.mode is 'binary'."),
     HIVE_SERVER2_THRIFT_SASL_QOP("hive.server2.thrift.sasl.qop", "auth",

Modified: hive/branches/branch-0.14/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java?rev=1649974&r1=1649973&r2=1649974&view=diff
==============================================================================
--- hive/branches/branch-0.14/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java (original)
+++ hive/branches/branch-0.14/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java Tue Jan  6 23:44:59 2015
@@ -19,6 +19,8 @@ package org.apache.hive.service.auth;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.net.SocketException;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -194,8 +196,8 @@ public class HiveAuthFactory {
     return TSSLTransportFactory.getClientSocket(host, port, loginTimeout, params);
   }
 
-  public static TServerSocket getServerSocket(String hiveHost, int portNum)
-    throws TTransportException {
+  public static TServerSocket getServerSocket(String hiveHost, int portNum, int socketTimeout,
+      boolean keepAlive) throws TTransportException {
     InetSocketAddress serverAddress;
     if (hiveHost == null || hiveHost.isEmpty()) {
       // Wildcard bind
@@ -203,14 +205,17 @@ public class HiveAuthFactory {
     } else {
       serverAddress = new InetSocketAddress(hiveHost, portNum);
     }
-    return new TServerSocket(serverAddress);
+    TServerSocket serverSocket = new TServerSocket(serverAddress, socketTimeout);
+    if (keepAlive) {
+      serverSocket = new TServerSocketKeepAlive(serverSocket.getServerSocket());
+    }
+    return serverSocket;
   }
 
   public static TServerSocket getServerSSLSocket(String hiveHost, int portNum, String keyStorePath,
-      String keyStorePassWord, List<String> sslVersionBlacklist) throws TTransportException,
-      UnknownHostException {
-    TSSLTransportFactory.TSSLTransportParameters params =
-        new TSSLTransportFactory.TSSLTransportParameters();
+      String keyStorePassWord, List<String> sslVersionBlacklist, int socketTimeout,
+      boolean keepAlive) throws TTransportException, UnknownHostException {
+    TSSLTransportFactory.TSSLTransportParameters params = new TSSLTransportFactory.TSSLTransportParameters();
     params.setKeyStore(keyStorePath, keyStorePassWord);
     InetSocketAddress serverAddress;
     if (hiveHost == null || hiveHost.isEmpty()) {
@@ -219,8 +224,8 @@ public class HiveAuthFactory {
     } else {
       serverAddress = new InetSocketAddress(hiveHost, portNum);
     }
-    TServerSocket thriftServerSocket =
-        TSSLTransportFactory.getServerSocket(portNum, 0, serverAddress.getAddress(), params);
+    TServerSocket thriftServerSocket = TSSLTransportFactory.getServerSocket(portNum, socketTimeout,
+        serverAddress.getAddress(), params);
     if (thriftServerSocket.getServerSocket() instanceof SSLServerSocket) {
       List<String> sslVersionBlacklistLocal = new ArrayList<String>();
       for (String sslVersion : sslVersionBlacklist) {
@@ -239,6 +244,9 @@ public class HiveAuthFactory {
       LOG.info("SSL Server Socket Enabled Protocols: "
           + Arrays.toString(sslServerSocket.getEnabledProtocols()));
     }
+    if (keepAlive) {
+      thriftServerSocket = new TServerSocketKeepAlive(thriftServerSocket.getServerSocket());
+    }
     return thriftServerSocket;
   }
 
@@ -325,4 +333,25 @@ public class HiveAuthFactory {
     }
   }
 
+  /**
+   * TServerSocketKeepAlive - like TServerSocket, but will enable keepalive for
+   * accepted sockets.
+   * 
+   */
+  static class TServerSocketKeepAlive extends TServerSocket {
+    public TServerSocketKeepAlive(ServerSocket serverSocket) throws TTransportException {
+      super(serverSocket);
+    }
+
+    @Override
+    protected TSocket acceptImpl() throws TTransportException {
+      TSocket ts = super.acceptImpl();
+      try {
+        ts.getSocket().setKeepAlive(true);
+      } catch (SocketException e) {
+        throw new TTransportException(e);
+      }
+      return ts;
+    }
+  }
 }

Modified: hive/branches/branch-0.14/service/src/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/service/src/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java?rev=1649974&r1=1649973&r2=1649974&view=diff
==============================================================================
--- hive/branches/branch-0.14/service/src/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java (original)
+++ hive/branches/branch-0.14/service/src/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java Tue Jan  6 23:44:59 2015
@@ -58,12 +58,16 @@ public class ThriftBinaryCLIService exte
       TTransportFactory transportFactory = hiveAuthFactory.getAuthTransFactory();
       TProcessorFactory processorFactory = hiveAuthFactory.getAuthProcFactory(this);
       TServerSocket serverSocket = null;
+      int socketTimeout =
+          (int) hiveConf.getTimeVar(ConfVars.HIVE_SERVER2_TCP_SOCKET_BLOCKING_TIMEOUT,
+              TimeUnit.SECONDS);
+      boolean keepAlive = hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_TCP_SOCKET_KEEPALIVE);
       List<String> sslVersionBlacklist = new ArrayList<String>();
       for (String sslVersion : hiveConf.getVar(ConfVars.HIVE_SSL_PROTOCOL_BLACKLIST).split(",")) {
         sslVersionBlacklist.add(sslVersion);
       }
       if (!hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_USE_SSL)) {
-        serverSocket = HiveAuthFactory.getServerSocket(hiveHost, portNum);
+        serverSocket = HiveAuthFactory.getServerSocket(hiveHost, portNum, socketTimeout, keepAlive);
       } else {
         String keyStorePath = hiveConf.getVar(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PATH).trim();
         if (keyStorePath.isEmpty()) {
@@ -73,7 +77,7 @@ public class ThriftBinaryCLIService exte
         String keyStorePassword = ShimLoader.getHadoopShims().getPassword(hiveConf,
             HiveConf.ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname);
         serverSocket = HiveAuthFactory.getServerSSLSocket(hiveHost, portNum, keyStorePath,
-            keyStorePassword, sslVersionBlacklist);
+            keyStorePassword, sslVersionBlacklist, socketTimeout, keepAlive);
       }
 
       // Server args