HDFS-8058. Erasure coding: use BlockInfo[] for both striped and contiguous blocks in INodeFile. Contributed by Zhe Zhang and Yi Liu.

This commit is contained in:
Zhe Zhang 2015-07-15 09:49:32 -07:00
parent f32d9a1758
commit 7e091de136
25 changed files with 273 additions and 494 deletions

View File

@ -356,3 +356,6 @@
HDFS-8619. Erasure Coding: revisit replica counting for striped blocks.
(Jing Zhao via yliu)
HDFS-8058. Erasure coding: use BlockInfo[] for both striped and contiguous
blocks in INodeFile. (Zhe Zhang and Yi Liu via zhz)

View File

@ -31,6 +31,7 @@ import org.apache.hadoop.util.LightWeightGSet;
*/
public abstract class BlockInfo extends Block
implements LightWeightGSet.LinkedElement {
public static final BlockInfo[] EMPTY_ARRAY = {};
private BlockCollection bc;
/** For implementing {@link LightWeightGSet.LinkedElement} interface */

View File

@ -26,7 +26,6 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
*/
@InterfaceAudience.Private
public class BlockInfoContiguous extends BlockInfo {
public static final BlockInfoContiguous[] EMPTY_ARRAY = {};
public BlockInfoContiguous(short size) {
super(size);

View File

@ -3037,13 +3037,13 @@ public class BlockManager {
/** Set replication for the blocks. */
public void setReplication(final short oldRepl, final short newRepl,
final String src, final BlockInfoContiguous... blocks) {
final String src, final BlockInfo... blocks) {
if (newRepl == oldRepl) {
return;
}
// update needReplication priority queues
for(BlockInfoContiguous b : blocks) {
for(BlockInfo b : blocks) {
updateNeededReplications(b, 0, newRepl-oldRepl);
}
@ -3051,7 +3051,7 @@ public class BlockManager {
// old replication > the new one; need to remove copies
LOG.info("Decreasing replication from " + oldRepl + " to " + newRepl
+ " for " + src);
for(BlockInfoContiguous b : blocks) {
for(BlockInfo b : blocks) {
processOverReplicatedBlock(b, newRepl, null, null);
}
} else { // replication factor is increased

View File

@ -25,14 +25,12 @@ import org.apache.hadoop.fs.XAttr;
import org.apache.hadoop.fs.XAttrSetFlag;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
import org.apache.hadoop.hdfs.util.EnumCounters;
@ -150,7 +148,7 @@ public class FSDirAttrOp {
}
final short[] blockRepls = new short[2]; // 0: old, 1: new
final BlockInfoContiguous[] blocks = unprotectedSetReplication(fsd, src,
final BlockInfo[] blocks = unprotectedSetReplication(fsd, src,
replication, blockRepls);
isFile = blocks != null;
if (isFile) {
@ -377,7 +375,7 @@ public class FSDirAttrOp {
}
}
static BlockInfoContiguous[] unprotectedSetReplication(
static BlockInfo[] unprotectedSetReplication(
FSDirectory fsd, String src, short replication, short[] blockRepls)
throws QuotaExceededException, UnresolvedLinkException,
SnapshotAccessControlException, UnsupportedActionException {
@ -417,7 +415,7 @@ public class FSDirAttrOp {
blockRepls[0] = oldBR;
blockRepls[1] = newBR;
}
return file.getContiguousBlocks();
return file.getBlocks();
}
static void unprotectedSetStoragePolicy(

View File

@ -234,7 +234,7 @@ class FSDirConcatOp {
int count = 0;
for (INodeFile nodeToRemove : srcList) {
if(nodeToRemove != null) {
nodeToRemove.setContiguousBlocks(null);
nodeToRemove.clearBlocks();
nodeToRemove.getParent().removeChild(nodeToRemove);
fsd.getINodeMap().remove(nodeToRemove);
count++;

View File

@ -44,7 +44,6 @@ 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.BlockInfoStripedUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
@ -481,25 +480,24 @@ class FSDirWriteFileOp {
long preferredBlockSize, boolean underConstruction, String clientName,
String clientMachine, byte storagePolicyId) {
final INodeFile newNode;
Preconditions.checkNotNull(existing);
assert fsd.hasWriteLock();
if (underConstruction) {
newNode = newINodeFile(id, permissions, modificationTime,
modificationTime, replication, preferredBlockSize, storagePolicyId);
newNode.toUnderConstruction(clientName, clientMachine);
} else {
newNode = newINodeFile(id, permissions, modificationTime, atime,
replication, preferredBlockSize, storagePolicyId);
}
newNode.setLocalName(localName);
try {
// check if the file is in an EC zone
final boolean isStriped = FSDirErasureCodingOp.isInErasureCodingZone(
fsd.getFSNamesystem(), existing);
if (underConstruction) {
newNode = newINodeFile(id, permissions, modificationTime,
modificationTime, replication, preferredBlockSize, storagePolicyId,
isStriped);
newNode.toUnderConstruction(clientName, clientMachine);
} else {
newNode = newINodeFile(id, permissions, modificationTime, atime,
replication, preferredBlockSize, storagePolicyId, isStriped);
}
newNode.setLocalName(localName);
INodesInPath iip = fsd.addINode(existing, newNode);
if (iip != null) {
// check if the file is in an EC zone
if (FSDirErasureCodingOp.isInErasureCodingZone(fsd.getFSNamesystem(),
iip)) {
newNode.addStripedBlocksFeature();
}
if (aclEntries != null) {
AclStorage.updateINodeAcl(newNode, aclEntries, CURRENT_STATE_ID);
}
@ -580,21 +578,18 @@ class FSDirWriteFileOp {
String clientName, String clientMachine)
throws IOException {
Preconditions.checkNotNull(existing);
long modTime = now();
INodeFile newNode = newINodeFile(fsd.allocateNewInodeId(), permissions,
modTime, modTime, replication, preferredBlockSize);
newNode.setLocalName(localName.getBytes(Charsets.UTF_8));
newNode.toUnderConstruction(clientName, clientMachine);
INodesInPath newiip;
fsd.writeLock();
try {
final boolean isStriped = FSDirErasureCodingOp.isInErasureCodingZone(
fsd.getFSNamesystem(), existing);
INodeFile newNode = newINodeFile(fsd.allocateNewInodeId(), permissions,
modTime, modTime, replication, preferredBlockSize, isStriped);
newNode.setLocalName(localName.getBytes(Charsets.UTF_8));
newNode.toUnderConstruction(clientName, clientMachine);
newiip = fsd.addINode(existing, newNode);
if (newiip != null
&& FSDirErasureCodingOp.isInErasureCodingZone(fsd.getFSNamesystem(),
newiip)) {
newNode.addStripedBlocksFeature();
}
} finally {
fsd.writeUnlock();
}
@ -788,16 +783,18 @@ class FSDirWriteFileOp {
private static INodeFile newINodeFile(
long id, PermissionStatus permissions, long mtime, long atime,
short replication, long preferredBlockSize, byte storagePolicyId) {
short replication, long preferredBlockSize, byte storagePolicyId,
boolean isStriped) {
return new INodeFile(id, null, permissions, mtime, atime,
BlockInfoContiguous.EMPTY_ARRAY, replication, preferredBlockSize,
storagePolicyId);
BlockInfo.EMPTY_ARRAY, replication, preferredBlockSize,
storagePolicyId, isStriped);
}
private static INodeFile newINodeFile(long id, PermissionStatus permissions,
long mtime, long atime, short replication, long preferredBlockSize) {
long mtime, long atime, short replication, long preferredBlockSize,
boolean isStriped) {
return newINodeFile(id, permissions, mtime, atime, replication,
preferredBlockSize, (byte)0);
preferredBlockSize, (byte)0, isStriped);
}
/**

View File

@ -55,9 +55,7 @@ import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
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.blockmanagement.BlockInfoContiguousUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
@ -76,7 +74,6 @@ import org.apache.hadoop.hdfs.util.ReadOnlyList;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.MD5Hash;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.erasurecode.ECSchema;
import org.apache.hadoop.util.StringUtils;
import com.google.common.annotations.VisibleForTesting;
@ -903,7 +900,7 @@ public class FSImageFormat {
final long preferredBlockSize = in.readLong();
return new INodeFileAttributes.SnapshotCopy(name, permissions, null, modificationTime,
accessTime, replication, preferredBlockSize, (byte) 0, null);
accessTime, replication, preferredBlockSize, (byte) 0, null, false);
}
public INodeDirectoryAttributes loadINodeDirectoryAttributes(DataInput in)

View File

@ -55,7 +55,6 @@ import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FilesUnderConstructio
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeDirectorySection;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.AclFeatureProto;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.StripedBlocksFeature;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.XAttrCompactProto;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.XAttrFeatureProto;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.QuotaByStorageTypeEntryProto;
@ -326,13 +325,22 @@ public final class FSImageFormatPBINode {
INodeSection.INodeFile f = n.getFile();
List<BlockProto> bp = f.getBlocksList();
short replication = (short) f.getReplication();
boolean isStriped = f.getIsStriped();
LoaderContext state = parent.getLoaderContext();
ECSchema schema = ErasureCodingSchemaManager.getSystemDefaultSchema();
BlockInfoContiguous[] blocks = null;
if (!f.hasStripedBlocks()) {
blocks = new BlockInfoContiguous[bp.size()];
for (int i = 0, e = bp.size(); i < e; ++i) {
blocks[i] = new BlockInfoContiguous(PBHelper.convert(bp.get(i)), replication);
if (isStriped) {
Preconditions.checkState(f.hasStripingCellSize());
}
BlockInfo[] blocks = new BlockInfo[bp.size()];
for (int i = 0; i < bp.size(); ++i) {
BlockProto b = bp.get(i);
if (isStriped) {
blocks[i] = new BlockInfoStriped(PBHelper.convert(b), schema,
(int)f.getStripingCellSize());
} else {
blocks[i] = new BlockInfoContiguous(PBHelper.convert(b),
replication);
}
}
@ -342,46 +350,31 @@ public final class FSImageFormatPBINode {
final INodeFile file = new INodeFile(n.getId(),
n.getName().toByteArray(), permissions, f.getModificationTime(),
f.getAccessTime(), blocks, replication, f.getPreferredBlockSize(),
(byte)f.getStoragePolicyID());
(byte)f.getStoragePolicyID(), isStriped);
if (f.hasAcl()) {
int[] entries = AclEntryStatusFormat.toInt(loadAclEntries(
f.getAcl(), state.getStringTable()));
file.addAclFeature(new AclFeature(entries));
}
if (f.hasXAttrs()) {
file.addXAttrFeature(new XAttrFeature(
loadXAttrs(f.getXAttrs(), state.getStringTable())));
}
FileWithStripedBlocksFeature stripeFeature = null;
if (f.hasStripedBlocks()) {
// TODO: HDFS-7859
ECSchema schema = ErasureCodingSchemaManager.getSystemDefaultSchema();
stripeFeature = file.addStripedBlocksFeature();
if (bp.size() > 0) {
// if a striped file has block, the cellSize must exist in proto
final int cellSize = f.getStripedBlocks().getCellSize();
for (BlockProto b : bp) {
stripeFeature.addBlock(new BlockInfoStriped(PBHelper.convert(b),
schema, cellSize));
}
}
}
// under-construction information
if (f.hasFileUC()) {
INodeSection.FileUnderConstructionFeature uc = f.getFileUC();
file.toUnderConstruction(uc.getClientName(), uc.getClientMachine());
BlockInfo lastBlk = file.getLastBlock();
if (lastBlk != null) {
if (blocks.length > 0) {
BlockInfo lastBlk = file.getLastBlock();
// replace the last block of file
final BlockInfo ucBlk;
if (stripeFeature != null) {
if (isStriped) {
BlockInfoStriped striped = (BlockInfoStriped) lastBlk;
ucBlk = new BlockInfoStripedUnderConstruction(striped,
striped.getSchema(), striped.getCellSize());
schema, (int)f.getStripingCellSize());
} else {
ucBlk = new BlockInfoContiguousUnderConstruction(lastBlk,
replication);
@ -500,7 +493,8 @@ public final class FSImageFormatPBINode {
.setPermission(buildPermissionStatus(file, state.getStringMap()))
.setPreferredBlockSize(file.getPreferredBlockSize())
.setReplication(file.getFileReplication())
.setStoragePolicyID(file.getLocalStoragePolicyID());
.setStoragePolicyID(file.getLocalStoragePolicyID())
.setIsStriped(file.isStriped());
AclFeature f = file.getAclFeature();
if (f != null) {
@ -654,28 +648,22 @@ public final class FSImageFormatPBINode {
private void save(OutputStream out, INodeFile n) throws IOException {
INodeSection.INodeFile.Builder b = buildINodeFile(n,
parent.getSaverContext());
BlockInfo[] blocks = n.getBlocks();
BlockInfoContiguous[] cBlks = n.getContiguousBlocks();
if (cBlks != null) {
for (Block block : cBlks) {
if (blocks != null) {
for (Block block : n.getBlocks()) {
b.addBlocks(PBHelper.convert(block));
}
}
FileWithStripedBlocksFeature sb = n.getStripedBlocksFeature();
if (sb != null) {
StripedBlocksFeature.Builder builder =
StripedBlocksFeature.newBuilder();
BlockInfoStriped[] sblocks = sb.getBlocks();
if (sblocks != null && sblocks.length > 0) {
final int cellSize = sblocks[0].getCellSize();
for (BlockInfoStriped sblk : sblocks) {
assert cellSize == sblk.getCellSize();
b.addBlocks(PBHelper.convert(sblk));
}
builder.setCellSize(cellSize);
if (n.isStriped()) {
if (blocks != null && blocks.length > 0) {
BlockInfo firstBlock = blocks[0];
Preconditions.checkState(firstBlock.isStriped());
b.setStripingCellSize(((BlockInfoStriped)firstBlock).getCellSize());
} else {
b.setStripingCellSize(HdfsConstants.BLOCK_STRIPED_CELL_SIZE);
}
b.setStripedBlocks(builder.build());
}
FileUnderConstructionFeature uc = n.getFileUnderConstructionFeature();

View File

@ -1,116 +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.namenode;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
/**
* Feature for file with striped blocks
*/
class FileWithStripedBlocksFeature implements INode.Feature {
private BlockInfoStriped[] blocks;
FileWithStripedBlocksFeature() {
blocks = new BlockInfoStriped[0];
}
FileWithStripedBlocksFeature(BlockInfoStriped[] blocks) {
Preconditions.checkArgument(blocks != null);
this.blocks = blocks;
}
BlockInfoStriped[] getBlocks() {
return this.blocks;
}
void setBlock(int index, BlockInfoStriped blk) {
blocks[index] = blk;
}
BlockInfoStriped getLastBlock() {
return blocks == null || blocks.length == 0 ?
null : blocks[blocks.length - 1];
}
int numBlocks() {
return blocks == null ? 0 : blocks.length;
}
void updateBlockCollection(INodeFile file) {
if (blocks != null) {
for (BlockInfoStriped blk : blocks) {
blk.setBlockCollection(file);
}
}
}
private void setBlocks(BlockInfoStriped[] blocks) {
this.blocks = blocks;
}
void addBlock(BlockInfoStriped newBlock) {
if (this.blocks == null) {
this.setBlocks(new BlockInfoStriped[]{newBlock});
} else {
int size = this.blocks.length;
BlockInfoStriped[] newlist = new BlockInfoStriped[size + 1];
System.arraycopy(this.blocks, 0, newlist, 0, size);
newlist[size] = newBlock;
this.setBlocks(newlist);
}
}
BlockInfoStripedUnderConstruction removeLastBlock(
Block oldblock) {
if (blocks == null || blocks.length == 0) {
return null;
}
int newSize = blocks.length - 1;
if (!blocks[newSize].equals(oldblock)) {
return null;
}
BlockInfoStripedUnderConstruction uc =
(BlockInfoStripedUnderConstruction) blocks[newSize];
//copy to a new list
BlockInfoStriped[] newlist = new BlockInfoStriped[newSize];
System.arraycopy(blocks, 0, newlist, 0, newSize);
setBlocks(newlist);
return uc;
}
void truncateStripedBlocks(int n) {
final BlockInfoStriped[] newBlocks;
if (n == 0) {
newBlocks = new BlockInfoStriped[0];
} else {
newBlocks = new BlockInfoStriped[n];
System.arraycopy(getBlocks(), 0, newBlocks, 0, n);
}
// set new blocks
setBlocks(newBlocks);
}
void clear() {
this.blocks = null;
}
}

View File

@ -40,9 +40,7 @@ import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
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.BlockInfoStriped;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
@ -84,12 +82,14 @@ public class INodeFile extends INodeWithAdditionalFields
/**
* Bit format:
* [4-bit storagePolicyID][12-bit replication][48-bit preferredBlockSize]
* [4-bit storagePolicyID][1-bit isStriped]
* [11-bit replication][48-bit preferredBlockSize]
*/
static enum HeaderFormat {
PREFERRED_BLOCK_SIZE(null, 48, 1),
REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 12, 0),
STORAGE_POLICY_ID(REPLICATION.BITS, BlockStoragePolicySuite.ID_BIT_LENGTH,
REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 11, 0),
IS_STRIPED(REPLICATION.BITS, 1, 0),
STORAGE_POLICY_ID(IS_STRIPED.BITS, BlockStoragePolicySuite.ID_BIT_LENGTH,
0);
private final LongBitFormat BITS;
@ -110,14 +110,27 @@ public class INodeFile extends INodeWithAdditionalFields
return (byte)STORAGE_POLICY_ID.BITS.retrieve(header);
}
static boolean isStriped(long header) {
long isStriped = IS_STRIPED.BITS.retrieve(header);
Preconditions.checkState(isStriped == 0 || isStriped == 1);
return isStriped == 1;
}
static long toLong(long preferredBlockSize, short replication,
byte storagePolicyID) {
boolean isStriped, byte storagePolicyID) {
long h = 0;
if (preferredBlockSize == 0) {
preferredBlockSize = PREFERRED_BLOCK_SIZE.BITS.getMin();
}
h = PREFERRED_BLOCK_SIZE.BITS.combine(preferredBlockSize, h);
h = REPLICATION.BITS.combine(replication, h);
// Replication factor for striped files is zero
if (isStriped) {
h = REPLICATION.BITS.combine(0L, h);
h = IS_STRIPED.BITS.combine(1L, h);
} else {
h = REPLICATION.BITS.combine(replication, h);
h = IS_STRIPED.BITS.combine(0L, h);
}
h = STORAGE_POLICY_ID.BITS.combine(storagePolicyID, h);
return h;
}
@ -126,21 +139,27 @@ public class INodeFile extends INodeWithAdditionalFields
private long header = 0L;
private BlockInfoContiguous[] blocks;
private BlockInfo[] blocks;
INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime,
long atime, BlockInfoContiguous[] blklist, short replication,
long atime, BlockInfo[] blklist, short replication,
long preferredBlockSize) {
this(id, name, permissions, mtime, atime, blklist, replication,
preferredBlockSize, (byte) 0);
preferredBlockSize, (byte) 0, false);
}
INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime,
long atime, BlockInfoContiguous[] blklist, short replication,
long preferredBlockSize, byte storagePolicyID) {
long atime, BlockInfo[] blklist, short replication,
long preferredBlockSize, byte storagePolicyID, boolean isStriped) {
super(id, name, permissions, mtime, atime);
header = HeaderFormat.toLong(preferredBlockSize, replication, storagePolicyID);
this.blocks = blklist;
header = HeaderFormat.toLong(preferredBlockSize, replication, isStriped,
storagePolicyID);
if (blklist != null && blklist.length > 0) {
for (BlockInfo b : blklist) {
Preconditions.checkArgument(b.isStriped() == isStriped);
}
}
setBlocks(blklist);
}
public INodeFile(INodeFile that) {
@ -177,27 +196,6 @@ public class INodeFile extends INodeWithAdditionalFields
&& getXAttrFeature() == other.getXAttrFeature();
}
/* Start of StripedBlock Feature */
public final FileWithStripedBlocksFeature getStripedBlocksFeature() {
return getFeature(FileWithStripedBlocksFeature.class);
}
public FileWithStripedBlocksFeature addStripedBlocksFeature() {
assert blocks == null || blocks.length == 0:
"The file contains contiguous blocks";
assert !isStriped();
this.setFileReplication((short) 0);
FileWithStripedBlocksFeature sb = new FileWithStripedBlocksFeature();
addFeature(sb);
return sb;
}
/** Used to make sure there is no contiguous block related info */
private boolean hasNoContiguousBlock() {
return (blocks == null || blocks.length == 0) && getFileReplication() == 0;
}
/* Start of Under-Construction Feature */
/**
@ -232,7 +230,7 @@ public class INodeFile extends INodeWithAdditionalFields
"file is no longer under construction");
FileUnderConstructionFeature uc = getFileUnderConstructionFeature();
if (uc != null) {
assertAllBlocksComplete(getBlocks());
assertAllBlocksComplete();
removeFeature(uc);
this.setModificationTime(mtime);
}
@ -240,32 +238,21 @@ public class INodeFile extends INodeWithAdditionalFields
}
/** Assert all blocks are complete. */
private void assertAllBlocksComplete(BlockInfo[] blks) {
if (blks == null) {
private void assertAllBlocksComplete() {
if (blocks == null) {
return;
}
for (int i = 0; i < blks.length; i++) {
Preconditions.checkState(blks[i].isComplete(), "Failed to finalize"
for (int i = 0; i < blocks.length; i++) {
Preconditions.checkState(blocks[i].isComplete(), "Failed to finalize"
+ " %s %s since blocks[%s] is non-complete, where blocks=%s.",
getClass().getSimpleName(), this, i, Arrays.asList(blks));
getClass().getSimpleName(), this, i, Arrays.asList(blocks));
}
}
/**
* Instead of adding a new block, this function is usually used while loading
* fsimage or converting the last block to UC/Complete.
*/
@Override // BlockCollection
public void setBlock(int index, BlockInfo blk) {
FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
if (sb == null) {
assert !blk.isStriped();
this.blocks[index] = (BlockInfoContiguous) blk;
} else {
assert blk.isStriped();
assert hasNoContiguousBlock();
sb.setBlock(index, (BlockInfoStriped) blk);
}
Preconditions.checkArgument(blk.isStriped() == this.isStriped());
this.blocks[index] = blk;
}
@Override // BlockCollection, the file should be under construction
@ -278,15 +265,13 @@ public class INodeFile extends INodeWithAdditionalFields
}
final BlockInfo ucBlock;
FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
if (sb == null) {
assert !lastBlock.isStriped();
ucBlock = ((BlockInfoContiguous) lastBlock)
if (isStriped()) {
Preconditions.checkState(lastBlock.isStriped());
ucBlock = ((BlockInfoStriped) lastBlock)
.convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations);
} else {
assert hasNoContiguousBlock();
assert lastBlock.isStriped();
ucBlock = ((BlockInfoStriped) lastBlock)
Preconditions.checkState(!lastBlock.isStriped());
ucBlock = ((BlockInfoContiguous) lastBlock)
.convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations);
}
setBlock(numBlocks() - 1, ucBlock);
@ -299,27 +284,21 @@ public class INodeFile extends INodeWithAdditionalFields
BlockInfoUnderConstruction removeLastBlock(Block oldblock) {
Preconditions.checkState(isUnderConstruction(),
"file is no longer under construction");
FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
if (sb == null) {
if (blocks == null || blocks.length == 0) {
return null;
}
int size_1 = blocks.length - 1;
if (!blocks[size_1].equals(oldblock)) {
return null;
}
BlockInfoContiguousUnderConstruction uc =
(BlockInfoContiguousUnderConstruction)blocks[size_1];
//copy to a new list
BlockInfoContiguous[] newlist = new BlockInfoContiguous[size_1];
System.arraycopy(blocks, 0, newlist, 0, size_1);
setContiguousBlocks(newlist);
return uc;
} else {
assert hasNoContiguousBlock();
return sb.removeLastBlock(oldblock);
if (blocks == null || blocks.length == 0) {
return null;
}
int size_1 = blocks.length - 1;
if (!blocks[size_1].equals(oldblock)) {
return null;
}
BlockInfoUnderConstruction uc =
(BlockInfoUnderConstruction)blocks[size_1];
//copy to a new list
BlockInfo[] newlist = new BlockInfo[size_1];
System.arraycopy(blocks, 0, newlist, 0, size_1);
setBlocks(newlist);
return uc;
}
/* End of Under-Construction Feature */
@ -401,7 +380,7 @@ public class INodeFile extends INodeWithAdditionalFields
/** The same as getFileReplication(null). */
@Override // INodeFileAttributes
// TODO striped
// TODO properly handle striped files
public final short getFileReplication() {
return getFileReplication(CURRENT_STATE_ID);
}
@ -466,6 +445,16 @@ public class INodeFile extends INodeWithAdditionalFields
setStoragePolicyID(storagePolicyId);
}
/**
* @return true if the file is in the striping layout.
*/
@VisibleForTesting
@Override
public boolean isStriped() {
return HeaderFormat.isStriped(header);
}
@Override // INodeFileAttributes
public long getHeaderLong() {
return header;
@ -474,17 +463,6 @@ public class INodeFile extends INodeWithAdditionalFields
/** @return the blocks of the file. */
@Override // BlockCollection
public BlockInfo[] getBlocks() {
FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
if (sb != null) {
assert hasNoContiguousBlock();
return sb.getBlocks();
} else {
return this.blocks;
}
}
/** Used by snapshot diff */
public BlockInfoContiguous[] getContiguousBlocks() {
return this.blocks;
}
@ -507,16 +485,11 @@ public class INodeFile extends INodeWithAdditionalFields
}
/** Used during concat to update the BlockCollection for each block */
private void updateBlockCollection() {
if (blocks != null && blocks.length > 0) {
for(BlockInfoContiguous b : blocks) {
void updateBlockCollection() {
if (blocks != null) {
for(BlockInfo b : blocks) {
b.setBlockCollection(this);
}
} else {
FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
if (sb != null) {
sb.updateBlockCollection(this);
}
}
}
@ -527,11 +500,12 @@ public class INodeFile extends INodeWithAdditionalFields
int size = this.blocks.length;
int totalAddedBlocks = 0;
for(INodeFile f : inodes) {
Preconditions.checkState(f.isStriped() == this.isStriped());
totalAddedBlocks += f.blocks.length;
}
BlockInfoContiguous[] newlist =
new BlockInfoContiguous[size + totalAddedBlocks];
BlockInfo[] newlist =
new BlockInfo[size + totalAddedBlocks];
System.arraycopy(this.blocks, 0, newlist, 0, size);
for(INodeFile in: inodes) {
@ -539,43 +513,35 @@ public class INodeFile extends INodeWithAdditionalFields
size += in.blocks.length;
}
setContiguousBlocks(newlist);
setBlocks(newlist);
updateBlockCollection();
}
/**
* add a contiguous block to the block list
* add a block to the block list
*/
private void addContiguousBlock(BlockInfoContiguous newblock) {
void addBlock(BlockInfo newblock) {
Preconditions.checkArgument(newblock.isStriped() == this.isStriped());
if (this.blocks == null) {
this.setContiguousBlocks(new BlockInfoContiguous[]{newblock});
this.setBlocks(new BlockInfo[]{newblock});
} else {
int size = this.blocks.length;
BlockInfoContiguous[] newlist = new BlockInfoContiguous[size + 1];
BlockInfo[] newlist = new BlockInfo[size + 1];
System.arraycopy(this.blocks, 0, newlist, 0, size);
newlist[size] = newblock;
this.setContiguousBlocks(newlist);
}
}
/** add a striped or contiguous block */
void addBlock(BlockInfo newblock) {
FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
if (sb == null) {
assert !newblock.isStriped();
addContiguousBlock((BlockInfoContiguous) newblock);
} else {
assert newblock.isStriped();
assert hasNoContiguousBlock();
sb.addBlock((BlockInfoStriped) newblock);
this.setBlocks(newlist);
}
}
/** Set the blocks. */
public void setContiguousBlocks(BlockInfoContiguous[] blocks) {
private void setBlocks(BlockInfo[] blocks) {
this.blocks = blocks;
}
public void clearBlocks() {
setBlocks(null);
}
@Override
public void cleanSubtree(ReclaimContext reclaimContext,
final int snapshot, int priorSnapshotId) {
@ -623,19 +589,13 @@ public class INodeFile extends INodeWithAdditionalFields
}
public void clearFile(ReclaimContext reclaimContext) {
BlockInfo[] blks = getBlocks();
if (blks != null && reclaimContext.collectedBlocks != null) {
for (BlockInfo blk : blks) {
if (blocks != null && reclaimContext.collectedBlocks != null) {
for (BlockInfo blk : blocks) {
reclaimContext.collectedBlocks.addDeleteBlock(blk);
blk.setBlockCollection(null);
}
}
setContiguousBlocks(null);
FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
if (sb != null) {
sb.clear();
}
clearBlocks();
if (getAclFeature() != null) {
AclStorage.removeAclFeature(getAclFeature());
}
@ -677,7 +637,7 @@ public class INodeFile extends INodeWithAdditionalFields
if (isStriped()) {
return computeQuotaUsageWithStriped(bsp, counts);
}
if (last < lastSnapshotId) {
ssDeltaNoReplication = computeFileSize(true, false);
replication = getFileReplication();
@ -790,33 +750,27 @@ public class INodeFile extends INodeWithAdditionalFields
* @return file size
*/
public final long computeFileSize(boolean includesLastUcBlock,
boolean usePreferredBlockSize4LastUcBlock) {
BlockInfo[] blockInfos = getBlocks();
// In case of contiguous blocks
if (blockInfos == null || blockInfos.length == 0) {
boolean usePreferredBlockSize4LastUcBlock) {
if (blocks == null || blocks.length == 0) {
return 0;
}
final int last = blockInfos.length - 1;
final int last = blocks.length - 1;
//check if the last block is BlockInfoUnderConstruction
long size = blockInfos[last].getNumBytes();
if (blockInfos[last] instanceof BlockInfoContiguousUnderConstruction) {
if (!includesLastUcBlock) {
size = 0;
} else if (usePreferredBlockSize4LastUcBlock) {
size = getPreferredBlockSize();
}
} else if (blockInfos[last] instanceof BlockInfoStripedUnderConstruction) {
if (!includesLastUcBlock) {
size = 0;
} else if (usePreferredBlockSize4LastUcBlock) {
BlockInfoStripedUnderConstruction blockInfoStripedUC
= (BlockInfoStripedUnderConstruction) blockInfos[last];
size = getPreferredBlockSize() * blockInfoStripedUC.getDataBlockNum();
}
BlockInfo lastBlk = blocks[last];
long size = lastBlk.getNumBytes();
if (lastBlk instanceof BlockInfoUnderConstruction) {
if (!includesLastUcBlock) {
size = 0;
} else if (usePreferredBlockSize4LastUcBlock) {
size = isStriped()?
getPreferredBlockSize() *
((BlockInfoStriped)lastBlk).getDataBlockNum() :
getPreferredBlockSize();
}
}
//sum other blocks
for (int i = 0; i < last; i++) {
size += blockInfos[i].getNumBytes();
size += blocks[i].getNumBytes();
}
return size;
}
@ -828,41 +782,30 @@ public class INodeFile extends INodeWithAdditionalFields
*/
public final QuotaCounts storagespaceConsumed(BlockStoragePolicy bsp) {
if (isStriped()) {
return storagespaceConsumedWithStriped();
return storagespaceConsumedStriped();
} else {
return storagespaceConsumedWithReplication(bsp);
return storagespaceConsumedContiguous(bsp);
}
}
// TODO: support EC with heterogeneous storage
public final QuotaCounts storagespaceConsumedWithStriped() {
public final QuotaCounts storagespaceConsumedStriped() {
QuotaCounts counts = new QuotaCounts.Builder().build();
BlockInfo[] blockInfos = getBlocks();
if (blockInfos == null || blockInfos.length == 0) {
if (blocks == null || blocks.length == 0) {
return counts;
}
long size;
final int last = blockInfos.length - 1;
if (blockInfos[last] instanceof BlockInfoStripedUnderConstruction) {
BlockInfoStripedUnderConstruction blockInfoStripedUC
=(BlockInfoStripedUnderConstruction)blockInfos[last];
size = getPreferredBlockSize() * blockInfoStripedUC.getTotalBlockNum();
} else {
// In case of last block is complete
BlockInfoStriped blockInfoStriped = (BlockInfoStriped)blockInfos[last];
size = blockInfoStriped.spaceConsumed();
for (BlockInfo b : blocks) {
Preconditions.checkState(b.isStriped());
long blockSize = b.isComplete() ?
((BlockInfoStriped)b).spaceConsumed() : getPreferredBlockSize() *
((BlockInfoStriped)b).getTotalBlockNum();
counts.addStorageSpace(blockSize);
}
for (int i = 0; i < last; i++) {
BlockInfoStriped blockInfoStriped = (BlockInfoStriped)blockInfos[i];
size += blockInfoStriped.spaceConsumed();
}
counts.addStorageSpace(size);
return counts;
}
public final QuotaCounts storagespaceConsumedWithReplication(
public final QuotaCounts storagespaceConsumedContiguous(
BlockStoragePolicy bsp) {
QuotaCounts counts = new QuotaCounts.Builder().build();
final Iterable<BlockInfo> blocks;
@ -874,7 +817,7 @@ public class INodeFile extends INodeWithAdditionalFields
Set<BlockInfo> allBlocks = new HashSet<>(Arrays.asList(getBlocks()));
List<FileDiff> diffs = sf.getDiffs().asList();
for(FileDiff diff : diffs) {
BlockInfoContiguous[] diffBlocks = diff.getBlocks();
BlockInfo[] diffBlocks = diff.getBlocks();
if (diffBlocks != null) {
allBlocks.addAll(Arrays.asList(diffBlocks));
}
@ -903,32 +846,20 @@ public class INodeFile extends INodeWithAdditionalFields
* Return the penultimate allocated block for this file.
*/
BlockInfo getPenultimateBlock() {
BlockInfo[] blks = getBlocks();
return (blks == null || blks.length <= 1) ?
null : blks[blks.length - 2];
if (blocks == null || blocks.length <= 1) {
return null;
}
return blocks[blocks.length - 2];
}
@Override
public BlockInfo getLastBlock() {
FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
if (sb == null) {
return blocks == null || blocks.length == 0 ?
null : blocks[blocks.length - 1];
} else {
assert hasNoContiguousBlock();
return sb.getLastBlock();
}
return blocks == null || blocks.length == 0? null: blocks[blocks.length-1];
}
@Override
public int numBlocks() {
FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
if (sb == null) {
return blocks == null ? 0 : blocks.length;
} else {
assert hasNoContiguousBlock();
return sb.numBlocks();
}
return blocks == null ? 0 : blocks.length;
}
@VisibleForTesting
@ -939,8 +870,7 @@ public class INodeFile extends INodeWithAdditionalFields
out.print(", fileSize=" + computeFileSize(snapshotId));
// only compare the first block
out.print(", blocks=");
BlockInfo[] blks = getBlocks();
out.print(blks == null || blks.length == 0? null: blks[0]);
out.print(blocks == null || blocks.length == 0? null: blocks[0]);
out.println();
}
@ -1033,24 +963,15 @@ public class INodeFile extends INodeWithAdditionalFields
}
void truncateBlocksTo(int n) {
FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
if (sb == null) {
truncateContiguousBlocks(n);
} else {
sb.truncateStripedBlocks(n);
}
}
private void truncateContiguousBlocks(int n) {
final BlockInfoContiguous[] newBlocks;
final BlockInfo[] newBlocks;
if (n == 0) {
newBlocks = BlockInfoContiguous.EMPTY_ARRAY;
newBlocks = BlockInfo.EMPTY_ARRAY;
} else {
newBlocks = new BlockInfoContiguous[n];
System.arraycopy(blocks, 0, newBlocks, 0, n);
newBlocks = new BlockInfo[n];
System.arraycopy(getBlocks(), 0, newBlocks, 0, n);
}
// set new blocks
setContiguousBlocks(newBlocks);
setBlocks(newBlocks);
}
/**
@ -1059,10 +980,11 @@ public class INodeFile extends INodeWithAdditionalFields
* snapshots. Since we do not support truncation with striped blocks,
* we only need to handle contiguous blocks here.
*/
public void collectBlocksBeyondSnapshot(BlockInfoContiguous[] snapshotBlocks,
public void collectBlocksBeyondSnapshot(BlockInfo[] snapshotBlocks,
BlocksMapUpdateInfo collectedBlocks) {
BlockInfoContiguous[] oldBlocks = this.blocks;
if (snapshotBlocks == null || oldBlocks == null)
Preconditions.checkState(!isStriped());
BlockInfo[] oldBlocks = getBlocks();
if(snapshotBlocks == null || oldBlocks == null)
return;
// Skip blocks in common between the file and the snapshot
int n = 0;
@ -1070,7 +992,7 @@ public class INodeFile extends INodeWithAdditionalFields
oldBlocks[n] == snapshotBlocks[n]) {
n++;
}
truncateContiguousBlocks(n);
truncateBlocksTo(n);
// Collect the remaining blocks of the file
while(n < oldBlocks.length) {
collectedBlocks.addDeleteBlock(oldBlocks[n++]);
@ -1085,7 +1007,7 @@ public class INodeFile extends INodeWithAdditionalFields
FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
if(sf == null)
return;
BlockInfoContiguous[] snapshotBlocks =
BlockInfo[] snapshotBlocks =
getDiffs().findEarlierSnapshotBlocks(snapshotId);
if(snapshotBlocks == null)
return;
@ -1099,23 +1021,14 @@ public class INodeFile extends INodeWithAdditionalFields
/**
* @return true if the block is contained in a snapshot or false otherwise.
*/
boolean isBlockInLatestSnapshot(BlockInfoContiguous block) {
boolean isBlockInLatestSnapshot(BlockInfo block) {
FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature();
if (sf == null || sf.getDiffs() == null) {
return false;
}
BlockInfoContiguous[] snapshotBlocks = getDiffs()
BlockInfo[] snapshotBlocks = getDiffs()
.findEarlierSnapshotBlocks(getDiffs().getLastSnapshotId());
return snapshotBlocks != null &&
Arrays.asList(snapshotBlocks).contains(block);
}
/**
* @return true if the file is in the striping layout.
*/
@VisibleForTesting
@Override
public boolean isStriped() {
return getStripedBlocksFeature() != null;
}
}

View File

@ -29,6 +29,9 @@ public interface INodeFileAttributes extends INodeAttributes {
/** @return the file replication. */
public short getFileReplication();
/** @return whether the file is striped (instead of contiguous) */
public boolean isStriped();
/** @return preferred block size in bytes */
public long getPreferredBlockSize();
@ -47,10 +50,10 @@ public interface INodeFileAttributes extends INodeAttributes {
public SnapshotCopy(byte[] name, PermissionStatus permissions,
AclFeature aclFeature, long modificationTime, long accessTime,
short replication, long preferredBlockSize,
byte storagePolicyID, XAttrFeature xAttrsFeature) {
byte storagePolicyID, XAttrFeature xAttrsFeature, boolean isStriped) {
super(name, permissions, aclFeature, modificationTime, accessTime,
xAttrsFeature);
header = HeaderFormat.toLong(preferredBlockSize, replication,
header = HeaderFormat.toLong(preferredBlockSize, replication, isStriped,
storagePolicyID);
}
@ -69,6 +72,11 @@ public interface INodeFileAttributes extends INodeAttributes {
return HeaderFormat.getReplication(header);
}
@Override
public boolean isStriped() {
return HeaderFormat.isStriped(header);
}
@Override
public long getPreferredBlockSize() {
return HeaderFormat.getPreferredBlockSize(header);

View File

@ -233,7 +233,8 @@ public class FSImageFormatPBSnapshot {
.toByteArray(), permission, acl, fileInPb.getModificationTime(),
fileInPb.getAccessTime(), (short) fileInPb.getReplication(),
fileInPb.getPreferredBlockSize(),
(byte)fileInPb.getStoragePolicyID(), xAttrs);
(byte)fileInPb.getStoragePolicyID(), xAttrs,
fileInPb.getIsStriped());
}
FileDiff diff = new FileDiff(pbf.getSnapshotId(), copy, null,

View File

@ -21,6 +21,7 @@ import java.io.DataOutput;
import java.io.IOException;
import java.util.Arrays;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
import org.apache.hadoop.hdfs.server.namenode.INode;
@ -38,7 +39,7 @@ public class FileDiff extends
/** The file size at snapshot creation time. */
private final long fileSize;
/** A copy of the INodeFile block list. Used in truncate. */
private BlockInfoContiguous[] blocks;
private BlockInfo[] blocks;
FileDiff(int snapshotId, INodeFile file) {
super(snapshotId, null, null);
@ -64,7 +65,7 @@ public class FileDiff extends
* up to the current {@link #fileSize}.
* Should be done only once.
*/
public void setBlocks(BlockInfoContiguous[] blocks) {
public void setBlocks(BlockInfo[] blocks) {
if(this.blocks != null)
return;
int numBlocks = 0;
@ -73,7 +74,7 @@ public class FileDiff extends
this.blocks = Arrays.copyOf(blocks, numBlocks);
}
public BlockInfoContiguous[] getBlocks() {
public BlockInfo[] getBlocks() {
return blocks;
}
@ -118,7 +119,7 @@ public class FileDiff extends
if (blocks == null || collectedBlocks == null) {
return;
}
for (BlockInfoContiguous blk : blocks) {
for (BlockInfo blk : blocks) {
collectedBlocks.addDeleteBlock(blk);
}
blocks = null;

View File

@ -56,20 +56,20 @@ public class FileDiffList extends
final FileDiff diff =
super.saveSelf2Snapshot(latestSnapshotId, iNodeFile, snapshotCopy);
if (withBlocks) { // Store blocks if this is the first update
BlockInfoContiguous[] blks = iNodeFile.getContiguousBlocks();
BlockInfo[] blks = iNodeFile.getBlocks();
assert blks != null;
diff.setBlocks(blks);
}
}
public BlockInfoContiguous[] findEarlierSnapshotBlocks(int snapshotId) {
public BlockInfo[] findEarlierSnapshotBlocks(int snapshotId) {
assert snapshotId != Snapshot.NO_SNAPSHOT_ID : "Wrong snapshot id";
if (snapshotId == Snapshot.CURRENT_STATE_ID) {
return null;
}
List<FileDiff> diffs = this.asList();
int i = Collections.binarySearch(diffs, snapshotId);
BlockInfoContiguous[] blocks = null;
BlockInfo[] blocks = null;
for(i = i >= 0 ? i : -i-2; i >= 0; i--) {
blocks = diffs.get(i).getBlocks();
if(blocks != null) {
@ -79,14 +79,14 @@ public class FileDiffList extends
return blocks;
}
public BlockInfoContiguous[] findLaterSnapshotBlocks(int snapshotId) {
public BlockInfo[] findLaterSnapshotBlocks(int snapshotId) {
assert snapshotId != Snapshot.NO_SNAPSHOT_ID : "Wrong snapshot id";
if (snapshotId == Snapshot.CURRENT_STATE_ID) {
return null;
}
List<FileDiff> diffs = this.asList();
int i = Collections.binarySearch(diffs, snapshotId);
BlockInfoContiguous[] blocks = null;
BlockInfo[] blocks = null;
for (i = i >= 0 ? i+1 : -i-1; i < diffs.size(); i++) {
blocks = diffs.get(i).getBlocks();
if (blocks != null) {
@ -103,7 +103,7 @@ public class FileDiffList extends
*/
void combineAndCollectSnapshotBlocks(
INode.ReclaimContext reclaimContext, INodeFile file, FileDiff removed) {
BlockInfoContiguous[] removedBlocks = removed.getBlocks();
BlockInfo[] removedBlocks = removed.getBlocks();
if (removedBlocks == null) {
FileWithSnapshotFeature sf = file.getFileWithSnapshotFeature();
assert sf != null : "FileWithSnapshotFeature is null";
@ -117,11 +117,11 @@ public class FileDiffList extends
if (earlierDiff != null) {
earlierDiff.setBlocks(removedBlocks);
}
BlockInfoContiguous[] earlierBlocks =
BlockInfo[] earlierBlocks =
(earlierDiff == null ? new BlockInfoContiguous[]{} : earlierDiff.getBlocks());
// Find later snapshot (or file itself) with blocks
BlockInfoContiguous[] laterBlocks = findLaterSnapshotBlocks(removed.getSnapshotId());
laterBlocks = (laterBlocks == null) ? file.getContiguousBlocks() : laterBlocks;
BlockInfo[] laterBlocks = findLaterSnapshotBlocks(removed.getSnapshotId());
laterBlocks = (laterBlocks == null) ? file.getBlocks() : laterBlocks;
// Skip blocks, which belong to either the earlier or the later lists
int i = 0;
for(; i < removedBlocks.length; i++) {

View File

@ -22,7 +22,7 @@ import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.namenode.AclFeature;
import org.apache.hadoop.hdfs.server.namenode.INode;
import org.apache.hadoop.hdfs.server.namenode.AclStorage;
@ -208,7 +208,7 @@ public class FileWithSnapshotFeature implements INode.Feature {
// Collect blocks that should be deleted
FileDiff last = diffs.getLast();
BlockInfoContiguous[] snapshotBlocks = last == null ? null : last.getBlocks();
BlockInfo[] snapshotBlocks = last == null ? null : last.getBlocks();
if(snapshotBlocks == null)
file.collectBlocksBeyondMax(max, reclaimContext.collectedBlocks());
else

View File

@ -92,11 +92,6 @@ message INodeSection {
optional string clientMachine = 2;
}
message StripedBlocksFeature {
// store striped blocks related information
optional uint32 cellSize = 1;
}
message AclFeatureProto {
/**
* An ACL entry is represented by a 32-bit integer in Big Endian
@ -145,7 +140,8 @@ message INodeSection {
optional AclFeatureProto acl = 8;
optional XAttrFeatureProto xAttrs = 9;
optional uint32 storagePolicyID = 10;
optional StripedBlocksFeature stripedBlocks = 11;
optional bool isStriped = 11;
optional uint64 stripingCellSize = 12;
}
message QuotaByStorageTypeEntryProto {

View File

@ -23,6 +23,7 @@ import java.io.IOException;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.common.GenerationStamp;
import org.apache.hadoop.hdfs.server.common.Storage;
@ -66,7 +67,7 @@ public class CreateEditsLog {
INodeDirectory dirInode = new INodeDirectory(inodeId.nextValue(), null, p,
0L);
editLog.logMkDir(BASE_PATH, dirInode);
BlockInfoContiguous[] blocks = new BlockInfoContiguous[blocksPerFile];
BlockInfo[] blocks = new BlockInfo[blocksPerFile];
for (int iB = 0; iB < blocksPerFile; ++iB) {
blocks[iB] =
new BlockInfoContiguous(new Block(0, blockSize, BLOCK_GENERATION_STAMP),
@ -97,7 +98,7 @@ public class CreateEditsLog {
editLog.logMkDir(currentDir, dirInode);
}
INodeFile fileUc = new INodeFile(inodeId.nextValue(), null,
p, 0L, 0L, BlockInfoContiguous.EMPTY_ARRAY, replication, blockSize);
p, 0L, 0L, BlockInfo.EMPTY_ARRAY, replication, blockSize);
fileUc.toUnderConstruction("", "");
editLog.logOpenFile(filePath, fileUc, false, false);
editLog.logCloseFile(filePath, inode);

View File

@ -333,7 +333,7 @@ public class TestAddStripedBlocks {
INodeFile fileNode = ns.getFSDirectory().getINode(filePath.toString()).
asFile();
Assert.assertTrue(fileNode.isStriped());
BlockInfoStriped stored = fileNode.getStripedBlocksFeature().getBlocks()[0];
BlockInfo stored = fileNode.getBlocks()[0];
BlockManagerTestUtil.updateState(ns.getBlockManager());
Assert.assertEquals(0, ns.getCorruptReplicaBlocks());

View File

@ -69,7 +69,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
@ -205,7 +205,7 @@ public class TestEditLog {
for (int i = 0; i < numTransactions; i++) {
INodeFile inode = new INodeFile(namesystem.dir.allocateNewInodeId(), null,
p, 0L, 0L, BlockInfoContiguous.EMPTY_ARRAY, replication, blockSize);
p, 0L, 0L, BlockInfo.EMPTY_ARRAY, replication, blockSize);
inode.toUnderConstruction("", "");
editLog.logOpenFile("/filename" + (startIndex + i), inode, false, false);

View File

@ -46,6 +46,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@ -468,7 +469,7 @@ public class TestFSEditLogLoader {
new Block(blkId, blkNumBytes, timestamp), testSchema, cellSize);
INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
file.toUnderConstruction(clientName, clientMachine);
file.getStripedBlocksFeature().addBlock(stripedBlk);
file.addBlock(stripedBlk);
fns.getEditLog().logAddBlock(testFilePath, file);
file.toCompleteFile(System.currentTimeMillis());
@ -483,14 +484,14 @@ public class TestFSEditLogLoader {
assertTrue(inodeLoaded.isStriped());
BlockInfoStriped[] blks = (BlockInfoStriped[])inodeLoaded.getBlocks();
BlockInfo[] blks = inodeLoaded.getBlocks();
assertEquals(1, blks.length);
assertEquals(blkId, blks[0].getBlockId());
assertEquals(blkNumBytes, blks[0].getNumBytes());
assertEquals(timestamp, blks[0].getGenerationStamp());
assertEquals(blockNum, blks[0].getDataBlockNum());
assertEquals(parityNum, blks[0].getParityBlockNum());
assertEquals(cellSize, blks[0].getCellSize());
assertEquals(blockNum, ((BlockInfoStriped)blks[0]).getDataBlockNum());
assertEquals(parityNum, ((BlockInfoStriped)blks[0]).getParityBlockNum());
assertEquals(cellSize, ((BlockInfoStriped)blks[0]).getCellSize());
cluster.shutdown();
cluster = null;
@ -536,7 +537,7 @@ public class TestFSEditLogLoader {
new Block(blkId, blkNumBytes, timestamp), testSchema, cellSize);
INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
file.toUnderConstruction(clientName, clientMachine);
file.getStripedBlocksFeature().addBlock(stripedBlk);
file.addBlock(stripedBlk);
fns.getEditLog().logAddBlock(testFilePath, file);
file.toCompleteFile(System.currentTimeMillis());
fns.enterSafeMode(false);
@ -564,14 +565,15 @@ public class TestFSEditLogLoader {
assertTrue(inodeLoaded.isStriped());
BlockInfoStriped[] blks = (BlockInfoStriped[])inodeLoaded.getBlocks();
BlockInfo[] blks = inodeLoaded.getBlocks();
assertEquals(1, blks.length);
assertTrue(blks[0].isStriped());
assertEquals(blkId, blks[0].getBlockId());
assertEquals(newBlkNumBytes, blks[0].getNumBytes());
assertEquals(newTimestamp, blks[0].getGenerationStamp());
assertEquals(blockNum, blks[0].getDataBlockNum());
assertEquals(parityNum, blks[0].getParityBlockNum());
assertEquals(cellSize, blks[0].getCellSize());
assertEquals(blockNum, ((BlockInfoStriped)blks[0]).getDataBlockNum());
assertEquals(parityNum, ((BlockInfoStriped)blks[0]).getParityBlockNum());
assertEquals(cellSize, ((BlockInfoStriped)blks[0]).getCellSize());
cluster.shutdown();
cluster = null;

View File

@ -152,9 +152,8 @@ public class TestFSImage {
short replication = 3;
long preferredBlockSize = 128*1024*1024;
INodeFile file = new INodeFile(id, name, permissionStatus, mtime, atime,
blks, replication, preferredBlockSize);
blks, replication, preferredBlockSize, (byte) 0, true);
ByteArrayOutputStream bs = new ByteArrayOutputStream();
file.addStripedBlocksFeature();
//construct StripedBlocks for the INode
BlockInfoStriped[] stripedBlks = new BlockInfoStriped[3];
@ -164,7 +163,7 @@ public class TestFSImage {
stripedBlks[i] = new BlockInfoStriped(
new Block(stripedBlkId + i, preferredBlockSize, timestamp),
testSchema, cellSize);
file.getStripedBlocksFeature().addBlock(stripedBlks[i]);
file.addBlock(stripedBlks[i]);
}
final String client = "testClient";
@ -206,7 +205,7 @@ public class TestFSImage {
assertEquals(isUC ? mtime : atime, fileByLoaded.getAccessTime());
// TODO for striped blocks, we currently save and load them as contiguous
// blocks to/from legacy fsimage
assertEquals(3, fileByLoaded.getContiguousBlocks().length);
assertEquals(3, fileByLoaded.getBlocks().length);
assertEquals(preferredBlockSize, fileByLoaded.getPreferredBlockSize());
if (isUC) {
@ -405,13 +404,12 @@ public class TestFSImage {
// check the information of striped blocks
FSNamesystem fsn = cluster.getNamesystem();
INodeFile inode = fsn.dir.getINode(file.toString()).asFile();
FileWithStripedBlocksFeature sb = inode.getStripedBlocksFeature();
assertNotNull(sb);
BlockInfoStriped[] blks = sb.getBlocks();
assertTrue(inode.isStriped());
BlockInfo[] blks = inode.getBlocks();
assertEquals(1, blks.length);
assertTrue(blks[0].isStriped());
assertEquals(HdfsConstants.NUM_DATA_BLOCKS, blks[0].getDataBlockNum());
assertEquals(HdfsConstants.NUM_PARITY_BLOCKS, blks[0].getParityBlockNum());
assertEquals(HdfsConstants.NUM_DATA_BLOCKS, ((BlockInfoStriped)blks[0]).getDataBlockNum());
assertEquals(HdfsConstants.NUM_PARITY_BLOCKS, ((BlockInfoStriped)blks[0]).getParityBlockNum());
} finally {
cluster.shutdown();
}

View File

@ -92,7 +92,7 @@ public class TestINodeFile {
private static INodeFile createINodeFile(byte storagePolicyID) {
return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
null, (short)3, 1024L, storagePolicyID);
null, (short)3, 1024L, storagePolicyID, false);
}
@Test

View File

@ -57,14 +57,13 @@ public class TestStripedINodeFile {
private static INodeFile createStripedINodeFile() {
return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
null, (short)0, 1024L, HdfsServerConstants.COLD_STORAGE_POLICY_ID);
null, (short)0, 1024L, HdfsServerConstants.COLD_STORAGE_POLICY_ID, true);
}
@Test
public void testBlockStripedFeature()
throws IOException, InterruptedException{
INodeFile inf = createStripedINodeFile();
inf.addStripedBlocksFeature();
assertTrue(inf.isStriped());
}
@ -80,7 +79,6 @@ public class TestStripedINodeFile {
public void testBlockStripedLength()
throws IOException, InterruptedException {
INodeFile inf = createStripedINodeFile();
inf.addStripedBlocksFeature();
Block blk = new Block(1);
BlockInfoStriped blockInfoStriped
= new BlockInfoStriped(blk, testSchema, cellSize);
@ -92,7 +90,6 @@ public class TestStripedINodeFile {
public void testBlockStripedConsumedSpace()
throws IOException, InterruptedException {
INodeFile inf = createStripedINodeFile();
inf.addStripedBlocksFeature();
Block blk = new Block(1);
BlockInfoStriped blockInfoStriped
= new BlockInfoStriped(blk, testSchema, cellSize);
@ -110,7 +107,7 @@ public class TestStripedINodeFile {
// a. <Cell Size> * (<Num Stripes> - 1) * <Total Block Num> = 0
// b. <Num Bytes> % <Num Bytes per Stripes> = 1
// c. <Last Stripe Length> * <Parity Block Num> = 1 * 3
assertEquals(4, inf.storagespaceConsumedWithStriped().getStorageSpace());
assertEquals(4, inf.storagespaceConsumedStriped().getStorageSpace());
assertEquals(4, inf.storagespaceConsumed(defaultPolicy).getStorageSpace());
}
@ -118,7 +115,6 @@ public class TestStripedINodeFile {
public void testMultipleBlockStripedConsumedSpace()
throws IOException, InterruptedException {
INodeFile inf = createStripedINodeFile();
inf.addStripedBlocksFeature();
Block blk1 = new Block(1);
BlockInfoStriped blockInfoStriped1
= new BlockInfoStriped(blk1, testSchema, cellSize);
@ -130,7 +126,7 @@ public class TestStripedINodeFile {
inf.addBlock(blockInfoStriped1);
inf.addBlock(blockInfoStriped2);
// This is the double size of one block in above case.
assertEquals(4 * 2, inf.storagespaceConsumedWithStriped().getStorageSpace());
assertEquals(4 * 2, inf.storagespaceConsumedStriped().getStorageSpace());
assertEquals(4 * 2, inf.storagespaceConsumed(defaultPolicy).getStorageSpace());
}
@ -138,7 +134,6 @@ public class TestStripedINodeFile {
public void testBlockStripedFileSize()
throws IOException, InterruptedException {
INodeFile inf = createStripedINodeFile();
inf.addStripedBlocksFeature();
Block blk = new Block(1);
BlockInfoStriped blockInfoStriped
= new BlockInfoStriped(blk, testSchema, cellSize);
@ -154,7 +149,6 @@ public class TestStripedINodeFile {
public void testBlockStripedUCFileSize()
throws IOException, InterruptedException {
INodeFile inf = createStripedINodeFile();
inf.addStripedBlocksFeature();
Block blk = new Block(1);
BlockInfoStripedUnderConstruction bInfoStripedUC
= new BlockInfoStripedUnderConstruction(blk, testSchema, cellSize);
@ -168,7 +162,6 @@ public class TestStripedINodeFile {
public void testBlockStripedComputeQuotaUsage()
throws IOException, InterruptedException {
INodeFile inf = createStripedINodeFile();
inf.addStripedBlocksFeature();
Block blk = new Block(1);
BlockInfoStriped blockInfoStriped
= new BlockInfoStriped(blk, testSchema, cellSize);
@ -190,7 +183,6 @@ public class TestStripedINodeFile {
public void testBlockStripedUCComputeQuotaUsage()
throws IOException, InterruptedException {
INodeFile inf = createStripedINodeFile();
inf.addStripedBlocksFeature();
Block blk = new Block(1);
BlockInfoStripedUnderConstruction bInfoStripedUC
= new BlockInfoStripedUnderConstruction(blk, testSchema, cellSize);

View File

@ -78,7 +78,7 @@ public class TestTruncateQuotaUpdate {
@Test
public void testTruncateWithSnapshotNoDivergence() {
INodeFile file = createMockFile(BLOCKSIZE * 2 + BLOCKSIZE / 2, REPLICATION);
addSnapshotFeature(file, file.getContiguousBlocks());
addSnapshotFeature(file, file.getBlocks());
// case 4: truncate to 1.5 blocks
// all the blocks are in snapshot. truncate need to allocate a new block
@ -152,7 +152,7 @@ public class TestTruncateQuotaUpdate {
return new BlockInfoContiguous(b, replication);
}
private static void addSnapshotFeature(INodeFile file, BlockInfoContiguous[] blocks) {
private static void addSnapshotFeature(INodeFile file, BlockInfo[] blocks) {
FileDiff diff = mock(FileDiff.class);
when(diff.getBlocks()).thenReturn(blocks);
FileDiffList diffList = new FileDiffList();