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 sz...@apache.org on 2012/04/10 01:25:21 UTC

svn commit: r1311518 - in /hadoop/common/branches/HDFS-3092/hadoop-common-project: hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/ hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ hadoop-auth/src/tes...

Author: szetszwo
Date: Mon Apr  9 23:25:17 2012
New Revision: 1311518

URL: http://svn.apache.org/viewvc?rev=1311518&view=rev
Log:
Merge r1310141 through r1311517 from trunk.

Added:
    hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/NotInMountpointException.java
      - copied unchanged from r1311517, hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/NotInMountpointException.java
Modified:
    hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java
    hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosName.java
    hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestAuthenticationFilter.java
    hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/CHANGES.txt   (contents, props changed)
    hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/docs/   (props changed)
    hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/   (props changed)
    hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
    hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java
    hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java
    hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java
    hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
    hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/NodeFencer.java
    hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java
    hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/test/core/   (props changed)
    hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemTestHelper.java
    hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestChRootedFileSystem.java
    hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestNodeFencer.java
    hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestShellCommandFencer.java

Modified: hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java?rev=1311518&r1=1311517&r2=1311518&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java (original)
+++ hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java Mon Apr  9 23:25:17 2012
@@ -331,7 +331,14 @@ public class AuthenticationFilter implem
     HttpServletResponse httpResponse = (HttpServletResponse) response;
     try {
       boolean newToken = false;
-      AuthenticationToken token = getToken(httpRequest);
+      AuthenticationToken token;
+      try {
+        token = getToken(httpRequest);
+      }
+      catch (AuthenticationException ex) {
+        LOG.warn("AuthenticationToken ignored: " + ex.getMessage());
+        token = null;
+      }
       if (token == null) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Request [{}] triggering authentication", getRequestURL(httpRequest));
@@ -371,6 +378,9 @@ public class AuthenticationFilter implem
         }
         filterChain.doFilter(httpRequest, httpResponse);
       }
+      else {
+        throw new AuthenticationException("Missing AuthenticationToken");
+      }
     } catch (AuthenticationException ex) {
       if (!httpResponse.isCommitted()) {
         Cookie cookie = createCookie("");

Modified: hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosName.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosName.java?rev=1311518&r1=1311517&r2=1311518&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosName.java (original)
+++ hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosName.java Mon Apr  9 23:25:17 2012
@@ -23,10 +23,11 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
-import java.lang.reflect.Method;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 /**
@@ -38,6 +39,8 @@ import org.apache.hadoop.classification.
 @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
 @InterfaceStability.Evolving
 public class KerberosName {
+  private static final Logger LOG = LoggerFactory.getLogger(KerberosName.class);
+
   /** The first component of the name */
   private final String serviceName;
   /** The second component of the name. It may be null. */
@@ -81,6 +84,7 @@ public class KerberosName {
     try {
       defaultRealm = KerberosUtil.getDefaultRealm();
     } catch (Exception ke) {
+        LOG.warn("Kerberos krb5 configuration not found, setting default realm to empty");
         defaultRealm="";
     }
   }

Modified: hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestAuthenticationFilter.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestAuthenticationFilter.java?rev=1311518&r1=1311517&r2=1311518&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestAuthenticationFilter.java (original)
+++ hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestAuthenticationFilter.java Mon Apr  9 23:25:17 2012
@@ -349,7 +349,7 @@ public class TestAuthenticationFilter ex
     }
   }
 
-  private void _testDoFilterAuthentication(boolean withDomainPath) throws Exception {
+  private void _testDoFilterAuthentication(boolean withDomainPath, boolean invalidToken) throws Exception {
     AuthenticationFilter filter = new AuthenticationFilter();
     try {
       FilterConfig config = Mockito.mock(FilterConfig.class);
@@ -380,6 +380,12 @@ public class TestAuthenticationFilter ex
       Mockito.when(request.getRequestURL()).thenReturn(new StringBuffer("http://foo:8080/bar"));
       Mockito.when(request.getQueryString()).thenReturn("authenticated=true");
 
+      if (invalidToken) {
+        Mockito.when(request.getCookies()).thenReturn(
+          new Cookie[] { new Cookie(AuthenticatedURL.AUTH_COOKIE, "foo")}
+        );
+      }
+
       HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
 
       FilterChain chain = Mockito.mock(FilterChain.class);
@@ -437,11 +443,15 @@ public class TestAuthenticationFilter ex
   }
 
   public void testDoFilterAuthentication() throws Exception {
-    _testDoFilterAuthentication(false);
+    _testDoFilterAuthentication(false, false);
+  }
+
+  public void testDoFilterAuthenticationWithInvalidToken() throws Exception {
+    _testDoFilterAuthentication(false, true);
   }
 
   public void testDoFilterAuthenticationWithDomainPath() throws Exception {
-    _testDoFilterAuthentication(true);
+    _testDoFilterAuthentication(true, false);
   }
 
   public void testDoFilterAuthenticated() throws Exception {

Modified: hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/CHANGES.txt?rev=1311518&r1=1311517&r2=1311518&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/CHANGES.txt (original)
+++ hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/CHANGES.txt Mon Apr  9 23:25:17 2012
@@ -252,6 +252,12 @@ Release 2.0.0 - UNRELEASED 
 
     HADOOP-8007. Use substitution tokens for fencing argument (todd)
 
+    HADOOP-8077. HA: fencing method should be able to be configured on
+    a per-NN or per-NS basis (todd)
+
+    HADOOP-8086. KerberosName silently sets defaultRealm to "" if the 
+    Kerberos config is not found, it should log a WARN (tucu)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -324,6 +330,14 @@ Release 2.0.0 - UNRELEASED 
     HADOOP-8243. Security support broken in CLI (manual) failover controller
     (todd)
 
+    HADOOP-8251. Fix SecurityUtil.fetchServiceTicket after HADOOP-6941 (todd)
+
+    HADOOP-8249. invalid hadoop-auth cookies should trigger authentication 
+    if info is avail before returning HTTP 401 (tucu)
+
+    HADOOP-8261. Har file system doesn't deal with FS URIs with a host but no
+    port. (atm)
+
   BREAKDOWN OF HADOOP-7454 SUBTASKS
 
     HADOOP-7455. HA: Introduce HA Service Protocol Interface. (suresh)
@@ -400,6 +414,9 @@ Release 0.23.3 - UNRELEASED
     HADOOP-8180. Remove hsqldb since its not needed from pom.xml (Ravi Prakash
     via tgraves)
 
+    HADOOP-8014. ViewFileSystem does not correctly implement getDefaultBlockSize,
+    getDefaultReplication, getContentSummary (John George via bobby)
+
 Release 0.23.2 - UNRELEASED 
 
   INCOMPATIBLE CHANGES

Propchange: hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/CHANGES.txt
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt:r1310141-1311517

Propchange: hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/docs/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/docs:r1310141-1311517

Propchange: hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java:r1310141-1311517

Modified: hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java?rev=1311518&r1=1311517&r2=1311518&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java (original)
+++ hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java Mon Apr  9 23:25:17 2012
@@ -579,7 +579,8 @@ public abstract class FileSystem extends
    *
    * The FileSystem will simply return an elt containing 'localhost'.
    *
-   * @param p path of file to get locations for
+   * @param p path is used to identify an FS since an FS could have
+   *          another FS that it could be delegating the call to
    * @param start offset into the given file
    * @param len length for which to get locations for
    */
@@ -602,10 +603,21 @@ public abstract class FileSystem extends
     return new FsServerDefaults(getDefaultBlockSize(), 
         conf.getInt("io.bytes.per.checksum", 512), 
         64 * 1024, 
-        getDefaultReplication(), 
+        getDefaultReplication(),
         conf.getInt("io.file.buffer.size", 4096));
   }
-  
+
+  /**
+   * Return a set of server default configuration values
+   * @param p path is used to identify an FS since an FS could have
+   *          another FS that it could be delegating the call to
+   * @return server default configuration values
+   * @throws IOException
+   */
+  public FsServerDefaults getServerDefaults(Path p) throws IOException {
+    return getServerDefaults();
+  }
+
   /**
    * Return the fully-qualified path of path f resolving the path
    * through any symlinks or mount point
@@ -653,8 +665,8 @@ public abstract class FileSystem extends
       throws IOException {
     return create(f, overwrite, 
                   getConf().getInt("io.file.buffer.size", 4096),
-                  getDefaultReplication(),
-                  getDefaultBlockSize());
+                  getDefaultReplication(f),
+                  getDefaultBlockSize(f));
   }
 
   /**
@@ -668,8 +680,8 @@ public abstract class FileSystem extends
       throws IOException {
     return create(f, true, 
                   getConf().getInt("io.file.buffer.size", 4096),
-                  getDefaultReplication(),
-                  getDefaultBlockSize(), progress);
+                  getDefaultReplication(f),
+                  getDefaultBlockSize(f), progress);
   }
 
   /**
@@ -683,7 +695,7 @@ public abstract class FileSystem extends
     return create(f, true, 
                   getConf().getInt("io.file.buffer.size", 4096),
                   replication,
-                  getDefaultBlockSize());
+                  getDefaultBlockSize(f));
   }
 
   /**
@@ -699,7 +711,7 @@ public abstract class FileSystem extends
     return create(f, true, 
                   getConf().getInt("io.file.buffer.size", 4096),
                   replication,
-                  getDefaultBlockSize(), progress);
+                  getDefaultBlockSize(f), progress);
   }
 
     
@@ -715,8 +727,8 @@ public abstract class FileSystem extends
                                    int bufferSize
                                    ) throws IOException {
     return create(f, overwrite, bufferSize, 
-                  getDefaultReplication(),
-                  getDefaultBlockSize());
+                  getDefaultReplication(f),
+                  getDefaultBlockSize(f));
   }
     
   /**
@@ -733,8 +745,8 @@ public abstract class FileSystem extends
                                    Progressable progress
                                    ) throws IOException {
     return create(f, overwrite, bufferSize, 
-                  getDefaultReplication(),
-                  getDefaultBlockSize(), progress);
+                  getDefaultReplication(f),
+                  getDefaultBlockSize(f), progress);
   }
     
     
@@ -1916,12 +1928,32 @@ public abstract class FileSystem extends
     return getConf().getLong("fs.local.block.size", 32 * 1024 * 1024);
   }
     
+  /** Return the number of bytes that large input files should be optimally
+   * be split into to minimize i/o time.  The given path will be used to
+   * locate the actual filesystem.  The full path does not have to exist.
+   * @param f path of file
+   * @return the default block size for the path's filesystem
+   */
+  public long getDefaultBlockSize(Path f) {
+    return getDefaultBlockSize();
+  }
+
   /**
    * Get the default replication.
    */
   public short getDefaultReplication() { return 1; }
 
   /**
+   * Get the default replication for a path.   The given path will be used to
+   * locate the actual filesystem.  The full path does not have to exist.
+   * @param path of the file
+   * @return default replication for the path's filesystem 
+   */
+  public short getDefaultReplication(Path path) {
+    return getDefaultReplication();
+  }
+  
+  /**
    * Return a file status object that represents the path.
    * @param f The path we want information from
    * @return a FileStatus object

Modified: hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java?rev=1311518&r1=1311517&r2=1311518&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java (original)
+++ hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java Mon Apr  9 23:25:17 2012
@@ -28,6 +28,7 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.Progressable;
@@ -267,6 +268,7 @@ public class FilterFileSystem extends Fi
     return fs.mkdirs(f, permission);
   }
 
+
   /**
    * The src file is on the local disk.  Add it to FS at
    * the given dst name.
@@ -336,19 +338,42 @@ public class FilterFileSystem extends Fi
     return fs.getUsed();
   }
   
-  /** Return the number of bytes that large input files should be optimally
-   * be split into to minimize i/o time. */
+  @Override
   public long getDefaultBlockSize() {
     return fs.getDefaultBlockSize();
   }
   
-  /**
-   * Get the default replication.
-   */
+  @Override
   public short getDefaultReplication() {
     return fs.getDefaultReplication();
   }
 
+  @Override
+  public FsServerDefaults getServerDefaults() throws IOException {
+    return fs.getServerDefaults();
+  }
+
+  // path variants delegate to underlying filesystem 
+  @Override
+  public ContentSummary getContentSummary(Path f) throws IOException {
+    return fs.getContentSummary(f);
+  }
+
+  @Override
+  public long getDefaultBlockSize(Path f) {
+    return fs.getDefaultBlockSize(f);
+  }
+
+  @Override
+  public short getDefaultReplication(Path f) {
+    return fs.getDefaultReplication(f);
+  }
+
+  @Override
+  public FsServerDefaults getServerDefaults(Path f) throws IOException {
+    return fs.getServerDefaults(f);
+  }
+
   /**
    * Get file status.
    */
@@ -441,4 +466,4 @@ public class FilterFileSystem extends Fi
       Credentials credentials) throws IOException {
     return fs.getDelegationTokens(renewer, credentials);
   }
-}
\ No newline at end of file
+}

Modified: hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java?rev=1311518&r1=1311517&r2=1311518&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java (original)
+++ hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java Mon Apr  9 23:25:17 2012
@@ -202,7 +202,8 @@ public class HarFileSystem extends Filte
     final String underLyingHost = i == host.length()? null: host.substring(i);
     int underLyingPort = rawURI.getPort();
     String auth = (underLyingHost == null && underLyingPort == -1)?
-                  null:(underLyingHost+":"+underLyingPort);
+                  null:(underLyingHost+
+                      (underLyingPort == -1 ? "" : ":"+underLyingPort));
     URI tmp = null;
     if (rawURI.getQuery() != null) {
       // query component not allowed

Modified: hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java?rev=1311518&r1=1311517&r2=1311518&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java (original)
+++ hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java Mon Apr  9 23:25:17 2012
@@ -23,6 +23,7 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileChecksum;
@@ -209,11 +210,6 @@ class ChRootedFileSystem extends FilterF
   }
 
   @Override
-  public FsServerDefaults getServerDefaults() throws IOException {
-    return super.getServerDefaults();
-  }
-
-  @Override
   public FileStatus[] listStatus(final Path f) 
       throws IOException {
     return super.listStatus(fullPath(f));
@@ -273,4 +269,42 @@ class ChRootedFileSystem extends FilterF
   public Path resolvePath(final Path p) throws IOException {
     return super.resolvePath(fullPath(p));
   }
+
+  @Override
+  public ContentSummary getContentSummary(Path f) throws IOException {
+    return super.getContentSummary(fullPath(f));
+  }
+  
+
+  private static Path rootPath = new Path(Path.SEPARATOR);
+
+  @Override
+  public long getDefaultBlockSize() {
+    return getDefaultBlockSize(fullPath(rootPath));
+  }
+  
+  @Override
+  public long getDefaultBlockSize(Path f) {
+    return super.getDefaultBlockSize(fullPath(f));
+  }  
+
+  @Override
+  public short getDefaultReplication() {
+    return getDefaultReplication(fullPath(rootPath));
+  }
+
+  @Override
+  public short getDefaultReplication(Path f) {
+    return super.getDefaultReplication(fullPath(f));
+  }
+  
+  @Override
+  public FsServerDefaults getServerDefaults() throws IOException {
+    return getServerDefaults(fullPath(rootPath));
+  }  
+
+  @Override
+  public FsServerDefaults getServerDefaults(Path f) throws IOException {
+    return super.getServerDefaults(fullPath(f));
+  }  
 }

Modified: hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java?rev=1311518&r1=1311517&r2=1311518&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java (original)
+++ hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java Mon Apr  9 23:25:17 2012
@@ -34,6 +34,7 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
@@ -41,6 +42,7 @@ import org.apache.hadoop.fs.FileChecksum
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsConstants;
+import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
@@ -471,6 +473,57 @@ public class ViewFileSystem extends File
   }
   
   @Override
+  public long getDefaultBlockSize() {
+    throw new NotInMountpointException("getDefaultBlockSize");
+  }
+
+  @Override
+  public short getDefaultReplication() {
+    throw new NotInMountpointException("getDefaultReplication");
+  }
+
+  @Override
+  public FsServerDefaults getServerDefaults() throws IOException {
+    throw new NotInMountpointException("getServerDefaults");
+  }
+
+  @Override
+  public long getDefaultBlockSize(Path f) {
+    try {
+      InodeTree.ResolveResult<FileSystem> res =
+        fsState.resolve(getUriPath(f), true);
+      return res.targetFileSystem.getDefaultBlockSize(res.remainingPath);
+    } catch (FileNotFoundException e) {
+      throw new NotInMountpointException(f, "getDefaultBlockSize"); 
+    }
+  }
+
+  @Override
+  public short getDefaultReplication(Path f) {
+    try {
+      InodeTree.ResolveResult<FileSystem> res =
+        fsState.resolve(getUriPath(f), true);
+      return res.targetFileSystem.getDefaultReplication(res.remainingPath);
+    } catch (FileNotFoundException e) {
+      throw new NotInMountpointException(f, "getDefaultReplication"); 
+    }
+  }
+
+  @Override
+  public FsServerDefaults getServerDefaults(Path f) throws IOException {
+    InodeTree.ResolveResult<FileSystem> res =
+      fsState.resolve(getUriPath(f), true);
+    return res.targetFileSystem.getServerDefaults(res.remainingPath);    
+  }
+
+  @Override
+  public ContentSummary getContentSummary(Path f) throws IOException {
+    InodeTree.ResolveResult<FileSystem> res = 
+      fsState.resolve(getUriPath(f), true);
+    return res.targetFileSystem.getContentSummary(res.remainingPath);
+  }
+
+  @Override
   public void setWriteChecksum(final boolean writeChecksum) { 
     List<InodeTree.MountPoint<FileSystem>> mountPoints = 
         fsState.getMountPoints();
@@ -742,5 +795,20 @@ public class ViewFileSystem extends File
     public void setVerifyChecksum(boolean verifyChecksum) {
       // Noop for viewfs
     }
+
+    @Override
+    public FsServerDefaults getServerDefaults(Path f) throws IOException {
+      throw new NotInMountpointException(f, "getServerDefaults");
+    }
+    
+    @Override
+    public long getDefaultBlockSize(Path f) {
+      throw new NotInMountpointException(f, "getDefaultBlockSize");
+    }
+
+    @Override
+    public short getDefaultReplication(Path f) {
+      throw new NotInMountpointException(f, "getDefaultReplication");
+    }
   }
 }

Modified: hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/NodeFencer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/NodeFencer.java?rev=1311518&r1=1311517&r2=1311518&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/NodeFencer.java (original)
+++ hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/NodeFencer.java Mon Apr  9 23:25:17 2012
@@ -53,9 +53,6 @@ import com.google.common.collect.Lists;
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class NodeFencer {
-  public static final String CONF_METHODS_KEY =
-    "dfs.ha.fencing.methods";
-  
   private static final String CLASS_RE = "([a-zA-Z0-9\\.\\$]+)";
   private static final Pattern CLASS_WITH_ARGUMENT =
     Pattern.compile(CLASS_RE + "\\((.+?)\\)");
@@ -76,18 +73,18 @@ public class NodeFencer {
   
   private final List<FenceMethodWithArg> methods;
   
-  public NodeFencer(Configuration conf)
+  NodeFencer(Configuration conf, String spec)
       throws BadFencingConfigurationException {
-    this.methods = parseMethods(conf);
+    this.methods = parseMethods(conf, spec);
   }
   
-  public static NodeFencer create(Configuration conf)
+  public static NodeFencer create(Configuration conf, String confKey)
       throws BadFencingConfigurationException {
-    String confStr = conf.get(CONF_METHODS_KEY);
+    String confStr = conf.get(confKey);
     if (confStr == null) {
       return null;
     }
-    return new NodeFencer(conf);
+    return new NodeFencer(conf, confStr);
   }
 
   public boolean fence(HAServiceTarget fromSvc) {
@@ -115,10 +112,10 @@ public class NodeFencer {
     return false;
   }
 
-  private static List<FenceMethodWithArg> parseMethods(Configuration conf)
+  private static List<FenceMethodWithArg> parseMethods(Configuration conf,
+      String spec)
       throws BadFencingConfigurationException {
-    String confStr = conf.get(CONF_METHODS_KEY);
-    String[] lines = confStr.split("\\s*\n\\s*");
+    String[] lines = spec.split("\\s*\n\\s*");
     
     List<FenceMethodWithArg> methods = Lists.newArrayList();
     for (String line : lines) {

Modified: hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java?rev=1311518&r1=1311517&r2=1311518&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java (original)
+++ hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java Mon Apr  9 23:25:17 2012
@@ -171,7 +171,7 @@ public class SecurityUtil {
       } else {
         principalClass = Class.forName("sun.security.krb5.PrincipalName");
         credentialsClass = Class.forName("sun.security.krb5.Credentials");
-        krb5utilClass = Class.forName("sun.security.jgss.krb5");
+        krb5utilClass = Class.forName("sun.security.jgss.krb5.Krb5Util");
       }
       @SuppressWarnings("rawtypes")
       Constructor principalConstructor = principalClass.getConstructor(String.class, 

Propchange: hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/test/core/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/core:r1310141-1311517

Modified: hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemTestHelper.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemTestHelper.java?rev=1311518&r1=1311517&r2=1311518&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemTestHelper.java (original)
+++ hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemTestHelper.java Mon Apr  9 23:25:17 2012
@@ -36,6 +36,7 @@ public final class FileSystemTestHelper 
     System.getProperty("test.build.data", "target/test/data") + "/test";
   private static final int DEFAULT_BLOCK_SIZE = 1024;
   private static final int DEFAULT_NUM_BLOCKS = 2;
+  private static final short DEFAULT_NUM_REPL = 1;
   private static String absTestRootDir = null;
 
   /** Hidden constructor */
@@ -99,9 +100,9 @@ public final class FileSystemTestHelper 
    * Create files with numBlocks blocks each with block size blockSize.
    */
   public static long createFile(FileSystem fSys, Path path, int numBlocks,
-      int blockSize, boolean createParent) throws IOException {
+      int blockSize, short numRepl, boolean createParent) throws IOException {
     FSDataOutputStream out = 
-      fSys.create(path, false, 4096, fSys.getDefaultReplication(), blockSize );
+      fSys.create(path, false, 4096, numRepl, blockSize );
 
     byte[] data = getFileData(numBlocks, blockSize);
     out.write(data, 0, data.length);
@@ -109,13 +110,19 @@ public final class FileSystemTestHelper 
     return data.length;
   }
 
+
+  public static long createFile(FileSystem fSys, Path path, int numBlocks,
+      int blockSize, boolean createParent) throws IOException {
+      return createFile(fSys, path, numBlocks, blockSize, fSys.getDefaultReplication(), true);
+  }
+
   public static long createFile(FileSystem fSys, Path path, int numBlocks,
       int blockSize) throws IOException {
       return createFile(fSys, path, numBlocks, blockSize, true);
-    }
+  }
 
   public static long createFile(FileSystem fSys, Path path) throws IOException {
-    return createFile(fSys, path, DEFAULT_NUM_BLOCKS, DEFAULT_BLOCK_SIZE, true);
+    return createFile(fSys, path, DEFAULT_NUM_BLOCKS, DEFAULT_BLOCK_SIZE, DEFAULT_NUM_REPL, true);
   }
 
   public static long createFile(FileSystem fSys, String name) throws IOException {

Modified: hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestChRootedFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestChRootedFileSystem.java?rev=1311518&r1=1311517&r2=1311518&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestChRootedFileSystem.java (original)
+++ hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestChRootedFileSystem.java Mon Apr  9 23:25:17 2012
@@ -23,6 +23,7 @@ import java.net.URI;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.FsConstants;
@@ -170,7 +171,15 @@ public class TestChRootedFileSystem {
     Assert.assertTrue(fSys.isDirectory(FileSystemTestHelper.getTestRootPath(fSys,"/newDir/dirFooBar")));
     Assert.assertTrue(fSysTarget.isDirectory(new Path(chrootedTo,"newDir/dirFooBar")));
   }
-  
+
+  @Test
+  public void testGetContentSummary() throws IOException {
+    // GetContentSummary of a dir
+    fSys.mkdirs(new Path("/newDir/dirFoo"));
+    ContentSummary cs = fSys.getContentSummary(new Path("/newDir/dirFoo"));
+    Assert.assertEquals(-1L, cs.getQuota());
+    Assert.assertEquals(-1L, cs.getSpaceQuota());
+  }
   
   /**
    * We would have liked renames across file system to fail but 

Modified: hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestNodeFencer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestNodeFencer.java?rev=1311518&r1=1311517&r2=1311518&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestNodeFencer.java (original)
+++ hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestNodeFencer.java Mon Apr  9 23:25:17 2012
@@ -132,8 +132,7 @@ public class TestNodeFencer {
       throws BadFencingConfigurationException {
     System.err.println("Testing configuration:\n" + confStr);
     Configuration conf = new Configuration();
-    conf.set(NodeFencer.CONF_METHODS_KEY, confStr);
-    return new NodeFencer(conf);
+    return new NodeFencer(conf, confStr);
   }
   
   /**

Modified: hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestShellCommandFencer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestShellCommandFencer.java?rev=1311518&r1=1311517&r2=1311518&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestShellCommandFencer.java (original)
+++ hadoop/common/branches/HDFS-3092/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestShellCommandFencer.java Mon Apr  9 23:25:17 2012
@@ -71,8 +71,7 @@ public class TestShellCommandFencer {
   public void testCheckNoArgs() {
     try {
       Configuration conf = new Configuration();
-      conf.set(NodeFencer.CONF_METHODS_KEY, "shell");
-      new NodeFencer(conf);
+      new NodeFencer(conf, "shell");
       fail("Didn't throw when passing no args to shell");
     } catch (BadFencingConfigurationException confe) {
       assertTrue(
@@ -85,8 +84,7 @@ public class TestShellCommandFencer {
   public void testCheckParensNoArgs() {
     try {
       Configuration conf = new Configuration();
-      conf.set(NodeFencer.CONF_METHODS_KEY, "shell()");
-      new NodeFencer(conf);
+      new NodeFencer(conf, "shell()");
       fail("Didn't throw when passing no args to shell");
     } catch (BadFencingConfigurationException confe) {
       assertTrue(