瀏覽代碼

HDFS-8441. Erasure Coding: make condition check earlier for setReplication. (waltersu4549)

Walter Su 10 年之前
父節點
當前提交
3d734df24c

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt

@@ -250,3 +250,6 @@
 
     HDFS-8294. Erasure Coding: Fix Findbug warnings present in erasure coding.
     (Rakesh R via zhz)
+
+    HDFS-8441. Erasure Coding: make condition check earlier for setReplication.
+    (waltersu4549)

+ 6 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java

@@ -380,7 +380,7 @@ public class FSDirAttrOp {
   static BlockInfoContiguous[] unprotectedSetReplication(
       FSDirectory fsd, String src, short replication, short[] blockRepls)
       throws QuotaExceededException, UnresolvedLinkException,
-             SnapshotAccessControlException {
+      SnapshotAccessControlException, UnsupportedActionException {
     assert fsd.hasWriteLock();
 
     final INodesInPath iip = fsd.getINodesInPath4Write(src, true);
@@ -389,6 +389,11 @@ public class FSDirAttrOp {
       return null;
     }
     INodeFile file = inode.asFile();
+    if (file.isStriped()) {
+      throw new UnsupportedActionException(
+          "Cannot set replication to a file with striped blocks");
+    }
+
     final short oldBR = file.getPreferredBlockReplication();
 
     // before setFileReplication, check for increasing block replication.

+ 19 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -2404,7 +2404,18 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     if (!DFSUtil.isValidName(src)) {
       throw new InvalidPathException(src);
     }
-    blockManager.verifyReplication(src, replication, clientMachine);
+
+    checkOperation(OperationCategory.READ);
+    readLock();
+    try {
+      checkOperation(OperationCategory.READ);
+      if (!isInECZone(src)) {
+        blockManager.verifyReplication(src, replication, clientMachine);
+      }
+    } finally {
+      readUnlock();
+    }
+    
     checkOperation(OperationCategory.WRITE);
     if (blockSize < minBlockSize) {
       throw new IOException("Specified block size is less than configured" +
@@ -7605,6 +7616,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     logAuditEvent(success, "createErasureCodingZone", srcArg, null, resultingStat);
   }
 
+  private boolean isInECZone(String src) throws IOException {
+    byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
+    src = FSDirectory.resolvePath(src, pathComponents, dir);
+    final INodesInPath iip = dir.getINodesInPath(src, true);
+    return dir.isInECZone(iip);
+  }
+
   /**
    * Get the erasure coding information for specified src
    */

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -429,8 +429,6 @@ public class INodeFile extends INodeWithAdditionalFields
   /** Set the replication factor of this file. */
   public final INodeFile setFileReplication(short replication,
       int latestSnapshotId) throws QuotaExceededException {
-    Preconditions.checkState(!isStriped(),
-        "Cannot set replication to a file with striped blocks");
     recordModification(latestSnapshotId);
     setFileReplication(replication);
     return this;

+ 23 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo;
@@ -35,6 +36,7 @@ import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
 import static org.junit.Assert.*;
 
 public class TestErasureCodingZones {
+  private Configuration conf;
   private MiniDFSCluster cluster;
   private DistributedFileSystem fs;
   private static final int BLOCK_SIZE = 1024;
@@ -42,7 +44,7 @@ public class TestErasureCodingZones {
 
   @Before
   public void setupCluster() throws IOException {
-    Configuration conf = new HdfsConfiguration();
+    conf = new HdfsConfiguration();
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
     cluster = new MiniDFSCluster.Builder(conf).
         numDataNodes(1).build();
@@ -149,6 +151,26 @@ public class TestErasureCodingZones {
     }
   }
 
+  @Test
+  public void testReplication() throws IOException {
+    final Path testDir = new Path("/ec");
+    fs.mkdir(testDir, FsPermission.getDirDefault());
+    fs.createErasureCodingZone(testDir, null, 0);
+    final Path fooFile = new Path(testDir, "foo");
+    // create ec file with replication=0
+    fs.create(fooFile, FsPermission.getFileDefault(), true,
+        conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+        (short)0, fs.getDefaultBlockSize(fooFile), null);
+
+    try {
+      fs.setReplication(fooFile, (short) 3);
+      fail("Shouldn't allow to set replication to a file with striped blocks");
+    } catch (IOException e) {
+      assertExceptionContains(
+          "Cannot set replication to a file with striped blocks", e);
+    }
+  }
+
   @Test
   public void testGetErasureCodingInfoWithSystemDefaultSchema() throws Exception {
     String src = "/ec";