You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by ar...@apache.org on 2013/12/09 18:38:22 UTC

svn commit: r1549626 [2/2] - in /hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/ src/main/java/org/apache/hadoop/hdfs/ src/main/java/org/apache/hadoop/hdfs/net/ src/main/java/org/apache/hadoop/hdfs/server/datanode/ s...

Modified: hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java?rev=1549626&r1=1549625&r2=1549626&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java Mon Dec  9 17:38:20 2013
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.tools;
 import java.io.File;
 import java.io.IOException;
 import java.net.InetSocketAddress;
+import java.net.URL;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -47,9 +48,9 @@ import org.apache.hadoop.hdfs.NameNodePr
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.TransferFsImage;
 import org.apache.hadoop.ipc.RPC;
@@ -547,8 +548,10 @@ public class DFSAdmin extends FsShell {
    * @throws IOException
    */
   public int fetchImage(final String[] argv, final int idx) throws IOException {
-    final String infoServer = DFSUtil.getInfoServer(
-        HAUtil.getAddressOfActive(getDFS()), getConf(), false);
+    Configuration conf = getConf();
+    final URL infoServer = DFSUtil.getInfoServer(
+        HAUtil.getAddressOfActive(getDFS()), conf,
+        DFSUtil.getHttpClientScheme(conf)).toURL();
     SecurityUtil.doAsCurrentUser(new PrivilegedExceptionAction<Void>() {
       @Override
       public Void run() throws Exception {

Modified: hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java?rev=1549626&r1=1549625&r2=1549626&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java Mon Dec  9 17:38:20 2013
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.PrintStream;
+import java.net.URI;
 import java.net.URL;
 import java.net.URLConnection;
 import java.net.URLEncoder;
@@ -37,7 +38,6 @@ import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck;
 import org.apache.hadoop.hdfs.web.URLConnectionFactory;
-import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.util.StringUtils;
@@ -227,7 +227,7 @@ public class DFSck extends Configured im
    * @return Returns http address or null if failure.
    * @throws IOException if we can't determine the active NN address
    */
-  private String getCurrentNamenodeAddress() throws IOException {
+  private URI getCurrentNamenodeAddress() throws IOException {
     //String nnAddress = null;
     Configuration conf = getConf();
 
@@ -245,19 +245,21 @@ public class DFSck extends Configured im
       return null;
     }
     
-    return DFSUtil.getInfoServer(HAUtil.getAddressOfActive(fs), conf, false);
+    return DFSUtil.getInfoServer(HAUtil.getAddressOfActive(fs), conf,
+        DFSUtil.getHttpClientScheme(conf));
   }
 
   private int doWork(final String[] args) throws IOException {
-    final StringBuilder url = new StringBuilder(HttpConfig.getSchemePrefix());
+    final StringBuilder url = new StringBuilder();
     
-    String namenodeAddress = getCurrentNamenodeAddress();
+    URI namenodeAddress = getCurrentNamenodeAddress();
     if (namenodeAddress == null) {
       //Error message already output in {@link #getCurrentNamenodeAddress()}
       System.err.println("DFSck exiting.");
       return 0;
     }
-    url.append(namenodeAddress);
+
+    url.append(namenodeAddress.toString());
     System.err.println("Connecting to namenode via " + url.toString());
     
     url.append("/fsck?ugi=").append(ugi.getShortUserName());

Modified: hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FileDistributionVisitor.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FileDistributionVisitor.java?rev=1549626&r1=1549625&r2=1549626&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FileDistributionVisitor.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FileDistributionVisitor.java Mon Dec  9 17:38:20 2013
@@ -100,6 +100,18 @@ class FileDistributionVisitor extends Te
 
   @Override
   void finish() throws IOException {
+    output();
+    super.finish();
+  }
+
+  @Override
+  void finishAbnormally() throws IOException {
+    System.out.println("*** Image processing finished abnormally.  Ending ***");
+    output();
+    super.finishAbnormally();
+  }
+
+  private void output() throws IOException {
     // write the distribution into the output file
     write("Size\tNumFiles\n");
     for(int i = 0; i < distribution.length; i++)
@@ -109,7 +121,6 @@ class FileDistributionVisitor extends Te
     System.out.println("totalBlocks = " + totalBlocks);
     System.out.println("totalSpace = " + totalSpace);
     System.out.println("maxFileSize = " + maxFileSize);
-    super.finish();
   }
 
   @Override

Modified: hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java?rev=1549626&r1=1549625&r2=1549626&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java Mon Dec  9 17:38:20 2013
@@ -129,7 +129,7 @@ class ImageLoaderCurrent implements Imag
       -40, -41, -42, -43, -44, -45, -46, -47, -48, -49 };
   private int imageVersion = 0;
   
-  private final Map<Long, String> subtreeMap = new HashMap<Long, String>();
+  private final Map<Long, Boolean> subtreeMap = new HashMap<Long, Boolean>();
   private final Map<Long, String> dirNodeMap = new HashMap<Long, String>();
 
   /* (non-Javadoc)
@@ -500,11 +500,15 @@ class ImageLoaderCurrent implements Imag
     // 1. load dir node id
     long inodeId = in.readLong();
     
-    String dirName = dirNodeMap.get(inodeId);
-    String oldValue = subtreeMap.put(inodeId, dirName);
-    if (oldValue != null) { // the subtree has been visited
-      return;
-    }
+    String dirName = dirNodeMap.remove(inodeId);
+    Boolean visitedRef = subtreeMap.get(inodeId);
+    if (visitedRef != null) {
+      if (visitedRef.booleanValue()) { // the subtree has been visited
+        return;
+      } else { // first time to visit
+        subtreeMap.put(inodeId, true);
+      }
+    } // else the dir is not linked by a RefNode, thus cannot be revisited
     
     // 2. load possible snapshots
     processSnapshots(in, v, dirName);
@@ -695,6 +699,8 @@ class ImageLoaderCurrent implements Imag
     
     if (numBlocks >= 0) { // File
       if (supportSnapshot) {
+        // make sure subtreeMap only contains entry for directory
+        subtreeMap.remove(inodeId);
         // process file diffs
         processFileDiffList(in, v, parentName);
         if (isSnapshotCopy) {
@@ -738,6 +744,11 @@ class ImageLoaderCurrent implements Imag
       
       final boolean firstReferred = in.readBoolean();
       if (firstReferred) {
+        // if a subtree is linked by multiple "parents", the corresponding dir
+        // must be referred by a reference node. we put the reference node into
+        // the subtreeMap here and let its value be false. when we later visit
+        // the subtree for the first time, we change the value to true.
+        subtreeMap.put(inodeId, false);
         v.visitEnclosingElement(ImageElement.SNAPSHOT_REF_INODE);
         processINode(in, v, skipBlocks, parentName, isSnapshotCopy);
         v.leaveEnclosingElement();  // referred inode    

Modified: hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java?rev=1549626&r1=1549625&r2=1549626&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java Mon Dec  9 17:38:20 2013
@@ -157,6 +157,8 @@ public class WebHdfsFileSystem extends F
       ) throws IOException {
     super.initialize(uri, conf);
     setConf(conf);
+    /** set user pattern based on configuration file */
+    UserParam.setUserPattern(conf.get(DFSConfigKeys.DFS_WEBHDFS_USER_PATTERN_KEY, DFSConfigKeys.DFS_WEBHDFS_USER_PATTERN_DEFAULT));
     connectionFactory = URLConnectionFactory
         .newDefaultURLConnectionFactory(conf);
     initializeTokenAspect();

Modified: hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/UserParam.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/UserParam.java?rev=1549626&r1=1549625&r2=1549626&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/UserParam.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/UserParam.java Mon Dec  9 17:38:20 2013
@@ -17,8 +17,10 @@
  */
 package org.apache.hadoop.hdfs.web.resources;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEBHDFS_USER_PATTERN_DEFAULT;
 import org.apache.hadoop.security.UserGroupInformation;
-
+import com.google.common.annotations.VisibleForTesting;
+ 
 import java.text.MessageFormat;
 import java.util.regex.Pattern;
 
@@ -29,8 +31,21 @@ public class UserParam extends StringPar
   /** Default parameter value. */
   public static final String DEFAULT = "";
 
-  private static final Domain DOMAIN = new Domain(NAME,
-    Pattern.compile("^[A-Za-z_][A-Za-z0-9._-]*[$]?$"));
+  private static Domain domain = new Domain(NAME, Pattern.compile(DFS_WEBHDFS_USER_PATTERN_DEFAULT));
+
+  @VisibleForTesting
+  public static Domain getUserPatternDomain() {
+    return domain;
+  }
+
+  @VisibleForTesting
+  public static void setUserPatternDomain(Domain dm) {
+    domain = dm;
+  }
+
+  public static void setUserPattern(String pattern) {
+    domain = new Domain(NAME, Pattern.compile(pattern));
+  }
 
   private static String validateLength(String str) {
     if (str == null) {
@@ -50,7 +65,7 @@ public class UserParam extends StringPar
    * @param str a string representation of the parameter value.
    */
   public UserParam(final String str) {
-    super(DOMAIN, str == null || str.equals(DEFAULT)? null : validateLength(str));
+    super(domain, str == null || str.equals(DEFAULT)? null : validateLength(str));
   }
 
   /**
@@ -64,4 +79,4 @@ public class UserParam extends StringPar
   public String getName() {
     return NAME;
   }
-}
\ No newline at end of file
+}

Modified: hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml?rev=1549626&r1=1549625&r2=1549626&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml Mon Dec  9 17:38:20 2013
@@ -1593,4 +1593,12 @@
   </description>
 </property>
 
+<property>
+  <name>dfs.webhdfs.user.provider.user.pattern</name>
+  <value>^[A-Za-z_][A-Za-z0-9._-]*[$]?$</value>
+  <description>
+    Valid pattern for user and group names for webhdfs, it must be a valid java regex.
+  </description>
+</property>
+
 </configuration>

Modified: hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java?rev=1549626&r1=1549625&r2=1549626&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java Mon Dec  9 17:38:20 2013
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hdfs;
 
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_BACKUP_ADDRESS_KEY;
@@ -431,20 +430,22 @@ public class TestDFSUtil {
   }
 
   @Test
-  public void testGetInfoServer() throws IOException {
+  public void testGetInfoServer() throws IOException, URISyntaxException {
     HdfsConfiguration conf = new HdfsConfiguration();
-    conf.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos");
-    UserGroupInformation.setConfiguration(conf);
     
-    String httpsport = DFSUtil.getInfoServer(null, conf, true);
-    assertEquals("0.0.0.0:"+DFS_NAMENODE_HTTPS_PORT_DEFAULT, httpsport);
-    
-    String httpport = DFSUtil.getInfoServer(null, conf, false);
-    assertEquals("0.0.0.0:"+DFS_NAMENODE_HTTP_PORT_DEFAULT, httpport);
-    
-    String httpAddress = DFSUtil.getInfoServer(new InetSocketAddress(
-        "localhost", 8020), conf, false);
-    assertEquals("localhost:" + DFS_NAMENODE_HTTP_PORT_DEFAULT, httpAddress);
+    URI httpsport = DFSUtil.getInfoServer(null, conf, "https");
+    assertEquals(new URI("https", null, "0.0.0.0",
+        DFS_NAMENODE_HTTPS_PORT_DEFAULT, null, null, null), httpsport);
+    
+    URI httpport = DFSUtil.getInfoServer(null, conf, "http");
+    assertEquals(new URI("http", null, "0.0.0.0",
+        DFS_NAMENODE_HTTP_PORT_DEFAULT, null, null, null), httpport);
+
+    URI httpAddress = DFSUtil.getInfoServer(new InetSocketAddress(
+        "localhost", 8020), conf, "http");
+    assertEquals(
+        URI.create("http://localhost:" + DFS_NAMENODE_HTTP_PORT_DEFAULT),
+        httpAddress);
   }
   
   @Test

Modified: hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitReadUnCached.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitReadUnCached.java?rev=1549626&r1=1549625&r2=1549626&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitReadUnCached.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitReadUnCached.java Mon Dec  9 17:38:20 2013
@@ -42,6 +42,10 @@ public class TestParallelShortCircuitRea
       new File(sockDir.getDir(), 
         "TestParallelShortCircuitReadUnCached._PORT.sock").getAbsolutePath());
     conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
+    // Enabling data transfer encryption should have no effect when using
+    // short-circuit local reads.  This is a regression test for HDFS-5353.
+    conf.setBoolean(DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_KEY, true);
+    conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
     conf.setBoolean(DFSConfigKeys.
         DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, false);
     conf.setBoolean(DFSConfigKeys.

Modified: hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPeerCache.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPeerCache.java?rev=1549626&r1=1549625&r2=1549626&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPeerCache.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPeerCache.java Mon Dec  9 17:38:20 2013
@@ -140,6 +140,11 @@ public class TestPeerCache {
     public int hashCode() {
       return dnId.hashCode() ^ (hasDomain ? 1 : 0);
     }
+
+    @Override
+    public boolean hasSecureChannel() {
+      return false;
+    }
   }
 
   @Test

Modified: hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPersistBlocks.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPersistBlocks.java?rev=1549626&r1=1549625&r2=1549626&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPersistBlocks.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPersistBlocks.java Mon Dec  9 17:38:20 2013
@@ -97,7 +97,6 @@ public class TestPersistBlocks {
     conf.setInt(
         CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY,
         0);
-    conf.setBoolean(DFSConfigKeys.DFS_PERSIST_BLOCKS_KEY, true);
     MiniDFSCluster cluster = null;
 
     long len = 0;
@@ -157,7 +156,6 @@ public class TestPersistBlocks {
     conf.setInt(
         CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY,
         0);
-    conf.setBoolean(DFSConfigKeys.DFS_PERSIST_BLOCKS_KEY, true);
     MiniDFSCluster cluster = null;
 
     long len = 0;
@@ -219,7 +217,6 @@ public class TestPersistBlocks {
     conf.setInt(
         CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY,
         0);
-    conf.setBoolean(DFSConfigKeys.DFS_PERSIST_BLOCKS_KEY, true);
     MiniDFSCluster cluster = null;
 
     FSDataOutputStream stream;
@@ -269,7 +266,6 @@ public class TestPersistBlocks {
     conf.setInt(
         CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY,
         0);
-    conf.setBoolean(DFSConfigKeys.DFS_PERSIST_BLOCKS_KEY, true);
     MiniDFSCluster cluster = null;
 
     FSDataOutputStream stream;

Modified: hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java?rev=1549626&r1=1549625&r2=1549626&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java Mon Dec  9 17:38:20 2013
@@ -434,7 +434,6 @@ public class TestBackupNode {
   public void testCanReadData() throws IOException {
     Path file1 = new Path("/fileToRead.dat");
     Configuration conf = new HdfsConfiguration();
-    conf.setBoolean(DFSConfigKeys.DFS_PERSIST_BLOCKS_KEY, true);
     MiniDFSCluster cluster = null;
     FileSystem fileSys = null;
     BackupNode backup = null;

Modified: hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java?rev=1549626&r1=1549625&r2=1549626&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java Mon Dec  9 17:38:20 2013
@@ -34,6 +34,7 @@ import java.io.RandomAccessFile;
 import java.lang.management.ManagementFactory;
 import java.net.InetSocketAddress;
 import java.net.URI;
+import java.net.URL;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
@@ -71,7 +72,6 @@ import org.apache.hadoop.hdfs.server.pro
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils.DelayAnswer;
 import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
@@ -218,6 +218,7 @@ public class TestCheckpoint {
     assertTrue("Removed directory wasn't what was expected",
                listRsd.size() > 0 && listRsd.get(listRsd.size() - 1).getRoot().
                toString().indexOf("storageDirToCheck") != -1);
+    nnStorage.close();
   }
 
   /*
@@ -1947,8 +1948,9 @@ public class TestCheckpoint {
           .format(true).build();
       
       NamenodeProtocols nn = cluster.getNameNodeRpc();
-      String fsName = NetUtils.getHostPortString(
-          cluster.getNameNode().getHttpAddress());
+      URL fsName = DFSUtil.getInfoServer(
+          cluster.getNameNode().getServiceRpcAddress(), conf,
+          DFSUtil.getHttpClientScheme(conf)).toURL();
 
       // Make a finalized log on the server side. 
       nn.rollEditLog();
@@ -1980,8 +1982,7 @@ public class TestCheckpoint {
       }
 
       try {
-        InetSocketAddress fakeAddr = new InetSocketAddress(1);
-        TransferFsImage.uploadImageFromStorage(fsName, fakeAddr, dstImage, 0);
+        TransferFsImage.uploadImageFromStorage(fsName, new URL("http://localhost:1234"), dstImage, 0);
         fail("Storage info was not verified");
       } catch (IOException ioe) {
         String msg = StringUtils.stringifyException(ioe);

Modified: hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java?rev=1549626&r1=1549625&r2=1549626&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java Mon Dec  9 17:38:20 2013
@@ -32,7 +32,6 @@ import org.apache.hadoop.hdfs.Distribute
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -239,7 +238,7 @@ public class TestSnapshotPathINodes {
     // The last INode should be the INode for sub1
     final INode last = nodesInPath.getLastINode();
     assertEquals(last.getFullPathName(), sub1.toString());
-    assertFalse(last instanceof INodeFileWithSnapshot);
+    assertFalse(last instanceof INodeFile);
     
     String[] invalidPathComponent = {"invalidDir", "foo", ".snapshot", "bar"};
     Path invalidPath = new Path(invalidPathComponent[0]);
@@ -287,7 +286,7 @@ public class TestSnapshotPathINodes {
       // Check the INode for file1 (snapshot file)
       final INode inode = inodes[inodes.length - 1];
       assertEquals(file1.getName(), inode.getLocalName());
-      assertEquals(INodeFileWithSnapshot.class, inode.getClass());
+      assertTrue(inode.asFile().isWithSnapshot());
     }
 
     // Check the INodes for path /TestSnapshot/sub1/file1
@@ -391,6 +390,8 @@ public class TestSnapshotPathINodes {
     // The last INode should be associated with file1
     assertEquals(inodes[components.length - 1].getFullPathName(),
         file1.toString());
+    // record the modification time of the inode
+    final long modTime = inodes[inodes.length - 1].getModificationTime();
     
     // Create a snapshot for the dir, and check the inodes for the path
     // pointing to a snapshot file
@@ -414,10 +415,10 @@ public class TestSnapshotPathINodes {
     // Check the INode for snapshot of file1
     INode snapshotFileNode = ssInodes[ssInodes.length - 1]; 
     assertEquals(snapshotFileNode.getLocalName(), file1.getName());
-    assertTrue(snapshotFileNode instanceof INodeFileWithSnapshot);
+    assertTrue(snapshotFileNode.asFile().isWithSnapshot());
     // The modification time of the snapshot INode should be the same with the
     // original INode before modification
-    assertEquals(inodes[inodes.length - 1].getModificationTime(),
+    assertEquals(modTime,
         snapshotFileNode.getModificationTime(ssNodesInPath.getPathSnapshot()));
 
     // Check the INode for /TestSnapshot/sub1/file1 again
@@ -432,7 +433,6 @@ public class TestSnapshotPathINodes {
     final int last = components.length - 1;
     assertEquals(newInodes[last].getFullPathName(), file1.toString());
     // The modification time of the INode for file3 should have been changed
-    Assert.assertFalse(inodes[last].getModificationTime()
-        == newInodes[last].getModificationTime());
+    Assert.assertFalse(modTime == newInodes[last].getModificationTime());
   }
 }

Modified: hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTransferFsImage.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTransferFsImage.java?rev=1549626&r1=1549625&r2=1549626&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTransferFsImage.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTransferFsImage.java Mon Dec  9 17:38:20 2013
@@ -34,11 +34,11 @@ import javax.servlet.http.HttpServletReq
 import javax.servlet.http.HttpServletResponse;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.http.HttpServerFunctionalTest;
-import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.junit.Test;
@@ -66,8 +66,9 @@ public class TestTransferFsImage {
         new File("/xxxxx-does-not-exist/blah"));
        
     try {
-      String fsName = NetUtils.getHostPortString(
-          cluster.getNameNode().getHttpAddress());
+      URL fsName = DFSUtil.getInfoServer(
+          cluster.getNameNode().getServiceRpcAddress(), conf,
+          DFSUtil.getHttpClientScheme(conf)).toURL();
       String id = "getimage=1&txid=0";
 
       TransferFsImage.getFileClient(fsName, id, localPath, mockStorage, false);      
@@ -98,8 +99,10 @@ public class TestTransferFsImage {
         );
        
     try {
-      String fsName = NetUtils.getHostPortString(
-          cluster.getNameNode().getHttpAddress());
+      URL fsName = DFSUtil.getInfoServer(
+          cluster.getNameNode().getServiceRpcAddress(), conf,
+          DFSUtil.getHttpClientScheme(conf)).toURL();
+
       String id = "getimage=1&txid=0";
 
       TransferFsImage.getFileClient(fsName, id, localPaths, mockStorage, false);      
@@ -123,7 +126,7 @@ public class TestTransferFsImage {
       URL serverURL = HttpServerFunctionalTest.getServerURL(testServer);
       TransferFsImage.timeout = 2000;
       try {
-        TransferFsImage.getFileClient(serverURL.getAuthority(), "txid=1", null,
+        TransferFsImage.getFileClient(serverURL, "txid=1", null,
             null, false);
         fail("TransferImage Should fail with timeout");
       } catch (SocketTimeoutException e) {

Modified: hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAConfiguration.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAConfiguration.java?rev=1549626&r1=1549625&r2=1549626&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAConfiguration.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAConfiguration.java Mon Dec  9 17:38:20 2013
@@ -24,6 +24,7 @@ import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.net.URI;
+import java.net.URL;
 import java.util.Collection;
 
 import org.apache.hadoop.conf.Configuration;
@@ -86,7 +87,8 @@ public class TestHAConfiguration {
     // 0.0.0.0, it should substitute the address from the RPC configuration
     // above.
     StandbyCheckpointer checkpointer = new StandbyCheckpointer(conf, fsn);
-    assertEquals("1.2.3.2:" + DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT,
+    assertEquals(new URL("http", "1.2.3.2",
+        DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT, ""),
         checkpointer.getActiveNNAddress());
   }
   

Modified: hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestINodeFileUnderConstructionWithSnapshot.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestINodeFileUnderConstructionWithSnapshot.java?rev=1549626&r1=1549625&r2=1549626&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestINodeFileUnderConstructionWithSnapshot.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestINodeFileUnderConstructionWithSnapshot.java Mon Dec  9 17:38:20 2013
@@ -176,7 +176,7 @@ public class TestINodeFileUnderConstruct
     dirNode = (INodeDirectorySnapshottable) fsdir.getINode(dir.toString());
     last = dirNode.getDiffs().getLast();
     Snapshot s1 = last.snapshot;
-    assertTrue(fileNode instanceof INodeFileWithSnapshot);
+    assertTrue(fileNode.isWithSnapshot());
     assertEquals(BLOCKSIZE * 3, fileNode.computeFileSize(s1));
     
     // 4. modify file --> append without closing stream --> take snapshot -->

Modified: hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java?rev=1549626&r1=1549625&r2=1549626&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java Mon Dec  9 17:38:20 2013
@@ -403,8 +403,7 @@ public class TestRenameWithSnapshots {
     final Path foo_s3 = SnapshotTestHelper.getSnapshotPath(sdir1, "s3",
         "foo");
     assertFalse(hdfs.exists(foo_s3));
-    INodeFileWithSnapshot sfoo = (INodeFileWithSnapshot) fsdir.getINode(
-        newfoo.toString()).asFile();
+    INodeFile sfoo = fsdir.getINode(newfoo.toString()).asFile();
     assertEquals("s2", sfoo.getDiffs().getLastSnapshot().getRoot()
         .getLocalName());
   }
@@ -604,8 +603,7 @@ public class TestRenameWithSnapshots {
     status = hdfs.getFileStatus(foo_s2);
     assertEquals(REPL, status.getReplication());
     
-    INodeFileWithSnapshot snode = (INodeFileWithSnapshot) fsdir.getINode(
-        newfoo.toString()).asFile();
+    INodeFile snode = fsdir.getINode(newfoo.toString()).asFile();
     assertEquals(1, snode.getDiffs().asList().size());
     assertEquals("s2", snode.getDiffs().getLastSnapshot().getRoot()
         .getLocalName());
@@ -763,8 +761,7 @@ public class TestRenameWithSnapshots {
         .asDirectory();
     assertEquals(1, foo.getDiffs().asList().size());
     assertEquals("s1", foo.getLastSnapshot().getRoot().getLocalName());
-    INodeFileWithSnapshot bar1 = (INodeFileWithSnapshot) fsdir.getINode4Write(
-        bar1_dir1.toString()).asFile();
+    INodeFile bar1 = fsdir.getINode4Write(bar1_dir1.toString()).asFile();
     assertEquals(1, bar1.getDiffs().asList().size());
     assertEquals("s1", bar1.getDiffs().getLastSnapshot().getRoot()
         .getLocalName());
@@ -774,7 +771,7 @@ public class TestRenameWithSnapshots {
     INodeReference.WithCount barWithCount = (WithCount) barRef
         .getReferredINode();
     assertEquals(2, barWithCount.getReferenceCount());
-    INodeFileWithSnapshot bar = (INodeFileWithSnapshot) barWithCount.asFile();
+    INodeFile bar = barWithCount.asFile();
     assertEquals(1, bar.getDiffs().asList().size());
     assertEquals("s1", bar.getDiffs().getLastSnapshot().getRoot()
         .getLocalName());
@@ -984,8 +981,7 @@ public class TestRenameWithSnapshots {
     assertEquals("s333", fooDiffs.get(2).snapshot.getRoot().getLocalName());
     assertEquals("s22", fooDiffs.get(1).snapshot.getRoot().getLocalName());
     assertEquals("s1", fooDiffs.get(0).snapshot.getRoot().getLocalName());
-    INodeFileWithSnapshot bar1 = (INodeFileWithSnapshot) fsdir.getINode4Write(
-        bar1_dir1.toString()).asFile();
+    INodeFile bar1 = fsdir.getINode4Write(bar1_dir1.toString()).asFile();
     List<FileDiff> bar1Diffs = bar1.getDiffs().asList();
     assertEquals(3, bar1Diffs.size());
     assertEquals("s333", bar1Diffs.get(2).snapshot.getRoot().getLocalName());
@@ -997,7 +993,7 @@ public class TestRenameWithSnapshots {
     INodeReference.WithCount barWithCount = (WithCount) barRef.getReferredINode();
     // 5 references: s1, s22, s333, s2222, current tree of sdir1
     assertEquals(5, barWithCount.getReferenceCount());
-    INodeFileWithSnapshot bar = (INodeFileWithSnapshot) barWithCount.asFile();
+    INodeFile bar = barWithCount.asFile();
     List<FileDiff> barDiffs = bar.getDiffs().asList();
     assertEquals(4, barDiffs.size());
     assertEquals("s2222", barDiffs.get(3).snapshot.getRoot().getLocalName());
@@ -1047,7 +1043,7 @@ public class TestRenameWithSnapshots {
     barRef = fsdir.getINode(bar_s2222.toString()).asReference();
     barWithCount = (WithCount) barRef.getReferredINode();
     assertEquals(4, barWithCount.getReferenceCount());
-    bar = (INodeFileWithSnapshot) barWithCount.asFile();
+    bar = barWithCount.asFile();
     barDiffs = bar.getDiffs().asList();
     assertEquals(4, barDiffs.size());
     assertEquals("s2222", barDiffs.get(3).snapshot.getRoot().getLocalName());
@@ -1229,7 +1225,7 @@ public class TestRenameWithSnapshots {
       fooRef = fsdir.getINode4Write(foo2.toString());
       assertTrue(fooRef instanceof INodeReference.DstReference);
       INodeFile fooNode = fooRef.asFile();
-      assertTrue(fooNode instanceof INodeFileWithSnapshot);
+      assertTrue(fooNode.isWithSnapshot());
       assertTrue(fooNode.isUnderConstruction());
     } finally {
       if (out != null) {
@@ -1240,7 +1236,7 @@ public class TestRenameWithSnapshots {
     fooRef = fsdir.getINode4Write(foo2.toString());
     assertTrue(fooRef instanceof INodeReference.DstReference);
     INodeFile fooNode = fooRef.asFile();
-    assertTrue(fooNode instanceof INodeFileWithSnapshot);
+    assertTrue(fooNode.isWithSnapshot());
     assertFalse(fooNode.isUnderConstruction());
     
     restartClusterAndCheckImage(true);
@@ -1715,8 +1711,7 @@ public class TestRenameWithSnapshots {
     assertTrue(diff.getChildrenDiff().getList(ListType.CREATED).isEmpty());
     
     // bar was converted to filewithsnapshot while renaming
-    INodeFileWithSnapshot barNode = (INodeFileWithSnapshot) fsdir
-        .getINode4Write(bar.toString());
+    INodeFile barNode = fsdir.getINode4Write(bar.toString()).asFile();
     assertSame(barNode, children.get(0));
     assertSame(fooNode, barNode.getParent());
     List<FileDiff> barDiffList = barNode.getDiffs().asList();

Modified: hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java?rev=1549626&r1=1549625&r2=1549626&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java Mon Dec  9 17:38:20 2013
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.na
 
 import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
@@ -167,7 +168,8 @@ public class TestSnapshotBlocksMap {
       Assert.assertSame(INodeFile.class, f1.getClass());
       hdfs.setReplication(file1, (short)2);
       f1 = assertBlockCollection(file1.toString(), 2, fsdir, blockmanager);
-      Assert.assertSame(INodeFileWithSnapshot.class, f1.getClass());
+      assertTrue(f1.isWithSnapshot());
+      assertFalse(f1.isUnderConstruction());
     }
     
     // Check the block information for file0

Modified: hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java?rev=1549626&r1=1549625&r2=1549626&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java Mon Dec  9 17:38:20 2013
@@ -277,10 +277,10 @@ public class TestSnapshotDeletion {
     TestSnapshotBlocksMap.assertBlockCollection(new Path(snapshotNoChangeDir,
         noChangeFileSCopy.getLocalName()).toString(), 1, fsdir, blockmanager);
     
-    INodeFileWithSnapshot metaChangeFile2SCopy = 
-        (INodeFileWithSnapshot) children.get(0);
+    INodeFile metaChangeFile2SCopy = children.get(0).asFile();
     assertEquals(metaChangeFile2.getName(), metaChangeFile2SCopy.getLocalName());
-    assertEquals(INodeFileWithSnapshot.class, metaChangeFile2SCopy.getClass());
+    assertTrue(metaChangeFile2SCopy.isWithSnapshot());
+    assertFalse(metaChangeFile2SCopy.isUnderConstruction());
     TestSnapshotBlocksMap.assertBlockCollection(new Path(snapshotNoChangeDir,
         metaChangeFile2SCopy.getLocalName()).toString(), 1, fsdir, blockmanager);
     
@@ -338,8 +338,9 @@ public class TestSnapshotDeletion {
     INode child = children.get(0);
     assertEquals(child.getLocalName(), metaChangeFile1.getName());
     // check snapshot copy of metaChangeFile1
-    assertEquals(INodeFileWithSnapshot.class, child.getClass());
-    INodeFileWithSnapshot metaChangeFile1SCopy = (INodeFileWithSnapshot) child;
+    INodeFile metaChangeFile1SCopy = child.asFile();
+    assertTrue(metaChangeFile1SCopy.isWithSnapshot());
+    assertFalse(metaChangeFile1SCopy.isUnderConstruction());
     assertEquals(REPLICATION_1,
         metaChangeFile1SCopy.getFileReplication(null));
     assertEquals(REPLICATION_1,

Modified: hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java?rev=1549626&r1=1549625&r2=1549626&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java Mon Dec  9 17:38:20 2013
@@ -261,6 +261,34 @@ public class TestWebHDFS {
     }
   }
 
+  @Test(timeout=300000)
+  public void testNumericalUserName() throws Exception {
+    final Configuration conf = WebHdfsTestUtil.createConf();
+    conf.set(DFSConfigKeys.DFS_WEBHDFS_USER_PATTERN_KEY, "^[A-Za-z0-9_][A-Za-z0-9._-]*[$]?$");
+    final MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    try {
+      cluster.waitActive();
+      WebHdfsTestUtil.getWebHdfsFileSystem(conf, WebHdfsFileSystem.SCHEME)
+          .setPermission(new Path("/"),
+              new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL));
+
+      UserGroupInformation.createUserForTesting("123", new String[]{"my-group"})
+        .doAs(new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws IOException, URISyntaxException {
+            FileSystem fs = WebHdfsTestUtil.getWebHdfsFileSystem(conf,
+                WebHdfsFileSystem.SCHEME);
+            Path d = new Path("/my-dir");
+            Assert.assertTrue(fs.mkdirs(d));
+            return null;
+          }
+        });
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
   /**
    * WebHdfs should be enabled by default after HDFS-5532
    * 

Modified: hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java?rev=1549626&r1=1549625&r2=1549626&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java Mon Dec  9 17:38:20 2013
@@ -285,4 +285,19 @@ public class TestParam {
       Assert.assertEquals(expected, computed.getValue());
     }
   }
+
+  @Test
+  public void testUserNameOkAfterResettingPattern() {
+    UserParam.Domain oldDomain = UserParam.getUserPatternDomain();
+
+    String newPattern = "^[A-Za-z0-9_][A-Za-z0-9._-]*[$]?$";
+    UserParam.setUserPattern(newPattern);
+
+    UserParam userParam = new UserParam("1x");
+    assertNotNull(userParam.getValue());
+    userParam = new UserParam("123");
+    assertNotNull(userParam.getValue());
+
+    UserParam.setUserPatternDomain(oldDomain);
+  }
 }