diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index 60e5577028501..0c6383b4dcb18 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -2994,6 +2994,21 @@ public RemoteIterator listEncryptionZones() 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 flag) throws IOException { checkOpen(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java index 55faf1662c6db..a94caad972a82 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java @@ -1362,6 +1362,14 @@ public EncryptionZone getEZForPath(String src) public BatchedEntries 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, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java index ae5745a760cf0..5be9bc6d8d7fc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java +++ b/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.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 ListEncryptionZonesResponseProto listEncryptionZones( } } + @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 { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java index 22da083c98882..43a0322c25c5b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java +++ b/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.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 BatchedEntries listEncryptionZones(long id) } } + @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 flag) throws IOException { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java index 26a7ab34ddb68..2ebc92713aebc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java @@ -396,4 +396,6 @@ enum BlockUCState { "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"; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java new file mode 100644 index 0000000000000..d4ff7c56fe142 --- /dev/null +++ b/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. + *

+ * 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 inodes = iip.getReadOnlyINodes(); + for (int i = inodes.size() - 1; i >= 0; i--) { + final INode inode = inodes.get(i); + if (inode == null) { + continue; + } + final List xAttrs = inode.getXAttrFeature() == null ? + new ArrayList(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 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."); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java index b69bb4265b3f6..127474cc295e1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java @@ -185,6 +185,7 @@ static boolean unprotectedRenameTo(FSDirectory fsd, String src, String dst, } 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 @@ static boolean unprotectedRenameTo(FSDirectory fsd, String src, String dst, BlockStoragePolicySuite bsps = fsd.getBlockStoragePolicySuite(); fsd.ezManager.checkMoveValidity(srcIIP, dstIIP, src); + fsd.ecZoneManager.checkMoveValidity(srcIIP, dstIIP, src); final INode dstInode = dstIIP.getLastINode(); List snapshottableDirs = new ArrayList<>(); if (dstInode != null) { // Destination exists diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java index 325d9d34e8115..fa17d9df9b024 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java +++ b/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.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 @@ static INodeFile addFileForEditLog( 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 @@ private static INodesInPath addFile( fsd.writeLock(); try { newiip = fsd.addINode(existing, newNode); - if (newiip != null && newNode.isStriped()) { - newNode.addStripedBlocksFeature(); - } } finally { fsd.writeUnlock(); } @@ -610,7 +605,7 @@ private static FileState analyzeFileState( } } 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 @@ private static FileState analyzeFileState( // 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() && diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java index 1be2ad090f421..af5793fa82bbe 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java @@ -205,6 +205,9 @@ public int getWriteHoldCount() { @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 int getWriteHoldCount() { namesystem = ns; this.editLog = ns.getEditLog(); ezManager = new EncryptionZoneManager(this, conf); + ecZoneManager = new ErasureCodingZoneManager(this); } FSNamesystem getFSNamesystem() { @@ -1221,6 +1225,25 @@ FileEncryptionInfo getFileEncryptionInfo(INode inode, int snapshotId, } } + 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) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java index 5fac43cc25b32..fa874d151d433 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java @@ -7510,6 +7510,46 @@ BatchedListEntries listEncryptionZones(long prevId) } } + /** + * 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 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 flag, boolean logRetryCache) throws IOException { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java index 51e24db657e92..032578a22602d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java @@ -420,7 +420,7 @@ public short getPreferredBlockReplication() { } max = maxInSnapshot > max ? maxInSnapshot : max; } - return isStriped()? + return isStriped() ? HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS : max; } @@ -1114,8 +1114,7 @@ boolean isBlockInLatestSnapshot(BlockInfoContiguous block) { */ @VisibleForTesting @Override - // TODO: move erasure coding policy to file XAttr public boolean isStriped() { - return getStoragePolicyID() == HdfsConstants.EC_STORAGE_POLICY_ID; + return getStripedBlocksFeature() != null; } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java index 0d416a6eb28dc..6e0333cc76bc9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java @@ -1820,6 +1820,22 @@ public BatchedEntries listEncryptionZones( 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 flag) throws IOException { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto index b44c556bbaa68..183aff89a41a4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto +++ b/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) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java index 13214f34484ae..ea69f976a6f16 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java @@ -119,9 +119,6 @@ public void testDefaultPolicies() { 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], " + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java new file mode 100644 index 0000000000000..49f08eefdf6f0 --- /dev/null +++ b/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); + } + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInitialEncoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInitialEncoding.java deleted file mode 100644 index a84f67b232b0c..0000000000000 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInitialEncoding.java +++ /dev/null @@ -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()); - } - -} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java index 215a4e4b8efb1..c3c823936c887 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java @@ -68,7 +68,7 @@ public void setup() throws IOException { .numDataNodes(GROUP_SIZE).build(); cluster.waitActive(); dfs = cluster.getFileSystem(); - dfs.setStoragePolicy(new Path("/"), HdfsConstants.EC_STORAGE_POLICY_NAME); + dfs.getClient().createErasureCodingZone("/"); } @After diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java index 051b74b4678d8..7dab7eb1280d5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java @@ -445,8 +445,7 @@ public void testAddNewStripedBlock() throws IOException{ //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 void testUpdateStripedBlocks() throws IOException{ //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); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java index 16f247add78b2..7fdf1af05589a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java @@ -33,18 +33,14 @@ 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.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 @@ private void testPersistHelper(Configuration conf) throws IOException { } } - 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 @@ private void testSaveAndLoadINodeFile(FSNamesystem fsn, Configuration conf, 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 @@ private void testSaveAndLoadINodeFile(FSNamesystem fsn, Configuration conf, * 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 void testSaveAndLoadInodeFile() throws IOException{ * 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 void testSupportBlockGroup() throws IOException { .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); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java index b9fd4fee16e72..d965ae721344c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java +++ b/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.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.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 static void createECFile(MiniDFSCluster cluster, Path file, Path dir, 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 {