HDFS-8367 BlockInfoStriped uses EC schema. Contributed by Kai Sasaki

This commit is contained in:
Kai Zheng 2015-05-19 00:10:30 +08:00 committed by Zhe Zhang
parent d8ea443af0
commit c9103e9cac
15 changed files with 99 additions and 80 deletions

View File

@ -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)

View File

@ -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) {

View File

@ -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

View File

@ -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;

View File

@ -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

View File

@ -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;

View File

@ -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);

View 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,

View File

@ -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);

View File

@ -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 {

View File

@ -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;

View File

@ -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);

View File

@ -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);

View File

@ -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]);
}

View File

@ -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);