|
@@ -62,7 +62,6 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
|
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
|
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory;
|
|
|
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
|
|
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
|
|
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;
|
|
@@ -88,12 +87,12 @@ import com.google.common.base.Preconditions;
|
|
|
* commands.
|
|
|
*/
|
|
|
public final class ErasureCodingWorker {
|
|
|
- private final Log LOG = DataNode.LOG;
|
|
|
+ private static final Log LOG = DataNode.LOG;
|
|
|
|
|
|
private final DataNode datanode;
|
|
|
- private Configuration conf;
|
|
|
+ private final Configuration conf;
|
|
|
|
|
|
- private ThreadPoolExecutor STRIPED_READ_TRHEAD_POOL;
|
|
|
+ private ThreadPoolExecutor STRIPED_READ_THREAD_POOL;
|
|
|
private final int STRIPED_READ_THRESHOLD_MILLIS;
|
|
|
private final int STRIPED_READ_BUFFER_SIZE;
|
|
|
|
|
@@ -121,7 +120,10 @@ public final class ErasureCodingWorker {
|
|
|
}
|
|
|
|
|
|
private void initializeStripedReadThreadPool(int num) {
|
|
|
- STRIPED_READ_TRHEAD_POOL = new ThreadPoolExecutor(1, num, 60,
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Using striped reads; pool threads=" + num);
|
|
|
+ }
|
|
|
+ STRIPED_READ_THREAD_POOL = new ThreadPoolExecutor(1, num, 60,
|
|
|
TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
|
|
|
new Daemon.DaemonFactory() {
|
|
|
private final AtomicInteger threadIndex = new AtomicInteger(0);
|
|
@@ -141,7 +143,7 @@ public final class ErasureCodingWorker {
|
|
|
super.rejectedExecution(runnable, e);
|
|
|
}
|
|
|
});
|
|
|
- STRIPED_READ_TRHEAD_POOL.allowCoreThreadTimeOut(true);
|
|
|
+ STRIPED_READ_THREAD_POOL.allowCoreThreadTimeOut(true);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -231,23 +233,23 @@ public final class ErasureCodingWorker {
|
|
|
|
|
|
// sources
|
|
|
private final short[] liveIndices;
|
|
|
- private DatanodeInfo[] sources;
|
|
|
+ private final DatanodeInfo[] sources;
|
|
|
|
|
|
- private List<StripedReader> stripedReaders;
|
|
|
+ private final List<StripedReader> stripedReaders;
|
|
|
|
|
|
// targets
|
|
|
- private DatanodeInfo[] targets;
|
|
|
- private StorageType[] targetStorageTypes;
|
|
|
+ private final DatanodeInfo[] targets;
|
|
|
+ private final StorageType[] targetStorageTypes;
|
|
|
|
|
|
- private short[] targetIndices;
|
|
|
- private ByteBuffer[] targetBuffers;
|
|
|
+ private final short[] targetIndices;
|
|
|
+ private final ByteBuffer[] targetBuffers;
|
|
|
|
|
|
- private Socket[] targetSockets;
|
|
|
- private DataOutputStream[] targetOutputStreams;
|
|
|
- private DataInputStream[] targetInputStreams;
|
|
|
+ private final Socket[] targetSockets;
|
|
|
+ private final DataOutputStream[] targetOutputStreams;
|
|
|
+ private final DataInputStream[] targetInputStreams;
|
|
|
|
|
|
- private long[] blockOffset4Targets;
|
|
|
- private long[] seqNo4Targets;
|
|
|
+ private final long[] blockOffset4Targets;
|
|
|
+ private final long[] seqNo4Targets;
|
|
|
|
|
|
private final int WRITE_PACKET_SIZE = 64 * 1024;
|
|
|
private DataChecksum checksum;
|
|
@@ -257,11 +259,11 @@ public final class ErasureCodingWorker {
|
|
|
private int bytesPerChecksum;
|
|
|
private int checksumSize;
|
|
|
|
|
|
- private CachingStrategy cachingStrategy;
|
|
|
+ private final CachingStrategy cachingStrategy;
|
|
|
|
|
|
- private Map<Future<Void>, Integer> futures = new HashMap<>();
|
|
|
- private CompletionService<Void> readService =
|
|
|
- new ExecutorCompletionService<>(STRIPED_READ_TRHEAD_POOL);
|
|
|
+ private final Map<Future<Void>, Integer> futures = new HashMap<>();
|
|
|
+ private final CompletionService<Void> readService =
|
|
|
+ new ExecutorCompletionService<>(STRIPED_READ_THREAD_POOL);
|
|
|
|
|
|
ReconstructAndTransferBlock(BlockECRecoveryInfo recoveryInfo) {
|
|
|
ECSchema schema = recoveryInfo.getECSchema();
|
|
@@ -277,7 +279,8 @@ public final class ErasureCodingWorker {
|
|
|
|
|
|
Preconditions.checkArgument(liveIndices.length >= dataBlkNum,
|
|
|
"No enough live striped blocks.");
|
|
|
- Preconditions.checkArgument(liveIndices.length == sources.length);
|
|
|
+ Preconditions.checkArgument(liveIndices.length == sources.length,
|
|
|
+ "liveBlockIndices and source dns should match");
|
|
|
|
|
|
targets = recoveryInfo.getTargetDnInfos();
|
|
|
targetStorageTypes = recoveryInfo.getTargetStorageTypes();
|
|
@@ -336,7 +339,6 @@ public final class ErasureCodingWorker {
|
|
|
if (nsuccess < dataBlkNum) {
|
|
|
String error = "Can't find minimum sources required by "
|
|
|
+ "recovery, block id: " + blockGroup.getBlockId();
|
|
|
- LOG.warn(error);
|
|
|
throw new IOException(error);
|
|
|
}
|
|
|
|
|
@@ -358,7 +360,6 @@ public final class ErasureCodingWorker {
|
|
|
boolean[] targetsStatus = new boolean[targets.length];
|
|
|
if (initTargetStreams(targetsStatus) == 0) {
|
|
|
String error = "All targets are failed.";
|
|
|
- LOG.warn(error);
|
|
|
throw new IOException(error);
|
|
|
}
|
|
|
|
|
@@ -372,7 +373,6 @@ public final class ErasureCodingWorker {
|
|
|
if (nsuccess < dataBlkNum) {
|
|
|
String error = "Can't read data from minimum number of sources "
|
|
|
+ "required by recovery, block id: " + blockGroup.getBlockId();
|
|
|
- LOG.warn(error);
|
|
|
throw new IOException(error);
|
|
|
}
|
|
|
|
|
@@ -385,7 +385,6 @@ public final class ErasureCodingWorker {
|
|
|
// step3: transfer data
|
|
|
if (transferData2Targets(targetsStatus) == 0) {
|
|
|
String error = "Transfer failed for all targets.";
|
|
|
- LOG.warn(error);
|
|
|
throw new IOException(error);
|
|
|
}
|
|
|
|
|
@@ -906,11 +905,11 @@ public final class ErasureCodingWorker {
|
|
|
}
|
|
|
|
|
|
private class StripedReader {
|
|
|
- short index;
|
|
|
- BlockReader blockReader;
|
|
|
- ByteBuffer buffer;
|
|
|
+ private final short index;
|
|
|
+ private BlockReader blockReader;
|
|
|
+ private ByteBuffer buffer;
|
|
|
|
|
|
- public StripedReader(short index) {
|
|
|
+ private StripedReader(short index) {
|
|
|
this.index = index;
|
|
|
}
|
|
|
}
|