Переглянути джерело

HDFS-9022. Move NameNode.getAddress() and NameNode.getUri() to hadoop-hdfs-client. Contributed by Mingliang Liu.

Haohui Mai 9 роки тому
батько
коміт
9eee97508f
27 змінених файлів з 124 додано та 109 видалено
  1. 41 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
  2. 2 2
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/RpcProgramMountd.java
  3. 2 2
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/DFSClientCache.java
  4. 4 3
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestWrites.java
  5. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  6. 3 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  7. 6 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  8. 6 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
  9. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java
  10. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageServlet.java
  11. 9 40
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  12. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
  13. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
  14. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/IPFailoverProxyProvider.java
  15. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java
  16. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetGroups.java
  17. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
  18. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientFailover.java
  19. 6 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShellGenericOptions.java
  20. 9 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDefaultNameNodePort.java
  21. 1 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java
  22. 3 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
  23. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPersistBlocks.java
  24. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeRollingUpgrade.java
  25. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
  26. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
  27. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java

+ 41 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java

@@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProviderFactory;
 import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.net.BasicInetPeer;
@@ -33,6 +34,7 @@ import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory;
@@ -587,4 +589,43 @@ public class DFSUtilClient {
       }
     }
   }
+
+  public static InetSocketAddress getNNAddress(String address) {
+    return NetUtils.createSocketAddr(address,
+        HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT);
+  }
+
+  public static InetSocketAddress getNNAddress(Configuration conf) {
+    URI filesystemURI = FileSystem.getDefaultUri(conf);
+    return getNNAddress(filesystemURI);
+  }
+
+  /**
+   * @return address of file system
+   */
+  public static InetSocketAddress getNNAddress(URI filesystemURI) {
+    String authority = filesystemURI.getAuthority();
+    if (authority == null) {
+      throw new IllegalArgumentException(String.format(
+          "Invalid URI for NameNode address (check %s): %s has no authority.",
+          FileSystem.FS_DEFAULT_NAME_KEY, filesystemURI.toString()));
+    }
+    if (!HdfsConstants.HDFS_URI_SCHEME.equalsIgnoreCase(
+        filesystemURI.getScheme())) {
+      throw new IllegalArgumentException(String.format(
+          "Invalid URI for NameNode address (check %s): " +
+          "%s is not of scheme '%s'.", FileSystem.FS_DEFAULT_NAME_KEY,
+          filesystemURI.toString(), HdfsConstants.HDFS_URI_SCHEME));
+    }
+    return getNNAddress(authority);
+  }
+
+  public static URI getNNUri(InetSocketAddress namenode) {
+    int port = namenode.getPort();
+    String portString =
+        (port == HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT) ?
+        "" : (":" + port);
+    return URI.create(HdfsConstants.HDFS_URI_SCHEME + "://"
+        + namenode.getHostName() + portString);
+  }
 }

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/RpcProgramMountd.java

@@ -27,10 +27,10 @@ import java.util.List;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
 import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.mount.MountEntry;
 import org.apache.hadoop.mount.MountInterface;
 import org.apache.hadoop.mount.MountResponse;
@@ -90,7 +90,7 @@ public class RpcProgramMountd extends RpcProgram implements MountInterface {
     UserGroupInformation.setConfiguration(config);
     SecurityUtil.login(config, NfsConfigKeys.DFS_NFS_KEYTAB_FILE_KEY,
         NfsConfigKeys.DFS_NFS_KERBEROS_PRINCIPAL_KEY);
-    this.dfsClient = new DFSClient(NameNode.getAddress(config), config);
+    this.dfsClient = new DFSClient(DFSUtilClient.getNNAddress(config), config);
   }
   
   @Override

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/DFSClientCache.java

@@ -33,8 +33,8 @@ import org.apache.commons.logging.Log;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSInputStream;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.io.MultipleIOException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.ShutdownHookManager;
@@ -173,7 +173,7 @@ class DFSClientCache {
         return ugi.doAs(new PrivilegedExceptionAction<DFSClient>() {
           @Override
           public DFSClient run() throws IOException {
-            return new DFSClient(NameNode.getAddress(config), config);
+            return new DFSClient(DFSUtilClient.getNNAddress(config), config);
           }
         });
       }

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestWrites.java

@@ -28,6 +28,7 @@ import java.util.Arrays;
 import java.util.concurrent.ConcurrentNavigableMap;
 
 import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
@@ -35,7 +36,6 @@ import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
 import org.apache.hadoop.hdfs.nfs.nfs3.OpenFileCtx.COMMIT_STATUS;
 import org.apache.hadoop.hdfs.nfs.nfs3.OpenFileCtx.CommitCtx;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.nfs.nfs3.FileHandle;
 import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
 import org.apache.hadoop.nfs.nfs3.Nfs3Constant.WriteStableHow;
@@ -480,7 +480,7 @@ public class TestWrites {
     try {
       cluster = new MiniDFSCluster.Builder(config).numDataNodes(1).build();
       cluster.waitActive();
-      client = new DFSClient(NameNode.getAddress(config), config);
+      client = new DFSClient(DFSUtilClient.getNNAddress(config), config);
 
       // Use emphral port in case tests are running in parallel
       config.setInt("nfs3.mountd.port", 0);
@@ -596,7 +596,8 @@ public class TestWrites {
       nfs3.startServiceInternal(false);
       nfsd = (RpcProgramNfs3) nfs3.getRpcProgram();
 
-      DFSClient dfsClient = new DFSClient(NameNode.getAddress(config), config);
+      DFSClient dfsClient = new DFSClient(DFSUtilClient.getNNAddress(config),
+          config);
       HdfsFileStatus status = dfsClient.getFileInfo("/");
       FileHandle rootHandle = new FileHandle(status.getFileId());
 

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -929,6 +929,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-7995. Implement chmod in the HDFS Web UI.
     (Ravi Prakash and Haohui Mai via wheat9)
 
+    HDFS-9022. Move NameNode.getAddress() and NameNode.getUri() to
+    hadoop-hdfs-client. (Mingliang Liu via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

+ 3 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -149,7 +149,6 @@ import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.io.DataOutputBuffer;
@@ -250,17 +249,17 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       = new HashMap<Long, DFSOutputStream>();
 
   /**
-   * Same as this(NameNode.getAddress(conf), conf);
+   * Same as this(NameNode.getNNAddress(conf), conf);
    * @see #DFSClient(InetSocketAddress, Configuration)
    * @deprecated Deprecated at 0.21
    */
   @Deprecated
   public DFSClient(Configuration conf) throws IOException {
-    this(NameNode.getAddress(conf), conf);
+    this(DFSUtilClient.getNNAddress(conf), conf);
   }
   
   public DFSClient(InetSocketAddress address, Configuration conf) throws IOException {
-    this(NameNode.getUri(address), conf);
+    this(DFSUtilClient.getNNUri(address), conf);
   }
 
   /**

+ 6 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java

@@ -411,7 +411,7 @@ public class DFSUtil {
           NameNode.initializeGenericKeys(confForNn, nsId, nnId);
           String principal = SecurityUtil.getServerPrincipal(confForNn
               .get(DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY),
-              NameNode.getAddress(confForNn).getHostName());
+              DFSUtilClient.getNNAddress(confForNn).getHostName());
           principals.add(principal);
         }
       } else {
@@ -419,7 +419,7 @@ public class DFSUtil {
         NameNode.initializeGenericKeys(confForNn, nsId, null);
         String principal = SecurityUtil.getServerPrincipal(confForNn
             .get(DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY),
-            NameNode.getAddress(confForNn).getHostName());
+            DFSUtilClient.getNNAddress(confForNn).getHostName());
         principals.add(principal);
       }
     }
@@ -495,7 +495,8 @@ public class DFSUtil {
     // Use default address as fall back
     String defaultAddress;
     try {
-      defaultAddress = NetUtils.getHostPortString(NameNode.getAddress(conf));
+      defaultAddress = NetUtils.getHostPortString(
+          DFSUtilClient.getNNAddress(conf));
     } catch (IllegalArgumentException e) {
       defaultAddress = null;
     }
@@ -531,7 +532,8 @@ public class DFSUtil {
     // Use default address as fall back
     String defaultAddress;
     try {
-      defaultAddress = NetUtils.getHostPortString(NameNode.getAddress(conf));
+      defaultAddress = NetUtils.getHostPortString(
+          DFSUtilClient.getNNAddress(conf));
     } catch (IllegalArgumentException e) {
       defaultAddress = null;
     }

+ 6 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java

@@ -165,8 +165,8 @@ public class NameNodeProxies {
   
     if (failoverProxyProvider == null) {
       // Non-HA case
-      return createNonHAProxy(conf, NameNode.getAddress(nameNodeUri), xface,
-          UserGroupInformation.getCurrentUser(), true,
+      return createNonHAProxy(conf, DFSUtilClient.getNNAddress(nameNodeUri),
+          xface, UserGroupInformation.getCurrentUser(), true,
           fallbackToSimpleAuth);
     } else {
       // HA case
@@ -183,10 +183,10 @@ public class NameNodeProxies {
                                                                 HdfsConstants.HDFS_URI_SCHEME);
       } else {
         dtService = SecurityUtil.buildTokenService(
-            NameNode.getAddress(nameNodeUri));
+            DFSUtilClient.getNNAddress(nameNodeUri));
       }
       return new ProxyAndInfo<T>(proxy, dtService,
-          NameNode.getAddress(nameNodeUri));
+          DFSUtilClient.getNNAddress(nameNodeUri));
     }
   }
   
@@ -249,10 +249,10 @@ public class NameNodeProxies {
                                                                 HdfsConstants.HDFS_URI_SCHEME);
       } else {
         dtService = SecurityUtil.buildTokenService(
-            NameNode.getAddress(nameNodeUri));
+            DFSUtilClient.getNNAddress(nameNodeUri));
       }
       return new ProxyAndInfo<T>(proxy, dtService,
-          NameNode.getAddress(nameNodeUri));
+          DFSUtilClient.getNNAddress(nameNodeUri));
     } else {
       LOG.warn("Currently creating proxy using " +
       		"LossyRetryInvocationHandler requires NN HA setup");

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java

@@ -26,6 +26,7 @@ import javax.servlet.http.HttpServletRequest;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
@@ -77,7 +78,7 @@ abstract class DfsServlet extends HttpServlet {
       NameNodeHttpServer.getNameNodeAddressFromContext(context);
     Configuration conf = new HdfsConfiguration(
         NameNodeHttpServer.getConfFromContext(context));
-    return NameNodeProxies.createProxy(conf, NameNode.getUri(nnAddr),
+    return NameNodeProxies.createProxy(conf, DFSUtilClient.getNNUri(nnAddr),
         ClientProtocol.class).getProxy();
   }
 

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageServlet.java

@@ -32,6 +32,7 @@ import javax.servlet.http.HttpServletResponse;
 
 import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -237,7 +238,7 @@ public class ImageServlet extends HttpServlet {
 
     validRequestors.add(SecurityUtil.getServerPrincipal(conf
         .get(DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY),
-        NameNode.getAddress(conf).getHostName()));
+        DFSUtilClient.getNNAddress(conf).getHostName()));
     try {
       validRequestors.add(
           SecurityUtil.getServerPrincipal(conf
@@ -261,7 +262,7 @@ public class ImageServlet extends HttpServlet {
       for (Configuration otherNnConf : otherNnConfs) {
         validRequestors.add(SecurityUtil.getServerPrincipal(otherNnConf
                 .get(DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY),
-            NameNode.getAddress(otherNnConf).getHostName()));
+            DFSUtilClient.getNNAddress(otherNnConf).getHostName()));
       }
     }
 

+ 9 - 40
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -445,10 +445,6 @@ public class NameNode implements NameNodeStatusMXBean {
     return clientNamenodeAddress;
   }
 
-  public static InetSocketAddress getAddress(String address) {
-    return NetUtils.createSocketAddr(address, DFS_NAMENODE_RPC_PORT_DEFAULT);
-  }
-  
   /**
    * Set the configuration property for the service rpc address
    * to address
@@ -470,45 +466,18 @@ public class NameNode implements NameNodeStatusMXBean {
                                                         boolean fallback) {
     String addr = conf.getTrimmed(DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY);
     if (addr == null || addr.isEmpty()) {
-      return fallback ? getAddress(conf) : null;
+      return fallback ? DFSUtilClient.getNNAddress(conf) : null;
     }
-    return getAddress(addr);
-  }
-
-  public static InetSocketAddress getAddress(Configuration conf) {
-    URI filesystemURI = FileSystem.getDefaultUri(conf);
-    return getAddress(filesystemURI);
+    return DFSUtilClient.getNNAddress(addr);
   }
 
-
+  @Deprecated
   /**
-   * @return address of file system
+   * @deprecated Use {@link DFSUtilClient#getNNUri(InetSocketAddress)} instead.
    */
-  public static InetSocketAddress getAddress(URI filesystemURI) {
-    String authority = filesystemURI.getAuthority();
-    if (authority == null) {
-      throw new IllegalArgumentException(String.format(
-          "Invalid URI for NameNode address (check %s): %s has no authority.",
-          FileSystem.FS_DEFAULT_NAME_KEY, filesystemURI.toString()));
-    }
-    if (!HdfsConstants.HDFS_URI_SCHEME.equalsIgnoreCase(
-        filesystemURI.getScheme())) {
-      throw new IllegalArgumentException(String.format(
-          "Invalid URI for NameNode address (check %s): %s is not of scheme '%s'.",
-          FileSystem.FS_DEFAULT_NAME_KEY, filesystemURI.toString(),
-          HdfsConstants.HDFS_URI_SCHEME));
-    }
-    return getAddress(authority);
-  }
-
   public static URI getUri(InetSocketAddress namenode) {
-    int port = namenode.getPort();
-    String portString = (port == DFS_NAMENODE_RPC_PORT_DEFAULT) ?
-        "" : (":" + port);
-    return URI.create(HdfsConstants.HDFS_URI_SCHEME + "://"
-        + namenode.getHostName()+portString);
+    return DFSUtilClient.getNNUri(namenode);
   }
-
   //
   // Common NameNode methods implementation for the active name-node role.
   //
@@ -529,7 +498,7 @@ public class NameNode implements NameNodeStatusMXBean {
   }
 
   protected InetSocketAddress getRpcServerAddress(Configuration conf) {
-    return getAddress(conf);
+    return DFSUtilClient.getNNAddress(conf);
   }
   
   /** Given a configuration get the bind host of the service rpc server
@@ -564,7 +533,7 @@ public class NameNode implements NameNodeStatusMXBean {
 
   protected void setRpcServerAddress(Configuration conf,
       InetSocketAddress rpcAddress) {
-    FileSystem.setDefaultUri(conf, getUri(rpcAddress));
+    FileSystem.setDefaultUri(conf, DFSUtilClient.getNNUri(rpcAddress));
   }
 
   protected InetSocketAddress getHttpServerAddress(Configuration conf) {
@@ -1012,7 +981,7 @@ public class NameNode implements NameNodeStatusMXBean {
     checkAllowFormat(conf);
 
     if (UserGroupInformation.isSecurityEnabled()) {
-      InetSocketAddress socAddr = getAddress(conf);
+      InetSocketAddress socAddr = DFSUtilClient.getNNAddress(conf);
       SecurityUtil.login(conf, DFS_NAMENODE_KEYTAB_FILE_KEY,
           DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY, socAddr.getHostName());
     }
@@ -1115,7 +1084,7 @@ public class NameNode implements NameNodeStatusMXBean {
     }
 
     if (UserGroupInformation.isSecurityEnabled()) {
-      InetSocketAddress socAddr = getAddress(conf);
+      InetSocketAddress socAddr = DFSUtilClient.getNNAddress(conf);
       SecurityUtil.login(conf, DFS_NAMENODE_KEYTAB_FILE_KEY,
           DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY, socAddr.getHostName());
     }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java

@@ -758,7 +758,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
 
   private void copyBlocksToLostFound(String parent, HdfsFileStatus file,
         LocatedBlocks blocks) throws IOException {
-    final DFSClient dfs = new DFSClient(NameNode.getAddress(conf), conf);
+    final DFSClient dfs = new DFSClient(DFSUtilClient.getNNAddress(conf), conf);
     final String fullName = file.getFullName(parent);
     OutputStream fos = null;
     try {

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java

@@ -38,6 +38,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@@ -101,7 +102,7 @@ public class BootstrapStandby implements Tool, Configurable {
     parseConfAndFindOtherNN();
     NameNode.checkAllowFormat(conf);
 
-    InetSocketAddress myAddr = NameNode.getAddress(conf);
+    InetSocketAddress myAddr = DFSUtilClient.getNNAddress(conf);
     SecurityUtil.login(conf, DFS_NAMENODE_KEYTAB_FILE_KEY,
         DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY, myAddr.getHostName());
 

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/IPFailoverProxyProvider.java

@@ -24,9 +24,9 @@ import java.net.URI;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -91,7 +91,7 @@ public class IPFailoverProxyProvider<T> extends
     if (nnProxyInfo == null) {
       try {
         // Create a proxy that is not wrapped in RetryProxy
-        InetSocketAddress nnAddr = NameNode.getAddress(nameNodeUri);
+        InetSocketAddress nnAddr = DFSUtilClient.getNNAddress(nameNodeUri);
         nnProxyInfo = new ProxyInfo<T>(NameNodeProxies.createNonHAProxy(
             conf, nnAddr, xface, UserGroupInformation.getCurrentUser(), 
             false).getProxy(), nnAddr.toString());

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java

@@ -39,6 +39,7 @@ import org.apache.hadoop.ha.HealthMonitor;
 import org.apache.hadoop.ha.ZKFailoverController;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
@@ -166,7 +167,7 @@ public class DFSZKFailoverController extends ZKFailoverController {
 
   @Override
   public void loginAsFCUser() throws IOException {
-    InetSocketAddress socAddr = NameNode.getAddress(conf);
+    InetSocketAddress socAddr = DFSUtilClient.getNNAddress(conf);
     SecurityUtil.login(conf, DFS_NAMENODE_KEYTAB_FILE_KEY,
         DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY, socAddr.getHostName());
   }

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetGroups.java

@@ -29,9 +29,9 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.NameNodeProxies;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.tools.GetGroupsBase;
 import org.apache.hadoop.tools.GetUserMappingsProtocol;
 import org.apache.hadoop.util.ToolRunner;
@@ -63,7 +63,7 @@ public class GetGroups extends GetGroupsBase {
   @Override
   protected InetSocketAddress getProtocolAddress(Configuration conf)
       throws IOException {
-    return NameNode.getAddress(conf);
+    return DFSUtilClient.getNNAddress(conf);
   }
   
   @Override

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java

@@ -1797,8 +1797,8 @@ public class DFSTestUtil {
       URI nameNodeUri, UserGroupInformation ugi)
       throws IOException {
     return NameNodeProxies.createNonHAProxy(conf,
-        NameNode.getAddress(nameNodeUri), NamenodeProtocol.class, ugi, false).
-        getProxy();
+        DFSUtilClient.getNNAddress(nameNodeUri), NamenodeProtocol.class, ugi,
+        false).getProxy();
   }
 
   /**

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientFailover.java

@@ -300,7 +300,7 @@ public class TestDFSClientFailover {
         Class<T> xface) {
       try {
         this.proxy = NameNodeProxies.createNonHAProxy(conf,
-            NameNode.getAddress(uri), xface,
+            DFSUtilClient.getNNAddress(uri), xface,
             UserGroupInformation.getCurrentUser(), false).getProxy();
         this.xface = xface;
       } catch (IOException ioe) {

+ 6 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShellGenericOptions.java

@@ -28,7 +28,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.Test;
 
@@ -53,7 +52,7 @@ public class TestDFSShellGenericOptions {
     }
   }
 
-  private void testFsOption(String [] args, String namenode) {        
+  private void testFsOption(String [] args, String namenode) {
     // prepare arguments to create a directory /data
     args[0] = "-fs";
     args[1] = namenode;
@@ -81,7 +80,7 @@ public class TestDFSShellGenericOptions {
       // prepare arguments to create a directory /data
       args[0] = "-conf";
       args[1] = siteFile.getPath();
-      execute(args, namenode); 
+      execute(args, namenode);
     } catch (FileNotFoundException e) {
       e.printStackTrace();
     } finally {
@@ -94,7 +93,7 @@ public class TestDFSShellGenericOptions {
     // prepare arguments to create a directory /data
     args[0] = "-D";
     args[1] = "fs.defaultFS="+namenode;
-    execute(args, namenode);        
+    execute(args, namenode);
   }
     
   private void execute(String [] args, String namenode) {
@@ -102,9 +101,9 @@ public class TestDFSShellGenericOptions {
     FileSystem fs=null;
     try {
       ToolRunner.run(shell, args);
-      fs = FileSystem.get(NameNode.getUri(NameNode.getAddress(namenode)),
-          shell.getConf());
-      assertTrue("Directory does not get created", 
+      fs = FileSystem.get(DFSUtilClient.getNNUri(
+          DFSUtilClient.getNNAddress(namenode)), shell.getConf());
+      assertTrue("Directory does not get created",
                  fs.isDirectory(new Path("/data")));
       fs.delete(new Path("/data"), true);
     } catch (Exception e) {

+ 9 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDefaultNameNodePort.java

@@ -34,13 +34,13 @@ public class TestDefaultNameNodePort {
 
   @Test
   public void testGetAddressFromString() throws Exception {
-    assertEquals(NameNode.getAddress("foo").getPort(),
+    assertEquals(DFSUtilClient.getNNAddress("foo").getPort(),
                  HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT);
-    assertEquals(NameNode.getAddress("hdfs://foo/").getPort(),
+    assertEquals(DFSUtilClient.getNNAddress("hdfs://foo/").getPort(),
                  HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT);
-    assertEquals(NameNode.getAddress("hdfs://foo:555").getPort(),
+    assertEquals(DFSUtilClient.getNNAddress("hdfs://foo:555").getPort(),
                  555);
-    assertEquals(NameNode.getAddress("foo:555").getPort(),
+    assertEquals(DFSUtilClient.getNNAddress("foo:555").getPort(),
                  555);
   }
 
@@ -48,20 +48,20 @@ public class TestDefaultNameNodePort {
   public void testGetAddressFromConf() throws Exception {
     Configuration conf = new HdfsConfiguration();
     FileSystem.setDefaultUri(conf, "hdfs://foo/");
-    assertEquals(NameNode.getAddress(conf).getPort(),
+    assertEquals(DFSUtilClient.getNNAddress(conf).getPort(),
         HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT);
     FileSystem.setDefaultUri(conf, "hdfs://foo:555/");
-    assertEquals(NameNode.getAddress(conf).getPort(), 555);
+    assertEquals(DFSUtilClient.getNNAddress(conf).getPort(), 555);
     FileSystem.setDefaultUri(conf, "foo");
-    assertEquals(NameNode.getAddress(conf).getPort(),
+    assertEquals(DFSUtilClient.getNNAddress(conf).getPort(),
         HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT);
   }
 
   @Test
   public void testGetUri() {
-    assertEquals(NameNode.getUri(new InetSocketAddress("foo", 555)),
+    assertEquals(DFSUtilClient.getNNUri(new InetSocketAddress("foo", 555)),
                  URI.create("hdfs://foo:555"));
-    assertEquals(NameNode.getUri(new InetSocketAddress("foo",
+    assertEquals(DFSUtilClient.getNNUri(new InetSocketAddress("foo",
             HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT)),
         URI.create("hdfs://foo"));
   }

+ 1 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java

@@ -35,12 +35,9 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.log4j.Level;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -73,7 +70,7 @@ public class TestFileStatus {
     cluster = new MiniDFSCluster.Builder(conf).build();
     fs = cluster.getFileSystem();
     fc = FileContext.getFileContext(cluster.getURI(0), conf);
-    dfsClient = new DFSClient(NameNode.getAddress(conf), conf);
+    dfsClient = new DFSClient(DFSUtilClient.getNNAddress(conf), conf);
     file1 = new Path("filestatus.dat");
     DFSTestUtil.createFile(fs, file1, fileSize, fileSize, blockSize, (short) 1,
         seed);

+ 3 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java

@@ -41,7 +41,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.ipc.RemoteException;
@@ -193,8 +192,8 @@ public class TestGetBlocks {
       DatanodeInfo[] dataNodes = null;
       boolean notWritten;
       do {
-        final DFSClient dfsclient = new DFSClient(NameNode.getAddress(CONF),
-            CONF);
+        final DFSClient dfsclient = new DFSClient(
+            DFSUtilClient.getNNAddress(CONF), CONF);
         locatedBlocks = dfsclient.getNamenode()
             .getBlockLocations("/tmp.txt", 0, fileLen).getLocatedBlocks();
         assertEquals(2, locatedBlocks.size());
@@ -216,7 +215,7 @@ public class TestGetBlocks {
       InetSocketAddress addr = new InetSocketAddress("localhost",
           cluster.getNameNodePort());
       NamenodeProtocol namenode = NameNodeProxies.createProxy(CONF,
-          NameNode.getUri(addr), NamenodeProtocol.class).getProxy();
+          DFSUtilClient.getNNUri(addr), NamenodeProtocol.class).getProxy();
 
       // get blocks of size fileLen from dataNodes[0]
       BlockWithLocations[] locs;

+ 2 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPersistBlocks.java

@@ -41,7 +41,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.namenode.FSImage;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.PathUtils;
@@ -225,7 +224,7 @@ public class TestPersistBlocks {
     try {
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
       FileSystem fs = cluster.getFileSystem();
-      NameNode.getAddress(conf).getPort();
+      DFSUtilClient.getNNAddress(conf).getPort();
       // Creating a file with 4096 blockSize to write multiple blocks
       stream = fs.create(FILE_PATH, true, BLOCK_SIZE, (short) 1, BLOCK_SIZE);
       stream.write(DATA_BEFORE_RESTART);
@@ -274,7 +273,7 @@ public class TestPersistBlocks {
     try {
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
       FileSystem fs = cluster.getFileSystem();
-      NameNode.getAddress(conf).getPort();
+      DFSUtilClient.getNNAddress(conf).getPort();
       // Creating a file with 4096 blockSize to write multiple blocks
       stream = fs.create(FILE_PATH, true, BLOCK_SIZE, (short) 1, BLOCK_SIZE);
       stream.write(DATA_BEFORE_RESTART, 0, DATA_BEFORE_RESTART.length / 2);

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeRollingUpgrade.java

@@ -34,13 +34,13 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSOutputStream;
 import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster.Builder;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.TestRollingUpgrade;
-import org.apache.hadoop.hdfs.client.BlockReportOptions;
 import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -294,9 +294,9 @@ public class TestDataNodeRollingUpgrade {
       String testFile2 = "/" + GenericTestUtils.getMethodName() + ".02.dat";
       String testFile3 = "/" + GenericTestUtils.getMethodName() + ".03.dat";
 
-      DFSClient client1 = new DFSClient(NameNode.getAddress(conf), conf);
-      DFSClient client2 = new DFSClient(NameNode.getAddress(conf), conf);
-      DFSClient client3 = new DFSClient(NameNode.getAddress(conf), conf);
+      DFSClient client1 = new DFSClient(DFSUtilClient.getNNAddress(conf), conf);
+      DFSClient client2 = new DFSClient(DFSUtilClient.getNNAddress(conf), conf);
+      DFSClient client3 = new DFSClient(DFSUtilClient.getNNAddress(conf), conf);
 
       DFSOutputStream s1 = (DFSOutputStream) client1.create(testFile1, true);
       DFSOutputStream s2 = (DFSOutputStream) client2.create(testFile2, true);

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java

@@ -37,6 +37,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -1497,7 +1498,7 @@ public class NNThroughputBenchmark implements Tool {
             UserGroupInformation.getCurrentUser());
         clientProto = dfs.getClient().getNamenode();
         dataNodeProto = new DatanodeProtocolClientSideTranslatorPB(
-            NameNode.getAddress(nnUri), config);
+            DFSUtilClient.getNNAddress(nnUri), config);
         refreshUserMappingsProto =
             DFSTestUtil.getRefreshUserMappingsProtocolProxy(config, nnUri);
         getBlockPoolId(dfs);

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java

@@ -53,9 +53,9 @@ import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -978,7 +978,7 @@ public class TestINodeFile {
       long parentId = fsdir.getINode("/").getId();
       String testPath = "/.reserved/.inodes/" + dirId + "/..";
 
-      client = new DFSClient(NameNode.getAddress(conf), conf);
+      client = new DFSClient(DFSUtilClient.getNNAddress(conf), conf);
       HdfsFileStatus status = client.getFileInfo(testPath);
       assertTrue(parentId == status.getFileId());
       

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java

@@ -35,6 +35,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
@@ -246,7 +247,7 @@ public class TestFailureToReadEdits {
     FileSystem fs0 = null;
     try {
       // Make sure that when the active restarts, it loads all the edits.
-      fs0 = FileSystem.get(NameNode.getUri(nn0.getNameNodeAddress()),
+      fs0 = FileSystem.get(DFSUtilClient.getNNUri(nn0.getNameNodeAddress()),
           conf);
       
       assertTrue(fs0.exists(new Path(TEST_DIR1)));