You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@reef.apache.org by se...@apache.org on 2015/04/21 03:15:42 UTC

[1/2] incubator-reef git commit: [REEF-246] Add LocalAddressProvider

Repository: incubator-reef
Updated Branches:
  refs/heads/master ac380af06 -> a72a40f8d


http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/address/HostnameBasedLocalAddressProvider.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/address/HostnameBasedLocalAddressProvider.java b/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/address/HostnameBasedLocalAddressProvider.java
new file mode 100644
index 0000000..844cb8e
--- /dev/null
+++ b/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/address/HostnameBasedLocalAddressProvider.java
@@ -0,0 +1,66 @@
+/*
+ * 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.
+ */
+package org.apache.reef.wake.remote.address;
+
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.Tang;
+
+import javax.inject.Inject;
+import java.net.Inet4Address;
+import java.net.UnknownHostException;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * A LocalAddressProvider that uses <code>Inet4Address.getLocalHost().getHostAddress()</code>
+ */
+public final class HostnameBasedLocalAddressProvider implements LocalAddressProvider {
+  private static final Logger LOG = Logger.getLogger(HostnameBasedLocalAddressProvider.class.getName());
+  private String cached = null;
+
+  /**
+   * The constructor is for Tang only.
+   */
+  @Inject
+  private HostnameBasedLocalAddressProvider() {
+    LOG.log(Level.INFO, "Instantiating HostnameBasedLocalAddressProvider");
+  }
+
+  @Override
+  public synchronized String getLocalAddress() {
+    if (null == cached) {
+      try {
+        cached = Inet4Address.getLocalHost().getHostAddress();
+      } catch (final UnknownHostException ex) {
+        final String message = "Unable to resolve LocalHost. This is fatal.";
+        LOG.log(Level.SEVERE, message, ex);
+        throw new RuntimeException(message, ex);
+      }
+    }
+    assert (null != cached);
+    return cached;
+  }
+
+  @Override
+  public Configuration getConfiguration() {
+    return Tang.Factory.getTang().newConfigurationBuilder()
+        .bind(LocalAddressProvider.class, HostnameBasedLocalAddressProvider.class)
+        .build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/address/LegacyLocalAddressProvider.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/address/LegacyLocalAddressProvider.java b/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/address/LegacyLocalAddressProvider.java
new file mode 100644
index 0000000..7ab18fc
--- /dev/null
+++ b/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/address/LegacyLocalAddressProvider.java
@@ -0,0 +1,126 @@
+/*
+ * 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.
+ */
+package org.apache.reef.wake.remote.address;
+
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.wake.exception.WakeRuntimeException;
+
+import javax.inject.Inject;
+import java.net.Inet4Address;
+import java.net.InetAddress;
+import java.net.NetworkInterface;
+import java.net.SocketException;
+import java.util.Comparator;
+import java.util.Enumeration;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * An implementation of LocalAddressProvider using the (removed) code from NetUtils.getLocalAddress()
+ */
+public final class LegacyLocalAddressProvider implements LocalAddressProvider {
+  private static final Logger LOG = Logger.getLogger(LegacyLocalAddressProvider.class.getName());
+  private final AtomicReference<String> cached = new AtomicReference<>();
+
+  /**
+   * Injectable constructor for Tang only.
+   */
+  @Inject
+  private LegacyLocalAddressProvider() {
+    LOG.log(Level.INFO, "Instantiating LegacyLocalAddressProvider");
+  }
+
+  @Override
+  public String getLocalAddress() {
+    // This method is surprisingly slow: It was causing unit test timeouts, so we memoize the result.
+    if (cached.get() == null) {
+      Enumeration<NetworkInterface> ifaces;
+      try {
+        ifaces = NetworkInterface.getNetworkInterfaces();
+        TreeSet<Inet4Address> sortedAddrs = new TreeSet<>(new AddressComparator());
+        // There is an idea of virtual / subinterfaces exposed by java here.
+        // We're not walking around looking for those because the javadoc says:
+
+        // "NOTE: can use getNetworkInterfaces()+getInetAddresses() to obtain all IP addresses for this node"
+
+        while (ifaces.hasMoreElements()) {
+          NetworkInterface iface = ifaces.nextElement();
+//          if(iface.isUp()) {  // leads to slowness and non-deterministic return values, so don't call isUp().
+          Enumeration<InetAddress> addrs = iface.getInetAddresses();
+          while (addrs.hasMoreElements()) {
+            InetAddress a = addrs.nextElement();
+            if (a instanceof Inet4Address) {
+              sortedAddrs.add((Inet4Address) a);
+            }
+//            }
+          }
+        }
+        if (sortedAddrs.isEmpty()) {
+          throw new WakeRuntimeException("This machine apparently doesn't have any IP addresses (not even 127.0.0.1) on interfaces that are up.");
+        }
+        cached.set(sortedAddrs.pollFirst().getHostAddress());
+        LOG.log(Level.FINE, "Local address is {0}", cached.get());
+      } catch (SocketException e) {
+        throw new WakeRuntimeException("Unable to get local host address",
+            e.getCause());
+      }
+    }
+    return cached.get();
+  }
+
+  @Override
+  public Configuration getConfiguration() {
+    return Tang.Factory.getTang().newConfigurationBuilder()
+        .bind(LocalAddressProvider.class, LegacyLocalAddressProvider.class)
+        .build();
+  }
+
+  private static class AddressComparator implements Comparator<Inet4Address> {
+
+    // get unsigned byte.
+    private static int u(byte b) {
+      return ((int) b);// & 0xff;
+    }
+
+    @Override
+    public int compare(Inet4Address aa, Inet4Address ba) {
+      byte[] a = aa.getAddress();
+      byte[] b = ba.getAddress();
+      // local subnet comes after all else.
+      if (a[0] == 127 && b[0] != 127) {
+        return 1;
+      }
+      if (a[0] != 127 && b[0] == 127) {
+        return -1;
+      }
+      for (int i = 0; i < 4; i++) {
+        if (u(a[i]) < u(b[i])) {
+          return -1;
+        }
+        if (u(a[i]) > u(b[i])) {
+          return 1;
+        }
+      }
+      return 0;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/address/LocalAddressProvider.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/address/LocalAddressProvider.java b/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/address/LocalAddressProvider.java
new file mode 100644
index 0000000..e0fd9c2
--- /dev/null
+++ b/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/address/LocalAddressProvider.java
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+package org.apache.reef.wake.remote.address;
+
+import org.apache.reef.tang.ConfigurationProvider;
+import org.apache.reef.tang.annotations.DefaultImplementation;
+
+/**
+ * Injectable class that provides the local address of the node to bind to.
+ */
+@DefaultImplementation(HostnameBasedLocalAddressProvider.class)
+public interface LocalAddressProvider extends ConfigurationProvider {
+
+  /**
+   * @return a String representation of the local address.
+   */
+  String getLocalAddress();
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/address/LocalAddressProviderFactory.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/address/LocalAddressProviderFactory.java b/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/address/LocalAddressProviderFactory.java
new file mode 100644
index 0000000..e6f4a3c
--- /dev/null
+++ b/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/address/LocalAddressProviderFactory.java
@@ -0,0 +1,69 @@
+/*
+ * 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.
+ */
+package org.apache.reef.wake.remote.address;
+
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.exceptions.InjectionException;
+
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * Uses Tang to create the default LocalAddressProvider.
+ *
+ * @deprecated Have an instance of LocalAddressProvider injected instead.
+ */
+@Deprecated
+public final class LocalAddressProviderFactory {
+  private static final Logger LOGGER = Logger.getLogger(LocalAddressProviderFactory.class.getName());
+  private static LocalAddressProvider instance = null;
+
+  /**
+   * This class shall never be instantiated.
+   */
+  private LocalAddressProviderFactory() {
+    // Intentionally left blank.
+  }
+
+  /**
+   * @return the default LocalAddressProvider
+   * @deprecated Have an instance of LocalAddressProvider injected instead.
+   */
+  public static LocalAddressProvider getInstance() {
+    if (null == instance) {
+      makeInstance();
+    }
+    return instance;
+  }
+
+  /**
+   * Makes the instance.
+   */
+  private static void makeInstance() {
+    assert (null == instance);
+    try {
+      LOGGER.log(Level.FINER, "Instantiating default LocalAddressProvider for legacy users.");
+      instance = Tang.Factory.getTang().newInjector().getInstance(LocalAddressProvider.class);
+      LOGGER.log(Level.FINER, "Instantiated default LocalAddressProvider for legacy users.");
+    } catch (InjectionException e) {
+      throw new RuntimeException("Unable to instantiate default LocalAddressProvider for legacy users.", e);
+    }
+    assert (null != instance);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/address/package-info.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/address/package-info.java b/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/address/package-info.java
new file mode 100644
index 0000000..79b2aa7
--- /dev/null
+++ b/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/address/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+/**
+ * Local address resolution.
+ */
+package org.apache.reef.wake.remote.address;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/impl/DefaultRemoteManagerImplementation.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/impl/DefaultRemoteManagerImplementation.java b/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/impl/DefaultRemoteManagerImplementation.java
index fc6b804..ff1679c 100644
--- a/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/impl/DefaultRemoteManagerImplementation.java
+++ b/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/impl/DefaultRemoteManagerImplementation.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -23,6 +23,8 @@ import org.apache.reef.wake.EStage;
 import org.apache.reef.wake.EventHandler;
 import org.apache.reef.wake.impl.StageManager;
 import org.apache.reef.wake.remote.*;
+import org.apache.reef.wake.remote.address.LocalAddressProvider;
+import org.apache.reef.wake.remote.address.LocalAddressProviderFactory;
 import org.apache.reef.wake.remote.transport.Transport;
 import org.apache.reef.wake.remote.transport.netty.NettyMessagingTransport;
 
@@ -58,6 +60,36 @@ public class DefaultRemoteManagerImplementation implements RemoteManager {
   private final RemoteSeqNumGenerator seqGen = new RemoteSeqNumGenerator();
   private RemoteIdentifier myIdentifier;
 
+
+  /**
+   * @deprecated have an instance injected instead.
+   */
+  @Deprecated
+  public <T> DefaultRemoteManagerImplementation(
+      final String name,
+      final String hostAddress,
+      final int listeningPort,
+      final Codec<T> codec,
+      final EventHandler<Throwable> errorHandler,
+      final boolean orderingGuarantee,
+      final int numberOfTries,
+      final int retryTimeout) {
+    this(name,
+        hostAddress,
+        listeningPort,
+        codec,
+        errorHandler,
+        orderingGuarantee,
+        numberOfTries,
+        retryTimeout,
+        LocalAddressProviderFactory.getInstance());
+
+  }
+
+  /**
+   * @deprecated have an instance injected instead.
+   */
+  @Deprecated
   @Inject
   public <T> DefaultRemoteManagerImplementation(
       final @Parameter(RemoteConfiguration.ManagerName.class) String name,
@@ -67,7 +99,8 @@ public class DefaultRemoteManagerImplementation implements RemoteManager {
       final @Parameter(RemoteConfiguration.ErrorHandler.class) EventHandler<Throwable> errorHandler,
       final @Parameter(RemoteConfiguration.OrderingGuarantee.class) boolean orderingGuarantee,
       final @Parameter(RemoteConfiguration.NumberOfTries.class) int numberOfTries,
-      final @Parameter(RemoteConfiguration.RetryTimeout.class) int retryTimeout) {
+      final @Parameter(RemoteConfiguration.RetryTimeout.class) int retryTimeout,
+      final LocalAddressProvider localAddressProvider) {
 
     this.name = name;
     this.handlerContainer = new HandlerContainer<>(name, codec);
@@ -78,7 +111,7 @@ public class DefaultRemoteManagerImplementation implements RemoteManager {
 
     if ("##UNKNOWN##".equals(hostAddress)) {
       this.transport = new NettyMessagingTransport(
-          NetUtils.getLocalAddress(), listeningPort, this.reRecvStage, this.reRecvStage, numberOfTries, retryTimeout);
+          localAddressProvider.getLocalAddress(), listeningPort, this.reRecvStage, this.reRecvStage, numberOfTries, retryTimeout);
     } else {
       this.transport = new NettyMessagingTransport(
           hostAddress, listeningPort, this.reRecvStage, this.reRecvStage, numberOfTries, retryTimeout);
@@ -93,10 +126,10 @@ public class DefaultRemoteManagerImplementation implements RemoteManager {
 
     StageManager.instance().register(this);
 
-    LOG.log(Level.FINEST, "RemoteManager {0} instantiated id {1} counter {2} listening on {3}:{4}",
+    LOG.log(Level.FINEST, "RemoteManager {0} instantiated id {1} counter {2} listening on {3}:{4}. Binding address provided by {5}",
         new Object[]{this.name, this.myIdentifier, counter.incrementAndGet(),
             this.transport.getLocalAddress().toString(),
-            this.transport.getListeningPort()}
+            this.transport.getListeningPort(), localAddressProvider}
     );
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/LargeMsgTest.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/LargeMsgTest.java b/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/LargeMsgTest.java
index db59b34..3defe01 100644
--- a/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/LargeMsgTest.java
+++ b/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/LargeMsgTest.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -18,13 +18,16 @@
  */
 package org.apache.reef.wake.test.remote;
 
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.exceptions.InjectionException;
 import org.apache.reef.wake.EStage;
 import org.apache.reef.wake.EventHandler;
 import org.apache.reef.wake.impl.LoggingEventHandler;
 import org.apache.reef.wake.impl.LoggingUtils;
 import org.apache.reef.wake.impl.ThreadPoolStage;
 import org.apache.reef.wake.impl.TimerStage;
-import org.apache.reef.wake.remote.NetUtils;
+import org.apache.reef.wake.remote.address.LocalAddressProvider;
+import org.apache.reef.wake.remote.address.LocalAddressProviderFactory;
 import org.apache.reef.wake.remote.impl.TransportEvent;
 import org.apache.reef.wake.remote.transport.Link;
 import org.apache.reef.wake.remote.transport.netty.NettyMessagingTransport;
@@ -35,7 +38,6 @@ import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.logging.Level;
 
@@ -43,11 +45,16 @@ import java.util.logging.Level;
  * Test transferring large messages
  */
 public class LargeMsgTest {
+  private final LocalAddressProvider localAddressProvider;
   private final static byte[][] values = new byte[3][];
   private final static int l0 = 1 << 25;
   private final static int l1 = 1 << 2;
   private final static int l2 = 1 << 21;
 
+  public LargeMsgTest() throws InjectionException {
+    this.localAddressProvider = LocalAddressProviderFactory.getInstance();
+  }
+
   @BeforeClass
   public static void setUpBeforeClass() {
     values[0] = new byte[l0];
@@ -78,7 +85,7 @@ public class LargeMsgTest {
     EStage<TransportEvent> serverStage = new ThreadPoolStage<>("server@7001",
         new ServerHandler(monitor, dataSize), 1, new LoggingEventHandler<Throwable>());
 
-    String hostAddress = NetUtils.getLocalAddress();
+    final String hostAddress = this.localAddressProvider.getLocalAddress();
     int port = 7001;
     NettyMessagingTransport transport = new NettyMessagingTransport(hostAddress, port, clientStage, serverStage, 1, 10000);
     final Link<byte[]> link = transport.open(new InetSocketAddress(hostAddress, port), new PassThroughEncoder(), null);

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/RemoteIdentifierFactoryTest.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/RemoteIdentifierFactoryTest.java b/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/RemoteIdentifierFactoryTest.java
index fa85651..c003cb5 100644
--- a/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/RemoteIdentifierFactoryTest.java
+++ b/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/RemoteIdentifierFactoryTest.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -18,14 +18,14 @@
  */
 package org.apache.reef.wake.test.remote;
 
+import org.apache.reef.tang.Injector;
+import org.apache.reef.tang.Tang;
 import org.apache.reef.wake.Identifier;
 import org.apache.reef.wake.IdentifierFactory;
 import org.apache.reef.wake.impl.DefaultIdentifierFactory;
 import org.apache.reef.wake.impl.LoggingEventHandler;
-import org.apache.reef.wake.remote.Codec;
-import org.apache.reef.wake.remote.NetUtils;
-import org.apache.reef.wake.remote.RemoteIdentifier;
-import org.apache.reef.wake.remote.RemoteManager;
+import org.apache.reef.wake.remote.*;
+import org.apache.reef.wake.remote.address.LocalAddressProvider;
 import org.apache.reef.wake.remote.impl.DefaultRemoteManagerImplementation;
 import org.apache.reef.wake.remote.impl.MultiCodec;
 import org.junit.Assert;
@@ -59,27 +59,25 @@ public class RemoteIdentifierFactoryTest {
 
   @Test
   public void testRemoteManagerIdentifier() throws Exception {
-    System.out.println(logPrefix + name.getMethodName());
+    final Injector injector = Tang.Factory.getTang().newInjector();
+    final LocalAddressProvider localAddressProvider = injector.getInstance(LocalAddressProvider.class);
 
-    int port = 9100;
-    Map<Class<?>, Codec<?>> clazzToCodecMap = new HashMap<Class<?>, Codec<?>>();
+    final int port = 9100;
+    final Map<Class<?>, Codec<?>> clazzToCodecMap = new HashMap<Class<?>, Codec<?>>();
     clazzToCodecMap.put(TestEvent.class, new TestEventCodec());
-    Codec<?> codec = new MultiCodec<Object>(clazzToCodecMap);
-
-    String hostAddress = NetUtils.getLocalAddress();
+    final Codec<?> codec = new MultiCodec<Object>(clazzToCodecMap);
 
-    RemoteManager rm = new DefaultRemoteManagerImplementation("TestRemoteManager",
-        hostAddress, port, codec, new LoggingEventHandler<Throwable>(), false, 1, 10000);
-    RemoteIdentifier id = rm.getMyIdentifier();
-    System.out.println(id.toString());
 
-    IdentifierFactory factory = new DefaultIdentifierFactory();
-    Identifier newid = factory.getNewInstance(id.toString());
-    System.out.println(newid.toString());
+    try (final RemoteManager rm = new DefaultRemoteManagerImplementation("TestRemoteManager",
+        localAddressProvider.getLocalAddress(), port, codec, new LoggingEventHandler<Throwable>(), false, 1, 10000,
+        localAddressProvider)) {
+      final RemoteIdentifier id = rm.getMyIdentifier();
 
-    Assert.assertTrue(id.equals(newid));
+      final IdentifierFactory factory = new DefaultIdentifierFactory();
+      final Identifier newId = factory.getNewInstance(id.toString());
 
-    rm.close();
+      Assert.assertEquals(id, newId);
+    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/RemoteManagerTest.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/RemoteManagerTest.java b/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/RemoteManagerTest.java
index 7e98229..888574f 100644
--- a/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/RemoteManagerTest.java
+++ b/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/RemoteManagerTest.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -18,11 +18,14 @@
  */
 package org.apache.reef.wake.test.remote;
 
+import org.apache.reef.tang.exceptions.InjectionException;
 import org.apache.reef.wake.EventHandler;
 import org.apache.reef.wake.impl.LoggingEventHandler;
 import org.apache.reef.wake.impl.LoggingUtils;
 import org.apache.reef.wake.impl.TimerStage;
 import org.apache.reef.wake.remote.*;
+import org.apache.reef.wake.remote.address.LocalAddressProvider;
+import org.apache.reef.wake.remote.address.LocalAddressProviderFactory;
 import org.apache.reef.wake.remote.impl.DefaultRemoteIdentifierFactoryImplementation;
 import org.apache.reef.wake.remote.impl.DefaultRemoteManagerImplementation;
 import org.apache.reef.wake.remote.impl.MultiCodec;
@@ -47,6 +50,12 @@ import java.util.logging.Level;
 
 public class RemoteManagerTest {
 
+  private final LocalAddressProvider localAddressProvider;
+
+  public RemoteManagerTest() throws InjectionException {
+    this.localAddressProvider = LocalAddressProviderFactory.getInstance();
+  }
+
   @Rule
   public final TestName name = new TestName();
 
@@ -68,10 +77,10 @@ public class RemoteManagerTest {
     clazzToCodecMap.put(TestEvent2.class, new ObjectSerializableCodec<TestEvent2>());
     Codec<?> codec = new MultiCodec<Object>(clazzToCodecMap);
 
-    String hostAddress = NetUtils.getLocalAddress();
+    final String hostAddress = localAddressProvider.getLocalAddress();
 
     final RemoteManager rm = new DefaultRemoteManagerImplementation(
-        "name", hostAddress, port, codec, new LoggingEventHandler<Throwable>(), false, 3, 10000);
+        "name", hostAddress, port, codec, new LoggingEventHandler<Throwable>(), false, 3, 10000, localAddressProvider);
 
     RemoteIdentifierFactory factory = new DefaultRemoteIdentifierFactoryImplementation();
     RemoteIdentifier remoteId = factory.getNewInstance("socket://" + hostAddress + ":" + port);
@@ -170,10 +179,10 @@ public class RemoteManagerTest {
     clazzToCodecMap.put(TestEvent2.class, new ObjectSerializableCodec<TestEvent2>());
     Codec<?> codec = new MultiCodec<Object>(clazzToCodecMap);
 
-    String hostAddress = NetUtils.getLocalAddress();
+    final String hostAddress = localAddressProvider.getLocalAddress();
 
     final RemoteManager rm = new DefaultRemoteManagerImplementation(
-        "name", hostAddress, port, codec, new LoggingEventHandler<Throwable>(), true, 3, 10000);
+        "name", hostAddress, port, codec, new LoggingEventHandler<Throwable>(), true, 3, 10000, localAddressProvider);
 
     RemoteIdentifierFactory factory = new DefaultRemoteIdentifierFactoryImplementation();
     RemoteIdentifier remoteId = factory.getNewInstance("socket://" + hostAddress + ":" + port);
@@ -213,10 +222,10 @@ public class RemoteManagerTest {
     clazzToCodecMap.put(TestEvent.class, new TestEventCodec());
     Codec<?> codec = new MultiCodec<Object>(clazzToCodecMap);
 
-    String hostAddress = NetUtils.getLocalAddress();
+    String hostAddress = localAddressProvider.getLocalAddress();
 
     final RemoteManager rm = new DefaultRemoteManagerImplementation(
-        "name", hostAddress, port, codec, new LoggingEventHandler<Throwable>(), false, 3, 10000);
+        "name", hostAddress, port, codec, new LoggingEventHandler<Throwable>(), false, 3, 10000, localAddressProvider);
 
     RemoteIdentifierFactory factory = new DefaultRemoteIdentifierFactoryImplementation();
     RemoteIdentifier remoteId = factory.getNewInstance("socket://" + hostAddress + ":" + port);
@@ -250,12 +259,12 @@ public class RemoteManagerTest {
     clazzToCodecMap.put(TestEvent.class, new ObjectSerializableCodec<TestEvent>());
     Codec<?> codec = new MultiCodec<Object>(clazzToCodecMap);
 
-    String hostAddress = NetUtils.getLocalAddress();
+    final String hostAddress = localAddressProvider.getLocalAddress();
 
     ExceptionHandler errorHandler = new ExceptionHandler(monitor);
 
     try (final RemoteManager rm = new DefaultRemoteManagerImplementation(
-        "name", hostAddress, port, codec, errorHandler, false, 3, 10000)) {
+        "name", hostAddress, port, codec, errorHandler, false, 3, 10000, localAddressProvider)) {
 
       RemoteIdentifierFactory factory = new DefaultRemoteIdentifierFactoryImplementation();
       RemoteIdentifier remoteId = factory.getNewInstance("socket://" + hostAddress + ":" + port);
@@ -281,9 +290,9 @@ public class RemoteManagerTest {
     clazzToCodecMap.put(TestEvent2.class, new ObjectSerializableCodec<TestEvent1>());
     Codec<?> codec = new MultiCodec<Object>(clazzToCodecMap);
 
-    String hostAddress = NetUtils.getLocalAddress();
+    String hostAddress = localAddressProvider.getLocalAddress();
     return new DefaultRemoteManagerImplementation(name, hostAddress, localPort,
-        codec, new LoggingEventHandler<Throwable>(), false, retry, retryTimeout);
+        codec, new LoggingEventHandler<Throwable>(), false, retry, retryTimeout, localAddressProvider);
   }
 
   private class SendingRemoteManagerThread implements Callable<Integer> {
@@ -304,7 +313,7 @@ public class RemoteManagerTest {
       Monitor monitor = new Monitor();
       TimerStage timer = new TimerStage(new TimeoutHandler(monitor), timeout, timeout);
 
-      String hostAddress = NetUtils.getLocalAddress();
+      final String hostAddress = localAddressProvider.getLocalAddress();
       RemoteIdentifierFactory factory = new DefaultRemoteIdentifierFactoryImplementation();
       RemoteIdentifier remoteId = factory.getNewInstance("socket://" + hostAddress + ":" + remotePort);
 

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/RemoteTest.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/RemoteTest.java b/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/RemoteTest.java
index 2e06b81..636315f 100644
--- a/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/RemoteTest.java
+++ b/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/RemoteTest.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -24,6 +24,8 @@ import org.apache.reef.wake.impl.LoggingUtils;
 import org.apache.reef.wake.impl.MultiEventHandler;
 import org.apache.reef.wake.impl.TimerStage;
 import org.apache.reef.wake.remote.*;
+import org.apache.reef.wake.remote.address.LocalAddressProvider;
+import org.apache.reef.wake.remote.address.LocalAddressProviderFactory;
 import org.apache.reef.wake.remote.impl.*;
 import org.apache.reef.wake.remote.transport.Transport;
 import org.apache.reef.wake.remote.transport.netty.NettyMessagingTransport;
@@ -41,12 +43,17 @@ import java.util.*;
 import java.util.logging.Level;
 
 public class RemoteTest {
-
+  private final LocalAddressProvider localAddressProvider;
   @Rule
   public final TestName name = new TestName();
 
   final String logPrefix = "TEST ";
 
+
+  public RemoteTest() {
+    this.localAddressProvider = LocalAddressProviderFactory.getInstance();
+  }
+
   @Test
   public void testRemoteEventCodec() throws UnknownHostException {
     System.out.println(logPrefix + name.getMethodName());
@@ -54,8 +61,8 @@ public class RemoteTest {
     ObjectSerializableCodec<TestEvent> codec = new ObjectSerializableCodec<TestEvent>();
 
     RemoteEventCodec<TestEvent> reCodec = new RemoteEventCodec<TestEvent>(codec);
-    SocketAddress localAddr = new InetSocketAddress(NetUtils.getLocalAddress(), 8000);
-    SocketAddress remoteAddr = new InetSocketAddress(NetUtils.getLocalAddress(), 9000);
+    SocketAddress localAddr = new InetSocketAddress(this.localAddressProvider.getLocalAddress(), 8000);
+    SocketAddress remoteAddr = new InetSocketAddress(this.localAddressProvider.getLocalAddress(), 9000);
 
     RemoteEvent<TestEvent> e1 = new RemoteEvent<TestEvent>(
         localAddr, remoteAddr, "stage1", "stage2", 1, new TestEvent("hello", 0.0));
@@ -77,7 +84,7 @@ public class RemoteTest {
     // receiver stage
     final RemoteReceiverStage reRecvStage = new RemoteReceiverStage(null, null, 10);
 
-    String hostAddress = NetUtils.getLocalAddress();
+    final String hostAddress = this.localAddressProvider.getLocalAddress();
 
     // transport
     Transport transport1 = new NettyMessagingTransport(hostAddress, 0, reRecvStage, reRecvStage, 1, 10000);
@@ -122,7 +129,7 @@ public class RemoteTest {
     final RemoteReceiverStage reRecvStage = new RemoteReceiverStage(
         new RemoteEventHandler(decoder, handler), new LoggingEventHandler<Throwable>(), 10);
 
-    String hostAddress = NetUtils.getLocalAddress();
+    final String hostAddress = this.localAddressProvider.getLocalAddress();
 
     // transport
     Transport transport = new NettyMessagingTransport(hostAddress, port, reRecvStage, reRecvStage, 1, 10000);

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/SmallMessagesTest.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/SmallMessagesTest.java b/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/SmallMessagesTest.java
index 42050d7..fdced04 100644
--- a/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/SmallMessagesTest.java
+++ b/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/SmallMessagesTest.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -22,7 +22,12 @@ import org.apache.reef.wake.EventHandler;
 import org.apache.reef.wake.impl.LoggingUtils;
 import org.apache.reef.wake.impl.MultiEventHandler;
 import org.apache.reef.wake.impl.TimerStage;
-import org.apache.reef.wake.remote.*;
+import org.apache.reef.wake.remote.Decoder;
+import org.apache.reef.wake.remote.Encoder;
+import org.apache.reef.wake.remote.RemoteIdentifier;
+import org.apache.reef.wake.remote.RemoteIdentifierFactory;
+import org.apache.reef.wake.remote.address.LocalAddressProvider;
+import org.apache.reef.wake.remote.address.LocalAddressProviderFactory;
 import org.apache.reef.wake.remote.impl.*;
 import org.apache.reef.wake.remote.transport.Transport;
 import org.apache.reef.wake.remote.transport.netty.NettyMessagingTransport;
@@ -37,6 +42,11 @@ import java.util.*;
 import java.util.logging.Level;
 
 public class SmallMessagesTest {
+  private final LocalAddressProvider localAddressProvider;
+
+  public SmallMessagesTest() {
+    this.localAddressProvider = LocalAddressProviderFactory.getInstance();
+  }
 
   @Rule
   public final TestName name = new TestName();
@@ -74,7 +84,7 @@ public class SmallMessagesTest {
     final RemoteReceiverStage reRecvStage = new RemoteReceiverStage(
         new RemoteEventHandler(decoder, handler), null, 10);
 
-    String hostAddress = NetUtils.getLocalAddress();
+    final String hostAddress = this.localAddressProvider.getLocalAddress();
 
     // transport
     Transport transport = new NettyMessagingTransport(hostAddress, port, reRecvStage, reRecvStage, 1, 10000);

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/TestRemote.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/TestRemote.java b/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/TestRemote.java
index 1ec61ce..e12a517 100644
--- a/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/TestRemote.java
+++ b/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/TestRemote.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -18,23 +18,35 @@
  */
 package org.apache.reef.wake.test.remote;
 
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.exceptions.InjectionException;
 import org.apache.reef.wake.EventHandler;
 import org.apache.reef.wake.impl.LoggingEventHandler;
 import org.apache.reef.wake.remote.*;
+import org.apache.reef.wake.remote.address.LocalAddressProvider;
+import org.apache.reef.wake.remote.address.LocalAddressProviderFactory;
 import org.apache.reef.wake.remote.impl.DefaultRemoteIdentifierFactoryImplementation;
 import org.apache.reef.wake.remote.impl.DefaultRemoteManagerImplementation;
 
+import javax.inject.Inject;
 import java.net.UnknownHostException;
 
-public class TestRemote {
+public class TestRemote implements Runnable {
+  private final LocalAddressProvider localAddressProvider;
 
-  public static void main(String[] args) {
-    String hostAddress = NetUtils.getLocalAddress();
+  @Inject
+  public TestRemote() throws InjectionException {
+    this.localAddressProvider = LocalAddressProviderFactory.getInstance();
+  }
+
+  @Override
+  public void run() {
+    final String hostAddress = localAddressProvider.getLocalAddress();
     int myPort = 10011;
     int remotePort = 10001;
     Codec<TestEvent> codec = new TestEventCodec();
     try (RemoteManager rm = new DefaultRemoteManagerImplementation("name", hostAddress,
-        myPort, codec, new LoggingEventHandler<Throwable>(), false, 1, 10000)) {
+        myPort, codec, new LoggingEventHandler<Throwable>(), false, 1, 10000, localAddressProvider)) {
       // proxy handler
       RemoteIdentifierFactory factory = new DefaultRemoteIdentifierFactoryImplementation();
       RemoteIdentifier remoteId = factory.getNewInstance("socket://" + hostAddress + ":" + remotePort);
@@ -50,6 +62,10 @@ public class TestRemote {
       e.printStackTrace();
     }
   }
+
+  public static void main(String[] args) throws InjectionException {
+    Tang.Factory.getTang().newInjector().getInstance(TestRemote.class).run();
+  }
 }
 
 class TestEventHandler implements EventHandler<RemoteMessage<TestEvent>> {

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/TransportRaceTest.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/TransportRaceTest.java b/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/TransportRaceTest.java
index 7b7cfde..bdb480d 100644
--- a/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/TransportRaceTest.java
+++ b/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/TransportRaceTest.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -24,7 +24,8 @@ import org.apache.reef.wake.impl.LoggingEventHandler;
 import org.apache.reef.wake.impl.LoggingUtils;
 import org.apache.reef.wake.impl.ThreadPoolStage;
 import org.apache.reef.wake.impl.TimerStage;
-import org.apache.reef.wake.remote.NetUtils;
+import org.apache.reef.wake.remote.address.LocalAddressProvider;
+import org.apache.reef.wake.remote.address.LocalAddressProviderFactory;
 import org.apache.reef.wake.remote.impl.TransportEvent;
 import org.apache.reef.wake.remote.transport.Link;
 import org.apache.reef.wake.remote.transport.netty.NettyMessagingTransport;
@@ -39,6 +40,11 @@ import java.util.logging.Level;
 
 
 public class TransportRaceTest {
+  private final LocalAddressProvider localAddressProvider;
+
+  public TransportRaceTest() {
+    this.localAddressProvider = LocalAddressProviderFactory.getInstance();
+  }
 
   @Test
   public void testRace() throws Exception {
@@ -52,7 +58,7 @@ public class TransportRaceTest {
     final ServerHandler serverHandler = new ServerHandler(monitor, msgsSent);
     EStage<TransportEvent> serverStage = new ThreadPoolStage<>("server@7001",
         serverHandler, 1, new LoggingEventHandler<Throwable>());
-    String hostAddress = NetUtils.getLocalAddress();
+    final String hostAddress = this.localAddressProvider.getLocalAddress();
     int port = 7001;
     NettyMessagingTransport transport = new NettyMessagingTransport(
         hostAddress, port, clientStage, serverStage, 1, 10000);

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/TransportTest.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/TransportTest.java b/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/TransportTest.java
index 9c1d045..8176a70 100644
--- a/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/TransportTest.java
+++ b/lang/java/reef-wake/wake/src/test/java/org/apache/reef/wake/test/remote/TransportTest.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -22,7 +22,8 @@ import org.apache.reef.wake.EStage;
 import org.apache.reef.wake.impl.LoggingUtils;
 import org.apache.reef.wake.impl.TimerStage;
 import org.apache.reef.wake.remote.Codec;
-import org.apache.reef.wake.remote.NetUtils;
+import org.apache.reef.wake.remote.address.LocalAddressProvider;
+import org.apache.reef.wake.remote.address.LocalAddressProviderFactory;
 import org.apache.reef.wake.remote.impl.ObjectSerializableCodec;
 import org.apache.reef.wake.remote.impl.TransportEvent;
 import org.apache.reef.wake.remote.transport.Link;
@@ -42,6 +43,11 @@ import java.util.logging.Level;
 
 
 public class TransportTest {
+  private final LocalAddressProvider localAddressProvider;
+
+  public TransportTest() {
+    this.localAddressProvider = LocalAddressProviderFactory.getInstance();
+  }
 
   final String logPrefix = "TEST ";
   @Rule
@@ -56,7 +62,8 @@ public class TransportTest {
     TimerStage timer = new TimerStage(new TimeoutHandler(monitor), 2000, 2000);
 
     final int expected = 2;
-    final String hostAddress = NetUtils.getLocalAddress();
+    final String hostAddress = this.localAddressProvider.getLocalAddress();
+    ;
     final int port = 9100;
 
     // Codec<String>
@@ -87,7 +94,7 @@ public class TransportTest {
     TimerStage timer = new TimerStage(new TimeoutHandler(monitor), 2000, 2000);
 
     final int expected = 2;
-    final String hostAddress = NetUtils.getLocalAddress();
+    final String hostAddress = this.localAddressProvider.getLocalAddress();
     final int port = 9100;
 
     // Codec<TestEvent>


[2/2] incubator-reef git commit: [REEF-246] Add LocalAddressProvider

Posted by se...@apache.org.
[REEF-246] Add LocalAddressProvider

This PR adds the concept of a `LocalAddressProvider` that provides the
address which network services shall bind to. This new interface
replaces `NetUtils.getLocalAddress()`.

This adds two implementations for this new interface:

  * `LegacyLocalAddressProvider` which uses the code found prior in
    `NetUtils.getLocalAddress()`.
  * `HostnameBasedLocalAddressProvider`, which uses the same code Apache
    Hadoop MapReduce uses for this.

The default implementation for `LocalAddressProvider` is
`HostnameBasedLocalAddressProvider`. `NetUtils.getLocalAddress()` has
been deprecated and uses the default implementation internally.

The remainder of the changes are dealing with the fallout of this
change. In fact, this is the majority of the changes. Most notably,
this removes the `bindVolatile` on the clock injector in `LaunchClass`.
This means that we now potentially instantiate two `RemoteManager`s.
[REEF-269] has been filled to track the cleanup.

JIRA:
  [REEF-246](https://issues.apache.org/jira/browse/REEF-246)

Pull Request:
  This closes #155


Project: http://git-wip-us.apache.org/repos/asf/incubator-reef/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-reef/commit/a72a40f8
Tree: http://git-wip-us.apache.org/repos/asf/incubator-reef/tree/a72a40f8
Diff: http://git-wip-us.apache.org/repos/asf/incubator-reef/diff/a72a40f8

Branch: refs/heads/master
Commit: a72a40f8d23231e0b07a0693d088b5e3cf6c3248
Parents: ac380af
Author: Markus Weimer <we...@apache.org>
Authored: Fri Apr 17 10:45:29 2015 -0700
Committer: Beysim Sezgin <be...@microsoft.com>
Committed: Mon Apr 20 18:12:44 2015 -0700

----------------------------------------------------------------------
 .../reef/javabridge/generic/JobDriver.java      |  13 +-
 .../client/CommonClientConfigurationModule.java |   7 +-
 .../reef/runtime/common/launch/LaunchClass.java |  10 +-
 .../group/impl/driver/GroupCommDriverImpl.java  |  31 +++--
 .../network/impl/MessagingTransportFactory.java |  22 +++-
 .../reef/io/network/impl/NetworkService.java    |  65 +++++++++-
 .../reef/io/network/naming/NameClient.java      |  61 +++++++--
 .../io/network/naming/NameLookupClient.java     |  65 ++++++++--
 .../io/network/naming/NameRegistryClient.java   |  32 +++--
 .../reef/io/network/naming/NameServerImpl.java  |  41 ++++--
 .../reef/services/network/NameClientTest.java   |  29 +++--
 .../reef/services/network/NamingTest.java       |  55 ++++----
 .../services/network/NetworkServiceTest.java    | 112 +++++++++--------
 .../local/client/LocalRuntimeConfiguration.java |  14 ++-
 .../runtime/local/driver/ContainerManager.java  |  11 +-
 .../runtime/mesos/util/MesosRemoteManager.java  |  10 +-
 .../yarn/client/YarnClientConfiguration.java    |  10 ++
 .../org/apache/reef/wake/remote/NetUtils.java   |  98 +++------------
 .../HostnameBasedLocalAddressProvider.java      |  66 ++++++++++
 .../address/LegacyLocalAddressProvider.java     | 126 +++++++++++++++++++
 .../remote/address/LocalAddressProvider.java    |  34 +++++
 .../address/LocalAddressProviderFactory.java    |  69 ++++++++++
 .../reef/wake/remote/address/package-info.java  |  22 ++++
 .../DefaultRemoteManagerImplementation.java     |  43 ++++++-
 .../reef/wake/test/remote/LargeMsgTest.java     |  15 ++-
 .../remote/RemoteIdentifierFactoryTest.java     |  38 +++---
 .../wake/test/remote/RemoteManagerTest.java     |  33 +++--
 .../reef/wake/test/remote/RemoteTest.java       |  19 ++-
 .../wake/test/remote/SmallMessagesTest.java     |  16 ++-
 .../reef/wake/test/remote/TestRemote.java       |  26 +++-
 .../wake/test/remote/TransportRaceTest.java     |  12 +-
 .../reef/wake/test/remote/TransportTest.java    |  15 ++-
 32 files changed, 912 insertions(+), 308 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-bridge-java/src/main/java/org/apache/reef/javabridge/generic/JobDriver.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-java/src/main/java/org/apache/reef/javabridge/generic/JobDriver.java b/lang/java/reef-bridge-java/src/main/java/org/apache/reef/javabridge/generic/JobDriver.java
index f7adb53..44077f0 100644
--- a/lang/java/reef-bridge-java/src/main/java/org/apache/reef/javabridge/generic/JobDriver.java
+++ b/lang/java/reef-bridge-java/src/main/java/org/apache/reef/javabridge/generic/JobDriver.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -35,7 +35,7 @@ import org.apache.reef.util.logging.CLRBufferedLogHandler;
 import org.apache.reef.util.logging.LoggingScope;
 import org.apache.reef.util.logging.LoggingScopeFactory;
 import org.apache.reef.wake.EventHandler;
-import org.apache.reef.wake.remote.NetUtils;
+import org.apache.reef.wake.remote.address.LocalAddressProvider;
 import org.apache.reef.wake.remote.impl.ObjectSerializableCodec;
 import org.apache.reef.wake.time.Clock;
 import org.apache.reef.wake.time.event.Alarm;
@@ -146,14 +146,15 @@ public final class JobDriver {
             final EvaluatorRequestor evaluatorRequestor,
             final DriverStatusManager driverStatusManager,
             final LoggingScopeFactory loggingScopeFactory,
-            final LibLoader libLoader) {
+            final LibLoader libLoader,
+            final LocalAddressProvider localAddressProvider) {
     this.clock = clock;
     this.httpServer = httpServer;
     this.jobMessageObserver = jobMessageObserver;
     this.evaluatorRequestor = evaluatorRequestor;
     this.nameServer = nameServer;
     this.driverStatusManager = driverStatusManager;
-    this.nameServerInfo = NetUtils.getLocalAddress() + ":" + this.nameServer.getPort();
+    this.nameServerInfo = localAddressProvider.getLocalAddress() + ":" + this.nameServer.getPort();
     this.loggingScopeFactory = loggingScopeFactory;
     this.libLoader = libLoader;
   }
@@ -274,7 +275,7 @@ public final class JobDriver {
       try (final LoggingScope ls = loggingScopeFactory.evaluatorAllocated(allocatedEvaluator.getId())) {
         synchronized (JobDriver.this) {
           LOG.log(Level.INFO, "AllocatedEvaluatorHandler.OnNext");
-            JobDriver.this.submitEvaluator(allocatedEvaluator, EvaluatorType.CLR);
+          JobDriver.this.submitEvaluator(allocatedEvaluator, EvaluatorType.CLR);
         }
       }
     }
@@ -504,7 +505,7 @@ public final class JobDriver {
     public void onNext(final ActiveContext context) {
       try (final LoggingScope ls = loggingScopeFactory.driverRestartActiveContextReceived(context.getId())) {
         JobDriver.this.contexts.put(context.getId(), context);
-      LOG.log(Level.INFO, "DriverRestartActiveContextHandler event received: " + context.getId());
+        LOG.log(Level.INFO, "DriverRestartActiveContextHandler event received: " + context.getId());
         clock.scheduleAlarm(0, new EventHandler<Alarm>() {
           @Override
           public void onNext(final Alarm time) {

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/client/CommonClientConfigurationModule.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/client/CommonClientConfigurationModule.java b/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/client/CommonClientConfigurationModule.java
index 59892e9..4ab54b9 100644
--- a/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/client/CommonClientConfigurationModule.java
+++ b/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/client/CommonClientConfigurationModule.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -21,7 +21,10 @@ package org.apache.reef.runtime.common.client;
 import org.apache.reef.tang.formats.ConfigurationModule;
 import org.apache.reef.tang.formats.ConfigurationModuleBuilder;
 
-
+/**
+ * @deprecated this class was never used.
+ */
+@Deprecated
 public class CommonClientConfigurationModule extends ConfigurationModuleBuilder {
   public final static ConfigurationModule CONF = new CommonClientConfigurationModule()
       .build();

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/launch/LaunchClass.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/launch/LaunchClass.java b/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/launch/LaunchClass.java
index 8c40439..dfe9882 100644
--- a/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/launch/LaunchClass.java
+++ b/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/launch/LaunchClass.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -22,7 +22,6 @@ import org.apache.reef.runtime.common.evaluator.PIDStoreStartHandler;
 import org.apache.reef.runtime.common.launch.parameters.ClockConfigurationPath;
 import org.apache.reef.runtime.common.launch.parameters.ErrorHandlerRID;
 import org.apache.reef.runtime.common.launch.parameters.LaunchID;
-import org.apache.reef.runtime.common.utils.RemoteManager;
 import org.apache.reef.tang.*;
 import org.apache.reef.tang.annotations.Name;
 import org.apache.reef.tang.annotations.NamedParameter;
@@ -46,7 +45,6 @@ import java.util.logging.Logger;
 public final class LaunchClass implements AutoCloseable, Runnable {
 
   private static final Logger LOG = Logger.getLogger(LaunchClass.class.getName());
-  private final RemoteManager remoteManager;
   private final String launchID;
   private final String errorHandlerID;
   private final String evaluatorConfigurationPath;
@@ -56,8 +54,7 @@ public final class LaunchClass implements AutoCloseable, Runnable {
   private WakeProfiler profiler;
 
   @Inject
-  LaunchClass(final RemoteManager remoteManager,
-              final REEFUncaughtExceptionHandler uncaughtExceptionHandler,
+  LaunchClass(final REEFUncaughtExceptionHandler uncaughtExceptionHandler,
               final REEFErrorHandler errorHandler,
               final @Parameter(LaunchID.class) String launchID,
               final @Parameter(ErrorHandlerRID.class) String errorHandlerID,
@@ -66,7 +63,6 @@ public final class LaunchClass implements AutoCloseable, Runnable {
               final ConfigurationSerializer configurationSerializer,
               final REEFVersion reefVersion) {
     reefVersion.logVersion();
-    this.remoteManager = remoteManager;
     this.launchID = launchID;
     this.errorHandlerID = errorHandlerID;
     this.evaluatorConfigurationPath = evaluatorConfigurationPath;
@@ -83,6 +79,7 @@ public final class LaunchClass implements AutoCloseable, Runnable {
       this.profiler = new WakeProfiler();
       ProfilingStopHandler.setProfiler(profiler); // TODO: This probably should be bound via Tang.
     }
+    LOG.log(Level.FINE, "Instantiated LaunchClass");
   }
 
   /**
@@ -146,7 +143,6 @@ public final class LaunchClass implements AutoCloseable, Runnable {
       if (isProfilingEnabled) {
         clockInjector.bindAspect(profiler);
       }
-      clockInjector.bindVolatileInstance(RemoteManager.class, this.remoteManager);
       return clockInjector.getInstance(Clock.class);
     } catch (final Throwable ex) {
       fail("Unable to instantiate the clock", ex);

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/GroupCommDriverImpl.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/GroupCommDriverImpl.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/GroupCommDriverImpl.java
index 8c01b31..8965553 100644
--- a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/GroupCommDriverImpl.java
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/GroupCommDriverImpl.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -54,7 +54,8 @@ import org.apache.reef.wake.impl.LoggingEventHandler;
 import org.apache.reef.wake.impl.SingleThreadStage;
 import org.apache.reef.wake.impl.SyncStage;
 import org.apache.reef.wake.impl.ThreadPoolStage;
-import org.apache.reef.wake.remote.NetUtils;
+import org.apache.reef.wake.remote.address.LocalAddressProvider;
+import org.apache.reef.wake.remote.address.LocalAddressProviderFactory;
 
 import javax.inject.Inject;
 import java.util.HashMap;
@@ -79,7 +80,7 @@ public class GroupCommDriverImpl implements GroupCommServiceDriver {
 
   private final IdentifierFactory idFac = new StringIdentifierFactory();
 
-  private final NameServer nameService = new NameServerImpl(0, idFac);
+  private final NameServer nameService;
 
   private final String nameServiceAddr;
   private final int nameServicePort;
@@ -103,14 +104,30 @@ public class GroupCommDriverImpl implements GroupCommServiceDriver {
   private final EStage<GroupCommunicationMessage> groupCommMessageStage;
   private final int fanOut;
 
-  @Inject
+  /**
+   * @deprecated Have an instance injected instead.
+   */
+  @Deprecated
   public GroupCommDriverImpl(final ConfigurationSerializer confSerializer,
                              @Parameter(DriverIdentifier.class) final String driverId,
                              @Parameter(TreeTopologyFanOut.class) final int fanOut) {
+    this(confSerializer, driverId, fanOut, LocalAddressProviderFactory.getInstance());
+  }
+
+  /**
+   * @deprecated Have an instance injected instead.
+   */
+  @Deprecated
+  @Inject
+  public GroupCommDriverImpl(final ConfigurationSerializer confSerializer,
+                             @Parameter(DriverIdentifier.class) final String driverId,
+                             @Parameter(TreeTopologyFanOut.class) final int fanOut,
+                             final LocalAddressProvider localAddressProvider) {
     assert (SingletonAsserter.assertSingleton(getClass()));
     this.driverId = driverId;
     this.fanOut = fanOut;
-    this.nameServiceAddr = NetUtils.getLocalAddress();
+    this.nameService = new NameServerImpl(0, idFac, localAddressProvider);
+    this.nameServiceAddr = localAddressProvider.getLocalAddress();
     this.nameServicePort = nameService.getPort();
     this.confSerializer = confSerializer;
     this.groupCommRunningTaskHandler = new BroadcastingEventHandler<>();
@@ -123,14 +140,14 @@ public class GroupCommDriverImpl implements GroupCommServiceDriver {
     this.groupCommMessageHandler = new GroupCommMessageHandler();
     this.groupCommMessageStage = new SingleThreadStage<>("GroupCommMessageStage", groupCommMessageHandler, 100 * 1000);
     this.netService = new NetworkService<>(idFac, 0, nameServiceAddr, nameServicePort,
-        new GroupCommunicationMessageCodec(), new MessagingTransportFactory(),
+        new GroupCommunicationMessageCodec(), new MessagingTransportFactory(localAddressProvider),
         new EventHandler<Message<GroupCommunicationMessage>>() {
 
           @Override
           public void onNext(final Message<GroupCommunicationMessage> msg) {
             groupCommMessageStage.onNext(Utils.getGCM(msg));
           }
-        }, new LoggingEventHandler<Exception>());
+        }, new LoggingEventHandler<Exception>(), localAddressProvider);
     this.netService.registerId(idFac.getNewInstance(driverId));
     this.senderStage = new ThreadPoolStage<>("SrcCtrlMsgSender", new CtrlMsgSender(idFac, netService), 5);
   }

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-io/src/main/java/org/apache/reef/io/network/impl/MessagingTransportFactory.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/impl/MessagingTransportFactory.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/impl/MessagingTransportFactory.java
index e169624..82fd720 100644
--- a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/impl/MessagingTransportFactory.java
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/impl/MessagingTransportFactory.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -21,7 +21,8 @@ package org.apache.reef.io.network.impl;
 import org.apache.reef.io.network.TransportFactory;
 import org.apache.reef.wake.EventHandler;
 import org.apache.reef.wake.impl.SyncStage;
-import org.apache.reef.wake.remote.NetUtils;
+import org.apache.reef.wake.remote.address.LocalAddressProvider;
+import org.apache.reef.wake.remote.address.LocalAddressProviderFactory;
 import org.apache.reef.wake.remote.impl.TransportEvent;
 import org.apache.reef.wake.remote.transport.Transport;
 import org.apache.reef.wake.remote.transport.netty.NettyMessagingTransport;
@@ -33,8 +34,23 @@ import javax.inject.Inject;
  */
 public class MessagingTransportFactory implements TransportFactory {
 
+  private final String localAddress;
+
+  /**
+   * @deprecated Have an instance injected instead.
+   */
+  @Deprecated
   @Inject
+  public MessagingTransportFactory(final LocalAddressProvider localAddressProvider) {
+    this.localAddress = localAddressProvider.getLocalAddress();
+  }
+
+  /**
+   * @deprecated Have an instance injected instead.
+   */
+  @Deprecated
   public MessagingTransportFactory() {
+    this.localAddress = LocalAddressProviderFactory.getInstance().getLocalAddress();
   }
 
   /**
@@ -51,7 +67,7 @@ public class MessagingTransportFactory implements TransportFactory {
                           final EventHandler<TransportEvent> serverHandler,
                           final EventHandler<Exception> exHandler) {
 
-    final Transport transport = new NettyMessagingTransport(NetUtils.getLocalAddress(),
+    final Transport transport = new NettyMessagingTransport(this.localAddress,
         port, new SyncStage<>(clientHandler), new SyncStage<>(serverHandler), 3, 10000);
 
     transport.registerErrorHandler(exHandler);

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-io/src/main/java/org/apache/reef/io/network/impl/NetworkService.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/impl/NetworkService.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/impl/NetworkService.java
index 37d6128..6deb7cb 100644
--- a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/impl/NetworkService.java
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/impl/NetworkService.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -36,14 +36,14 @@ import org.apache.reef.wake.*;
 import org.apache.reef.wake.impl.LoggingEventHandler;
 import org.apache.reef.wake.impl.SingleThreadStage;
 import org.apache.reef.wake.remote.Codec;
+import org.apache.reef.wake.remote.address.LocalAddressProvider;
+import org.apache.reef.wake.remote.address.LocalAddressProviderFactory;
 import org.apache.reef.wake.remote.impl.TransportEvent;
-import org.apache.reef.wake.remote.transport.LinkListener;
 import org.apache.reef.wake.remote.transport.Transport;
 import org.apache.reef.wake.remote.transport.netty.LoggingLinkListener;
 
 import javax.inject.Inject;
 import java.net.InetSocketAddress;
-import java.net.SocketAddress;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.logging.Level;
@@ -80,6 +80,35 @@ public final class NetworkService<T> implements Stage, ConnectionFactory<T> {
   private final EStage<Identifier> nameServiceUnregisteringStage;
   private Identifier myId;
 
+  /**
+   * @param factory
+   * @param nsPort
+   * @param nameServerAddr
+   * @param nameServerPort
+   * @param codec
+   * @param tpFactory
+   * @param recvHandler
+   * @param exHandler
+   * @deprecated have an instance injected instead.
+   */
+  @Deprecated
+  public NetworkService(final IdentifierFactory factory,
+                        final int nsPort,
+                        final String nameServerAddr,
+                        final int nameServerPort,
+                        final Codec<T> codec,
+                        final TransportFactory tpFactory,
+                        final EventHandler<Message<T>> recvHandler,
+                        final EventHandler<Exception> exHandler,
+                        final LocalAddressProvider localAddressProvider) {
+    this(factory, nsPort, nameServerAddr, nameServerPort,
+        retryCount, retryTimeout, codec, tpFactory, recvHandler, exHandler, localAddressProvider);
+  }
+
+  /**
+   * @deprecated have an instance injected instead.
+   */
+  @Deprecated
   public NetworkService(final IdentifierFactory factory,
                         final int nsPort,
                         final String nameServerAddr,
@@ -89,10 +118,13 @@ public final class NetworkService<T> implements Stage, ConnectionFactory<T> {
                         final EventHandler<Message<T>> recvHandler,
                         final EventHandler<Exception> exHandler) {
     this(factory, nsPort, nameServerAddr, nameServerPort,
-        retryCount, retryTimeout, codec, tpFactory, recvHandler, exHandler);
+        retryCount, retryTimeout, codec, tpFactory, recvHandler, exHandler, LocalAddressProviderFactory.getInstance());
   }
 
-  @Inject
+  /**
+   * @deprecated have an instance injected instead.
+   */
+  @Deprecated
   public NetworkService(
       final @Parameter(NetworkServiceParameters.NetworkServiceIdentifierFactory.class) IdentifierFactory factory,
       final @Parameter(NetworkServiceParameters.NetworkServicePort.class) int nsPort,
@@ -104,6 +136,27 @@ public final class NetworkService<T> implements Stage, ConnectionFactory<T> {
       final @Parameter(NetworkServiceParameters.NetworkServiceTransportFactory.class) TransportFactory tpFactory,
       final @Parameter(NetworkServiceParameters.NetworkServiceHandler.class) EventHandler<Message<T>> recvHandler,
       final @Parameter(NetworkServiceParameters.NetworkServiceExceptionHandler.class) EventHandler<Exception> exHandler) {
+    this(factory, nsPort, nameServerAddr, nameServerPort, retryCount, retryTimeout, codec, tpFactory, recvHandler, exHandler,
+        LocalAddressProviderFactory.getInstance());
+  }
+
+  /**
+   * @deprecated have an instance injected instead.
+   */
+  @Deprecated
+  @Inject
+  public NetworkService(
+      final @Parameter(NetworkServiceParameters.NetworkServiceIdentifierFactory.class) IdentifierFactory factory,
+      final @Parameter(NetworkServiceParameters.NetworkServicePort.class) int nsPort,
+      final @Parameter(NameServerParameters.NameServerAddr.class) String nameServerAddr,
+      final @Parameter(NameServerParameters.NameServerPort.class) int nameServerPort,
+      final @Parameter(NameLookupClient.RetryCount.class) int retryCount,
+      final @Parameter(NameLookupClient.RetryTimeout.class) int retryTimeout,
+      final @Parameter(NetworkServiceParameters.NetworkServiceCodec.class) Codec<T> codec,
+      final @Parameter(NetworkServiceParameters.NetworkServiceTransportFactory.class) TransportFactory tpFactory,
+      final @Parameter(NetworkServiceParameters.NetworkServiceHandler.class) EventHandler<Message<T>> recvHandler,
+      final @Parameter(NetworkServiceParameters.NetworkServiceExceptionHandler.class) EventHandler<Exception> exHandler,
+      final LocalAddressProvider localAddressProvider) {
 
     this.factory = factory;
     this.codec = codec;
@@ -112,7 +165,7 @@ public final class NetworkService<T> implements Stage, ConnectionFactory<T> {
         new MessageHandler<T>(recvHandler, codec, factory), exHandler);
 
     this.nameClient = new NameClient(nameServerAddr, nameServerPort,
-        factory, retryCount, retryTimeout, new NameCache(30000));
+        factory, retryCount, retryTimeout, new NameCache(30000), localAddressProvider);
 
     this.nameServiceRegisteringStage = new SingleThreadStage<>(
         "NameServiceRegisterer", new EventHandler<Tuple<Identifier, InetSocketAddress>>() {

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-io/src/main/java/org/apache/reef/io/network/naming/NameClient.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/naming/NameClient.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/naming/NameClient.java
index 966ac94..52688aa 100644
--- a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/naming/NameClient.java
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/naming/NameClient.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -30,7 +30,8 @@ import org.apache.reef.wake.IdentifierFactory;
 import org.apache.reef.wake.Stage;
 import org.apache.reef.wake.impl.SyncStage;
 import org.apache.reef.wake.remote.Codec;
-import org.apache.reef.wake.remote.NetUtils;
+import org.apache.reef.wake.remote.address.LocalAddressProvider;
+import org.apache.reef.wake.remote.address.LocalAddressProviderFactory;
 import org.apache.reef.wake.remote.impl.TransportEvent;
 import org.apache.reef.wake.remote.transport.Transport;
 import org.apache.reef.wake.remote.transport.netty.NettyMessagingTransport;
@@ -52,6 +53,17 @@ public class NameClient implements Stage, Naming {
   private NameRegistryClient registryClient;
   private Transport transport;
 
+
+  @Deprecated
+  public NameClient(final String serverAddr,
+                    final int serverPort,
+                    final IdentifierFactory factory,
+                    final int retryCount,
+                    final int retryTimeout,
+                    final Cache<Identifier, InetSocketAddress> cache) {
+    this(serverAddr, serverPort, 10000, factory, retryCount, retryTimeout, cache, LocalAddressProviderFactory.getInstance());
+  }
+
   /**
    * Constructs a naming client
    *
@@ -60,10 +72,32 @@ public class NameClient implements Stage, Naming {
    * @param factory    an identifier factory
    * @param cache      a cache
    */
-  public NameClient(String serverAddr, int serverPort,
-                    IdentifierFactory factory, int retryCount, int retryTimeout,
-                    Cache<Identifier, InetSocketAddress> cache) {
-    this(serverAddr, serverPort, 10000, factory, retryCount, retryTimeout, cache);
+  public NameClient(final String serverAddr,
+                    final int serverPort,
+                    final IdentifierFactory factory,
+                    final int retryCount,
+                    final int retryTimeout,
+                    final Cache<Identifier, InetSocketAddress> cache,
+                    final LocalAddressProvider localAddressProvider) {
+    this(serverAddr, serverPort, 10000, factory, retryCount, retryTimeout, cache, localAddressProvider);
+  }
+
+  @Deprecated
+  public NameClient(final String serverAddr,
+                    final int serverPort,
+                    final long timeout,
+                    final IdentifierFactory factory,
+                    final int retryCount,
+                    final int retryTimeout,
+                    final Cache<Identifier, InetSocketAddress> cache) {
+    this(serverAddr,
+        serverPort,
+        timeout,
+        factory,
+        retryCount,
+        retryTimeout,
+        cache,
+        LocalAddressProviderFactory.getInstance());
   }
 
   /**
@@ -75,15 +109,20 @@ public class NameClient implements Stage, Naming {
    * @param factory    an identifier factory
    * @param cache      a cache
    */
-  public NameClient(final String serverAddr, final int serverPort, final long timeout,
-                    final IdentifierFactory factory, final int retryCount, final int retryTimeout,
-                    final Cache<Identifier, InetSocketAddress> cache) {
+  public NameClient(final String serverAddr,
+                    final int serverPort,
+                    final long timeout,
+                    final IdentifierFactory factory,
+                    final int retryCount,
+                    final int retryTimeout,
+                    final Cache<Identifier, InetSocketAddress> cache,
+                    final LocalAddressProvider localAddressProvider) {
 
     final BlockingQueue<NamingLookupResponse> replyLookupQueue = new LinkedBlockingQueue<NamingLookupResponse>();
     final BlockingQueue<NamingRegisterResponse> replyRegisterQueue = new LinkedBlockingQueue<NamingRegisterResponse>();
     final Codec<NamingMessage> codec = NamingCodecFactory.createFullCodec(factory);
 
-    this.transport = new NettyMessagingTransport(NetUtils.getLocalAddress(), 0,
+    this.transport = new NettyMessagingTransport(localAddressProvider.getLocalAddress(), 0,
         new SyncStage<>(new NamingClientEventHandler(
             new NamingResponseHandler(replyLookupQueue, replyRegisterQueue), codec)),
         null, retryCount, retryTimeout);
@@ -104,7 +143,7 @@ public class NameClient implements Stage, Naming {
   @Override
   public void register(final Identifier id, final InetSocketAddress addr)
       throws Exception {
-    LOG.log(Level.FINE, "Refister {0} : {1}", new Object[]{id, addr});
+    LOG.log(Level.FINE, "Register {0} : {1}", new Object[]{id, addr});
     this.registryClient.register(id, addr);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-io/src/main/java/org/apache/reef/io/network/naming/NameLookupClient.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/naming/NameLookupClient.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/naming/NameLookupClient.java
index 3a19b99..a402ebb 100644
--- a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/naming/NameLookupClient.java
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/naming/NameLookupClient.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -33,7 +33,8 @@ import org.apache.reef.wake.IdentifierFactory;
 import org.apache.reef.wake.Stage;
 import org.apache.reef.wake.impl.SyncStage;
 import org.apache.reef.wake.remote.Codec;
-import org.apache.reef.wake.remote.NetUtils;
+import org.apache.reef.wake.remote.address.LocalAddressProvider;
+import org.apache.reef.wake.remote.address.LocalAddressProviderFactory;
 import org.apache.reef.wake.remote.impl.TransportEvent;
 import org.apache.reef.wake.remote.transport.Link;
 import org.apache.reef.wake.remote.transport.Transport;
@@ -74,10 +75,51 @@ public class NameLookupClient implements Stage, NamingLookup {
    * @param factory    an identifier factory
    * @param cache      an cache
    */
-  public NameLookupClient(final String serverAddr, final int serverPort,
-                          final IdentifierFactory factory, final int retryCount, final int retryTimeout,
+  public NameLookupClient(final String serverAddr,
+                          final int serverPort,
+                          final IdentifierFactory factory,
+                          final int retryCount,
+                          final int retryTimeout,
+                          final Cache<Identifier, InetSocketAddress> cache,
+                          final LocalAddressProvider localAddressProvider) {
+    this(serverAddr, serverPort, 10000, factory, retryCount, retryTimeout, cache, localAddressProvider);
+  }
+
+  /**
+   * Constructs a naming lookup client
+   *
+   * @param serverAddr a server address
+   * @param serverPort a server port number
+   * @param factory    an identifier factory
+   * @param cache      an cache
+   */
+  @Deprecated
+  public NameLookupClient(final String serverAddr,
+                          final int serverPort,
+                          final IdentifierFactory factory,
+                          final int retryCount,
+                          final int retryTimeout,
+                          final Cache<Identifier, InetSocketAddress> cache) {
+    this(serverAddr, serverPort, 10000, factory, retryCount, retryTimeout, cache, LocalAddressProviderFactory.getInstance());
+  }
+
+  @Deprecated
+  public NameLookupClient(final String serverAddr,
+                          final int serverPort,
+                          final long timeout,
+                          final IdentifierFactory factory,
+                          final int retryCount,
+                          final int retryTimeout,
                           final Cache<Identifier, InetSocketAddress> cache) {
-    this(serverAddr, serverPort, 10000, factory, retryCount, retryTimeout, cache);
+    this(serverAddr,
+        serverPort,
+        timeout,
+        factory,
+        retryCount,
+        retryTimeout,
+        cache,
+        LocalAddressProviderFactory.getInstance());
+
   }
 
   /**
@@ -89,9 +131,14 @@ public class NameLookupClient implements Stage, NamingLookup {
    * @param factory    an identifier factory
    * @param cache      an cache
    */
-  public NameLookupClient(final String serverAddr, final int serverPort, final long timeout,
-                          final IdentifierFactory factory, final int retryCount, final int retryTimeout,
-                          final Cache<Identifier, InetSocketAddress> cache) {
+  public NameLookupClient(final String serverAddr,
+                          final int serverPort,
+                          final long timeout,
+                          final IdentifierFactory factory,
+                          final int retryCount,
+                          final int retryTimeout,
+                          final Cache<Identifier, InetSocketAddress> cache,
+                          final LocalAddressProvider localAddressProvider) {
 
     this.serverSocketAddr = new InetSocketAddress(serverAddr, serverPort);
     this.timeout = timeout;
@@ -99,7 +146,7 @@ public class NameLookupClient implements Stage, NamingLookup {
     this.codec = NamingCodecFactory.createLookupCodec(factory);
     this.replyQueue = new LinkedBlockingQueue<>();
 
-    this.transport = new NettyMessagingTransport(NetUtils.getLocalAddress(), 0,
+    this.transport = new NettyMessagingTransport(localAddressProvider.getLocalAddress(), 0,
         new SyncStage<>(new NamingLookupClientHandler(
             new NamingLookupResponseHandler(this.replyQueue), this.codec)),
         null, retryCount, retryTimeout);

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-io/src/main/java/org/apache/reef/io/network/naming/NameRegistryClient.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/naming/NameRegistryClient.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/naming/NameRegistryClient.java
index 938dab8..ba48ed7 100644
--- a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/naming/NameRegistryClient.java
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/naming/NameRegistryClient.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -30,10 +30,10 @@ import org.apache.reef.wake.IdentifierFactory;
 import org.apache.reef.wake.Stage;
 import org.apache.reef.wake.impl.SyncStage;
 import org.apache.reef.wake.remote.Codec;
-import org.apache.reef.wake.remote.NetUtils;
+import org.apache.reef.wake.remote.address.LocalAddressProvider;
+import org.apache.reef.wake.remote.address.LocalAddressProviderFactory;
 import org.apache.reef.wake.remote.impl.TransportEvent;
 import org.apache.reef.wake.remote.transport.Link;
-import org.apache.reef.wake.remote.transport.LinkListener;
 import org.apache.reef.wake.remote.transport.Transport;
 import org.apache.reef.wake.remote.transport.netty.LoggingLinkListener;
 import org.apache.reef.wake.remote.transport.netty.NettyMessagingTransport;
@@ -68,8 +68,8 @@ public class NameRegistryClient implements Stage, NamingRegistry {
    * @param factory    an identifier factory
    */
   public NameRegistryClient(
-      final String serverAddr, final int serverPort, final IdentifierFactory factory) {
-    this(serverAddr, serverPort, 10000, factory);
+      final String serverAddr, final int serverPort, final IdentifierFactory factory, final LocalAddressProvider localAddressProvider) {
+    this(serverAddr, serverPort, 10000, factory, localAddressProvider);
   }
 
   /**
@@ -80,18 +80,34 @@ public class NameRegistryClient implements Stage, NamingRegistry {
    * @param timeout    timeout in ms
    * @param factory    an identifier factory
    */
-  public NameRegistryClient(final String serverAddr, final int serverPort,
-                            final long timeout, final IdentifierFactory factory) {
+  public NameRegistryClient(final String serverAddr,
+                            final int serverPort,
+                            final long timeout,
+                            final IdentifierFactory factory,
+                            final LocalAddressProvider localAddressProvider) {
 
     this.serverSocketAddr = new InetSocketAddress(serverAddr, serverPort);
     this.timeout = timeout;
     this.codec = NamingCodecFactory.createRegistryCodec(factory);
     this.replyQueue = new LinkedBlockingQueue<>();
-    this.transport = new NettyMessagingTransport(NetUtils.getLocalAddress(), 0,
+    this.transport = new NettyMessagingTransport(localAddressProvider.getLocalAddress(), 0,
         new SyncStage<>(new NamingRegistryClientHandler(new NamingRegistryResponseHandler(replyQueue), codec)),
         null, 3, 10000);
   }
 
+  @Deprecated
+  public NameRegistryClient(final String serverAddr,
+                            final int serverPort,
+                            final long timeout,
+                            final IdentifierFactory factory) {
+
+    this(serverAddr,
+        serverPort,
+        timeout,
+        factory,
+        LocalAddressProviderFactory.getInstance());
+  }
+
   public NameRegistryClient(final String serverAddr, final int serverPort,
                             final long timeout, final IdentifierFactory factory,
                             final BlockingQueue<NamingRegisterResponse> replyQueue,

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-io/src/main/java/org/apache/reef/io/network/naming/NameServerImpl.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/naming/NameServerImpl.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/naming/NameServerImpl.java
index ff241a0..d9384aa 100644
--- a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/naming/NameServerImpl.java
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/naming/NameServerImpl.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -27,7 +27,8 @@ import org.apache.reef.wake.IdentifierFactory;
 import org.apache.reef.wake.impl.MultiEventHandler;
 import org.apache.reef.wake.impl.SyncStage;
 import org.apache.reef.wake.remote.Codec;
-import org.apache.reef.wake.remote.NetUtils;
+import org.apache.reef.wake.remote.address.LocalAddressProvider;
+import org.apache.reef.wake.remote.address.LocalAddressProviderFactory;
 import org.apache.reef.wake.remote.impl.TransportEvent;
 import org.apache.reef.wake.remote.transport.Transport;
 import org.apache.reef.wake.remote.transport.netty.NettyMessagingTransport;
@@ -35,7 +36,6 @@ import org.apache.reef.webserver.AvroReefServiceInfo;
 import org.apache.reef.webserver.ReefEventStateManager;
 
 import javax.inject.Inject;
-import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.*;
 import java.util.logging.Level;
@@ -52,6 +52,7 @@ public class NameServerImpl implements NameServer {
   private final Map<Identifier, InetSocketAddress> idToAddrMap;
   private final ReefEventStateManager reefEventStateManager;
   private final int port;
+  private final LocalAddressProvider localAddressProvider;
 
   /**
    * @param port    a listening port number
@@ -63,13 +64,15 @@ public class NameServerImpl implements NameServer {
   @Deprecated
   public NameServerImpl(
       final int port,
-      final IdentifierFactory factory) {
+      final IdentifierFactory factory,
+      final LocalAddressProvider localAddressProvider) {
 
+    this.localAddressProvider = localAddressProvider;
     this.reefEventStateManager = null;
     final Codec<NamingMessage> codec = NamingCodecFactory.createFullCodec(factory);
     final EventHandler<NamingMessage> handler = createEventHandler(codec);
 
-    this.transport = new NettyMessagingTransport(NetUtils.getLocalAddress(), port, null,
+    this.transport = new NettyMessagingTransport(localAddressProvider.getLocalAddress(), port, null,
         new SyncStage<>(new NamingServerHandler(handler, codec)), 3, 10000);
 
     this.port = transport.getListeningPort();
@@ -78,6 +81,25 @@ public class NameServerImpl implements NameServer {
     LOG.log(Level.FINE, "NameServer starting, listening at port {0}", this.port);
   }
 
+  /**
+   * @deprecated have an instance injected instead
+   */
+  @Deprecated
+  public NameServerImpl(final int port, final IdentifierFactory factory) {
+    this(port, factory, LocalAddressProviderFactory.getInstance());
+
+  }
+
+  /**
+   * @deprecated have an instance injected instead
+   */
+  @Deprecated
+  public NameServerImpl(
+      final @Parameter(NameServerParameters.NameServerPort.class) int port,
+      final @Parameter(NameServerParameters.NameServerIdentifierFactory.class) IdentifierFactory factory,
+      final ReefEventStateManager reefEventStateManager) {
+    this(port, factory, reefEventStateManager, LocalAddressProviderFactory.getInstance());
+  }
 
   /**
    * Constructs a name server
@@ -90,13 +112,16 @@ public class NameServerImpl implements NameServer {
   public NameServerImpl(
       final @Parameter(NameServerParameters.NameServerPort.class) int port,
       final @Parameter(NameServerParameters.NameServerIdentifierFactory.class) IdentifierFactory factory,
-      final ReefEventStateManager reefEventStateManager) {
+      final ReefEventStateManager reefEventStateManager,
+      final LocalAddressProvider localAddressProvider) {
+
+    this.localAddressProvider = localAddressProvider;
 
     this.reefEventStateManager = reefEventStateManager;
     final Codec<NamingMessage> codec = NamingCodecFactory.createFullCodec(factory);
     final EventHandler<NamingMessage> handler = createEventHandler(codec);
 
-    this.transport = new NettyMessagingTransport(NetUtils.getLocalAddress(), port, null,
+    this.transport = new NettyMessagingTransport(localAddressProvider.getLocalAddress(), port, null,
         new SyncStage<>(new NamingServerHandler(handler, codec)), 3, 10000);
 
     this.port = transport.getListeningPort();
@@ -195,7 +220,7 @@ public class NameServerImpl implements NameServer {
   }
 
   private String getNameServerId() {
-    return NetUtils.getLocalAddress() + ":" + getPort();
+    return this.localAddressProvider.getLocalAddress() + ":" + getPort();
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-io/src/test/java/org/apache/reef/services/network/NameClientTest.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/test/java/org/apache/reef/services/network/NameClientTest.java b/lang/java/reef-io/src/test/java/org/apache/reef/services/network/NameClientTest.java
index f1f1a60..18787db 100644
--- a/lang/java/reef-io/src/test/java/org/apache/reef/services/network/NameClientTest.java
+++ b/lang/java/reef-io/src/test/java/org/apache/reef/services/network/NameClientTest.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -25,7 +25,8 @@ import org.apache.reef.tang.Tang;
 import org.apache.reef.tang.exceptions.InjectionException;
 import org.apache.reef.wake.Identifier;
 import org.apache.reef.wake.IdentifierFactory;
-import org.apache.reef.wake.remote.NetUtils;
+import org.apache.reef.wake.remote.address.LocalAddressProvider;
+import org.apache.reef.wake.remote.address.LocalAddressProviderFactory;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -36,6 +37,12 @@ import java.util.concurrent.ExecutionException;
 
 public class NameClientTest {
 
+  private final LocalAddressProvider localAddressProvider;
+
+  public NameClientTest() throws InjectionException {
+    this.localAddressProvider = LocalAddressProviderFactory.getInstance();
+  }
+
   static int retryCount, retryTimeout;
 
   static {
@@ -69,13 +76,14 @@ public class NameClientTest {
    */
   @Test
   public final void testClose() throws Exception {
+    final String localAddress = localAddressProvider.getLocalAddress();
     IdentifierFactory factory = new StringIdentifierFactory();
-    try (NameServer server = new NameServerImpl(0, factory)) {
+    try (NameServer server = new NameServerImpl(0, factory, this.localAddressProvider)) {
       int serverPort = server.getPort();
-      try (NameClient client = new NameClient(NetUtils.getLocalAddress(), serverPort, factory, retryCount, retryTimeout,
-          new NameCache(10000))) {
+      try (NameClient client = new NameClient(localAddress, serverPort, factory, retryCount, retryTimeout,
+          new NameCache(10000), localAddressProvider)) {
         Identifier id = factory.getNewInstance("Task1");
-        client.register(id, new InetSocketAddress(NetUtils.getLocalAddress(), 7001));
+        client.register(id, new InetSocketAddress(localAddress, 7001));
         client.unregister(id);
         Thread.sleep(100);
       }
@@ -92,12 +100,13 @@ public class NameClientTest {
   @Test
   public final void testLookup() throws Exception {
     IdentifierFactory factory = new StringIdentifierFactory();
-    try (NameServer server = new NameServerImpl(0, factory)) {
+    final String localAddress = localAddressProvider.getLocalAddress();
+    try (NameServer server = new NameServerImpl(0, factory, this.localAddressProvider)) {
       int serverPort = server.getPort();
-      try (NameClient client = new NameClient(NetUtils.getLocalAddress(), serverPort, factory, retryCount, retryTimeout,
-          new NameCache(150))) {
+      try (NameClient client = new NameClient(localAddress, serverPort, factory, retryCount, retryTimeout,
+          new NameCache(150), localAddressProvider)) {
         Identifier id = factory.getNewInstance("Task1");
-        client.register(id, new InetSocketAddress(NetUtils.getLocalAddress(), 7001));
+        client.register(id, new InetSocketAddress(localAddress, 7001));
         client.lookup(id);// caches the entry
         client.unregister(id);
         Thread.sleep(100);

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-io/src/test/java/org/apache/reef/services/network/NamingTest.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/test/java/org/apache/reef/services/network/NamingTest.java b/lang/java/reef-io/src/test/java/org/apache/reef/services/network/NamingTest.java
index 71382e2..139595b 100644
--- a/lang/java/reef-io/src/test/java/org/apache/reef/services/network/NamingTest.java
+++ b/lang/java/reef-io/src/test/java/org/apache/reef/services/network/NamingTest.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -26,7 +26,8 @@ import org.apache.reef.tang.Tang;
 import org.apache.reef.tang.exceptions.InjectionException;
 import org.apache.reef.wake.Identifier;
 import org.apache.reef.wake.IdentifierFactory;
-import org.apache.reef.wake.remote.NetUtils;
+import org.apache.reef.wake.remote.address.LocalAddressProvider;
+import org.apache.reef.wake.remote.address.LocalAddressProviderFactory;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
@@ -61,12 +62,17 @@ public class NamingTest {
     }
   }
 
+  private final LocalAddressProvider localAddressProvider;
   @Rule
   public final TestName name = new TestName();
   final long TTL = 30000;
   final IdentifierFactory factory = new StringIdentifierFactory();
   int port;
 
+  public NamingTest() throws InjectionException {
+    this.localAddressProvider = LocalAddressProviderFactory.getInstance();
+  }
+
   /**
    * NameServer and NameLookupClient test
    *
@@ -75,23 +81,24 @@ public class NamingTest {
   @Test
   public void testNamingLookup() throws Exception {
 
+    final String localAddress = localAddressProvider.getLocalAddress();
     LOG.log(Level.FINEST, this.name.getMethodName());
 
     // names 
     final Map<Identifier, InetSocketAddress> idToAddrMap = new HashMap<Identifier, InetSocketAddress>();
-    idToAddrMap.put(this.factory.getNewInstance("task1"), new InetSocketAddress(NetUtils.getLocalAddress(), 7001));
-    idToAddrMap.put(this.factory.getNewInstance("task2"), new InetSocketAddress(NetUtils.getLocalAddress(), 7002));
+    idToAddrMap.put(this.factory.getNewInstance("task1"), new InetSocketAddress(localAddress, 7001));
+    idToAddrMap.put(this.factory.getNewInstance("task2"), new InetSocketAddress(localAddress, 7002));
 
     // run a server
-    final NameServer server = new NameServerImpl(0, this.factory);
+    final NameServer server = new NameServerImpl(0, this.factory, this.localAddressProvider);
     this.port = server.getPort();
     for (final Identifier id : idToAddrMap.keySet()) {
       server.register(id, idToAddrMap.get(id));
     }
 
     // run a client
-    final NameLookupClient client = new NameLookupClient(NetUtils.getLocalAddress(), this.port,
-        10000, this.factory, retryCount, retryTimeout, new NameCache(this.TTL));
+    final NameLookupClient client = new NameLookupClient(localAddress, this.port,
+        10000, this.factory, retryCount, retryTimeout, new NameCache(this.TTL), this.localAddressProvider);
 
     final Identifier id1 = this.factory.getNewInstance("task1");
     final Identifier id2 = this.factory.getNewInstance("task2");
@@ -122,6 +129,7 @@ public class NamingTest {
 
     LOG.log(Level.FINEST, this.name.getMethodName());
 
+    final String localAddress = localAddressProvider.getLocalAddress();
     // test it 3 times to make failure likely
     for (int i = 0; i < 3; i++) {
 
@@ -129,20 +137,20 @@ public class NamingTest {
 
       // names 
       final Map<Identifier, InetSocketAddress> idToAddrMap = new HashMap<Identifier, InetSocketAddress>();
-      idToAddrMap.put(this.factory.getNewInstance("task1"), new InetSocketAddress(NetUtils.getLocalAddress(), 7001));
-      idToAddrMap.put(this.factory.getNewInstance("task2"), new InetSocketAddress(NetUtils.getLocalAddress(), 7002));
-      idToAddrMap.put(this.factory.getNewInstance("task3"), new InetSocketAddress(NetUtils.getLocalAddress(), 7003));
+      idToAddrMap.put(this.factory.getNewInstance("task1"), new InetSocketAddress(localAddress, 7001));
+      idToAddrMap.put(this.factory.getNewInstance("task2"), new InetSocketAddress(localAddress, 7002));
+      idToAddrMap.put(this.factory.getNewInstance("task3"), new InetSocketAddress(localAddress, 7003));
 
       // run a server
-      final NameServer server = new NameServerImpl(0, this.factory);
+      final NameServer server = new NameServerImpl(0, this.factory, this.localAddressProvider);
       this.port = server.getPort();
       for (final Identifier id : idToAddrMap.keySet()) {
         server.register(id, idToAddrMap.get(id));
       }
 
       // run a client
-      final NameLookupClient client = new NameLookupClient(NetUtils.getLocalAddress(), this.port,
-          10000, this.factory, retryCount, retryTimeout, new NameCache(this.TTL));
+      final NameLookupClient client = new NameLookupClient(localAddress, this.port,
+          10000, this.factory, retryCount, retryTimeout, new NameCache(this.TTL), this.localAddressProvider);
 
       final Identifier id1 = this.factory.getNewInstance("task1");
       final Identifier id2 = this.factory.getNewInstance("task2");
@@ -217,18 +225,18 @@ public class NamingTest {
 
     LOG.log(Level.FINEST, this.name.getMethodName());
 
-    final NameServer server = new NameServerImpl(0, this.factory);
+    final NameServer server = new NameServerImpl(0, this.factory, this.localAddressProvider);
     this.port = server.getPort();
+    final String localAddress = localAddressProvider.getLocalAddress();
 
     // names to start with
     final Map<Identifier, InetSocketAddress> idToAddrMap = new HashMap<Identifier, InetSocketAddress>();
-    idToAddrMap.put(this.factory.getNewInstance("task1"), new InetSocketAddress(NetUtils.getLocalAddress(), 7001));
-    idToAddrMap.put(this.factory.getNewInstance("task2"), new InetSocketAddress(NetUtils.getLocalAddress(), 7002));
+    idToAddrMap.put(this.factory.getNewInstance("task1"), new InetSocketAddress(localAddress, 7001));
+    idToAddrMap.put(this.factory.getNewInstance("task2"), new InetSocketAddress(localAddress, 7002));
 
     // registration
     // invoke registration from the client side
-    final NameRegistryClient client = new NameRegistryClient(
-        NetUtils.getLocalAddress(), this.port, this.factory);
+    final NameRegistryClient client = new NameRegistryClient(localAddress, this.port, this.factory, this.localAddressProvider);
     for (final Identifier id : idToAddrMap.keySet()) {
       client.register(id, idToAddrMap.get(id));
     }
@@ -278,17 +286,18 @@ public class NamingTest {
 
     LOG.log(Level.FINEST, this.name.getMethodName());
 
-    final NameServer server = new NameServerImpl(0, this.factory);
+    final String localAddress = localAddressProvider.getLocalAddress();
+    final NameServer server = new NameServerImpl(0, this.factory, this.localAddressProvider);
     this.port = server.getPort();
 
     final Map<Identifier, InetSocketAddress> idToAddrMap = new HashMap<Identifier, InetSocketAddress>();
-    idToAddrMap.put(this.factory.getNewInstance("task1"), new InetSocketAddress(NetUtils.getLocalAddress(), 7001));
-    idToAddrMap.put(this.factory.getNewInstance("task2"), new InetSocketAddress(NetUtils.getLocalAddress(), 7002));
+    idToAddrMap.put(this.factory.getNewInstance("task1"), new InetSocketAddress(localAddress, 7001));
+    idToAddrMap.put(this.factory.getNewInstance("task2"), new InetSocketAddress(localAddress, 7002));
 
     // registration
     // invoke registration from the client side
-    final NameClient client = new NameClient(NetUtils.getLocalAddress(), this.port,
-        this.factory, retryCount, retryTimeout, new NameCache(this.TTL));
+    final NameClient client = new NameClient(localAddress, this.port,
+        this.factory, retryCount, retryTimeout, new NameCache(this.TTL), this.localAddressProvider);
     for (final Identifier id : idToAddrMap.keySet()) {
       client.register(id, idToAddrMap.get(id));
     }

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-io/src/test/java/org/apache/reef/services/network/NetworkServiceTest.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/test/java/org/apache/reef/services/network/NetworkServiceTest.java b/lang/java/reef-io/src/test/java/org/apache/reef/services/network/NetworkServiceTest.java
index 17765f1..f68ef5a 100644
--- a/lang/java/reef-io/src/test/java/org/apache/reef/services/network/NetworkServiceTest.java
+++ b/lang/java/reef-io/src/test/java/org/apache/reef/services/network/NetworkServiceTest.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -28,10 +28,13 @@ import org.apache.reef.io.network.naming.NameServerImpl;
 import org.apache.reef.io.network.util.StringIdentifierFactory;
 import org.apache.reef.services.network.util.Monitor;
 import org.apache.reef.services.network.util.StringCodec;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.exceptions.InjectionException;
 import org.apache.reef.wake.EventHandler;
 import org.apache.reef.wake.Identifier;
 import org.apache.reef.wake.IdentifierFactory;
-import org.apache.reef.wake.remote.NetUtils;
+import org.apache.reef.wake.remote.address.LocalAddressProvider;
+import org.apache.reef.wake.remote.address.LocalAddressProviderFactory;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestName;
@@ -48,6 +51,14 @@ import java.util.logging.Logger;
 public class NetworkServiceTest {
   private static final Logger LOG = Logger.getLogger(NetworkServiceTest.class.getName());
 
+  private final LocalAddressProvider localAddressProvider;
+  private final String localAddress;
+
+  public NetworkServiceTest() throws InjectionException {
+    localAddressProvider = LocalAddressProviderFactory.getInstance();
+    localAddress = localAddressProvider.getLocalAddress();
+  }
+
   @Rule
   public TestName name = new TestName();
 
@@ -59,9 +70,8 @@ public class NetworkServiceTest {
     LOG.log(Level.FINEST, name.getMethodName());
 
     IdentifierFactory factory = new StringIdentifierFactory();
-    String nameServerAddr = NetUtils.getLocalAddress();
 
-    NameServer server = new NameServerImpl(0, factory);
+    NameServer server = new NameServerImpl(0, factory, localAddressProvider);
     int nameServerPort = server.getPort();
 
     final int numMessages = 10;
@@ -71,21 +81,21 @@ public class NetworkServiceTest {
     // network service
     final String name2 = "task2";
     NetworkService<String> ns2 = new NetworkService<String>(
-        factory, 0, nameServerAddr, nameServerPort,
-        new StringCodec(), new MessagingTransportFactory(),
-        new MessageHandler<String>(name2, monitor, numMessages), new ExceptionHandler());
+        factory, 0, this.localAddress, nameServerPort,
+        new StringCodec(), new MessagingTransportFactory(localAddressProvider),
+        new MessageHandler<String>(name2, monitor, numMessages), new ExceptionHandler(), localAddressProvider);
     ns2.registerId(factory.getNewInstance(name2));
     final int port2 = ns2.getTransport().getListeningPort();
-    server.register(factory.getNewInstance("task2"), new InetSocketAddress(nameServerAddr, port2));
+    server.register(factory.getNewInstance("task2"), new InetSocketAddress(this.localAddress, port2));
 
     LOG.log(Level.FINEST, "=== Test network service sender start");
     final String name1 = "task1";
-    final NetworkService<String> ns1 = new NetworkService<String>(factory, 0, nameServerAddr, nameServerPort,
-        new StringCodec(), new MessagingTransportFactory(),
-        new MessageHandler<String>(name1, null, 0), new ExceptionHandler());
+    final NetworkService<String> ns1 = new NetworkService<String>(factory, 0, this.localAddress, nameServerPort,
+        new StringCodec(), new MessagingTransportFactory(localAddressProvider),
+        new MessageHandler<String>(name1, null, 0), new ExceptionHandler(), localAddressProvider);
     ns1.registerId(factory.getNewInstance(name1));
     final int port1 = ns1.getTransport().getListeningPort();
-    server.register(factory.getNewInstance("task1"), new InetSocketAddress(nameServerAddr, port1));
+    server.register(factory.getNewInstance("task1"), new InetSocketAddress(this.localAddress, port1));
 
     final Identifier destId = factory.getNewInstance(name2);
     final Connection<String> conn = ns1.newConnection(destId);
@@ -115,9 +125,8 @@ public class NetworkServiceTest {
     LOG.log(Level.FINEST, name.getMethodName());
 
     IdentifierFactory factory = new StringIdentifierFactory();
-    String nameServerAddr = NetUtils.getLocalAddress();
 
-    NameServer server = new NameServerImpl(0, factory);
+    NameServer server = new NameServerImpl(0, factory, localAddressProvider);
     int nameServerPort = server.getPort();
 
     final int[] messageSizes = {1, 16, 32, 64, 512, 64 * 1024, 1024 * 1024};
@@ -130,22 +139,22 @@ public class NetworkServiceTest {
       // network service
       final String name2 = "task2";
       NetworkService<String> ns2 = new NetworkService<String>(
-          factory, 0, nameServerAddr, nameServerPort,
-          new StringCodec(), new MessagingTransportFactory(),
-          new MessageHandler<String>(name2, monitor, numMessages), new ExceptionHandler());
+          factory, 0, this.localAddress, nameServerPort,
+          new StringCodec(), new MessagingTransportFactory(localAddressProvider),
+          new MessageHandler<String>(name2, monitor, numMessages), new ExceptionHandler(), localAddressProvider);
       ns2.registerId(factory.getNewInstance(name2));
       final int port2 = ns2.getTransport().getListeningPort();
-      server.register(factory.getNewInstance("task2"), new InetSocketAddress(nameServerAddr, port2));
+      server.register(factory.getNewInstance("task2"), new InetSocketAddress(this.localAddress, port2));
 
       LOG.log(Level.FINEST, "=== Test network service sender start");
       final String name1 = "task1";
       NetworkService<String> ns1 = new NetworkService<String>(
-          factory, 0, nameServerAddr, nameServerPort,
-          new StringCodec(), new MessagingTransportFactory(),
-          new MessageHandler<String>(name1, null, 0), new ExceptionHandler());
+          factory, 0, this.localAddress, nameServerPort,
+          new StringCodec(), new MessagingTransportFactory(localAddressProvider),
+          new MessageHandler<String>(name1, null, 0), new ExceptionHandler(), localAddressProvider);
       ns1.registerId(factory.getNewInstance(name1));
       final int port1 = ns1.getTransport().getListeningPort();
-      server.register(factory.getNewInstance("task1"), new InetSocketAddress(nameServerAddr, port1));
+      server.register(factory.getNewInstance("task1"), new InetSocketAddress(this.localAddress, port1));
 
       Identifier destId = factory.getNewInstance(name2);
       Connection<String> conn = ns1.newConnection(destId);
@@ -187,9 +196,8 @@ public class NetworkServiceTest {
     LOG.log(Level.FINEST, name.getMethodName());
 
     final IdentifierFactory factory = new StringIdentifierFactory();
-    final String nameServerAddr = NetUtils.getLocalAddress();
 
-    final NameServer server = new NameServerImpl(0, factory);
+    final NameServer server = new NameServerImpl(0, factory, localAddressProvider);
     final int nameServerPort = server.getPort();
 
     BlockingQueue<Object> barrier = new LinkedBlockingQueue<Object>();
@@ -212,22 +220,22 @@ public class NetworkServiceTest {
             // network service
             final String name2 = "task2-" + tt;
             NetworkService<String> ns2 = new NetworkService<String>(
-                factory, 0, nameServerAddr, nameServerPort,
-                new StringCodec(), new MessagingTransportFactory(),
-                new MessageHandler<String>(name2, monitor, numMessages), new ExceptionHandler());
+                factory, 0, localAddress, nameServerPort,
+                new StringCodec(), new MessagingTransportFactory(localAddressProvider),
+                new MessageHandler<String>(name2, monitor, numMessages), new ExceptionHandler(), localAddressProvider);
             ns2.registerId(factory.getNewInstance(name2));
             final int port2 = ns2.getTransport().getListeningPort();
-            server.register(factory.getNewInstance(name2), new InetSocketAddress(nameServerAddr, port2));
+            server.register(factory.getNewInstance(name2), new InetSocketAddress(localAddress, port2));
 
             LOG.log(Level.FINEST, "=== Test network service sender start");
             final String name1 = "task1-" + tt;
             NetworkService<String> ns1 = new NetworkService<String>(
-                factory, 0, nameServerAddr, nameServerPort,
-                new StringCodec(), new MessagingTransportFactory(),
-                new MessageHandler<String>(name1, null, 0), new ExceptionHandler());
+                factory, 0, localAddress, nameServerPort,
+                new StringCodec(), new MessagingTransportFactory(localAddressProvider),
+                new MessageHandler<String>(name1, null, 0), new ExceptionHandler(), localAddressProvider);
             ns1.registerId(factory.getNewInstance(name1));
             final int port1 = ns1.getTransport().getListeningPort();
-            server.register(factory.getNewInstance(name1), new InetSocketAddress(nameServerAddr, port1));
+            server.register(factory.getNewInstance(name1), new InetSocketAddress(localAddress, port1));
 
             Identifier destId = factory.getNewInstance(name2);
             Connection<String> conn = ns1.newConnection(destId);
@@ -280,9 +288,8 @@ public class NetworkServiceTest {
     LOG.log(Level.FINEST, name.getMethodName());
 
     IdentifierFactory factory = new StringIdentifierFactory();
-    String nameServerAddr = NetUtils.getLocalAddress();
 
-    NameServer server = new NameServerImpl(0, factory);
+    NameServer server = new NameServerImpl(0, factory, localAddressProvider);
     int nameServerPort = server.getPort();
 
     final int[] messageSizes = {2000};// {1,16,32,64,512,64*1024,1024*1024};
@@ -297,22 +304,22 @@ public class NetworkServiceTest {
       // network service
       final String name2 = "task2";
       NetworkService<String> ns2 = new NetworkService<String>(
-          factory, 0, nameServerAddr, nameServerPort,
-          new StringCodec(), new MessagingTransportFactory(),
-          new MessageHandler<String>(name2, monitor, totalNumMessages), new ExceptionHandler());
+          factory, 0, this.localAddress, nameServerPort,
+          new StringCodec(), new MessagingTransportFactory(localAddressProvider),
+          new MessageHandler<String>(name2, monitor, totalNumMessages), new ExceptionHandler(), localAddressProvider);
       ns2.registerId(factory.getNewInstance(name2));
       final int port2 = ns2.getTransport().getListeningPort();
-      server.register(factory.getNewInstance("task2"), new InetSocketAddress(nameServerAddr, port2));
+      server.register(factory.getNewInstance("task2"), new InetSocketAddress(this.localAddress, port2));
 
       LOG.log(Level.FINEST, "=== Test network service sender start");
       final String name1 = "task1";
       NetworkService<String> ns1 = new NetworkService<String>(
-          factory, 0, nameServerAddr, nameServerPort,
-          new StringCodec(), new MessagingTransportFactory(),
-          new MessageHandler<String>(name1, null, 0), new ExceptionHandler());
+          factory, 0, this.localAddress, nameServerPort,
+          new StringCodec(), new MessagingTransportFactory(localAddressProvider),
+          new MessageHandler<String>(name1, null, 0), new ExceptionHandler(), localAddressProvider);
       ns1.registerId(factory.getNewInstance(name1));
       final int port1 = ns1.getTransport().getListeningPort();
-      server.register(factory.getNewInstance("task1"), new InetSocketAddress(nameServerAddr, port1));
+      server.register(factory.getNewInstance("task1"), new InetSocketAddress(this.localAddress, port1));
 
       Identifier destId = factory.getNewInstance(name2);
       final Connection<String> conn = ns1.newConnection(destId);
@@ -370,9 +377,8 @@ public class NetworkServiceTest {
     LOG.log(Level.FINEST, name.getMethodName());
 
     IdentifierFactory factory = new StringIdentifierFactory();
-    String nameServerAddr = NetUtils.getLocalAddress();
 
-    NameServer server = new NameServerImpl(0, factory);
+    NameServer server = new NameServerImpl(0, factory, localAddressProvider);
     int nameServerPort = server.getPort();
 
     final int batchSize = 1024 * 1024;
@@ -386,22 +392,22 @@ public class NetworkServiceTest {
       // network service
       final String name2 = "task2";
       NetworkService<String> ns2 = new NetworkService<String>(
-          factory, 0, nameServerAddr, nameServerPort,
-          new StringCodec(), new MessagingTransportFactory(),
-          new MessageHandler<String>(name2, monitor, numMessages), new ExceptionHandler());
+          factory, 0, this.localAddress, nameServerPort,
+          new StringCodec(), new MessagingTransportFactory(localAddressProvider),
+          new MessageHandler<String>(name2, monitor, numMessages), new ExceptionHandler(), localAddressProvider);
       ns2.registerId(factory.getNewInstance(name2));
       final int port2 = ns2.getTransport().getListeningPort();
-      server.register(factory.getNewInstance("task2"), new InetSocketAddress(nameServerAddr, port2));
+      server.register(factory.getNewInstance("task2"), new InetSocketAddress(this.localAddress, port2));
 
       LOG.log(Level.FINEST, "=== Test network service sender start");
       final String name1 = "task1";
       NetworkService<String> ns1 = new NetworkService<String>(
-          factory, 0, nameServerAddr, nameServerPort,
-          new StringCodec(), new MessagingTransportFactory(),
-          new MessageHandler<String>(name1, null, 0), new ExceptionHandler());
+          factory, 0, this.localAddress, nameServerPort,
+          new StringCodec(), new MessagingTransportFactory(localAddressProvider),
+          new MessageHandler<String>(name1, null, 0), new ExceptionHandler(), localAddressProvider);
       ns1.registerId(factory.getNewInstance(name1));
       final int port1 = ns1.getTransport().getListeningPort();
-      server.register(factory.getNewInstance("task1"), new InetSocketAddress(nameServerAddr, port1));
+      server.register(factory.getNewInstance("task1"), new InetSocketAddress(this.localAddress, port1));
 
       Identifier destId = factory.getNewInstance(name2);
       Connection<String> conn = ns1.newConnection(destId);
@@ -463,7 +469,7 @@ public class NetworkServiceTest {
 
       LOG.log(Level.FINEST,
           "OUT: {0} received {1} from {2} to {3}",
-          new Object[] { name, value.getData(), value.getSrcId(), value.getDestId() });
+          new Object[]{name, value.getData(), value.getSrcId(), value.getDestId()});
 
       for (final T obj : value.getData()) {
         LOG.log(Level.FINEST, "OUT: data: {0}", obj);

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/client/LocalRuntimeConfiguration.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/client/LocalRuntimeConfiguration.java b/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/client/LocalRuntimeConfiguration.java
index 03b3087..cbcdf66 100644
--- a/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/client/LocalRuntimeConfiguration.java
+++ b/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/client/LocalRuntimeConfiguration.java
@@ -31,6 +31,7 @@ import org.apache.reef.tang.formats.ConfigurationModule;
 import org.apache.reef.tang.formats.ConfigurationModuleBuilder;
 import org.apache.reef.tang.formats.OptionalImpl;
 import org.apache.reef.tang.formats.OptionalParameter;
+import org.apache.reef.wake.remote.address.LocalAddressProvider;
 
 import java.util.concurrent.ExecutorService;
 
@@ -66,17 +67,28 @@ public class LocalRuntimeConfiguration extends ConfigurationModuleBuilder {
   public static final OptionalImpl<ConfigurationProvider> DRIVER_CONFIGURATION_PROVIDERS = new OptionalImpl<>();
 
   /**
+   * The class used to resolve the local address for Wake and HTTP to bind to.
+   * Note that you will likely want to bind the same class also to DRIVER_CONFIGURATION_PROVIDERS to make sure that
+   * the Driver (and the Evaluators) also use it.
+   */
+  public static final OptionalImpl<LocalAddressProvider> LOCAL_ADDRESS_PROVIDER = new OptionalImpl<>();
+
+  /**
    * The ConfigurationModule for the local resourcemanager.
    */
   public static final ConfigurationModule CONF = new LocalRuntimeConfiguration()
       .merge(CommonRuntimeConfiguration.CONF)
+          // Bind the local runtime
       .bindImplementation(JobSubmissionHandler.class, LocalJobSubmissionHandler.class)
       .bindConstructor(ExecutorService.class, ExecutorServiceConstructor.class)
+      .bindImplementation(RuntimeClasspathProvider.class, LocalClasspathProvider.class)
+          // Bind parameters of the local runtime
       .bindNamedParameter(MaxNumberOfEvaluators.class, MAX_NUMBER_OF_EVALUATORS)
       .bindNamedParameter(RootFolder.class, RUNTIME_ROOT_FOLDER)
       .bindNamedParameter(JVMHeapSlack.class, JVM_HEAP_SLACK)
       .bindSetEntry(DriverConfigurationProviders.class, DRIVER_CONFIGURATION_PROVIDERS)
-      .bindImplementation(RuntimeClasspathProvider.class, LocalClasspathProvider.class)
+          // Bind LocalAddressProvider
+      .bindImplementation(LocalAddressProvider.class, LOCAL_ADDRESS_PROVIDER)
       .build();
 
 

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/ContainerManager.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/ContainerManager.java b/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/ContainerManager.java
index c1a746c..65290c1 100644
--- a/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/ContainerManager.java
+++ b/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/ContainerManager.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -31,7 +31,7 @@ import org.apache.reef.runtime.local.client.parameters.RootFolder;
 import org.apache.reef.runtime.local.process.ReefRunnableProcessObserver;
 import org.apache.reef.tang.annotations.Parameter;
 import org.apache.reef.wake.EventHandler;
-import org.apache.reef.wake.remote.NetUtils;
+import org.apache.reef.wake.remote.address.LocalAddressProvider;
 import org.apache.reef.wake.remote.RemoteMessage;
 import org.apache.reef.wake.time.Time;
 import org.apache.reef.wake.time.runtime.RuntimeClock;
@@ -72,6 +72,7 @@ final class ContainerManager implements AutoCloseable {
   private final File rootFolder;
   private final REEFFileNames fileNames;
   private final ReefRunnableProcessObserver processObserver;
+  private final String localAddress;
 
   @Inject
   ContainerManager(
@@ -82,7 +83,8 @@ final class ContainerManager implements AutoCloseable {
       final @Parameter(RootFolder.class) String rootFolderName,
       final @Parameter(RuntimeParameters.NodeDescriptorHandler.class)
       EventHandler<DriverRuntimeProtocol.NodeDescriptorProto> nodeDescriptorHandler,
-      final ReefRunnableProcessObserver processObserver) {
+      final ReefRunnableProcessObserver processObserver,
+      final LocalAddressProvider localAddressProvider) {
 
     this.capacity = capacity;
     this.fileNames = fileNames;
@@ -90,6 +92,7 @@ final class ContainerManager implements AutoCloseable {
     this.errorHandlerRID = remoteManager.getMyIdentifier();
     this.nodeDescriptorHandler = nodeDescriptorHandler;
     this.rootFolder = new File(rootFolderName);
+    this.localAddress = localAddressProvider.getLocalAddress();
 
     LOG.log(Level.FINEST, "Initializing Container Manager with {0} containers", capacity);
 
@@ -131,7 +134,7 @@ final class ContainerManager implements AutoCloseable {
       nodeDescriptorHandler.onNext(DriverRuntimeProtocol.NodeDescriptorProto.newBuilder()
           .setIdentifier(id)
           .setRackName("/default-rack")
-          .setHostName(NetUtils.getLocalAddress())
+          .setHostName(this.localAddress)
           .setPort(i)
           .setMemorySize(512) // TODO: Find the actual system memory on this machine.
           .build());

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-runtime-mesos/src/main/java/org/apache/reef/runtime/mesos/util/MesosRemoteManager.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-mesos/src/main/java/org/apache/reef/runtime/mesos/util/MesosRemoteManager.java b/lang/java/reef-runtime-mesos/src/main/java/org/apache/reef/runtime/mesos/util/MesosRemoteManager.java
index b32475d..3c3bfd9 100644
--- a/lang/java/reef-runtime-mesos/src/main/java/org/apache/reef/runtime/mesos/util/MesosRemoteManager.java
+++ b/lang/java/reef-runtime-mesos/src/main/java/org/apache/reef/runtime/mesos/util/MesosRemoteManager.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -19,12 +19,11 @@
 package org.apache.reef.runtime.mesos.util;
 
 import org.apache.reef.wake.EventHandler;
+import org.apache.reef.wake.remote.address.LocalAddressProvider;
 import org.apache.reef.wake.remote.RemoteIdentifierFactory;
 import org.apache.reef.wake.remote.RemoteManager;
 import org.apache.reef.wake.remote.RemoteMessage;
 import org.apache.reef.wake.remote.impl.DefaultRemoteManagerImplementation;
-import org.apache.reef.wake.remote.impl.ObjectSerializableCodec;
-import org.apache.reef.wake.remote.impl.StringCodec;
 
 import javax.inject.Inject;
 
@@ -40,10 +39,11 @@ public final class MesosRemoteManager {
   @Inject
   MesosRemoteManager(final RemoteIdentifierFactory factory,
                      final MesosErrorHandler mesosErrorHandler,
-                     final MesosRemoteManagerCodec codec) {
+                     final MesosRemoteManagerCodec codec,
+                     final LocalAddressProvider localAddressProvider) {
     this.factory = factory;
     this.raw = new DefaultRemoteManagerImplementation("MESOS_EXECUTOR", "##UNKNOWN##", 0,
-        codec, mesosErrorHandler, false, 3, 10000);
+        codec, mesosErrorHandler, false, 3, 10000, localAddressProvider);
   }
 
   public <T> EventHandler<T> getHandler(

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-runtime-yarn/src/main/java/org/apache/reef/runtime/yarn/client/YarnClientConfiguration.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-yarn/src/main/java/org/apache/reef/runtime/yarn/client/YarnClientConfiguration.java b/lang/java/reef-runtime-yarn/src/main/java/org/apache/reef/runtime/yarn/client/YarnClientConfiguration.java
index 4351a3f..5b46da6 100644
--- a/lang/java/reef-runtime-yarn/src/main/java/org/apache/reef/runtime/yarn/client/YarnClientConfiguration.java
+++ b/lang/java/reef-runtime-yarn/src/main/java/org/apache/reef/runtime/yarn/client/YarnClientConfiguration.java
@@ -35,6 +35,7 @@ import org.apache.reef.tang.formats.ConfigurationModuleBuilder;
 import org.apache.reef.tang.formats.OptionalImpl;
 import org.apache.reef.tang.formats.OptionalParameter;
 import org.apache.reef.util.logging.LoggingSetup;
+import org.apache.reef.wake.remote.address.LocalAddressProvider;
 
 /**
  * A ConfigurationModule for the YARN resourcemanager.
@@ -56,6 +57,13 @@ public class YarnClientConfiguration extends ConfigurationModuleBuilder {
    */
   public static final OptionalImpl<ConfigurationProvider> DRIVER_CONFIGURATION_PROVIDERS = new OptionalImpl<>();
 
+  /**
+   * The class used to resolve the local address for Wake and HTTP to bind to.
+   * Note that you will likely want to bind the same class also to DRIVER_CONFIGURATION_PROVIDERS to make sure that
+   * the Driver (and the Evaluators) also use it.
+   */
+  public static final OptionalImpl<LocalAddressProvider> LOCAL_ADDRESS_PROVIDER = new OptionalImpl<>();
+
   public static final ConfigurationModule CONF = new YarnClientConfiguration()
       .merge(CommonRuntimeConfiguration.CONF)
           // Bind YARN
@@ -68,6 +76,8 @@ public class YarnClientConfiguration extends ConfigurationModuleBuilder {
           // Bind external constructors. Taken from  YarnExternalConstructors.registerClientConstructors
       .bindConstructor(org.apache.hadoop.yarn.conf.YarnConfiguration.class, YarnConfigurationConstructor.class)
       .bindSetEntry(DriverConfigurationProviders.class, DRIVER_CONFIGURATION_PROVIDERS)
+          // Bind LocalAddressProvider
+      .bindImplementation(LocalAddressProvider.class, LOCAL_ADDRESS_PROVIDER)
       .build();
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/a72a40f8/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/NetUtils.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/NetUtils.java b/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/NetUtils.java
index d83f4c8..a5cbc87 100644
--- a/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/NetUtils.java
+++ b/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/NetUtils.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -18,89 +18,21 @@
  */
 package org.apache.reef.wake.remote;
 
-import org.apache.reef.wake.exception.WakeRuntimeException;
-
-import java.net.Inet4Address;
-import java.net.InetAddress;
-import java.net.NetworkInterface;
-import java.net.SocketException;
-import java.util.Comparator;
-import java.util.Enumeration;
-import java.util.TreeSet;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-public class NetUtils {
-  private static final Logger LOG = Logger.getLogger(NetUtils.class.getName());
-
-  private static AtomicReference<String> cached = new AtomicReference<>();
+import org.apache.reef.wake.remote.address.LocalAddressProvider;
+import org.apache.reef.wake.remote.address.LocalAddressProviderFactory;
 
+/**
+ * @deprecated Use <code>org.apache.reef.wake.remote.address.LocalAddressProvider</code> instead.
+ */
+@Deprecated
+public final class NetUtils {
+  private static final LocalAddressProvider localAddressProvider = LocalAddressProviderFactory.getInstance();
+
+  /**
+   * @deprecated Use <code>org.apache.reef.wake.remote.address.LocalAddressProvider#getLocalAddress()</code> instead.
+   */
+  @Deprecated
   public static String getLocalAddress() {
-    // This method is surprisingly slow: It was causing unit test timeouts, so we memoize the result.
-    if (cached.get() == null) {
-      Enumeration<NetworkInterface> ifaces;
-      try {
-        ifaces = NetworkInterface.getNetworkInterfaces();
-        TreeSet<Inet4Address> sortedAddrs = new TreeSet<>(new AddressComparator());
-        // There is an idea of virtual / subinterfaces exposed by java here.
-        // We're not walking around looking for those because the javadoc says:
-
-        // "NOTE: can use getNetworkInterfaces()+getInetAddresses() to obtain all IP addresses for this node"
-
-        while (ifaces.hasMoreElements()) {
-          NetworkInterface iface = ifaces.nextElement();
-//          if(iface.isUp()) {  // leads to slowness and non-deterministic return values, so don't call isUp().
-          Enumeration<InetAddress> addrs = iface.getInetAddresses();
-          while (addrs.hasMoreElements()) {
-            InetAddress a = addrs.nextElement();
-            if (a instanceof Inet4Address) {
-              sortedAddrs.add((Inet4Address) a);
-            }
-//            }
-          }
-        }
-        if (sortedAddrs.isEmpty()) {
-          throw new WakeRuntimeException("This machine apparently doesn't have any IP addresses (not even 127.0.0.1) on interfaces that are up.");
-        }
-        cached.set(sortedAddrs.pollFirst().getHostAddress());
-        LOG.log(Level.FINE, "Local address is {0}", cached.get());
-      } catch (SocketException e) {
-        throw new WakeRuntimeException("Unable to get local host address",
-            e.getCause());
-      }
-    }
-    return cached.get();
-  }
-
-  private static class AddressComparator implements Comparator<Inet4Address> {
-
-    // get unsigned byte.
-    private static int u(byte b) {
-      return ((int) b);// & 0xff;
-    }
-
-    @Override
-    public int compare(Inet4Address aa, Inet4Address ba) {
-      byte[] a = aa.getAddress();
-      byte[] b = ba.getAddress();
-      // local subnet comes after all else.
-      if (a[0] == 127 && b[0] != 127) {
-        return 1;
-      }
-      if (a[0] != 127 && b[0] == 127) {
-        return -1;
-      }
-      for (int i = 0; i < 4; i++) {
-        if (u(a[i]) < u(b[i])) {
-          return -1;
-        }
-        if (u(a[i]) > u(b[i])) {
-          return 1;
-        }
-      }
-      return 0;
-    }
+    return localAddressProvider.getLocalAddress();
   }
-
 }