|
@@ -17,16 +17,21 @@
|
|
*/
|
|
*/
|
|
package org.apache.hadoop.hdfs.server.datanode;
|
|
package org.apache.hadoop.hdfs.server.datanode;
|
|
|
|
|
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
import com.google.common.base.Joiner;
|
|
import com.google.common.base.Joiner;
|
|
|
|
+import com.google.common.base.Preconditions;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
|
|
import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
|
|
import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
|
|
import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
|
|
|
|
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
|
|
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
|
|
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
|
|
|
|
+import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringStripedBlock;
|
|
import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
|
|
import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
|
|
import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
|
|
import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
|
|
import org.apache.hadoop.ipc.RemoteException;
|
|
import org.apache.hadoop.ipc.RemoteException;
|
|
@@ -37,7 +42,12 @@ import java.io.IOException;
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
import java.util.Arrays;
|
|
import java.util.Arrays;
|
|
import java.util.Collection;
|
|
import java.util.Collection;
|
|
|
|
+import java.util.HashMap;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
|
|
+import java.util.Map;
|
|
|
|
+
|
|
|
|
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BLOCK_GROUP_INDEX_MASK;
|
|
|
|
+import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getInternalBlockLength;
|
|
|
|
|
|
/**
|
|
/**
|
|
* This class handles the block recovery work commands.
|
|
* This class handles the block recovery work commands.
|
|
@@ -78,6 +88,10 @@ public class BlockRecoveryWorker {
|
|
newLength);
|
|
newLength);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ public ReplicaRecoveryInfo getReplicaRecoveryInfo(){
|
|
|
|
+ return rInfo;
|
|
|
|
+ }
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public String toString() {
|
|
public String toString() {
|
|
return "block:" + rInfo + " node:" + id;
|
|
return "block:" + rInfo + " node:" + id;
|
|
@@ -294,12 +308,8 @@ public class BlockRecoveryWorker {
|
|
// we never know the actual state of the replica on failed data-nodes.
|
|
// we never know the actual state of the replica on failed data-nodes.
|
|
// The recovery should be started over.
|
|
// The recovery should be started over.
|
|
if (!failedList.isEmpty()) {
|
|
if (!failedList.isEmpty()) {
|
|
- StringBuilder b = new StringBuilder();
|
|
|
|
- for(DatanodeID id : failedList) {
|
|
|
|
- b.append("\n " + id);
|
|
|
|
- }
|
|
|
|
- throw new IOException("Cannot recover " + block + ", the following "
|
|
|
|
- + failedList.size() + " data-nodes failed {" + b + "\n}");
|
|
|
|
|
|
+ throw new IOException("Cannot recover " + block
|
|
|
|
+ + ", the following datanodes failed: " + failedList);
|
|
}
|
|
}
|
|
|
|
|
|
// Notify the name-node about successfully recovered replicas.
|
|
// Notify the name-node about successfully recovered replicas.
|
|
@@ -323,6 +333,215 @@ public class BlockRecoveryWorker {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * blk_0 blk_1 blk_2 blk_3 blk_4 blk_5 blk_6 blk_7 blk_8
|
|
|
|
+ * 64k 64k 64k 64k 64k 64k 64k 64k 64k <-- stripe_0
|
|
|
|
+ * 64k 64k 64k 64k 64k 64k 64k 64k 64k
|
|
|
|
+ * 64k 64k 64k 64k 64k 64k 64k 61k <-- startStripeIdx
|
|
|
|
+ * 64k 64k 64k 64k 64k 64k 64k
|
|
|
|
+ * 64k 64k 64k 64k 64k 64k 59k
|
|
|
|
+ * 64k 64k 64k 64k 64k 64k
|
|
|
|
+ * 64k 64k 64k 64k 64k 64k <-- last full stripe
|
|
|
|
+ * 64k 64k 13k 64k 55k 3k <-- target last stripe
|
|
|
|
+ * 64k 64k 64k 1k
|
|
|
|
+ * 64k 64k 58k
|
|
|
|
+ * 64k 64k
|
|
|
|
+ * 64k 19k
|
|
|
|
+ * 64k <-- total visible stripe
|
|
|
|
+ *
|
|
|
|
+ * Due to different speed of streamers, the internal blocks in a block group
|
|
|
|
+ * could have different lengths when the block group isn't ended normally.
|
|
|
|
+ * The purpose of this class is to recover the UnderConstruction block group,
|
|
|
|
+ * so all internal blocks end at the same stripe.
|
|
|
|
+ *
|
|
|
|
+ * The steps:
|
|
|
|
+ * 1. get all blocks lengths from DataNodes.
|
|
|
|
+ * 2. calculate safe length, which is at the target last stripe.
|
|
|
|
+ * 3. decode and feed blk_6~8, make them end at last full stripe. (the last
|
|
|
|
+ * full stripe means the last decodable stripe.)
|
|
|
|
+ * 4. encode the target last stripe, with the remaining sequential data. In
|
|
|
|
+ * this case, the sequential data is 64k+64k+13k. Feed blk_6~8 the parity cells.
|
|
|
|
+ * Overwrite the parity cell if have to.
|
|
|
|
+ * 5. truncate the stripes from visible stripe, to target last stripe.
|
|
|
|
+ * TODO: implement step 3,4
|
|
|
|
+ */
|
|
|
|
+ public class RecoveryTaskStriped {
|
|
|
|
+ private final RecoveringBlock rBlock;
|
|
|
|
+ private final ExtendedBlock block;
|
|
|
|
+ private final String bpid;
|
|
|
|
+ private final DatanodeInfo[] locs;
|
|
|
|
+ private final long recoveryId;
|
|
|
|
+
|
|
|
|
+ private final int[] blockIndices;
|
|
|
|
+ private final ErasureCodingPolicy ecPolicy;
|
|
|
|
+
|
|
|
|
+ RecoveryTaskStriped(RecoveringStripedBlock rBlock) {
|
|
|
|
+ this.rBlock = rBlock;
|
|
|
|
+ // TODO: support truncate
|
|
|
|
+ Preconditions.checkArgument(rBlock.getNewBlock() == null);
|
|
|
|
+
|
|
|
|
+ block = rBlock.getBlock();
|
|
|
|
+ bpid = block.getBlockPoolId();
|
|
|
|
+ locs = rBlock.getLocations();
|
|
|
|
+ recoveryId = rBlock.getNewGenerationStamp();
|
|
|
|
+ blockIndices = rBlock.getBlockIndices();
|
|
|
|
+ ecPolicy = rBlock.getErasureCodingPolicy();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ protected void recover() throws IOException {
|
|
|
|
+ checkLocations(locs.length);
|
|
|
|
+
|
|
|
|
+ Map<Long, BlockRecord> syncBlocks = new HashMap<>(locs.length);
|
|
|
|
+ final int dataBlkNum = ecPolicy.getNumDataUnits();
|
|
|
|
+ final int totalBlkNum = dataBlkNum + ecPolicy.getNumParityUnits();
|
|
|
|
+ //check generation stamps
|
|
|
|
+ for (int i = 0; i < locs.length; i++) {
|
|
|
|
+ DatanodeID id = locs[i];
|
|
|
|
+ try {
|
|
|
|
+ DatanodeID bpReg = new DatanodeID(
|
|
|
|
+ datanode.getBPOfferService(bpid).bpRegistration);
|
|
|
|
+ InterDatanodeProtocol proxyDN = bpReg.equals(id) ?
|
|
|
|
+ datanode : DataNode.createInterDataNodeProtocolProxy(id, conf,
|
|
|
|
+ dnConf.socketTimeout, dnConf.connectToDnViaHostname);
|
|
|
|
+ ExtendedBlock internalBlk = new ExtendedBlock(block);
|
|
|
|
+ final long blockId = block.getBlockId() + blockIndices[i];
|
|
|
|
+ internalBlk.setBlockId(blockId);
|
|
|
|
+ ReplicaRecoveryInfo info = callInitReplicaRecovery(proxyDN,
|
|
|
|
+ new RecoveringBlock(internalBlk, null, recoveryId));
|
|
|
|
+
|
|
|
|
+ if (info != null &&
|
|
|
|
+ info.getGenerationStamp() >= block.getGenerationStamp() &&
|
|
|
|
+ info.getNumBytes() > 0) {
|
|
|
|
+ final BlockRecord existing = syncBlocks.get(blockId);
|
|
|
|
+ if (existing == null ||
|
|
|
|
+ info.getNumBytes() > existing.rInfo.getNumBytes()) {
|
|
|
|
+ // if we have >1 replicas for the same internal block, we
|
|
|
|
+ // simply choose the one with larger length.
|
|
|
|
+ // TODO: better usage of redundant replicas
|
|
|
|
+ syncBlocks.put(blockId, new BlockRecord(id, proxyDN, 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) {
|
|
|
|
+ InterDatanodeProtocol.LOG.warn(
|
|
|
|
+ "Failed to obtain replica info for block (=" + block
|
|
|
|
+ + ") from datanode (=" + id + ")", e);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ checkLocations(syncBlocks.size());
|
|
|
|
+
|
|
|
|
+ final long safeLength = getSafeLength(syncBlocks);
|
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
|
+ LOG.debug("Recovering block " + block
|
|
|
|
+ + ", length=" + block.getNumBytes() + ", safeLength=" + safeLength
|
|
|
|
+ + ", syncList=" + syncBlocks);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // If some internal blocks reach the safe length, convert them to RUR
|
|
|
|
+ List<BlockRecord> rurList = new ArrayList<>(locs.length);
|
|
|
|
+ for (BlockRecord r : syncBlocks.values()) {
|
|
|
|
+ int blockIndex = (int) (r.rInfo.getBlockId() & BLOCK_GROUP_INDEX_MASK);
|
|
|
|
+ long newSize = getInternalBlockLength(safeLength, ecPolicy.getCellSize(),
|
|
|
|
+ dataBlkNum, blockIndex);
|
|
|
|
+ if (r.rInfo.getNumBytes() >= newSize) {
|
|
|
|
+ rurList.add(r);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ assert rurList.size() >= dataBlkNum : "incorrect safe length";
|
|
|
|
+
|
|
|
|
+ // Recovery the striped block by truncating internal blocks to the safe
|
|
|
|
+ // length. Abort if there is any failure in this step.
|
|
|
|
+ truncatePartialBlock(rurList, safeLength);
|
|
|
|
+
|
|
|
|
+ // notify Namenode the new size and locations
|
|
|
|
+ final DatanodeID[] newLocs = new DatanodeID[totalBlkNum];
|
|
|
|
+ final String[] newStorages = new String[totalBlkNum];
|
|
|
|
+ for (int i = 0; i < totalBlkNum; i++) {
|
|
|
|
+ newLocs[blockIndices[i]] = DatanodeID.EMPTY_DATANODE_ID;
|
|
|
|
+ newStorages[blockIndices[i]] = "";
|
|
|
|
+ }
|
|
|
|
+ for (BlockRecord r : rurList) {
|
|
|
|
+ int index = (int) (r.rInfo.getBlockId() &
|
|
|
|
+ HdfsServerConstants.BLOCK_GROUP_INDEX_MASK);
|
|
|
|
+ newLocs[index] = r.id;
|
|
|
|
+ newStorages[index] = r.storageID;
|
|
|
|
+ }
|
|
|
|
+ ExtendedBlock newBlock = new ExtendedBlock(bpid, block.getBlockId(),
|
|
|
|
+ safeLength, recoveryId);
|
|
|
|
+ DatanodeProtocolClientSideTranslatorPB nn = getActiveNamenodeForBP(bpid);
|
|
|
|
+ nn.commitBlockSynchronization(block, newBlock.getGenerationStamp(),
|
|
|
|
+ newBlock.getNumBytes(), true, false, newLocs, newStorages);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void truncatePartialBlock(List<BlockRecord> rurList,
|
|
|
|
+ long safeLength) throws IOException {
|
|
|
|
+ int cellSize = ecPolicy.getCellSize();
|
|
|
|
+ int dataBlkNum = ecPolicy.getNumDataUnits();
|
|
|
|
+ List<DatanodeID> failedList = new ArrayList<>();
|
|
|
|
+ for (BlockRecord r : rurList) {
|
|
|
|
+ int blockIndex = (int) (r.rInfo.getBlockId() & BLOCK_GROUP_INDEX_MASK);
|
|
|
|
+ long newSize = getInternalBlockLength(safeLength, cellSize, dataBlkNum,
|
|
|
|
+ blockIndex);
|
|
|
|
+ try {
|
|
|
|
+ r.updateReplicaUnderRecovery(bpid, recoveryId, r.rInfo.getBlockId(),
|
|
|
|
+ newSize);
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ InterDatanodeProtocol.LOG.warn("Failed to updateBlock (newblock="
|
|
|
|
+ + ", datanode=" + r.id + ")", e);
|
|
|
|
+ failedList.add(r.id);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // If any of the data-nodes failed, the recovery fails, because
|
|
|
|
+ // we never know the actual state of the replica on failed data-nodes.
|
|
|
|
+ // The recovery should be started over.
|
|
|
|
+ if (!failedList.isEmpty()) {
|
|
|
|
+ throw new IOException("Cannot recover " + block
|
|
|
|
+ + ", the following datanodes failed: " + failedList);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * TODO: the current implementation depends on the assumption that the
|
|
|
|
+ * parity cells are only generated based on the full stripe. This is not
|
|
|
|
+ * true after we support hflush.
|
|
|
|
+ */
|
|
|
|
+ @VisibleForTesting
|
|
|
|
+ long getSafeLength(Map<Long, BlockRecord> syncBlocks) {
|
|
|
|
+ final int cellSize = ecPolicy.getCellSize();
|
|
|
|
+ final int dataBlkNum = ecPolicy.getNumDataUnits();
|
|
|
|
+ Preconditions.checkArgument(syncBlocks.size() >= dataBlkNum);
|
|
|
|
+ final int stripeSize = dataBlkNum * cellSize;
|
|
|
|
+ long[] blockLengths = new long[syncBlocks.size()];
|
|
|
|
+ int i = 0;
|
|
|
|
+ for (BlockRecord r : syncBlocks.values()) {
|
|
|
|
+ ReplicaRecoveryInfo rInfo = r.getReplicaRecoveryInfo();
|
|
|
|
+ blockLengths[i++] = rInfo.getNumBytes();
|
|
|
|
+ }
|
|
|
|
+ Arrays.sort(blockLengths);
|
|
|
|
+ // full stripe is a stripe has at least dataBlkNum full cells.
|
|
|
|
+ // lastFullStripeIdx is the index of the last full stripe.
|
|
|
|
+ int lastFullStripeIdx =
|
|
|
|
+ (int) (blockLengths[blockLengths.length - dataBlkNum] / cellSize);
|
|
|
|
+ return lastFullStripeIdx * stripeSize; // return the safeLength
|
|
|
|
+ // TODO: Include lastFullStripeIdx+1 stripe in safeLength, if there exists
|
|
|
|
+ // such a stripe (and it must be partial).
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void checkLocations(int locationCount)
|
|
|
|
+ throws IOException {
|
|
|
|
+ if (locationCount < ecPolicy.getNumDataUnits()) {
|
|
|
|
+ throw new IOException(block + " has no enough internal blocks" +
|
|
|
|
+ ", unable to start recovery. Locations=" + Arrays.asList(locs));
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
private static void logRecoverBlock(String who, RecoveringBlock rb) {
|
|
private static void logRecoverBlock(String who, RecoveringBlock rb) {
|
|
ExtendedBlock block = rb.getBlock();
|
|
ExtendedBlock block = rb.getBlock();
|
|
DatanodeInfo[] targets = rb.getLocations();
|
|
DatanodeInfo[] targets = rb.getLocations();
|
|
@@ -379,8 +598,11 @@ public class BlockRecoveryWorker {
|
|
for(RecoveringBlock b : blocks) {
|
|
for(RecoveringBlock b : blocks) {
|
|
try {
|
|
try {
|
|
logRecoverBlock(who, b);
|
|
logRecoverBlock(who, b);
|
|
- RecoveryTaskContiguous task = new RecoveryTaskContiguous(b);
|
|
|
|
- task.recover();
|
|
|
|
|
|
+ if (b.isStriped()) {
|
|
|
|
+ new RecoveryTaskStriped((RecoveringStripedBlock) b).recover();
|
|
|
|
+ } else {
|
|
|
|
+ new RecoveryTaskContiguous(b).recover();
|
|
|
|
+ }
|
|
} catch (IOException e) {
|
|
} catch (IOException e) {
|
|
LOG.warn("recoverBlocks FAILED: " + b, e);
|
|
LOG.warn("recoverBlocks FAILED: " + b, e);
|
|
}
|
|
}
|