|
@@ -31,7 +31,6 @@ import java.util.Map;
|
|
import java.util.concurrent.ThreadLocalRandom;
|
|
import java.util.concurrent.ThreadLocalRandom;
|
|
import java.util.concurrent.atomic.AtomicBoolean;
|
|
import java.util.concurrent.atomic.AtomicBoolean;
|
|
|
|
|
|
-import com.google.common.base.Joiner;
|
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
|
|
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
|
|
import org.apache.hadoop.hdfs.client.BlockReportOptions;
|
|
import org.apache.hadoop.hdfs.client.BlockReportOptions;
|
|
@@ -51,9 +50,7 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
|
|
import org.apache.hadoop.hdfs.server.protocol.DisallowedDatanodeException;
|
|
import org.apache.hadoop.hdfs.server.protocol.DisallowedDatanodeException;
|
|
import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
|
|
import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
|
|
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
|
|
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
|
|
-import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
|
|
|
|
import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
|
|
import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
|
|
-import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
|
|
|
|
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
|
|
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
|
|
import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
|
|
import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
|
|
import org.apache.hadoop.io.IOUtils;
|
|
import org.apache.hadoop.io.IOUtils;
|
|
@@ -61,10 +58,10 @@ import org.apache.hadoop.ipc.RemoteException;
|
|
import org.apache.hadoop.util.Time;
|
|
import org.apache.hadoop.util.Time;
|
|
import org.apache.hadoop.util.VersionInfo;
|
|
import org.apache.hadoop.util.VersionInfo;
|
|
import org.apache.hadoop.util.VersionUtil;
|
|
import org.apache.hadoop.util.VersionUtil;
|
|
|
|
+import org.slf4j.Logger;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
-import com.google.common.collect.Maps;
|
|
|
|
-import org.slf4j.Logger;
|
|
|
|
|
|
+import com.google.common.base.Joiner;
|
|
|
|
|
|
/**
|
|
/**
|
|
* A thread per active or standby namenode to perform:
|
|
* A thread per active or standby namenode to perform:
|
|
@@ -95,25 +92,14 @@ class BPServiceActor implements Runnable {
|
|
}
|
|
}
|
|
|
|
|
|
private volatile RunningState runningState = RunningState.CONNECTING;
|
|
private volatile RunningState runningState = RunningState.CONNECTING;
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Between block reports (which happen on the order of once an hour) the
|
|
|
|
- * DN reports smaller incremental changes to its block list. This map,
|
|
|
|
- * 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<DatanodeStorage, PerStoragePendingIncrementalBR>
|
|
|
|
- pendingIncrementalBRperStorage = Maps.newHashMap();
|
|
|
|
-
|
|
|
|
- // IBR = Incremental Block Report. If this flag is set then an IBR will be
|
|
|
|
- // sent immediately by the actor thread without waiting for the IBR timer
|
|
|
|
- // to elapse.
|
|
|
|
- private volatile boolean sendImmediateIBR = false;
|
|
|
|
private volatile boolean shouldServiceRun = true;
|
|
private volatile boolean shouldServiceRun = true;
|
|
private final DataNode dn;
|
|
private final DataNode dn;
|
|
private final DNConf dnConf;
|
|
private final DNConf dnConf;
|
|
private long prevBlockReportId;
|
|
private long prevBlockReportId;
|
|
|
|
|
|
|
|
+ private final IncrementalBlockReportManager ibrManager
|
|
|
|
+ = new IncrementalBlockReportManager();
|
|
|
|
+
|
|
private DatanodeRegistration bpRegistration;
|
|
private DatanodeRegistration bpRegistration;
|
|
final LinkedList<BPServiceActorAction> bpThreadQueue
|
|
final LinkedList<BPServiceActorAction> bpThreadQueue
|
|
= new LinkedList<BPServiceActorAction>();
|
|
= new LinkedList<BPServiceActorAction>();
|
|
@@ -131,6 +117,10 @@ class BPServiceActor implements Runnable {
|
|
return bpRegistration;
|
|
return bpRegistration;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ IncrementalBlockReportManager getIbrManager() {
|
|
|
|
+ return ibrManager;
|
|
|
|
+ }
|
|
|
|
+
|
|
boolean isAlive() {
|
|
boolean isAlive() {
|
|
if (!shouldServiceRun || !bpThread.isAlive()) {
|
|
if (!shouldServiceRun || !bpThread.isAlive()) {
|
|
return false;
|
|
return false;
|
|
@@ -231,141 +221,20 @@ class BPServiceActor implements Runnable {
|
|
register(nsInfo);
|
|
register(nsInfo);
|
|
}
|
|
}
|
|
|
|
|
|
- /**
|
|
|
|
- * Report received blocks and delete hints to the Namenode for each
|
|
|
|
- * storage.
|
|
|
|
- *
|
|
|
|
- * @throws IOException
|
|
|
|
- */
|
|
|
|
- private void reportReceivedDeletedBlocks() throws IOException {
|
|
|
|
-
|
|
|
|
- // Generate a list of the pending reports for each storage under the lock
|
|
|
|
- ArrayList<StorageReceivedDeletedBlocks> reports =
|
|
|
|
- new ArrayList<StorageReceivedDeletedBlocks>(pendingIncrementalBRperStorage.size());
|
|
|
|
- synchronized (pendingIncrementalBRperStorage) {
|
|
|
|
- for (Map.Entry<DatanodeStorage, PerStoragePendingIncrementalBR> entry :
|
|
|
|
- pendingIncrementalBRperStorage.entrySet()) {
|
|
|
|
- final DatanodeStorage storage = entry.getKey();
|
|
|
|
- final PerStoragePendingIncrementalBR perStorageMap = entry.getValue();
|
|
|
|
-
|
|
|
|
- if (perStorageMap.getBlockInfoCount() > 0) {
|
|
|
|
- // Send newly-received and deleted blockids to namenode
|
|
|
|
- ReceivedDeletedBlockInfo[] rdbi = perStorageMap.dequeueBlockInfos();
|
|
|
|
- reports.add(new StorageReceivedDeletedBlocks(storage, rdbi));
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- sendImmediateIBR = false;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- if (reports.size() == 0) {
|
|
|
|
- // Nothing new to report.
|
|
|
|
- return;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- // Send incremental block reports to the Namenode outside the lock
|
|
|
|
- boolean success = false;
|
|
|
|
- final long startTime = monotonicNow();
|
|
|
|
- try {
|
|
|
|
- bpNamenode.blockReceivedAndDeleted(bpRegistration,
|
|
|
|
- bpos.getBlockPoolId(),
|
|
|
|
- reports.toArray(new StorageReceivedDeletedBlocks[reports.size()]));
|
|
|
|
- success = true;
|
|
|
|
- } finally {
|
|
|
|
- dn.getMetrics().addIncrementalBlockReport(monotonicNow() - startTime);
|
|
|
|
- if (!success) {
|
|
|
|
- synchronized (pendingIncrementalBRperStorage) {
|
|
|
|
- for (StorageReceivedDeletedBlocks report : reports) {
|
|
|
|
- // 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.
|
|
|
|
- PerStoragePendingIncrementalBR perStorageMap =
|
|
|
|
- pendingIncrementalBRperStorage.get(report.getStorage());
|
|
|
|
- perStorageMap.putMissingBlockInfos(report.getBlocks());
|
|
|
|
- sendImmediateIBR = true;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * @return pending incremental block report for given {@code storage}
|
|
|
|
- */
|
|
|
|
- private PerStoragePendingIncrementalBR getIncrementalBRMapForStorage(
|
|
|
|
- DatanodeStorage storage) {
|
|
|
|
- PerStoragePendingIncrementalBR mapForStorage =
|
|
|
|
- pendingIncrementalBRperStorage.get(storage);
|
|
|
|
-
|
|
|
|
- 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(storage, mapForStorage);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- return mapForStorage;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Add a blockInfo for notification to NameNode. If another entry
|
|
|
|
- * exists for the same block it is removed.
|
|
|
|
- *
|
|
|
|
- * Caller must synchronize access using pendingIncrementalBRperStorage.
|
|
|
|
- */
|
|
|
|
- void addPendingReplicationBlockInfo(ReceivedDeletedBlockInfo bInfo,
|
|
|
|
- DatanodeStorage storage) {
|
|
|
|
- // Make sure another entry for the same block is first removed.
|
|
|
|
- // There may only be one such entry.
|
|
|
|
- for (Map.Entry<DatanodeStorage, PerStoragePendingIncrementalBR> entry :
|
|
|
|
- pendingIncrementalBRperStorage.entrySet()) {
|
|
|
|
- if (entry.getValue().removeBlockInfo(bInfo)) {
|
|
|
|
- break;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- getIncrementalBRMapForStorage(storage).putBlockInfo(bInfo);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /*
|
|
|
|
- * 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 notifyNamenodeBlock(ReceivedDeletedBlockInfo bInfo,
|
|
|
|
- String storageUuid, boolean now) {
|
|
|
|
- synchronized (pendingIncrementalBRperStorage) {
|
|
|
|
- addPendingReplicationBlockInfo(
|
|
|
|
- bInfo, dn.getFSDataset().getStorage(storageUuid));
|
|
|
|
- sendImmediateIBR = true;
|
|
|
|
- // If now is true, the report is sent right away.
|
|
|
|
- // Otherwise, it will be sent out in the next heartbeat.
|
|
|
|
- if (now) {
|
|
|
|
- pendingIncrementalBRperStorage.notifyAll();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- void notifyNamenodeDeletedBlock(
|
|
|
|
- ReceivedDeletedBlockInfo bInfo, String storageUuid) {
|
|
|
|
- synchronized (pendingIncrementalBRperStorage) {
|
|
|
|
- addPendingReplicationBlockInfo(
|
|
|
|
- bInfo, dn.getFSDataset().getStorage(storageUuid));
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
|
|
|
|
/**
|
|
/**
|
|
* Run an immediate block report on this thread. Used by tests.
|
|
* Run an immediate block report on this thread. Used by tests.
|
|
*/
|
|
*/
|
|
@VisibleForTesting
|
|
@VisibleForTesting
|
|
void triggerBlockReportForTests() {
|
|
void triggerBlockReportForTests() {
|
|
- synchronized (pendingIncrementalBRperStorage) {
|
|
|
|
|
|
+ synchronized (ibrManager) {
|
|
scheduler.scheduleHeartbeat();
|
|
scheduler.scheduleHeartbeat();
|
|
long oldBlockReportTime = scheduler.nextBlockReportTime;
|
|
long oldBlockReportTime = scheduler.nextBlockReportTime;
|
|
scheduler.forceFullBlockReportNow();
|
|
scheduler.forceFullBlockReportNow();
|
|
- pendingIncrementalBRperStorage.notifyAll();
|
|
|
|
|
|
+ ibrManager.notifyAll();
|
|
while (oldBlockReportTime == scheduler.nextBlockReportTime) {
|
|
while (oldBlockReportTime == scheduler.nextBlockReportTime) {
|
|
try {
|
|
try {
|
|
- pendingIncrementalBRperStorage.wait(100);
|
|
|
|
|
|
+ ibrManager.wait(100);
|
|
} catch (InterruptedException e) {
|
|
} catch (InterruptedException e) {
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
@@ -375,28 +244,12 @@ class BPServiceActor implements Runnable {
|
|
|
|
|
|
@VisibleForTesting
|
|
@VisibleForTesting
|
|
void triggerHeartbeatForTests() {
|
|
void triggerHeartbeatForTests() {
|
|
- synchronized (pendingIncrementalBRperStorage) {
|
|
|
|
|
|
+ synchronized (ibrManager) {
|
|
final long nextHeartbeatTime = scheduler.scheduleHeartbeat();
|
|
final long nextHeartbeatTime = scheduler.scheduleHeartbeat();
|
|
- pendingIncrementalBRperStorage.notifyAll();
|
|
|
|
|
|
+ ibrManager.notifyAll();
|
|
while (nextHeartbeatTime - scheduler.nextHeartbeatTime >= 0) {
|
|
while (nextHeartbeatTime - scheduler.nextHeartbeatTime >= 0) {
|
|
try {
|
|
try {
|
|
- pendingIncrementalBRperStorage.wait(100);
|
|
|
|
- } catch (InterruptedException e) {
|
|
|
|
- return;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @VisibleForTesting
|
|
|
|
- void triggerDeletionReportForTests() {
|
|
|
|
- synchronized (pendingIncrementalBRperStorage) {
|
|
|
|
- sendImmediateIBR = true;
|
|
|
|
- pendingIncrementalBRperStorage.notifyAll();
|
|
|
|
-
|
|
|
|
- while (sendImmediateIBR) {
|
|
|
|
- try {
|
|
|
|
- pendingIncrementalBRperStorage.wait(100);
|
|
|
|
|
|
+ ibrManager.wait(100);
|
|
} catch (InterruptedException e) {
|
|
} catch (InterruptedException e) {
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
@@ -404,11 +257,6 @@ class BPServiceActor implements Runnable {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- @VisibleForTesting
|
|
|
|
- boolean hasPendingIBR() {
|
|
|
|
- return sendImmediateIBR;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
private long generateUniqueBlockReportId() {
|
|
private long generateUniqueBlockReportId() {
|
|
// Initialize the block report ID the first time through.
|
|
// Initialize the block report ID the first time through.
|
|
// Note that 0 is used on the NN to indicate "uninitialized", so we should
|
|
// Note that 0 is used on the NN to indicate "uninitialized", so we should
|
|
@@ -432,7 +280,8 @@ class BPServiceActor implements Runnable {
|
|
// we have a chance that we will miss the delHint information
|
|
// we have a chance that we will miss the delHint information
|
|
// or we will report an RBW replica after the BlockReport already reports
|
|
// or we will report an RBW replica after the BlockReport already reports
|
|
// a FINALIZED one.
|
|
// a FINALIZED one.
|
|
- reportReceivedDeletedBlocks();
|
|
|
|
|
|
+ ibrManager.sendIBRs(bpNamenode, bpRegistration,
|
|
|
|
+ bpos.getBlockPoolId(), dn.getMetrics());
|
|
|
|
|
|
long brCreateStartTime = monotonicNow();
|
|
long brCreateStartTime = monotonicNow();
|
|
Map<DatanodeStorage, BlockListAsLongs> perVolumeBlockLists =
|
|
Map<DatanodeStorage, BlockListAsLongs> perVolumeBlockLists =
|
|
@@ -697,8 +546,9 @@ class BPServiceActor implements Runnable {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
- if (sendImmediateIBR || sendHeartbeat) {
|
|
|
|
- reportReceivedDeletedBlocks();
|
|
|
|
|
|
+ if (ibrManager.sendImmediately() || sendHeartbeat) {
|
|
|
|
+ ibrManager.sendIBRs(bpNamenode, bpRegistration,
|
|
|
|
+ bpos.getBlockPoolId(), dn.getMetrics());
|
|
}
|
|
}
|
|
|
|
|
|
List<DatanodeCommand> cmds = null;
|
|
List<DatanodeCommand> cmds = null;
|
|
@@ -723,10 +573,10 @@ class BPServiceActor implements Runnable {
|
|
// or work arrives, and then iterate again.
|
|
// or work arrives, and then iterate again.
|
|
//
|
|
//
|
|
long waitTime = scheduler.getHeartbeatWaitTime();
|
|
long waitTime = scheduler.getHeartbeatWaitTime();
|
|
- synchronized(pendingIncrementalBRperStorage) {
|
|
|
|
- if (waitTime > 0 && !sendImmediateIBR) {
|
|
|
|
|
|
+ synchronized(ibrManager) {
|
|
|
|
+ if (waitTime > 0 && !ibrManager.sendImmediately()) {
|
|
try {
|
|
try {
|
|
- pendingIncrementalBRperStorage.wait(waitTime);
|
|
|
|
|
|
+ ibrManager.wait(waitTime);
|
|
} catch (InterruptedException ie) {
|
|
} catch (InterruptedException ie) {
|
|
LOG.warn("BPOfferService for " + this + " interrupted");
|
|
LOG.warn("BPOfferService for " + this + " interrupted");
|
|
}
|
|
}
|
|
@@ -915,82 +765,15 @@ class BPServiceActor implements Runnable {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- private static class PerStoragePendingIncrementalBR {
|
|
|
|
- private final 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.
|
|
|
|
- * @return the number of missing blocks that we added.
|
|
|
|
- */
|
|
|
|
- int putMissingBlockInfos(ReceivedDeletedBlockInfo[] blockArray) {
|
|
|
|
- int blocksPut = 0;
|
|
|
|
- for (ReceivedDeletedBlockInfo rdbi : blockArray) {
|
|
|
|
- if (!pendingIncrementalBR.containsKey(rdbi.getBlock().getBlockId())) {
|
|
|
|
- pendingIncrementalBR.put(rdbi.getBlock().getBlockId(), rdbi);
|
|
|
|
- ++blocksPut;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- return blocksPut;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Add pending incremental block report for a single block.
|
|
|
|
- * @param blockInfo
|
|
|
|
- */
|
|
|
|
- void putBlockInfo(ReceivedDeletedBlockInfo blockInfo) {
|
|
|
|
- pendingIncrementalBR.put(blockInfo.getBlock().getBlockId(), blockInfo);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Remove pending incremental block report for a single block if it
|
|
|
|
- * exists.
|
|
|
|
- *
|
|
|
|
- * @param blockInfo
|
|
|
|
- * @return true if a report was removed, false if no report existed for
|
|
|
|
- * the given block.
|
|
|
|
- */
|
|
|
|
- boolean removeBlockInfo(ReceivedDeletedBlockInfo blockInfo) {
|
|
|
|
- return (pendingIncrementalBR.remove(blockInfo.getBlock().getBlockId()) != null);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- void triggerBlockReport(BlockReportOptions options) throws IOException {
|
|
|
|
|
|
+ void triggerBlockReport(BlockReportOptions options) {
|
|
if (options.isIncremental()) {
|
|
if (options.isIncremental()) {
|
|
LOG.info(bpos.toString() + ": scheduling an incremental block report.");
|
|
LOG.info(bpos.toString() + ": scheduling an incremental block report.");
|
|
- synchronized(pendingIncrementalBRperStorage) {
|
|
|
|
- sendImmediateIBR = true;
|
|
|
|
- pendingIncrementalBRperStorage.notifyAll();
|
|
|
|
- }
|
|
|
|
|
|
+ ibrManager.triggerIBR();
|
|
} else {
|
|
} else {
|
|
LOG.info(bpos.toString() + ": scheduling a full block report.");
|
|
LOG.info(bpos.toString() + ": scheduling a full block report.");
|
|
- synchronized(pendingIncrementalBRperStorage) {
|
|
|
|
|
|
+ synchronized(ibrManager) {
|
|
scheduler.forceFullBlockReportNow();
|
|
scheduler.forceFullBlockReportNow();
|
|
- pendingIncrementalBRperStorage.notifyAll();
|
|
|
|
|
|
+ ibrManager.notifyAll();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|