HDFS-9710. DN can be configured to send block receipt IBRs in batches. Contributed by Tsz-Wo Nicholas Sze.

Backport HDFS-11837 by Vinitha Reddy Gankidi

(cherry picked from commit 5b95971f8a)
This commit is contained in:
Konstantin V Shvachko 2017-05-24 11:38:53 -07:00
parent f99a48aa63
commit ac1b8ff78f
17 changed files with 123 additions and 73 deletions

View File

@ -111,6 +111,9 @@ Release 2.7.4 - UNRELEASED
HDFS-9412. getBlocks occupies FSLock and takes too long to complete.
Contributed by He Tianyi. Backport HDFS-11855 by Vinitha Reddy Gankidi.
HDFS-9710. DN can be configured to send block receipt IBRs in batches.
(Tsz-Wo Nicholas Sze. Backport HDFS-11837 by Vinitha Reddy Gankidi)
BUG FIXES
HDFS-8307. Spurious DNS Queries from hdfs shell. (Andres Perez via aengineer)

View File

@ -554,6 +554,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final String DFS_DF_INTERVAL_KEY = "dfs.df.interval";
public static final int DFS_DF_INTERVAL_DEFAULT = 60000;
public static final String DFS_BLOCKREPORT_INCREMENTAL_INTERVAL_MSEC_KEY
= "dfs.blockreport.incremental.intervalMsec";
public static final long DFS_BLOCKREPORT_INCREMENTAL_INTERVAL_MSEC_DEFAULT
= 0;
public static final String DFS_BLOCKREPORT_INTERVAL_MSEC_KEY = "dfs.blockreport.intervalMsec";
public static final long DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT = 6 * 60 * 60 * 1000;
public static final String DFS_BLOCKREPORT_INITIAL_DELAY_KEY = "dfs.blockreport.initialDelay";

View File

@ -230,29 +230,32 @@ class BPOfferService {
* till namenode is informed before responding with success to the
* client? For now we don't.
*/
void notifyNamenodeReceivedBlock(
ExtendedBlock block, String delHint, String storageUuid) {
void notifyNamenodeReceivedBlock(ExtendedBlock block, String delHint,
String storageUuid, boolean isOnTransientStorage) {
notifyNamenodeBlock(block, BlockStatus.RECEIVED_BLOCK, delHint,
storageUuid);
storageUuid, isOnTransientStorage);
}
void notifyNamenodeReceivingBlock(ExtendedBlock block, String storageUuid) {
notifyNamenodeBlock(block, BlockStatus.RECEIVING_BLOCK, null, storageUuid);
notifyNamenodeBlock(block, BlockStatus.RECEIVING_BLOCK, null, storageUuid,
false);
}
void notifyNamenodeDeletedBlock(ExtendedBlock block, String storageUuid) {
notifyNamenodeBlock(block, BlockStatus.DELETED_BLOCK, null, storageUuid);
notifyNamenodeBlock(block, BlockStatus.DELETED_BLOCK, null, storageUuid,
false);
}
private void notifyNamenodeBlock(ExtendedBlock block, BlockStatus status,
String delHint, String storageUuid) {
String delHint, String storageUuid, boolean isOnTransientStorage) {
checkBlock(block);
final ReceivedDeletedBlockInfo info = new ReceivedDeletedBlockInfo(
block.getLocalBlock(), status, delHint);
final DatanodeStorage storage = dn.getFSDataset().getStorage(storageUuid);
for (BPServiceActor actor : bpServices) {
actor.getIbrManager().notifyNamenodeBlock(info, storage);
actor.getIbrManager().notifyNamenodeBlock(info, storage,
isOnTransientStorage);
}
}

View File

@ -96,8 +96,7 @@ class BPServiceActor implements Runnable {
private final DNConf dnConf;
private long prevBlockReportId = 0;
private final IncrementalBlockReportManager ibrManager
= new IncrementalBlockReportManager();
private final IncrementalBlockReportManager ibrManager;
private DatanodeRegistration bpRegistration;
final LinkedList<BPServiceActorAction> bpThreadQueue
@ -108,6 +107,7 @@ class BPServiceActor implements Runnable {
this.dn = bpos.getDataNode();
this.nnAddr = nnAddr;
this.dnConf = dn.getDnConf();
this.ibrManager = new IncrementalBlockReportManager(dnConf.ibrInterval);
scheduler = new Scheduler(dnConf.heartBeatInterval, dnConf.blockReportInterval);
}
@ -537,20 +537,9 @@ class BPServiceActor implements Runnable {
DatanodeCommand cmd = cacheReport();
processCommand(new DatanodeCommand[]{ cmd });
//
// There is no work to do; sleep until hearbeat timer elapses,
// or work arrives, and then iterate again.
//
long waitTime = scheduler.getHeartbeatWaitTime();
synchronized(ibrManager) {
if (waitTime > 0 && !ibrManager.sendImmediately()) {
try {
ibrManager.wait(waitTime);
} catch (InterruptedException ie) {
LOG.warn("BPOfferService for " + this + " interrupted");
}
}
} // synchronized
ibrManager.waitTillNextIBR(scheduler.getHeartbeatWaitTime());
} catch(RemoteException re) {
String reClass = re.getClassName();
if (UnregisteredNodeException.class.getName().equals(reClass) ||
@ -742,7 +731,7 @@ class BPServiceActor implements Runnable {
void triggerBlockReport(BlockReportOptions options) {
if (options.isIncremental()) {
LOG.info(bpos.toString() + ": scheduling an incremental block report.");
ibrManager.triggerIBR();
ibrManager.triggerIBR(true);
} else {
LOG.info(bpos.toString() + ": scheduling a full block report.");
synchronized(ibrManager) {

View File

@ -286,8 +286,8 @@ class BlockReceiver implements Closeable {
/** Return the datanode object. */
DataNode getDataNode() {return datanode;}
String getStorageUuid() {
return replicaInfo.getStorageUuid();
Replica getReplica() {
return replicaInfo;
}
/**
@ -1425,8 +1425,8 @@ class BlockReceiver implements Closeable {
datanode.data.setPinning(block);
}
datanode.closeBlock(
block, DataNode.EMPTY_DEL_HINT, replicaInfo.getStorageUuid());
datanode.closeBlock(block, null, replicaInfo.getStorageUuid(),
replicaInfo.isOnTransientStorage());
if (ClientTraceLog.isInfoEnabled() && isClient) {
long offset = 0;
DatanodeRegistration dnR = datanode.getDNRegistrationForBP(block

View File

@ -85,6 +85,7 @@ public class DNConf {
final long blockReportInterval;
final long blockReportSplitThreshold;
final long initialBlockReportDelay;
final long ibrInterval;
final long cacheReportInterval;
final long dfsclientSlowIoWarningThresholdMs;
final long datanodeSlowIoWarningThresholdMs;
@ -142,6 +143,9 @@ public class DNConf {
DFSConfigKeys.DFS_DATANODE_USE_DN_HOSTNAME_DEFAULT);
this.blockReportInterval = conf.getLong(DFS_BLOCKREPORT_INTERVAL_MSEC_KEY,
DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT);
this.ibrInterval = conf.getLong(
DFSConfigKeys.DFS_BLOCKREPORT_INCREMENTAL_INTERVAL_MSEC_KEY,
DFSConfigKeys.DFS_BLOCKREPORT_INCREMENTAL_INTERVAL_MSEC_DEFAULT);
this.blockReportSplitThreshold = conf.getLong(DFS_BLOCKREPORT_SPLIT_THRESHOLD_KEY,
DFS_BLOCKREPORT_SPLIT_THRESHOLD_DEFAULT);
this.cacheReportInterval = conf.getLong(DFS_CACHEREPORT_INTERVAL_MSEC_KEY,

View File

@ -289,7 +289,6 @@ public class DataNode extends ReconfigurableBase
volatile FsDatasetSpi<? extends FsVolumeSpi> data = null;
private String clusterId = null;
public final static String EMPTY_DEL_HINT = "";
final AtomicInteger xmitsInProgress = new AtomicInteger();
Daemon dataXceiverServer = null;
DataXceiverServer xserver = null;
@ -960,11 +959,12 @@ public class DataNode extends ReconfigurableBase
}
// calls specific to BP
public void notifyNamenodeReceivedBlock(
ExtendedBlock block, String delHint, String storageUuid) {
public void notifyNamenodeReceivedBlock(ExtendedBlock block, String delHint,
String storageUuid, boolean isOnTransientStorage) {
BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
if(bpos != null) {
bpos.notifyNamenodeReceivedBlock(block, delHint, storageUuid);
bpos.notifyNamenodeReceivedBlock(block, delHint, storageUuid,
isOnTransientStorage);
} else {
LOG.error("Cannot find BPOfferService for reporting block received for bpid="
+ block.getBlockPoolId());
@ -2227,15 +2227,11 @@ public class DataNode extends ReconfigurableBase
* @param delHint hint on which excess block to delete
* @param storageUuid UUID of the storage where block is stored
*/
void closeBlock(ExtendedBlock block, String delHint, String storageUuid) {
void closeBlock(ExtendedBlock block, String delHint, String storageUuid,
boolean isTransientStorage) {
metrics.incrBlocksWritten();
BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
if(bpos != null) {
bpos.notifyNamenodeReceivedBlock(block, delHint, storageUuid);
} else {
LOG.warn("Cannot find BPOfferService for reporting block received for bpid="
+ block.getBlockPoolId());
}
notifyNamenodeReceivedBlock(block, delHint, storageUuid,
isTransientStorage);
}
/** Start a single datanode daemon and wait for it to finish.
@ -2602,7 +2598,7 @@ public class DataNode extends ReconfigurableBase
public String updateReplicaUnderRecovery(final ExtendedBlock oldBlock,
final long recoveryId, final long newBlockId, final long newLength)
throws IOException {
final String storageID = data.updateReplicaUnderRecovery(oldBlock,
final Replica r = data.updateReplicaUnderRecovery(oldBlock,
recoveryId, newBlockId, newLength);
// Notify the namenode of the updated block info. This is important
// for HA, since otherwise the standby node may lose track of the
@ -2611,7 +2607,9 @@ public class DataNode extends ReconfigurableBase
newBlock.setGenerationStamp(recoveryId);
newBlock.setBlockId(newBlockId);
newBlock.setNumBytes(newLength);
notifyNamenodeReceivedBlock(newBlock, "", storageID);
final String storageID = r.getStorageUuid();
notifyNamenodeReceivedBlock(newBlock, null, storageID,
r.isOnTransientStorage());
return storageID;
}

View File

@ -667,7 +667,9 @@ class DataXceiver extends Receiver implements Runnable {
String firstBadLink = ""; // first datanode that failed in connection setup
Status mirrorInStatus = SUCCESS;
final String storageUuid;
final boolean isOnTransientStorage;
try {
final Replica replica;
if (isDatanode ||
stage != BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
// open a block receiver
@ -677,12 +679,13 @@ class DataXceiver extends Receiver implements Runnable {
stage, latestGenerationStamp, minBytesRcvd, maxBytesRcvd,
clientname, srcDataNode, datanode, requestedChecksum,
cachingStrategy, allowLazyPersist, pinning);
storageUuid = blockReceiver.getStorageUuid();
replica = blockReceiver.getReplica();
} else {
storageUuid = datanode.data.recoverClose(
replica = datanode.data.recoverClose(
block, latestGenerationStamp, minBytesRcvd);
}
storageUuid = replica.getStorageUuid();
isOnTransientStorage = replica.isOnTransientStorage();
//
// Connect to downstream machine, if appropriate
@ -823,7 +826,7 @@ class DataXceiver extends Receiver implements Runnable {
// the block is finalized in the PacketResponder.
if (isDatanode ||
stage == BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
datanode.closeBlock(block, DataNode.EMPTY_DEL_HINT, storageUuid);
datanode.closeBlock(block, null, storageUuid, isOnTransientStorage);
LOG.info("Received " + block + " src: " + remoteAddress + " dest: "
+ localAddress + " of size " + block.getNumBytes());
}
@ -1137,8 +1140,9 @@ class DataXceiver extends Receiver implements Runnable {
dataXceiverServer.balanceThrottler, null, true);
// notify name node
final Replica r = blockReceiver.getReplica();
datanode.notifyNamenodeReceivedBlock(
block, delHint, blockReceiver.getStorageUuid());
block, delHint, r.getStorageUuid(), r.isOnTransientStorage());
LOG.info("Moved " + block + " from " + peer.getRemoteAddressString()
+ ", delHint=" + delHint);

View File

@ -21,6 +21,7 @@ import static org.apache.hadoop.util.Time.monotonicNow;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
@ -33,6 +34,8 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus;
import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Maps;
@ -42,6 +45,9 @@ import com.google.common.collect.Maps;
*/
@InterfaceAudience.Private
class IncrementalBlockReportManager {
private static final Logger LOG = LoggerFactory.getLogger(
IncrementalBlockReportManager.class);
private static class PerStorageIBR {
/** The blocks in this IBR. */
final Map<Block, ReceivedDeletedBlockInfo> blocks = Maps.newHashMap();
@ -103,8 +109,29 @@ class IncrementalBlockReportManager {
*/
private volatile boolean readyToSend = false;
/** The time interval between two IBRs. */
private final long ibrInterval;
/** The timestamp of the last IBR. */
private volatile long lastIBR;
IncrementalBlockReportManager(final long ibrInterval) {
this.ibrInterval = ibrInterval;
this.lastIBR = monotonicNow() - ibrInterval;
}
boolean sendImmediately() {
return readyToSend;
return readyToSend && monotonicNow() - ibrInterval >= lastIBR;
}
synchronized void waitTillNextIBR(long waitTime) {
if (waitTime > 0 && !sendImmediately()) {
try {
wait(ibrInterval > 0 && ibrInterval < waitTime? ibrInterval: waitTime);
} catch (InterruptedException ie) {
LOG.warn(getClass().getSimpleName() + " interrupted");
}
}
}
private synchronized StorageReceivedDeletedBlocks[] generateIBRs() {
@ -144,6 +171,9 @@ class IncrementalBlockReportManager {
}
// Send incremental block reports to the Namenode outside the lock
if (LOG.isDebugEnabled()) {
LOG.debug("call blockReceivedAndDeleted: " + Arrays.toString(reports));
}
boolean success = false;
final long startTime = monotonicNow();
try {
@ -151,7 +181,9 @@ class IncrementalBlockReportManager {
success = true;
} finally {
metrics.addIncrementalBlockReport(monotonicNow() - startTime);
if (!success) {
if (success) {
lastIBR = startTime;
} else {
// 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.
@ -191,7 +223,7 @@ class IncrementalBlockReportManager {
}
synchronized void notifyNamenodeBlock(ReceivedDeletedBlockInfo rdbi,
DatanodeStorage storage) {
DatanodeStorage storage, boolean isOnTransientStorage) {
addRDBI(rdbi, storage);
final BlockStatus status = rdbi.getStatus();
@ -200,18 +232,23 @@ class IncrementalBlockReportManager {
readyToSend = true;
} else if (status == BlockStatus.RECEIVED_BLOCK) {
// the report is sent right away.
triggerIBR();
triggerIBR(isOnTransientStorage);
}
}
synchronized void triggerIBR() {
synchronized void triggerIBR(boolean force) {
readyToSend = true;
if (force) {
lastIBR = monotonicNow() - ibrInterval;
}
if (sendImmediately()) {
notifyAll();
}
}
@VisibleForTesting
synchronized void triggerDeletionReportForTests() {
triggerIBR();
triggerIBR(true);
while (sendImmediately()) {
try {

View File

@ -285,7 +285,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
* @return the storage uuid of the replica.
* @throws IOException
*/
public String recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen
Replica recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen
) throws IOException;
/**
@ -435,7 +435,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
* Update replica's generation stamp and length and finalize it.
* @return the ID of storage that stores the block
*/
public String updateReplicaUnderRecovery(ExtendedBlock oldBlock,
Replica updateReplicaUnderRecovery(ExtendedBlock oldBlock,
long recoveryId, long newBlockId, long newLength) throws IOException;
/**

View File

@ -1233,7 +1233,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
}
@Override // FsDatasetSpi
public synchronized String recoverClose(ExtendedBlock b, long newGS,
public synchronized Replica recoverClose(ExtendedBlock b, long newGS,
long expectedBlockLen) throws IOException {
LOG.info("Recover failed close " + b);
// check replica's state
@ -1244,7 +1244,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
if (replicaInfo.getState() == ReplicaState.RBW) {
finalizeReplica(b.getBlockPoolId(), replicaInfo);
}
return replicaInfo.getStorageUuid();
return replicaInfo;
}
/**
@ -2368,7 +2368,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
}
@Override // FsDatasetSpi
public synchronized String updateReplicaUnderRecovery(
public synchronized Replica updateReplicaUnderRecovery(
final ExtendedBlock oldBlock,
final long recoveryId,
final long newBlockId,
@ -2428,8 +2428,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
//check replica files after update
checkReplicaFiles(finalized);
//return storage ID
return getVolume(new ExtendedBlock(bpid, finalized)).getStorageID();
return finalized;
}
private FinalizedReplica updateReplicaUnderRecovery(
@ -2804,7 +2803,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
datanode.getShortCircuitRegistry().processBlockInvalidation(
ExtendedBlockId.fromExtendedBlock(extendedBlock));
datanode.notifyNamenodeReceivedBlock(
extendedBlock, null, newReplicaInfo.getStorageUuid());
extendedBlock, null, newReplicaInfo.getStorageUuid(),
newReplicaInfo.isOnTransientStorage());
// Remove the old replicas
if (blockFile.delete() || !blockFile.exists()) {

View File

@ -18,6 +18,8 @@
package org.apache.hadoop.hdfs.server.protocol;
import java.util.Arrays;
/**
* Report of block received and deleted per Datanode
* storage.
@ -51,4 +53,9 @@ public class StorageReceivedDeletedBlocks {
this.storage = storage;
this.blocks = blocks;
}
@Override
public String toString() {
return storage + Arrays.toString(blocks);
}
}

View File

@ -876,7 +876,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
}
@Override // FsDatasetSpi
public String 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());
@ -890,7 +890,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
map.remove(b.getLocalBlock());
binfo.theBlock.setGenerationStamp(newGS);
map.put(binfo.theBlock, binfo);
return binfo.getStorageUuid();
return binfo;
}
@Override // FsDatasetSpi
@ -1171,12 +1171,11 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
}
@Override // FsDatasetSpi
public String updateReplicaUnderRecovery(ExtendedBlock oldBlock,
public Replica updateReplicaUnderRecovery(ExtendedBlock oldBlock,
long recoveryId,
long newBlockId,
long newlength) {
// Caller does not care about the exact Storage UUID returned.
return datanodeUuid;
long newlength) throws IOException {
return getMap(oldBlock.getBlockPoolId()).get(oldBlock.getLocalBlock());
}
@Override // FsDatasetSpi

View File

@ -202,7 +202,7 @@ public class TestBPOfferService {
waitForBlockReport(mockNN2);
// When we receive a block, it should report it to both NNs
bpos.notifyNamenodeReceivedBlock(FAKE_BLOCK, "", "");
bpos.notifyNamenodeReceivedBlock(FAKE_BLOCK, null, "", false);
ReceivedDeletedBlockInfo[] ret = waitForBlockReceived(FAKE_BLOCK, mockNN1);
assertEquals(1, ret.length);
@ -726,7 +726,7 @@ public class TestBPOfferService {
DatanodeStorage storage = Mockito.mock(DatanodeStorage.class);
Mockito.doReturn(storage).when(mockFSDataset).getStorage("storage0");
// Add IBRs
bpos.notifyNamenodeReceivedBlock(FAKE_BLOCK, null, "storage0");
bpos.notifyNamenodeReceivedBlock(FAKE_BLOCK, null, "storage0", false);
// Send heartbeat so that the BpServiceActor can send IBR to
// namenode
bpos.triggerHeartbeatForTests();

View File

@ -84,7 +84,7 @@ public class TestIncrementalBlockReports {
ReceivedDeletedBlockInfo rdbi = new ReceivedDeletedBlockInfo(
getDummyBlock(), BlockStatus.RECEIVED_BLOCK, null);
DatanodeStorage s = singletonDn.getFSDataset().getStorage(storageUuid);
actor.getIbrManager().notifyNamenodeBlock(rdbi, s);
actor.getIbrManager().notifyNamenodeBlock(rdbi, s, false);
}
/**

View File

@ -177,7 +177,7 @@ public class ExternalDatasetImpl implements FsDatasetSpi<ExternalVolumeImpl> {
}
@Override
public String recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen)
public Replica recoverClose(ExtendedBlock b, long newGS, long expectedBlkLen)
throws IOException {
return null;
}
@ -272,7 +272,7 @@ public class ExternalDatasetImpl implements FsDatasetSpi<ExternalVolumeImpl> {
}
@Override
public String updateReplicaUnderRecovery(ExtendedBlock oldBlock,
public Replica updateReplicaUnderRecovery(ExtendedBlock oldBlock,
long recoveryId, long newBlockId, long newLength) throws IOException {
return null;
}

View File

@ -46,6 +46,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
import org.apache.hadoop.hdfs.server.datanode.Replica;
import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
import org.apache.hadoop.hdfs.server.datanode.ReplicaUnderRecovery;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
@ -378,10 +379,11 @@ public class TestInterDatanodeProtocol {
}
//update
final String storageID = fsdataset.updateReplicaUnderRecovery(
final Replica r = fsdataset.updateReplicaUnderRecovery(
new ExtendedBlock(b.getBlockPoolId(), rri), recoveryid,
rri.getBlockId(), newlength);
assertTrue(storageID != null);
assertTrue(r != null);
assertTrue(r.getStorageUuid() != null);
} finally {
if (cluster != null) cluster.shutdown();