|
@@ -67,7 +67,7 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactor
|
|
|
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
|
-import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
|
|
|
+import org.apache.hadoop.hdfs.server.protocol.BlockECReconstructionCommand.BlockECReconstructionInfo;
|
|
|
import org.apache.hadoop.hdfs.util.StripedBlockUtil;
|
|
|
import org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunkReadResult;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
@@ -83,10 +83,10 @@ import com.google.common.base.Preconditions;
|
|
|
import org.slf4j.Logger;
|
|
|
|
|
|
/**
|
|
|
- * ErasureCodingWorker handles the erasure coding recovery work commands. These
|
|
|
- * commands would be issued from Namenode as part of Datanode's heart beat
|
|
|
- * response. BPOfferService delegates the work to this class for handling EC
|
|
|
- * commands.
|
|
|
+ * ErasureCodingWorker handles the erasure coding reconstruction work commands.
|
|
|
+ * These commands would be issued from Namenode as part of Datanode's heart
|
|
|
+ * beat response. BPOfferService delegates the work to this class for handling
|
|
|
+ * EC commands.
|
|
|
*/
|
|
|
@InterfaceAudience.Private
|
|
|
public final class ErasureCodingWorker {
|
|
@@ -95,28 +95,28 @@ public final class ErasureCodingWorker {
|
|
|
private final DataNode datanode;
|
|
|
private final Configuration conf;
|
|
|
|
|
|
- private ThreadPoolExecutor STRIPED_BLK_RECOVERY_THREAD_POOL;
|
|
|
- private ThreadPoolExecutor STRIPED_READ_THREAD_POOL;
|
|
|
- private final int STRIPED_READ_TIMEOUT_MILLIS;
|
|
|
- private final int STRIPED_READ_BUFFER_SIZE;
|
|
|
+ private ThreadPoolExecutor EC_RECONSTRUCTION_STRIPED_BLK_THREAD_POOL;
|
|
|
+ private ThreadPoolExecutor EC_RECONSTRUCTION_STRIPED_READ_THREAD_POOL;
|
|
|
+ private final int EC_RECONSTRUCTION_STRIPED_READ_TIMEOUT_MILLIS;
|
|
|
+ private final int EC_RECONSTRUCTION_STRIPED_READ_BUFFER_SIZE;
|
|
|
|
|
|
public ErasureCodingWorker(Configuration conf, DataNode datanode) {
|
|
|
this.datanode = datanode;
|
|
|
this.conf = conf;
|
|
|
|
|
|
- STRIPED_READ_TIMEOUT_MILLIS = conf.getInt(
|
|
|
- DFSConfigKeys.DFS_DATANODE_STRIPED_READ_TIMEOUT_MILLIS_KEY,
|
|
|
- DFSConfigKeys.DFS_DATANODE_STRIPED_READ_TIMEOUT_MILLIS_DEFAULT);
|
|
|
+ EC_RECONSTRUCTION_STRIPED_READ_TIMEOUT_MILLIS = conf.getInt(
|
|
|
+ DFSConfigKeys.DFS_DN_EC_RECONSTRUCTION_STRIPED_READ_TIMEOUT_MILLIS_KEY,
|
|
|
+ DFSConfigKeys.DFS_DN_EC_RECONSTRUCTION_STRIPED_READ_TIMEOUT_MILLIS_DEFAULT);
|
|
|
initializeStripedReadThreadPool(conf.getInt(
|
|
|
- DFSConfigKeys.DFS_DATANODE_STRIPED_READ_THREADS_KEY,
|
|
|
- DFSConfigKeys.DFS_DATANODE_STRIPED_READ_THREADS_DEFAULT));
|
|
|
- STRIPED_READ_BUFFER_SIZE = conf.getInt(
|
|
|
- DFSConfigKeys.DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_KEY,
|
|
|
- DFSConfigKeys.DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_DEFAULT);
|
|
|
-
|
|
|
- initializeStripedBlkRecoveryThreadPool(conf.getInt(
|
|
|
- DFSConfigKeys.DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_KEY,
|
|
|
- DFSConfigKeys.DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_DEFAULT));
|
|
|
+ DFSConfigKeys.DFS_DN_EC_RECONSTRUCTION_STRIPED_READ_THREADS_KEY,
|
|
|
+ DFSConfigKeys.DFS_DN_EC_RECONSTRUCTION_STRIPED_READ_THREADS_DEFAULT));
|
|
|
+ EC_RECONSTRUCTION_STRIPED_READ_BUFFER_SIZE = conf.getInt(
|
|
|
+ DFSConfigKeys.DFS_DN_EC_RECONSTRUCTION_STRIPED_READ_BUFFER_SIZE_KEY,
|
|
|
+ DFSConfigKeys.DFS_DN_EC_RECONSTRUCTION_STRIPED_READ_BUFFER_SIZE_DEFAULT);
|
|
|
+
|
|
|
+ initializeStripedBlkReconstructionThreadPool(conf.getInt(
|
|
|
+ DFSConfigKeys.DFS_DN_EC_RECONSTRUCTION_STRIPED_BLK_THREADS_KEY,
|
|
|
+ DFSConfigKeys.DFS_DN_EC_RECONSTRUCTION_STRIPED_BLK_THREADS_DEFAULT));
|
|
|
}
|
|
|
|
|
|
private RawErasureDecoder newDecoder(int numDataUnits, int numParityUnits) {
|
|
@@ -126,8 +126,8 @@ public final class ErasureCodingWorker {
|
|
|
private void initializeStripedReadThreadPool(int num) {
|
|
|
LOG.debug("Using striped reads; pool threads=" + num);
|
|
|
|
|
|
- STRIPED_READ_THREAD_POOL = new ThreadPoolExecutor(1, num, 60,
|
|
|
- TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
|
|
|
+ EC_RECONSTRUCTION_STRIPED_READ_THREAD_POOL = new ThreadPoolExecutor(1, num,
|
|
|
+ 60, TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
|
|
|
new Daemon.DaemonFactory() {
|
|
|
private final AtomicInteger threadIndex = new AtomicInteger(0);
|
|
|
|
|
@@ -146,48 +146,50 @@ public final class ErasureCodingWorker {
|
|
|
super.rejectedExecution(runnable, e);
|
|
|
}
|
|
|
});
|
|
|
- STRIPED_READ_THREAD_POOL.allowCoreThreadTimeOut(true);
|
|
|
+ EC_RECONSTRUCTION_STRIPED_READ_THREAD_POOL.allowCoreThreadTimeOut(true);
|
|
|
}
|
|
|
|
|
|
- private void initializeStripedBlkRecoveryThreadPool(int num) {
|
|
|
- LOG.debug("Using striped block recovery; pool threads=" + num);
|
|
|
- STRIPED_BLK_RECOVERY_THREAD_POOL = new ThreadPoolExecutor(2, num, 60,
|
|
|
- TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>(),
|
|
|
+ private void initializeStripedBlkReconstructionThreadPool(int num) {
|
|
|
+ LOG.debug("Using striped block reconstruction; pool threads=" + num);
|
|
|
+ EC_RECONSTRUCTION_STRIPED_BLK_THREAD_POOL = new ThreadPoolExecutor(2, num,
|
|
|
+ 60, TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>(),
|
|
|
new Daemon.DaemonFactory() {
|
|
|
private final AtomicInteger threadIdx = new AtomicInteger(0);
|
|
|
|
|
|
@Override
|
|
|
public Thread newThread(Runnable r) {
|
|
|
Thread t = super.newThread(r);
|
|
|
- t.setName("stripedBlockRecovery-" + threadIdx.getAndIncrement());
|
|
|
+ t.setName(
|
|
|
+ "stripedBlockReconstruction-" + threadIdx.getAndIncrement());
|
|
|
return t;
|
|
|
}
|
|
|
});
|
|
|
- STRIPED_BLK_RECOVERY_THREAD_POOL.allowCoreThreadTimeOut(true);
|
|
|
+ EC_RECONSTRUCTION_STRIPED_BLK_THREAD_POOL.allowCoreThreadTimeOut(true);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Handles the Erasure Coding recovery work commands.
|
|
|
- *
|
|
|
+ * Handles the Erasure Coding reconstruction work commands.
|
|
|
+ *
|
|
|
* @param ecTasks
|
|
|
- * BlockECRecoveryInfo
|
|
|
+ * BlockECReconstructionInfo
|
|
|
*/
|
|
|
- public void processErasureCodingTasks(Collection<BlockECRecoveryInfo> ecTasks) {
|
|
|
- for (BlockECRecoveryInfo recoveryInfo : ecTasks) {
|
|
|
+ public void processErasureCodingTasks(
|
|
|
+ Collection<BlockECReconstructionInfo> ecTasks) {
|
|
|
+ for (BlockECReconstructionInfo reconstructionInfo : ecTasks) {
|
|
|
try {
|
|
|
- STRIPED_BLK_RECOVERY_THREAD_POOL
|
|
|
- .submit(new ReconstructAndTransferBlock(recoveryInfo));
|
|
|
+ EC_RECONSTRUCTION_STRIPED_BLK_THREAD_POOL
|
|
|
+ .submit(new ReconstructAndTransferBlock(reconstructionInfo));
|
|
|
} catch (Throwable e) {
|
|
|
- LOG.warn("Failed to recover striped block "
|
|
|
- + recoveryInfo.getExtendedBlock().getLocalBlock(), e);
|
|
|
+ LOG.warn("Failed to reconstruct striped block "
|
|
|
+ + reconstructionInfo.getExtendedBlock().getLocalBlock(), e);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * ReconstructAndTransferBlock recover one or more missed striped block in the
|
|
|
- * striped block group, the minimum number of live striped blocks should be
|
|
|
- * no less than data block number.
|
|
|
+ * ReconstructAndTransferBlock reconstruct one or more missed striped block
|
|
|
+ * in the striped block group, the minimum number of live striped blocks
|
|
|
+ * should be no less than data block number.
|
|
|
*
|
|
|
* | <- Striped Block Group -> |
|
|
|
* blk_0 blk_1 blk_2(*) blk_3 ... <- A striped block group
|
|
@@ -203,12 +205,12 @@ public final class ErasureCodingWorker {
|
|
|
* ... ... ... ...
|
|
|
*
|
|
|
*
|
|
|
- * We use following steps to recover striped block group, in each round, we
|
|
|
- * recover <code>bufferSize</code> data until finish, the
|
|
|
+ * We use following steps to reconstruct striped block group, in each round,
|
|
|
+ * we reconstruct <code>bufferSize</code> data until finish, the
|
|
|
* <code>bufferSize</code> is configurable and may be less or larger than
|
|
|
* cell size:
|
|
|
* step1: read <code>bufferSize</code> data from minimum number of sources
|
|
|
- * required by recovery.
|
|
|
+ * required by reconstruction.
|
|
|
* step2: decode data for targets.
|
|
|
* step3: transfer data to targets.
|
|
|
*
|
|
@@ -217,25 +219,25 @@ public final class ErasureCodingWorker {
|
|
|
* will be scheduled. The best sources are remembered for next round and
|
|
|
* may be updated in each round.
|
|
|
*
|
|
|
- * In step2, typically if source blocks we read are all data blocks, we
|
|
|
+ * In step2, typically if source blocks we read are all data blocks, we
|
|
|
* need to call encode, and if there is one parity block, we need to call
|
|
|
- * decode. Notice we only read once and recover all missed striped block
|
|
|
+ * decode. Notice we only read once and reconstruct all missed striped block
|
|
|
* if they are more than one.
|
|
|
*
|
|
|
- * In step3, send the recovered data to targets by constructing packet
|
|
|
- * and send them directly. Same as continuous block replication, we
|
|
|
- * don't check the packet ack. Since the datanode doing the recovery work
|
|
|
- * are one of the source datanodes, so the recovered data are sent
|
|
|
+ * In step3, send the reconstructed data to targets by constructing packet
|
|
|
+ * and send them directly. Same as continuous block replication, we
|
|
|
+ * don't check the packet ack. Since the datanode doing the reconstruction
|
|
|
+ * work are one of the source datanodes, so the reconstructed data are sent
|
|
|
* remotely.
|
|
|
*
|
|
|
* There are some points we can do further improvements in next phase:
|
|
|
* 1. we can read the block file directly on the local datanode,
|
|
|
* currently we use remote block reader. (Notice short-circuit is not
|
|
|
* a good choice, see inline comments).
|
|
|
- * 2. We need to check the packet ack for EC recovery? Since EC recovery
|
|
|
- * is more expensive than continuous block replication, it needs to
|
|
|
- * read from several other datanodes, should we make sure the
|
|
|
- * recovered result received by targets?
|
|
|
+ * 2. We need to check the packet ack for EC reconstruction? Since EC
|
|
|
+ * reconstruction is more expensive than continuous block replication,
|
|
|
+ * it needs to read from several other datanodes, should we make sure
|
|
|
+ * the reconstructed result received by targets?
|
|
|
*/
|
|
|
private class ReconstructAndTransferBlock implements Runnable {
|
|
|
private final int dataBlkNum;
|
|
@@ -288,20 +290,22 @@ public final class ErasureCodingWorker {
|
|
|
|
|
|
private final Map<Future<Void>, Integer> futures = new HashMap<>();
|
|
|
private final CompletionService<Void> readService =
|
|
|
- new ExecutorCompletionService<>(STRIPED_READ_THREAD_POOL);
|
|
|
+ new ExecutorCompletionService<>(
|
|
|
+ EC_RECONSTRUCTION_STRIPED_READ_THREAD_POOL);
|
|
|
|
|
|
- ReconstructAndTransferBlock(BlockECRecoveryInfo recoveryInfo) {
|
|
|
- ErasureCodingPolicy ecPolicy = recoveryInfo.getErasureCodingPolicy();
|
|
|
+ ReconstructAndTransferBlock(BlockECReconstructionInfo reconstructionInfo) {
|
|
|
+ ErasureCodingPolicy ecPolicy = reconstructionInfo
|
|
|
+ .getErasureCodingPolicy();
|
|
|
dataBlkNum = ecPolicy.getNumDataUnits();
|
|
|
parityBlkNum = ecPolicy.getNumParityUnits();
|
|
|
cellSize = ecPolicy.getCellSize();
|
|
|
|
|
|
- blockGroup = recoveryInfo.getExtendedBlock();
|
|
|
+ blockGroup = reconstructionInfo.getExtendedBlock();
|
|
|
final int cellsNum = (int)((blockGroup.getNumBytes() - 1) / cellSize + 1);
|
|
|
minRequiredSources = Math.min(cellsNum, dataBlkNum);
|
|
|
|
|
|
- liveIndices = recoveryInfo.getLiveBlockIndices();
|
|
|
- sources = recoveryInfo.getSourceDnInfos();
|
|
|
+ liveIndices = reconstructionInfo.getLiveBlockIndices();
|
|
|
+ sources = reconstructionInfo.getSourceDnInfos();
|
|
|
stripedReaders = new ArrayList<>(sources.length);
|
|
|
|
|
|
Preconditions.checkArgument(liveIndices.length >= minRequiredSources,
|
|
@@ -315,8 +319,8 @@ public final class ErasureCodingWorker {
|
|
|
zeroStripeIndices = new short[dataBlkNum - minRequiredSources];
|
|
|
}
|
|
|
|
|
|
- targets = recoveryInfo.getTargetDnInfos();
|
|
|
- targetStorageTypes = recoveryInfo.getTargetStorageTypes();
|
|
|
+ targets = reconstructionInfo.getTargetDnInfos();
|
|
|
+ targetStorageTypes = reconstructionInfo.getTargetStorageTypes();
|
|
|
targetIndices = new short[targets.length];
|
|
|
targetBuffers = new ByteBuffer[targets.length];
|
|
|
|
|
@@ -402,7 +406,7 @@ public final class ErasureCodingWorker {
|
|
|
|
|
|
if (nsuccess < minRequiredSources) {
|
|
|
String error = "Can't find minimum sources required by "
|
|
|
- + "recovery, block id: " + blockGroup.getBlockId();
|
|
|
+ + "reconstruction, block id: " + blockGroup.getBlockId();
|
|
|
throw new IOException(error);
|
|
|
}
|
|
|
|
|
@@ -441,21 +445,21 @@ public final class ErasureCodingWorker {
|
|
|
getBlockLen(blockGroup, targetIndex));
|
|
|
}
|
|
|
while (positionInBlock < maxTargetLength) {
|
|
|
- final int toRecover = (int) Math.min(
|
|
|
+ final int toReconstruct = (int) Math.min(
|
|
|
bufferSize, maxTargetLength - positionInBlock);
|
|
|
// step1: read from minimum source DNs required for reconstruction.
|
|
|
// The returned success list is the source DNs we do real read from
|
|
|
Map<ExtendedBlock, Set<DatanodeInfo>> corruptionMap = new HashMap<>();
|
|
|
try {
|
|
|
- success = readMinimumStripedData4Recovery(success, toRecover,
|
|
|
- corruptionMap);
|
|
|
+ success = readMinimumStripedData4Reconstruction(success,
|
|
|
+ toReconstruct, corruptionMap);
|
|
|
} finally {
|
|
|
// report corrupted blocks to NN
|
|
|
reportCorruptedBlocks(corruptionMap);
|
|
|
}
|
|
|
|
|
|
// step2: decode to reconstruct targets
|
|
|
- recoverTargets(success, targetsStatus, toRecover);
|
|
|
+ reconstructTargets(success, targetsStatus, toReconstruct);
|
|
|
|
|
|
// step3: transfer data
|
|
|
if (transferData2Targets(targetsStatus) == 0) {
|
|
@@ -464,7 +468,7 @@ public final class ErasureCodingWorker {
|
|
|
}
|
|
|
|
|
|
clearBuffers();
|
|
|
- positionInBlock += toRecover;
|
|
|
+ positionInBlock += toReconstruct;
|
|
|
}
|
|
|
|
|
|
endTargetBlocks(targetsStatus);
|
|
@@ -472,7 +476,7 @@ public final class ErasureCodingWorker {
|
|
|
// Currently we don't check the acks for packets, this is similar as
|
|
|
// block replication.
|
|
|
} catch (Throwable e) {
|
|
|
- LOG.warn("Failed to recover striped block: " + blockGroup, e);
|
|
|
+ LOG.warn("Failed to reconstruct striped block: " + blockGroup, e);
|
|
|
} finally {
|
|
|
datanode.decrementXmitsInProgress();
|
|
|
// close block readers
|
|
@@ -493,7 +497,7 @@ public final class ErasureCodingWorker {
|
|
|
checksum = blockReader.getDataChecksum();
|
|
|
bytesPerChecksum = checksum.getBytesPerChecksum();
|
|
|
// The bufferSize is flat to divide bytesPerChecksum
|
|
|
- int readBufferSize = STRIPED_READ_BUFFER_SIZE;
|
|
|
+ int readBufferSize = EC_RECONSTRUCTION_STRIPED_READ_BUFFER_SIZE;
|
|
|
bufferSize = readBufferSize < bytesPerChecksum ? bytesPerChecksum :
|
|
|
readBufferSize - readBufferSize % bytesPerChecksum;
|
|
|
} else {
|
|
@@ -521,11 +525,11 @@ public final class ErasureCodingWorker {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /** the reading length should not exceed the length for recovery */
|
|
|
- private int getReadLength(int index, int recoverLength) {
|
|
|
+ /** the reading length should not exceed the length for reconstruction. */
|
|
|
+ private int getReadLength(int index, int reconstructLength) {
|
|
|
long blockLen = getBlockLen(blockGroup, index);
|
|
|
long remaining = blockLen - positionInBlock;
|
|
|
- return (int) Math.min(remaining, recoverLength);
|
|
|
+ return (int) Math.min(remaining, reconstructLength);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -538,15 +542,16 @@ public final class ErasureCodingWorker {
|
|
|
* operations and next iteration read.
|
|
|
*
|
|
|
* @param success the initial success list of source DNs we think best
|
|
|
- * @param recoverLength the length to recover.
|
|
|
+ * @param reconstructLength the length to reconstruct.
|
|
|
* @return updated success list of source DNs we do real read
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- private int[] readMinimumStripedData4Recovery(final int[] success,
|
|
|
- int recoverLength, Map<ExtendedBlock, Set<DatanodeInfo>> corruptionMap)
|
|
|
- throws IOException {
|
|
|
- Preconditions.checkArgument(recoverLength >= 0 &&
|
|
|
- recoverLength <= bufferSize);
|
|
|
+ private int[] readMinimumStripedData4Reconstruction(final int[] success,
|
|
|
+ int reconstructLength,
|
|
|
+ Map<ExtendedBlock, Set<DatanodeInfo>> corruptionMap)
|
|
|
+ throws IOException {
|
|
|
+ Preconditions.checkArgument(reconstructLength >= 0 &&
|
|
|
+ reconstructLength <= bufferSize);
|
|
|
int nsuccess = 0;
|
|
|
int[] newSuccess = new int[minRequiredSources];
|
|
|
BitSet used = new BitSet(sources.length);
|
|
@@ -557,7 +562,7 @@ public final class ErasureCodingWorker {
|
|
|
for (int i = 0; i < minRequiredSources; i++) {
|
|
|
StripedReader reader = stripedReaders.get(success[i]);
|
|
|
final int toRead = getReadLength(liveIndices[success[i]],
|
|
|
- recoverLength);
|
|
|
+ reconstructLength);
|
|
|
if (toRead > 0) {
|
|
|
Callable<Void> readCallable = readFromBlock(reader, reader.buffer,
|
|
|
toRead, corruptionMap);
|
|
@@ -573,9 +578,9 @@ public final class ErasureCodingWorker {
|
|
|
|
|
|
while (!futures.isEmpty()) {
|
|
|
try {
|
|
|
- StripingChunkReadResult result =
|
|
|
- StripedBlockUtil.getNextCompletedStripedRead(
|
|
|
- readService, futures, STRIPED_READ_TIMEOUT_MILLIS);
|
|
|
+ StripingChunkReadResult result = StripedBlockUtil
|
|
|
+ .getNextCompletedStripedRead(readService, futures,
|
|
|
+ EC_RECONSTRUCTION_STRIPED_READ_TIMEOUT_MILLIS);
|
|
|
int resultIndex = -1;
|
|
|
if (result.state == StripingChunkReadResult.SUCCESSFUL) {
|
|
|
resultIndex = result.index;
|
|
@@ -585,10 +590,12 @@ public final class ErasureCodingWorker {
|
|
|
StripedReader failedReader = stripedReaders.get(result.index);
|
|
|
closeBlockReader(failedReader.blockReader);
|
|
|
failedReader.blockReader = null;
|
|
|
- resultIndex = scheduleNewRead(used, recoverLength, corruptionMap);
|
|
|
+ resultIndex = scheduleNewRead(used, reconstructLength,
|
|
|
+ corruptionMap);
|
|
|
} else if (result.state == StripingChunkReadResult.TIMEOUT) {
|
|
|
// If timeout, we also schedule a new read.
|
|
|
- resultIndex = scheduleNewRead(used, recoverLength, corruptionMap);
|
|
|
+ resultIndex = scheduleNewRead(used, reconstructLength,
|
|
|
+ corruptionMap);
|
|
|
}
|
|
|
if (resultIndex >= 0) {
|
|
|
newSuccess[nsuccess++] = resultIndex;
|
|
@@ -643,20 +650,20 @@ public final class ErasureCodingWorker {
|
|
|
return Arrays.copyOf(result, m);
|
|
|
}
|
|
|
|
|
|
- private void recoverTargets(int[] success, boolean[] targetsStatus,
|
|
|
- int toRecoverLen) {
|
|
|
+ private void reconstructTargets(int[] success, boolean[] targetsStatus,
|
|
|
+ int toReconstructLen) {
|
|
|
initDecoderIfNecessary();
|
|
|
ByteBuffer[] inputs = new ByteBuffer[dataBlkNum + parityBlkNum];
|
|
|
for (int i = 0; i < success.length; i++) {
|
|
|
StripedReader reader = stripedReaders.get(success[i]);
|
|
|
ByteBuffer buffer = reader.buffer;
|
|
|
- paddingBufferToLen(buffer, toRecoverLen);
|
|
|
+ paddingBufferToLen(buffer, toReconstructLen);
|
|
|
inputs[reader.index] = (ByteBuffer)buffer.flip();
|
|
|
}
|
|
|
if (success.length < dataBlkNum) {
|
|
|
for (int i = 0; i < zeroStripeBuffers.length; i++) {
|
|
|
ByteBuffer buffer = zeroStripeBuffers[i];
|
|
|
- paddingBufferToLen(buffer, toRecoverLen);
|
|
|
+ paddingBufferToLen(buffer, toReconstructLen);
|
|
|
int index = zeroStripeIndices[i];
|
|
|
inputs[index] = (ByteBuffer)buffer.flip();
|
|
|
}
|
|
@@ -666,7 +673,7 @@ public final class ErasureCodingWorker {
|
|
|
int m = 0;
|
|
|
for (int i = 0; i < targetBuffers.length; i++) {
|
|
|
if (targetsStatus[i]) {
|
|
|
- targetBuffers[i].limit(toRecoverLen);
|
|
|
+ targetBuffers[i].limit(toReconstructLen);
|
|
|
outputs[m++] = targetBuffers[i];
|
|
|
}
|
|
|
}
|
|
@@ -678,7 +685,7 @@ public final class ErasureCodingWorker {
|
|
|
long remaining = blockLen - positionInBlock;
|
|
|
if (remaining <= 0) {
|
|
|
targetBuffers[i].limit(0);
|
|
|
- } else if (remaining < toRecoverLen) {
|
|
|
+ } else if (remaining < toReconstructLen) {
|
|
|
targetBuffers[i].limit((int)remaining);
|
|
|
}
|
|
|
}
|
|
@@ -696,7 +703,7 @@ public final class ErasureCodingWorker {
|
|
|
* @param used the used source DNs in this iteration.
|
|
|
* @return the array index of source DN if don't need to do real read.
|
|
|
*/
|
|
|
- private int scheduleNewRead(BitSet used, int recoverLength,
|
|
|
+ private int scheduleNewRead(BitSet used, int reconstructLen,
|
|
|
Map<ExtendedBlock, Set<DatanodeInfo>> corruptionMap) {
|
|
|
StripedReader reader = null;
|
|
|
// step1: initially we may only have <code>minRequiredSources</code>
|
|
@@ -707,7 +714,7 @@ public final class ErasureCodingWorker {
|
|
|
int toRead = 0;
|
|
|
while (reader == null && m < sources.length) {
|
|
|
reader = addStripedReader(m, positionInBlock);
|
|
|
- toRead = getReadLength(liveIndices[m], recoverLength);
|
|
|
+ toRead = getReadLength(liveIndices[m], reconstructLen);
|
|
|
if (toRead > 0) {
|
|
|
if (reader.blockReader == null) {
|
|
|
reader = null;
|
|
@@ -727,7 +734,7 @@ public final class ErasureCodingWorker {
|
|
|
for (int i = 0; reader == null && i < stripedReaders.size(); i++) {
|
|
|
if (!used.get(i)) {
|
|
|
StripedReader r = stripedReaders.get(i);
|
|
|
- toRead = getReadLength(liveIndices[i], recoverLength);
|
|
|
+ toRead = getReadLength(liveIndices[i], reconstructLen);
|
|
|
if (toRead > 0) {
|
|
|
closeBlockReader(r.blockReader);
|
|
|
r.blockReader = newBlockReader(
|