You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ar...@apache.org on 2013/08/21 19:47:15 UTC

svn commit: r1516230 [2/2] - in /hadoop/common/branches/HDFS-2832/hadoop-common-project: hadoop-auth/ hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/ hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/ hadoo...

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Globber.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Globber.java?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Globber.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Globber.java Wed Aug 21 17:47:10 2013
@@ -62,6 +62,18 @@ class Globber {
     }
   }
 
+  private FileStatus getFileLinkStatus(Path path) {
+    try {
+      if (fs != null) {
+        return fs.getFileLinkStatus(path);
+      } else {
+        return fc.getFileLinkStatus(path);
+      }
+    } catch (IOException e) {
+      return null;
+    }
+  }
+
   private FileStatus[] listStatus(Path path) {
     try {
       if (fs != null) {
@@ -99,29 +111,41 @@ class Globber {
   }
 
   private String schemeFromPath(Path path) throws IOException {
-    String scheme = pathPattern.toUri().getScheme();
+    String scheme = path.toUri().getScheme();
     if (scheme == null) {
       if (fs != null) {
         scheme = fs.getUri().getScheme();
       } else {
-        scheme = fc.getFSofPath(path).getUri().getScheme();
+        scheme = fc.getDefaultFileSystem().getUri().getScheme();
       }
     }
     return scheme;
   }
 
   private String authorityFromPath(Path path) throws IOException {
-    String authority = pathPattern.toUri().getAuthority();
+    String authority = path.toUri().getAuthority();
     if (authority == null) {
       if (fs != null) {
         authority = fs.getUri().getAuthority();
       } else {
-        authority = fc.getFSofPath(path).getUri().getAuthority();
+        authority = fc.getDefaultFileSystem().getUri().getAuthority();
       }
     }
     return authority ;
   }
 
+  /**
+   * The glob filter builds a regexp per path component.  If the component
+   * does not contain a shell metachar, then it falls back to appending the
+   * raw string to the list of built up paths.  This raw path needs to have
+   * the quoting removed.  Ie. convert all occurrences of "\X" to "X"
+   * @param name of the path component
+   * @return the unquoted path component
+   */
+  private static String unquotePathComponent(String name) {
+    return name.replaceAll("\\\\(.)", "$1");
+  }
+
   public FileStatus[] glob() throws IOException {
     // First we get the scheme and authority of the pattern that was passed
     // in.
@@ -176,14 +200,30 @@ class Globber {
               resolvedCandidate.isDirectory() == false) {
             continue;
           }
-          FileStatus[] children = listStatus(candidate.getPath());
-          for (FileStatus child : children) {
-            // Set the child path based on the parent path.
-            // This keeps the symlinks in our path.
-            child.setPath(new Path(candidate.getPath(),
-                    child.getPath().getName()));
-            if (globFilter.accept(child.getPath())) {
-              newCandidates.add(child);
+          // For components without pattern, we get its FileStatus directly
+          // using getFileLinkStatus for two reasons:
+          // 1. It should be faster to only get FileStatus needed rather than
+          //    get all children.
+          // 2. Some special filesystem directories (e.g. HDFS snapshot
+          //    directories) are not returned by listStatus, but do exist if
+          //    checked explicitly via getFileLinkStatus.
+          if (globFilter.hasPattern()) {
+            FileStatus[] children = listStatus(candidate.getPath());
+            for (FileStatus child : children) {
+              // Set the child path based on the parent path.
+              // This keeps the symlinks in our path.
+              child.setPath(new Path(candidate.getPath(),
+                      child.getPath().getName()));
+              if (globFilter.accept(child.getPath())) {
+                newCandidates.add(child);
+              }
+            }
+          } else {
+            Path p = new Path(candidate.getPath(), unquotePathComponent(component));
+            FileStatus s = getFileLinkStatus(p);
+            if (s != null) {
+              s.setPath(p);
+              newCandidates.add(s);
             }
           }
         }

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HardLink.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HardLink.java?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HardLink.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HardLink.java Wed Aug 21 17:47:10 2013
@@ -41,15 +41,6 @@ import org.apache.hadoop.util.Shell;
  */
 public class HardLink { 
 
-  public enum OSType {
-    OS_TYPE_UNIX,
-    OS_TYPE_WIN,
-    OS_TYPE_SOLARIS,
-    OS_TYPE_MAC,
-    OS_TYPE_FREEBSD
-  }
-  
-  public static OSType osType;
   private static HardLinkCommandGetter getHardLinkCommand;
   
   public final LinkStats linkStats; //not static
@@ -57,19 +48,18 @@ public class HardLink { 
   //initialize the command "getters" statically, so can use their 
   //methods without instantiating the HardLink object
   static { 
-    osType = getOSType();
-    if (osType == OSType.OS_TYPE_WIN) {
+    if (Shell.WINDOWS) {
       // Windows
       getHardLinkCommand = new HardLinkCGWin();
     } else {
-      // Unix
+      // Unix or Linux
       getHardLinkCommand = new HardLinkCGUnix();
       //override getLinkCountCommand for the particular Unix variant
       //Linux is already set as the default - {"stat","-c%h", null}
-      if (osType == OSType.OS_TYPE_MAC || osType == OSType.OS_TYPE_FREEBSD) {
+      if (Shell.MAC || Shell.FREEBSD) {
         String[] linkCountCmdTemplate = {"/usr/bin/stat","-f%l", null};
         HardLinkCGUnix.setLinkCountCmdTemplate(linkCountCmdTemplate);
-      } else if (osType == OSType.OS_TYPE_SOLARIS) {
+      } else if (Shell.SOLARIS) {
         String[] linkCountCmdTemplate = {"ls","-l", null};
         HardLinkCGUnix.setLinkCountCmdTemplate(linkCountCmdTemplate);        
       }
@@ -80,26 +70,6 @@ public class HardLink { 
     linkStats = new LinkStats();
   }
   
-  static private OSType getOSType() {
-    String osName = System.getProperty("os.name");
-    if (Shell.WINDOWS) {
-      return OSType.OS_TYPE_WIN;
-    }
-    else if (osName.contains("SunOS") 
-            || osName.contains("Solaris")) {
-       return OSType.OS_TYPE_SOLARIS;
-    }
-    else if (osName.contains("Mac")) {
-       return OSType.OS_TYPE_MAC;
-    }
-    else if (osName.contains("FreeBSD")) {
-       return OSType.OS_TYPE_FREEBSD;
-    }
-    else {
-      return OSType.OS_TYPE_UNIX;
-    }
-  }
-  
   /**
    * This abstract class bridges the OS-dependent implementations of the 
    * needed functionality for creating hardlinks and querying link counts.
@@ -548,7 +518,7 @@ public class HardLink { 
       if (inpMsg == null || exitValue != 0) {
         throw createIOException(fileName, inpMsg, errMsg, exitValue, null);
       }
-      if (osType == OSType.OS_TYPE_SOLARIS) {
+      if (Shell.SOLARIS) {
         String[] result = inpMsg.split("\\s+");
         return Integer.parseInt(result[1]);
       } else {

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java Wed Aug 21 17:47:10 2013
@@ -51,6 +51,7 @@ import org.apache.hadoop.util.StringUtil
 public class RawLocalFileSystem extends FileSystem {
   static final URI NAME = URI.create("file:///");
   private Path workingDir;
+  private static final boolean useDeprecatedFileStatus = !Stat.isAvailable();
   
   public RawLocalFileSystem() {
     workingDir = getInitialWorkingDirectory();
@@ -385,8 +386,11 @@ public class RawLocalFileSystem extends 
       throw new FileNotFoundException("File " + f + " does not exist");
     }
     if (localf.isFile()) {
+      if (!useDeprecatedFileStatus) {
+        return new FileStatus[] { getFileStatus(f) };
+      }
       return new FileStatus[] {
-        new RawLocalFileStatus(localf, getDefaultBlockSize(f), this) };
+        new DeprecatedRawLocalFileStatus(localf, getDefaultBlockSize(f), this)};
     }
 
     File[] names = localf.listFiles();
@@ -516,15 +520,22 @@ public class RawLocalFileSystem extends 
   
   @Override
   public FileStatus getFileStatus(Path f) throws IOException {
+    return getFileLinkStatusInternal(f, true);
+  }
+
+  @Deprecated
+  private FileStatus deprecatedGetFileStatus(Path f) throws IOException {
     File path = pathToFile(f);
     if (path.exists()) {
-      return new RawLocalFileStatus(pathToFile(f), getDefaultBlockSize(f), this);
+      return new DeprecatedRawLocalFileStatus(pathToFile(f),
+          getDefaultBlockSize(f), this);
     } else {
       throw new FileNotFoundException("File " + f + " does not exist");
     }
   }
 
-  static class RawLocalFileStatus extends FileStatus {
+  @Deprecated
+  static class DeprecatedRawLocalFileStatus extends FileStatus {
     /* We can add extra fields here. It breaks at least CopyFiles.FilePair().
      * We recognize if the information is already loaded by check if
      * onwer.equals("").
@@ -533,7 +544,7 @@ public class RawLocalFileSystem extends 
       return !super.getOwner().isEmpty(); 
     }
     
-    RawLocalFileStatus(File f, long defaultBlockSize, FileSystem fs) { 
+    DeprecatedRawLocalFileStatus(File f, long defaultBlockSize, FileSystem fs) {
       super(f.length(), f.isDirectory(), 1, defaultBlockSize,
           f.lastModified(), new Path(f.getPath()).makeQualified(fs.getUri(),
             fs.getWorkingDirectory()));
@@ -699,7 +710,7 @@ public class RawLocalFileSystem extends 
    */
   @Override
   public FileStatus getFileLinkStatus(final Path f) throws IOException {
-    FileStatus fi = getFileLinkStatusInternal(f);
+    FileStatus fi = getFileLinkStatusInternal(f, false);
     // getFileLinkStatus is supposed to return a symlink with a
     // qualified path
     if (fi.isSymlink()) {
@@ -710,7 +721,35 @@ public class RawLocalFileSystem extends 
     return fi;
   }
 
-  private FileStatus getFileLinkStatusInternal(final Path f) throws IOException {
+  /**
+   * Public {@link FileStatus} methods delegate to this function, which in turn
+   * either call the new {@link Stat} based implementation or the deprecated
+   * methods based on platform support.
+   * 
+   * @param f Path to stat
+   * @param dereference whether to dereference the final path component if a
+   *          symlink
+   * @return FileStatus of f
+   * @throws IOException
+   */
+  private FileStatus getFileLinkStatusInternal(final Path f,
+      boolean dereference) throws IOException {
+    if (!useDeprecatedFileStatus) {
+      return getNativeFileLinkStatus(f, dereference);
+    } else if (dereference) {
+      return deprecatedGetFileStatus(f);
+    } else {
+      return deprecatedGetFileLinkStatusInternal(f);
+    }
+  }
+
+  /**
+   * Deprecated. Remains for legacy support. Should be removed when {@link Stat}
+   * gains support for Windows and other operating systems.
+   */
+  @Deprecated
+  private FileStatus deprecatedGetFileLinkStatusInternal(final Path f)
+      throws IOException {
     String target = FileUtil.readLink(new File(f.toString()));
 
     try {
@@ -746,10 +785,31 @@ public class RawLocalFileSystem extends 
       throw e;
     }
   }
+  /**
+   * Calls out to platform's native stat(1) implementation to get file metadata
+   * (permissions, user, group, atime, mtime, etc). This works around the lack
+   * of lstat(2) in Java 6.
+   * 
+   *  Currently, the {@link Stat} class used to do this only supports Linux
+   *  and FreeBSD, so the old {@link #deprecatedGetFileLinkStatusInternal(Path)}
+   *  implementation (deprecated) remains further OS support is added.
+   *
+   * @param f File to stat
+   * @param dereference whether to dereference symlinks
+   * @return FileStatus of f
+   * @throws IOException
+   */
+  private FileStatus getNativeFileLinkStatus(final Path f,
+      boolean dereference) throws IOException {
+    checkPath(f);
+    Stat stat = new Stat(f, getDefaultBlockSize(f), dereference, this);
+    FileStatus status = stat.getFileStatus();
+    return status;
+  }
 
   @Override
   public Path getLinkTarget(Path f) throws IOException {
-    FileStatus fi = getFileLinkStatusInternal(f);
+    FileStatus fi = getFileLinkStatusInternal(f, false);
     // return an unqualified symlink target
     return fi.getSymlink();
   }

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/local/RawLocalFs.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/local/RawLocalFs.java?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/local/RawLocalFs.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/local/RawLocalFs.java Wed Aug 21 17:47:10 2013
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.fs.local;
 
-import java.io.File;
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
@@ -28,13 +26,9 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.AbstractFileSystem;
 import org.apache.hadoop.fs.DelegateToFileSystem;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FsConstants;
 import org.apache.hadoop.fs.FsServerDefaults;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RawLocalFileSystem;
-import org.apache.hadoop.fs.permission.FsPermission;
 
 /**
  * The RawLocalFs implementation of AbstractFileSystem.
@@ -72,90 +66,12 @@ public class RawLocalFs extends Delegate
   public FsServerDefaults getServerDefaults() throws IOException {
     return LocalConfigKeys.getServerDefaults();
   }
-  
-  @Override
-  public boolean supportsSymlinks() {
-    return true;
-  }
-
-  @Override
-  public void createSymlink(Path target, Path link, boolean createParent)
-      throws IOException {
-    final String targetScheme = target.toUri().getScheme();
-    if (targetScheme != null && !"file".equals(targetScheme)) {
-      throw new IOException("Unable to create symlink to non-local file "+
-          "system: "+target.toString());
-    }
-
-    if (createParent) {
-      mkdir(link.getParent(), FsPermission.getDirDefault(), true);
-    }
-
-    // NB: Use createSymbolicLink in java.nio.file.Path once available
-    int result = FileUtil.symLink(target.toString(), link.toString());
-    if (result != 0) {
-      throw new IOException("Error " + result + " creating symlink " +
-          link + " to " + target);
-    }
-  }
 
-  /**
-   * Return a FileStatus representing the given path. If the path refers 
-   * to a symlink return a FileStatus representing the link rather than
-   * the object the link refers to.
-   */
-  @Override
-  public FileStatus getFileLinkStatus(final Path f) throws IOException {
-    String target = FileUtil.readLink(new File(f.toString()));
-    try {
-      FileStatus fs = getFileStatus(f);
-      // If f refers to a regular file or directory      
-      if (target.isEmpty()) {
-        return fs;
-      }
-      // Otherwise f refers to a symlink
-      return new FileStatus(fs.getLen(), 
-          false,
-          fs.getReplication(), 
-          fs.getBlockSize(),
-          fs.getModificationTime(),
-          fs.getAccessTime(),
-          fs.getPermission(),
-          fs.getOwner(),
-          fs.getGroup(),
-          new Path(target),
-          f);
-    } catch (FileNotFoundException e) {
-      /* The exists method in the File class returns false for dangling 
-       * links so we can get a FileNotFoundException for links that exist.
-       * It's also possible that we raced with a delete of the link. Use
-       * the readBasicFileAttributes method in java.nio.file.attributes 
-       * when available.
-       */
-      if (!target.isEmpty()) {
-        return new FileStatus(0, false, 0, 0, 0, 0, FsPermission.getDefault(), 
-            "", "", new Path(target), f);        
-      }
-      // f refers to a file or directory that does not exist
-      throw e;
-    }
-  }
-  
-   @Override
-   public boolean isValidName(String src) {
-     // Different local file systems have different validation rules.  Skip
-     // validation here and just let the OS handle it.  This is consistent with
-     // RawLocalFileSystem.
-     return true;
-   }
-  
   @Override
-  public Path getLinkTarget(Path f) throws IOException {
-    /* We should never get here. Valid local links are resolved transparently
-     * by the underlying local file system and accessing a dangling link will 
-     * result in an IOException, not an UnresolvedLinkException, so FileContext
-     * should never call this function.
-     */
-    throw new AssertionError();
+  public boolean isValidName(String src) {
+    // Different local file systems have different validation rules. Skip
+    // validation here and just let the OS handle it. This is consistent with
+    // RawLocalFileSystem.
+    return true;
   }
 }

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java Wed Aug 21 17:47:10 2013
@@ -133,7 +133,8 @@ class CopyCommands {  
       "Copy files that match the file pattern <src> to a\n" +
       "destination.  When copying multiple files, the destination\n" +
       "must be a directory. Passing -p preserves access and\n" +
-      "modification times, ownership and the mode.\n";
+      "modification times, ownership and the mode. Passing -f\n" +
+      "overwrites the destination if it already exists.\n";
     
     @Override
     protected void processOptions(LinkedList<String> args) throws IOException {
@@ -186,7 +187,8 @@ class CopyCommands {  
       "into fs. Copying fails if the file already\n" +
       "exists, unless the -f flag is given. Passing\n" +
       "-p preserves access and modification times,\n" +
-      "ownership and the mode.\n";
+      "ownership and the mode. Passing -f overwrites\n" +
+      "the destination if it already exists.\n";
 
     @Override
     protected void processOptions(LinkedList<String> args) throws IOException {

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/SnapshotCommands.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/SnapshotCommands.java?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/SnapshotCommands.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/SnapshotCommands.java Wed Aug 21 17:47:10 2013
@@ -79,7 +79,7 @@ class SnapshotCommands extends FsCommand
     protected void processArguments(LinkedList<PathData> items)
     throws IOException {
       super.processArguments(items);
-      if (exitCode != 0) { // check for error collecting paths
+      if (numErrors != 0) { // check for error collecting paths
         return;
       }
       assert(items.size() == 1);
@@ -119,7 +119,7 @@ class SnapshotCommands extends FsCommand
     protected void processArguments(LinkedList<PathData> items)
         throws IOException {
       super.processArguments(items);
-      if (exitCode != 0) { // check for error collecting paths
+      if (numErrors != 0) { // check for error collecting paths
         return;
       }
       assert (items.size() == 1);
@@ -160,7 +160,7 @@ class SnapshotCommands extends FsCommand
     protected void processArguments(LinkedList<PathData> items)
         throws IOException {
       super.processArguments(items);
-      if (exitCode != 0) { // check for error collecting paths
+      if (numErrors != 0) { // check for error collecting paths
         return;
       }
       Preconditions.checkArgument(items.size() == 1);

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java Wed Aug 21 17:47:10 2013
@@ -47,6 +47,7 @@ import javax.servlet.http.HttpServletRes
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.ConfServlet;
@@ -119,18 +120,117 @@ public class HttpServer implements Filte
   protected final Map<Context, Boolean> defaultContexts =
       new HashMap<Context, Boolean>();
   protected final List<String> filterNames = new ArrayList<String>();
-  private static final int MAX_RETRIES = 10;
   static final String STATE_DESCRIPTION_ALIVE = " - alive";
   static final String STATE_DESCRIPTION_NOT_LIVE = " - not live";
 
   private final boolean listenerStartedExternally;
   
+  /**
+   * Class to construct instances of HTTP server with specific options.
+   */
+  public static class Builder {
+    String name;
+    String bindAddress;
+    Integer port;
+    Boolean findPort;
+    Configuration conf;
+    Connector connector;
+    String[] pathSpecs;
+    AccessControlList adminsAcl;
+    boolean securityEnabled = false;
+    String usernameConfKey = null;
+    String keytabConfKey = null;
+    
+    public Builder setName(String name){
+      this.name = name;
+      return this;
+    }
+    
+    public Builder setBindAddress(String bindAddress){
+      this.bindAddress = bindAddress;
+      return this;
+    }
+    
+    public Builder setPort(int port) {
+      this.port = port;
+      return this;
+    }
+    
+    public Builder setFindPort(boolean findPort) {
+      this.findPort = findPort;
+      return this;
+    }
+    
+    public Builder setConf(Configuration conf) {
+      this.conf = conf;
+      return this;
+    }
+    
+    public Builder setConnector(Connector connector) {
+      this.connector = connector;
+      return this;
+    }
+    
+    public Builder setPathSpec(String[] pathSpec) {
+      this.pathSpecs = pathSpec;
+      return this;
+    }
+    
+    public Builder setACL(AccessControlList acl) {
+      this.adminsAcl = acl;
+      return this;
+    }
+    
+    public Builder setSecurityEnabled(boolean securityEnabled) {
+      this.securityEnabled = securityEnabled;
+      return this;
+    }
+    
+    public Builder setUsernameConfKey(String usernameConfKey) {
+      this.usernameConfKey = usernameConfKey;
+      return this;
+    }
+    
+    public Builder setKeytabConfKey(String keytabConfKey) {
+      this.keytabConfKey = keytabConfKey;
+      return this;
+    }
+    
+    public HttpServer build() throws IOException {
+      if (this.name == null) {
+        throw new HadoopIllegalArgumentException("name is not set");
+      }
+      if (this.bindAddress == null) {
+        throw new HadoopIllegalArgumentException("bindAddress is not set");
+      }
+      if (this.port == null) {
+        throw new HadoopIllegalArgumentException("port is not set");
+      }
+      if (this.findPort == null) {
+        throw new HadoopIllegalArgumentException("findPort is not set");
+      }
+      
+      if (this.conf == null) {
+        conf = new Configuration();
+      }
+      
+      HttpServer server = new HttpServer(this.name, this.bindAddress, this.port,
+      this.findPort, this.conf, this.adminsAcl, this.connector, this.pathSpecs);
+      if (this.securityEnabled) {
+        server.initSpnego(this.conf, this.usernameConfKey, this.keytabConfKey);
+      }
+      return server;
+    }
+  }
+  
   /** Same as this(name, bindAddress, port, findPort, null); */
+  @Deprecated
   public HttpServer(String name, String bindAddress, int port, boolean findPort
       ) throws IOException {
     this(name, bindAddress, port, findPort, new Configuration());
   }
-
+  
+  @Deprecated
   public HttpServer(String name, String bindAddress, int port,
       boolean findPort, Configuration conf, Connector connector) throws IOException {
     this(name, bindAddress, port, findPort, conf, null, connector, null);
@@ -150,6 +250,7 @@ public class HttpServer implements Filte
    * @param pathSpecs Path specifications that this httpserver will be serving. 
    *        These will be added to any filters.
    */
+  @Deprecated
   public HttpServer(String name, String bindAddress, int port,
       boolean findPort, Configuration conf, String[] pathSpecs) throws IOException {
     this(name, bindAddress, port, findPort, conf, null, null, pathSpecs);
@@ -164,11 +265,13 @@ public class HttpServer implements Filte
    *        increment by 1 until it finds a free port.
    * @param conf Configuration 
    */
+  @Deprecated
   public HttpServer(String name, String bindAddress, int port,
       boolean findPort, Configuration conf) throws IOException {
     this(name, bindAddress, port, findPort, conf, null, null, null);
   }
 
+  @Deprecated
   public HttpServer(String name, String bindAddress, int port,
       boolean findPort, Configuration conf, AccessControlList adminsAcl) 
       throws IOException {
@@ -186,6 +289,7 @@ public class HttpServer implements Filte
    * @param conf Configuration 
    * @param adminsAcl {@link AccessControlList} of the admins
    */
+  @Deprecated
   public HttpServer(String name, String bindAddress, int port,
       boolean findPort, Configuration conf, AccessControlList adminsAcl, 
       Connector connector) throws IOException {
@@ -529,7 +633,7 @@ public class HttpServer implements Filte
   /**
    * Define a filter for a context and set up default url mappings.
    */
-  protected void defineFilter(Context ctx, String name,
+  public void defineFilter(Context ctx, String name,
       String classname, Map<String,String> parameters, String[] urls) {
 
     FilterHolder holder = new FilterHolder();
@@ -569,6 +673,10 @@ public class HttpServer implements Filte
   public Object getAttribute(String name) {
     return webAppContext.getAttribute(name);
   }
+  
+  public WebAppContext getWebAppContext(){
+    return this.webAppContext;
+  }
 
   /**
    * Get the pathname to the webapps files.

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java Wed Aug 21 17:47:10 2013
@@ -136,8 +136,6 @@ public class RetryInvocationHandler<T> i
             msg += ". Trying to fail over " + formatSleepMessage(action.delayMillis);
             if (LOG.isDebugEnabled()) {
               LOG.debug(msg, e);
-            } else {
-              LOG.warn(msg);
             }
           } else {
             if(LOG.isDebugEnabled()) {

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java Wed Aug 21 17:47:10 2013
@@ -1311,7 +1311,15 @@ public abstract class Server {
           Throwable cause = e;
           while (cause != null) {
             if (cause instanceof InvalidToken) {
-              sendToClient = (InvalidToken) cause;
+              // FIXME: hadoop method signatures are restricting the SASL
+              // callbacks to only returning InvalidToken, but some services
+              // need to throw other exceptions (ex. NN + StandyException),
+              // so for now we'll tunnel the real exceptions via an
+              // InvalidToken's cause which normally is not set 
+              if (cause.getCause() != null) {
+                cause = cause.getCause();
+              }
+              sendToClient = (IOException) cause;
               break;
             }
             cause = cause.getCause();

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/DefaultMetricsSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/DefaultMetricsSystem.java?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/DefaultMetricsSystem.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/DefaultMetricsSystem.java Wed Aug 21 17:47:10 2013
@@ -46,8 +46,8 @@ public enum DefaultMetricsSystem {
   @VisibleForTesting
   volatile boolean miniClusterMode = false;
   
-  final UniqueNames mBeanNames = new UniqueNames();
-  final UniqueNames sourceNames = new UniqueNames();
+  transient final UniqueNames mBeanNames = new UniqueNames();
+  transient final UniqueNames sourceNames = new UniqueNames();
 
   /**
    * Convenience method to initialize the metrics system

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcServer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcServer.java?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcServer.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcServer.java Wed Aug 21 17:47:10 2013
@@ -104,7 +104,7 @@ public class SaslRpcServer {
         if (LOG.isDebugEnabled())
           LOG.debug("Kerberos principal name is " + fullName);
         // don't use KerberosName because we don't want auth_to_local
-        String[] parts = fullName.split("[/@]", 2);
+        String[] parts = fullName.split("[/@]", 3);
         protocol = parts[0];
         // should verify service host is present here rather than in create()
         // but lazy tests are using a UGI that isn't a SPN...
@@ -127,7 +127,6 @@ public class SaslRpcServer {
     final CallbackHandler callback;
     switch (authMethod) {
       case TOKEN: {
-        secretManager.checkAvailableForRead();
         callback = new SaslDigestCallbackHandler(secretManager, connection);
         break;
       }

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java Wed Aug 21 17:47:10 2013
@@ -58,6 +58,45 @@ abstract public class Shell {
   /** Windows CreateProcess synchronization object */
   public static final Object WindowsProcessLaunchLock = new Object();
 
+  // OSType detection
+
+  public enum OSType {
+    OS_TYPE_LINUX,
+    OS_TYPE_WIN,
+    OS_TYPE_SOLARIS,
+    OS_TYPE_MAC,
+    OS_TYPE_FREEBSD,
+    OS_TYPE_OTHER
+  }
+
+  public static final OSType osType = getOSType();
+
+  static private OSType getOSType() {
+    String osName = System.getProperty("os.name");
+    if (osName.startsWith("Windows")) {
+      return OSType.OS_TYPE_WIN;
+    } else if (osName.contains("SunOS") || osName.contains("Solaris")) {
+      return OSType.OS_TYPE_SOLARIS;
+    } else if (osName.contains("Mac")) {
+      return OSType.OS_TYPE_MAC;
+    } else if (osName.contains("FreeBSD")) {
+      return OSType.OS_TYPE_FREEBSD;
+    } else if (osName.startsWith("Linux")) {
+      return OSType.OS_TYPE_LINUX;
+    } else {
+      // Some other form of Unix
+      return OSType.OS_TYPE_OTHER;
+    }
+  }
+
+  // Helper static vars for each platform
+  public static final boolean WINDOWS = (osType == OSType.OS_TYPE_WIN);
+  public static final boolean SOLARIS = (osType == OSType.OS_TYPE_SOLARIS);
+  public static final boolean MAC     = (osType == OSType.OS_TYPE_MAC);
+  public static final boolean FREEBSD = (osType == OSType.OS_TYPE_FREEBSD);
+  public static final boolean LINUX   = (osType == OSType.OS_TYPE_LINUX);
+  public static final boolean OTHER   = (osType == OSType.OS_TYPE_OTHER);
+
   /** a Unix command to get the current user's groups list */
   public static String[] getGroupsCommand() {
     return (WINDOWS)? new String[]{"cmd", "/c", "groups"}
@@ -282,13 +321,6 @@ abstract public class Shell {
     return exeFile.getCanonicalPath();
   }
 
-  /** Set to true on Windows platforms */
-  public static final boolean WINDOWS /* borrowed from Path.WINDOWS */
-                = System.getProperty("os.name").startsWith("Windows");
-
-  public static final boolean LINUX
-                = System.getProperty("os.name").startsWith("Linux");
-  
   /** a Windows utility to emulate Unix commands */
   public static final String WINUTILS = getWinUtilsPath();
 
@@ -336,6 +368,7 @@ abstract public class Shell {
 
   private long    interval;   // refresh interval in msec
   private long    lastTime;   // last time the command was performed
+  final private boolean redirectErrorStream; // merge stdout and stderr
   private Map<String, String> environment; // env for the command execution
   private File dir;
   private Process process; // sub process used to execute the command
@@ -348,13 +381,18 @@ abstract public class Shell {
     this(0L);
   }
   
+  public Shell(long interval) {
+    this(interval, false);
+  }
+
   /**
    * @param interval the minimum duration to wait before re-executing the 
    *        command.
    */
-  public Shell( long interval ) {
+  public Shell(long interval, boolean redirectErrorStream) {
     this.interval = interval;
     this.lastTime = (interval<0) ? 0 : -interval;
+    this.redirectErrorStream = redirectErrorStream;
   }
   
   /** set the environment for the command 
@@ -393,6 +431,8 @@ abstract public class Shell {
     if (dir != null) {
       builder.directory(this.dir);
     }
+
+    builder.redirectErrorStream(redirectErrorStream);
     
     if (Shell.WINDOWS) {
       synchronized (WindowsProcessLaunchLock) {

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/VersionInfo.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/VersionInfo.java?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/VersionInfo.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/VersionInfo.java Wed Aug 21 17:47:10 2013
@@ -90,6 +90,10 @@ public class VersionInfo {
       " source checksum " + _getSrcChecksum();
   }
 
+  protected String _getProtocVersion() {
+    return info.getProperty("protocVersion", "Unknown");
+  }
+
   private static VersionInfo COMMON_VERSION_INFO = new VersionInfo("common");
   /**
    * Get the Hadoop version.
@@ -153,12 +157,20 @@ public class VersionInfo {
   public static String getBuildVersion(){
     return COMMON_VERSION_INFO._getBuildVersion();
   }
-  
+
+  /**
+   * Returns the protoc version used for the build.
+   */
+  public static String getProtocVersion(){
+    return COMMON_VERSION_INFO._getProtocVersion();
+  }
+
   public static void main(String[] args) {
     LOG.debug("version: "+ getVersion());
     System.out.println("Hadoop " + getVersion());
     System.out.println("Subversion " + getUrl() + " -r " + getRevision());
     System.out.println("Compiled by " + getUser() + " on " + getDate());
+    System.out.println("Compiled with protoc " + getProtocVersion());
     System.out.println("From source with checksum " + getSrcChecksum());
     System.out.println("This command was run using " + 
         ClassUtil.findContainingJar(VersionInfo.class));

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/resources/common-version-info.properties
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/resources/common-version-info.properties?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/resources/common-version-info.properties (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/resources/common-version-info.properties Wed Aug 21 17:47:10 2013
@@ -23,3 +23,4 @@ user=${user.name}
 date=${version-info.build.time}
 url=${version-info.scm.uri}
 srcChecksum=${version-info.source.md5}
+protocVersion=${protobuf.version}
\ No newline at end of file

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/site/apt/CLIMiniCluster.apt.vm
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/site/apt/CLIMiniCluster.apt.vm?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/site/apt/CLIMiniCluster.apt.vm (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/site/apt/CLIMiniCluster.apt.vm Wed Aug 21 17:47:10 2013
@@ -42,7 +42,7 @@ Hadoop MapReduce Next Generation - CLI M
 $ mvn clean install -DskipTests
 $ mvn package -Pdist -Dtar -DskipTests -Dmaven.javadoc.skip
 +---+
-  <<NOTE:>> You will need protoc installed of version 2.4.1 or greater.
+  <<NOTE:>> You will need protoc 2.5.0 installed.
 
   The tarball should be available in <<<hadoop-dist/target/>>> directory. 
 

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/site/apt/FileSystemShell.apt.vm
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/site/apt/FileSystemShell.apt.vm?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/site/apt/FileSystemShell.apt.vm (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/site/apt/FileSystemShell.apt.vm Wed Aug 21 17:47:10 2013
@@ -86,11 +86,14 @@ chgrp
 
    Usage: <<<hdfs dfs -chgrp [-R] GROUP URI [URI ...]>>>
 
-   Change group association of files. With -R, make the change recursively
-   through the directory structure. The user must be the owner of files, or
+   Change group association of files. The user must be the owner of files, or
    else a super-user. Additional information is in the
    {{{betterurl}Permissions Guide}}.
 
+   Options
+
+     * The -R option will make the change recursively through the directory structure.
+
 chmod
 
    Usage: <<<hdfs dfs -chmod [-R] <MODE[,MODE]... | OCTALMODE> URI [URI ...]>>>
@@ -100,14 +103,21 @@ chmod
    else a super-user. Additional information is in the
    {{{betterurl}Permissions Guide}}.
 
+   Options
+
+     * The -R option will make the change recursively through the directory structure.
+
 chown
 
    Usage: <<<hdfs dfs -chown [-R] [OWNER][:[GROUP]] URI [URI ]>>>
 
-   Change the owner of files. With -R, make the change recursively through the
-   directory structure. The user must be a super-user. Additional information
+   Change the owner of files. The user must be a super-user. Additional information
    is in the {{{betterurl}Permissions Guide}}.
 
+   Options
+
+     * The -R option will make the change recursively through the directory structure.
+
 copyFromLocal
 
    Usage: <<<hdfs dfs -copyFromLocal <localsrc> URI>>>
@@ -115,6 +125,10 @@ copyFromLocal
    Similar to put command, except that the source is restricted to a local
    file reference.
 
+   Options:
+
+     * The -f option will overwrite the destination if it already exists.
+
 copyToLocal
 
    Usage: <<<hdfs dfs -copyToLocal [-ignorecrc] [-crc] URI <localdst> >>>
@@ -145,11 +159,15 @@ count
 
 cp
 
-   Usage: <<<hdfs dfs -cp URI [URI ...] <dest> >>>
+   Usage: <<<hdfs dfs -cp [-f] URI [URI ...] <dest> >>>
 
    Copy files from source to destination. This command allows multiple sources
    as well in which case the destination must be a directory.
 
+    Options:
+
+      * The -f option will overwrite the destination if it already exists.
+
    Example:
 
      * <<<hdfs dfs -cp /user/hadoop/file1 /user/hadoop/file2>>>
@@ -232,7 +250,7 @@ ls
 permissions number_of_replicas userid groupid filesize modification_date modification_time filename
 +---+
 
-   For a directory it returns list of its direct children as in unix.A directory is listed as:
+   For a directory it returns list of its direct children as in Unix. A directory is listed as:
 
 +---+
 permissions userid groupid modification_date modification_time dirname
@@ -256,8 +274,11 @@ mkdir
 
    Usage: <<<hdfs dfs -mkdir [-p] <paths> >>>
 
-   Takes path uri's as argument and creates directories.  With -p the behavior
-   is much like unix mkdir -p creating parent directories along the path.
+   Takes path uri's as argument and creates directories.
+
+   Options:
+
+     * The -p option behavior is much like Unix mkdir -p, creating parent directories along the path.
 
    Example:
 
@@ -362,8 +383,11 @@ setrep
 
    Usage: <<<hdfs dfs -setrep [-R] <path> >>>
 
-   Changes the replication factor of a file. -R option is for recursively
-   increasing the replication factor of files within a directory.
+   Changes the replication factor of a file.
+
+   Options:
+
+     * The -R option will recursively increase the replication factor of files within a directory.
 
    Example:
 
@@ -390,8 +414,11 @@ tail
 
    Usage: <<<hdfs dfs -tail [-f] URI>>>
 
-   Displays last kilobyte of the file to stdout. -f option can be used as in
-   Unix.
+   Displays last kilobyte of the file to stdout.
+
+   Options:
+
+     * The -f option will output appended data as the file grows, as in Unix.
 
    Example:
 
@@ -406,13 +433,9 @@ test
 
    Options:
 
-*----+------------+
-| -e | check to see if the file exists. Return 0 if true.
-*----+------------+
-| -z | check to see if the file is zero length. Return 0 if true.
-*----+------------+
-| -d | check to see if the path is directory. Return 0 if true.
-*----+------------+
+     * The -e option will check to see if the file exists, returning 0 if true.
+     * The -z option will check to see if the file is zero length, returning 0 if true.
+     * The -d option will check to see if the path is directory, returning 0 if true.
 
    Example:
 

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/site/apt/SingleCluster.apt.vm
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/site/apt/SingleCluster.apt.vm?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/site/apt/SingleCluster.apt.vm (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/site/apt/SingleCluster.apt.vm Wed Aug 21 17:47:10 2013
@@ -32,7 +32,7 @@ $ mvn clean install -DskipTests
 $ cd hadoop-mapreduce-project
 $ mvn clean install assembly:assembly -Pnative
 +---+
-  <<NOTE:>> You will need protoc installed of version 2.4.1 or greater.
+  <<NOTE:>> You will need protoc 2.5.0 installed.
 
   To ignore the native builds in mapreduce you can omit the <<<-Pnative>>> argument
   for maven. The tarball should be available in <<<target/>>> directory. 

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java Wed Aug 21 17:47:10 2013
@@ -1272,7 +1272,23 @@ public class TestConfiguration extends T
    Class<?> clazz = config.getClassByNameOrNull("java.lang.Object");
    assertNotNull(clazz);
   }
-  
+
+  public void testGetFinalParameters() throws Exception {
+    out=new BufferedWriter(new FileWriter(CONFIG));
+    startConfig();
+    declareProperty("my.var", "x", "x", true);
+    endConfig();
+    Path fileResource = new Path(CONFIG);
+    Configuration conf = new Configuration();
+    Set<String> finalParameters = conf.getFinalParameters();
+    assertFalse("my.var already exists", finalParameters.contains("my.var"));
+    conf.addResource(fileResource);
+    assertEquals("my.var is undefined", "x", conf.get("my.var"));
+    assertFalse("finalparams not copied", finalParameters.contains("my.var"));
+    finalParameters = conf.getFinalParameters();
+    assertTrue("my.var is not final", finalParameters.contains("my.var"));
+  }
+
   public static void main(String[] argv) throws Exception {
     junit.textui.TestRunner.main(new String[]{
       TestConfiguration.class.getName()

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java Wed Aug 21 17:47:10 2013
@@ -30,6 +30,7 @@ import org.apache.hadoop.fs.Options.Rena
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.junit.After;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -632,6 +633,20 @@ public abstract class FileContextMainOpe
         filteredPaths));
   }
   
+  protected Path getHiddenPathForTest() {
+    return null;
+  }
+  
+  @Test
+  public void testGlobStatusFilterWithHiddenPathTrivialFilter()
+      throws Exception {
+    Path hidden = getHiddenPathForTest();
+    Assume.assumeNotNull(hidden);
+    FileStatus[] filteredPaths = fc.util().globStatus(hidden, DEFAULT_FILTER);
+    Assert.assertNotNull(filteredPaths);
+    Assert.assertEquals(1, filteredPaths.length);
+  }
+
   @Test
   public void testWriteReadAndDeleteEmptyFile() throws Exception {
     writeReadAndDelete(0);

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellReturnCode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellReturnCode.java?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellReturnCode.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellReturnCode.java Wed Aug 21 17:47:10 2013
@@ -517,6 +517,26 @@ public class TestFsShellReturnCode {
       }
       return stat;
     }
+
+    @Override
+    public FileStatus getFileLinkStatus(Path p) throws IOException {
+      String f = makeQualified(p).toString();
+      FileStatus stat = super.getFileLinkStatus(p);
+      
+      stat.getPermission();
+      if (owners.containsKey(f)) {
+        stat.setOwner("STUB-"+owners.get(f));      
+      } else {
+        stat.setOwner("REAL-"+stat.getOwner());
+      }
+      if (groups.containsKey(f)) {
+        stat.setGroup("STUB-"+groups.get(f));      
+      } else {
+        stat.setGroup("REAL-"+stat.getGroup());
+      }
+      return stat;
+    }
+
   }
   
   static class MyFsShell extends FsShell {

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java Wed Aug 21 17:47:10 2013
@@ -26,6 +26,7 @@ import org.apache.hadoop.util.StringUtil
 import static org.apache.hadoop.fs.FileSystemTestHelper.*;
 
 import java.io.*;
+import java.net.URI;
 import java.util.Arrays;
 import java.util.Random;
 
@@ -363,12 +364,12 @@ public class TestLocalFileSystem {
 
     FileStatus status = fileSys.getFileStatus(path);
     assertTrue("check we're actually changing something", newModTime != status.getModificationTime());
-    assertEquals(0, status.getAccessTime());
+    long accessTime = status.getAccessTime();
 
     fileSys.setTimes(path, newModTime, -1);
     status = fileSys.getFileStatus(path);
     assertEquals(newModTime, status.getModificationTime());
-    assertEquals(0, status.getAccessTime());
+    assertEquals(accessTime, status.getAccessTime());
   }
 
   /**
@@ -520,4 +521,18 @@ public class TestLocalFileSystem {
       fail(s);
     }
   }
+
+  @Test
+  public void testStripFragmentFromPath() throws Exception {
+    FileSystem fs = FileSystem.getLocal(new Configuration());
+    Path pathQualified = TEST_PATH.makeQualified(fs.getUri(),
+        fs.getWorkingDirectory());
+    Path pathWithFragment = new Path(
+        new URI(pathQualified.toString() + "#glacier"));
+    // Create test file with fragment
+    FileSystemTestHelper.createFile(fs, pathWithFragment);
+    Path resolved = fs.resolvePath(pathWithFragment);
+    assertEquals("resolvePath did not strip fragment from Path", pathQualified,
+        resolved);
+  }
 }

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestSymlinkLocalFS.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestSymlinkLocalFS.java?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestSymlinkLocalFS.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestSymlinkLocalFS.java Wed Aug 21 17:47:10 2013
@@ -31,6 +31,7 @@ import java.net.URISyntaxException;
 
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.Test;
 
 /**
@@ -134,6 +135,7 @@ abstract public class TestSymlinkLocalFS
     Path fileAbs  = new Path(testBaseDir1()+"/file");
     Path fileQual = new Path(testURI().toString(), fileAbs);
     Path link     = new Path(testBaseDir1()+"/linkToFile");
+    Path linkQual = new Path(testURI().toString(), link.toString());
     wrapper.createSymlink(fileAbs, link, false);
     // Deleting the link using FileContext currently fails because
     // resolve looks up LocalFs rather than RawLocalFs for the path 
@@ -151,18 +153,15 @@ abstract public class TestSymlinkLocalFS
       // Expected. File's exists method returns false for dangling links
     }
     // We can stat a dangling link
+    UserGroupInformation user = UserGroupInformation.getCurrentUser();
     FileStatus fsd = wrapper.getFileLinkStatus(link);
     assertEquals(fileQual, fsd.getSymlink());
     assertTrue(fsd.isSymlink());
     assertFalse(fsd.isDirectory());
-    assertEquals("", fsd.getOwner());
-    assertEquals("", fsd.getGroup());
-    assertEquals(link, fsd.getPath());
-    assertEquals(0, fsd.getLen());
-    assertEquals(0, fsd.getBlockSize());
-    assertEquals(0, fsd.getReplication());
-    assertEquals(0, fsd.getAccessTime());
-    assertEquals(FsPermission.getDefault(), fsd.getPermission());
+    assertEquals(user.getUserName(), fsd.getOwner());
+    // Compare against user's primary group
+    assertEquals(user.getGroupNames()[0], fsd.getGroup());
+    assertEquals(linkQual, fsd.getPath());
     // Accessing the link 
     try {
       readFile(link);

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java Wed Aug 21 17:47:10 2013
@@ -116,7 +116,8 @@ public class HttpServerFunctionalTest ex
   public static HttpServer createServer(String host, int port)
       throws IOException {
     prepareTestWebapp();
-    return new HttpServer(TEST, host, port, true);
+    return new HttpServer.Builder().setName(TEST).setBindAddress(host)
+        .setPort(port).setFindPort(true).build();
   }
 
   /**
@@ -126,7 +127,8 @@ public class HttpServerFunctionalTest ex
    * @throws IOException if it could not be created
    */
   public static HttpServer createServer(String webapp) throws IOException {
-    return new HttpServer(webapp, "0.0.0.0", 0, true);
+    return new HttpServer.Builder().setName(webapp).setBindAddress("0.0.0.0")
+        .setPort(0).setFindPort(true).build();
   }
   /**
    * Create an HttpServer instance for the given webapp
@@ -137,13 +139,16 @@ public class HttpServerFunctionalTest ex
    */
   public static HttpServer createServer(String webapp, Configuration conf)
       throws IOException {
-    return new HttpServer(webapp, "0.0.0.0", 0, true, conf);
+    return new HttpServer.Builder().setName(webapp).setBindAddress("0.0.0.0")
+        .setPort(0).setFindPort(true).setConf(conf).build();
   }
 
   public static HttpServer createServer(String webapp, Configuration conf, AccessControlList adminsAcl)
       throws IOException {
-    return new HttpServer(webapp, "0.0.0.0", 0, true, conf, adminsAcl);
+    return new HttpServer.Builder().setName(webapp).setBindAddress("0.0.0.0")
+        .setPort(0).setFindPort(true).setConf(conf).setACL(adminsAcl).build();
   }
+  
   /**
    * Create an HttpServer instance for the given webapp
    * @param webapp the webapp to work with
@@ -154,7 +159,8 @@ public class HttpServerFunctionalTest ex
    */
   public static HttpServer createServer(String webapp, Configuration conf,
       String[] pathSpecs) throws IOException {
-    return new HttpServer(webapp, "0.0.0.0", 0, true, conf, pathSpecs);
+    return new HttpServer.Builder().setName(webapp).setBindAddress("0.0.0.0")
+        .setPort(0).setFindPort(true).setConf(conf).setPathSpec(pathSpecs).build();
   }
 
   /**

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java Wed Aug 21 17:47:10 2013
@@ -121,7 +121,6 @@ public class TestHttpServer extends Http
 
   @SuppressWarnings("serial")
   public static class LongHeaderServlet extends HttpServlet {
-    @SuppressWarnings("unchecked")
     @Override
     public void doGet(HttpServletRequest request,
                       HttpServletResponse response
@@ -362,7 +361,8 @@ public class TestHttpServer extends Http
     MyGroupsProvider.mapping.put("userA", Arrays.asList("groupA"));
     MyGroupsProvider.mapping.put("userB", Arrays.asList("groupB"));
 
-    HttpServer myServer = new HttpServer("test", "0.0.0.0", 0, true, conf);
+    HttpServer myServer = new HttpServer.Builder().setName("test")
+        .setBindAddress("0.0.0.0").setPort(0).setFindPort(true).build();
     myServer.setAttribute(HttpServer.CONF_CONTEXT_ATTRIBUTE, conf);
     myServer.start();
     int port = myServer.getPort();
@@ -403,8 +403,9 @@ public class TestHttpServer extends Http
     MyGroupsProvider.mapping.put("userD", Arrays.asList("groupD"));
     MyGroupsProvider.mapping.put("userE", Arrays.asList("groupE"));
 
-    HttpServer myServer = new HttpServer("test", "0.0.0.0", 0, true, conf,
-        new AccessControlList("userA,userB groupC,groupD"));
+    HttpServer myServer = new HttpServer.Builder().setName("test")
+        .setBindAddress("0.0.0.0").setPort(0).setFindPort(true).setConf(conf)
+        .setACL(new AccessControlList("userA,userB groupC,groupD")).build();
     myServer.setAttribute(HttpServer.CONF_CONTEXT_ATTRIBUTE, conf);
     myServer.start();
     int port = myServer.getPort();

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/log/TestLogLevel.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/log/TestLogLevel.java?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/log/TestLogLevel.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/log/TestLogLevel.java Wed Aug 21 17:47:10 2013
@@ -42,7 +42,10 @@ public class TestLogLevel extends TestCa
       log.error("log.error1");
       assertTrue(!Level.ERROR.equals(log.getEffectiveLevel()));
 
-      HttpServer server = new HttpServer("..", "localhost", 22222, true);
+      HttpServer server = new HttpServer.Builder().setName("..")
+          .setBindAddress("localhost").setPort(22222).setFindPort(true)
+          .build();
+      
       server.start();
       int port = server.getPort();
 

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java Wed Aug 21 17:47:10 2013
@@ -38,6 +38,7 @@ import org.apache.hadoop.security.UserGr
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenInfo;
+import org.junit.Before;
 import org.junit.Test;
 import org.apache.hadoop.ipc.TestSaslRPC.TestTokenSecretManager;
 import org.apache.hadoop.ipc.TestSaslRPC.TestTokenIdentifier;
@@ -58,7 +59,7 @@ public class TestDoAsEffectiveUser {
       GROUP2_NAME };
   private static final String ADDRESS = "0.0.0.0";
   private TestProtocol proxy;
-  private static Configuration masterConf = new Configuration();
+  private static final Configuration masterConf = new Configuration();
   
   
   public static final Log LOG = LogFactory
@@ -70,6 +71,10 @@ public class TestDoAsEffectiveUser {
         "RULE:[2:$1@$0](.*@HADOOP.APACHE.ORG)s/@.*//" +
         "RULE:[1:$1@$0](.*@HADOOP.APACHE.ORG)s/@.*//"
         + "DEFAULT");
+  }
+
+  @Before
+  public void setMasterConf() {
     UserGroupInformation.setConfiguration(masterConf);
   }
 

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml Wed Aug 21 17:47:10 2013
@@ -296,7 +296,11 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^( |\t)*modification times, ownership and the mode.( )*</expected-output>
+          <expected-output>^( |\t)*modification times, ownership and the mode. Passing -f( )*</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^( |\t)*overwrites the destination if it already exists.( )*</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -400,7 +404,11 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^( |\t)*ownership and the mode.( )*</expected-output>
+          <expected-output>^( |\t)*ownership and the mode. Passing -f overwrites( )*</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^( |\t)*the destination if it already exists.( )*</expected-output>
         </comparator>
       </comparators>
     </test>

Propchange: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-minikdc/
------------------------------------------------------------------------------
--- svn:ignore (original)
+++ svn:ignore Wed Aug 21 17:47:10 2013
@@ -1 +1,4 @@
+.classpath
+.project
+.settings
 target

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-minikdc/pom.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-minikdc/pom.xml?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-minikdc/pom.xml (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-minikdc/pom.xml Wed Aug 21 17:47:10 2013
@@ -38,7 +38,7 @@
     <dependency>
       <groupId>org.apache.directory.server</groupId>
       <artifactId>apacheds-all</artifactId>
-      <version>2.0.0-M14</version>
+      <version>2.0.0-M15</version>
       <scope>compile</scope>
     </dependency>
     <dependency>

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-minikdc/src/main/java/org/apache/hadoop/minikdc/MiniKdc.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-minikdc/src/main/java/org/apache/hadoop/minikdc/MiniKdc.java?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-minikdc/src/main/java/org/apache/hadoop/minikdc/MiniKdc.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-minikdc/src/main/java/org/apache/hadoop/minikdc/MiniKdc.java Wed Aug 21 17:47:10 2013
@@ -37,7 +37,7 @@ import org.apache.directory.server.core.
 import org.apache.directory.server.core.partition.ldif.LdifPartition;
 import org.apache.directory.server.kerberos.kdc.KdcServer;
 import org.apache.directory.server.kerberos.shared.crypto.encryption.KerberosKeyFactory;
-import org.apache.directory.server.kerberos.shared.keytab.HackedKeytab;
+import org.apache.directory.server.kerberos.shared.keytab.Keytab;
 import org.apache.directory.server.kerberos.shared.keytab.KeytabEntry;
 import org.apache.directory.server.protocol.shared.transport.TcpTransport;
 import org.apache.directory.server.protocol.shared.transport.UdpTransport;
@@ -514,7 +514,7 @@ public class MiniKdc {
   public void createPrincipal(File keytabFile, String ... principals)
           throws Exception {
     String generatedPassword = UUID.randomUUID().toString();
-    HackedKeytab keytab = new HackedKeytab();
+    Keytab keytab = new Keytab();
     List<KeytabEntry> entries = new ArrayList<KeytabEntry>();
     for (String principal : principals) {
       createPrincipal(principal, generatedPassword);
@@ -529,6 +529,6 @@ public class MiniKdc {
       }
     }
     keytab.setEntries(entries);
-    keytab.write(keytabFile, principals.length);
+    keytab.write(keytabFile);
   }
 }
\ No newline at end of file

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-minikdc/src/test/java/org/apache/hadoop/minikdc/TestMiniKdc.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-minikdc/src/test/java/org/apache/hadoop/minikdc/TestMiniKdc.java?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-minikdc/src/test/java/org/apache/hadoop/minikdc/TestMiniKdc.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-minikdc/src/test/java/org/apache/hadoop/minikdc/TestMiniKdc.java Wed Aug 21 17:47:10 2013
@@ -137,7 +137,7 @@ public class TestMiniKdc extends Kerbero
               subject.getPrincipals().iterator().next().getClass());
       Assert.assertEquals(principal + "@" + kdc.getRealm(),
               subject.getPrincipals().iterator().next().getName());
-      loginContext.login();
+      loginContext.logout();
 
       //server login
       subject = new Subject(false, principals, new HashSet<Object>(),
@@ -151,7 +151,7 @@ public class TestMiniKdc extends Kerbero
               subject.getPrincipals().iterator().next().getClass());
       Assert.assertEquals(principal + "@" + kdc.getRealm(),
               subject.getPrincipals().iterator().next().getName());
-      loginContext.login();
+      loginContext.logout();
 
     } finally {
       if (loginContext != null) {

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/READDIR3Response.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/READDIR3Response.java?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/READDIR3Response.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/READDIR3Response.java Wed Aug 21 17:47:10 2013
@@ -17,12 +17,14 @@
  */
 package org.apache.hadoop.nfs.nfs3.response;
 
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
 import org.apache.hadoop.nfs.nfs3.Nfs3FileAttributes;
 import org.apache.hadoop.nfs.nfs3.Nfs3Status;
 import org.apache.hadoop.oncrpc.XDR;
 
-import com.google.common.collect.ObjectArrays;
-
 /**
  * READDIR3 Response
  */
@@ -56,12 +58,11 @@ public class READDIR3Response extends NF
   }
 
   public static class DirList3 {
-    final Entry3 entries[];
+    final List<Entry3> entries;
     final boolean eof;
     
     public DirList3(Entry3[] entries, boolean eof) {
-      this.entries = ObjectArrays.newArray(entries, entries.length);
-      System.arraycopy(this.entries, 0, entries, 0, entries.length);
+      this.entries = Collections.unmodifiableList(Arrays.asList(entries));
       this.eof = eof;
     }
   }
@@ -102,12 +103,11 @@ public class READDIR3Response extends NF
 
     if (getStatus() == Nfs3Status.NFS3_OK) {
       xdr.writeLongAsHyper(cookieVerf);
-      Entry3[] f = dirList.entries;
-      for (int i = 0; i < f.length; i++) {
+      for (Entry3 e : dirList.entries) {
         xdr.writeBoolean(true); // Value follows
-        xdr.writeLongAsHyper(f[i].getFileId());
-        xdr.writeString(f[i].getName());
-        xdr.writeLongAsHyper(f[i].getCookie());
+        xdr.writeLongAsHyper(e.getFileId());
+        xdr.writeString(e.getName());
+        xdr.writeLongAsHyper(e.getCookie());
       }
 
       xdr.writeBoolean(false);

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/READDIRPLUS3Response.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/READDIRPLUS3Response.java?rev=1516230&r1=1516229&r2=1516230&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/READDIRPLUS3Response.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/READDIRPLUS3Response.java Wed Aug 21 17:47:10 2013
@@ -17,13 +17,15 @@
  */
 package org.apache.hadoop.nfs.nfs3.response;
 
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
 import org.apache.hadoop.nfs.nfs3.FileHandle;
 import org.apache.hadoop.nfs.nfs3.Nfs3FileAttributes;
 import org.apache.hadoop.nfs.nfs3.Nfs3Status;
 import org.apache.hadoop.oncrpc.XDR;
 
-import com.google.common.collect.ObjectArrays;
-
 /**
  * READDIRPLUS3 Response
  */
@@ -60,16 +62,15 @@ public class READDIRPLUS3Response  exten
   }
 
   public static class DirListPlus3 {
-    EntryPlus3 entries[];
+    List<EntryPlus3> entries;
     boolean eof;
     
     public DirListPlus3(EntryPlus3[] entries, boolean eof) {
-      this.entries = ObjectArrays.newArray(entries, entries.length);
-      System.arraycopy(this.entries, 0, entries, 0, entries.length);
+      this.entries = Collections.unmodifiableList(Arrays.asList(entries));
       this.eof = eof;
     }
 
-    EntryPlus3[] getEntries() {
+    List<EntryPlus3> getEntries() {
       return entries;
     }
     
@@ -101,10 +102,9 @@ public class READDIRPLUS3Response  exten
     
     if (getStatus() == Nfs3Status.NFS3_OK) {
       out.writeLongAsHyper(cookieVerf);
-      EntryPlus3[] f = dirListPlus.getEntries();
-      for (int i = 0; i < f.length; i++) {
+      for (EntryPlus3 f : dirListPlus.getEntries()) {
         out.writeBoolean(true); // next
-        f[i].seralize(out);
+        f.seralize(out);
       }
 
       out.writeBoolean(false);