HDFS-8823. Move replication factor into individual blocks. Contributed by Haohui Mai.
This commit is contained in:
parent
7087e700e0
commit
745d04be59
|
@ -824,7 +824,9 @@ Release 2.8.0 - UNRELEASED
|
|||
|
||||
HDFS-8828. Utilize Snapshot diff report to build diff copy list in distcp.
|
||||
(Yufei Gu via Yongjun Zhang)
|
||||
|
||||
|
||||
HDFS-8823. Move replication factor into individual blocks. (wheat9)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than
|
||||
|
|
|
@ -55,12 +55,6 @@ public interface BlockCollection {
|
|||
public long getPreferredBlockSize();
|
||||
|
||||
/**
|
||||
* Get block replication for the collection
|
||||
* @return block replication value
|
||||
*/
|
||||
public short getPreferredBlockReplication();
|
||||
|
||||
/**
|
||||
* @return the storage policy ID.
|
||||
*/
|
||||
public byte getStoragePolicyID();
|
||||
|
|
|
@ -40,9 +40,14 @@ public abstract class BlockInfo extends Block
|
|||
|
||||
public static final BlockInfo[] EMPTY_ARRAY = {};
|
||||
|
||||
/**
|
||||
* Replication factor
|
||||
*/
|
||||
private short replication;
|
||||
|
||||
private BlockCollection bc;
|
||||
|
||||
/** For implementing {@link LightWeightGSet.LinkedElement} interface */
|
||||
/** For implementing {@link LightWeightGSet.LinkedElement} interface. */
|
||||
private LightWeightGSet.LinkedElement nextLinkedElement;
|
||||
|
||||
/**
|
||||
|
@ -68,12 +73,14 @@ public abstract class BlockInfo extends Block
|
|||
public BlockInfo(short replication) {
|
||||
this.triplets = new Object[3*replication];
|
||||
this.bc = null;
|
||||
this.replication = replication;
|
||||
}
|
||||
|
||||
public BlockInfo(Block blk, short replication) {
|
||||
super(blk);
|
||||
this.triplets = new Object[3*replication];
|
||||
this.bc = null;
|
||||
this.replication = replication;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -81,11 +88,18 @@ public abstract class BlockInfo extends Block
|
|||
* @param from BlockInfo to copy from.
|
||||
*/
|
||||
protected BlockInfo(BlockInfo from) {
|
||||
super(from);
|
||||
this.triplets = new Object[from.triplets.length];
|
||||
this(from, from.getReplication());
|
||||
this.bc = from.bc;
|
||||
}
|
||||
|
||||
public short getReplication() {
|
||||
return replication;
|
||||
}
|
||||
|
||||
public void setReplication(short repl) {
|
||||
this.replication = repl;
|
||||
}
|
||||
|
||||
public BlockCollection getBlockCollection() {
|
||||
return bc;
|
||||
}
|
||||
|
|
|
@ -1187,8 +1187,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
addToInvalidates(b.corrupted, node);
|
||||
return;
|
||||
}
|
||||
short expectedReplicas =
|
||||
b.corrupted.getBlockCollection().getPreferredBlockReplication();
|
||||
short expectedReplicas = b.corrupted.getReplication();
|
||||
|
||||
// Add replica to the data-node if it is not already there
|
||||
if (storageInfo != null) {
|
||||
|
@ -1363,7 +1362,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
continue;
|
||||
}
|
||||
|
||||
requiredReplication = bc.getPreferredBlockReplication();
|
||||
requiredReplication = getExpectedReplicaNum(block);
|
||||
|
||||
// get a source data-node
|
||||
containingNodes = new ArrayList<DatanodeDescriptor>();
|
||||
|
@ -1447,7 +1446,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
rw.targets = null;
|
||||
continue;
|
||||
}
|
||||
requiredReplication = bc.getPreferredBlockReplication();
|
||||
requiredReplication = getExpectedReplicaNum(block);
|
||||
|
||||
// do not schedule more if enough replicas is already pending
|
||||
NumberReplicas numReplicas = countNodes(block);
|
||||
|
@ -1712,7 +1711,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
continue;
|
||||
}
|
||||
NumberReplicas num = countNodes(timedOutItems[i]);
|
||||
if (isNeededReplication(bi, getReplication(bi), num.liveReplicas())) {
|
||||
if (isNeededReplication(bi, num.liveReplicas())) {
|
||||
neededReplications.add(bi, num.liveReplicas(),
|
||||
num.decommissionedAndDecommissioning(), getReplication(bi));
|
||||
}
|
||||
|
@ -2637,8 +2636,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
}
|
||||
|
||||
// handle underReplication/overReplication
|
||||
short fileReplication = bc.getPreferredBlockReplication();
|
||||
if (!isNeededReplication(storedBlock, fileReplication, numCurrentReplica)) {
|
||||
short fileReplication = getExpectedReplicaNum(storedBlock);
|
||||
if (!isNeededReplication(storedBlock, numCurrentReplica)) {
|
||||
neededReplications.remove(storedBlock, numCurrentReplica,
|
||||
num.decommissionedAndDecommissioning(), fileReplication);
|
||||
} else {
|
||||
|
@ -2867,12 +2866,11 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
return MisReplicationResult.UNDER_CONSTRUCTION;
|
||||
}
|
||||
// calculate current replication
|
||||
short expectedReplication =
|
||||
block.getBlockCollection().getPreferredBlockReplication();
|
||||
short expectedReplication = getExpectedReplicaNum(block);
|
||||
NumberReplicas num = countNodes(block);
|
||||
int numCurrentReplica = num.liveReplicas();
|
||||
// add to under-replicated queue if need to be
|
||||
if (isNeededReplication(block, expectedReplication, numCurrentReplica)) {
|
||||
if (isNeededReplication(block, numCurrentReplica)) {
|
||||
if (neededReplications.add(block, numCurrentReplica, num
|
||||
.decommissionedAndDecommissioning(), expectedReplication)) {
|
||||
return MisReplicationResult.UNDER_REPLICATED;
|
||||
|
@ -2898,27 +2896,18 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
}
|
||||
|
||||
/** Set replication for the blocks. */
|
||||
public void setReplication(final short oldRepl, final short newRepl,
|
||||
final String src, final BlockInfo... blocks) {
|
||||
public void setReplication(
|
||||
final short oldRepl, final short newRepl, final BlockInfo b) {
|
||||
if (newRepl == oldRepl) {
|
||||
return;
|
||||
}
|
||||
|
||||
// update needReplication priority queues
|
||||
for(BlockInfo b : blocks) {
|
||||
updateNeededReplications(b, 0, newRepl-oldRepl);
|
||||
}
|
||||
|
||||
b.setReplication(newRepl);
|
||||
updateNeededReplications(b, 0, newRepl - oldRepl);
|
||||
|
||||
if (oldRepl > newRepl) {
|
||||
// old replication > the new one; need to remove copies
|
||||
LOG.info("Decreasing replication from " + oldRepl + " to " + newRepl
|
||||
+ " for " + src);
|
||||
for(BlockInfo b : blocks) {
|
||||
processOverReplicatedBlock(b, newRepl, null, null);
|
||||
}
|
||||
} else { // replication factor is increased
|
||||
LOG.info("Increasing replication from " + oldRepl + " to " + newRepl
|
||||
+ " for " + src);
|
||||
processOverReplicatedBlock(b, newRepl, null, null);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -3385,8 +3374,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
int numOverReplicated = 0;
|
||||
while(it.hasNext()) {
|
||||
final BlockInfo block = it.next();
|
||||
BlockCollection bc = blocksMap.getBlockCollection(block);
|
||||
short expectedReplication = bc.getPreferredBlockReplication();
|
||||
short expectedReplication = block.getReplication();
|
||||
NumberReplicas num = countNodes(block);
|
||||
int numCurrentReplica = num.liveReplicas();
|
||||
if (numCurrentReplica > expectedReplication) {
|
||||
|
@ -3478,7 +3466,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
}
|
||||
NumberReplicas repl = countNodes(block);
|
||||
int curExpectedReplicas = getReplication(block);
|
||||
if (isNeededReplication(block, curExpectedReplicas, repl.liveReplicas())) {
|
||||
if (isNeededReplication(block, repl.liveReplicas())) {
|
||||
neededReplications.update(block, repl.liveReplicas(), repl
|
||||
.decommissionedAndDecommissioning(), curExpectedReplicas,
|
||||
curReplicasDelta, expectedReplicasDelta);
|
||||
|
@ -3500,10 +3488,10 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
* process it as an over replicated block.
|
||||
*/
|
||||
public void checkReplication(BlockCollection bc) {
|
||||
final short expected = bc.getPreferredBlockReplication();
|
||||
for (BlockInfo block : bc.getBlocks()) {
|
||||
final short expected = block.getReplication();
|
||||
final NumberReplicas n = countNodes(block);
|
||||
if (isNeededReplication(block, expected, n.liveReplicas())) {
|
||||
if (isNeededReplication(block, n.liveReplicas())) {
|
||||
neededReplications.add(block, n.liveReplicas(),
|
||||
n.decommissionedAndDecommissioning(), expected);
|
||||
} else if (n.liveReplicas() > expected) {
|
||||
|
@ -3535,12 +3523,10 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
* @return 0 if the block is not found;
|
||||
* otherwise, return the replication factor of the block.
|
||||
*/
|
||||
private int getReplication(Block block) {
|
||||
final BlockCollection bc = blocksMap.getBlockCollection(block);
|
||||
return bc == null? 0: bc.getPreferredBlockReplication();
|
||||
private int getReplication(BlockInfo block) {
|
||||
return getExpectedReplicaNum(block);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Get blocks to invalidate for <i>nodeId</i>
|
||||
* in {@link #invalidateBlocks}.
|
||||
|
@ -3581,7 +3567,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
return toInvalidate.size();
|
||||
}
|
||||
|
||||
boolean blockHasEnoughRacks(Block b) {
|
||||
boolean blockHasEnoughRacks(BlockInfo b) {
|
||||
if (!this.shouldCheckForEnoughRacks) {
|
||||
return true;
|
||||
}
|
||||
|
@ -3617,8 +3603,13 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
* A block needs replication if the number of replicas is less than expected
|
||||
* or if it does not have enough racks.
|
||||
*/
|
||||
boolean isNeededReplication(Block b, int expected, int current) {
|
||||
return current < expected || !blockHasEnoughRacks(b);
|
||||
boolean isNeededReplication(BlockInfo storedBlock, int current) {
|
||||
int expected = storedBlock.getReplication();
|
||||
return current < expected || !blockHasEnoughRacks(storedBlock);
|
||||
}
|
||||
|
||||
public short getExpectedReplicaNum(BlockInfo block) {
|
||||
return block.getReplication();
|
||||
}
|
||||
|
||||
public long getMissingBlocksCount() {
|
||||
|
|
|
@ -36,7 +36,6 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.protocol.Block;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
import org.apache.hadoop.hdfs.server.namenode.Namesystem;
|
||||
import org.apache.hadoop.hdfs.util.CyclicIteration;
|
||||
|
@ -241,9 +240,9 @@ public class DecommissionManager {
|
|||
private boolean isSufficientlyReplicated(BlockInfo block,
|
||||
BlockCollection bc,
|
||||
NumberReplicas numberReplicas) {
|
||||
final int numExpected = bc.getPreferredBlockReplication();
|
||||
final int numExpected = block.getReplication();
|
||||
final int numLive = numberReplicas.liveReplicas();
|
||||
if (!blockManager.isNeededReplication(block, numExpected, numLive)) {
|
||||
if (!blockManager.isNeededReplication(block, numLive)) {
|
||||
// Block doesn't need replication. Skip.
|
||||
LOG.trace("Block {} does not need replication.", block);
|
||||
return true;
|
||||
|
@ -274,11 +273,12 @@ public class DecommissionManager {
|
|||
return false;
|
||||
}
|
||||
|
||||
private static void logBlockReplicationInfo(Block block, BlockCollection bc,
|
||||
private static void logBlockReplicationInfo(BlockInfo block,
|
||||
BlockCollection bc,
|
||||
DatanodeDescriptor srcNode, NumberReplicas num,
|
||||
Iterable<DatanodeStorageInfo> storages) {
|
||||
int curReplicas = num.liveReplicas();
|
||||
int curExpectedReplicas = bc.getPreferredBlockReplication();
|
||||
int curExpectedReplicas = block.getReplication();
|
||||
StringBuilder nodeList = new StringBuilder();
|
||||
for (DatanodeStorageInfo storage : storages) {
|
||||
final DatanodeDescriptor node = storage.getDatanodeDescriptor();
|
||||
|
@ -540,8 +540,7 @@ public class DecommissionManager {
|
|||
|
||||
// Schedule under-replicated blocks for replication if not already
|
||||
// pending
|
||||
if (blockManager.isNeededReplication(block,
|
||||
bc.getPreferredBlockReplication(), liveReplicas)) {
|
||||
if (blockManager.isNeededReplication(block, liveReplicas)) {
|
||||
if (!blockManager.neededReplications.contains(block) &&
|
||||
blockManager.pendingReplications.getNumReplicas(block) == 0 &&
|
||||
namesystem.isPopulatingReplQueues()) {
|
||||
|
@ -549,7 +548,7 @@ public class DecommissionManager {
|
|||
blockManager.neededReplications.add(block,
|
||||
curReplicas,
|
||||
num.decommissionedAndDecommissioning(),
|
||||
bc.getPreferredBlockReplication());
|
||||
block.getReplication());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -245,7 +245,7 @@ final class FSDirAppendOp {
|
|||
final BlockInfo lastBlock = file.getLastBlock();
|
||||
if (lastBlock != null) {
|
||||
final long diff = file.getPreferredBlockSize() - lastBlock.getNumBytes();
|
||||
final short repl = file.getPreferredBlockReplication();
|
||||
final short repl = lastBlock.getReplication();
|
||||
delta.addStorageSpace(diff * repl);
|
||||
final BlockStoragePolicy policy = fsn.getFSDirectory()
|
||||
.getBlockStoragePolicySuite().getPolicy(file.getStoragePolicyID());
|
||||
|
|
|
@ -147,13 +147,11 @@ public class FSDirAttrOp {
|
|||
fsd.checkPathAccess(pc, iip, FsAction.WRITE);
|
||||
}
|
||||
|
||||
final short[] blockRepls = new short[2]; // 0: old, 1: new
|
||||
final BlockInfo[] blocks = unprotectedSetReplication(fsd, src,
|
||||
replication, blockRepls);
|
||||
replication);
|
||||
isFile = blocks != null;
|
||||
if (isFile) {
|
||||
fsd.getEditLog().logSetReplication(src, replication);
|
||||
bm.setReplication(blockRepls[0], blockRepls[1], src, blocks);
|
||||
}
|
||||
} finally {
|
||||
fsd.writeUnlock();
|
||||
|
@ -399,39 +397,51 @@ public class FSDirAttrOp {
|
|||
}
|
||||
|
||||
static BlockInfo[] unprotectedSetReplication(
|
||||
FSDirectory fsd, String src, short replication, short[] blockRepls)
|
||||
FSDirectory fsd, String src, short replication)
|
||||
throws QuotaExceededException, UnresolvedLinkException,
|
||||
SnapshotAccessControlException {
|
||||
assert fsd.hasWriteLock();
|
||||
|
||||
final BlockManager bm = fsd.getBlockManager();
|
||||
final INodesInPath iip = fsd.getINodesInPath4Write(src, true);
|
||||
final INode inode = iip.getLastINode();
|
||||
if (inode == null || !inode.isFile()) {
|
||||
return null;
|
||||
}
|
||||
INodeFile file = inode.asFile();
|
||||
final short oldBR = file.getPreferredBlockReplication();
|
||||
|
||||
// before setFileReplication, check for increasing block replication.
|
||||
// if replication > oldBR, then newBR == replication.
|
||||
// if replication < oldBR, we don't know newBR yet.
|
||||
if (replication > oldBR) {
|
||||
long dsDelta = file.storagespaceConsumed(null).getStorageSpace() / oldBR;
|
||||
fsd.updateCount(iip, 0L, dsDelta, oldBR, replication, true);
|
||||
// Make sure the directory has sufficient quotas
|
||||
short oldBR = file.getPreferredBlockReplication();
|
||||
|
||||
// Ensure the quota does not exceed
|
||||
if (oldBR < replication) {
|
||||
long size = file.computeFileSize(true, true);
|
||||
fsd.updateCount(iip, 0L, size, oldBR, replication, true);
|
||||
}
|
||||
|
||||
file.setFileReplication(replication, iip.getLatestSnapshotId());
|
||||
short targetReplication = (short) Math.max(
|
||||
replication, file.getPreferredBlockReplication());
|
||||
|
||||
final short newBR = file.getPreferredBlockReplication();
|
||||
// check newBR < oldBR case.
|
||||
if (newBR < oldBR) {
|
||||
long dsDelta = file.storagespaceConsumed(null).getStorageSpace() / newBR;
|
||||
fsd.updateCount(iip, 0L, dsDelta, oldBR, newBR, true);
|
||||
for (BlockInfo b : file.getBlocks()) {
|
||||
if (oldBR == targetReplication) {
|
||||
continue;
|
||||
}
|
||||
if (oldBR > replication) {
|
||||
fsd.updateCount(iip, 0L, b.getNumBytes(), oldBR, targetReplication,
|
||||
true);
|
||||
}
|
||||
bm.setReplication(oldBR, targetReplication, b);
|
||||
}
|
||||
|
||||
if (blockRepls != null) {
|
||||
blockRepls[0] = oldBR;
|
||||
blockRepls[1] = newBR;
|
||||
if (oldBR != -1) {
|
||||
if (oldBR > targetReplication) {
|
||||
FSDirectory.LOG.info("Decreasing replication from {} to {} for {}",
|
||||
oldBR, targetReplication, src);
|
||||
} else {
|
||||
FSDirectory.LOG.info("Increasing replication from {} to {} for {}",
|
||||
oldBR, targetReplication, src);
|
||||
}
|
||||
}
|
||||
return file.getBlocks();
|
||||
}
|
||||
|
|
|
@ -170,7 +170,7 @@ class FSDirConcatOp {
|
|||
QuotaCounts deltas = new QuotaCounts.Builder().build();
|
||||
final short targetRepl = target.getPreferredBlockReplication();
|
||||
for (INodeFile src : srcList) {
|
||||
short srcRepl = src.getPreferredBlockReplication();
|
||||
short srcRepl = src.getFileReplication();
|
||||
long fileSize = src.computeFileSize();
|
||||
if (targetRepl != srcRepl) {
|
||||
deltas.addStorageSpace(fileSize * (targetRepl - srcRepl));
|
||||
|
@ -223,7 +223,7 @@ class FSDirConcatOp {
|
|||
// the target file can be included in a snapshot
|
||||
trgInode.recordModification(targetIIP.getLatestSnapshotId());
|
||||
INodeDirectory trgParent = targetIIP.getINode(-2).asDirectory();
|
||||
trgInode.concatBlocks(srcList);
|
||||
trgInode.concatBlocks(srcList, fsd.getBlockManager());
|
||||
|
||||
// since we are in the same dir - we can use same parent to remove files
|
||||
int count = 0;
|
||||
|
|
|
@ -47,6 +47,7 @@ class FSDirDeleteOp {
|
|||
NameNode.stateChangeLog.debug("DIR* FSDirectory.delete: " + iip.getPath());
|
||||
}
|
||||
long filesRemoved = -1;
|
||||
FSNamesystem fsn = fsd.getFSNamesystem();
|
||||
fsd.writeLock();
|
||||
try {
|
||||
if (deleteAllowed(iip, iip.getPath()) ) {
|
||||
|
@ -58,7 +59,9 @@ class FSDirDeleteOp {
|
|||
if (unprotectedDelete(fsd, iip, context, mtime)) {
|
||||
filesRemoved = context.quotaDelta().getNsDelta();
|
||||
}
|
||||
fsd.getFSNamesystem().removeSnapshottableDirs(snapshottableDirs);
|
||||
fsd.updateReplicationFactor(context.collectedBlocks()
|
||||
.toUpdateReplicationInfo());
|
||||
fsn.removeSnapshottableDirs(snapshottableDirs);
|
||||
fsd.updateCount(iip, context.quotaDelta(), false);
|
||||
}
|
||||
} finally {
|
||||
|
|
|
@ -729,8 +729,8 @@ class FSDirRenameOp {
|
|||
Preconditions.checkState(oldDstChild != null);
|
||||
List<INode> removedINodes = new ChunkedArrayList<>();
|
||||
List<Long> removedUCFiles = new ChunkedArrayList<>();
|
||||
INode.ReclaimContext context = new INode.ReclaimContext(bsps,
|
||||
collectedBlocks, removedINodes, removedUCFiles);
|
||||
INode.ReclaimContext context = new INode.ReclaimContext(
|
||||
bsps, collectedBlocks, removedINodes, removedUCFiles);
|
||||
final boolean filesDeleted;
|
||||
if (!oldDstChild.isInLatestSnapshot(dstIIP.getLatestSnapshotId())) {
|
||||
oldDstChild.destroyAndCollectBlocks(context);
|
||||
|
@ -740,6 +740,9 @@ class FSDirRenameOp {
|
|||
dstIIP.getLatestSnapshotId());
|
||||
filesDeleted = context.quotaDelta().getNsDelta() >= 0;
|
||||
}
|
||||
fsd.updateReplicationFactor(context.collectedBlocks()
|
||||
.toUpdateReplicationInfo());
|
||||
|
||||
fsd.getFSNamesystem().removeLeasesAndINodes(
|
||||
removedUCFiles, removedINodes, false);
|
||||
return filesDeleted;
|
||||
|
|
|
@ -184,6 +184,8 @@ class FSDirSnapshotOp {
|
|||
snapshotManager.deleteSnapshot(iip, snapshotName, context);
|
||||
fsd.updateCount(iip, context.quotaDelta(), false);
|
||||
fsd.removeFromInodeMap(removedINodes);
|
||||
fsd.updateReplicationFactor(context.collectedBlocks()
|
||||
.toUpdateReplicationInfo());
|
||||
} finally {
|
||||
fsd.writeUnlock();
|
||||
}
|
||||
|
|
|
@ -517,7 +517,7 @@ class FSDirWriteFileOp {
|
|||
|
||||
// check quota limits and updated space consumed
|
||||
fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
|
||||
fileINode.getPreferredBlockReplication(), true);
|
||||
fileINode.getFileReplication(), true);
|
||||
|
||||
// associate new last block for the file
|
||||
BlockInfo blockInfo = new BlockInfoContiguous(block,
|
||||
|
|
|
@ -48,9 +48,11 @@ import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
|
|||
import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
|
||||
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
||||
import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo.UpdatedReplicationInfo;
|
||||
import org.apache.hadoop.hdfs.util.ByteArray;
|
||||
import org.apache.hadoop.hdfs.util.EnumCounters;
|
||||
import org.apache.hadoop.security.AccessControlException;
|
||||
|
@ -63,6 +65,7 @@ import java.io.FileNotFoundException;
|
|||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.EnumSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -450,6 +453,20 @@ public class FSDirectory implements Closeable {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Tell the block manager to update the replication factors when delete
|
||||
* happens. Deleting a file or a snapshot might decrease the replication
|
||||
* factor of the blocks as the blocks are always replicated to the highest
|
||||
* replication factor among all snapshots.
|
||||
*/
|
||||
void updateReplicationFactor(Collection<UpdatedReplicationInfo> blocks) {
|
||||
BlockManager bm = getBlockManager();
|
||||
for (UpdatedReplicationInfo e : blocks) {
|
||||
BlockInfo b = e.block();
|
||||
bm.setReplication(b.getReplication(), e.targetReplication(), b);
|
||||
}
|
||||
}
|
||||
|
||||
/** Updates namespace, storagespace and typespaces consumed for all
|
||||
* directories until the parent directory of file represented by path.
|
||||
*
|
||||
|
|
|
@ -514,7 +514,7 @@ public class FSEditLogLoader {
|
|||
short replication = fsNamesys.getBlockManager().adjustReplication(
|
||||
setReplicationOp.replication);
|
||||
FSDirAttrOp.unprotectedSetReplication(fsDir, renameReservedPathsOnUpgrade(
|
||||
setReplicationOp.path, logVersion), replication, null);
|
||||
setReplicationOp.path, logVersion), replication);
|
||||
break;
|
||||
}
|
||||
case OP_CONCAT_DELETE: {
|
||||
|
@ -1058,7 +1058,7 @@ public class FSEditLogLoader {
|
|||
// versions of Hadoop. Current versions always log
|
||||
// OP_ADD operations as each block is allocated.
|
||||
newBI = new BlockInfoContiguous(newBlock,
|
||||
file.getPreferredBlockReplication());
|
||||
file.getFileReplication());
|
||||
}
|
||||
fsNamesys.getBlockManager().addBlockCollection(newBI, file);
|
||||
file.addBlock(newBI);
|
||||
|
|
|
@ -901,15 +901,14 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
|
|||
|
||||
/**
|
||||
* @param bsps
|
||||
* block storage policy suite to calculate intended storage type
|
||||
* usage
|
||||
* block storage policy suite to calculate intended storage type
|
||||
* usage
|
||||
* @param collectedBlocks
|
||||
* blocks collected from the descents for further block
|
||||
* deletion/update will be added to the given map.
|
||||
* blocks collected from the descents for further block
|
||||
* deletion/update will be added to the given map.
|
||||
* @param removedINodes
|
||||
* INodes collected from the descents for further cleaning up of
|
||||
* INodes collected from the descents for further cleaning up of
|
||||
* @param removedUCFiles
|
||||
* files that the NN need to remove the leases
|
||||
*/
|
||||
public ReclaimContext(
|
||||
BlockStoragePolicySuite bsps, BlocksMapUpdateInfo collectedBlocks,
|
||||
|
@ -947,13 +946,44 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
|
|||
* Information used for updating the blocksMap when deleting files.
|
||||
*/
|
||||
public static class BlocksMapUpdateInfo {
|
||||
/**
|
||||
* The blocks whose replication factor need to be updated.
|
||||
*/
|
||||
public static class UpdatedReplicationInfo {
|
||||
/**
|
||||
* the expected replication after the update.
|
||||
*/
|
||||
private final short targetReplication;
|
||||
/**
|
||||
* The block whose replication needs to be updated.
|
||||
*/
|
||||
private final BlockInfo block;
|
||||
|
||||
public UpdatedReplicationInfo(short targetReplication, BlockInfo block) {
|
||||
this.targetReplication = targetReplication;
|
||||
this.block = block;
|
||||
}
|
||||
|
||||
public BlockInfo block() {
|
||||
return block;
|
||||
}
|
||||
|
||||
public short targetReplication() {
|
||||
return targetReplication;
|
||||
}
|
||||
}
|
||||
/**
|
||||
* The list of blocks that need to be removed from blocksMap
|
||||
*/
|
||||
private final List<BlockInfo> toDeleteList;
|
||||
/**
|
||||
* The list of blocks whose replication factor needs to be adjusted
|
||||
*/
|
||||
private final List<UpdatedReplicationInfo> toUpdateReplicationInfo;
|
||||
|
||||
public BlocksMapUpdateInfo() {
|
||||
toDeleteList = new ChunkedArrayList<>();
|
||||
toUpdateReplicationInfo = new ChunkedArrayList<>();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -962,7 +992,11 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
|
|||
public List<BlockInfo> getToDeleteList() {
|
||||
return toDeleteList;
|
||||
}
|
||||
|
||||
|
||||
public List<UpdatedReplicationInfo> toUpdateReplicationInfo() {
|
||||
return toUpdateReplicationInfo;
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a to-be-deleted block into the
|
||||
* {@link BlocksMapUpdateInfo#toDeleteList}
|
||||
|
@ -978,6 +1012,10 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
|
|||
toDeleteList.remove(block);
|
||||
}
|
||||
|
||||
public void addUpdateReplicationFactor(BlockInfo block, short targetRepl) {
|
||||
toUpdateReplicationInfo.add(
|
||||
new UpdatedReplicationInfo(targetRepl, block));
|
||||
}
|
||||
/**
|
||||
* Clear {@link BlocksMapUpdateInfo#toDeleteList}
|
||||
*/
|
||||
|
|
|
@ -37,6 +37,7 @@ import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
|
|||
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.BlockManager;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
|
||||
|
@ -353,12 +354,11 @@ public class INodeFile extends INodeWithAdditionalFields
|
|||
return getFileReplication(CURRENT_STATE_ID);
|
||||
}
|
||||
|
||||
@Override // BlockCollection
|
||||
public short getPreferredBlockReplication() {
|
||||
short max = getFileReplication(CURRENT_STATE_ID);
|
||||
FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature();
|
||||
if (sf != null) {
|
||||
short maxInSnapshot = sf.getMaxBlockRepInDiffs();
|
||||
short maxInSnapshot = sf.getMaxBlockRepInDiffs(null);
|
||||
if (sf.isCurrentFileDeleted()) {
|
||||
return maxInSnapshot;
|
||||
}
|
||||
|
@ -439,19 +439,10 @@ public class INodeFile extends INodeWithAdditionalFields
|
|||
return (snapshotBlocks == null) ? getBlocks() : snapshotBlocks;
|
||||
}
|
||||
|
||||
/** Used during concat to update the BlockCollection for each block. */
|
||||
private void updateBlockCollection() {
|
||||
if (blocks != null) {
|
||||
for(BlockInfo b : blocks) {
|
||||
b.setBlockCollection(this);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* append array of blocks to this.blocks
|
||||
*/
|
||||
void concatBlocks(INodeFile[] inodes) {
|
||||
void concatBlocks(INodeFile[] inodes, BlockManager bm) {
|
||||
int size = this.blocks.length;
|
||||
int totalAddedBlocks = 0;
|
||||
for(INodeFile f : inodes) {
|
||||
|
@ -468,7 +459,14 @@ public class INodeFile extends INodeWithAdditionalFields
|
|||
}
|
||||
|
||||
setBlocks(newlist);
|
||||
updateBlockCollection();
|
||||
for(BlockInfo b : blocks) {
|
||||
b.setBlockCollection(this);
|
||||
short oldRepl = b.getReplication();
|
||||
short repl = getPreferredBlockReplication();
|
||||
if (oldRepl != repl) {
|
||||
bm.setReplication(oldRepl, repl, b);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -857,10 +855,9 @@ public class INodeFile extends INodeWithAdditionalFields
|
|||
truncatedBytes -= bi.getNumBytes();
|
||||
}
|
||||
|
||||
delta.addStorageSpace(-truncatedBytes * getPreferredBlockReplication());
|
||||
delta.addStorageSpace(-truncatedBytes * bi.getReplication());
|
||||
if (bsps != null) {
|
||||
List<StorageType> types = bsps.chooseStorageTypes(
|
||||
getPreferredBlockReplication());
|
||||
List<StorageType> types = bsps.chooseStorageTypes(bi.getReplication());
|
||||
for (StorageType t : types) {
|
||||
if (t.supportTypeQuota()) {
|
||||
delta.addTypeSpace(t, -truncatedBytes);
|
||||
|
|
|
@ -254,8 +254,9 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
|
|||
NumberReplicas numberReplicas= bm.countNodes(blockInfo);
|
||||
out.println("Block Id: " + blockId);
|
||||
out.println("Block belongs to: "+iNode.getFullPathName());
|
||||
out.println("No. of Expected Replica: " +
|
||||
bc.getPreferredBlockReplication());
|
||||
if (blockInfo != null) {
|
||||
out.println("No. of Expected Replica: " + blockInfo.getReplication());
|
||||
}
|
||||
out.println("No. of live Replica: " + numberReplicas.liveReplicas());
|
||||
out.println("No. of excess Replica: " + numberReplicas.excessReplicas());
|
||||
out.println("No. of stale Replica: " +
|
||||
|
|
|
@ -44,6 +44,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.BlockManager;
|
||||
import org.apache.hadoop.hdfs.server.namenode.AclEntryStatusFormat;
|
||||
import org.apache.hadoop.hdfs.server.namenode.AclFeature;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
|
||||
|
@ -208,6 +209,7 @@ public class FSImageFormatPBSnapshot {
|
|||
throws IOException {
|
||||
final FileDiffList diffs = new FileDiffList();
|
||||
final LoaderContext state = parent.getLoaderContext();
|
||||
final BlockManager bm = fsn.getBlockManager();
|
||||
for (int i = 0; i < size; i++) {
|
||||
SnapshotDiffSection.FileDiff pbf = SnapshotDiffSection.FileDiff
|
||||
.parseDelimitedFrom(in);
|
||||
|
@ -243,9 +245,9 @@ public class FSImageFormatPBSnapshot {
|
|||
BlockInfo[] blocks = new BlockInfo[bpl.size()];
|
||||
for(int j = 0, e = bpl.size(); j < e; ++j) {
|
||||
Block blk = PBHelper.convert(bpl.get(j));
|
||||
BlockInfo storedBlock = fsn.getBlockManager().getStoredBlock(blk);
|
||||
BlockInfo storedBlock = bm.getStoredBlock(blk);
|
||||
if(storedBlock == null) {
|
||||
storedBlock = fsn.getBlockManager().addBlockCollection(
|
||||
storedBlock = bm.addBlockCollection(
|
||||
new BlockInfoContiguous(blk, copy.getFileReplication()), file);
|
||||
}
|
||||
blocks[j] = storedBlock;
|
||||
|
@ -256,6 +258,12 @@ public class FSImageFormatPBSnapshot {
|
|||
diffs.addFirst(diff);
|
||||
}
|
||||
file.addSnapshotFeature(diffs);
|
||||
short repl = file.getPreferredBlockReplication();
|
||||
for (BlockInfo b : file.getBlocks()) {
|
||||
if (b.getReplication() < repl) {
|
||||
bm.setReplication(b.getReplication(), repl, b);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Load the created list in a DirectoryDiff */
|
||||
|
|
|
@ -65,10 +65,10 @@ public class FileWithSnapshotFeature implements INode.Feature {
|
|||
}
|
||||
|
||||
/** @return the max replication factor in diffs */
|
||||
public short getMaxBlockRepInDiffs() {
|
||||
public short getMaxBlockRepInDiffs(FileDiff excluded) {
|
||||
short max = 0;
|
||||
for(FileDiff d : getDiffs()) {
|
||||
if (d.snapshotINode != null) {
|
||||
if (d != excluded && d.snapshotINode != null) {
|
||||
final short replication = d.snapshotINode.getFileReplication();
|
||||
if (replication > max) {
|
||||
max = replication;
|
||||
|
@ -147,28 +147,27 @@ public class FileWithSnapshotFeature implements INode.Feature {
|
|||
byte storagePolicyID = file.getStoragePolicyID();
|
||||
BlockStoragePolicy bsp = null;
|
||||
if (storagePolicyID != HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED) {
|
||||
bsp = reclaimContext.storagePolicySuite().getPolicy(file.getStoragePolicyID());
|
||||
bsp = reclaimContext.storagePolicySuite().
|
||||
getPolicy(file.getStoragePolicyID());
|
||||
}
|
||||
|
||||
|
||||
QuotaCounts oldCounts = file.storagespaceConsumed(null);
|
||||
long oldStoragespace;
|
||||
QuotaCounts oldCounts;
|
||||
if (removed.snapshotINode != null) {
|
||||
short replication = removed.snapshotINode.getFileReplication();
|
||||
short currentRepl = file.getPreferredBlockReplication();
|
||||
if (replication > currentRepl) {
|
||||
long oldFileSizeNoRep = currentRepl == 0
|
||||
? file.computeFileSize(true, true)
|
||||
: oldCounts.getStorageSpace() /
|
||||
file.getPreferredBlockReplication();
|
||||
oldStoragespace = oldFileSizeNoRep * replication;
|
||||
oldCounts.setStorageSpace(oldStoragespace);
|
||||
oldCounts = new QuotaCounts.Builder().build();
|
||||
BlockInfo[] blocks = file.getBlocks() == null ? new
|
||||
BlockInfo[0] : file.getBlocks();
|
||||
for (BlockInfo b: blocks) {
|
||||
short replication = b.getReplication();
|
||||
long blockSize = b.isComplete() ? b.getNumBytes() : file
|
||||
.getPreferredBlockSize();
|
||||
|
||||
oldCounts.addStorageSpace(blockSize * replication);
|
||||
|
||||
if (bsp != null) {
|
||||
List<StorageType> oldTypeChosen = bsp.chooseStorageTypes(replication);
|
||||
for (StorageType t : oldTypeChosen) {
|
||||
if (t.supportTypeQuota()) {
|
||||
oldCounts.addTypeSpace(t, oldFileSizeNoRep);
|
||||
oldCounts.addTypeSpace(t, blockSize);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -178,10 +177,21 @@ public class FileWithSnapshotFeature implements INode.Feature {
|
|||
if (aclFeature != null) {
|
||||
AclStorage.removeAclFeature(aclFeature);
|
||||
}
|
||||
} else {
|
||||
oldCounts = file.storagespaceConsumed(null);
|
||||
}
|
||||
|
||||
getDiffs().combineAndCollectSnapshotBlocks(reclaimContext, file, removed);
|
||||
|
||||
if (file.getBlocks() != null) {
|
||||
short replInDiff = getMaxBlockRepInDiffs(removed);
|
||||
short repl = (short) Math.max(file.getPreferredBlockReplication(),
|
||||
replInDiff);
|
||||
for (BlockInfo b : file.getBlocks()) {
|
||||
if (repl != b.getReplication()) {
|
||||
reclaimContext.collectedBlocks().addUpdateReplicationFactor(b, repl);
|
||||
}
|
||||
}
|
||||
}
|
||||
QuotaCounts current = file.storagespaceConsumed(bsp);
|
||||
reclaimContext.quotaDelta().add(oldCounts.subtract(current));
|
||||
}
|
||||
|
|
|
@ -434,7 +434,6 @@ public class TestBlockManager {
|
|||
|
||||
private BlockInfo addBlockOnNodes(long blockId, List<DatanodeDescriptor> nodes) {
|
||||
BlockCollection bc = Mockito.mock(BlockCollection.class);
|
||||
Mockito.doReturn((short)3).when(bc).getPreferredBlockReplication();
|
||||
BlockInfo blockInfo = blockOnNodes(blockId, nodes);
|
||||
|
||||
bm.blocksMap.addBlockCollection(blockInfo, bc);
|
||||
|
@ -741,7 +740,6 @@ public class TestBlockManager {
|
|||
BlockInfo blockInfo =
|
||||
new BlockInfoContiguous(block, (short) 3);
|
||||
BlockCollection bc = Mockito.mock(BlockCollection.class);
|
||||
Mockito.doReturn((short) 3).when(bc).getPreferredBlockReplication();
|
||||
bm.blocksMap.addBlockCollection(blockInfo, bc);
|
||||
return blockInfo;
|
||||
}
|
||||
|
@ -751,7 +749,6 @@ public class TestBlockManager {
|
|||
BlockInfo blockInfo = new BlockInfoContiguous(block, (short) 3);
|
||||
blockInfo.convertToBlockUnderConstruction(UNDER_CONSTRUCTION, null);
|
||||
BlockCollection bc = Mockito.mock(BlockCollection.class);
|
||||
Mockito.doReturn((short) 3).when(bc).getPreferredBlockReplication();
|
||||
bm.blocksMap.addBlockCollection(blockInfo, bc);
|
||||
return blockInfo;
|
||||
}
|
||||
|
|
|
@ -190,7 +190,6 @@ public class TestPendingReplication {
|
|||
DatanodeStorageInfo.toDatanodeDescriptors(
|
||||
DFSTestUtil.createDatanodeStorageInfos(1)));
|
||||
BlockCollection bc = Mockito.mock(BlockCollection.class);
|
||||
Mockito.doReturn((short) 3).when(bc).getPreferredBlockReplication();
|
||||
// Place into blocksmap with GenerationStamp = 1
|
||||
blockInfo.setGenerationStamp(1);
|
||||
blocksMap.addBlockCollection(blockInfo, bc);
|
||||
|
|
|
@ -1225,7 +1225,6 @@ public class TestReplicationPolicy {
|
|||
BlockInfoContiguous info = new BlockInfoContiguous(block1, (short) 1);
|
||||
info.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION, null);
|
||||
BlockCollection bc = mock(BlockCollection.class);
|
||||
when(bc.getPreferredBlockReplication()).thenReturn((short)1);
|
||||
bm.addBlockCollection(info, bc);
|
||||
|
||||
// Adding this block will increase its current replication, and that will
|
||||
|
@ -1269,7 +1268,6 @@ public class TestReplicationPolicy {
|
|||
final BlockCollection mbc = mock(BlockCollection.class);
|
||||
when(mbc.getLastBlock()).thenReturn(info);
|
||||
when(mbc.getPreferredBlockSize()).thenReturn(block1.getNumBytes() + 1);
|
||||
when(mbc.getPreferredBlockReplication()).thenReturn((short)1);
|
||||
when(mbc.isUnderConstruction()).thenReturn(true);
|
||||
ContentSummary cs = mock(ContentSummary.class);
|
||||
when(cs.getLength()).thenReturn((long)1);
|
||||
|
@ -1326,7 +1324,7 @@ public class TestReplicationPolicy {
|
|||
chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(1);
|
||||
assertTheChosenBlocks(chosenBlocks, 1, 0, 0, 0, 0);
|
||||
|
||||
bm.setReplication((short)0, (short)1, "", block1);
|
||||
bm.setReplication((short)0, (short)1, block1);
|
||||
|
||||
// Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
|
||||
// from QUEUE_VERY_UNDER_REPLICATED.
|
||||
|
|
|
@ -55,6 +55,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
|
|||
import org.apache.hadoop.hdfs.DFSUtil;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.protocol.Block;
|
||||
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
||||
|
@ -63,6 +64,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.blockmanagement.BlockManager;
|
||||
import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
|
||||
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
|
@ -271,8 +273,9 @@ public class TestINodeFile {
|
|||
INodeFile origFile = createINodeFiles(1, "origfile")[0];
|
||||
assertEquals("Number of blocks didn't match", origFile.numBlocks(), 1L);
|
||||
|
||||
INodeFile[] appendFiles = createINodeFiles(4, "appendfile");
|
||||
origFile.concatBlocks(appendFiles);
|
||||
INodeFile[] appendFiles = createINodeFiles(4, "appendfile");
|
||||
BlockManager bm = Mockito.mock(BlockManager.class);
|
||||
origFile.concatBlocks(appendFiles, bm);
|
||||
assertEquals("Number of blocks didn't match", origFile.numBlocks(), 5L);
|
||||
}
|
||||
|
||||
|
|
|
@ -22,6 +22,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.BlockManager;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
|
||||
import org.apache.hadoop.hdfs.server.namenode.INode;
|
||||
import org.apache.hadoop.hdfs.server.namenode.INodeFile;
|
||||
|
@ -53,12 +54,16 @@ public class TestFileWithSnapshotFeature {
|
|||
BlockInfo[] blocks = new BlockInfo[] {
|
||||
new BlockInfoContiguous(new Block(1, BLOCK_SIZE, 1), REPL_1)
|
||||
};
|
||||
BlockManager bm = mock(BlockManager.class);
|
||||
|
||||
// No snapshot
|
||||
INodeFile file = mock(INodeFile.class);
|
||||
when(file.getFileWithSnapshotFeature()).thenReturn(sf);
|
||||
when(file.getBlocks()).thenReturn(blocks);
|
||||
when(file.getStoragePolicyID()).thenReturn((byte) 1);
|
||||
Whitebox.setInternalState(file, "header", (long) REPL_1 << 48);
|
||||
when(file.getPreferredBlockReplication()).thenReturn(REPL_1);
|
||||
|
||||
when(bsps.getPolicy(anyByte())).thenReturn(bsp);
|
||||
INode.BlocksMapUpdateInfo collectedBlocks = mock(
|
||||
INode.BlocksMapUpdateInfo.class);
|
||||
|
@ -72,7 +77,6 @@ public class TestFileWithSnapshotFeature {
|
|||
|
||||
// INode only exists in the snapshot
|
||||
INodeFile snapshotINode = mock(INodeFile.class);
|
||||
when(file.getPreferredBlockReplication()).thenReturn(REPL_1);
|
||||
Whitebox.setInternalState(snapshotINode, "header", (long) REPL_3 << 48);
|
||||
Whitebox.setInternalState(diff, "snapshotINode", snapshotINode);
|
||||
when(diff.getSnapshotINode()).thenReturn(snapshotINode);
|
||||
|
@ -81,6 +85,7 @@ public class TestFileWithSnapshotFeature {
|
|||
.thenReturn(Lists.newArrayList(SSD));
|
||||
when(bsp.chooseStorageTypes(REPL_3))
|
||||
.thenReturn(Lists.newArrayList(DISK));
|
||||
blocks[0].setReplication(REPL_3);
|
||||
sf.updateQuotaAndCollectBlocks(ctx, file, diff);
|
||||
counts = ctx.quotaDelta().getCountsCopy();
|
||||
Assert.assertEquals((REPL_3 - REPL_1) * BLOCK_SIZE,
|
||||
|
|
|
@ -782,7 +782,7 @@ public class TestSnapshotDeletion {
|
|||
// modify file10, to check if the posterior diff was set correctly
|
||||
hdfs.setReplication(file10, REPLICATION);
|
||||
checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 7, 20 * BLOCKSIZE);
|
||||
|
||||
|
||||
Path file10_s1 = SnapshotTestHelper.getSnapshotPath(snapshotRoot, "s1",
|
||||
modDirStr + "file10");
|
||||
Path file11_s1 = SnapshotTestHelper.getSnapshotPath(snapshotRoot, "s1",
|
||||
|
@ -830,7 +830,7 @@ public class TestSnapshotDeletion {
|
|||
blockmanager);
|
||||
TestSnapshotBlocksMap.assertBlockCollection(file13_s1.toString(), 1, fsdir,
|
||||
blockmanager);
|
||||
|
||||
|
||||
// make sure file14 and file15 are not included in s1
|
||||
Path file14_s1 = SnapshotTestHelper.getSnapshotPath(snapshotRoot, "s1",
|
||||
modDirStr + "file14");
|
||||
|
@ -841,14 +841,18 @@ public class TestSnapshotDeletion {
|
|||
for (BlockInfo b : blocks_14) {
|
||||
assertNull(blockmanager.getBlockCollection(b));
|
||||
}
|
||||
|
||||
|
||||
INodeFile nodeFile13 = (INodeFile) fsdir.getINode(file13.toString());
|
||||
assertEquals(REPLICATION_1, nodeFile13.getPreferredBlockReplication());
|
||||
for (BlockInfo b: nodeFile13.getBlocks()) {
|
||||
assertEquals(REPLICATION_1, b.getReplication());
|
||||
}
|
||||
TestSnapshotBlocksMap.assertBlockCollection(file13.toString(), 1, fsdir,
|
||||
blockmanager);
|
||||
|
||||
|
||||
INodeFile nodeFile12 = (INodeFile) fsdir.getINode(file12_s1.toString());
|
||||
assertEquals(REPLICATION_1, nodeFile12.getPreferredBlockReplication());
|
||||
for (BlockInfo b: nodeFile12.getBlocks()) {
|
||||
assertEquals(REPLICATION_1, b.getReplication());
|
||||
}
|
||||
}
|
||||
|
||||
/** Test deleting snapshots with modification on the metadata of directory */
|
||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hdfs.DFSTestUtil;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
||||
import org.apache.hadoop.hdfs.server.namenode.INode;
|
||||
|
@ -38,10 +39,9 @@ import org.junit.Before;
|
|||
import org.junit.Test;
|
||||
|
||||
/**
|
||||
* This class tests the replication handling/calculation of snapshots. In
|
||||
* particular, {@link INodeFile#getFileReplication()} and
|
||||
* {@link INodeFile#getPreferredBlockReplication()} are tested to make sure
|
||||
* the number of replication is calculated correctly with/without snapshots.
|
||||
* This class tests the replication handling/calculation of snapshots to make
|
||||
* sure the number of replication is calculated correctly with/without
|
||||
* snapshots.
|
||||
*/
|
||||
public class TestSnapshotReplication {
|
||||
|
||||
|
@ -79,9 +79,7 @@ public class TestSnapshotReplication {
|
|||
}
|
||||
|
||||
/**
|
||||
* Check the replication of a given file. We test both
|
||||
* {@link INodeFile#getFileReplication()} and
|
||||
* {@link INodeFile#getPreferredBlockReplication()}.
|
||||
* Check the replication of a given file.
|
||||
*
|
||||
* @param file The given file
|
||||
* @param replication The expected replication number
|
||||
|
@ -98,8 +96,9 @@ public class TestSnapshotReplication {
|
|||
// Check the correctness of getPreferredBlockReplication()
|
||||
INode inode = fsdir.getINode(file1.toString());
|
||||
assertTrue(inode instanceof INodeFile);
|
||||
assertEquals(blockReplication,
|
||||
((INodeFile) inode).getPreferredBlockReplication());
|
||||
for (BlockInfo b: inode.asFile().getBlocks()) {
|
||||
assertEquals(blockReplication, b.getReplication());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -141,8 +140,9 @@ public class TestSnapshotReplication {
|
|||
// First check the getPreferredBlockReplication for the INode of
|
||||
// the currentFile
|
||||
final INodeFile inodeOfCurrentFile = getINodeFile(currentFile);
|
||||
assertEquals(expectedBlockRep,
|
||||
inodeOfCurrentFile.getPreferredBlockReplication());
|
||||
for (BlockInfo b : inodeOfCurrentFile.getBlocks()) {
|
||||
assertEquals(expectedBlockRep, b.getReplication());
|
||||
}
|
||||
// Then check replication for every snapshot
|
||||
for (Path ss : snapshotRepMap.keySet()) {
|
||||
final INodesInPath iip = fsdir.getINodesInPath(ss.toString(), true);
|
||||
|
@ -150,7 +150,9 @@ public class TestSnapshotReplication {
|
|||
// The replication number derived from the
|
||||
// INodeFileWithLink#getPreferredBlockReplication should
|
||||
// always == expectedBlockRep
|
||||
assertEquals(expectedBlockRep, ssInode.getPreferredBlockReplication());
|
||||
for (BlockInfo b : ssInode.getBlocks()) {
|
||||
assertEquals(expectedBlockRep, b.getReplication());
|
||||
}
|
||||
// Also check the number derived from INodeFile#getFileReplication
|
||||
assertEquals(snapshotRepMap.get(ss).shortValue(),
|
||||
ssInode.getFileReplication(iip.getPathSnapshotId()));
|
||||
|
@ -224,7 +226,10 @@ public class TestSnapshotReplication {
|
|||
// The replication number derived from the
|
||||
// INodeFileWithLink#getPreferredBlockReplication should
|
||||
// always == expectedBlockRep
|
||||
assertEquals(REPLICATION, ssInode.getPreferredBlockReplication());
|
||||
for (BlockInfo b : ssInode.getBlocks()) {
|
||||
assertEquals(REPLICATION, b.getReplication());
|
||||
}
|
||||
|
||||
// Also check the number derived from INodeFile#getFileReplication
|
||||
assertEquals(snapshotRepMap.get(ss).shortValue(),
|
||||
ssInode.getFileReplication());
|
||||
|
|
Loading…
Reference in New Issue