HDFS-8367 BlockInfoStriped uses EC schema. Contributed by Kai Sasaki
This commit is contained in:
parent
d8ea443af0
commit
c9103e9cac
@ -215,3 +215,5 @@
|
||||
|
||||
HDFS-8391. NN should consider current EC tasks handling count from DN while
|
||||
assigning new tasks. (umamahesh)
|
||||
|
||||
HDFS-8367. BlockInfoStriped uses EC schema. (Kai Sasaki via Kai Zheng)
|
||||
|
@ -203,6 +203,7 @@
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
|
||||
import org.apache.hadoop.hdfs.server.common.StorageInfo;
|
||||
import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
|
||||
import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager;
|
||||
import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
|
||||
import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
|
||||
import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand;
|
||||
@ -445,9 +446,8 @@ public static Block convert(BlockProto b) {
|
||||
return new Block(b.getBlockId(), b.getNumBytes(), b.getGenStamp());
|
||||
}
|
||||
|
||||
public static BlockInfoStriped convert(StripedBlockProto p) {
|
||||
return new BlockInfoStriped(convert(p.getBlock()),
|
||||
(short) p.getDataBlockNum(), (short) p.getParityBlockNum());
|
||||
public static BlockInfoStriped convert(StripedBlockProto p, ECSchema schema) {
|
||||
return new BlockInfoStriped(convert(p.getBlock()), schema);
|
||||
}
|
||||
|
||||
public static StripedBlockProto convert(BlockInfoStriped blk) {
|
||||
|
@ -19,7 +19,9 @@
|
||||
|
||||
import org.apache.hadoop.hdfs.protocol.Block;
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
|
||||
import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager;
|
||||
import org.apache.hadoop.hdfs.util.StripedBlockUtil;
|
||||
import org.apache.hadoop.io.erasurecode.ECSchema;
|
||||
|
||||
import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
|
||||
|
||||
@ -39,6 +41,7 @@
|
||||
public class BlockInfoStriped extends BlockInfo {
|
||||
private final short dataBlockNum;
|
||||
private final short parityBlockNum;
|
||||
private final ECSchema schema;
|
||||
/**
|
||||
* Always the same size with triplets. Record the block index for each triplet
|
||||
* TODO: actually this is only necessary for over-replicated block. Thus can
|
||||
@ -46,16 +49,17 @@ public class BlockInfoStriped extends BlockInfo {
|
||||
*/
|
||||
private byte[] indices;
|
||||
|
||||
public BlockInfoStriped(Block blk, short dataBlockNum, short parityBlockNum) {
|
||||
super(blk, (short) (dataBlockNum + parityBlockNum));
|
||||
indices = new byte[dataBlockNum + parityBlockNum];
|
||||
public BlockInfoStriped(Block blk, ECSchema schema) {
|
||||
super(blk, (short) (schema.getNumDataUnits() + schema.getNumParityUnits()));
|
||||
indices = new byte[schema.getNumDataUnits() + schema.getNumParityUnits()];
|
||||
initIndices();
|
||||
this.dataBlockNum = dataBlockNum;
|
||||
this.parityBlockNum = parityBlockNum;
|
||||
this.schema = schema;
|
||||
this.dataBlockNum = (short)schema.getNumDataUnits();
|
||||
this.parityBlockNum = (short)schema.getNumParityUnits();
|
||||
}
|
||||
|
||||
BlockInfoStriped(BlockInfoStriped b) {
|
||||
this(b, b.dataBlockNum, b.parityBlockNum);
|
||||
this(b, b.getSchema());
|
||||
this.setBlockCollection(b.getBlockCollection());
|
||||
}
|
||||
|
||||
@ -71,6 +75,10 @@ public short getParityBlockNum() {
|
||||
return parityBlockNum;
|
||||
}
|
||||
|
||||
public ECSchema getSchema() {
|
||||
return schema;
|
||||
}
|
||||
|
||||
private void initIndices() {
|
||||
for (int i = 0; i < indices.length; i++) {
|
||||
indices[i] = -1;
|
||||
@ -231,8 +239,8 @@ public BlockInfoStripedUnderConstruction convertToBlockUnderConstruction(
|
||||
BlockUCState s, DatanodeStorageInfo[] targets) {
|
||||
final BlockInfoStripedUnderConstruction ucBlock;
|
||||
if(isComplete()) {
|
||||
ucBlock = new BlockInfoStripedUnderConstruction(this, getDataBlockNum(),
|
||||
getParityBlockNum(), s, targets);
|
||||
ucBlock = new BlockInfoStripedUnderConstruction(this, schema,
|
||||
s, targets);
|
||||
ucBlock.setBlockCollection(getBlockCollection());
|
||||
} else {
|
||||
// the block is already under construction
|
||||
|
@ -21,6 +21,7 @@
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
||||
import org.apache.hadoop.io.erasurecode.ECSchema;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
@ -56,17 +57,16 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped
|
||||
/**
|
||||
* Constructor with null storage targets.
|
||||
*/
|
||||
public BlockInfoStripedUnderConstruction(Block blk, short dataBlockNum,
|
||||
short parityBlockNum) {
|
||||
this(blk, dataBlockNum, parityBlockNum, UNDER_CONSTRUCTION, null);
|
||||
public BlockInfoStripedUnderConstruction(Block blk, ECSchema schema) {
|
||||
this(blk, schema, UNDER_CONSTRUCTION, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a striped block that is currently being constructed.
|
||||
*/
|
||||
public BlockInfoStripedUnderConstruction(Block blk, short dataBlockNum,
|
||||
short parityBlockNum, BlockUCState state, DatanodeStorageInfo[] targets) {
|
||||
super(blk, dataBlockNum, parityBlockNum);
|
||||
public BlockInfoStripedUnderConstruction(Block blk, ECSchema schema,
|
||||
BlockUCState state, DatanodeStorageInfo[] targets) {
|
||||
super(blk, schema);
|
||||
assert getBlockUCState() != COMPLETE :
|
||||
"BlockInfoStripedUnderConstruction cannot be in COMPLETE state";
|
||||
this.blockUCState = state;
|
||||
|
@ -532,8 +532,8 @@ private static BlockInfo addBlock(FSDirectory fsd, String path,
|
||||
// check quota limits and updated space consumed
|
||||
fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
|
||||
numLocations, true);
|
||||
blockInfo = new BlockInfoStripedUnderConstruction(block, numDataUnits,
|
||||
numParityUnits, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
|
||||
blockInfo = new BlockInfoStripedUnderConstruction(block, ecSchema,
|
||||
HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
|
||||
targets);
|
||||
} else {
|
||||
// check quota limits and updated space consumed
|
||||
|
@ -54,6 +54,9 @@
|
||||
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
|
||||
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;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
||||
|
@ -104,6 +104,7 @@
|
||||
import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress.Counter;
|
||||
import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
|
||||
import org.apache.hadoop.hdfs.util.Holder;
|
||||
import org.apache.hadoop.io.erasurecode.ECSchema;
|
||||
import org.apache.hadoop.util.ChunkedArrayList;
|
||||
|
||||
import com.google.common.base.Joiner;
|
||||
@ -416,8 +417,8 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir,
|
||||
// Update the salient file attributes.
|
||||
newFile.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID);
|
||||
newFile.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID);
|
||||
// TODO whether the file is striped should later be retrieved from iip
|
||||
updateBlocks(fsDir, addCloseOp, iip, newFile, fsDir.isInECZone(iip));
|
||||
updateBlocks(fsDir, addCloseOp, iip, newFile,
|
||||
fsDir.getECSchema(iip), fsDir.isInECZone(iip));
|
||||
break;
|
||||
}
|
||||
case OP_CLOSE: {
|
||||
@ -437,8 +438,8 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir,
|
||||
// Update the salient file attributes.
|
||||
file.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID);
|
||||
file.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID);
|
||||
// TODO whether the file is striped should later be retrieved from iip
|
||||
updateBlocks(fsDir, addCloseOp, iip, file, fsDir.isInECZone(iip));
|
||||
updateBlocks(fsDir, addCloseOp, iip, file,
|
||||
fsDir.getECSchema(iip), fsDir.isInECZone(iip));
|
||||
|
||||
// Now close the file
|
||||
if (!file.isUnderConstruction() &&
|
||||
@ -496,9 +497,9 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir,
|
||||
INodesInPath iip = fsDir.getINodesInPath(path, true);
|
||||
INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path);
|
||||
// Update in-memory data structures
|
||||
// TODO whether the file is striped should later be retrieved from iip
|
||||
updateBlocks(fsDir, updateOp, iip, oldFile, fsDir.isInECZone(iip));
|
||||
|
||||
updateBlocks(fsDir, updateOp, iip, oldFile,
|
||||
fsDir.getECSchema(iip), fsDir.isInECZone(iip));
|
||||
|
||||
if (toAddRetryCache) {
|
||||
fsNamesys.addCacheEntry(updateOp.rpcClientId, updateOp.rpcCallId);
|
||||
}
|
||||
@ -514,8 +515,8 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir,
|
||||
INodesInPath iip = fsDir.getINodesInPath(path, true);
|
||||
INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path);
|
||||
// add the new block to the INodeFile
|
||||
// TODO whether the file is striped should later be retrieved from iip
|
||||
addNewBlock(addBlockOp, oldFile, fsDir.isInECZone(iip));
|
||||
addNewBlock(addBlockOp, oldFile,
|
||||
fsDir.getECSchema(iip), fsDir.isInECZone(iip));
|
||||
break;
|
||||
}
|
||||
case OP_SET_REPLICATION: {
|
||||
@ -956,7 +957,8 @@ private static String formatEditLogReplayError(EditLogInputStream in,
|
||||
/**
|
||||
* Add a new block into the given INodeFile
|
||||
*/
|
||||
private void addNewBlock(AddBlockOp op, INodeFile file, boolean isStriped)
|
||||
private void addNewBlock(AddBlockOp op, INodeFile file,
|
||||
ECSchema schema, boolean isStriped)
|
||||
throws IOException {
|
||||
BlockInfo[] oldBlocks = file.getBlocks();
|
||||
Block pBlock = op.getPenultimateBlock();
|
||||
@ -985,8 +987,7 @@ private void addNewBlock(AddBlockOp op, INodeFile file, boolean isStriped)
|
||||
// add the new block
|
||||
final BlockInfo newBlockInfo;
|
||||
if (isStriped) {
|
||||
newBlockInfo = new BlockInfoStripedUnderConstruction(newBlock,
|
||||
HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS);
|
||||
newBlockInfo = new BlockInfoStripedUnderConstruction(newBlock, schema);
|
||||
} else {
|
||||
newBlockInfo = new BlockInfoContiguousUnderConstruction(newBlock,
|
||||
file.getPreferredBlockReplication());
|
||||
@ -1001,7 +1002,8 @@ private void addNewBlock(AddBlockOp op, INodeFile file, boolean isStriped)
|
||||
* @throws IOException
|
||||
*/
|
||||
private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op,
|
||||
INodesInPath iip, INodeFile file, boolean isStriped) throws IOException {
|
||||
INodesInPath iip, INodeFile file, ECSchema schema,
|
||||
boolean isStriped) throws IOException {
|
||||
// Update its block list
|
||||
BlockInfo[] oldBlocks = file.getBlocks();
|
||||
Block[] newBlocks = op.getBlocks();
|
||||
@ -1069,8 +1071,7 @@ private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op,
|
||||
// what about an old-version fsync() where fsync isn't called
|
||||
// until several blocks in?
|
||||
if (isStriped) {
|
||||
newBI = new BlockInfoStripedUnderConstruction(newBlock,
|
||||
HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS);
|
||||
newBI = new BlockInfoStripedUnderConstruction(newBlock, schema);
|
||||
} else {
|
||||
newBI = new BlockInfoContiguousUnderConstruction(newBlock,
|
||||
file.getPreferredBlockReplication());
|
||||
@ -1080,8 +1081,9 @@ private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op,
|
||||
// is only executed when loading edits written by prior
|
||||
// versions of Hadoop. Current versions always log
|
||||
// OP_ADD operations as each block is allocated.
|
||||
// TODO: ECSchema can be restored from persisted file (HDFS-7859).
|
||||
newBI = isStriped ? new BlockInfoStriped(newBlock,
|
||||
HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS) :
|
||||
ErasureCodingSchemaManager.getSystemDefaultSchema()) :
|
||||
new BlockInfoContiguous(newBlock, file.getPreferredBlockReplication());
|
||||
}
|
||||
fsNamesys.getBlockManager().addBlockCollectionWithCheck(newBI, file);
|
||||
|
@ -76,6 +76,7 @@
|
||||
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;
|
||||
@ -762,6 +763,9 @@ INode loadINode(final byte[] localName, boolean isSnapshotINode,
|
||||
NameNodeLayoutVersion.Feature.ERASURE_CODING, imgVersion)
|
||||
&& (in.readBoolean());
|
||||
final int numBlocks = in.readInt();
|
||||
// TODO: ECSchema can be restored from persisted file (HDFS-7859).
|
||||
final ECSchema schema = isStriped ?
|
||||
ErasureCodingSchemaManager.getSystemDefaultSchema() : null;
|
||||
|
||||
if (numBlocks >= 0) {
|
||||
// file
|
||||
@ -771,8 +775,7 @@ INode loadINode(final byte[] localName, boolean isSnapshotINode,
|
||||
if (isStriped) {
|
||||
blocks = new Block[numBlocks];
|
||||
for (int j = 0; j < numBlocks; j++) {
|
||||
blocks[j] = new BlockInfoStriped(new Block(),
|
||||
HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS);
|
||||
blocks[j] = new BlockInfoStriped(new Block(), schema);
|
||||
blocks[j].readFields(in);
|
||||
}
|
||||
} else {
|
||||
@ -804,8 +807,7 @@ INode loadINode(final byte[] localName, boolean isSnapshotINode,
|
||||
BlockInfoStriped lastStripedBlk = (BlockInfoStriped) lastBlk;
|
||||
blocks[blocks.length - 1]
|
||||
= new BlockInfoStripedUnderConstruction(lastBlk,
|
||||
lastStripedBlk.getDataBlockNum(),
|
||||
lastStripedBlk.getParityBlockNum());
|
||||
lastStripedBlk.getSchema());
|
||||
} else {
|
||||
blocks[blocks.length - 1]
|
||||
= new BlockInfoContiguousUnderConstruction(lastBlk,
|
||||
|
@ -69,6 +69,7 @@
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.protobuf.ByteString;
|
||||
import org.apache.hadoop.io.erasurecode.ECSchema;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public final class FSImageFormatPBINode {
|
||||
@ -354,10 +355,12 @@ private INodeFile loadINodeFile(INodeSection.INode n) {
|
||||
|
||||
FileWithStripedBlocksFeature stripeFeature = null;
|
||||
if (f.hasStripedBlocks()) {
|
||||
// TODO: HDFS-7859
|
||||
ECSchema schema = ErasureCodingSchemaManager.getSystemDefaultSchema();
|
||||
StripedBlocksFeature sb = f.getStripedBlocks();
|
||||
stripeFeature = file.addStripedBlocksFeature();
|
||||
for (StripedBlockProto sp : sb.getBlocksList()) {
|
||||
stripeFeature.addBlock(PBHelper.convert(sp));
|
||||
stripeFeature.addBlock(PBHelper.convert(sp, schema));
|
||||
}
|
||||
}
|
||||
|
||||
@ -372,7 +375,7 @@ private INodeFile loadINodeFile(INodeSection.INode n) {
|
||||
if (stripeFeature != null) {
|
||||
BlockInfoStriped striped = (BlockInfoStriped) lastBlk;
|
||||
ucBlk = new BlockInfoStripedUnderConstruction(striped,
|
||||
striped.getDataBlockNum(), striped.getParityBlockNum());
|
||||
striped.getSchema());
|
||||
} else {
|
||||
ucBlk = new BlockInfoContiguousUnderConstruction(lastBlk,
|
||||
replication);
|
||||
|
@ -50,6 +50,7 @@
|
||||
import org.apache.hadoop.io.ShortWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.io.WritableUtils;
|
||||
import org.apache.hadoop.io.erasurecode.ECSchema;
|
||||
import org.xml.sax.ContentHandler;
|
||||
import org.xml.sax.SAXException;
|
||||
|
||||
@ -130,7 +131,11 @@ static INodeFile readINodeUnderConstruction(
|
||||
final boolean isStriped = NameNodeLayoutVersion.supports(
|
||||
NameNodeLayoutVersion.Feature.ERASURE_CODING, imgVersion)
|
||||
&& (in.readBoolean());
|
||||
|
||||
|
||||
// TODO: ECSchema can be restored from persisted file (HDFS-7859).
|
||||
final ECSchema schema = isStriped ?
|
||||
ErasureCodingSchemaManager.getSystemDefaultSchema() : null;
|
||||
|
||||
int numBlocks = in.readInt();
|
||||
|
||||
final BlockInfoContiguous[] blocksContiguous;
|
||||
@ -140,15 +145,12 @@ static INodeFile readINodeUnderConstruction(
|
||||
blocksStriped = new BlockInfoStriped[numBlocks];
|
||||
int i = 0;
|
||||
for (; i < numBlocks - 1; i++) {
|
||||
blocksStriped[i] = new BlockInfoStriped(new Block(),
|
||||
HdfsConstants.NUM_DATA_BLOCKS,
|
||||
HdfsConstants.NUM_PARITY_BLOCKS);
|
||||
blocksStriped[i] = new BlockInfoStriped(new Block(), schema);
|
||||
blocksStriped[i].readFields(in);
|
||||
}
|
||||
if (numBlocks > 0) {
|
||||
blocksStriped[i] = new BlockInfoStripedUnderConstruction(new Block(),
|
||||
HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS,
|
||||
BlockUCState.UNDER_CONSTRUCTION, null);
|
||||
schema, BlockUCState.UNDER_CONSTRUCTION, null);
|
||||
blocksStriped[i].readFields(in);
|
||||
}
|
||||
} else {
|
||||
|
@ -3672,7 +3672,7 @@ void commitOrCompleteLastBlock(
|
||||
fileINode.getPreferredBlockSize() * numBlocks;
|
||||
|
||||
final BlockInfoStriped striped = new BlockInfoStriped(commitBlock,
|
||||
numDataUnits, numParityUnits);
|
||||
ecSchema);
|
||||
final long actualBlockGroupSize = striped.spaceConsumed();
|
||||
|
||||
diff = fullBlockGroupSize - actualBlockGroupSize;
|
||||
|
@ -20,6 +20,8 @@
|
||||
import org.apache.hadoop.hdfs.DFSTestUtil;
|
||||
import org.apache.hadoop.hdfs.protocol.Block;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBlockResult;
|
||||
import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager;
|
||||
import org.apache.hadoop.io.erasurecode.ECSchema;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import org.mockito.internal.util.reflection.Whitebox;
|
||||
@ -42,8 +44,10 @@ public class TestBlockInfoStriped {
|
||||
private static final int TOTAL_NUM_BLOCKS = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS;
|
||||
private static final long BASE_ID = -1600;
|
||||
private static final Block baseBlock = new Block(BASE_ID);
|
||||
private static final ECSchema testSchema
|
||||
= ErasureCodingSchemaManager.getSystemDefaultSchema();
|
||||
private final BlockInfoStriped info = new BlockInfoStriped(baseBlock,
|
||||
NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS);
|
||||
testSchema);
|
||||
|
||||
private Block[] createReportedBlocks(int num) {
|
||||
Block[] blocks = new Block[num];
|
||||
@ -231,7 +235,7 @@ public void testWrite() {
|
||||
ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
|
||||
DataOutput out = new DataOutputStream(byteStream);
|
||||
BlockInfoStriped blk = new BlockInfoStriped(new Block(blkID, numBytes,
|
||||
generationStamp), NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS);
|
||||
generationStamp), testSchema);
|
||||
|
||||
try {
|
||||
blk.write(out);
|
||||
|
@ -55,6 +55,7 @@
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
|
||||
import org.apache.hadoop.hdfs.protocol.Block;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.io.erasurecode.ECSchema;
|
||||
import org.apache.hadoop.test.PathUtils;
|
||||
import org.apache.log4j.Level;
|
||||
import org.junit.Test;
|
||||
@ -72,6 +73,9 @@ public class TestFSEditLogLoader {
|
||||
private static final File TEST_DIR = PathUtils.getTestDir(TestFSEditLogLoader.class);
|
||||
|
||||
private static final int NUM_DATA_NODES = 0;
|
||||
|
||||
private static final ECSchema testSchema
|
||||
= ErasureCodingSchemaManager.getSystemDefaultSchema();
|
||||
|
||||
@Test
|
||||
public void testDisplayRecentEditLogOpCodes() throws IOException {
|
||||
@ -459,7 +463,7 @@ public void testAddNewStripedBlock() throws IOException{
|
||||
|
||||
// Add a striped block to the file
|
||||
BlockInfoStriped stripedBlk = new BlockInfoStriped(
|
||||
new Block(blkId, blkNumBytes, timestamp), blockNum, parityNum);
|
||||
new Block(blkId, blkNumBytes, timestamp), testSchema);
|
||||
INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
|
||||
file.toUnderConstruction(clientName, clientMachine);
|
||||
file.getStripedBlocksFeature().addBlock(stripedBlk);
|
||||
@ -525,7 +529,7 @@ public void testUpdateStripedBlocks() throws IOException{
|
||||
Path p = new Path(testFilePath);
|
||||
DFSTestUtil.createFile(fs, p, 0, (short) 1, 1);
|
||||
BlockInfoStriped stripedBlk = new BlockInfoStriped(
|
||||
new Block(blkId, blkNumBytes, timestamp), blockNum, parityNum);
|
||||
new Block(blkId, blkNumBytes, timestamp), testSchema);
|
||||
INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
|
||||
file.toUnderConstruction(clientName, clientMachine);
|
||||
file.getStripedBlocksFeature().addBlock(stripedBlk);
|
||||
|
@ -32,6 +32,7 @@
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
|
||||
import org.apache.hadoop.io.erasurecode.ECSchema;
|
||||
import org.junit.Assert;
|
||||
|
||||
import org.apache.hadoop.fs.permission.PermissionStatus;
|
||||
@ -67,6 +68,9 @@ public class TestFSImage {
|
||||
|
||||
private static final String HADOOP_2_7_ZER0_BLOCK_SIZE_TGZ =
|
||||
"image-with-zero-block-size.tar.gz";
|
||||
private static final ECSchema testSchema
|
||||
= ErasureCodingSchemaManager.getSystemDefaultSchema();
|
||||
|
||||
@Test
|
||||
public void testPersist() throws IOException {
|
||||
Configuration conf = new Configuration();
|
||||
@ -158,7 +162,7 @@ private void testSaveAndLoadStripedINodeFile(FSNamesystem fsn, Configuration con
|
||||
for (int i = 0; i < stripedBlks.length; i++) {
|
||||
stripedBlks[i] = new BlockInfoStriped(
|
||||
new Block(stripedBlkId + i, preferredBlockSize, timestamp),
|
||||
HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS);
|
||||
testSchema);
|
||||
file.getStripedBlocksFeature().addBlock(stripedBlks[i]);
|
||||
}
|
||||
|
||||
|
@ -51,6 +51,9 @@ public class TestStripedINodeFile {
|
||||
private final BlockStoragePolicy defaultPolicy =
|
||||
defaultSuite.getDefaultPolicy();
|
||||
|
||||
private static final ECSchema testSchema
|
||||
= ErasureCodingSchemaManager.getSystemDefaultSchema();
|
||||
|
||||
private static INodeFile createStripedINodeFile() {
|
||||
return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
|
||||
null, (short)0, 1024L, HdfsServerConstants.COLD_STORAGE_POLICY_ID);
|
||||
@ -69,9 +72,7 @@ public void testBlockStripedTotalBlockCount() {
|
||||
ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
|
||||
Block blk = new Block(1);
|
||||
BlockInfoStriped blockInfoStriped
|
||||
= new BlockInfoStriped(blk,
|
||||
(short)defaultSchema.getNumDataUnits(),
|
||||
(short)defaultSchema.getNumParityUnits());
|
||||
= new BlockInfoStriped(blk, testSchema);
|
||||
assertEquals(9, blockInfoStriped.getTotalBlockNum());
|
||||
}
|
||||
|
||||
@ -83,9 +84,7 @@ public void testBlockStripedLength()
|
||||
inf.addStripedBlocksFeature();
|
||||
Block blk = new Block(1);
|
||||
BlockInfoStriped blockInfoStriped
|
||||
= new BlockInfoStriped(blk,
|
||||
(short)defaultSchema.getNumDataUnits(),
|
||||
(short)defaultSchema.getNumParityUnits());
|
||||
= new BlockInfoStriped(blk, testSchema);
|
||||
inf.addBlock(blockInfoStriped);
|
||||
assertEquals(1, inf.getBlocks().length);
|
||||
}
|
||||
@ -98,9 +97,7 @@ public void testBlockStripedConsumedSpace()
|
||||
inf.addStripedBlocksFeature();
|
||||
Block blk = new Block(1);
|
||||
BlockInfoStriped blockInfoStriped
|
||||
= new BlockInfoStriped(blk,
|
||||
(short)defaultSchema.getNumDataUnits(),
|
||||
(short)defaultSchema.getNumParityUnits());
|
||||
= new BlockInfoStriped(blk, testSchema);
|
||||
blockInfoStriped.setNumBytes(1);
|
||||
inf.addBlock(blockInfoStriped);
|
||||
// 0. Calculate the total bytes per stripes <Num Bytes per Stripes>
|
||||
@ -127,15 +124,11 @@ public void testMultipleBlockStripedConsumedSpace()
|
||||
inf.addStripedBlocksFeature();
|
||||
Block blk1 = new Block(1);
|
||||
BlockInfoStriped blockInfoStriped1
|
||||
= new BlockInfoStriped(blk1,
|
||||
(short)defaultSchema.getNumDataUnits(),
|
||||
(short)defaultSchema.getNumParityUnits());
|
||||
= new BlockInfoStriped(blk1, testSchema);
|
||||
blockInfoStriped1.setNumBytes(1);
|
||||
Block blk2 = new Block(2);
|
||||
BlockInfoStriped blockInfoStriped2
|
||||
= new BlockInfoStriped(blk2,
|
||||
(short)defaultSchema.getNumDataUnits(),
|
||||
(short)defaultSchema.getNumParityUnits());
|
||||
= new BlockInfoStriped(blk2, testSchema);
|
||||
blockInfoStriped2.setNumBytes(1);
|
||||
inf.addBlock(blockInfoStriped1);
|
||||
inf.addBlock(blockInfoStriped2);
|
||||
@ -152,9 +145,7 @@ public void testBlockStripedFileSize()
|
||||
inf.addStripedBlocksFeature();
|
||||
Block blk = new Block(1);
|
||||
BlockInfoStriped blockInfoStriped
|
||||
= new BlockInfoStriped(blk,
|
||||
(short)defaultSchema.getNumDataUnits(),
|
||||
(short)defaultSchema.getNumParityUnits());
|
||||
= new BlockInfoStriped(blk, testSchema);
|
||||
blockInfoStriped.setNumBytes(100);
|
||||
inf.addBlock(blockInfoStriped);
|
||||
// Compute file size should return actual data
|
||||
@ -171,9 +162,7 @@ public void testBlockStripedUCFileSize()
|
||||
inf.addStripedBlocksFeature();
|
||||
Block blk = new Block(1);
|
||||
BlockInfoStripedUnderConstruction bInfoStripedUC
|
||||
= new BlockInfoStripedUnderConstruction(blk,
|
||||
(short)defaultSchema.getNumDataUnits(),
|
||||
(short)defaultSchema.getNumParityUnits());
|
||||
= new BlockInfoStripedUnderConstruction(blk, testSchema);
|
||||
bInfoStripedUC.setNumBytes(100);
|
||||
inf.addBlock(bInfoStripedUC);
|
||||
assertEquals(100, inf.computeFileSize());
|
||||
@ -188,9 +177,7 @@ public void testBlockStripedComputeQuotaUsage()
|
||||
inf.addStripedBlocksFeature();
|
||||
Block blk = new Block(1);
|
||||
BlockInfoStriped blockInfoStriped
|
||||
= new BlockInfoStriped(blk,
|
||||
(short)defaultSchema.getNumDataUnits(),
|
||||
(short)defaultSchema.getNumParityUnits());
|
||||
= new BlockInfoStriped(blk, testSchema);
|
||||
blockInfoStriped.setNumBytes(100);
|
||||
inf.addBlock(blockInfoStriped);
|
||||
|
||||
@ -213,9 +200,7 @@ public void testBlockStripedUCComputeQuotaUsage()
|
||||
inf.addStripedBlocksFeature();
|
||||
Block blk = new Block(1);
|
||||
BlockInfoStripedUnderConstruction bInfoStripedUC
|
||||
= new BlockInfoStripedUnderConstruction(blk,
|
||||
(short)defaultSchema.getNumDataUnits(),
|
||||
(short)defaultSchema.getNumParityUnits());
|
||||
= new BlockInfoStripedUnderConstruction(blk, testSchema);
|
||||
bInfoStripedUC.setNumBytes(100);
|
||||
inf.addBlock(bInfoStripedUC);
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user