HDFS-8489. Subclass BlockInfo to represent contiguous blocks. Contributed by Zhe Zhang.

(cherry picked from commit cdc13efb1a)
This commit is contained in:
Jing Zhao 2015-06-01 11:37:47 -07:00
parent 26ad3293ec
commit 640ea994ed
20 changed files with 186 additions and 100 deletions

View File

@ -248,6 +248,9 @@ Release 2.8.0 - UNRELEASED
HDFS-8443. Document dfs.namenode.service.handler.count in hdfs-site.xml.
(J.Andreina via aajisaka)
HDFS-8489. Subclass BlockInfo to represent contiguous blocks.
(Zhe Zhang via jing9)
OPTIMIZATIONS
HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

View File

@ -33,7 +33,7 @@
* the block are stored.
*/
@InterfaceAudience.Private
public class BlockInfo extends Block
public abstract class BlockInfo extends Block
implements LightWeightGSet.LinkedElement {
public static final BlockInfo[] EMPTY_ARRAY = {};
@ -54,7 +54,7 @@ public class BlockInfo extends Block
* per replica is 42 bytes (LinkedList#Entry object per replica) versus 16
* bytes using the triplets.
*/
private Object[] triplets;
protected Object[] triplets;
/**
* Construct an entry for blocksmap
@ -105,7 +105,7 @@ DatanodeStorageInfo getStorageInfo(int index) {
return (DatanodeStorageInfo)triplets[index*3];
}
private BlockInfo getPrevious(int index) {
BlockInfo getPrevious(int index) {
assert this.triplets != null : "BlockInfo is not initialized";
assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound";
BlockInfo info = (BlockInfo)triplets[index*3+1];
@ -125,7 +125,7 @@ BlockInfo getNext(int index) {
return info;
}
private void setStorageInfo(int index, DatanodeStorageInfo storage) {
void setStorageInfo(int index, DatanodeStorageInfo storage) {
assert this.triplets != null : "BlockInfo is not initialized";
assert index >= 0 && index*3 < triplets.length : "Index is out of bound";
triplets[index*3] = storage;
@ -139,7 +139,7 @@ private void setStorageInfo(int index, DatanodeStorageInfo storage) {
* @param to - block to be set to previous on the list of blocks
* @return current previous block on the list of blocks
*/
private BlockInfo setPrevious(int index, BlockInfo to) {
BlockInfo setPrevious(int index, BlockInfo to) {
assert this.triplets != null : "BlockInfo is not initialized";
assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound";
BlockInfo info = (BlockInfo)triplets[index*3+1];
@ -155,7 +155,7 @@ private BlockInfo setPrevious(int index, BlockInfo to) {
* @param to - block to be set to next on the list of blocks
* * @return current next block on the list of blocks
*/
private BlockInfo setNext(int index, BlockInfo to) {
BlockInfo setNext(int index, BlockInfo to) {
assert this.triplets != null : "BlockInfo is not initialized";
assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound";
BlockInfo info = (BlockInfo)triplets[index*3+2];
@ -169,69 +169,27 @@ public int getCapacity() {
return triplets.length / 3;
}
/**
* Ensure that there is enough space to include num more triplets.
* @return first free triplet index.
*/
private int ensureCapacity(int num) {
assert this.triplets != null : "BlockInfo is not initialized";
int last = numNodes();
if(triplets.length >= (last+num)*3)
return last;
/* Not enough space left. Create a new array. Should normally
* happen only when replication is manually increased by the user. */
Object[] old = triplets;
triplets = new Object[(last+num)*3];
System.arraycopy(old, 0, triplets, 0, last*3);
return last;
}
/**
* Count the number of data-nodes the block belongs to.
*/
public int numNodes() {
assert this.triplets != null : "BlockInfo is not initialized";
assert triplets.length % 3 == 0 : "Malformed BlockInfo";
for(int idx = getCapacity()-1; idx >= 0; idx--) {
if(getDatanode(idx) != null)
return idx+1;
}
return 0;
}
public abstract int numNodes();
/**
* Add a {@link DatanodeStorageInfo} location for a block
* Add a {@link DatanodeStorageInfo} location for a block.
*/
boolean addStorage(DatanodeStorageInfo storage) {
// find the last null node
int lastNode = ensureCapacity(1);
setStorageInfo(lastNode, storage);
setNext(lastNode, null);
setPrevious(lastNode, null);
return true;
}
abstract boolean addStorage(DatanodeStorageInfo storage);
/**
* Remove {@link DatanodeStorageInfo} location for a block
*/
boolean removeStorage(DatanodeStorageInfo storage) {
int dnIndex = findStorageInfo(storage);
if(dnIndex < 0) // the node is not found
return false;
assert getPrevious(dnIndex) == null && getNext(dnIndex) == null :
"Block is still in the list and must be removed first.";
// find the last not null node
int lastNode = numNodes()-1;
// replace current node triplet by the lastNode one
setStorageInfo(dnIndex, getStorageInfo(lastNode));
setNext(dnIndex, getNext(lastNode));
setPrevious(dnIndex, getPrevious(lastNode));
// set the last triplet to null
setStorageInfo(lastNode, null);
setNext(lastNode, null);
setPrevious(lastNode, null);
return true;
}
abstract boolean removeStorage(DatanodeStorageInfo storage);
/**
* Replace the current BlockInfo with the new one in corresponding
* DatanodeStorageInfo's linked list
*/
abstract void replaceBlock(BlockInfo newBlock);
/**
* Find specified DatanodeStorageInfo.

View File

@ -0,0 +1,123 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.blockmanagement;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.protocol.Block;
/**
* Subclass of {@link BlockInfo}, used for a block with replication scheme.
*/
@InterfaceAudience.Private
public class BlockInfoContiguous extends BlockInfo {
public static final BlockInfoContiguous[] EMPTY_ARRAY = {};
public BlockInfoContiguous(short size) {
super(size);
}
public BlockInfoContiguous(Block blk, short size) {
super(blk, size);
}
/**
* Copy construction.
* This is used to convert BlockReplicationInfoUnderConstruction
* @param from BlockReplicationInfo to copy from.
*/
protected BlockInfoContiguous(BlockInfoContiguous from) {
super(from);
}
/**
* Ensure that there is enough space to include num more triplets.
* @return first free triplet index.
*/
private int ensureCapacity(int num) {
assert this.triplets != null : "BlockInfo is not initialized";
int last = numNodes();
if (triplets.length >= (last+num)*3) {
return last;
}
/* Not enough space left. Create a new array. Should normally
* happen only when replication is manually increased by the user. */
Object[] old = triplets;
triplets = new Object[(last+num)*3];
System.arraycopy(old, 0, triplets, 0, last * 3);
return last;
}
@Override
boolean addStorage(DatanodeStorageInfo storage) {
// find the last null node
int lastNode = ensureCapacity(1);
setStorageInfo(lastNode, storage);
setNext(lastNode, null);
setPrevious(lastNode, null);
return true;
}
@Override
boolean removeStorage(DatanodeStorageInfo storage) {
int dnIndex = findStorageInfo(storage);
if (dnIndex < 0) { // the node is not found
return false;
}
assert getPrevious(dnIndex) == null && getNext(dnIndex) == null :
"Block is still in the list and must be removed first.";
// find the last not null node
int lastNode = numNodes()-1;
// replace current node triplet by the lastNode one
setStorageInfo(dnIndex, getStorageInfo(lastNode));
setNext(dnIndex, getNext(lastNode));
setPrevious(dnIndex, getPrevious(lastNode));
// set the last triplet to null
setStorageInfo(lastNode, null);
setNext(lastNode, null);
setPrevious(lastNode, null);
return true;
}
@Override
public int numNodes() {
assert this.triplets != null : "BlockInfo is not initialized";
assert triplets.length % 3 == 0 : "Malformed BlockInfo";
for (int idx = getCapacity()-1; idx >= 0; idx--) {
if (getDatanode(idx) != null) {
return idx + 1;
}
}
return 0;
}
@Override
void replaceBlock(BlockInfo newBlock) {
assert newBlock instanceof BlockInfoContiguous;
for (int i = this.numNodes() - 1; i >= 0; i--) {
final DatanodeStorageInfo storage = this.getStorageInfo(i);
final boolean removed = storage.removeBlock(this);
assert removed : "currentBlock not found.";
final DatanodeStorageInfo.AddBlockResult result = storage.addBlock(
newBlock);
assert result == DatanodeStorageInfo.AddBlockResult.ADDED :
"newBlock already exists.";
}
}
}

View File

@ -31,7 +31,7 @@
* Represents a block that is currently being constructed.<br>
* This is usually the last block of a file opened for write or append.
*/
public class BlockInfoContiguousUnderConstruction extends BlockInfo {
public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
/** Block state. See {@link BlockUCState} */
private BlockUCState blockUCState;
@ -165,7 +165,8 @@ public BlockInfoContiguousUnderConstruction(Block blk, short replication) {
/**
* Create a block that is currently being constructed.
*/
public BlockInfoContiguousUnderConstruction(Block blk, short replication, BlockUCState state, DatanodeStorageInfo[] targets) {
public BlockInfoContiguousUnderConstruction(Block blk, short replication,
BlockUCState state, DatanodeStorageInfo[] targets) {
super(blk, replication);
assert getBlockUCState() != BlockUCState.COMPLETE :
"BlockInfoUnderConstruction cannot be in COMPLETE state";
@ -185,7 +186,7 @@ assert getBlockUCState() != BlockUCState.COMPLETE :
BlockInfo convertToCompleteBlock() throws IOException {
assert getBlockUCState() != BlockUCState.COMPLETE :
"Trying to convert a COMPLETE block";
return new BlockInfo(this);
return new BlockInfoContiguous(this);
}
/** Set expected locations */

View File

@ -1767,7 +1767,8 @@ private static class BlockToMarkCorrupt {
BlockToMarkCorrupt(BlockInfo stored, long gs, String reason,
Reason reasonCode) {
this(new BlockInfo(stored), stored, reason, reasonCode);
this(new BlockInfoContiguous((BlockInfoContiguous)stored), stored,
reason, reasonCode);
//the corrupted block in datanode has a different generation stamp
corrupted.setGenerationStamp(gs);
}
@ -2146,7 +2147,7 @@ private void reportDiff(DatanodeStorageInfo storageInfo,
// place a delimiter in the list which separates blocks
// that have been reported from those that have not
BlockInfo delimiter = new BlockInfo(new Block(), (short) 1);
BlockInfo delimiter = new BlockInfoContiguous(new Block(), (short) 1);
AddBlockResult result = storageInfo.addBlock(delimiter);
assert result == AddBlockResult.ADDED
: "Delimiting block cannot be present in the node";

View File

@ -20,12 +20,10 @@
import java.util.Iterator;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBlockResult;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.util.GSet;
import org.apache.hadoop.util.LightWeightGSet;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.collect.Iterables;
@ -222,16 +220,7 @@ BlockInfo replaceBlock(BlockInfo newBlock) {
BlockInfo currentBlock = blocks.get(newBlock);
assert currentBlock != null : "the block if not in blocksMap";
// replace block in data-node lists
for (int i = currentBlock.numNodes() - 1; i >= 0; i--) {
final DatanodeDescriptor dn = currentBlock.getDatanode(i);
final DatanodeStorageInfo storage = currentBlock.findStorageInfo(dn);
final boolean removed = storage.removeBlock(currentBlock);
Preconditions.checkState(removed, "currentBlock not found.");
final AddBlockResult result = storage.addBlock(newBlock);
Preconditions.checkState(result == AddBlockResult.ADDED,
"newBlock already exists.");
}
currentBlock.replaceBlock(newBlock);
// replace block in the map itself
blocks.put(newBlock);
return newBlock;

View File

@ -44,6 +44,7 @@
import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
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.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
@ -1058,7 +1059,7 @@ 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.
newBI = new BlockInfo(newBlock,
newBI = new BlockInfoContiguous(newBlock,
file.getPreferredBlockReplication());
}
fsNamesys.getBlockManager().addBlockCollection(newBI, file);

View File

@ -53,6 +53,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.BlockInfoContiguousUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@ -755,7 +756,7 @@ INode loadINode(final byte[] localName, boolean isSnapshotINode,
// read blocks
BlockInfo[] blocks = new BlockInfo[numBlocks];
for (int j = 0; j < numBlocks; j++) {
blocks[j] = new BlockInfo(replication);
blocks[j] = new BlockInfoContiguous(replication);
blocks[j].readFields(in);
}

View File

@ -43,6 +43,7 @@
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
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.BlockManager;
import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.LoaderContext;
@ -324,7 +325,8 @@ private INodeFile loadINodeFile(INodeSection.INode n) {
BlockInfo[] blocks = new BlockInfo[bp.size()];
for (int i = 0, e = bp.size(); i < e; ++i) {
blocks[i] = new BlockInfo(PBHelper.convert(bp.get(i)), replication);
blocks[i] =
new BlockInfoContiguous(PBHelper.convert(bp.get(i)), replication);
}
final PermissionStatus permissions = loadPermission(f.getPermission(),
parent.getLoaderContext().getStringTable());

View File

@ -34,6 +34,7 @@
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
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.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat;
@ -131,7 +132,7 @@ static INodeFile readINodeUnderConstruction(
int i = 0;
for (; i < numBlocks-1; i++) {
blk.readFields(in);
blocks[i] = new BlockInfo(blk, blockReplication);
blocks[i] = new BlockInfoContiguous(blk, blockReplication);
}
// last block is UNDER_CONSTRUCTION
if(numBlocks > 0) {

View File

@ -43,6 +43,7 @@
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
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.namenode.AclEntryStatusFormat;
import org.apache.hadoop.hdfs.server.namenode.AclFeature;
import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
@ -246,7 +247,7 @@ private void loadFileDiffList(InputStream in, INodeFile file, int size)
BlockInfo storedBlock = fsn.getBlockManager().getStoredBlock(blk);
if(storedBlock == null) {
storedBlock = fsn.getBlockManager().addBlockCollection(
new BlockInfo(blk, copy.getFileReplication()), file);
new BlockInfoContiguous(blk, copy.getFileReplication()), file);
}
blocks[j] = storedBlock;
}

View File

@ -49,7 +49,7 @@ public class TestBlockInfo {
@Test
public void testIsDeleted() {
BlockInfo blockInfo = new BlockInfo((short) 3);
BlockInfo blockInfo = new BlockInfoContiguous((short) 3);
BlockCollection bc = Mockito.mock(BlockCollection.class);
blockInfo.setBlockCollection(bc);
Assert.assertFalse(blockInfo.isDeleted());
@ -59,7 +59,7 @@ public void testIsDeleted() {
@Test
public void testAddStorage() throws Exception {
BlockInfo blockInfo = new BlockInfo((short) 3);
BlockInfo blockInfo = new BlockInfoContiguous((short) 3);
final DatanodeStorageInfo storage = DFSTestUtil.createDatanodeStorageInfo("storageID", "127.0.0.1");
@ -71,9 +71,9 @@ public void testAddStorage() throws Exception {
@Test
public void testCopyConstructor() {
BlockInfo old = new BlockInfo((short) 3);
BlockInfo old = new BlockInfoContiguous((short) 3);
try {
BlockInfo copy = new BlockInfo(old);
BlockInfo copy = new BlockInfoContiguous((BlockInfoContiguous)old);
assertEquals(old.getBlockCollection(), copy.getBlockCollection());
assertEquals(old.getCapacity(), copy.getCapacity());
} catch (Exception e) {
@ -92,7 +92,7 @@ public void testReplaceStorage() throws Exception {
// Create a few dummy blocks and add them to the first storage.
for (int i = 0; i < NUM_BLOCKS; ++i) {
blockInfos[i] = new BlockInfo((short) 3);
blockInfos[i] = new BlockInfoContiguous((short) 3);
storage1.addBlock(blockInfos[i]);
}
@ -118,7 +118,7 @@ public void testBlockListMoveToHead() throws Exception {
LOG.info("Building block list...");
for (int i = 0; i < MAX_BLOCKS; i++) {
blockList.add(new Block(i, 0, GenerationStamp.LAST_RESERVED_STAMP));
blockInfoList.add(new BlockInfo(blockList.get(i), (short) 3));
blockInfoList.add(new BlockInfoContiguous(blockList.get(i), (short) 3));
dd.addBlock(blockInfoList.get(i));
// index of the datanode should be 0
@ -182,4 +182,4 @@ public void testBlockListMoveToHead() throws Exception {
blockInfoList.get(j), dd.getBlockListHeadForTesting());
}
}
}
}

View File

@ -389,7 +389,7 @@ private void fulfillPipeline(BlockInfo blockInfo,
private BlockInfo blockOnNodes(long blkId, List<DatanodeDescriptor> nodes) {
Block block = new Block(blkId);
BlockInfo blockInfo = new BlockInfo(block, (short) 3);
BlockInfo blockInfo = new BlockInfoContiguous(block, (short) 3);
for (DatanodeDescriptor dn : nodes) {
for (DatanodeStorageInfo storage : dn.getStorageInfos()) {
@ -738,7 +738,7 @@ public void testSafeModeIBRBeforeFirstFullBR() throws Exception {
private BlockInfo addBlockToBM(long blkId) {
Block block = new Block(blkId);
BlockInfo blockInfo =
new BlockInfo(block, (short) 3);
new BlockInfoContiguous(block, (short) 3);
BlockCollection bc = Mockito.mock(BlockCollection.class);
Mockito.doReturn((short) 3).when(bc).getPreferredBlockReplication();
bm.blocksMap.addBlockCollection(blockInfo, bc);

View File

@ -58,8 +58,8 @@ public void testGetInvalidateBlocks() throws Exception {
public void testBlocksCounter() throws Exception {
DatanodeDescriptor dd = BlockManagerTestUtil.getLocalDatanodeDescriptor(true);
assertEquals(0, dd.numBlocks());
BlockInfo blk = new BlockInfo(new Block(1L), (short) 1);
BlockInfo blk1 = new BlockInfo(new Block(2L), (short) 2);
BlockInfo blk = new BlockInfoContiguous(new Block(1L), (short) 1);
BlockInfo blk1 = new BlockInfoContiguous(new Block(2L), (short) 2);
DatanodeStorageInfo[] storages = dd.getStorageInfos();
assertTrue(storages.length > 0);
// add first block

View File

@ -178,7 +178,7 @@ public void testProcessPendingReplications() throws Exception {
//
block = new Block(1, 1, 0);
blockInfo = new BlockInfo(block, (short) 3);
blockInfo = new BlockInfoContiguous(block, (short) 3);
pendingReplications.increment(block,
DatanodeStorageInfo.toDatanodeDescriptors(

View File

@ -24,6 +24,7 @@
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;
@ -69,8 +70,8 @@ static void addFiles(FSEditLog editLog, int numFiles, short replication,
BlockInfo[] blocks = new BlockInfo[blocksPerFile];
for (int iB = 0; iB < blocksPerFile; ++iB) {
blocks[iB] =
new BlockInfo(new Block(0, blockSize, BLOCK_GENERATION_STAMP),
replication);
new BlockInfoContiguous(new Block(0, blockSize, BLOCK_GENERATION_STAMP),
replication);
}
long currentBlockId = startingBlockId;

View File

@ -23,6 +23,7 @@
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
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.DatanodeStorageInfo;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@ -106,7 +107,7 @@ public void testCommitBlockSynchronization() throws IOException {
lastBlock, genStamp, length, false, false, newTargets, null);
// Simulate 'completing' the block.
BlockInfo completedBlockInfo = new BlockInfo(block, (short) 1);
BlockInfo completedBlockInfo = new BlockInfoContiguous(block, (short) 1);
completedBlockInfo.setBlockCollection(file);
completedBlockInfo.setGenerationStamp(genStamp);
doReturn(completedBlockInfo).when(namesystemSpy)
@ -178,7 +179,7 @@ public void testCommitBlockSynchronizationWithClose() throws IOException {
namesystemSpy.commitBlockSynchronization(
lastBlock, genStamp, length, true, false, newTargets, null);
BlockInfo completedBlockInfo = new BlockInfo(block, (short) 1);
BlockInfo completedBlockInfo = new BlockInfoContiguous(block, (short) 1);
completedBlockInfo.setBlockCollection(file);
completedBlockInfo.setGenerationStamp(genStamp);
doReturn(completedBlockInfo).when(namesystemSpy)

View File

@ -60,6 +60,7 @@
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
import org.apache.hadoop.io.IOUtils;
@ -289,7 +290,7 @@ private INodeFile[] createINodeFiles(int nCount, String fileNamePrefix) {
iNodes[i] = new INodeFile(i, null, perm, 0L, 0L, null, replication,
preferredBlockSize, (byte)0);
iNodes[i].setLocalName(DFSUtil.string2Bytes(fileNamePrefix + i));
BlockInfo newblock = new BlockInfo(replication);
BlockInfo newblock = new BlockInfoContiguous(replication);
iNodes[i].addBlock(newblock);
}

View File

@ -21,6 +21,7 @@
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.namenode.snapshot.FileDiff;
import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshotFeature;
@ -148,7 +149,7 @@ private INodeFile createMockFile(long size, short replication) {
private BlockInfo newBlock(long size, short replication) {
Block b = new Block(++nextMockBlockId, size, ++nextMockGenstamp);
return new BlockInfo(b, replication);
return new BlockInfoContiguous(b, replication);
}
private static void addSnapshotFeature(INodeFile file, BlockInfo[] blocks) {

View File

@ -21,6 +21,7 @@
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
import org.apache.hadoop.hdfs.server.namenode.INode;
import org.apache.hadoop.hdfs.server.namenode.INodeFile;
@ -50,7 +51,7 @@ public void testUpdateQuotaAndCollectBlocks() {
BlockStoragePolicySuite bsps = mock(BlockStoragePolicySuite.class);
BlockStoragePolicy bsp = mock(BlockStoragePolicy.class);
BlockInfo[] blocks = new BlockInfo[] {
new BlockInfo(new Block(1, BLOCK_SIZE, 1), REPL_1)
new BlockInfoContiguous(new Block(1, BLOCK_SIZE, 1), REPL_1)
};
// No snapshot