You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by en...@apache.org on 2015/02/06 23:36:43 UTC

hbase git commit: HBASE-12956 Binding to 0.0.0.0 is broken after HBASE-10569

Repository: hbase
Updated Branches:
  refs/heads/master 57319c536 -> 3b56d2a0b


HBASE-12956 Binding to 0.0.0.0 is broken after HBASE-10569

Signed-off-by: Enis Soztutar <en...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/3b56d2a0
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/3b56d2a0
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/3b56d2a0

Branch: refs/heads/master
Commit: 3b56d2a0bc36f9dcb901bb709b8d9ae58df955ff
Parents: 57319c5
Author: Esteban Gutierrez <es...@cloudera.com>
Authored: Fri Feb 6 11:50:07 2015 -0800
Committer: Enis Soztutar <en...@apache.org>
Committed: Fri Feb 6 14:35:20 2015 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/ipc/RpcServer.java    | 18 ++++++++++--------
 .../hadoop/hbase/regionserver/RSRpcServices.java  | 13 +++++++------
 .../hadoop/hbase/TestHBaseTestingUtility.java     | 12 ++++++++++++
 3 files changed, 29 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3b56d2a0/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index 5eb9a52..b469fac 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -196,7 +196,7 @@ public class RpcServer implements RpcServerInterface {
   static final ThreadLocal<MonitoredRPCHandler> MONITORED_RPC
       = new ThreadLocal<MonitoredRPCHandler>();
 
-  protected final InetSocketAddress isa;
+  protected final InetSocketAddress bindAddress;
   protected int port;                             // port we listen on
   private int readThreads;                        // number of read threads
   protected int maxIdleTime;                      // the maximum idle time after
@@ -525,8 +525,8 @@ public class RpcServer implements RpcServerInterface {
       acceptChannel = ServerSocketChannel.open();
       acceptChannel.configureBlocking(false);
 
-      // Bind the server socket to the local host and port
-      bind(acceptChannel.socket(), isa, backlogLength);
+      // Bind the server socket to the binding addrees (can be different from the default interface)
+      bind(acceptChannel.socket(), bindAddress, backlogLength);
       port = acceptChannel.socket().getLocalPort(); //Could be an ephemeral port
       // create a selector;
       selector= Selector.open();
@@ -534,7 +534,8 @@ public class RpcServer implements RpcServerInterface {
       readers = new Reader[readThreads];
       readPool = Executors.newFixedThreadPool(readThreads,
         new ThreadFactoryBuilder().setNameFormat(
-          "RpcServer.reader=%d,port=" + port).setDaemon(true).build());
+          "RpcServer.reader=%d,bindAddress=" + bindAddress.getHostName() +
+          ",port=" + port).setDaemon(true).build());
       for (int i = 0; i < readThreads; ++i) {
         Reader reader = new Reader();
         readers[i] = reader;
@@ -1873,17 +1874,18 @@ public class RpcServer implements RpcServerInterface {
    * instance else pass null for no authentication check.
    * @param name Used keying this rpc servers' metrics and for naming the Listener thread.
    * @param services A list of services.
-   * @param isa Where to listen
+   * @param bindAddres Where to listen
    * @throws IOException
    */
   public RpcServer(final Server server, final String name,
       final List<BlockingServiceAndInterface> services,
-      final InetSocketAddress isa, Configuration conf,
+      final InetSocketAddress bindAddress, Configuration conf,
       RpcScheduler scheduler)
-  throws IOException {
+      throws IOException {
+
     this.server = server;
     this.services = services;
-    this.isa = isa;
+    this.bindAddress = bindAddress;
     this.conf = conf;
     this.socketSendBufferSize = 0;
     this.maxQueueSize =

http://git-wip-us.apache.org/repos/asf/hbase/blob/3b56d2a0/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 68530d3..3653cfb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -781,6 +781,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       HConstants.DEFAULT_REGIONSERVER_PORT);
     // Creation of a HSA will force a resolve.
     InetSocketAddress initialIsa = new InetSocketAddress(hostname, port);
+    InetSocketAddress bindAddress = new InetSocketAddress(
+      rs.conf.get("hbase.regionserver.ipc.address", hostname), port);
     if (initialIsa.getAddress() == null) {
       throw new IllegalArgumentException("Failed resolve of " + initialIsa);
     }
@@ -789,7 +791,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     // Set how many times to retry talking to another server over HConnection.
     ConnectionUtils.setServerSideHConnectionRetriesConfig(rs.conf, name, LOG);
     rpcServer = new RpcServer(rs, name, getServices(),
-      initialIsa, // BindAddress is IP we got for this server.
+      bindAddress, // use final bindAddress for this server.
       rs.conf,
       rpcSchedulerFactory.create(rs.conf, this, rs));
 
@@ -800,17 +802,16 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY,
       HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE);
 
-    // Set our address.
-    isa = rpcServer.getListenerAddress();
+    // Set our address, however we need the final port that was given to rpcServer
+    isa = new InetSocketAddress(initialIsa.getHostName(), rpcServer.getListenerAddress().getPort());
     rpcServer.setErrorHandler(this);
     rs.setName(name);
   }
 
   public static String getHostname(Configuration conf) throws UnknownHostException {
-    return conf.get("hbase.regionserver.ipc.address",
-        Strings.domainNamePointerToHostName(DNS.getDefaultHost(
+    return Strings.domainNamePointerToHostName(DNS.getDefaultHost(
             conf.get("hbase.regionserver.dns.interface", "default"),
-            conf.get("hbase.regionserver.dns.nameserver", "default"))));
+            conf.get("hbase.regionserver.dns.nameserver", "default")));
   }
 
   RegionScanner getScanner(long scannerId) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/3b56d2a0/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java
index abbcb4c..56720a3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java
@@ -125,6 +125,18 @@ public class TestHBaseTestingUtility {
     }
   }
 
+  @Test
+  public void testMiniClusterBindToWildcard() throws Exception {
+    HBaseTestingUtility hbt = new HBaseTestingUtility();
+    hbt.getConfiguration().set("hbase.regionserver.ipc.address", "0.0.0.0");
+    MiniHBaseCluster cluster = hbt.startMiniCluster();
+    try {
+      assertEquals(1, cluster.getLiveRegionServerThreads().size());
+    } finally {
+      hbt.shutdownMiniCluster();
+    }
+  }
+
   /**
    *  Test that we can start and stop multiple time a cluster
    *   with the same HBaseTestingUtility.