ソースを参照

HDFS-9348. Erasure Coding: DFS GetErasureCodingPolicy API on a non-existent file should be handled properly. (Rakesh R via umamahesh)

Uma Mahesh 9 年 前
コミット
5b714a2819

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

@@ -864,6 +864,9 @@ Trunk (Unreleased)
       HDFS-9275. Wait previous ErasureCodingWork to finish before schedule
       another one. (Walter Su via yliu)
 
+      HDFS-9348. Erasure Coding: DFS GetErasureCodingPolicy API on a non-existent 
+      file should be handled properly. (Rakesh R via umamahesh)
+
 Release 2.8.0 - UNRELEASED
 
   NEW FEATURES

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java

@@ -21,6 +21,7 @@ import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.EnumSet;
@@ -154,12 +155,16 @@ final class FSDirErasureCodingOp {
    * @param src path
    * @return {@link ErasureCodingPolicy}
    * @throws IOException
+   * @throws FileNotFoundException if the path does not exist.
    */
   static ErasureCodingPolicy getErasureCodingPolicy(final FSNamesystem fsn,
       final String src) throws IOException {
     assert fsn.hasReadLock();
 
     final INodesInPath iip = getINodesInPath(fsn, src);
+    if (iip.getLastINode() == null) {
+      throw new FileNotFoundException("Path not found: " + iip.getPath());
+    }
     return getErasureCodingPolicyForPath(fsn, iip);
   }
 

+ 33 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java

@@ -25,12 +25,14 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.client.HdfsAdmin;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.Collection;
 
@@ -57,10 +59,12 @@ public class TestErasureCodingPolicies {
 
   @After
   public void shutdownCluster() throws IOException {
-    cluster.shutdown();
+    if (cluster != null) {
+      cluster.shutdown();
+    }
   }
 
-  @Test
+  @Test(timeout = 60000)
   public void testBasicSetECPolicy()
       throws IOException, InterruptedException {
     final Path testDir = new Path("/ec");
@@ -115,7 +119,7 @@ public class TestErasureCodingPolicies {
     }
   }
 
-  @Test
+  @Test(timeout = 60000)
   public void testMoveValidity() throws IOException, InterruptedException {
     final Path srcECDir = new Path("/srcEC");
     final Path dstECDir = new Path("/dstEC");
@@ -152,7 +156,7 @@ public class TestErasureCodingPolicies {
     fs.rename(nonECFile, dstECDir);
   }
 
-  @Test
+  @Test(timeout = 60000)
   public void testReplication() throws IOException {
     final Path testDir = new Path("/ec");
     fs.mkdir(testDir, FsPermission.getDirDefault());
@@ -166,7 +170,7 @@ public class TestErasureCodingPolicies {
     fs.setReplication(fooFile, (short) 3);
   }
 
-  @Test
+  @Test(timeout = 60000)
   public void testGetErasureCodingPolicyWithSystemDefaultECPolicy() throws Exception {
     String src = "/ec";
     final Path ecDir = new Path(src);
@@ -182,7 +186,7 @@ public class TestErasureCodingPolicies {
     verifyErasureCodingInfo(src + "/child1", sysDefaultECPolicy);
   }
 
-  @Test
+  @Test(timeout = 60000)
   public void testGetErasureCodingPolicy() throws Exception {
     ErasureCodingPolicy[] sysECPolicies = ErasureCodingPolicyManager.getSystemPolices();
     assertTrue("System ecPolicies should be of only 1 for now",
@@ -211,7 +215,7 @@ public class TestErasureCodingPolicies {
         usingECPolicy, ecPolicy);
   }
 
-  @Test
+  @Test(timeout = 60000)
   public void testCreationErasureCodingZoneWithInvalidPolicy()
       throws IOException {
     ECSchema rsSchema = new ECSchema("rs", 4, 2);
@@ -219,7 +223,7 @@ public class TestErasureCodingPolicies {
     int cellSize = 128 * 1024;
     ErasureCodingPolicy ecPolicy=
         new ErasureCodingPolicy(policyName,rsSchema,cellSize);
-    String src = "/ecZone4-2";
+    String src = "/ecDir4-2";
     final Path ecDir = new Path(src);
     try {
       fs.mkdir(ecDir, FsPermission.getDirDefault());
@@ -232,7 +236,7 @@ public class TestErasureCodingPolicies {
     }
   }
 
-  @Test
+  @Test(timeout = 60000)
   public void testGetAllErasureCodingPolicies() throws Exception {
     ErasureCodingPolicy[] sysECPolicies = ErasureCodingPolicyManager
         .getSystemPolices();
@@ -246,4 +250,24 @@ public class TestErasureCodingPolicies {
     assertEquals("Erasure coding policy mismatches",
         sysECPolicies[0], allECPolicies.iterator().next());
   }
+
+  @Test(timeout = 60000)
+  public void testGetErasureCodingPolicyOnANonExistentFile() throws Exception {
+    Path path = new Path("/ecDir");
+    try {
+      fs.getErasureCodingPolicy(path);
+      fail("FileNotFoundException should be thrown for a non-existent"
+          + " file path");
+    } catch (FileNotFoundException e) {
+      assertExceptionContains("Path not found: " + path, e);
+    }
+    HdfsAdmin dfsAdmin = new HdfsAdmin(cluster.getURI(), conf);
+    try {
+      dfsAdmin.getErasureCodingPolicy(path);
+      fail("FileNotFoundException should be thrown for a non-existent"
+          + " file path");
+    } catch (FileNotFoundException e) {
+      assertExceptionContains("Path not found: " + path, e);
+    }
+  }
 }