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 bo...@apache.org on 2012/04/06 19:58:15 UTC

svn commit: r1310513 - in /hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common: ./ src/main/java/org/apache/hadoop/fs/ src/main/java/org/apache/hadoop/fs/viewfs/ src/test/java/org/apache/hadoop/fs/ src/test/java/org/apache/hadoop/fs/...

Author: bobby
Date: Fri Apr  6 17:58:15 2012
New Revision: 1310513

URL: http://svn.apache.org/viewvc?rev=1310513&view=rev
Log:
svn merge -c 1310507 from trunk. FIXES: HADOOP-8014. ViewFileSystem does not correctly implement getDefaultBlockSize, getDefaultReplication, getContentSummary (John George via bobby)

Modified:
    hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/CHANGES.txt
    hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
    hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java
    hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java
    hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
    hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemTestHelper.java
    hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestChRootedFileSystem.java

Modified: hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/CHANGES.txt?rev=1310513&r1=1310512&r2=1310513&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/CHANGES.txt (original)
+++ hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/CHANGES.txt Fri Apr  6 17:58:15 2012
@@ -24,6 +24,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 
 
   NEW FEATURES

Modified: hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java?rev=1310513&r1=1310512&r2=1310513&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java Fri Apr  6 17:58:15 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/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java?rev=1310513&r1=1310512&r2=1310513&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java Fri Apr  6 17:58:15 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/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java?rev=1310513&r1=1310512&r2=1310513&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java Fri Apr  6 17:58:15 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/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java?rev=1310513&r1=1310512&r2=1310513&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java Fri Apr  6 17:58:15 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/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemTestHelper.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemTestHelper.java?rev=1310513&r1=1310512&r2=1310513&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemTestHelper.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemTestHelper.java Fri Apr  6 17:58:15 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/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestChRootedFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestChRootedFileSystem.java?rev=1310513&r1=1310512&r2=1310513&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestChRootedFileSystem.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestChRootedFileSystem.java Fri Apr  6 17:58:15 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