Browse Source

HADOOP-9631. ViewFs should use underlying FileSystem's server side defaults. Contributed by Lohit Vijayarenu and Erik Krogen.

(cherry picked from commit 59d69257a888347f0fb9c51bb000afc986b64f98)
(cherry picked from commit f98f4bb6ca45e6f807ca747d673690232e39da52)
(cherry picked from commit 8ca9915d0f833bff6dae75fdcd071b6ae79b6320)
Zhe Zhang 8 years ago
parent
commit
4cc53b51f6

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

@@ -120,6 +120,9 @@ Release 2.7.4 - UNRELEASED
 
 
     HADOOP-13433 Race in UGI.reloginFromKeytab. (Duo Zhang via xiao)
     HADOOP-13433 Race in UGI.reloginFromKeytab. (Duo Zhang via xiao)
 
 
+    HADOOP-9631. ViewFs should use underlying FileSystem's server side defaults.
+    (Lohit Vijayarenu and Erik Krogen via zhz)
+
 Release 2.7.3 - 2016-08-25
 Release 2.7.3 - 2016-08-25
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 14 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java

@@ -449,9 +449,21 @@ public abstract class AbstractFileSystem {
    * @return server default configuration values
    * @return server default configuration values
    * 
    * 
    * @throws IOException an I/O error occurred
    * @throws IOException an I/O error occurred
+   * @deprecated use {@link #getServerDefaults(Path)} instead
    */
    */
+  @Deprecated
   public abstract FsServerDefaults getServerDefaults() throws IOException; 
   public abstract FsServerDefaults getServerDefaults() throws IOException; 
 
 
+  /**
+   * Return a set of server default configuration values based on path.
+   * @param f path to fetch server defaults
+   * @return server default configuration values for path
+   * @throws IOException an I/O error occurred
+   */
+  public FsServerDefaults getServerDefaults(final Path f) throws IOException {
+    return getServerDefaults();
+  }
+
   /**
   /**
    * Return the fully-qualified path of path f resolving the path
    * Return the fully-qualified path of path f resolving the path
    * through any internal symlinks or mount point
    * through any internal symlinks or mount point
@@ -547,7 +559,7 @@ public abstract class AbstractFileSystem {
     }
     }
 
 
 
 
-    FsServerDefaults ssDef = getServerDefaults();
+    FsServerDefaults ssDef = getServerDefaults(f);
     if (ssDef.getBlockSize() % ssDef.getBytesPerChecksum() != 0) {
     if (ssDef.getBlockSize() % ssDef.getBytesPerChecksum() != 0) {
       throw new IOException("Internal error: default blockSize is" + 
       throw new IOException("Internal error: default blockSize is" + 
           " not a multiple of default bytesPerChecksum ");
           " not a multiple of default bytesPerChecksum ");
@@ -625,7 +637,7 @@ public abstract class AbstractFileSystem {
    */
    */
   public FSDataInputStream open(final Path f) throws AccessControlException,
   public FSDataInputStream open(final Path f) throws AccessControlException,
       FileNotFoundException, UnresolvedLinkException, IOException {
       FileNotFoundException, UnresolvedLinkException, IOException {
-    return open(f, getServerDefaults().getFileBufferSize());
+    return open(f, getServerDefaults(f).getFileBufferSize());
   }
   }
 
 
   /**
   /**

+ 7 - 6
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFs.java

@@ -57,7 +57,7 @@ public abstract class ChecksumFs extends FilterFs {
     throws IOException, URISyntaxException {
     throws IOException, URISyntaxException {
     super(theFs);
     super(theFs);
     defaultBytesPerChecksum = 
     defaultBytesPerChecksum = 
-      getMyFs().getServerDefaults().getBytesPerChecksum();
+      getMyFs().getServerDefaults(new Path("/")).getBytesPerChecksum();
   }
   }
   
   
   /**
   /**
@@ -96,9 +96,10 @@ public abstract class ChecksumFs extends FilterFs {
     return defaultBytesPerChecksum;
     return defaultBytesPerChecksum;
   }
   }
 
 
-  private int getSumBufferSize(int bytesPerSum, int bufferSize)
+  private int getSumBufferSize(int bytesPerSum, int bufferSize, Path file)
     throws IOException {
     throws IOException {
-    int defaultBufferSize =  getMyFs().getServerDefaults().getFileBufferSize();
+    int defaultBufferSize = getMyFs().getServerDefaults(file)
+        .getFileBufferSize();
     int proportionalBufferSize = bufferSize / bytesPerSum;
     int proportionalBufferSize = bufferSize / bytesPerSum;
     return Math.max(bytesPerSum,
     return Math.max(bytesPerSum,
                     Math.max(proportionalBufferSize, defaultBufferSize));
                     Math.max(proportionalBufferSize, defaultBufferSize));
@@ -121,7 +122,7 @@ public abstract class ChecksumFs extends FilterFs {
     
     
     public ChecksumFSInputChecker(ChecksumFs fs, Path file)
     public ChecksumFSInputChecker(ChecksumFs fs, Path file)
       throws IOException, UnresolvedLinkException {
       throws IOException, UnresolvedLinkException {
-      this(fs, file, fs.getServerDefaults().getFileBufferSize());
+      this(fs, file, fs.getServerDefaults(file).getFileBufferSize());
     }
     }
     
     
     public ChecksumFSInputChecker(ChecksumFs fs, Path file, int bufferSize)
     public ChecksumFSInputChecker(ChecksumFs fs, Path file, int bufferSize)
@@ -132,7 +133,7 @@ public abstract class ChecksumFs extends FilterFs {
       Path sumFile = fs.getChecksumFile(file);
       Path sumFile = fs.getChecksumFile(file);
       try {
       try {
         int sumBufferSize = fs.getSumBufferSize(fs.getBytesPerSum(),
         int sumBufferSize = fs.getSumBufferSize(fs.getBytesPerSum(),
-                                                bufferSize);
+            bufferSize, file);
         sums = fs.getRawFs().open(sumFile, sumBufferSize);
         sums = fs.getRawFs().open(sumFile, sumBufferSize);
 
 
         byte[] version = new byte[CHECKSUM_VERSION.length];
         byte[] version = new byte[CHECKSUM_VERSION.length];
@@ -355,7 +356,7 @@ public abstract class ChecksumFs extends FilterFs {
       
       
       // Now create the chekcsumfile; adjust the buffsize
       // Now create the chekcsumfile; adjust the buffsize
       int bytesPerSum = fs.getBytesPerSum();
       int bytesPerSum = fs.getBytesPerSum();
-      int sumBufferSize = fs.getSumBufferSize(bytesPerSum, bufferSize);
+      int sumBufferSize = fs.getSumBufferSize(bytesPerSum, bufferSize, file);
       this.sums = fs.getRawFs().createInternal(fs.getChecksumFile(file),
       this.sums = fs.getRawFs().createInternal(fs.getChecksumFile(file),
           EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE),
           EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE),
           absolutePermission, sumBufferSize, replication, blockSize, progress,
           absolutePermission, sumBufferSize, replication, blockSize, progress,

+ 6 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DelegateToFileSystem.java

@@ -149,10 +149,16 @@ public abstract class DelegateToFileSystem extends AbstractFileSystem {
   }
   }
 
 
   @Override
   @Override
+  @Deprecated
   public FsServerDefaults getServerDefaults() throws IOException {
   public FsServerDefaults getServerDefaults() throws IOException {
     return fsImpl.getServerDefaults();
     return fsImpl.getServerDefaults();
   }
   }
   
   
+  @Override
+  public FsServerDefaults getServerDefaults(final Path f) throws IOException {
+    return fsImpl.getServerDefaults(f);
+  }
+
   @Override
   @Override
   public Path getHomeDirectory() {
   public Path getHomeDirectory() {
     return fsImpl.getHomeDirectory();
     return fsImpl.getHomeDirectory();

+ 5 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFs.java

@@ -146,10 +146,15 @@ public abstract class FilterFs extends AbstractFileSystem {
   }
   }
 
 
   @Override
   @Override
+  @Deprecated
   public FsServerDefaults getServerDefaults() throws IOException {
   public FsServerDefaults getServerDefaults() throws IOException {
     return myFs.getServerDefaults();
     return myFs.getServerDefaults();
   }
   }
   
   
+  @Override
+  public FsServerDefaults getServerDefaults(final Path f) throws IOException {
+    return myFs.getServerDefaults(f);
+  }
 
 
   @Override
   @Override
   public Path resolvePath(final Path p) throws FileNotFoundException,
   public Path resolvePath(final Path p) throws FileNotFoundException,

+ 7 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ftp/FtpFs.java

@@ -29,6 +29,7 @@ import org.apache.hadoop.fs.AbstractFileSystem;
 import org.apache.hadoop.fs.DelegateToFileSystem;
 import org.apache.hadoop.fs.DelegateToFileSystem;
 import org.apache.hadoop.fs.FsConstants;
 import org.apache.hadoop.fs.FsConstants;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.FsServerDefaults;
+import org.apache.hadoop.fs.Path;
 
 
 /**
 /**
  * The FtpFs implementation of AbstractFileSystem.
  * The FtpFs implementation of AbstractFileSystem.
@@ -57,7 +58,13 @@ public class FtpFs extends DelegateToFileSystem {
   }
   }
   
   
   @Override
   @Override
+  @Deprecated
   public FsServerDefaults getServerDefaults() throws IOException {
   public FsServerDefaults getServerDefaults() throws IOException {
     return FtpConfigKeys.getServerDefaults();
     return FtpConfigKeys.getServerDefaults();
   }
   }
+
+  @Override
+  public FsServerDefaults getServerDefaults(final Path f) throws IOException {
+    return FtpConfigKeys.getServerDefaults();
+  }
 }
 }

+ 8 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/local/RawLocalFs.java

@@ -28,6 +28,7 @@ import org.apache.hadoop.fs.AbstractFileSystem;
 import org.apache.hadoop.fs.DelegateToFileSystem;
 import org.apache.hadoop.fs.DelegateToFileSystem;
 import org.apache.hadoop.fs.FsConstants;
 import org.apache.hadoop.fs.FsConstants;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.FsServerDefaults;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RawLocalFileSystem;
 import org.apache.hadoop.fs.RawLocalFileSystem;
 
 
 /**
 /**
@@ -63,6 +64,13 @@ public class RawLocalFs extends DelegateToFileSystem {
   }
   }
   
   
   @Override
   @Override
+  public FsServerDefaults getServerDefaults(final Path f)
+      throws IOException {
+    return LocalConfigKeys.getServerDefaults();
+  }
+
+  @Override
+  @Deprecated
   public FsServerDefaults getServerDefaults() throws IOException {
   public FsServerDefaults getServerDefaults() throws IOException {
     return LocalConfigKeys.getServerDefaults();
     return LocalConfigKeys.getServerDefaults();
   }
   }

+ 6 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFs.java

@@ -221,10 +221,16 @@ class ChRootedFs extends AbstractFileSystem {
   }
   }
 
 
   @Override
   @Override
+  @Deprecated
   public FsServerDefaults getServerDefaults() throws IOException {
   public FsServerDefaults getServerDefaults() throws IOException {
     return myFs.getServerDefaults();
     return myFs.getServerDefaults();
   }
   }
 
 
+  @Override
+  public FsServerDefaults getServerDefaults(final Path f) throws IOException {
+    return myFs.getServerDefaults(fullPath(f));
+  }
+
   @Override
   @Override
   public int getUriDefaultPort() {
   public int getUriDefaultPort() {
     return myFs.getUriDefaultPort();
     return myFs.getUriDefaultPort();

+ 19 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java

@@ -239,10 +239,22 @@ public class ViewFs extends AbstractFileSystem {
   }
   }
 
 
   @Override
   @Override
+  @Deprecated
   public FsServerDefaults getServerDefaults() throws IOException {
   public FsServerDefaults getServerDefaults() throws IOException {
     return LocalConfigKeys.getServerDefaults(); 
     return LocalConfigKeys.getServerDefaults(); 
   }
   }
 
 
+  @Override
+  public FsServerDefaults getServerDefaults(final Path f) throws IOException {
+    InodeTree.ResolveResult<AbstractFileSystem> res;
+    try {
+      res = fsState.resolve(getUriPath(f), true);
+    } catch (FileNotFoundException fnfe) {
+      return LocalConfigKeys.getServerDefaults();
+    }
+    return res.targetFileSystem.getServerDefaults(res.remainingPath);
+  }
+
   @Override
   @Override
   public int getUriDefaultPort() {
   public int getUriDefaultPort() {
     return -1;
     return -1;
@@ -871,8 +883,14 @@ public class ViewFs extends AbstractFileSystem {
     }
     }
 
 
     @Override
     @Override
+    @Deprecated
     public FsServerDefaults getServerDefaults() throws IOException {
     public FsServerDefaults getServerDefaults() throws IOException {
-      throw new IOException("FsServerDefaults not implemented yet");
+      return LocalConfigKeys.getServerDefaults();
+    }
+
+    @Override
+    public FsServerDefaults getServerDefaults(final Path f) throws IOException {
+      return LocalConfigKeys.getServerDefaults();
     }
     }
 
 
     @Override
     @Override

+ 1 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestAfsCheckPath.java

@@ -117,6 +117,7 @@ public class TestAfsCheckPath {
     }
     }
 
 
     @Override
     @Override
+    @Deprecated
     public FsServerDefaults getServerDefaults() throws IOException {
     public FsServerDefaults getServerDefaults() throws IOException {
       // deliberately empty
       // deliberately empty
       return null;
       return null;

+ 49 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFsBaseTest.java

@@ -38,6 +38,7 @@ import java.io.IOException;
 import java.net.URI;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
 import java.util.ArrayList;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
@@ -45,8 +46,10 @@ import java.util.Map;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.AbstractFileSystem;
 import org.apache.hadoop.fs.AbstractFileSystem;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileContextTestHelper;
 import org.apache.hadoop.fs.FileContextTestHelper;
+import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.FileContextTestHelper.fileType;
 import org.apache.hadoop.fs.FileContextTestHelper.fileType;
@@ -54,6 +57,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FsConstants;
 import org.apache.hadoop.fs.FsConstants;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.fs.local.LocalConfigKeys;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.AclUtil;
 import org.apache.hadoop.fs.permission.AclUtil;
@@ -797,6 +801,51 @@ public class ViewFsBaseTest {
     fcView.removeXAttr(new Path("/internalDir"), "xattrName");
     fcView.removeXAttr(new Path("/internalDir"), "xattrName");
   }
   }
 
 
+  @Test
+  public void testRespectsServerDefaults() throws Exception {
+    FsServerDefaults targetDefs =
+        fcTarget.getDefaultFileSystem().getServerDefaults(new Path("/"));
+    FsServerDefaults viewDefs =
+        fcView.getDefaultFileSystem().getServerDefaults(new Path("/data"));
+    assertEquals(targetDefs.getReplication(), viewDefs.getReplication());
+    assertEquals(targetDefs.getBlockSize(), viewDefs.getBlockSize());
+    assertEquals(targetDefs.getBytesPerChecksum(),
+        viewDefs.getBytesPerChecksum());
+    assertEquals(targetDefs.getFileBufferSize(),
+        viewDefs.getFileBufferSize());
+    assertEquals(targetDefs.getWritePacketSize(),
+        viewDefs.getWritePacketSize());
+    assertEquals(targetDefs.getEncryptDataTransfer(),
+        viewDefs.getEncryptDataTransfer());
+    assertEquals(targetDefs.getTrashInterval(), viewDefs.getTrashInterval());
+    assertEquals(targetDefs.getChecksumType(), viewDefs.getChecksumType());
+
+    fcView.create(new Path("/data/file"), EnumSet.of(CreateFlag.CREATE))
+        .close();
+    FileStatus stat =
+        fcTarget.getFileStatus(new Path(targetTestRoot, "data/file"));
+    assertEquals(targetDefs.getReplication(), stat.getReplication());
+  }
+
+  @Test
+  public void testServerDefaultsInternalDir() throws Exception {
+    FsServerDefaults localDefs = LocalConfigKeys.getServerDefaults();
+    FsServerDefaults viewDefs = fcView
+        .getDefaultFileSystem().getServerDefaults(new Path("/internalDir"));
+    assertEquals(localDefs.getReplication(), viewDefs.getReplication());
+    assertEquals(localDefs.getBlockSize(), viewDefs.getBlockSize());
+    assertEquals(localDefs.getBytesPerChecksum(),
+        viewDefs.getBytesPerChecksum());
+    assertEquals(localDefs.getFileBufferSize(),
+        viewDefs.getFileBufferSize());
+    assertEquals(localDefs.getWritePacketSize(),
+        viewDefs.getWritePacketSize());
+    assertEquals(localDefs.getEncryptDataTransfer(),
+        viewDefs.getEncryptDataTransfer());
+    assertEquals(localDefs.getTrashInterval(), viewDefs.getTrashInterval());
+    assertEquals(localDefs.getChecksumType(), viewDefs.getChecksumType());
+  }
+
   // Confirm that listLocatedStatus is delegated properly to the underlying
   // Confirm that listLocatedStatus is delegated properly to the underlying
   // AbstractFileSystem to allow for optimizations
   // AbstractFileSystem to allow for optimizations
   @Test
   @Test

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/Hdfs.java

@@ -152,10 +152,16 @@ public class Hdfs extends AbstractFileSystem {
   }
   }
 
 
   @Override
   @Override
+  @Deprecated
   public FsServerDefaults getServerDefaults() throws IOException {
   public FsServerDefaults getServerDefaults() throws IOException {
     return dfs.getServerDefaults();
     return dfs.getServerDefaults();
   }
   }
 
 
+  @Override
+  public FsServerDefaults getServerDefaults(final Path f) throws IOException {
+    return dfs.getServerDefaults();
+  }
+
   @Override
   @Override
   public RemoteIterator<LocatedFileStatus> listLocatedStatus(
   public RemoteIterator<LocatedFileStatus> listLocatedStatus(
       final Path p)
       final Path p)