|
@@ -23,7 +23,6 @@ import java.io.IOException;
|
|
|
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 @@ class BPServiceActor implements Runnable {
|
|
|
* @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 @@ class BPServiceActor implements Runnable {
|
|
|
*/
|
|
|
@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 @@ class BPServiceActor implements Runnable {
|
|
|
|
|
|
@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 @@ class BPServiceActor implements Runnable {
|
|
|
|
|
|
@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 @@ class BPServiceActor implements Runnable {
|
|
|
//
|
|
|
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 @@ class BPServiceActor implements Runnable {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ 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);
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|