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 2014/11/05 18:29:03 UTC

svn commit: r1636920 - in /hive/branches/branch-0.14/service/src/java/org/apache/hive/service: cli/thrift/ThriftCLIService.java server/HiveServer2.java

Author: vgumashta
Date: Wed Nov  5 17:29:02 2014
New Revision: 1636920

URL: http://svn.apache.org/r1636920
Log:
HIVE-8733: HiveServer2 dynamic service discovery not picking correct IP address when hive.server2.thrift.bind.host is not set (Vaibhav Gumashta reviewed by Thejas Nair)

Modified:
    hive/branches/branch-0.14/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
    hive/branches/branch-0.14/service/src/java/org/apache/hive/service/server/HiveServer2.java

Modified: hive/branches/branch-0.14/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java?rev=1636920&r1=1636919&r2=1636920&view=diff
==============================================================================
--- hive/branches/branch-0.14/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java (original)
+++ hive/branches/branch-0.14/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java Wed Nov  5 17:29:02 2014
@@ -19,7 +19,8 @@
 package org.apache.hive.service.cli.thrift;
 
 import java.io.IOException;
-import java.net.InetSocketAddress;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
@@ -31,6 +32,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hive.service.AbstractService;
+import org.apache.hive.service.ServiceException;
 import org.apache.hive.service.auth.HiveAuthFactory;
 import org.apache.hive.service.auth.TSetIpAddressProcessor;
 import org.apache.hive.service.cli.*;
@@ -53,7 +55,7 @@ public abstract class ThriftCLIService e
   protected static HiveAuthFactory hiveAuthFactory;
 
   protected int portNum;
-  protected InetSocketAddress serverAddress;
+  protected InetAddress serverAddress;
   protected String hiveHost;
   protected TServer server;
   protected org.eclipse.jetty.server.Server httpServer;
@@ -75,13 +77,21 @@ public abstract class ThriftCLIService e
   @Override
   public synchronized void init(HiveConf hiveConf) {
     this.hiveConf = hiveConf;
-
     // Initialize common server configs needed in both binary & http modes
     String portString;
     hiveHost = System.getenv("HIVE_SERVER2_THRIFT_BIND_HOST");
     if (hiveHost == null) {
       hiveHost = hiveConf.getVar(ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST);
     }
+    try {
+      if (hiveHost != null && !hiveHost.isEmpty()) {
+        serverAddress = InetAddress.getByName(hiveHost);
+      } else {
+        serverAddress = InetAddress.getLocalHost();
+      }
+    } catch (UnknownHostException e) {
+      throw new ServiceException(e);
+    }
     // HTTP mode
     if (HiveServer2.isHTTPTransportMode(hiveConf)) {
       workerKeepAliveTime =
@@ -105,11 +115,6 @@ public abstract class ThriftCLIService e
         portNum = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_PORT);
       }
     }
-    if (hiveHost != null && !hiveHost.isEmpty()) {
-      serverAddress = new InetSocketAddress(hiveHost, portNum);
-    } else {
-      serverAddress = new InetSocketAddress(portNum);
-    }
     minWorkerThreads = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_MIN_WORKER_THREADS);
     maxWorkerThreads = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_MAX_WORKER_THREADS);
     super.init(hiveConf);
@@ -148,7 +153,7 @@ public abstract class ThriftCLIService e
     return portNum;
   }
 
-  public InetSocketAddress getServerAddress() {
+  public InetAddress getServerAddress() {
     return serverAddress;
   }
 

Modified: hive/branches/branch-0.14/service/src/java/org/apache/hive/service/server/HiveServer2.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/service/src/java/org/apache/hive/service/server/HiveServer2.java?rev=1636920&r1=1636919&r2=1636920&view=diff
==============================================================================
--- hive/branches/branch-0.14/service/src/java/org/apache/hive/service/server/HiveServer2.java (original)
+++ hive/branches/branch-0.14/service/src/java/org/apache/hive/service/server/HiveServer2.java Wed Nov  5 17:29:02 2014
@@ -252,7 +252,7 @@ public class HiveServer2 extends Composi
     if ((thriftCLIService == null) || (thriftCLIService.getServerAddress() == null)) {
       throw new Exception("Unable to get the server address; it hasn't been initialized yet.");
     }
-    return thriftCLIService.getServerAddress().getHostName() + ":"
+    return thriftCLIService.getServerAddress().getHostAddress() + ":"
         + thriftCLIService.getPortNumber();
   }