فهرست منبع

HADOOP-13715. Add isErasureCoded() API to FileStatus class. Contributed by Manoj Govindassamy.

Andrew Wang 8 سال پیش
والد
کامیت
52b00600df
20فایلهای تغییر یافته به همراه256 افزوده شده و 30 حذف شده
  1. 10 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java
  2. 7 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java
  3. 9 0
      hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
  4. 27 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
  5. 6 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewfsFileStatus.java
  6. 14 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/FsPermissionExtension.java
  7. 8 5
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
  8. 5 2
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
  9. 3 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
  10. 23 1
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
  11. 17 1
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHdfsHelper.java
  12. 6 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
  13. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
  14. 26 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicyWithSnapshot.java
  15. 10 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java
  16. 11 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECPolicy.java
  17. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java
  18. 60 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
  19. 3 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AGetFileStatus.java
  20. 4 2
      hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestGetFileStatus.java

+ 10 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java

@@ -215,7 +215,16 @@ public class FileStatus implements Writable, Comparable<FileStatus>,
   public boolean isEncrypted() {
   public boolean isEncrypted() {
     return permission.getEncryptedBit();
     return permission.getEncryptedBit();
   }
   }
-  
+
+  /**
+   * Tell whether the underlying file or directory is erasure coded or not.
+   *
+   * @return true if the underlying file or directory is erasure coded.
+   */
+  public boolean isErasureCoded() {
+    return permission.getErasureCodedBit();
+  }
+
   /**
   /**
    * Get the owner of the file.
    * Get the owner of the file.
    * @return owner of the file. The string could be empty if there is no
    * @return owner of the file. The string could be empty if there is no

+ 7 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java

@@ -312,6 +312,13 @@ public class FsPermission implements Writable, Serializable,
     return false;
     return false;
   }
   }
 
 
+  /**
+   * Returns true if the file or directory is erasure coded.
+   */
+  public boolean getErasureCodedBit() {
+    return false;
+  }
+
   /** Set the user file creation mask (umask) */
   /** Set the user file creation mask (umask) */
   public static void setUMask(Configuration conf, FsPermission umask) {
   public static void setUMask(Configuration conf, FsPermission umask) {
     conf.set(UMASK_LABEL, String.format("%1$03o", umask.toShort()));
     conf.set(UMASK_LABEL, String.format("%1$03o", umask.toShort()));

+ 9 - 0
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md

@@ -90,7 +90,16 @@ Get the status of a path
             stat.isEncrypted = True
             stat.isEncrypted = True
         else
         else
             stat.isEncrypted = False
             stat.isEncrypted = False
+        if isErasureCoded(FS, p) :
+            stat.isErasureCoded = True
+        else
+            stat.isErasureCoded = False
 
 
+The returned `FileStatus` status of the path additionally carries details on
+Encryption and Erasure Coding information. `getFileStatus(Path p).isEncrypted()`
+can be queried to find if the path is Encrypted.
+Likewise, `getFileStatus(Path p).isErasureCoded()` will tell if the path is
+Erasure Coded or not.
 
 
 ### `Path getHomeDirectory()`
 ### `Path getHomeDirectory()`
 
 

+ 27 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java

@@ -519,6 +519,33 @@ public class ContractTestUtils extends Assert {
                fileStatus.isDirectory());
                fileStatus.isDirectory());
   }
   }
 
 
+  /**
+   * Assert that a path is Erasure Coded.
+   *
+   * @param fs filesystem
+   * @param path path of the file or directory
+   * @throws IOException on File IO problems
+   */
+  public static void assertErasureCoded(final FileSystem fs, final Path path)
+      throws IOException {
+    FileStatus fileStatus = fs.getFileStatus(path);
+    assertTrue(path + " must be erasure coded!", fileStatus.isErasureCoded());
+  }
+
+  /**
+   * Assert that a path is not Erasure Coded.
+   *
+   * @param fs filesystem
+   * @param path path of the file or directory
+   * @throws IOException on File IO problems
+   */
+  public static void assertNotErasureCoded(final FileSystem fs,
+      final Path path) throws IOException {
+    FileStatus fileStatus = fs.getFileStatus(path);
+    assertFalse(path + " should not be erasure coded!",
+        fileStatus.isErasureCoded());
+  }
+
   /**
   /**
    * Write the text to a file, returning the converted byte array
    * Write the text to a file, returning the converted byte array
    * for use in validating the round trip.
    * for use in validating the round trip.

+ 6 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewfsFileStatus.java

@@ -28,6 +28,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
 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.contract.ContractTestUtils;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -70,8 +71,11 @@ public class TestViewfsFileStatus {
     ConfigUtil.addLink(conf, "/foo/bar/baz", TEST_DIR.toURI());
     ConfigUtil.addLink(conf, "/foo/bar/baz", TEST_DIR.toURI());
     FileSystem vfs = FileSystem.get(FsConstants.VIEWFS_URI, conf);
     FileSystem vfs = FileSystem.get(FsConstants.VIEWFS_URI, conf);
     assertEquals(ViewFileSystem.class, vfs.getClass());
     assertEquals(ViewFileSystem.class, vfs.getClass());
-    FileStatus stat = vfs.getFileStatus(new Path("/foo/bar/baz", testfilename));
+    Path path = new Path("/foo/bar/baz", testfilename);
+    FileStatus stat = vfs.getFileStatus(path);
     assertEquals(content.length, stat.getLen());
     assertEquals(content.length, stat.getLen());
+    ContractTestUtils.assertNotErasureCoded(vfs, path);
+
     // check serialization/deserialization
     // check serialization/deserialization
     DataOutputBuffer dob = new DataOutputBuffer();
     DataOutputBuffer dob = new DataOutputBuffer();
     stat.write(dob);
     stat.write(dob);
@@ -80,6 +84,7 @@ public class TestViewfsFileStatus {
     FileStatus deSer = new FileStatus();
     FileStatus deSer = new FileStatus();
     deSer.readFields(dib);
     deSer.readFields(dib);
     assertEquals(content.length, deSer.getLen());
     assertEquals(content.length, deSer.getLen());
+    assertFalse(deSer.isErasureCoded());
   }
   }
 
 
   // Tests that ViewFileSystem.getFileChecksum calls res.targetFileSystem
   // Tests that ViewFileSystem.getFileChecksum calls res.targetFileSystem

+ 14 - 3
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/FsPermissionExtension.java

@@ -33,8 +33,10 @@ public class FsPermissionExtension extends FsPermission {
 
 
   private final static short ACL_BIT = 1 << 12;
   private final static short ACL_BIT = 1 << 12;
   private final static short ENCRYPTED_BIT = 1 << 13;
   private final static short ENCRYPTED_BIT = 1 << 13;
+  private final static short ERASURE_CODED_BIT = 1 << 14;
   private final boolean aclBit;
   private final boolean aclBit;
   private final boolean encryptedBit;
   private final boolean encryptedBit;
+  private final boolean erasureCodedBit;
 
 
   /**
   /**
    * Constructs a new FsPermissionExtension based on the given FsPermission.
    * Constructs a new FsPermissionExtension based on the given FsPermission.
@@ -42,10 +44,11 @@ public class FsPermissionExtension extends FsPermission {
    * @param perm FsPermission containing permission bits
    * @param perm FsPermission containing permission bits
    */
    */
   public FsPermissionExtension(FsPermission perm, boolean hasAcl,
   public FsPermissionExtension(FsPermission perm, boolean hasAcl,
-      boolean isEncrypted) {
+      boolean isEncrypted, boolean isErasureCoded) {
     super(perm.toShort());
     super(perm.toShort());
     aclBit = hasAcl;
     aclBit = hasAcl;
     encryptedBit = isEncrypted;
     encryptedBit = isEncrypted;
+    erasureCodedBit = isErasureCoded;
   }
   }
 
 
   /**
   /**
@@ -57,12 +60,15 @@ public class FsPermissionExtension extends FsPermission {
     super(perm);
     super(perm);
     aclBit = (perm & ACL_BIT) != 0;
     aclBit = (perm & ACL_BIT) != 0;
     encryptedBit = (perm & ENCRYPTED_BIT) != 0;
     encryptedBit = (perm & ENCRYPTED_BIT) != 0;
+    erasureCodedBit = (perm & ERASURE_CODED_BIT) != 0;
   }
   }
 
 
   @Override
   @Override
   public short toExtendedShort() {
   public short toExtendedShort() {
-    return (short)(toShort() |
-        (aclBit ? ACL_BIT : 0) | (encryptedBit ? ENCRYPTED_BIT : 0));
+    return (short)(toShort()
+        | (aclBit ? ACL_BIT : 0)
+        | (encryptedBit ? ENCRYPTED_BIT : 0)
+        | (erasureCodedBit ? ERASURE_CODED_BIT : 0));
   }
   }
 
 
   @Override
   @Override
@@ -75,6 +81,11 @@ public class FsPermissionExtension extends FsPermission {
     return encryptedBit;
     return encryptedBit;
   }
   }
 
 
+  @Override
+  public boolean getErasureCodedBit() {
+    return erasureCodedBit;
+  }
+
   @Override
   @Override
   public boolean equals(Object o) {
   public boolean equals(Object o) {
     // This intentionally delegates to the base class.  This is only overridden
     // This intentionally delegates to the base class.  This is only overridden

+ 8 - 5
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java

@@ -98,12 +98,13 @@ class JsonUtilClient {
 
 
   /** Convert a string to a FsPermission object. */
   /** Convert a string to a FsPermission object. */
   static FsPermission toFsPermission(
   static FsPermission toFsPermission(
-      final String s, Boolean aclBit, Boolean encBit) {
+      final String s, Boolean aclBit, Boolean encBit, Boolean erasureBit) {
     FsPermission perm = new FsPermission(Short.parseShort(s, 8));
     FsPermission perm = new FsPermission(Short.parseShort(s, 8));
     final boolean aBit = (aclBit != null) ? aclBit : false;
     final boolean aBit = (aclBit != null) ? aclBit : false;
     final boolean eBit = (encBit != null) ? encBit : false;
     final boolean eBit = (encBit != null) ? encBit : false;
-    if (aBit || eBit) {
-      return new FsPermissionExtension(perm, aBit, eBit);
+    final boolean ecBit = (erasureBit != null) ? erasureBit : false;
+    if (aBit || eBit || ecBit) {
+      return new FsPermissionExtension(perm, aBit, eBit, ecBit);
     } else {
     } else {
       return perm;
       return perm;
     }
     }
@@ -129,7 +130,8 @@ class JsonUtilClient {
     final String group = (String) m.get("group");
     final String group = (String) m.get("group");
     final FsPermission permission = toFsPermission((String) m.get("permission"),
     final FsPermission permission = toFsPermission((String) m.get("permission"),
         (Boolean) m.get("aclBit"),
         (Boolean) m.get("aclBit"),
-        (Boolean) m.get("encBit"));
+        (Boolean) m.get("encBit"),
+        (Boolean) m.get("ecBit"));
     final long aTime = ((Number) m.get("accessTime")).longValue();
     final long aTime = ((Number) m.get("accessTime")).longValue();
     final long mTime = ((Number) m.get("modificationTime")).longValue();
     final long mTime = ((Number) m.get("modificationTime")).longValue();
     final long blockSize = ((Number) m.get("blockSize")).longValue();
     final long blockSize = ((Number) m.get("blockSize")).longValue();
@@ -464,7 +466,8 @@ class JsonUtilClient {
     String permString = (String) m.get("permission");
     String permString = (String) m.get("permission");
     if (permString != null) {
     if (permString != null) {
       final FsPermission permission = toFsPermission(permString,
       final FsPermission permission = toFsPermission(permString,
-          (Boolean) m.get("aclBit"), (Boolean) m.get("encBit"));
+          (Boolean) m.get("aclBit"), (Boolean) m.get("encBit"),
+          (Boolean) m.get("ecBit"));
       aclStatusBuilder.setPermission(permission);
       aclStatusBuilder.setPermission(permission);
     }
     }
     final List<?> entries = (List<?>) m.get("entries");
     final List<?> entries = (List<?>) m.get("entries");

+ 5 - 2
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java

@@ -193,6 +193,7 @@ public class HttpFSFileSystem extends FileSystem
   public static final String ACL_BIT_JSON = "aclBit";
   public static final String ACL_BIT_JSON = "aclBit";
 
 
   public static final String ENC_BIT_JSON = "encBit";
   public static final String ENC_BIT_JSON = "encBit";
+  public static final String EC_BIT_JSON = "ecBit";
 
 
   public static final String DIRECTORY_LISTING_JSON = "DirectoryListing";
   public static final String DIRECTORY_LISTING_JSON = "DirectoryListing";
   public static final String PARTIAL_LISTING_JSON = "partialListing";
   public static final String PARTIAL_LISTING_JSON = "partialListing";
@@ -1042,11 +1043,13 @@ public class HttpFSFileSystem extends FileSystem
     final String s = (String) json.get(PERMISSION_JSON);
     final String s = (String) json.get(PERMISSION_JSON);
     final Boolean aclBit = (Boolean) json.get(ACL_BIT_JSON);
     final Boolean aclBit = (Boolean) json.get(ACL_BIT_JSON);
     final Boolean encBit = (Boolean) json.get(ENC_BIT_JSON);
     final Boolean encBit = (Boolean) json.get(ENC_BIT_JSON);
+    final Boolean erasureBit = (Boolean) json.get(EC_BIT_JSON);
     FsPermission perm = new FsPermission(Short.parseShort(s, 8));
     FsPermission perm = new FsPermission(Short.parseShort(s, 8));
     final boolean aBit = (aclBit != null) ? aclBit : false;
     final boolean aBit = (aclBit != null) ? aclBit : false;
     final boolean eBit = (encBit != null) ? encBit : false;
     final boolean eBit = (encBit != null) ? encBit : false;
-    if (aBit || eBit) {
-      return new FsPermissionExtension(perm, aBit, eBit);
+    final boolean ecBit = (erasureBit != null) ? erasureBit : false;
+    if (aBit || eBit || ecBit) {
+      return new FsPermissionExtension(perm, aBit, eBit, ecBit);
     } else {
     } else {
       return perm;
       return perm;
     }
     }

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java

@@ -114,6 +114,9 @@ public class FSOperations {
     if (fileStatus.getPermission().getEncryptedBit()) {
     if (fileStatus.getPermission().getEncryptedBit()) {
       json.put(HttpFSFileSystem.ENC_BIT_JSON, true);
       json.put(HttpFSFileSystem.ENC_BIT_JSON, true);
     }
     }
+    if (fileStatus.getPermission().getErasureCodedBit()) {
+      json.put(HttpFSFileSystem.EC_BIT_JSON, true);
+    }
     return json;
     return json;
   }
   }
 
 

+ 23 - 1
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java

@@ -28,6 +28,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.http.server.HttpFSServerWebApp;
 import org.apache.hadoop.fs.http.server.HttpFSServerWebApp;
 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;
@@ -944,6 +945,24 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
     assertFalse(httpStatus.isEncrypted());
     assertFalse(httpStatus.isEncrypted());
   }
   }
 
 
+  private void testErasureCoding() throws Exception {
+    Assume.assumeFalse("Assume its not a local FS!", isLocalFS());
+    FileSystem proxyFs = FileSystem.get(getProxiedFSConf());
+    FileSystem httpFS = getHttpFSFileSystem();
+    Path filePath = new Path(getProxiedFSTestDir(), "foo.txt");
+    proxyFs.create(filePath).close();
+
+    ContractTestUtils.assertNotErasureCoded(httpFS, getProxiedFSTestDir());
+    ContractTestUtils.assertNotErasureCoded(httpFS, filePath);
+    ContractTestUtils.assertErasureCoded(httpFS,
+        TestHdfsHelper.ERASURE_CODING_DIR);
+    ContractTestUtils.assertErasureCoded(httpFS,
+        TestHdfsHelper.ERASURE_CODING_FILE);
+
+    proxyFs.close();
+    httpFS.close();
+  }
+
   private void testStoragePolicy() throws Exception {
   private void testStoragePolicy() throws Exception {
     Assume.assumeFalse("Assume its not a local FS", isLocalFS());
     Assume.assumeFalse("Assume its not a local FS", isLocalFS());
     FileSystem fs = FileSystem.get(getProxiedFSConf());
     FileSystem fs = FileSystem.get(getProxiedFSConf());
@@ -993,7 +1012,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
     WORKING_DIRECTORY, MKDIRS, SET_TIMES, SET_PERMISSION, SET_OWNER, 
     WORKING_DIRECTORY, MKDIRS, SET_TIMES, SET_PERMISSION, SET_OWNER, 
     SET_REPLICATION, CHECKSUM, CONTENT_SUMMARY, FILEACLS, DIRACLS, SET_XATTR,
     SET_REPLICATION, CHECKSUM, CONTENT_SUMMARY, FILEACLS, DIRACLS, SET_XATTR,
     GET_XATTRS, REMOVE_XATTR, LIST_XATTRS, ENCRYPTION, LIST_STATUS_BATCH,
     GET_XATTRS, REMOVE_XATTR, LIST_XATTRS, ENCRYPTION, LIST_STATUS_BATCH,
-    GETTRASHROOT, STORAGEPOLICY
+    GETTRASHROOT, STORAGEPOLICY, ERASURE_CODING
   }
   }
 
 
   private void operation(Operation op) throws Exception {
   private void operation(Operation op) throws Exception {
@@ -1079,6 +1098,9 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
     case STORAGEPOLICY:
     case STORAGEPOLICY:
       testStoragePolicy();
       testStoragePolicy();
       break;
       break;
+    case ERASURE_CODING:
+      testErasureCoding();
+      break;
     }
     }
   }
   }
 
 

+ 17 - 1
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHdfsHelper.java

@@ -31,6 +31,9 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
 import org.junit.Test;
 import org.junit.Test;
 import org.junit.runners.model.FrameworkMethod;
 import org.junit.runners.model.FrameworkMethod;
 import org.junit.runners.model.Statement;
 import org.junit.runners.model.Statement;
@@ -136,6 +139,10 @@ public class TestHdfsHelper extends TestDirHelper {
 
 
   public static final Path ENCRYPTION_ZONE = new Path("/ez");
   public static final Path ENCRYPTION_ZONE = new Path("/ez");
   public static final Path ENCRYPTED_FILE = new Path("/ez/encfile");
   public static final Path ENCRYPTED_FILE = new Path("/ez/encfile");
+  public static final Path ERASURE_CODING_DIR = new Path("/ec");
+  public static final Path ERASURE_CODING_FILE = new Path("/ec/ecfile");
+  public static final ErasureCodingPolicy ERASURE_CODING_POLICY =
+      ErasureCodingPolicyManager.getPolicyByID(HdfsConstants.XOR_2_1_POLICY_ID);
 
 
   private static MiniDFSCluster MINI_DFS = null;
   private static MiniDFSCluster MINI_DFS = null;
 
 
@@ -161,8 +168,12 @@ public class TestHdfsHelper extends TestDirHelper {
           new Path(helper.getTestRootDir(), "test.jks").toUri();
           new Path(helper.getTestRootDir(), "test.jks").toUri();
       conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
       conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
           jceksPath);
           jceksPath);
+      conf.set(DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_ENABLED_KEY,
+          ERASURE_CODING_POLICY.getName());
       MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
       MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
-      builder.numDataNodes(2);
+      int totalDataNodes = ERASURE_CODING_POLICY.getNumDataUnits() +
+          ERASURE_CODING_POLICY.getNumParityUnits();
+      builder.numDataNodes(totalDataNodes);
       MiniDFSCluster miniHdfs = builder.build();
       MiniDFSCluster miniHdfs = builder.build();
       final String testkey = "testkey";
       final String testkey = "testkey";
       DFSTestUtil.createKey(testkey, miniHdfs, conf);
       DFSTestUtil.createKey(testkey, miniHdfs, conf);
@@ -179,6 +190,11 @@ public class TestHdfsHelper extends TestDirHelper {
       fileSystem.createEncryptionZone(ENCRYPTION_ZONE, testkey);
       fileSystem.createEncryptionZone(ENCRYPTION_ZONE, testkey);
       fileSystem.create(ENCRYPTED_FILE).close();
       fileSystem.create(ENCRYPTED_FILE).close();
 
 
+      fileSystem.mkdirs(ERASURE_CODING_DIR);
+      fileSystem.setErasureCodingPolicy(ERASURE_CODING_DIR,
+          ERASURE_CODING_POLICY.getName());
+      fileSystem.create(ERASURE_CODING_FILE).close();
+
       MINI_DFS = miniHdfs;
       MINI_DFS = miniHdfs;
     }
     }
     return MINI_DFS;
     return MINI_DFS;

+ 6 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java

@@ -414,6 +414,7 @@ class FSDirStatAndListingOp {
 
 
     final ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp
     final ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp
         .unprotectedGetErasureCodingPolicy(fsd.getFSNamesystem(), iip);
         .unprotectedGetErasureCodingPolicy(fsd.getFSNamesystem(), iip);
+    final boolean isErasureCoded = (ecPolicy != null);
 
 
     if (node.isFile()) {
     if (node.isFile()) {
       final INodeFile fileNode = node.asFile();
       final INodeFile fileNode = node.asFile();
@@ -448,7 +449,7 @@ class FSDirStatAndListingOp {
         blocksize,
         blocksize,
         node.getModificationTime(snapshot),
         node.getModificationTime(snapshot),
         node.getAccessTime(snapshot),
         node.getAccessTime(snapshot),
-        getPermissionForFileStatus(nodeAttrs, isEncrypted),
+        getPermissionForFileStatus(nodeAttrs, isEncrypted, isErasureCoded),
         nodeAttrs.getUserName(),
         nodeAttrs.getUserName(),
         nodeAttrs.getGroupName(),
         nodeAttrs.getGroupName(),
         node.isSymlink() ? node.asSymlink().getSymlink() : null,
         node.isSymlink() ? node.asSymlink().getSymlink() : null,
@@ -489,11 +490,12 @@ class FSDirStatAndListingOp {
    * and encrypted bit on if it represents an encrypted file/dir.
    * and encrypted bit on if it represents an encrypted file/dir.
    */
    */
   private static FsPermission getPermissionForFileStatus(
   private static FsPermission getPermissionForFileStatus(
-      INodeAttributes node, boolean isEncrypted) {
+      INodeAttributes node, boolean isEncrypted, boolean isErasureCoded) {
     FsPermission perm = node.getFsPermission();
     FsPermission perm = node.getFsPermission();
     boolean hasAcl = node.getAclFeature() != null;
     boolean hasAcl = node.getAclFeature() != null;
-    if (hasAcl || isEncrypted) {
-      perm = new FsPermissionExtension(perm, hasAcl, isEncrypted);
+    if (hasAcl || isEncrypted || isErasureCoded) {
+      perm = new FsPermissionExtension(perm, hasAcl,
+          isEncrypted, isErasureCoded);
     }
     }
     return perm;
     return perm;
   }
   }

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java

@@ -124,6 +124,9 @@ public class JsonUtil {
     if (perm.getEncryptedBit()) {
     if (perm.getEncryptedBit()) {
       m.put("encBit", true);
       m.put("encBit", true);
     }
     }
+    if (perm.getErasureCodedBit()) {
+      m.put("ecBit", true);
+    }
     m.put("accessTime", status.getAccessTime());
     m.put("accessTime", status.getAccessTime());
     m.put("modificationTime", status.getModificationTime());
     m.put("modificationTime", status.getModificationTime());
     m.put("blockSize", status.getBlockSize());
     m.put("blockSize", status.getBlockSize());
@@ -376,6 +379,9 @@ public class JsonUtil {
       if (perm.getEncryptedBit()) {
       if (perm.getEncryptedBit()) {
         m.put("encBit", true);
         m.put("encBit", true);
       }
       }
+      if (perm.getErasureCodedBit()) {
+        m.put("ecBit", true);
+      }
     }
     }
     final Map<String, Map<String, Object>> finalMap =
     final Map<String, Map<String, Object>> finalMap =
         new TreeMap<String, Map<String, Object>>();
         new TreeMap<String, Map<String, Object>>();

+ 26 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicyWithSnapshot.java

@@ -25,6 +25,7 @@ import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
@@ -199,4 +200,29 @@ public class TestErasureCodingPolicyWithSnapshot {
     assertEquals("Got unexpected erasure coding policy", sysDefaultPolicy,
     assertEquals("Got unexpected erasure coding policy", sysDefaultPolicy,
         fs.getErasureCodingPolicy(snap1));
         fs.getErasureCodingPolicy(snap1));
   }
   }
+
+  @Test (timeout = 300000)
+  public void testFileStatusAcrossNNRestart() throws IOException {
+    final int len = 1024;
+    final Path normalFile = new Path("/", "normalFile");
+    DFSTestUtil.createFile(fs, normalFile, len, (short) 1, 0xFEED);
+
+    final Path ecDir = new Path("/ecdir");
+    final Path ecFile = new Path(ecDir, "ecFile");
+    fs.mkdirs(ecDir);
+
+    // Set erasure coding policy
+    fs.setErasureCodingPolicy(ecDir, sysDefaultPolicy.getName());
+    DFSTestUtil.createFile(fs, ecFile, len, (short) 1, 0xFEED);
+
+    // Verify FileStatus for normal and EC files
+    ContractTestUtils.assertNotErasureCoded(fs, normalFile);
+    ContractTestUtils.assertErasureCoded(fs, ecFile);
+
+    cluster.restartNameNode(true);
+
+    // Verify FileStatus for normal and EC files
+    ContractTestUtils.assertNotErasureCoded(fs, normalFile);
+    ContractTestUtils.assertErasureCoded(fs, ecFile);
+  }
 }
 }

+ 10 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java

@@ -33,6 +33,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RemoteException;
@@ -97,7 +98,8 @@ public class TestFileStatus {
     Path path = new Path("/");
     Path path = new Path("/");
     assertTrue("/ should be a directory", 
     assertTrue("/ should be a directory", 
                fs.getFileStatus(path).isDirectory());
                fs.getFileStatus(path).isDirectory());
-    
+    ContractTestUtils.assertNotErasureCoded(fs, path);
+
     // Make sure getFileInfo returns null for files which do not exist
     // Make sure getFileInfo returns null for files which do not exist
     HdfsFileStatus fileInfo = dfsClient.getFileInfo("/noSuchFile");
     HdfsFileStatus fileInfo = dfsClient.getFileInfo("/noSuchFile");
     assertEquals("Non-existant file should result in null", null, fileInfo);
     assertEquals("Non-existant file should result in null", null, fileInfo);
@@ -133,7 +135,8 @@ public class TestFileStatus {
     assertEquals(blockSize, status.getBlockSize());
     assertEquals(blockSize, status.getBlockSize());
     assertEquals(1, status.getReplication());
     assertEquals(1, status.getReplication());
     assertEquals(fileSize, status.getLen());
     assertEquals(fileSize, status.getLen());
-    assertEquals(file1.makeQualified(fs.getUri(), 
+    ContractTestUtils.assertNotErasureCoded(fs, file1);
+    assertEquals(file1.makeQualified(fs.getUri(),
         fs.getWorkingDirectory()).toString(), 
         fs.getWorkingDirectory()).toString(), 
         status.getPath().toString());
         status.getPath().toString());
   }
   }
@@ -148,10 +151,11 @@ public class TestFileStatus {
     assertEquals(blockSize, status.getBlockSize());
     assertEquals(blockSize, status.getBlockSize());
     assertEquals(1, status.getReplication());
     assertEquals(1, status.getReplication());
     assertEquals(fileSize, status.getLen());
     assertEquals(fileSize, status.getLen());
-    assertEquals(file1.makeQualified(fs.getUri(), 
+    ContractTestUtils.assertNotErasureCoded(fs, file1);
+    assertEquals(file1.makeQualified(fs.getUri(),
         fs.getWorkingDirectory()).toString(), 
         fs.getWorkingDirectory()).toString(), 
         status.getPath().toString());
         status.getPath().toString());
-    
+
     RemoteIterator<FileStatus> itor = fc.listStatus(file1);
     RemoteIterator<FileStatus> itor = fc.listStatus(file1);
     status = itor.next();
     status = itor.next();
     assertEquals(stats[0], status);
     assertEquals(stats[0], status);
@@ -196,7 +200,8 @@ public class TestFileStatus {
     FileStatus status = fs.getFileStatus(dir);
     FileStatus status = fs.getFileStatus(dir);
     assertTrue(dir + " should be a directory", status.isDirectory());
     assertTrue(dir + " should be a directory", status.isDirectory());
     assertTrue(dir + " should be zero size ", status.getLen() == 0);
     assertTrue(dir + " should be zero size ", status.getLen() == 0);
-    assertEquals(dir.makeQualified(fs.getUri(), 
+    ContractTestUtils.assertNotErasureCoded(fs, dir);
+    assertEquals(dir.makeQualified(fs.getUri(),
         fs.getWorkingDirectory()).toString(), 
         fs.getWorkingDirectory()).toString(), 
         status.getPath().toString());
         status.getPath().toString());
     
     

+ 11 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECPolicy.java

@@ -23,8 +23,8 @@ import static org.junit.Assert.assertTrue;
 
 
 import java.io.IOException;
 import java.io.IOException;
 
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.junit.After;
 import org.junit.After;
@@ -66,25 +66,32 @@ public class TestFileStatusWithECPolicy {
     // test directory doesn't have an EC policy
     // test directory doesn't have an EC policy
     final Path dir = new Path("/foo");
     final Path dir = new Path("/foo");
     assertTrue(fs.mkdir(dir, FsPermission.getDirDefault()));
     assertTrue(fs.mkdir(dir, FsPermission.getDirDefault()));
+    ContractTestUtils.assertNotErasureCoded(fs, dir);
     assertNull(client.getFileInfo(dir.toString()).getErasureCodingPolicy());
     assertNull(client.getFileInfo(dir.toString()).getErasureCodingPolicy());
     // test file doesn't have an EC policy
     // test file doesn't have an EC policy
     final Path file = new Path(dir, "foo");
     final Path file = new Path(dir, "foo");
     fs.create(file).close();
     fs.create(file).close();
     assertNull(client.getFileInfo(file.toString()).getErasureCodingPolicy());
     assertNull(client.getFileInfo(file.toString()).getErasureCodingPolicy());
+    ContractTestUtils.assertNotErasureCoded(fs, file);
     fs.delete(file, true);
     fs.delete(file, true);
 
 
-    final ErasureCodingPolicy ecPolicy1 = StripedFileTestUtil.getDefaultECPolicy();
+    final ErasureCodingPolicy ecPolicy1 =
+        StripedFileTestUtil.getDefaultECPolicy();
     // set EC policy on dir
     // set EC policy on dir
     fs.setErasureCodingPolicy(dir, ecPolicy1.getName());
     fs.setErasureCodingPolicy(dir, ecPolicy1.getName());
-    final ErasureCodingPolicy ecPolicy2 = client.getFileInfo(dir.toUri().getPath()).getErasureCodingPolicy();
+    ContractTestUtils.assertErasureCoded(fs, dir);
+    final ErasureCodingPolicy ecPolicy2 =
+        client.getFileInfo(dir.toUri().getPath()).getErasureCodingPolicy();
     assertNotNull(ecPolicy2);
     assertNotNull(ecPolicy2);
     assertTrue(ecPolicy1.equals(ecPolicy2));
     assertTrue(ecPolicy1.equals(ecPolicy2));
 
 
     // test file doesn't have an EC policy
     // test file doesn't have an EC policy
     fs.create(file).close();
     fs.create(file).close();
     final ErasureCodingPolicy ecPolicy3 =
     final ErasureCodingPolicy ecPolicy3 =
-        fs.getClient().getFileInfo(file.toUri().getPath()).getErasureCodingPolicy();
+        fs.getClient().getFileInfo(file.toUri().getPath())
+            .getErasureCodingPolicy();
     assertNotNull(ecPolicy3);
     assertNotNull(ecPolicy3);
     assertTrue(ecPolicy1.equals(ecPolicy3));
     assertTrue(ecPolicy1.equals(ecPolicy3));
+    ContractTestUtils.assertErasureCoded(fs, file);
   }
   }
 }
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java

@@ -862,7 +862,7 @@ public abstract class FSAclBaseTest {
     assertPermission((short)0700);
     assertPermission((short)0700);
     fs.setPermission(path,
     fs.setPermission(path,
       new FsPermissionExtension(FsPermission.
       new FsPermissionExtension(FsPermission.
-          createImmutable((short)0755), true, true));
+          createImmutable((short)0755), true, true, true));
     INode inode = cluster.getNamesystem().getFSDirectory()
     INode inode = cluster.getNamesystem().getFSDirectory()
         .getINode(path.toUri().getPath(), DirOp.READ_LINK);
         .getINode(path.toUri().getPath(), DirOp.READ_LINK);
     assertNotNull(inode);
     assertNotNull(inode);

+ 60 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java

@@ -65,6 +65,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclEntryScope;
 import org.apache.hadoop.fs.permission.AclEntryScope;
 import org.apache.hadoop.fs.permission.AclEntryType;
 import org.apache.hadoop.fs.permission.AclEntryType;
@@ -85,6 +86,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
 import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
 import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
@@ -513,6 +515,64 @@ public class TestWebHDFS {
     }
     }
   }
   }
 
 
+  @Test (timeout = 60000)
+  public void testWebHdfsErasureCodingFiles() throws Exception {
+    MiniDFSCluster cluster = null;
+    final Configuration conf = WebHdfsTestUtil.createConf();
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_ENABLED_KEY,
+        ErasureCodingPolicyManager.getPolicyByID(
+            HdfsConstants.XOR_2_1_POLICY_ID).getName());
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
+      cluster.waitActive();
+      final DistributedFileSystem dfs = cluster.getFileSystem();
+      final WebHdfsFileSystem webHdfs = WebHdfsTestUtil
+          .getWebHdfsFileSystem(conf, WebHdfsConstants.WEBHDFS_SCHEME);
+
+      final Path ecDir = new Path("/ec");
+      dfs.mkdirs(ecDir);
+      dfs.setErasureCodingPolicy(ecDir,
+          ErasureCodingPolicyManager.getPolicyByID(
+              HdfsConstants.XOR_2_1_POLICY_ID).getName());
+      final Path ecFile = new Path(ecDir, "ec-file.log");
+      DFSTestUtil.createFile(dfs, ecFile, 1024 * 10, (short) 1, 0xFEED);
+
+      final Path normalDir = new Path("/dir");
+      dfs.mkdirs(normalDir);
+      final Path normalFile = new Path(normalDir, "file.log");
+      DFSTestUtil.createFile(dfs, normalFile, 1024 * 10, (short) 1, 0xFEED);
+
+      FileStatus expectedECDirStatus = dfs.getFileStatus(ecDir);
+      FileStatus actualECDirStatus = webHdfs.getFileStatus(ecDir);
+      Assert.assertEquals(expectedECDirStatus.isErasureCoded(),
+          actualECDirStatus.isErasureCoded());
+      ContractTestUtils.assertErasureCoded(dfs, ecDir);
+
+      FileStatus expectedECFileStatus = dfs.getFileStatus(ecFile);
+      FileStatus actualECFileStatus = webHdfs.getFileStatus(ecFile);
+      Assert.assertEquals(expectedECFileStatus.isErasureCoded(),
+          actualECFileStatus.isErasureCoded());
+      ContractTestUtils.assertErasureCoded(dfs, ecFile);
+
+      FileStatus expectedNormalDirStatus = dfs.getFileStatus(normalDir);
+      FileStatus actualNormalDirStatus = webHdfs.getFileStatus(normalDir);
+      Assert.assertEquals(expectedNormalDirStatus.isErasureCoded(),
+          actualNormalDirStatus.isErasureCoded());
+      ContractTestUtils.assertNotErasureCoded(dfs, normalDir);
+
+      FileStatus expectedNormalFileStatus = dfs.getFileStatus(normalFile);
+      FileStatus actualNormalFileStatus = webHdfs.getFileStatus(normalDir);
+      Assert.assertEquals(expectedNormalFileStatus.isErasureCoded(),
+          actualNormalFileStatus.isErasureCoded());
+      ContractTestUtils.assertNotErasureCoded(dfs, normalFile);
+
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
   /**
   /**
    * Test snapshot creation through WebHdfs
    * Test snapshot creation through WebHdfs
    */
    */

+ 3 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AGetFileStatus.java

@@ -35,6 +35,7 @@ import com.amazonaws.services.s3.model.S3ObjectSummary;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 
 
+import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.hamcrest.BaseMatcher;
 import org.hamcrest.BaseMatcher;
 import org.hamcrest.Description;
 import org.hamcrest.Description;
 import org.hamcrest.Matcher;
 import org.hamcrest.Matcher;
@@ -60,6 +61,7 @@ public class TestS3AGetFileStatus extends AbstractS3AMockTest {
     assertTrue(stat.isFile());
     assertTrue(stat.isFile());
     assertEquals(meta.getContentLength(), stat.getLen());
     assertEquals(meta.getContentLength(), stat.getLen());
     assertEquals(meta.getLastModified().getTime(), stat.getModificationTime());
     assertEquals(meta.getLastModified().getTime(), stat.getModificationTime());
+    ContractTestUtils.assertNotErasureCoded(fs, path);
   }
   }
 
 
   @Test
   @Test
@@ -98,6 +100,7 @@ public class TestS3AGetFileStatus extends AbstractS3AMockTest {
     assertNotNull(stat);
     assertNotNull(stat);
     assertEquals(fs.makeQualified(path), stat.getPath());
     assertEquals(fs.makeQualified(path), stat.getPath());
     assertTrue(stat.isDirectory());
     assertTrue(stat.isDirectory());
+    ContractTestUtils.assertNotErasureCoded(fs, path);
   }
   }
 
 
   @Test
   @Test

+ 4 - 2
hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestGetFileStatus.java

@@ -51,8 +51,8 @@ public class TestGetFileStatus extends AdlMockWebServer {
     getMockServer().enqueue(new MockResponse().setResponseCode(200)
     getMockServer().enqueue(new MockResponse().setResponseCode(200)
         .setBody(TestADLResponseData.getGetFileStatusJSONResponse()));
         .setBody(TestADLResponseData.getGetFileStatusJSONResponse()));
     long startTime = Time.monotonicNow();
     long startTime = Time.monotonicNow();
-    FileStatus fileStatus = getMockAdlFileSystem()
-        .getFileStatus(new Path("/test1/test2"));
+    Path path = new Path("/test1/test2");
+    FileStatus fileStatus = getMockAdlFileSystem().getFileStatus(path);
     long endTime = Time.monotonicNow();
     long endTime = Time.monotonicNow();
     LOG.debug("Time : " + (endTime - startTime));
     LOG.debug("Time : " + (endTime - startTime));
     Assert.assertTrue(fileStatus.isFile());
     Assert.assertTrue(fileStatus.isFile());
@@ -65,6 +65,8 @@ public class TestGetFileStatus extends AdlMockWebServer {
     Assert.assertEquals(new FsPermission("777"), fileStatus.getPermission());
     Assert.assertEquals(new FsPermission("777"), fileStatus.getPermission());
     Assert.assertEquals("NotSupportYet", fileStatus.getOwner());
     Assert.assertEquals("NotSupportYet", fileStatus.getOwner());
     Assert.assertEquals("NotSupportYet", fileStatus.getGroup());
     Assert.assertEquals("NotSupportYet", fileStatus.getGroup());
+    Assert.assertFalse(path + " should not be erasure coded!",
+        fileStatus.isErasureCoded());
   }
   }
 
 
     @Test
     @Test