You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by fp...@apache.org on 2013/06/15 19:05:46 UTC

svn commit: r1493400 - in /zookeeper/bookkeeper/trunk: ./ bookkeeper-server/conf/ bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ bookkeeper-server/src/main/java/org/apache/boo...

Author: fpj
Date: Sat Jun 15 17:05:45 2013
New Revision: 1493400

URL: http://svn.apache.org/r1493400
Log:
BOOKKEEPER-618: Better resolution of bookie address (ivank via fpj)


Added:
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/net/
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/net/DNS.java
Modified:
    zookeeper/bookkeeper/trunk/CHANGES.txt
    zookeeper/bookkeeper/trunk/bookkeeper-server/conf/bk_server.conf
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieNettyServer.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieServer.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LocalBookKeeper.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerChecker.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestPerChannelBookieClient.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorBookieTest.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/BookieAutoRecoveryTest.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java

Modified: zookeeper/bookkeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/CHANGES.txt?rev=1493400&r1=1493399&r2=1493400&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/CHANGES.txt (original)
+++ zookeeper/bookkeeper/trunk/CHANGES.txt Sat Jun 15 17:05:45 2013
@@ -92,6 +92,8 @@ Trunk (unreleased changes)
 
       BOOKKEEPER-627: LedgerDirsMonitor is missing thread name (rakesh via fpj)
 
+      BOOKKEEPER-618: Better resolution of bookie address (ivank via fpj)
+
     NEW FEATURE:
 
       BOOKKEEPER-562: Ability to tell if a ledger is closed or not (fpj)

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/conf/bk_server.conf
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/conf/bk_server.conf?rev=1493400&r1=1493399&r2=1493400&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/conf/bk_server.conf (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/conf/bk_server.conf Sat Jun 15 17:05:45 2013
@@ -25,6 +25,23 @@
 # Port that bookie server listen on
 bookiePort=3181
 
+# Set the network interface that the bookie should listen on.
+# If not set, the bookie will listen on all interfaces.
+#listeningInterface=eth0
+
+# Whether the bookie allowed to use a loopback interface as its primary
+# interface(i.e. the interface it uses to establish its identity)?
+# By default, loopback interfaces are not allowed as the primary
+# interface.
+# Using a loopback interface as the primary interface usually indicates
+# a configuration error. For example, its fairly common in some VPS setups
+# to not configure a hostname, or to have the hostname resolve to
+# 127.0.0.1. If this is the case, then all bookies in the cluster will
+# establish their identities as 127.0.0.1:3181, and only one will be able
+# to join the cluster. For VPSs configured like this, you should explicitly
+# set the listening interface.
+#allowLoopback=false
+
 # Directory Bookkeeper outputs its write ahead log
 journalDirectory=/tmp/bk-txn
 

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java?rev=1493400&r1=1493399&r2=1493400&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java Sat Jun 15 17:05:45 2013
@@ -57,6 +57,7 @@ import org.apache.bookkeeper.util.IOUtil
 import org.apache.bookkeeper.util.MathUtils;
 import org.apache.bookkeeper.util.ZkUtils;
 import org.apache.bookkeeper.util.StringUtils;
+import org.apache.bookkeeper.util.net.DNS;
 import org.apache.bookkeeper.zookeeper.ZooKeeperWatcherBase;
 import org.apache.commons.io.FileUtils;
 import org.slf4j.Logger;
@@ -347,8 +348,20 @@ public class Bookie extends Thread {
      */
     public static InetSocketAddress getBookieAddress(ServerConfiguration conf)
             throws UnknownHostException {
-        return new InetSocketAddress(InetAddress.getLocalHost()
-                .getHostAddress(), conf.getBookiePort());
+        String iface = conf.getListeningInterface();
+        if (iface == null) {
+            iface = "default";
+        }
+        InetSocketAddress addr = new InetSocketAddress(
+                DNS.getDefaultHost(iface),
+                conf.getBookiePort());
+        if (addr.getAddress().isLoopbackAddress()
+            && !conf.getAllowLoopback()) {
+            throw new UnknownHostException("Trying to listen on loopback address, "
+                    + addr + " but this is forbidden by default "
+                    + "(see ServerConfiguration#getAllowLoopback())");
+        }
+        return addr;
     }
 
     private String getInstanceId(ZooKeeper zk) throws KeeperException,
@@ -410,8 +423,7 @@ public class Bookie extends Thread {
     }
 
     private String getMyId() throws UnknownHostException {
-        return InetAddress.getLocalHost().getHostAddress() + ":"
-                + conf.getBookiePort();
+        return StringUtils.addrToString(Bookie.getBookieAddress(conf));
     }
 
     void readJournal() throws IOException, BookieException {

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java?rev=1493400&r1=1493399&r2=1493400&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java Sat Jun 15 17:05:45 2013
@@ -267,7 +267,7 @@ public class LedgerMetadata {
             LedgerMetadataFormat.Segment.Builder segmentBuilder = LedgerMetadataFormat.Segment.newBuilder();
             segmentBuilder.setFirstEntryId(entry.getKey());
             for (InetSocketAddress addr : entry.getValue()) {
-                segmentBuilder.addEnsembleMember(addr.getAddress().getHostAddress() + ":" + addr.getPort());
+                segmentBuilder.addEnsembleMember(StringUtils.addrToString(addr));
             }
             builder.addSegment(segmentBuilder.build());
         }

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java?rev=1493400&r1=1493399&r2=1493400&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java Sat Jun 15 17:05:45 2013
@@ -48,6 +48,9 @@ public class ServerConfiguration extends
     protected final static String MAX_BACKUP_JOURNALS = "journalMaxBackups";
     // Bookie Parameters
     protected final static String BOOKIE_PORT = "bookiePort";
+    protected final static String LISTENING_INTERFACE = "listeningInterface";
+    protected final static String ALLOW_LOOPBACK = "allowLoopback";
+
     protected final static String JOURNAL_DIR = "journalDirectory";
     protected final static String LEDGER_DIRS = "ledgerDirectories";
     // NIO Parameters
@@ -284,6 +287,64 @@ public class ServerConfiguration extends
     }
 
     /**
+     * Get the network interface that the bookie should
+     * listen for connections on. If this is null, then the bookie
+     * will listen for connections on all interfaces.
+     *
+     * @return the network interface to listen on, e.g. eth0, or
+     *         null if none is specified
+     */
+    public String getListeningInterface() {
+        return this.getString(LISTENING_INTERFACE);
+    }
+
+    /**
+     * Set the network interface that the bookie should listen on.
+     * If not set, the bookie will listen on all interfaces.
+     *
+     * @param iface the interface to listen on
+     */
+    public ServerConfiguration setListeningInterface(String iface) {
+        this.setProperty(LISTENING_INTERFACE, iface);
+        return this;
+    }
+
+    /**
+     * Is the bookie allowed to use a loopback interface as its primary
+     * interface(i.e. the interface it uses to establish its identity)?
+     *
+     * By default, loopback interfaces are not allowed as the primary
+     * interface.
+     *
+     * Using a loopback interface as the primary interface usually indicates
+     * a configuration error. For example, its fairly common in some VPS setups
+     * to not configure a hostname, or to have the hostname resolve to
+     * 127.0.0.1. If this is the case, then all bookies in the cluster will
+     * establish their identities as 127.0.0.1:3181, and only one will be able
+     * to join the cluster. For VPSs configured like this, you should explicitly
+     * set the listening interface.
+     *
+     * @see #setListeningInterface(String)
+     * @return whether a loopback interface can be used as the primary interface
+     */
+    public boolean getAllowLoopback() {
+        return this.getBoolean(ALLOW_LOOPBACK, false);
+    }
+
+    /**
+     * Configure the bookie to allow loopback interfaces to be used
+     * as the primary bookie interface.
+     *
+     * @see #getAllowLoopback
+     * @param allow whether to allow loopback interfaces
+     * @return server configuration
+     */
+    public ServerConfiguration setAllowLoopback(boolean allow) {
+        this.setProperty(ALLOW_LOOPBACK, allow);
+        return this;
+    }
+
+    /**
      * Get dir name to store journal files
      *
      * @return journal dir name

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java?rev=1493400&r1=1493399&r2=1493400&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java Sat Jun 15 17:05:45 2013
@@ -23,6 +23,8 @@ import org.apache.bookkeeper.replication
 import org.apache.bookkeeper.replication.ReplicationException.UnavailableException;
 import org.apache.bookkeeper.util.BookKeeperConstants;
 import org.apache.bookkeeper.util.ZkUtils;
+import org.apache.bookkeeper.util.net.DNS;
+
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback;
 import org.apache.bookkeeper.proto.DataFormats.LedgerRereplicationLayoutFormat;
 import org.apache.bookkeeper.proto.DataFormats.UnderreplicatedLedgerFormat;
@@ -114,7 +116,7 @@ public class ZkLedgerUnderreplicationMan
 
         LockDataFormat.Builder lockDataBuilder = LockDataFormat.newBuilder();
         try {
-            lockDataBuilder.setBookieId(InetAddress.getLocalHost().getHostAddress().toString());
+            lockDataBuilder.setBookieId(DNS.getDefaultHost("default"));
         } catch (UnknownHostException uhe) {
             // if we cant get the address, ignore. it's optional
             // in the data structure in any case

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieNettyServer.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieNettyServer.java?rev=1493400&r1=1493399&r2=1493400&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieNettyServer.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieNettyServer.java Sat Jun 15 17:05:45 2013
@@ -65,7 +65,7 @@ class BookieNettyServer {
     Object suspensionLock = new Object();
     boolean suspended = false;
 
-    InetSocketAddress localAddress = null;
+    final InetSocketAddress bindAddress;
 
     BookieNettyServer(ServerConfiguration conf, Bookie bookie)
             throws IOException, KeeperException, InterruptedException, BookieException  {
@@ -77,7 +77,12 @@ class BookieNettyServer {
         serverChannelFactory = new NioServerSocketChannelFactory(
                 Executors.newCachedThreadPool(tfb.setNameFormat(base + "-boss-%d").build()),
                 Executors.newCachedThreadPool(tfb.setNameFormat(base + "-worker-%d").build()));
-
+        if (conf.getListeningInterface() == null) {
+            // listen on all interfaces
+            bindAddress = new InetSocketAddress(conf.getBookiePort());
+        } else {
+            bindAddress = Bookie.getBookieAddress(conf);
+        }
     }
 
     boolean isRunning() {
@@ -101,24 +106,14 @@ class BookieNettyServer {
         }
     }
 
-    InetSocketAddress getLocalAddress() {
-        if (localAddress != null) {
-            return localAddress;
-        } else {
-            return new InetSocketAddress(conf.getBookiePort());
-        }
-    }
-
     void start() {
         ServerBootstrap bootstrap = new ServerBootstrap(serverChannelFactory);
         bootstrap.setPipelineFactory(new BookiePipelineFactory());
         bootstrap.setOption("child.tcpNoDelay", conf.getServerTcpNoDelay());
         bootstrap.setOption("child.soLinger", 2);
 
-        Channel listen = bootstrap.bind(new InetSocketAddress(conf.getBookiePort()));
+        Channel listen = bootstrap.bind(bindAddress);
 
-        assert(listen.getLocalAddress() instanceof InetSocketAddress);
-        localAddress = (InetSocketAddress)listen.getLocalAddress();
         allChannels.add(listen);
         isRunning.set(true);
     }

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieServer.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieServer.java?rev=1493400&r1=1493399&r2=1493400&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieServer.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieServer.java Sat Jun 15 17:05:45 2013
@@ -100,12 +100,9 @@ public class BookieServer {
         registerJMX();
     }
 
-    public InetSocketAddress getLocalAddress() {
-        try {
-            return Bookie.getBookieAddress(conf);
-        } catch (UnknownHostException uhe) {
-            return nettyServer.getLocalAddress();
-        }
+    @VisibleForTesting
+    public InetSocketAddress getLocalAddress() throws UnknownHostException {
+        return Bookie.getBookieAddress(conf);
     }
 
     @VisibleForTesting

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LocalBookKeeper.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LocalBookKeeper.java?rev=1493400&r1=1493399&r2=1493400&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LocalBookKeeper.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LocalBookKeeper.java Sat Jun 15 17:05:45 2013
@@ -149,6 +149,7 @@ public class LocalBookKeeper {
                                   + ZooKeeperDefaultPort);
             bsConfs[i].setJournalDirName(tmpDirs[i].getPath());
             bsConfs[i].setLedgerDirNames(new String[] { tmpDirs[i].getPath() });
+            bsConfs[i].setAllowLoopback(true);
 
             bs[i] = new BookieServer(bsConfs[i]);
             bs[i].start();

Added: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/net/DNS.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/net/DNS.java?rev=1493400&view=auto
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/net/DNS.java (added)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/net/DNS.java Sat Jun 15 17:05:45 2013
@@ -0,0 +1,356 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+// This code has been copied from hadoop-common 2.0.4-alpha
+package org.apache.bookkeeper.util.net;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.InetAddress;
+import java.net.NetworkInterface;
+import java.net.SocketException;
+import java.net.UnknownHostException;
+import java.util.Collections;
+import java.util.Enumeration;
+import java.util.LinkedHashSet;
+import java.util.Vector;
+
+import javax.naming.NamingException;
+import javax.naming.directory.Attributes;
+import javax.naming.directory.DirContext;
+import javax.naming.directory.InitialDirContext;
+
+/**
+ *
+ * A class that provides direct and reverse lookup functionalities, allowing
+ * the querying of specific network interfaces or nameservers.
+ *
+ *
+ */
+public class DNS {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DNS.class);
+
+  /**
+   * The cached hostname -initially null.
+   */
+
+  private static final String cachedHostname = resolveLocalHostname();
+  private static final String cachedHostAddress = resolveLocalHostIPAddress();
+  private static final String LOCALHOST = "localhost";
+
+  /**
+   * Returns the hostname associated with the specified IP address by the
+   * provided nameserver.
+   *
+   * Loopback addresses
+   * @param hostIp The address to reverse lookup
+   * @param ns The host name of a reachable DNS server
+   * @return The host name associated with the provided IP
+   * @throws NamingException If a NamingException is encountered
+   */
+  public static String reverseDns(InetAddress hostIp, String ns)
+    throws NamingException {
+    //
+    // Builds the reverse IP lookup form
+    // This is formed by reversing the IP numbers and appending in-addr.arpa
+    //
+    String[] parts = hostIp.getHostAddress().split("\\.");
+    String reverseIP = parts[3] + "." + parts[2] + "." + parts[1] + "."
+      + parts[0] + ".in-addr.arpa";
+
+    DirContext ictx = new InitialDirContext();
+    Attributes attribute;
+    try {
+      attribute = ictx.getAttributes("dns://"               // Use "dns:///" if the default
+                         + ((ns == null) ? "" : ns) +
+                         // nameserver is to be used
+                         "/" + reverseIP, new String[] { "PTR" });
+    } finally {
+      ictx.close();
+    }
+
+    return attribute.get("PTR").get().toString();
+  }
+
+  /**
+   * @return NetworkInterface for the given subinterface name (eg eth0:0)
+   *    or null if no interface with the given name can be found
+   */
+  private static NetworkInterface getSubinterface(String strInterface)
+      throws SocketException {
+    Enumeration<NetworkInterface> nifs =
+      NetworkInterface.getNetworkInterfaces();
+
+    while (nifs.hasMoreElements()) {
+      Enumeration<NetworkInterface> subNifs =
+        nifs.nextElement().getSubInterfaces();
+
+      while (subNifs.hasMoreElements()) {
+        NetworkInterface nif = subNifs.nextElement();
+        if (nif.getName().equals(strInterface)) {
+          return nif;
+        }
+      }
+    }
+    return null;
+  }
+
+  /**
+   * @param nif network interface to get addresses for
+   * @return set containing addresses for each subinterface of nif,
+   *    see below for the rationale for using an ordered set
+   */
+  private static LinkedHashSet<InetAddress> getSubinterfaceInetAddrs(
+      NetworkInterface nif) {
+    LinkedHashSet<InetAddress> addrs = new LinkedHashSet<InetAddress>();
+    Enumeration<NetworkInterface> subNifs = nif.getSubInterfaces();
+    while (subNifs.hasMoreElements()) {
+      NetworkInterface subNif = subNifs.nextElement();
+      addrs.addAll(Collections.list(subNif.getInetAddresses()));
+    }
+    return addrs;
+  }
+
+  /**
+   * Like {@link DNS#getIPs(String, boolean), but returns all
+   * IPs associated with the given interface and its subinterfaces.
+   */
+  public static String[] getIPs(String strInterface)
+      throws UnknownHostException {
+    return getIPs(strInterface, true);
+  }
+
+  /**
+   * Returns all the IPs associated with the provided interface, if any, in
+   * textual form.
+   *
+   * @param strInterface
+   *            The name of the network interface or sub-interface to query
+   *            (eg eth0 or eth0:0) or the string "default"
+   * @param returnSubinterfaces
+   *            Whether to return IPs associated with subinterfaces of
+   *            the given interface
+   * @return A string vector of all the IPs associated with the provided
+   *         interface. The local host IP is returned if the interface
+   *         name "default" is specified or there is an I/O error looking
+   *         for the given interface.
+   * @throws UnknownHostException
+   *             If the given interface is invalid
+   *
+   */
+  public static String[] getIPs(String strInterface,
+      boolean returnSubinterfaces) throws UnknownHostException {
+    if ("default".equals(strInterface)) {
+      return new String[] { cachedHostAddress };
+    }
+    NetworkInterface netIf;
+    try {
+      netIf = NetworkInterface.getByName(strInterface);
+      if (netIf == null) {
+        netIf = getSubinterface(strInterface);
+      }
+    } catch (SocketException e) {
+      LOG.warn("I/O error finding interface " + strInterface +
+          ": " + e.getMessage());
+      return new String[] { cachedHostAddress };
+    }
+    if (netIf == null) {
+      throw new UnknownHostException("No such interface " + strInterface);
+    }
+
+    // NB: Using a LinkedHashSet to preserve the order for callers
+    // that depend on a particular element being 1st in the array.
+    // For example, getDefaultIP always returns the first element.
+    LinkedHashSet<InetAddress> allAddrs = new LinkedHashSet<InetAddress>();
+    allAddrs.addAll(Collections.list(netIf.getInetAddresses()));
+    if (!returnSubinterfaces) {
+      allAddrs.removeAll(getSubinterfaceInetAddrs(netIf));
+    }
+
+    String ips[] = new String[allAddrs.size()];
+    int i = 0;
+    for (InetAddress addr : allAddrs) {
+      ips[i++] = addr.getHostAddress();
+    }
+    return ips;
+  }
+
+
+  /**
+   * Returns the first available IP address associated with the provided
+   * network interface or the local host IP if "default" is given.
+   *
+   * @param strInterface
+   *            The name of the network interface or subinterface to query
+   *             (e.g. eth0 or eth0:0) or the string "default"
+   * @return The IP address in text form, the local host IP is returned
+   *         if the interface name "default" is specified
+   * @throws UnknownHostException
+   *             If the given interface is invalid
+   */
+  public static String getDefaultIP(String strInterface)
+    throws UnknownHostException {
+    String[] ips = getIPs(strInterface);
+    return ips[0];
+  }
+
+  /**
+   * Returns all the host names associated by the provided nameserver with the
+   * address bound to the specified network interface
+   *
+   * @param strInterface
+   *            The name of the network interface or subinterface to query
+   *            (e.g. eth0 or eth0:0)
+   * @param nameserver
+   *            The DNS host name
+   * @return A string vector of all host names associated with the IPs tied to
+   *         the specified interface
+   * @throws UnknownHostException if the given interface is invalid
+   */
+  public static String[] getHosts(String strInterface, String nameserver)
+    throws UnknownHostException {
+    String[] ips = getIPs(strInterface);
+    Vector<String> hosts = new Vector<String>();
+    for (int ctr = 0; ctr < ips.length; ctr++) {
+      try {
+        hosts.add(reverseDns(InetAddress.getByName(ips[ctr]),
+                             nameserver));
+      } catch (UnknownHostException ignored) {
+      } catch (NamingException ignored) {
+      }
+    }
+    if (hosts.isEmpty()) {
+      LOG.warn("Unable to determine hostname for interface " + strInterface);
+      return new String[] { cachedHostname };
+    } else {
+      return hosts.toArray(new String[hosts.size()]);
+    }
+  }
+
+
+  /**
+   * Determine the local hostname; retrieving it from cache if it is known
+   * If we cannot determine our host name, return "localhost"
+   * @return the local hostname or "localhost"
+   */
+  private static String resolveLocalHostname() {
+    String localhost;
+    try {
+      localhost = InetAddress.getLocalHost().getCanonicalHostName();
+    } catch (UnknownHostException e) {
+      LOG.warn("Unable to determine local hostname "
+          + "-falling back to \"" + LOCALHOST + "\"", e);
+      localhost = LOCALHOST;
+    }
+    return localhost;
+  }
+
+
+  /**
+   * Get the IPAddress of the local host as a string.
+   * This will be a loop back value if the local host address cannot be
+   * determined.
+   * If the loopback address of "localhost" does not resolve, then the system's
+   * network is in such a state that nothing is going to work. A message is
+   * logged at the error level and a null pointer returned, a pointer
+   * which will trigger failures later on the application
+   * @return the IPAddress of the local host or null for a serious problem.
+   */
+  private static String resolveLocalHostIPAddress() {
+    String address;
+      try {
+        address = InetAddress.getLocalHost().getHostAddress();
+      } catch (UnknownHostException e) {
+        LOG.warn("Unable to determine address of the host"
+                + "-falling back to \"" + LOCALHOST + "\" address", e);
+        try {
+          address = InetAddress.getByName(LOCALHOST).getHostAddress();
+        } catch (UnknownHostException noLocalHostAddressException) {
+          //at this point, deep trouble
+          LOG.error("Unable to determine local loopback address "
+                  + "of \"" + LOCALHOST + "\" " +
+                  "-this system's network configuration is unsupported", e);
+          address = null;
+        }
+      }
+    return address;
+  }
+
+  /**
+   * Returns all the host names associated by the default nameserver with the
+   * address bound to the specified network interface
+   *
+   * @param strInterface
+   *            The name of the network interface to query (e.g. eth0)
+   * @return The list of host names associated with IPs bound to the network
+   *         interface
+   * @throws UnknownHostException
+   *             If one is encountered while querying the default interface
+   *
+   */
+  public static String[] getHosts(String strInterface)
+    throws UnknownHostException {
+    return getHosts(strInterface, null);
+  }
+
+  /**
+   * Returns the default (first) host name associated by the provided
+   * nameserver with the address bound to the specified network interface
+   *
+   * @param strInterface
+   *            The name of the network interface to query (e.g. eth0)
+   * @param nameserver
+   *            The DNS host name
+   * @return The default host names associated with IPs bound to the network
+   *         interface
+   * @throws UnknownHostException
+   *             If one is encountered while querying the default interface
+   */
+  public static String getDefaultHost(String strInterface, String nameserver)
+    throws UnknownHostException {
+    if ("default".equals(strInterface)) {
+      return cachedHostname;
+    }
+
+    if ("default".equals(nameserver)) {
+      return getDefaultHost(strInterface);
+    }
+
+    String[] hosts = getHosts(strInterface, nameserver);
+    return hosts[0];
+  }
+
+  /**
+   * Returns the default (first) host name associated by the default
+   * nameserver with the address bound to the specified network interface
+   *
+   * @param strInterface
+   *            The name of the network interface to query (e.g. eth0).
+   *            Must not be null.
+   * @return The default host name associated with IPs bound to the network
+   *         interface
+   * @throws UnknownHostException
+   *             If one is encountered while querying the default interface
+   */
+  public static String getDefaultHost(String strInterface)
+    throws UnknownHostException {
+    return getDefaultHost(strInterface, null);
+  }
+
+}

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerChecker.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerChecker.java?rev=1493400&r1=1493399&r2=1493400&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerChecker.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerChecker.java Sat Jun 15 17:05:45 2013
@@ -454,7 +454,7 @@ public class TestLedgerChecker extends B
     }
 
     private void killBookie(ArrayList<InetSocketAddress> firstEnsemble,
-            InetSocketAddress ensemble) throws InterruptedException {
+            InetSocketAddress ensemble) throws Exception {
         LOG.info("Killing " + ensemble + " from ensemble=" + firstEnsemble);
         killBookie(ensemble);
     }

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestPerChannelBookieClient.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestPerChannelBookieClient.java?rev=1493400&r1=1493399&r2=1493400&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestPerChannelBookieClient.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestPerChannelBookieClient.java Sat Jun 15 17:05:45 2013
@@ -55,7 +55,7 @@ public class TestPerChannelBookieClient 
      * {@link https://issues.apache.org/jira/browse/BOOKKEEPER-485}.
      */
     @Test(timeout=60000)
-    public void testConnectCloseRace() {
+    public void testConnectCloseRace() throws Exception {
         ClientSocketChannelFactory channelFactory
             = new NioClientSocketChannelFactory(Executors.newCachedThreadPool(),
                                                 Executors.newCachedThreadPool());
@@ -85,7 +85,7 @@ public class TestPerChannelBookieClient 
      * correctly, this causes the netty channel to get orphaned.
      */
     @Test(timeout=60000)
-    public void testConnectRace() {
+    public void testConnectRace() throws Exception {
         GenericCallback<Void> nullop = new GenericCallback<Void>() {
             @Override
             public void operationComplete(int rc, Void result) {
@@ -111,4 +111,4 @@ public class TestPerChannelBookieClient 
         channelFactory.releaseExternalResources();
         executor.shutdown();
     }
-}
\ No newline at end of file
+}

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorBookieTest.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorBookieTest.java?rev=1493400&r1=1493399&r2=1493400&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorBookieTest.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorBookieTest.java Sat Jun 15 17:05:45 2013
@@ -253,8 +253,7 @@ public class AuditorBookieTest extends B
         }
     }
 
-    private BookieServer verifyAuditor() throws KeeperException,
-            InterruptedException {
+    private BookieServer verifyAuditor() throws Exception {
         List<BookieServer> auditors = getAuditorBookie();
         Assert.assertEquals("Multiple Bookies acting as Auditor!", 1, auditors
                 .size());
@@ -262,8 +261,7 @@ public class AuditorBookieTest extends B
         return auditors.get(0);
     }
 
-    private List<BookieServer> getAuditorBookie() throws KeeperException,
-            InterruptedException {
+    private List<BookieServer> getAuditorBookie() throws Exception {
         List<BookieServer> auditors = new LinkedList<BookieServer>();
         byte[] data = zkc.getData(electionPath, false, null);
         Assert.assertNotNull("Auditor election failed", data);
@@ -286,7 +284,7 @@ public class AuditorBookieTest extends B
     }
 
     private BookieServer waitForNewAuditor(BookieServer auditor)
-            throws InterruptedException, KeeperException {
+            throws Exception {
         BookieServer newAuditor = null;
         int retryCount = 8;
         while (retryCount > 0) {

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java?rev=1493400&r1=1493399&r2=1493400&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java Sat Jun 15 17:05:45 2013
@@ -321,8 +321,7 @@ public class AuditorLedgerCheckerTest ex
         }
     }
 
-    private String shutdownBookie(int bkShutdownIndex) throws IOException,
-            InterruptedException {
+    private String shutdownBookie(int bkShutdownIndex) throws Exception {
         BookieServer bkServer = bs.get(bkShutdownIndex);
         String bookieAddr = StringUtils.addrToString(bkServer.getLocalAddress());
         LOG.debug("Shutting down bookie:" + bookieAddr);

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/BookieAutoRecoveryTest.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/BookieAutoRecoveryTest.java?rev=1493400&r1=1493399&r2=1493400&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/BookieAutoRecoveryTest.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/BookieAutoRecoveryTest.java Sat Jun 15 17:05:45 2013
@@ -358,7 +358,7 @@ public class BookieAutoRecoveryTest exte
 
     private void verifyLedgerEnsembleMetadataAfterReplication(
             BookieServer newBookieServer, LedgerHandle lh,
-            int ledgerReplicaIndex) throws BKException, InterruptedException {
+            int ledgerReplicaIndex) throws Exception {
         LedgerHandle openLedger = bkc
                 .openLedger(lh.getId(), digestType, PASSWD);
 

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java?rev=1493400&r1=1493399&r2=1493400&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java Sat Jun 15 17:05:45 2013
@@ -529,7 +529,7 @@ public class TestReplicationWorker exten
     }
 
     private void killAllBookies(LedgerHandle lh, InetSocketAddress excludeBK)
-            throws InterruptedException {
+            throws Exception {
         // Killing all bookies except newly replicated bookie
         Set<Entry<Long, ArrayList<InetSocketAddress>>> entrySet = LedgerHandleAdapter
                 .getLedgerMetadata(lh).getEnsembles().entrySet();

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java?rev=1493400&r1=1493399&r2=1493400&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java Sat Jun 15 17:05:45 2013
@@ -175,7 +175,7 @@ public abstract class BookKeeperClusterT
         }
     }
 
-    protected ServerConfiguration newServerConfiguration() throws IOException {
+    protected ServerConfiguration newServerConfiguration() throws Exception {
         File f = File.createTempFile("bookie", "test");
         tmpDirs.add(f);
         f.delete();
@@ -191,6 +191,7 @@ public abstract class BookKeeperClusterT
         conf.setBookiePort(port);
         conf.setZkServers(zkServers);
         conf.setJournalDirName(journalDir.getPath());
+        conf.setAllowLoopback(true);
         String[] ledgerDirNames = new String[ledgerDirs.length];
         for (int i=0; i<ledgerDirs.length; i++) {
             ledgerDirNames[i] = ledgerDirs[i].getPath();
@@ -202,7 +203,7 @@ public abstract class BookKeeperClusterT
     /**
      * Get bookie address for bookie at index
      */
-    public InetSocketAddress getBookie(int index) throws IllegalArgumentException {
+    public InetSocketAddress getBookie(int index) throws Exception {
         if (bs.size() <= index || index < 0) {
             throw new IllegalArgumentException("Invalid index, there are only " + bs.size()
                                                + " bookies. Asked for " + index);
@@ -219,7 +220,7 @@ public abstract class BookKeeperClusterT
      * @return the configuration of killed bookie
      * @throws InterruptedException
      */
-    public ServerConfiguration killBookie(InetSocketAddress addr) throws InterruptedException {
+    public ServerConfiguration killBookie(InetSocketAddress addr) throws Exception {
         BookieServer toRemove = null;
         int toRemoveIndex = 0;
         for (BookieServer server : bs) {
@@ -248,7 +249,7 @@ public abstract class BookKeeperClusterT
      * @throws InterruptedException
      * @throws IOException
      */
-    public ServerConfiguration killBookie(int index) throws InterruptedException, IOException {
+    public ServerConfiguration killBookie(int index) throws Exception {
         if (index >= bs.size()) {
             throw new IOException("Bookie does not exist");
         }
@@ -271,7 +272,7 @@ public abstract class BookKeeperClusterT
      * @throws IOException
      */
     public CountDownLatch sleepBookie(InetSocketAddress addr, final int seconds)
-            throws InterruptedException, IOException {
+            throws Exception {
         for (final BookieServer bookie : bs) {
             if (bookie.getLocalAddress().equals(addr)) {
                 final CountDownLatch l = new CountDownLatch(1);
@@ -306,7 +307,7 @@ public abstract class BookKeeperClusterT
      * @throws IOException
      */
     public void sleepBookie(InetSocketAddress addr, final CountDownLatch l)
-            throws InterruptedException, IOException {
+            throws Exception {
         for (final BookieServer bookie : bs) {
             if (bookie.getLocalAddress().equals(addr)) {
                 Thread sleeper = new Thread() {
@@ -337,8 +338,7 @@ public abstract class BookKeeperClusterT
      * @throws BookieException
      */
     public void restartBookies()
-            throws InterruptedException, IOException, KeeperException,
-            BookieException, UnavailableException, CompatibilityException {
+            throws Exception {
         restartBookies(null);
     }
 
@@ -354,8 +354,7 @@ public abstract class BookKeeperClusterT
      * @throws BookieException
      */
     public void restartBookies(ServerConfiguration newConf)
-            throws InterruptedException, IOException, KeeperException,
-            BookieException, UnavailableException, CompatibilityException {
+            throws Exception {
         // shut down bookie server
         for (BookieServer server : bs) {
             server.shutdown();
@@ -384,8 +383,7 @@ public abstract class BookKeeperClusterT
      * @throws IOException
      */
     public int startNewBookie()
-            throws IOException, InterruptedException, KeeperException,
-            BookieException, UnavailableException, CompatibilityException {
+            throws Exception {
         ServerConfiguration conf = newServerConfiguration();
         bsConfs.add(conf);
         bs.add(startBookie(conf));
@@ -402,8 +400,7 @@ public abstract class BookKeeperClusterT
      *
      */
     protected BookieServer startBookie(ServerConfiguration conf)
-            throws IOException, InterruptedException, KeeperException,
-            BookieException, UnavailableException, CompatibilityException {
+            throws Exception {
         BookieServer server = new BookieServer(conf);
         server.start();
 
@@ -430,8 +427,7 @@ public abstract class BookKeeperClusterT
      * recovery for this bookie, if isAutoRecoveryEnabled is true.
      */
     protected BookieServer startBookie(ServerConfiguration conf, final Bookie b)
-            throws IOException, InterruptedException, KeeperException,
-            BookieException, UnavailableException, CompatibilityException {
+            throws Exception {
         BookieServer server = new BookieServer(conf) {
             @Override
             protected Bookie newBookie(ServerConfiguration conf) {
@@ -486,9 +482,7 @@ public abstract class BookKeeperClusterT
     }
 
     private void startAutoRecovery(BookieServer bserver,
-            ServerConfiguration conf) throws CompatibilityException,
-            KeeperException, InterruptedException, IOException,
-            UnavailableException {
+                                   ServerConfiguration conf) throws Exception {
         if (isAutoRecoveryEnabled()) {
             AutoRecoveryMain autoRecoveryProcess = new AutoRecoveryMain(conf);
             autoRecoveryProcess.start();
@@ -498,7 +492,7 @@ public abstract class BookKeeperClusterT
         }
     }
 
-    private void stopAutoRecoveryService(BookieServer toRemove) {
+    private void stopAutoRecoveryService(BookieServer toRemove) throws Exception {
         AutoRecoveryMain autoRecoveryMain = autoRecoveryProcesses
                 .remove(toRemove);
         if (null != autoRecoveryMain && isAutoRecoveryEnabled()) {
@@ -512,21 +506,8 @@ public abstract class BookKeeperClusterT
      * Will starts the auto recovery process for the bookie servers. One auto
      * recovery process per each bookie server, if isAutoRecoveryEnabled is
      * enabled.
-     *
-     * @throws CompatibilityException
-     *             - Compatibility error
-     * @throws KeeperException
-     *             - ZK exception
-     * @throws InterruptedException
-     *             - interrupted exception
-     * @throws IOException
-     *             - IOException
-     * @throws UnavailableException
-     *             - replication service has become unavailable
-     */
-    public void startReplicationService() throws CompatibilityException,
-            KeeperException, InterruptedException, IOException,
-            UnavailableException {
+     */
+    public void startReplicationService() throws Exception {
         int index = -1;
         for (BookieServer bserver : bs) {
             startAutoRecovery(bserver, bsConfs.get(++index));
@@ -537,7 +518,7 @@ public abstract class BookKeeperClusterT
      * Will stops all the auto recovery processes for the bookie cluster, if
      * isAutoRecoveryEnabled is true.
      */
-    public void stopReplicationService() {
+    public void stopReplicationService() throws Exception{
         if(false == isAutoRecoveryEnabled()){
             return;
         }