HDFS-5051. Propagate cache status information from the DataNode to the NameNode (Andrew Wang via Colin Patrick McCabe)
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-4949@1513653 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
2a4031940c
commit
52ccc6c6d5
|
@ -9,6 +9,9 @@ HDFS-4949 (Unreleased)
|
||||||
IMPROVEMENTS
|
IMPROVEMENTS
|
||||||
HDFS-5049. Add JNI mlock support. (Andrew Wang via Colin Patrick McCabe)
|
HDFS-5049. Add JNI mlock support. (Andrew Wang via Colin Patrick McCabe)
|
||||||
|
|
||||||
|
HDFS-5051. Propagate cache status information from the DataNode to the
|
||||||
|
NameNode (Andrew Wang via Colin Patrick McCabe)
|
||||||
|
|
||||||
OPTIMIZATIONS
|
OPTIMIZATIONS
|
||||||
|
|
||||||
BUG FIXES
|
BUG FIXES
|
||||||
|
|
|
@ -354,6 +354,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
|
||||||
public static final long DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT = 60 * 60 * 1000;
|
public static final long DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT = 60 * 60 * 1000;
|
||||||
public static final String DFS_BLOCKREPORT_INITIAL_DELAY_KEY = "dfs.blockreport.initialDelay";
|
public static final String DFS_BLOCKREPORT_INITIAL_DELAY_KEY = "dfs.blockreport.initialDelay";
|
||||||
public static final int DFS_BLOCKREPORT_INITIAL_DELAY_DEFAULT = 0;
|
public static final int DFS_BLOCKREPORT_INITIAL_DELAY_DEFAULT = 0;
|
||||||
|
public static final String DFS_CACHEREPORT_INTERVAL_MSEC_KEY = "dfs.cachereport.intervalMsec";
|
||||||
|
public static final long DFS_CACHEREPORT_INTERVAL_MSEC_DEFAULT = 10 * 1000;
|
||||||
public static final String DFS_BLOCK_INVALIDATE_LIMIT_KEY = "dfs.block.invalidate.limit";
|
public static final String DFS_BLOCK_INVALIDATE_LIMIT_KEY = "dfs.block.invalidate.limit";
|
||||||
public static final int DFS_BLOCK_INVALIDATE_LIMIT_DEFAULT = 1000;
|
public static final int DFS_BLOCK_INVALIDATE_LIMIT_DEFAULT = 1000;
|
||||||
public static final String DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED_KEY = "dfs.corruptfilesreturned.max";
|
public static final String DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED_KEY = "dfs.corruptfilesreturned.max";
|
||||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.hadoop.hdfs.protocolPB;
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.net.InetSocketAddress;
|
import java.net.InetSocketAddress;
|
||||||
|
import java.util.ArrayList;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
@ -36,6 +37,8 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto;
|
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto;
|
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto;
|
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CacheReportRequestProto;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CacheReportResponseProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto;
|
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto;
|
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto;
|
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto;
|
||||||
|
@ -202,6 +205,29 @@ public class DatanodeProtocolClientSideTranslatorPB implements
|
||||||
return resp.hasCmd() ? PBHelper.convert(resp.getCmd()) : null;
|
return resp.hasCmd() ? PBHelper.convert(resp.getCmd()) : null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DatanodeCommand cacheReport(DatanodeRegistration registration,
|
||||||
|
String poolId, long[] blocks) throws IOException {
|
||||||
|
CacheReportRequestProto.Builder builder =
|
||||||
|
CacheReportRequestProto.newBuilder()
|
||||||
|
.setRegistration(PBHelper.convert(registration))
|
||||||
|
.setBlockPoolId(poolId);
|
||||||
|
for (int i=0; i<blocks.length; i++) {
|
||||||
|
builder.addBlocks(blocks[i]);
|
||||||
|
}
|
||||||
|
|
||||||
|
CacheReportResponseProto resp;
|
||||||
|
try {
|
||||||
|
resp = rpcProxy.cacheReport(NULL_CONTROLLER, builder.build());
|
||||||
|
} catch (ServiceException se) {
|
||||||
|
throw ProtobufHelper.getRemoteException(se);
|
||||||
|
}
|
||||||
|
if (resp.hasCmd()) {
|
||||||
|
return PBHelper.convert(resp.getCmd());
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void blockReceivedAndDeleted(DatanodeRegistration registration,
|
public void blockReceivedAndDeleted(DatanodeRegistration registration,
|
||||||
String poolId, StorageReceivedDeletedBlocks[] receivedAndDeletedBlocks)
|
String poolId, StorageReceivedDeletedBlocks[] receivedAndDeletedBlocks)
|
||||||
|
|
|
@ -27,6 +27,8 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceive
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto;
|
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto;
|
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto;
|
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CacheReportRequestProto;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CacheReportResponseProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto;
|
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto;
|
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto;
|
||||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto;
|
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto;
|
||||||
|
@ -55,6 +57,7 @@ import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
|
||||||
import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
|
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 com.google.common.primitives.Longs;
|
||||||
import com.google.protobuf.RpcController;
|
import com.google.protobuf.RpcController;
|
||||||
import com.google.protobuf.ServiceException;
|
import com.google.protobuf.ServiceException;
|
||||||
|
|
||||||
|
@ -159,6 +162,27 @@ public class DatanodeProtocolServerSideTranslatorPB implements
|
||||||
return builder.build();
|
return builder.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public CacheReportResponseProto cacheReport(RpcController controller,
|
||||||
|
CacheReportRequestProto request) throws ServiceException {
|
||||||
|
DatanodeCommand cmd = null;
|
||||||
|
try {
|
||||||
|
cmd = impl.cacheReport(
|
||||||
|
PBHelper.convert(request.getRegistration()),
|
||||||
|
request.getBlockPoolId(),
|
||||||
|
Longs.toArray(request.getBlocksList()));
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new ServiceException(e);
|
||||||
|
}
|
||||||
|
CacheReportResponseProto.Builder builder =
|
||||||
|
CacheReportResponseProto.newBuilder();
|
||||||
|
if (cmd != null) {
|
||||||
|
builder.setCmd(PBHelper.convert(cmd));
|
||||||
|
}
|
||||||
|
return builder.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public BlockReceivedAndDeletedResponseProto blockReceivedAndDeleted(
|
public BlockReceivedAndDeletedResponseProto blockReceivedAndDeleted(
|
||||||
RpcController controller, BlockReceivedAndDeletedRequestProto request)
|
RpcController controller, BlockReceivedAndDeletedRequestProto request)
|
||||||
|
|
|
@ -1664,6 +1664,15 @@ public class BlockManager {
|
||||||
+ ", processing time: " + (endTime - startTime) + " msecs");
|
+ ", processing time: " + (endTime - startTime) + " msecs");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The given datanode is reporting all of its cached blocks.
|
||||||
|
* Update the cache state of blocks in the block map.
|
||||||
|
*/
|
||||||
|
public void processCacheReport(final DatanodeID nodeID, final String poolId,
|
||||||
|
final BlockListAsLongs newReport) throws IOException {
|
||||||
|
// TODO: Implement me!
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Rescan the list of blocks which were previously postponed.
|
* Rescan the list of blocks which were previously postponed.
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -368,6 +368,12 @@ class BPOfferService {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void scheduleCacheReport(long delay) {
|
||||||
|
for (BPServiceActor actor: bpServices) {
|
||||||
|
actor.scheduleCacheReport(delay);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Ask each of the actors to report a bad block hosted on another DN.
|
* Ask each of the actors to report a bad block hosted on another DN.
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -84,6 +84,8 @@ class BPServiceActor implements Runnable {
|
||||||
|
|
||||||
boolean resetBlockReportTime = true;
|
boolean resetBlockReportTime = true;
|
||||||
|
|
||||||
|
volatile long lastCacheReport = 0;
|
||||||
|
|
||||||
Thread bpThread;
|
Thread bpThread;
|
||||||
DatanodeProtocolClientSideTranslatorPB bpNamenode;
|
DatanodeProtocolClientSideTranslatorPB bpNamenode;
|
||||||
private volatile long lastHeartbeat = 0;
|
private volatile long lastHeartbeat = 0;
|
||||||
|
@ -239,6 +241,17 @@ class BPServiceActor implements Runnable {
|
||||||
resetBlockReportTime = true; // reset future BRs for randomness
|
resetBlockReportTime = true; // reset future BRs for randomness
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void scheduleCacheReport(long delay) {
|
||||||
|
if (delay > 0) {
|
||||||
|
// Uniform random jitter by the delay
|
||||||
|
lastCacheReport = Time.monotonicNow()
|
||||||
|
- dnConf.cacheReportInterval
|
||||||
|
+ DFSUtil.getRandom().nextInt(((int)delay));
|
||||||
|
} else { // send at next heartbeat
|
||||||
|
lastCacheReport = lastCacheReport - dnConf.cacheReportInterval;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
void reportBadBlocks(ExtendedBlock block) {
|
void reportBadBlocks(ExtendedBlock block) {
|
||||||
if (bpRegistration == null) {
|
if (bpRegistration == null) {
|
||||||
return;
|
return;
|
||||||
|
@ -430,6 +443,15 @@ class BPServiceActor implements Runnable {
|
||||||
return cmd;
|
return cmd;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
DatanodeCommand cacheReport() throws IOException {
|
||||||
|
// send cache report if timer has expired.
|
||||||
|
DatanodeCommand cmd = null;
|
||||||
|
long startTime = Time.monotonicNow();
|
||||||
|
if (startTime - lastCacheReport > dnConf.cacheReportInterval) {
|
||||||
|
// TODO: Implement me!
|
||||||
|
}
|
||||||
|
return cmd;
|
||||||
|
}
|
||||||
|
|
||||||
HeartbeatResponse sendHeartBeat() throws IOException {
|
HeartbeatResponse sendHeartBeat() throws IOException {
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
|
@ -496,11 +518,12 @@ class BPServiceActor implements Runnable {
|
||||||
* forever calling remote NameNode functions.
|
* forever calling remote NameNode functions.
|
||||||
*/
|
*/
|
||||||
private void offerService() throws Exception {
|
private void offerService() throws Exception {
|
||||||
LOG.info("For namenode " + nnAddr + " using DELETEREPORT_INTERVAL of "
|
LOG.info("For namenode " + nnAddr + " using"
|
||||||
+ dnConf.deleteReportInterval + " msec " + " BLOCKREPORT_INTERVAL of "
|
+ " DELETEREPORT_INTERVAL of " + dnConf.deleteReportInterval + " msec "
|
||||||
+ dnConf.blockReportInterval + "msec" + " Initial delay: "
|
+ " BLOCKREPORT_INTERVAL of " + dnConf.blockReportInterval + "msec"
|
||||||
+ dnConf.initialBlockReportDelay + "msec" + "; heartBeatInterval="
|
+ " CACHEREPORT_INTERVAL of " + dnConf.cacheReportInterval + "msec"
|
||||||
+ dnConf.heartBeatInterval);
|
+ " Initial delay: " + dnConf.initialBlockReportDelay + "msec"
|
||||||
|
+ "; heartBeatInterval=" + dnConf.heartBeatInterval);
|
||||||
|
|
||||||
//
|
//
|
||||||
// Now loop for a long time....
|
// Now loop for a long time....
|
||||||
|
@ -555,6 +578,9 @@ class BPServiceActor implements Runnable {
|
||||||
DatanodeCommand cmd = blockReport();
|
DatanodeCommand cmd = blockReport();
|
||||||
processCommand(new DatanodeCommand[]{ cmd });
|
processCommand(new DatanodeCommand[]{ cmd });
|
||||||
|
|
||||||
|
cmd = cacheReport();
|
||||||
|
processCommand(new DatanodeCommand[]{ cmd });
|
||||||
|
|
||||||
// Now safe to start scanning the block pool.
|
// Now safe to start scanning the block pool.
|
||||||
// If it has already been started, this is a no-op.
|
// If it has already been started, this is a no-op.
|
||||||
if (dn.blockScanner != null) {
|
if (dn.blockScanner != null) {
|
||||||
|
|
|
@ -70,6 +70,7 @@ public class DNConf {
|
||||||
final long blockReportInterval;
|
final long blockReportInterval;
|
||||||
final long deleteReportInterval;
|
final long deleteReportInterval;
|
||||||
final long initialBlockReportDelay;
|
final long initialBlockReportDelay;
|
||||||
|
final long cacheReportInterval;
|
||||||
final int writePacketSize;
|
final int writePacketSize;
|
||||||
|
|
||||||
final String minimumNameNodeVersion;
|
final String minimumNameNodeVersion;
|
||||||
|
@ -114,6 +115,8 @@ public class DNConf {
|
||||||
DFSConfigKeys.DFS_DATANODE_USE_DN_HOSTNAME_DEFAULT);
|
DFSConfigKeys.DFS_DATANODE_USE_DN_HOSTNAME_DEFAULT);
|
||||||
this.blockReportInterval = conf.getLong(DFS_BLOCKREPORT_INTERVAL_MSEC_KEY,
|
this.blockReportInterval = conf.getLong(DFS_BLOCKREPORT_INTERVAL_MSEC_KEY,
|
||||||
DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT);
|
DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT);
|
||||||
|
this.cacheReportInterval = conf.getLong(DFS_BLOCKREPORT_INTERVAL_MSEC_KEY,
|
||||||
|
DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT);
|
||||||
|
|
||||||
long initBRDelay = conf.getLong(
|
long initBRDelay = conf.getLong(
|
||||||
DFS_BLOCKREPORT_INITIAL_DELAY_KEY,
|
DFS_BLOCKREPORT_INITIAL_DELAY_KEY,
|
||||||
|
|
|
@ -1916,6 +1916,7 @@ public class DataNode extends Configured
|
||||||
public void scheduleAllBlockReport(long delay) {
|
public void scheduleAllBlockReport(long delay) {
|
||||||
for(BPOfferService bpos : blockPoolManager.getAllNamenodeThreads()) {
|
for(BPOfferService bpos : blockPoolManager.getAllNamenodeThreads()) {
|
||||||
bpos.scheduleBlockReport(delay);
|
bpos.scheduleBlockReport(delay);
|
||||||
|
bpos.scheduleCacheReport(delay);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -951,6 +951,18 @@ class NameNodeRpcServer implements NamenodeProtocols {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DatanodeCommand cacheReport(DatanodeRegistration nodeReg,
|
||||||
|
String poolId, long[] blocks) throws IOException {
|
||||||
|
verifyRequest(nodeReg);
|
||||||
|
BlockListAsLongs blist = new BlockListAsLongs(blocks);
|
||||||
|
namesystem.getBlockManager().processCacheReport(nodeReg, poolId, blist);
|
||||||
|
if (nn.getFSImage().isUpgradeFinalized() && !nn.isStandbyState()) {
|
||||||
|
return new FinalizeCommand(poolId);
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
@Override // DatanodeProtocol
|
@Override // DatanodeProtocol
|
||||||
public void blockReceivedAndDeleted(DatanodeRegistration nodeReg, String poolId,
|
public void blockReceivedAndDeleted(DatanodeRegistration nodeReg, String poolId,
|
||||||
StorageReceivedDeletedBlocks[] receivedAndDeletedBlocks) throws IOException {
|
StorageReceivedDeletedBlocks[] receivedAndDeletedBlocks) throws IOException {
|
||||||
|
|
|
@ -22,10 +22,10 @@ import java.io.*;
|
||||||
|
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
|
||||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||||
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||||
import org.apache.hadoop.io.retry.AtMostOnce;
|
|
||||||
import org.apache.hadoop.io.retry.Idempotent;
|
import org.apache.hadoop.io.retry.Idempotent;
|
||||||
import org.apache.hadoop.security.KerberosInfo;
|
import org.apache.hadoop.security.KerberosInfo;
|
||||||
|
|
||||||
|
@ -128,6 +128,25 @@ public interface DatanodeProtocol {
|
||||||
public DatanodeCommand blockReport(DatanodeRegistration registration,
|
public DatanodeCommand blockReport(DatanodeRegistration registration,
|
||||||
String poolId, StorageBlockReport[] reports) throws IOException;
|
String poolId, StorageBlockReport[] reports) throws IOException;
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Communicates the complete list of locally cached blocks to the NameNode.
|
||||||
|
*
|
||||||
|
* This method is similar to
|
||||||
|
* {@link #blockReport(DatanodeRegistration, String, StorageBlockReport[])},
|
||||||
|
* which is used to communicated blocks stored on disk.
|
||||||
|
*
|
||||||
|
* @param registration
|
||||||
|
* @param poolId block pool ID for the blocks
|
||||||
|
* @param blocks a Long[] array from {@link BlockListAsLongs} that describes
|
||||||
|
* the list of cached blocks. This is more memory-efficient than a Block[].
|
||||||
|
* @return
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
@Idempotent
|
||||||
|
public DatanodeCommand cacheReport(DatanodeRegistration registration,
|
||||||
|
String poolId, long[] blocks) throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* blockReceivedAndDeleted() allows the DataNode to tell the NameNode about
|
* blockReceivedAndDeleted() allows the DataNode to tell the NameNode about
|
||||||
* recently-received and -deleted block data.
|
* recently-received and -deleted block data.
|
||||||
|
|
|
@ -205,9 +205,11 @@ message HeartbeatResponseProto {
|
||||||
/**
|
/**
|
||||||
* registration - datanode registration information
|
* registration - datanode registration information
|
||||||
* blockPoolID - block pool ID of the reported blocks
|
* blockPoolID - block pool ID of the reported blocks
|
||||||
* blocks - each block is represented as two longs in the array.
|
* blocks - each block is represented as multiple longs in the array.
|
||||||
* first long represents block ID
|
* first long represents block ID
|
||||||
* second long represents length
|
* second long represents length
|
||||||
|
* third long represents gen stamp
|
||||||
|
* fourth long (if under construction) represents replica state
|
||||||
*/
|
*/
|
||||||
message BlockReportRequestProto {
|
message BlockReportRequestProto {
|
||||||
required DatanodeRegistrationProto registration = 1;
|
required DatanodeRegistrationProto registration = 1;
|
||||||
|
@ -230,6 +232,21 @@ message BlockReportResponseProto {
|
||||||
optional DatanodeCommandProto cmd = 1;
|
optional DatanodeCommandProto cmd = 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* registration - datanode registration information
|
||||||
|
* blockPoolId - block pool ID of the reported blocks
|
||||||
|
* blocks - representation of blocks as longs for efficiency reasons
|
||||||
|
*/
|
||||||
|
message CacheReportRequestProto {
|
||||||
|
required DatanodeRegistrationProto registration = 1;
|
||||||
|
required string blockPoolId = 2;
|
||||||
|
repeated uint64 blocks = 3 [packed=true];
|
||||||
|
}
|
||||||
|
|
||||||
|
message CacheReportResponseProto {
|
||||||
|
optional DatanodeCommandProto cmd = 1;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Data structure to send received or deleted block information
|
* Data structure to send received or deleted block information
|
||||||
* from datanode to namenode.
|
* from datanode to namenode.
|
||||||
|
@ -347,6 +364,11 @@ service DatanodeProtocolService {
|
||||||
*/
|
*/
|
||||||
rpc blockReport(BlockReportRequestProto) returns(BlockReportResponseProto);
|
rpc blockReport(BlockReportRequestProto) returns(BlockReportResponseProto);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Report cached blocks at a datanode to the namenode
|
||||||
|
*/
|
||||||
|
rpc cacheReport(CacheReportRequestProto) returns(CacheReportResponseProto);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Incremental block report from the DN. This contains info about recently
|
* Incremental block report from the DN. This contains info about recently
|
||||||
* received and deleted blocks, as well as when blocks start being
|
* received and deleted blocks, as well as when blocks start being
|
||||||
|
|
Loading…
Reference in New Issue