HDFS-5390. Send one incremental block report per storage directory.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2832@1534891 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Arpit Agarwal 2013-10-23 01:28:48 +00:00
parent b884af72c5
commit 01f37e42f0
16 changed files with 204 additions and 83 deletions

View File

@ -42,3 +42,6 @@ IMPROVEMENTS:
HDFS-5398. NameNode changes to process storage reports per storage
directory. (Arpit Agarwal)
HDFS-5390. Send one incremental block report per storage directory.
(Arpit Agarwal)

View File

@ -192,7 +192,8 @@ void reportBadBlocks(ExtendedBlock block) {
* till namenode is informed before responding with success to the
* client? For now we don't.
*/
void notifyNamenodeReceivedBlock(ExtendedBlock block, String delHint) {
void notifyNamenodeReceivedBlock(
ExtendedBlock block, String delHint, String storageUuid) {
checkBlock(block);
checkDelHint(delHint);
ReceivedDeletedBlockInfo bInfo = new ReceivedDeletedBlockInfo(
@ -201,7 +202,7 @@ void notifyNamenodeReceivedBlock(ExtendedBlock block, String delHint) {
delHint);
for (BPServiceActor actor : bpServices) {
actor.notifyNamenodeBlockImmediately(bInfo);
actor.notifyNamenodeBlockImmediately(bInfo, storageUuid);
}
}
@ -218,23 +219,23 @@ private void checkDelHint(String delHint) {
"delHint is null");
}
void notifyNamenodeDeletedBlock(ExtendedBlock block) {
void notifyNamenodeDeletedBlock(ExtendedBlock block, String storageUuid) {
checkBlock(block);
ReceivedDeletedBlockInfo bInfo = new ReceivedDeletedBlockInfo(
block.getLocalBlock(), BlockStatus.DELETED_BLOCK, null);
for (BPServiceActor actor : bpServices) {
actor.notifyNamenodeDeletedBlock(bInfo);
actor.notifyNamenodeDeletedBlock(bInfo, storageUuid);
}
}
void notifyNamenodeReceivingBlock(ExtendedBlock block) {
void notifyNamenodeReceivingBlock(ExtendedBlock block, String storageUuid) {
checkBlock(block);
ReceivedDeletedBlockInfo bInfo = new ReceivedDeletedBlockInfo(
block.getLocalBlock(), BlockStatus.RECEIVING_BLOCK, null);
for (BPServiceActor actor : bpServices) {
actor.notifyNamenodeBlockImmediately(bInfo);
actor.notifyNamenodeBlockImmediately(bInfo, storageUuid);
}
}

View File

@ -23,7 +23,6 @@
import java.net.InetSocketAddress;
import java.net.SocketTimeoutException;
import java.util.Collection;
import java.util.Iterator;
import java.util.Map;
import org.apache.commons.logging.Log;
@ -94,9 +93,9 @@ class BPServiceActor implements Runnable {
* keyed by block ID, contains the pending changes which have yet to be
* reported to the NN. Access should be synchronized on this object.
*/
private final Map<Long, ReceivedDeletedBlockInfo> pendingIncrementalBR
= Maps.newHashMap();
private final Map<String, PerStoragePendingIncrementalBR>
pendingIncrementalBRperStorage = Maps.newConcurrentMap();
private volatile int pendingReceivedRequests = 0;
private volatile boolean shouldServiceRun = true;
private final DataNode dn;
@ -263,64 +262,84 @@ void reportBadBlocks(ExtendedBlock block) {
* @throws IOException
*/
private void reportReceivedDeletedBlocks() throws IOException {
// check if there are newly received blocks
ReceivedDeletedBlockInfo[] receivedAndDeletedBlockArray = null;
synchronized (pendingIncrementalBR) {
int numBlocks = pendingIncrementalBR.size();
if (numBlocks > 0) {
//
// Send newly-received and deleted blockids to namenode
//
receivedAndDeletedBlockArray = pendingIncrementalBR
.values().toArray(new ReceivedDeletedBlockInfo[numBlocks]);
// For each storage, check if there are newly received blocks and if
// so then send an incremental report to the NameNode.
for (Map.Entry<String, PerStoragePendingIncrementalBR> entry :
pendingIncrementalBRperStorage.entrySet()) {
final String storageUuid = entry.getKey();
final PerStoragePendingIncrementalBR perStorageMap = entry.getValue();
ReceivedDeletedBlockInfo[] receivedAndDeletedBlockArray = null;
// TODO: We can probably use finer-grained synchronization now.
synchronized (pendingIncrementalBRperStorage) {
if (perStorageMap.getBlockInfoCount() > 0) {
// Send newly-received and deleted blockids to namenode
receivedAndDeletedBlockArray = perStorageMap.dequeueBlockInfos();
pendingReceivedRequests -= receivedAndDeletedBlockArray.length;
}
}
pendingIncrementalBR.clear();
}
if (receivedAndDeletedBlockArray != null) {
StorageReceivedDeletedBlocks[] report = { new StorageReceivedDeletedBlocks(
bpRegistration.getDatanodeUuid(), receivedAndDeletedBlockArray) };
boolean success = false;
try {
bpNamenode.blockReceivedAndDeleted(bpRegistration, bpos.getBlockPoolId(),
report);
success = true;
} finally {
synchronized (pendingIncrementalBR) {
if (!success) {
// If we didn't succeed in sending the report, put all of the
// blocks back onto our queue, but only in the case where we didn't
// put something newer in the meantime.
for (ReceivedDeletedBlockInfo rdbi : receivedAndDeletedBlockArray) {
if (!pendingIncrementalBR.containsKey(rdbi.getBlock().getBlockId())) {
pendingIncrementalBR.put(rdbi.getBlock().getBlockId(), rdbi);
}
if (receivedAndDeletedBlockArray != null) {
StorageReceivedDeletedBlocks[] report = { new StorageReceivedDeletedBlocks(
storageUuid, receivedAndDeletedBlockArray) };
boolean success = false;
try {
bpNamenode.blockReceivedAndDeleted(bpRegistration, bpos.getBlockPoolId(),
report);
success = true;
} finally {
synchronized (pendingIncrementalBRperStorage) {
if (!success) {
// If we didn't succeed in sending the report, put all of the
// blocks back onto our queue, but only in the case where we
// didn't put something newer in the meantime.
perStorageMap.putMissingBlockInfos(receivedAndDeletedBlockArray);
pendingReceivedRequests += perStorageMap.getBlockInfoCount();
}
}
pendingReceivedRequests = pendingIncrementalBR.size();
}
}
}
}
/**
* Retrieve the incremental BR state for a given storage UUID
* @param storageUuid
* @return
*/
private PerStoragePendingIncrementalBR getIncrementalBRMapForStorage(
String storageUuid) {
PerStoragePendingIncrementalBR mapForStorage =
pendingIncrementalBRperStorage.get(storageUuid);
if (mapForStorage == null) {
// This is the first time we are adding incremental BR state for
// this storage so create a new map. This is required once per
// storage, per service actor.
mapForStorage = new PerStoragePendingIncrementalBR();
pendingIncrementalBRperStorage.put(storageUuid, mapForStorage);
}
return mapForStorage;
}
/*
* Informing the name node could take a long long time! Should we wait
* till namenode is informed before responding with success to the
* client? For now we don't.
*/
void notifyNamenodeBlockImmediately(ReceivedDeletedBlockInfo bInfo) {
synchronized (pendingIncrementalBR) {
pendingIncrementalBR.put(
bInfo.getBlock().getBlockId(), bInfo);
void notifyNamenodeBlockImmediately(
ReceivedDeletedBlockInfo bInfo, String storageUuid) {
synchronized (pendingIncrementalBRperStorage) {
getIncrementalBRMapForStorage(storageUuid).putBlockInfo(bInfo);
pendingReceivedRequests++;
pendingIncrementalBR.notifyAll();
pendingIncrementalBRperStorage.notifyAll();
}
}
void notifyNamenodeDeletedBlock(ReceivedDeletedBlockInfo bInfo) {
synchronized (pendingIncrementalBR) {
pendingIncrementalBR.put(
bInfo.getBlock().getBlockId(), bInfo);
void notifyNamenodeDeletedBlock(
ReceivedDeletedBlockInfo bInfo, String storageUuid) {
synchronized (pendingIncrementalBRperStorage) {
getIncrementalBRMapForStorage(storageUuid).putBlockInfo(bInfo);
}
}
@ -329,13 +348,13 @@ void notifyNamenodeDeletedBlock(ReceivedDeletedBlockInfo bInfo) {
*/
@VisibleForTesting
void triggerBlockReportForTests() {
synchronized (pendingIncrementalBR) {
synchronized (pendingIncrementalBRperStorage) {
lastBlockReport = 0;
lastHeartbeat = 0;
pendingIncrementalBR.notifyAll();
pendingIncrementalBRperStorage.notifyAll();
while (lastBlockReport == 0) {
try {
pendingIncrementalBR.wait(100);
pendingIncrementalBRperStorage.wait(100);
} catch (InterruptedException e) {
return;
}
@ -345,12 +364,12 @@ void triggerBlockReportForTests() {
@VisibleForTesting
void triggerHeartbeatForTests() {
synchronized (pendingIncrementalBR) {
synchronized (pendingIncrementalBRperStorage) {
lastHeartbeat = 0;
pendingIncrementalBR.notifyAll();
pendingIncrementalBRperStorage.notifyAll();
while (lastHeartbeat == 0) {
try {
pendingIncrementalBR.wait(100);
pendingIncrementalBRperStorage.wait(100);
} catch (InterruptedException e) {
return;
}
@ -360,13 +379,13 @@ void triggerHeartbeatForTests() {
@VisibleForTesting
void triggerDeletionReportForTests() {
synchronized (pendingIncrementalBR) {
synchronized (pendingIncrementalBRperStorage) {
lastDeletedReport = 0;
pendingIncrementalBR.notifyAll();
pendingIncrementalBRperStorage.notifyAll();
while (lastDeletedReport == 0) {
try {
pendingIncrementalBR.wait(100);
pendingIncrementalBRperStorage.wait(100);
} catch (InterruptedException e) {
return;
}
@ -582,10 +601,10 @@ private void offerService() throws Exception {
//
long waitTime = dnConf.heartBeatInterval -
(Time.now() - lastHeartbeat);
synchronized(pendingIncrementalBR) {
synchronized(pendingIncrementalBRperStorage) {
if (waitTime > 0 && pendingReceivedRequests == 0) {
try {
pendingIncrementalBR.wait(waitTime);
pendingIncrementalBRperStorage.wait(waitTime);
} catch (InterruptedException ie) {
LOG.warn("BPOfferService for " + this + " interrupted");
}
@ -756,4 +775,52 @@ void reRegister() throws IOException {
}
}
private static class PerStoragePendingIncrementalBR {
private Map<Long, ReceivedDeletedBlockInfo> pendingIncrementalBR =
Maps.newHashMap();
/**
* Return the number of blocks on this storage that have pending
* incremental block reports.
* @return
*/
int getBlockInfoCount() {
return pendingIncrementalBR.size();
}
/**
* Dequeue and return all pending incremental block report state.
* @return
*/
ReceivedDeletedBlockInfo[] dequeueBlockInfos() {
ReceivedDeletedBlockInfo[] blockInfos =
pendingIncrementalBR.values().toArray(
new ReceivedDeletedBlockInfo[getBlockInfoCount()]);
pendingIncrementalBR.clear();
return blockInfos;
}
/**
* Add blocks from blockArray to pendingIncrementalBR, unless the
* block already exists in pendingIncrementalBR.
* @param blockArray list of blocks to add.
*/
void putMissingBlockInfos(ReceivedDeletedBlockInfo[] blockArray) {
for (ReceivedDeletedBlockInfo rdbi : blockArray) {
if (!pendingIncrementalBR.containsKey(rdbi.getBlock().getBlockId())) {
pendingIncrementalBR.put(rdbi.getBlock().getBlockId(), rdbi);
}
}
}
/**
* Add pending incremental block report for a single block.
* @param blockID
* @param blockInfo
*/
void putBlockInfo(ReceivedDeletedBlockInfo blockInfo) {
pendingIncrementalBR.put(blockInfo.getBlock().getBlockId(), blockInfo);
}
}
}

View File

@ -162,7 +162,8 @@ class BlockReceiver implements Closeable {
switch (stage) {
case PIPELINE_SETUP_CREATE:
replicaInfo = datanode.data.createRbw(block);
datanode.notifyNamenodeReceivingBlock(block);
datanode.notifyNamenodeReceivingBlock(
block, replicaInfo.getStorageUuid());
break;
case PIPELINE_SETUP_STREAMING_RECOVERY:
replicaInfo = datanode.data.recoverRbw(
@ -176,7 +177,8 @@ class BlockReceiver implements Closeable {
block.getLocalBlock());
}
block.setGenerationStamp(newGs);
datanode.notifyNamenodeReceivingBlock(block);
datanode.notifyNamenodeReceivingBlock(
block, replicaInfo.getStorageUuid());
break;
case PIPELINE_SETUP_APPEND_RECOVERY:
replicaInfo = datanode.data.recoverAppend(block, newGs, minBytesRcvd);
@ -185,7 +187,8 @@ class BlockReceiver implements Closeable {
block.getLocalBlock());
}
block.setGenerationStamp(newGs);
datanode.notifyNamenodeReceivingBlock(block);
datanode.notifyNamenodeReceivingBlock(
block, replicaInfo.getStorageUuid());
break;
case TRANSFER_RBW:
case TRANSFER_FINALIZED:
@ -252,6 +255,10 @@ class BlockReceiver implements Closeable {
/** Return the datanode object. */
DataNode getDataNode() {return datanode;}
public Replica getReplicaInfo() {
return replicaInfo;
}
/**
* close files.
*/
@ -1072,7 +1079,8 @@ private void finalizeBlock(long startTime) throws IOException {
: 0;
block.setNumBytes(replicaInfo.getNumBytes());
datanode.data.finalizeBlock(block);
datanode.closeBlock(block, DataNode.EMPTY_DEL_HINT);
datanode.closeBlock(
block, DataNode.EMPTY_DEL_HINT, replicaInfo.getStorageUuid());
if (ClientTraceLog.isInfoEnabled() && isClient) {
long offset = 0;
DatanodeRegistration dnR = datanode.getDNRegistrationForBP(block

View File

@ -520,10 +520,11 @@ static DomainPeerServer getDomainPeerServer(Configuration conf,
}
// calls specific to BP
protected void notifyNamenodeReceivedBlock(ExtendedBlock block, String delHint) {
protected void notifyNamenodeReceivedBlock(
ExtendedBlock block, String delHint, String storageUuid) {
BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
if(bpos != null) {
bpos.notifyNamenodeReceivedBlock(block, delHint);
bpos.notifyNamenodeReceivedBlock(block, delHint, storageUuid);
} else {
LOG.error("Cannot find BPOfferService for reporting block received for bpid="
+ block.getBlockPoolId());
@ -531,10 +532,11 @@ protected void notifyNamenodeReceivedBlock(ExtendedBlock block, String delHint)
}
// calls specific to BP
protected void notifyNamenodeReceivingBlock(ExtendedBlock block) {
protected void notifyNamenodeReceivingBlock(
ExtendedBlock block, String storageUuid) {
BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
if(bpos != null) {
bpos.notifyNamenodeReceivingBlock(block);
bpos.notifyNamenodeReceivingBlock(block, storageUuid);
} else {
LOG.error("Cannot find BPOfferService for reporting block receiving for bpid="
+ block.getBlockPoolId());
@ -542,10 +544,10 @@ protected void notifyNamenodeReceivingBlock(ExtendedBlock block) {
}
/** Notify the corresponding namenode to delete the block. */
public void notifyNamenodeDeletedBlock(ExtendedBlock block) {
public void notifyNamenodeDeletedBlock(ExtendedBlock block, String storageUuid) {
BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
if (bpos != null) {
bpos.notifyNamenodeDeletedBlock(block);
bpos.notifyNamenodeDeletedBlock(block, storageUuid);
} else {
LOG.error("Cannot find BPOfferService for reporting block deleted for bpid="
+ block.getBlockPoolId());
@ -1528,11 +1530,11 @@ public void run() {
* @param block
* @param delHint
*/
void closeBlock(ExtendedBlock block, String delHint) {
void closeBlock(ExtendedBlock block, String delHint, String storageUuid) {
metrics.incrBlocksWritten();
BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
if(bpos != null) {
bpos.notifyNamenodeReceivedBlock(block, delHint);
bpos.notifyNamenodeReceivedBlock(block, delHint, storageUuid);
} else {
LOG.warn("Cannot find BPOfferService for reporting block received for bpid="
+ block.getBlockPoolId());
@ -1892,7 +1894,7 @@ public String updateReplicaUnderRecovery(final ExtendedBlock oldBlock,
ExtendedBlock newBlock = new ExtendedBlock(oldBlock);
newBlock.setGenerationStamp(recoveryId);
newBlock.setNumBytes(newLength);
notifyNamenodeReceivedBlock(newBlock, "");
notifyNamenodeReceivedBlock(newBlock, "", storageID);
return storageID;
}

View File

@ -447,6 +447,7 @@ public void writeBlock(final ExtendedBlock block,
String mirrorNode = null; // the name:port of next target
String firstBadLink = ""; // first datanode that failed in connection setup
Status mirrorInStatus = SUCCESS;
Replica replica;
try {
if (isDatanode ||
stage != BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
@ -457,8 +458,10 @@ public void writeBlock(final ExtendedBlock block,
stage, latestGenerationStamp, minBytesRcvd, maxBytesRcvd,
clientname, srcDataNode, datanode, requestedChecksum,
cachingStrategy);
replica = blockReceiver.getReplicaInfo();
} else {
datanode.data.recoverClose(block, latestGenerationStamp, minBytesRcvd);
replica =
datanode.data.recoverClose(block, latestGenerationStamp, minBytesRcvd);
}
//
@ -590,7 +593,8 @@ public void writeBlock(final ExtendedBlock block,
// the block is finalized in the PacketResponder.
if (isDatanode ||
stage == BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
datanode.closeBlock(block, DataNode.EMPTY_DEL_HINT);
datanode.closeBlock(
block, DataNode.EMPTY_DEL_HINT, replica.getStorageUuid());
LOG.info("Received " + block + " src: " + remoteAddress + " dest: "
+ localAddress + " of size " + block.getNumBytes());
}
@ -859,7 +863,8 @@ public void replaceBlock(final ExtendedBlock block,
dataXceiverServer.balanceThrottler, null);
// notify name node
datanode.notifyNamenodeReceivedBlock(block, delHint);
datanode.notifyNamenodeReceivedBlock(
block, delHint, blockReceiver.getReplicaInfo().getStorageUuid());
LOG.info("Moved " + block + " from " + peer.getRemoteAddressString());

View File

@ -54,4 +54,9 @@ public interface Replica {
* @return the number of bytes that are visible to readers
*/
public long getVisibleLength();
/**
* Return the storageUuid of the volume that stores this replica.
*/
public String getStorageUuid();
}

View File

@ -137,6 +137,14 @@ public FsVolumeSpi getVolume() {
void setVolume(FsVolumeSpi vol) {
this.volume = vol;
}
/**
* Get the storageUuid of the volume that stores this replica.
*/
@Override
public String getStorageUuid() {
return volume.getStorageID();
}
/**
* Return the parent directory path where this replica is located

View File

@ -243,7 +243,7 @@ public ReplicaInPipelineInterface recoverAppend(ExtendedBlock b, long newGS,
* @param expectedBlockLen the number of bytes the replica is expected to have
* @throws IOException
*/
public void recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen
public Replica recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen
) throws IOException;
/**

View File

@ -26,6 +26,9 @@
* This is an interface for the underlying volume.
*/
public interface FsVolumeSpi {
/** @return the StorageUuid of the volume */
public String getStorageID();
/** @return a list of block pools. */
public String[] getBlockPoolList();

View File

@ -195,7 +195,7 @@ public void run() {
+ " at file " + blockFile + ". Ignored.");
} else {
if(block.getLocalBlock().getNumBytes() != BlockCommand.NO_ACK){
datanode.notifyNamenodeDeletedBlock(block);
datanode.notifyNamenodeDeletedBlock(block, volume.getStorageID());
}
volume.decDfsUsed(block.getBlockPoolId(), dfsBytes);
LOG.info("Deleted " + block.getBlockPoolId() + " "

View File

@ -699,7 +699,7 @@ public synchronized ReplicaInPipeline recoverAppend(ExtendedBlock b,
}
@Override // FsDatasetSpi
public void recoverClose(ExtendedBlock b, long newGS,
public Replica recoverClose(ExtendedBlock b, long newGS,
long expectedBlockLen) throws IOException {
LOG.info("Recover failed close " + b);
// check replica's state
@ -710,6 +710,7 @@ public void recoverClose(ExtendedBlock b, long newGS,
if (replicaInfo.getState() == ReplicaState.RBW) {
finalizeReplica(b.getBlockPoolId(), replicaInfo);
}
return replicaInfo;
}
/**

View File

@ -290,6 +290,7 @@ void deleteBPDirectories(String bpid, boolean force) throws IOException {
}
}
@Override
public String getStorageID() {
return storageID;
}

View File

@ -131,6 +131,11 @@ private class BInfo implements ReplicaInPipelineInterface {
}
}
@Override
public String getStorageUuid() {
return storage.getStorageUuid();
}
@Override
synchronized public long getGenerationStamp() {
return theBlock.getGenerationStamp();
@ -314,6 +319,8 @@ void free(long amount) {
private static class SimulatedStorage {
private Map<String, SimulatedBPStorage> map =
new HashMap<String, SimulatedBPStorage>();
private final String storageUuid = "SimulatedStorage-UUID";
private long capacity; // in bytes
synchronized long getFree() {
@ -375,6 +382,10 @@ private SimulatedBPStorage getBPStorage(String bpid) throws IOException {
}
return bpStorage;
}
public String getStorageUuid() {
return storageUuid;
}
}
private final Map<String, Map<Block, BInfo>> blockMap
@ -625,7 +636,7 @@ public synchronized ReplicaInPipelineInterface recoverAppend(ExtendedBlock b,
}
@Override // FsDatasetSpi
public void recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen)
public Replica recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen)
throws IOException {
final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
BInfo binfo = map.get(b.getLocalBlock());
@ -639,6 +650,7 @@ public void recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen)
map.remove(b.getLocalBlock());
binfo.theBlock.setGenerationStamp(newGS);
map.put(binfo.theBlock, binfo);
return binfo;
}
@Override // FsDatasetSpi

View File

@ -176,7 +176,7 @@ public void testBasicFunctionality() throws Exception {
waitForBlockReport(mockNN2);
// When we receive a block, it should report it to both NNs
bpos.notifyNamenodeReceivedBlock(FAKE_BLOCK, "");
bpos.notifyNamenodeReceivedBlock(FAKE_BLOCK, "", "");
ReceivedDeletedBlockInfo[] ret = waitForBlockReceived(FAKE_BLOCK, mockNN1);
assertEquals(1, ret.length);

View File

@ -412,6 +412,11 @@ public File getFinalizedDir(String bpid) throws IOException {
public StorageType getStorageType() {
return StorageType.DEFAULT;
}
@Override
public String getStorageID() {
return "";
}
}
private final static TestFsVolumeSpi TEST_VOLUME = new TestFsVolumeSpi();