Skip to content

Commit

Permalink
HDFS-7839. Erasure coding: implement facilities in NameNode to create…
Browse files Browse the repository at this point in the history
… and manage EC zones. Contributed by Zhe Zhang
  • Loading branch information
zhe-thoughts authored and Zhe Zhang committed May 26, 2015
1 parent 578019d commit 1af8c14
Show file tree
Hide file tree
Showing 20 changed files with 426 additions and 114 deletions.
Expand Up @@ -2994,6 +2994,21 @@ public RemoteIterator<EncryptionZone> 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<XAttrSetFlag> flag) throws IOException {
checkOpen();
Expand Down
Expand Up @@ -1362,6 +1362,14 @@ public EncryptionZone getEZForPath(String src)
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,
Expand Down
Expand Up @@ -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;
Expand Down Expand Up @@ -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 {
Expand Down
Expand Up @@ -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;
Expand Down Expand Up @@ -1406,6 +1408,20 @@ public BatchedEntries<EncryptionZone> 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<XAttrSetFlag> flag)
throws IOException {
Expand Down
Expand Up @@ -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";
}
@@ -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.");
}
}
}
Expand Up @@ -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);
Expand Down Expand Up @@ -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<INodeDirectory> snapshottableDirs = new ArrayList<>();
if (dstInode != null) { // Destination exists
Expand Down
Expand Up @@ -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;
Expand Down Expand Up @@ -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);
}
Expand Down Expand Up @@ -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();
}
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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() &&
Expand Down
Expand Up @@ -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.
Expand Down Expand Up @@ -296,6 +299,7 @@ public int getWriteHoldCount() {
namesystem = ns;
this.editLog = ns.getEditLog();
ezManager = new EncryptionZoneManager(this, conf);
ecZoneManager = new ErasureCodingZoneManager(this);
}

FSNamesystem getFSNamesystem() {
Expand Down Expand Up @@ -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) {
Expand Down
Expand Up @@ -7510,6 +7510,46 @@ BatchedListEntries<EncryptionZone> 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<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 {
Expand Down
Expand Up @@ -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;
}

Expand Down Expand Up @@ -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;
}
}

0 comments on commit 1af8c14

Please sign in to comment.