append操作在namenode这端主要逻辑在FSNameSystem的appendFileInternal函数中处理,内部会调用
前言
在DFSClient写文件的时候,对于文件的每一个block,生成一个pipeline,然后按照这个pipeline进行数据传输,但是可能在数据传输过程中,DFSClient发生中断,例如断网等,此时该block在NameNode中处于UnderConstruction状态,该block所在的文件的写锁(通过LeaseManager实现)被该DFSClient占据着,无法释放。为此LeaseManager会对超过一定时间不活跃的DFSClient所占用的文件进行Recovery。
同时,BlockInfoContiguousUnderConstruction类有一个成员变量replicas用于存放“expected DataNodes which make up of the pipeline for that Block”. 该成员变量是在pipeline生成之后构造的,也就是说它并不用于指导生成pipeline,它的用途即用于Recovery。需要说明的是,在BlockInfoContiguousUnderConstruction的基类BlockInfoContiguous中有一个变量triplets用于存在block所有的replication所处的DataNodeStorageInfo。replicas记录的是"expected DataNodes",而triplets记录的是“stored DataNodes”。 下面主要介绍recovery的具体过程。
lease recovery
hdfs lease是为了实现一个文件在一个时刻只能被一个客户端写。客户端写文件或者append之前都需要向namenode申请这个文件的lease,在客户端写数据的过程中,客户端中的单独线程会不断的renew lease,不断的延长独占写的时间.
lease有两个limit,一个是soft limit,默认60s,一个是hard limit,默认1小时。lease soft limit过期之前,该客户端拥有对这个文件的独立访问权,其他客户端不能剥夺该客户端独占写这个文件的权利。lease soft limit过期后,任何一个客户端都可以回lease,继而得到这个文件的lease,获得对这个文件的独占访问权。lease hard limit过期后,namenode强制关闭文件,撤销lease.
考虑客户端写文件的过程中宕机,那么在lease soft limit过期之前,其他的客户端不能写这个文件,等到lease soft limit过期后,其他客户端可以写这个文件,在写文件之前,会首先检查文件是不是没有关闭,如果没有,那么就会进入lease recovery和block recovery阶段,这个阶段的目的是使文件的最后一个block的所有副本数据达到一致,因为客户端写block的多个副本是pipeline写,pipeline中的副本数据不一致很正常。
模拟场景:客户端写文件过程中客户端进程断掉,然后重新启动新客户端对文件进行append操作。
FileSystem fs = FileSystem.get(configuration);
FSDataOutputStream out = fs.append(path);
out.write(byte[]);
append操作在namenode这端主要逻辑在FSNameSystem的appendFileInternal函数中处理,内部会调用
// Opening an existing file for append - may need to recover lease.
recoverLeaseInternal(RecoverLeaseOp.APPEND_FILE,
iip, src, holder, clientMachine, false);
recoverLeaseInternal方法主要检查是否需要首先对文件进行lease recovery
boolean recoverLeaseInternal(RecoverLeaseOp op, INodesInPath iip,
String src, String holder, String clientMachine, boolean force)
throws IOException {
assert hasWriteLock();
INodeFile file = iip.getLastINode().asFile();
if (file.isUnderConstruction()) {
//
// If the file is under construction , then it must be in our
// leases. Find the appropriate lease record.
//
Lease lease = leaseManager.getLease(holder);
if (!force && lease != null) {
Lease leaseFile = leaseManager.getLeaseByPath(src);
if (leaseFile != null && leaseFile.equals(lease)) {
// We found the lease for this file but the original
// holder is trying to obtain it again.
throw new AlreadyBeingCreatedException(
op.getExceptionMessage(src, holder, clientMachine,
holder + " is already the current lease holder."));
}
}
//
// Find the original holder.
//
FileUnderConstructionFeature uc = file.getFileUnderConstructionFeature();
String clientName = uc.getClientName();
lease = leaseManager.getLease(clientName);
if (lease == null) {
throw new AlreadyBeingCreatedException(
op.getExceptionMessage(src, holder, clientMachine,
"the file is under construction but no leases found."));
}
if (force) {
// close now: no need to wait for soft lease expiration and
// close only the file src
LOG.info("recoverLease: " + lease + ", src=" + src +
" from client " + clientName);
return internalReleaseLease(lease, src, iip, holder);
} else {
assert lease.getHolder().equals(clientName) :
"Current lease holder " + lease.getHolder() +
" does not match file creator " + clientName;
//
// If the original holder has not renewed in the last SOFTLIMIT
// period, then start lease recovery.
//
if (lease.expiredSoftLimit()) {
LOG.info("startFile: recover " + lease + ", src=" + src + " client "
+ clientName);
if (internalReleaseLease(lease, src, iip, null)) {
return true;
} else {
throw new RecoveryInProgressException(
op.getExceptionMessage(src, holder, clientMachine,
"lease recovery is in progress. Try again later."));
}
} else {
final BlockInfoContiguous lastBlock = file.getLastBlock();
if (lastBlock != null
&& lastBlock.getBlockUCState() == BlockUCState.UNDER_RECOVERY) {
throw new RecoveryInProgressException(
op.getExceptionMessage(src, holder, clientMachine,
"another recovery is in progress by "
+ clientName + " on " + uc.getClientMachine()));
} else {
throw new AlreadyBeingCreatedException(
op.getExceptionMessage(src, holder, clientMachine,
"this file lease is currently owned by "
+ clientName + " on " + uc.getClientMachine()));
}
}
}
} else {
return true;
}
}
recoverLeaseInternal方法具体流程:
1.获取文件状态,如果是UnderConstruction状态,说明文件不处于关闭状态,有可能需要执行lease recovery和block recovery流程;
2.在leasemanager中通过src获取lease,如果不为空且就是当前client的lease,则返回AlreadyBeingCreatedException异常;
3.获取持有当前UC状态下文件的client的租约,如果有空,则直接返回异常AlreadyBeingCreatedException(the file is under construction but no leases found.)
4.第三步获取的lease,如果不为空,则继续检查这个lease是否超过soft limit,如果过期则调用internalReleaseLease.
下面来介绍block recovery的具体过程。
block recovery
internalReleaseLease代码:
boolean internalReleaseLease(Lease lease, String src, INodesInPath iip,
String recoveryLeaseHolder) throws IOException {
LOG.info("Recovering " + lease + ", src=" + src);
assert !isInSafeMode();
assert hasWriteLock();
final INodeFile pendingFile = iip.getLastINode().asFile();
int nrBlocks = pendingFile.numBlocks();
BlockInfoContiguous[] blocks = pendingFile.getBlocks();
int nrCompleteBlocks;
BlockInfoContiguous curBlock = null;
for(nrCompleteBlocks = 0; nrCompleteBlocks < nrBlocks; nrCompleteBlocks++) {
curBlock = blocks[nrCompleteBlocks];
if(!curBlock.isComplete())
break;
assert blockManager.checkMinReplication(curBlock) :
"A COMPLETE block is not minimally replicated in " + src;
}
// If there are no incomplete blocks associated with this file,
// then reap lease immediately and close the file.
if(nrCompleteBlocks == nrBlocks) {
finalizeINodeFileUnderConstruction(src, pendingFile,
iip.getLatestSnapshotId());
NameNode.stateChangeLog.warn("BLOCK*"
+ " internalReleaseLease: All existing blocks are COMPLETE,"
+ " lease removed, file closed.");
return true; // closed!
}
// Only the last and the penultimate blocks may be in non COMPLETE state.
// If the penultimate block is not COMPLETE, then it must be COMMITTED.
if(nrCompleteBlocks < nrBlocks - 2 ||
nrCompleteBlocks == nrBlocks - 2 &&
curBlock != null &&
curBlock.getBlockUCState() != BlockUCState.COMMITTED) {
final String message = "DIR* NameSystem.internalReleaseLease: "
+ "attempt to release a create lock on "
+ src + " but file is already closed.";
NameNode.stateChangeLog.warn(message);
throw new IOException(message);
}
// The last block is not COMPLETE, and
// that the penultimate block if exists is either COMPLETE or COMMITTED
final BlockInfoContiguous lastBlock = pendingFile.getLastBlock();
BlockUCState lastBlockState = lastBlock.getBlockUCState();
BlockInfoContiguous penultimateBlock = pendingFile.getPenultimateBlock();
// If penultimate block doesn't exist then its minReplication is met
boolean penultimateBlockMinReplication = penultimateBlock == null ? true :
blockManager.checkMinReplication(penultimateBlock);
switch(lastBlockState) {
case COMPLETE:
assert false : "Already checked that the last block is incomplete";
break;
case COMMITTED:
// Close file if committed blocks are minimally replicated
if(penultimateBlockMinReplication &&
blockManager.checkMinReplication(lastBlock)) {
finalizeINodeFileUnderConstruction(src, pendingFile,
iip.getLatestSnapshotId());
NameNode.stateChangeLog.warn("BLOCK*"
+ " internalReleaseLease: Committed blocks are minimally replicated,"
+ " lease removed, file closed.");
return true; // closed!
}
// Cannot close file right now, since some blocks
// are not yet minimally replicated.
// This may potentially cause infinite loop in lease recovery
// if there are no valid replicas on data-nodes.
String message = "DIR* NameSystem.internalReleaseLease: " +
"Failed to release lease for file " + src +
". Committed blocks are waiting to be minimally replicated." +
" Try again later.";
NameNode.stateChangeLog.warn(message);
throw new AlreadyBeingCreatedException(message);
case UNDER_CONSTRUCTION:
case UNDER_RECOVERY:
final BlockInfoContiguousUnderConstruction uc = (BlockInfoContiguousUnderConstruction)lastBlock;
// determine if last block was intended to be truncated
Block recoveryBlock = uc.getTruncateBlock();
boolean truncateRecovery = recoveryBlock != null;
boolean copyOnTruncate = truncateRecovery &&
recoveryBlock.getBlockId() != uc.getBlockId();
assert !copyOnTruncate ||
recoveryBlock.getBlockId() < uc.getBlockId() &&
recoveryBlock.getGenerationStamp() < uc.getGenerationStamp() &&
recoveryBlock.getNumBytes() > uc.getNumBytes() :
"wrong recoveryBlock";
// setup the last block locations from the blockManager if not known
if (uc.getNumExpectedLocations() == 0) {
uc.setExpectedLocations(blockManager.getStorages(lastBlock));
}
if (uc.getNumExpectedLocations() == 0 && uc.getNumBytes() == 0) {
// There is no datanode reported to this block.
// may be client have crashed before writing data to pipeline.
// This blocks doesn't need any recovery.
// We can remove this block and close the file.
pendingFile.removeLastBlock(lastBlock);
finalizeINodeFileUnderConstruction(src, pendingFile,
iip.getLatestSnapshotId());
NameNode.stateChangeLog.warn("BLOCK* internalReleaseLease: "
+ "Removed empty last block and closed file.");
return true;
}
// start recovery of the last block for this file
long blockRecoveryId = nextGenerationStamp(blockIdManager.isLegacyBlock(uc));
lease = reassignLease(lease, src, recoveryLeaseHolder, pendingFile);
if(copyOnTruncate) {
uc.setGenerationStamp(blockRecoveryId);
} else if(truncateRecovery) {
recoveryBlock.setGenerationStamp(blockRecoveryId);
}
uc.initializeBlockRecovery(blockRecoveryId);
leaseManager.renewLease(lease);
// Cannot close file right now, since the last block requires recovery.
// This may potentially cause infinite loop in lease recovery
// if there are no valid replicas on data-nodes.
NameNode.stateChangeLog.warn(
"DIR* NameSystem.internalReleaseLease: " +
"File " + src + " has not been closed." +
" Lease recovery is in progress. " +
"RecoveryId = " + blockRecoveryId + " for block " + lastBlock);
break;
}
return false;
}
具体流程:
1. 检查文件Inode里所有block的状态,如果都是completed状态,则直接调用finalizeINodeFileUnderConstruction方法,从namenode 端leasemanager中移除对应的lease,更改Inode状态,直接关闭文件。
private void finalizeINodeFileUnderConstruction(String src,
INodeFile pendingFile, int latestSnapshot) throws IOException {
assert hasWriteLock();
FileUnderConstructionFeature uc = pendingFile.getFileUnderConstructionFeature();
if (uc == null) {
throw new IOException("Cannot finalize file " + src
+ " because it is not under construction");
}
leaseManager.removeLease(uc.getClientName(), src);
pendingFile.recordModification(latestSnapshot);
// The file is no longer pending.
// Create permanent INode, update blocks. No need to replace the inode here
// since we just remove the uc feature from pendingFile
pendingFile.toCompleteFile(now());
waitForLoadingFSImage();
// close file and persist block allocations for this file
closeFile(src, pendingFile);
blockManager.checkReplication(pendingFile);
}
2. 如果未完成状态的block不是最后一个block,则直接抛出异常。
// Only the last and the penultimate blocks may be in non COMPLETE state.
// If the penultimate block is not COMPLETE, then it must be COMMITTED.
if(nrCompleteBlocks < nrBlocks - 2 ||
nrCompleteBlocks == nrBlocks - 2 &&
curBlock != null &&
curBlock.getBlockUCState() != BlockUCState.COMMITTED) {
final String message = "DIR* NameSystem.internalReleaseLease: "
+ "attempt to release a create lock on "
+ src + " but file is already closed.";
NameNode.stateChangeLog.warn(message);
throw new IOException(message);
}
3.然后检查最后一个block的状态:
3.1 如果处于COMMITTED状态,则检查最后两个块是否都满足最小副本数要求(默认为1),如果满足,也是调用finalizeINodeFileUnderConstruction方法,删除租约,关闭文件。 如果不满足,则直接抛异常:等待block满足最小副本数。
// If penultimate block doesn't exist then its minReplication is met
boolean penultimateBlockMinReplication = penultimateBlock == null ? true :
blockManager.checkMinReplication(penultimateBlock);
.....................
.....................
case COMMITTED:
// Close file if committed blocks are minimally replicated
if(penultimateBlockMinReplication &&
blockManager.checkMinReplication(lastBlock)) {
finalizeINodeFileUnderConstruction(src, pendingFile,
iip.getLatestSnapshotId());
NameNode.stateChangeLog.warn("BLOCK*"
+ " internalReleaseLease: Committed blocks are minimally replicated,"
+ " lease removed, file closed.");
return true; // closed!
}
// Cannot close file right now, since some blocks
// are not yet minimally replicated.
// This may potentially cause infinite loop in lease recovery
// if there are no valid replicas on data-nodes.
String message = "DIR* NameSystem.internalReleaseLease: " +
"Failed to release lease for file " + src +
". Committed blocks are waiting to be minimally replicated." +
" Try again later.";
NameNode.stateChangeLog.warn(message);
throw new AlreadyBeingCreatedException(message);
3.2 如果最后一个块处于UNDER_CONSTRUCTION或者UNDER_RECOVERY状态,
如果最后一个block没有任何datanode汇报上来,说明可能是pipeline还没有建立成功,这种情况下,只需要把最后一个block从INode中溢出,并且关闭文件.
if (uc.getNumExpectedLocations() == 0 && uc.getNumBytes() == 0) {
// There is no datanode reported to this block.
// may be client have crashed before writing data to pipeline.
// This blocks doesn't need any recovery.
// We can remove this block and close the file.
pendingFile.removeLastBlock(lastBlock);
finalizeINodeFileUnderConstruction(src, pendingFile,
iip.getLatestSnapshotId());
NameNode.stateChangeLog.warn("BLOCK* internalReleaseLease: "
+ "Removed empty last block and closed file.");
return true;
}
继续往下走,进入block recovery流程阶段:
a. 申请一个block recovery id来标示这次block recovery过程.block recovery id实际是一个新分配的generation stamp;
b. 然后调用initializeBlockRecovery方法,执行block recovery流程的初始化工作: 设置block状态为under recovery,从block的多个副本中选择一个副本所在的datanode作为primary data node,然后将这个block放入这个datanode的recoverBlocks列表中,随后,namenode在处理datanode的定期心跳中,会将这个datanode的所有的recoverBlocks都在心跳回复中发送给datanode,以BlockRecoveryCommand的形式.
datanodemanager: handlerHeartbeat
//check lease recovery
BlockInfoContiguousUnderConstruction[] blocks = nodeinfo
.getLeaseRecoveryCommand(Integer.MAX_VALUE);
if (blocks != null) {
BlockRecoveryCommand brCommand = new BlockRecoveryCommand(
blocks.length);
for (BlockInfoContiguousUnderConstruction b : blocks) {
final DatanodeStorageInfo[] storages = b.getExpectedStorageLocations();
// Skip stale nodes during recovery - not heart beated for some time (30s by default).
final List<DatanodeStorageInfo> recoveryLocations =
new ArrayList<DatanodeStorageInfo>(storages.length);
for (int i = 0; i < storages.length; i++) {
if (!storages[i].getDatanodeDescriptor().isStale(staleInterval)) {
recoveryLocations.add(storages[i]);
}
}
// If we are performing a truncate recovery than set recovery fields
// to old block.
boolean truncateRecovery = b.getTruncateBlock() != null;
boolean copyOnTruncateRecovery = truncateRecovery &&
b.getTruncateBlock().getBlockId() != b.getBlockId();
ExtendedBlock primaryBlock = (copyOnTruncateRecovery) ?
new ExtendedBlock(blockPoolId, b.getTruncateBlock()) :
new ExtendedBlock(blockPoolId, b);
// If we only get 1 replica after eliminating stale nodes, then choose all
// replicas for recovery and let the primary data node handle failures.
DatanodeInfo[] recoveryInfos;
if (recoveryLocations.size() > 1) {
if (recoveryLocations.size() != storages.length) {
LOG.info("Skipped stale nodes for recovery : " +
(storages.length - recoveryLocations.size()));
}
recoveryInfos =
DatanodeStorageInfo.toDatanodeInfos(recoveryLocations);
} else {
// If too many replicas are stale, then choose all replicas to participate
// in block recovery.
recoveryInfos = DatanodeStorageInfo.toDatanodeInfos(storages);
}
if(truncateRecovery) {
Block recoveryBlock = (copyOnTruncateRecovery) ? b :
b.getTruncateBlock();
brCommand.add(new RecoveringBlock(primaryBlock, recoveryInfos,
recoveryBlock));
} else {
brCommand.add(new RecoveringBlock(primaryBlock, recoveryInfos,
b.getBlockRecoveryId()));
}
}
return new DatanodeCommand[] { brCommand };
}
DataNode端:
DataNode端的BPServiceActor处理心跳回复,在offerService()函数中,从心跳回复中拿出所有的DataNodeCommand处理。在processCommandFromActive函数中检查,command类型是DNA_RECOVERBLOCK,说明是block recovery命令,调用DataNode的recoverBlocks处理.
case DatanodeProtocol.DNA_RECOVERBLOCK:
String who = "NameNode at " + actor.getNNSocketAddress();
dn.recoverBlocks(who, ((BlockRecoveryCommand)cmd).getRecoveringBlocks());
break;
datanode recoverBlocks: 启动一个后台线程专门来处理这件事,对于每个需要recover的block:
public Daemon recoverBlocks(
final String who,
final Collection<RecoveringBlock> blocks) {
Daemon d = new Daemon(threadGroup, new Runnable() {
/** Recover a list of blocks. It is run by the primary datanode. */
@Override
public void run() {
for(RecoveringBlock b : blocks) {
try {
logRecoverBlock(who, b);
recoverBlock(b);
} catch (IOException e) {
LOG.warn("recoverBlocks FAILED: " + b, e);
}
}
}
});
d.start();
return d;
}
进入recoverBlock方法:
/** Recover a block */
private void recoverBlock(RecoveringBlock rBlock) throws IOException {
ExtendedBlock block = rBlock.getBlock();
String blookPoolId = block.getBlockPoolId();
DatanodeID[] datanodeids = rBlock.getLocations();
List<BlockRecord> syncList = new ArrayList<BlockRecord>(datanodeids.length);
int errorCount = 0;
//check generation stamps
for(DatanodeID id : datanodeids) {
try {
BPOfferService bpos = blockPoolManager.get(blookPoolId);
DatanodeRegistration bpReg = bpos.bpRegistration;
InterDatanodeProtocol datanode = bpReg.equals(id)?
this: DataNode.createInterDataNodeProtocolProxy(id, getConf(),
dnConf.socketTimeout, dnConf.connectToDnViaHostname);
ReplicaRecoveryInfo info = callInitReplicaRecovery(datanode, rBlock);
if (info != null &&
info.getGenerationStamp() >= block.getGenerationStamp() &&
info.getNumBytes() > 0) {
syncList.add(new BlockRecord(id, datanode, info));
}
} catch (RecoveryInProgressException ripE) {
InterDatanodeProtocol.LOG.warn(
"Recovery for replica " + block + " on data-node " + id
+ " is already in progress. Recovery id = "
+ rBlock.getNewGenerationStamp() + " is aborted.", ripE);
return;
} catch (IOException e) {
++errorCount;
InterDatanodeProtocol.LOG.warn(
"Failed to obtain replica info for block (=" + block
+ ") from datanode (=" + id + ")", e);
}
}
if (errorCount == datanodeids.length) {
throw new IOException("All datanodes failed: block=" + block
+ ", datanodeids=" + Arrays.asList(datanodeids));
}
syncBlock(rBlock, syncList);
}
从recoveringBlock上拿到所有副本所在节点信息,和其他副本所在节点建立连接,datanode交互的接口定义在InterDatanodeProtocol中;
然后调用callInitReplicaRecovery方法初始化datanode节点上需要恢复的replica.
/**
* Convenience method, which unwraps RemoteException.
* @throws IOException not a RemoteException.
*/
private static ReplicaRecoveryInfo callInitReplicaRecovery(
InterDatanodeProtocol datanode,
RecoveringBlock rBlock) throws IOException {
try {
return datanode.initReplicaRecovery(rBlock);
} catch(RemoteException re) {
throw re.unwrapRemoteException();
}
}
最终调用FsDatasetImpl下initReplicaRecovery方法:
/** static version of {@link #initReplicaRecovery(RecoveringBlock)}. */
static ReplicaRecoveryInfo initReplicaRecovery(String bpid, ReplicaMap map,
Block block, long recoveryId, long xceiverStopTimeout) throws IOException {
final ReplicaInfo replica = map.get(bpid, block.getBlockId());
LOG.info("initReplicaRecovery: " + block + ", recoveryId=" + recoveryId
+ ", replica=" + replica);
//check replica
if (replica == null) {
return null;
}
//stop writer if there is any
if (replica instanceof ReplicaInPipeline) {
final ReplicaInPipeline rip = (ReplicaInPipeline)replica;
rip.stopWriter(xceiverStopTimeout);
//check replica bytes on disk.
if (rip.getBytesOnDisk() < rip.getVisibleLength()) {
throw new IOException("THIS IS NOT SUPPOSED TO HAPPEN:"
+ " getBytesOnDisk() < getVisibleLength(), rip=" + rip);
}
//check the replica's files
checkReplicaFiles(rip);
}
//check generation stamp
if (replica.getGenerationStamp() < block.getGenerationStamp()) {
throw new IOException(
"replica.getGenerationStamp() < block.getGenerationStamp(), block="
+ block + ", replica=" + replica);
}
//check recovery id
if (replica.getGenerationStamp() >= recoveryId) {
throw new IOException("THIS IS NOT SUPPOSED TO HAPPEN:"
+ " replica.getGenerationStamp() >= recoveryId = " + recoveryId
+ ", block=" + block + ", replica=" + replica);
}
//check RUR
final ReplicaUnderRecovery rur;
if (replica.getState() == ReplicaState.RUR) {
rur = (ReplicaUnderRecovery)replica;
if (rur.getRecoveryID() >= recoveryId) {
throw new RecoveryInProgressException(
"rur.getRecoveryID() >= recoveryId = " + recoveryId
+ ", block=" + block + ", rur=" + rur);
}
final long oldRecoveryID = rur.getRecoveryID();
rur.setRecoveryID(recoveryId);
LOG.info("initReplicaRecovery: update recovery id for " + block
+ " from " + oldRecoveryID + " to " + recoveryId);
}
else {
rur = new ReplicaUnderRecovery(replica, recoveryId);
map.add(bpid, rur);
LOG.info("initReplicaRecovery: changing replica state for "
+ block + " from " + replica.getState()
+ " to " + rur.getState());
}
return rur.createInfo();
}
首先,检查副本的状态,如果当前副本的状态是正在写的过程中,那么调用replica的stopWriter停止这个写线程,停止的方法就是interupt这个写线程(写pipeline时,datanode创建replica时会将当前写线程的handle存到replica中),从这可以看出blcok recovery优先级很高。然后做一些check,比如副本在磁盘上的文件是否存在,meta文件是否存在等,然后,检查generation stamp,namenode记录的generation stamp不能比实际的大,recovery id不能比副本的generation stamp小,最后,创建一个ReplicaUnderRecovery,放入replica map中,这里还会检查,如果replica已经处于under recovery状态,则看当前的block recovery过程的recovery id和它谁大,如果更大,则强占它。
接着,将三个副本的信息(包括recovery前的副本的信息)都加入一个列表,然后开始sync,sync就是根据三个副本的原来的状态,做一些选择,规则如下,这是两个副本的情况: