Browse Source

HDFS-7839. Erasure coding: implement facilities in NameNode to create and manage EC zones. Contributed by Zhe Zhang

Zhe Zhang 10 năm trước cách đây
mục cha
commit
1af8c14862
20 tập tin đã thay đổi với 426 bổ sung114 xóa
  1. 15 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  2. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  3. 14 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  4. 16 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  5. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
  6. 112 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
  7. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
  8. 3 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
  9. 23 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  10. 40 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  11. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  12. 16 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  13. 9 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
  14. 0 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
  15. 151 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java
  16. 0 75
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInitialEncoding.java
  17. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
  18. 2 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
  19. 9 14
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
  20. 1 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java

+ 15 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -2994,6 +2994,21 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     return new EncryptionZoneIterator(namenode, traceSampler);
   }
 
+  public void createErasureCodingZone(String src)
+      throws IOException {
+    checkOpen();
+    TraceScope scope = getPathTraceScope("createErasureCodingZone", src);
+    try {
+      namenode.createErasureCodingZone(src);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+          SafeModeException.class,
+          UnresolvedPathException.class);
+    } finally {
+      scope.close();
+    }
+  }
+
   public void setXAttr(String src, String name, byte[] value, 
       EnumSet<XAttrSetFlag> flag) throws IOException {
     checkOpen();

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -1362,6 +1362,14 @@ public interface ClientProtocol {
   public BatchedEntries<EncryptionZone> listEncryptionZones(
       long prevId) throws IOException;
 
+  /**
+   * Create an erasure coding zone (currently with hardcoded schema)
+   * TODO: Configurable and pluggable schemas (HDFS-7337)
+   */
+  @Idempotent
+  public void createErasureCodingZone(String src)
+      throws IOException;
+
   /**
    * Set xattr of a file or directory.
    * The name must be prefixed with the namespace followed by ".". For example,

+ 14 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java

@@ -192,6 +192,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Update
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathResponseProto;
@@ -1390,6 +1392,18 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
     }
   }
 
+  @Override
+  public CreateErasureCodingZoneResponseProto createErasureCodingZone(
+      RpcController controller, CreateErasureCodingZoneRequestProto req)
+      throws ServiceException {
+    try {
+      server.createErasureCodingZone(req.getSrc());
+      return CreateErasureCodingZoneResponseProto.newBuilder().build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
   @Override
   public SetXAttrResponseProto setXAttr(RpcController controller,
       SetXAttrRequestProto req) throws ServiceException {

+ 16 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -160,6 +160,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Trunca
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetStoragePolicyRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto;
@@ -1406,6 +1408,20 @@ public class ClientNamenodeProtocolTranslatorPB implements
     }
   }
 
+  @Override
+  public void createErasureCodingZone(String src)
+      throws IOException {
+    final CreateErasureCodingZoneRequestProto.Builder builder =
+        CreateErasureCodingZoneRequestProto.newBuilder();
+    builder.setSrc(src);
+    CreateErasureCodingZoneRequestProto req = builder.build();
+    try {
+      rpcProxy.createErasureCodingZone(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
   @Override
   public void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag)
       throws IOException {

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java

@@ -396,4 +396,6 @@ public interface HdfsServerConstants {
       "raw.hdfs.crypto.file.encryption.info";
   String SECURITY_XATTR_UNREADABLE_BY_SUPERUSER =
       "security.hdfs.unreadable.by.superuser";
+  public static final String XATTR_ERASURECODING_ZONE =
+      "raw.hdfs.erasurecoding.zone";
 }

+ 112 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java

@@ -0,0 +1,112 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.apache.hadoop.fs.XAttr;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.hdfs.XAttrHelper;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.List;
+
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_ERASURECODING_ZONE;
+
+/**
+ * Manages the list of erasure coding zones in the filesystem.
+ * <p/>
+ * The ErasureCodingZoneManager has its own lock, but relies on the FSDirectory
+ * lock being held for many operations. The FSDirectory lock should not be
+ * taken if the manager lock is already held.
+ * TODO: consolidate zone logic w/ encrypt. zones {@link EncryptionZoneManager}
+ */
+public class ErasureCodingZoneManager {
+  private final FSDirectory dir;
+
+  /**
+   * Construct a new ErasureCodingZoneManager.
+   *
+   * @param dir Enclosing FSDirectory
+   */
+  public ErasureCodingZoneManager(FSDirectory dir) {
+    this.dir = dir;
+  }
+
+  boolean getECPolicy(INodesInPath iip) {
+    assert dir.hasReadLock();
+    Preconditions.checkNotNull(iip);
+    List<INode> inodes = iip.getReadOnlyINodes();
+    for (int i = inodes.size() - 1; i >= 0; i--) {
+      final INode inode = inodes.get(i);
+      if (inode == null) {
+        continue;
+      }
+      final List<XAttr> xAttrs = inode.getXAttrFeature() == null ?
+          new ArrayList<XAttr>(0)
+          : inode.getXAttrFeature().getXAttrs();
+      for (XAttr xAttr : xAttrs) {
+        if (XATTR_ERASURECODING_ZONE.equals(XAttrHelper.getPrefixName(xAttr))) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
+  XAttr createErasureCodingZone(String src)
+      throws IOException {
+    assert dir.hasWriteLock();
+    final INodesInPath srcIIP = dir.getINodesInPath4Write(src, false);
+    if (dir.isNonEmptyDirectory(srcIIP)) {
+      throw new IOException(
+          "Attempt to create an erasure coding zone for a " +
+              "non-empty directory.");
+    }
+    if (srcIIP != null &&
+        srcIIP.getLastINode() != null &&
+        !srcIIP.getLastINode().isDirectory()) {
+      throw new IOException("Attempt to create an erasure coding zone " +
+          "for a file.");
+    }
+    if (getECPolicy(srcIIP)) {
+      throw new IOException("Directory " + src + " is already in an " +
+          "erasure coding zone.");
+    }
+    final XAttr ecXAttr = XAttrHelper
+        .buildXAttr(XATTR_ERASURECODING_ZONE, null);
+    final List<XAttr> xattrs = Lists.newArrayListWithCapacity(1);
+    xattrs.add(ecXAttr);
+    FSDirXAttrOp.unprotectedSetXAttrs(dir, src, xattrs,
+        EnumSet.of(XAttrSetFlag.CREATE));
+    return ecXAttr;
+  }
+
+  void checkMoveValidity(INodesInPath srcIIP, INodesInPath dstIIP, String src)
+      throws IOException {
+    assert dir.hasReadLock();
+    if (getECPolicy(srcIIP)
+        != getECPolicy(dstIIP)) {
+      throw new IOException(
+          src + " can't be moved because the source and destination have " +
+              "different erasure coding policies.");
+    }
+  }
+}

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

@@ -185,6 +185,7 @@ class FSDirRenameOp {
     }
 
     fsd.ezManager.checkMoveValidity(srcIIP, dstIIP, src);
+    fsd.ecZoneManager.checkMoveValidity(srcIIP, dstIIP, src);
     // Ensure dst has quota to accommodate rename
     verifyFsLimitsForRename(fsd, srcIIP, dstIIP);
     verifyQuotaForRename(fsd, srcIIP, dstIIP);
@@ -357,6 +358,7 @@ class FSDirRenameOp {
 
     BlockStoragePolicySuite bsps = fsd.getBlockStoragePolicySuite();
     fsd.ezManager.checkMoveValidity(srcIIP, dstIIP, src);
+    fsd.ecZoneManager.checkMoveValidity(srcIIP, dstIIP, src);
     final INode dstInode = dstIIP.getLastINode();
     List<INodeDirectory> snapshottableDirs = new ArrayList<>();
     if (dstInode != null) { // Destination exists

+ 3 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java

@@ -42,6 +42,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
@@ -482,9 +483,6 @@ class FSDirWriteFileOp {
     try {
       INodesInPath iip = fsd.addINode(existing, newNode);
       if (iip != null) {
-        if (newNode.isStriped()) {
-          newNode.addStripedBlocksFeature();
-        }
         if (aclEntries != null) {
           AclStorage.updateINodeAcl(newNode, aclEntries, CURRENT_STATE_ID);
         }
@@ -560,9 +558,6 @@ class FSDirWriteFileOp {
     fsd.writeLock();
     try {
       newiip = fsd.addINode(existing, newNode);
-      if (newiip != null && newNode.isStriped()) {
-        newNode.addStripedBlocksFeature();
-      }
     } finally {
       fsd.writeUnlock();
     }
@@ -610,7 +605,7 @@ class FSDirWriteFileOp {
       }
     }
     final INodeFile file = fsn.checkLease(src, clientName, inode, fileId);
-    BlockInfoContiguous lastBlockInFile = file.getLastBlock();
+    BlockInfo lastBlockInFile = file.getLastBlock();
     if (!Block.matchingIdAndGenStamp(previousBlock, lastBlockInFile)) {
       // The block that the client claims is the current last block
       // doesn't match up with what we think is the last block. There are
@@ -638,7 +633,7 @@ class FSDirWriteFileOp {
       //    changed the namesystem state yet.
       //    We run this analysis again in Part II where case 4 is impossible.
 
-      BlockInfoContiguous penultimateBlock = file.getPenultimateBlock();
+      BlockInfo penultimateBlock = file.getPenultimateBlock();
       if (previous == null &&
           lastBlockInFile != null &&
           lastBlockInFile.getNumBytes() >= file.getPreferredBlockSize() &&

+ 23 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -205,6 +205,9 @@ public class FSDirectory implements Closeable {
   @VisibleForTesting
   public final EncryptionZoneManager ezManager;
 
+  @VisibleForTesting
+  public final ErasureCodingZoneManager ecZoneManager;
+
   /**
    * Caches frequently used file names used in {@link INode} to reuse 
    * byte[] objects and reduce heap usage.
@@ -296,6 +299,7 @@ public class FSDirectory implements Closeable {
     namesystem = ns;
     this.editLog = ns.getEditLog();
     ezManager = new EncryptionZoneManager(this, conf);
+    ecZoneManager = new ErasureCodingZoneManager(this);
   }
     
   FSNamesystem getFSNamesystem() {
@@ -1221,6 +1225,25 @@ public class FSDirectory implements Closeable {
     }
   }
 
+  XAttr createErasureCodingZone(String src)
+      throws IOException {
+    writeLock();
+    try {
+      return ecZoneManager.createErasureCodingZone(src);
+    } finally {
+      writeUnlock();
+    }
+  }
+
+  public boolean getECPolicy(INodesInPath iip) {
+    readLock();
+    try {
+      return ecZoneManager.getECPolicy(iip);
+    } finally {
+      readUnlock();
+    }
+  }
+
   static INode resolveLastINode(INodesInPath iip) throws FileNotFoundException {
     INode inode = iip.getLastINode();
     if (inode == null) {

+ 40 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -7510,6 +7510,46 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
   }
 
+  /**
+   * Create an erasure coding zone on directory src.
+   *
+   * @param src     the path of a directory which will be the root of the
+   *                erasure coding zone. The directory must be empty.
+   * @throws AccessControlException  if the caller is not the superuser.
+   * @throws UnresolvedLinkException if the path can't be resolved.
+   * @throws SafeModeException       if the Namenode is in safe mode.
+   */
+  void createErasureCodingZone(final String srcArg,
+      final boolean logRetryCache)
+      throws IOException, UnresolvedLinkException,
+      SafeModeException, AccessControlException {
+    String src = srcArg;
+    HdfsFileStatus resultingStat = null;
+    checkSuperuserPrivilege();
+    checkOperation(OperationCategory.WRITE);
+    final byte[][] pathComponents =
+        FSDirectory.getPathComponentsForReservedPath(src);
+    FSPermissionChecker pc = getPermissionChecker();
+    writeLock();
+    try {
+      checkSuperuserPrivilege();
+      checkOperation(OperationCategory.WRITE);
+      checkNameNodeSafeMode("Cannot create erasure coding zone on " + src);
+      src = dir.resolvePath(pc, src, pathComponents);
+
+      final XAttr ecXAttr = dir.createErasureCodingZone(src);
+      List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
+      xAttrs.add(ecXAttr);
+      getEditLog().logSetXAttrs(src, xAttrs, logRetryCache);
+      final INodesInPath iip = dir.getINodesInPath4Write(src, false);
+      resultingStat = dir.getAuditFileInfo(iip);
+    } finally {
+      writeUnlock();
+    }
+    getEditLog().logSync();
+    logAuditEvent(true, "createErasureCodingZone", srcArg, null, resultingStat);
+  }
+
   void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag,
                 boolean logRetryCache)
       throws IOException {

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

@@ -420,7 +420,7 @@ public class INodeFile extends INodeWithAdditionalFields
       }
       max = maxInSnapshot > max ? maxInSnapshot : max;
     }
-    return isStriped()?
+    return isStriped() ?
         HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS : max;
   }
 
@@ -1114,8 +1114,7 @@ public class INodeFile extends INodeWithAdditionalFields
    */
   @VisibleForTesting
   @Override
-  // TODO: move erasure coding policy to file XAttr
   public boolean isStriped() {
-    return getStoragePolicyID() == HdfsConstants.EC_STORAGE_POLICY_ID;
+    return getStripedBlocksFeature() != null;
   }
 }

+ 16 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java

@@ -1820,6 +1820,22 @@ class NameNodeRpcServer implements NamenodeProtocols {
     return namesystem.listEncryptionZones(prevId);
   }
 
+  @Override // ClientProtocol
+  public void createErasureCodingZone(String src)
+    throws IOException {
+    checkNNStartup();
+    final CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
+    if (cacheEntry != null && cacheEntry.isSuccess()) {
+      return;
+    }
+    boolean success = false;
+    try {
+      namesystem.createErasureCodingZone(src, cacheEntry != null);
+    } finally {
+      RetryCache.setState(cacheEntry, success);
+    }
+  }
+
   @Override // ClientProtocol
   public void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag)
       throws IOException {

+ 9 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto

@@ -714,6 +714,13 @@ message GetEditsFromTxidResponseProto {
   required EventsListProto eventsList = 1;
 }
 
+message CreateErasureCodingZoneRequestProto {
+  required string src = 1;
+}
+
+message CreateErasureCodingZoneResponseProto {
+}
+
 service ClientNamenodeProtocol {
   rpc getBlockLocations(GetBlockLocationsRequestProto)
       returns(GetBlockLocationsResponseProto);
@@ -856,6 +863,8 @@ service ClientNamenodeProtocol {
       returns(ListEncryptionZonesResponseProto);
   rpc getEZForPath(GetEZForPathRequestProto)
       returns(GetEZForPathResponseProto);
+  rpc createErasureCodingZone(CreateErasureCodingZoneRequestProto)
+      returns(CreateErasureCodingZoneResponseProto);
   rpc getCurrentEditLogTxid(GetCurrentEditLogTxidRequestProto)
       returns(GetCurrentEditLogTxidResponseProto);
   rpc getEditsFromTxid(GetEditsFromTxidRequestProto)

+ 0 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java

@@ -119,9 +119,6 @@ public class TestBlockStoragePolicy {
     expectedPolicyStrings.put(COLD,
         "BlockStoragePolicy{COLD:" + COLD + ", storageTypes=[ARCHIVE], " +
             "creationFallbacks=[], replicationFallbacks=[]}");
-    expectedPolicyStrings.put(EC,
-        "BlockStoragePolicy{EC:" + EC + ", storageTypes=[DISK], " +
-            "creationFallbacks=[], replicationFallbacks=[ARCHIVE]}");
     expectedPolicyStrings.put(WARM,
         "BlockStoragePolicy{WARM:" + WARM + ", storageTypes=[DISK, ARCHIVE], " +
             "creationFallbacks=[DISK, ARCHIVE], " +

+ 151 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java

@@ -0,0 +1,151 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class TestErasureCodingZones {
+  private final int NUM_OF_DATANODES = 3;
+  private Configuration conf;
+  private MiniDFSCluster cluster;
+  private DistributedFileSystem fs;
+  private static final int BLOCK_SIZE = 1024;
+  private FSNamesystem namesystem;
+
+  @Before
+  public void setupCluster() throws IOException {
+    conf = new HdfsConfiguration();
+    conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    cluster = new MiniDFSCluster.Builder(conf).
+        numDataNodes(NUM_OF_DATANODES).build();
+    cluster.waitActive();
+    fs = cluster.getFileSystem();
+    namesystem = cluster.getNamesystem();
+  }
+
+  @After
+  public void shutdownCluster() throws IOException {
+    cluster.shutdown();
+  }
+
+  @Test
+  public void testCreateECZone()
+      throws IOException, InterruptedException {
+    final Path testDir = new Path("/ec");
+    fs.mkdir(testDir, FsPermission.getDirDefault());
+
+    /* Normal creation of an erasure coding zone */
+    fs.getClient().createErasureCodingZone(testDir.toString());
+
+    /* Verify files under the zone are striped */
+    final Path ECFilePath = new Path(testDir, "foo");
+    fs.create(ECFilePath);
+    INode inode = namesystem.getFSDirectory().getINode(ECFilePath.toString());
+    assertTrue(inode.asFile().isStriped());
+
+    /* Verify that EC zone cannot be created on non-empty dir */
+    final Path notEmpty = new Path("/nonEmpty");
+    fs.mkdir(notEmpty, FsPermission.getDirDefault());
+    fs.create(new Path(notEmpty, "foo"));
+    try {
+      fs.getClient().createErasureCodingZone(notEmpty.toString());
+      fail("Erasure coding zone on non-empty dir");
+    } catch (IOException e) {
+      assertExceptionContains("erasure coding zone for a non-empty directory", e);
+    }
+
+    /* Verify that nested EC zones cannot be created */
+    final Path zone1 = new Path("/zone1");
+    final Path zone2 = new Path(zone1, "zone2");
+    fs.mkdir(zone1, FsPermission.getDirDefault());
+    fs.getClient().createErasureCodingZone(zone1.toString());
+    fs.mkdir(zone2, FsPermission.getDirDefault());
+    try {
+      fs.getClient().createErasureCodingZone(zone2.toString());
+      fail("Nested erasure coding zones");
+    } catch (IOException e) {
+      assertExceptionContains("already in an erasure coding zone", e);
+    }
+
+    /* Verify that EC zone cannot be created on a file */
+    final Path fPath = new Path("/file");
+    fs.create(fPath);
+    try {
+      fs.getClient().createErasureCodingZone(fPath.toString());
+      fail("Erasure coding zone on file");
+    } catch (IOException e) {
+      assertExceptionContains("erasure coding zone for a file", e);
+    }
+  }
+
+  @Test
+  public void testMoveValidity() throws IOException, InterruptedException {
+    final Path srcECDir = new Path("/srcEC");
+    final Path dstECDir = new Path("/dstEC");
+    fs.mkdir(srcECDir, FsPermission.getDirDefault());
+    fs.mkdir(dstECDir, FsPermission.getDirDefault());
+    fs.getClient().createErasureCodingZone(srcECDir.toString());
+    fs.getClient().createErasureCodingZone(dstECDir.toString());
+    final Path srcFile = new Path(srcECDir, "foo");
+    fs.create(srcFile);
+
+    /* Verify that a file can be moved between 2 EC zones */
+    try {
+      fs.rename(srcFile, dstECDir);
+    } catch (IOException e) {
+      fail("A file should be able to move between 2 EC zones " + e);
+    }
+
+    // Move the file back
+    fs.rename(new Path(dstECDir, "foo"), srcECDir);
+
+    /* Verify that a file cannot be moved from a non-EC dir to an EC zone */
+    final Path nonECDir = new Path("/nonEC");
+    fs.mkdir(nonECDir, FsPermission.getDirDefault());
+    try {
+      fs.rename(srcFile, nonECDir);
+      fail("A file shouldn't be able to move from a non-EC dir to an EC zone");
+    } catch (IOException e) {
+      assertExceptionContains("can't be moved because the source and " +
+          "destination have different erasure coding policies", e);
+    }
+
+    /* Verify that a file cannot be moved from an EC zone to a non-EC dir */
+    final Path nonECFile = new Path(nonECDir, "nonECFile");
+    fs.create(nonECFile);
+    try {
+      fs.rename(nonECFile, dstECDir);
+    } catch (IOException e) {
+      assertExceptionContains("can't be moved because the source and " +
+          "destination have different erasure coding policies", e);
+    }
+  }
+}

+ 0 - 75
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInitialEncoding.java

@@ -1,75 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.blockmanagement;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.*;
-import org.apache.hadoop.hdfs.client.HdfsAdmin;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.hdfs.server.namenode.INode;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.EC_STORAGE_POLICY_NAME;
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.EC_STORAGE_POLICY_ID;
-import static org.junit.Assert.assertEquals;
-
-public class TestBlockInitialEncoding {
-  private final int NUM_OF_DATANODES = 3;
-  private Configuration conf;
-  private MiniDFSCluster cluster;
-  private DistributedFileSystem fs;
-  private static final int BLOCK_SIZE = 1024;
-  private HdfsAdmin dfsAdmin;
-  private FSNamesystem namesystem;
-
-  @Before
-  public void setupCluster() throws IOException {
-    conf = new HdfsConfiguration();
-    conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
-    cluster = new MiniDFSCluster.Builder(conf).
-        numDataNodes(NUM_OF_DATANODES).build();
-    cluster.waitActive();
-    fs = cluster.getFileSystem();
-    dfsAdmin = new HdfsAdmin(cluster.getURI(), conf);
-    namesystem = cluster.getNamesystem();
-  }
-
-  @After
-  public void shutdownCluster() throws IOException {
-    cluster.shutdown();
-  }
-
-  @Test
-  public void testBlockInitialEncoding()
-      throws IOException, InterruptedException {
-    final Path testDir = new Path("/test");
-    fs.mkdir(testDir, FsPermission.getDirDefault());
-    dfsAdmin.setStoragePolicy(testDir, EC_STORAGE_POLICY_NAME);
-    final Path ECFilePath = new Path("/test/foo.ec");
-    DFSTestUtil.createFile(fs, ECFilePath, 4 * BLOCK_SIZE, (short) 3, 0);
-    INode inode = namesystem.getFSDirectory().getINode(ECFilePath.toString());
-    assertEquals(EC_STORAGE_POLICY_ID, inode.getStoragePolicyID());
-  }
-
-}

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

@@ -68,7 +68,7 @@ public class TestAddStripedBlocks {
         .numDataNodes(GROUP_SIZE).build();
     cluster.waitActive();
     dfs = cluster.getFileSystem();
-    dfs.setStoragePolicy(new Path("/"), HdfsConstants.EC_STORAGE_POLICY_NAME);
+    dfs.getClient().createErasureCodingZone("/");
   }
 
   @After

+ 2 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java

@@ -445,8 +445,7 @@ public class TestFSEditLogLoader {
 
       //set the storage policy of the directory
       fs.mkdir(new Path(testDir), new FsPermission("755"));
-      fs.setStoragePolicy(new Path(testDir),
-          HdfsConstants.EC_STORAGE_POLICY_NAME);
+      fs.getClient().getNamenode().createErasureCodingZone(testDir);
 
       // Create a file with striped block
       Path p = new Path(testFilePath);
@@ -518,8 +517,7 @@ public class TestFSEditLogLoader {
 
       //set the storage policy of the directory
       fs.mkdir(new Path(testDir), new FsPermission("755"));
-      fs.setStoragePolicy(new Path(testDir),
-          HdfsConstants.EC_STORAGE_POLICY_NAME);
+      fs.getClient().getNamenode().createErasureCodingZone(testDir);
 
       //create a file with striped blocks
       Path p = new Path(testFilePath);

+ 9 - 14
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java

@@ -33,18 +33,14 @@ import java.io.IOException;
 import java.util.EnumSet;
 
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.junit.Assert;
 
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -59,7 +55,6 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
 import org.apache.hadoop.hdfs.util.MD5FileUtils;
@@ -137,9 +132,10 @@ public class TestFSImage {
     }
   }
 
-  private void testSaveAndLoadINodeFile(FSNamesystem fsn, Configuration conf,
-      boolean isUC) throws IOException{
+  private void testSaveAndLoadStripedINodeFile(FSNamesystem fsn, Configuration conf,
+                                               boolean isUC) throws IOException{
     // contruct a INode with StripedBlock for saving and loading
+    fsn.createErasureCodingZone("/", false);
     long id = 123456789;
     byte[] name = "testSaveAndLoadInodeFile_testfile".getBytes();
     PermissionStatus permissionStatus = new PermissionStatus("testuser_a",
@@ -149,9 +145,8 @@ public class TestFSImage {
     BlockInfoContiguous[] blks = new BlockInfoContiguous[0];
     short replication = 3;
     long preferredBlockSize = 128*1024*1024;
-    byte storagePolicyID = HdfsConstants.EC_STORAGE_POLICY_ID;
     INodeFile file = new INodeFile(id, name, permissionStatus, mtime, atime,
-        blks, replication, preferredBlockSize, storagePolicyID);
+        blks, replication, preferredBlockSize);
     ByteArrayOutputStream bs = new ByteArrayOutputStream();
     file.addStripedBlocksFeature();
 
@@ -237,13 +232,13 @@ public class TestFSImage {
    * FSImageSerialization and loaded by FSImageFormat#Loader.
    */
   @Test
-  public void testSaveAndLoadInodeFile() throws IOException{
+  public void testSaveAndLoadStripedINodeFile() throws IOException{
     Configuration conf = new Configuration();
     MiniDFSCluster cluster = null;
     try {
       cluster = new MiniDFSCluster.Builder(conf).build();
       cluster.waitActive();
-      testSaveAndLoadINodeFile(cluster.getNamesystem(), conf, false);
+      testSaveAndLoadStripedINodeFile(cluster.getNamesystem(), conf, false);
     } finally {
       if (cluster != null) {
         cluster.shutdown();
@@ -256,14 +251,14 @@ public class TestFSImage {
    * saved and loaded by FSImageSerialization
    */
   @Test
-  public void testSaveAndLoadInodeFileUC() throws IOException{
+  public void testSaveAndLoadStripedINodeFileUC() throws IOException{
     // construct a INode with StripedBlock for saving and loading
     Configuration conf = new Configuration();
     MiniDFSCluster cluster = null;
     try {
       cluster = new MiniDFSCluster.Builder(conf).build();
       cluster.waitActive();
-      testSaveAndLoadINodeFile(cluster.getNamesystem(), conf, true);
+      testSaveAndLoadStripedINodeFile(cluster.getNamesystem(), conf, true);
     } finally {
       if (cluster != null) {
         cluster.shutdown();
@@ -402,7 +397,7 @@ public class TestFSImage {
           .build();
       cluster.waitActive();
       DistributedFileSystem fs = cluster.getFileSystem();
-      fs.setStoragePolicy(new Path("/"), HdfsConstants.EC_STORAGE_POLICY_NAME);
+      fs.getClient().getNamenode().createErasureCodingZone("/");
       Path file = new Path("/striped");
       FSDataOutputStream out = fs.create(file);
       byte[] bytes = DFSTestUtil.generateSequentialBytes(0, BLOCK_SIZE);

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

@@ -27,7 +27,6 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.client.HdfsAdmin;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
@@ -37,23 +36,19 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockECRecoveryInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.io.IOUtils;
 import org.junit.After;
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
 import java.io.IOException;
-import java.util.Iterator;
 import java.util.List;
 import java.util.UUID;
 
 import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CHUNK_SIZE;
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.EC_STORAGE_POLICY_NAME;
 import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -93,7 +88,7 @@ public class TestRecoverStripedBlocks {
       int numBlocks) throws Exception {
     DistributedFileSystem dfs = cluster.getFileSystem();
     dfs.mkdirs(dir);
-    dfs.setStoragePolicy(dir, EC_STORAGE_POLICY_NAME);
+    dfs.getClient().getNamenode().createErasureCodingZone(dir.toString());
 
     FSDataOutputStream out = null;
     try {