|
@@ -85,7 +85,10 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
|
|
|
private long opStartTime; //the start time of receiving an Op
|
|
|
|
|
|
public DataXceiver(Socket s, DataNode datanode,
|
|
|
- DataXceiverServer dataXceiverServer) {
|
|
|
+ DataXceiverServer dataXceiverServer) throws IOException {
|
|
|
+ super(new DataInputStream(new BufferedInputStream(
|
|
|
+ NetUtils.getInputStream(s), FSConstants.SMALL_BUFFER_SIZE)));
|
|
|
+
|
|
|
this.s = s;
|
|
|
this.isLocal = s.getInetAddress().equals(s.getLocalAddress());
|
|
|
this.datanode = datanode;
|
|
@@ -127,13 +130,9 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
|
|
|
public void run() {
|
|
|
updateCurrentThreadName("Waiting for operation");
|
|
|
|
|
|
- DataInputStream in=null;
|
|
|
int opsProcessed = 0;
|
|
|
Op op = null;
|
|
|
try {
|
|
|
- in = new DataInputStream(
|
|
|
- new BufferedInputStream(NetUtils.getInputStream(s),
|
|
|
- SMALL_BUFFER_SIZE));
|
|
|
int stdTimeout = s.getSoTimeout();
|
|
|
|
|
|
// We process requests in a loop, and stay around for a short timeout.
|
|
@@ -145,7 +144,7 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
|
|
|
assert socketKeepaliveTimeout > 0;
|
|
|
s.setSoTimeout(socketKeepaliveTimeout);
|
|
|
}
|
|
|
- op = readOp(in);
|
|
|
+ op = readOp();
|
|
|
} catch (InterruptedIOException ignored) {
|
|
|
// Time out while we wait for client rpc
|
|
|
break;
|
|
@@ -176,7 +175,7 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
|
|
|
}
|
|
|
|
|
|
opStartTime = now();
|
|
|
- processOp(op, in);
|
|
|
+ processOp(op);
|
|
|
++opsProcessed;
|
|
|
} while (!s.isClosed() && socketKeepaliveTimeout > 0);
|
|
|
} catch (Throwable t) {
|
|
@@ -196,13 +195,12 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Read a block from the disk.
|
|
|
- */
|
|
|
@Override
|
|
|
- protected void opReadBlock(DataInputStream in, ExtendedBlock block,
|
|
|
- long startOffset, long length, String clientName,
|
|
|
- Token<BlockTokenIdentifier> blockToken) throws IOException {
|
|
|
+ public void readBlock(final ExtendedBlock block,
|
|
|
+ final Token<BlockTokenIdentifier> blockToken,
|
|
|
+ final String clientName,
|
|
|
+ final long blockOffset,
|
|
|
+ final long length) throws IOException {
|
|
|
OutputStream baseStream = NetUtils.getOutputStream(s,
|
|
|
datanode.socketWriteTimeout);
|
|
|
DataOutputStream out = new DataOutputStream(
|
|
@@ -225,7 +223,7 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
|
|
|
updateCurrentThreadName("Sending block " + block);
|
|
|
try {
|
|
|
try {
|
|
|
- blockSender = new BlockSender(block, startOffset, length,
|
|
|
+ blockSender = new BlockSender(block, blockOffset, length,
|
|
|
true, true, false, datanode, clientTraceFmt);
|
|
|
} catch(IOException e) {
|
|
|
LOG.info("opReadBlock " + block + " received exception " + e);
|
|
@@ -284,16 +282,17 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
|
|
|
datanode.metrics.incrReadsFromClient(isLocal);
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Write a block to disk.
|
|
|
- */
|
|
|
@Override
|
|
|
- protected void opWriteBlock(final DataInputStream in, final ExtendedBlock block,
|
|
|
- final int pipelineSize, final BlockConstructionStage stage,
|
|
|
- final long newGs, final long minBytesRcvd, final long maxBytesRcvd,
|
|
|
- final String clientname, final DatanodeInfo srcDataNode,
|
|
|
- final DatanodeInfo[] targets, final Token<BlockTokenIdentifier> blockToken
|
|
|
- ) throws IOException {
|
|
|
+ public void writeBlock(final ExtendedBlock block,
|
|
|
+ final Token<BlockTokenIdentifier> blockToken,
|
|
|
+ final String clientname,
|
|
|
+ final DatanodeInfo[] targets,
|
|
|
+ final DatanodeInfo srcDataNode,
|
|
|
+ final BlockConstructionStage stage,
|
|
|
+ final int pipelineSize,
|
|
|
+ final long minBytesRcvd,
|
|
|
+ final long maxBytesRcvd,
|
|
|
+ final long latestGenerationStamp) throws IOException {
|
|
|
updateCurrentThreadName("Receiving block " + block + " client=" + clientname);
|
|
|
final boolean isDatanode = clientname.length() == 0;
|
|
|
final boolean isClient = !isDatanode;
|
|
@@ -308,7 +307,7 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
|
|
|
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("opWriteBlock: stage=" + stage + ", clientname=" + clientname
|
|
|
- + "\n block =" + block + ", newGs=" + newGs
|
|
|
+ + "\n block =" + block + ", newGs=" + latestGenerationStamp
|
|
|
+ ", bytesRcvd=[" + minBytesRcvd + ", " + maxBytesRcvd + "]"
|
|
|
+ "\n targets=" + Arrays.asList(targets)
|
|
|
+ "; pipelineSize=" + pipelineSize + ", srcDataNode=" + srcDataNode
|
|
@@ -351,10 +350,10 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
|
|
|
blockReceiver = new BlockReceiver(block, in,
|
|
|
s.getRemoteSocketAddress().toString(),
|
|
|
s.getLocalSocketAddress().toString(),
|
|
|
- stage, newGs, minBytesRcvd, maxBytesRcvd,
|
|
|
+ stage, latestGenerationStamp, minBytesRcvd, maxBytesRcvd,
|
|
|
clientname, srcDataNode, datanode);
|
|
|
} else {
|
|
|
- datanode.data.recoverClose(block, newGs, minBytesRcvd);
|
|
|
+ datanode.data.recoverClose(block, latestGenerationStamp, minBytesRcvd);
|
|
|
}
|
|
|
|
|
|
//
|
|
@@ -380,9 +379,9 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
|
|
|
SMALL_BUFFER_SIZE));
|
|
|
mirrorIn = new DataInputStream(NetUtils.getInputStream(mirrorSock));
|
|
|
|
|
|
- Sender.opWriteBlock(mirrorOut, originalBlock,
|
|
|
- pipelineSize, stage, newGs, minBytesRcvd, maxBytesRcvd, clientname,
|
|
|
- srcDataNode, targets, blockToken);
|
|
|
+ new Sender(mirrorOut).writeBlock(originalBlock, blockToken,
|
|
|
+ clientname, targets, srcDataNode, stage, pipelineSize,
|
|
|
+ minBytesRcvd, maxBytesRcvd, latestGenerationStamp);
|
|
|
|
|
|
if (blockReceiver != null) { // send checksum header
|
|
|
blockReceiver.writeChecksumHeader(mirrorOut);
|
|
@@ -464,7 +463,7 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
|
|
|
// update its generation stamp
|
|
|
if (isClient &&
|
|
|
stage == BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
|
|
|
- block.setGenerationStamp(newGs);
|
|
|
+ block.setGenerationStamp(latestGenerationStamp);
|
|
|
block.setNumBytes(minBytesRcvd);
|
|
|
}
|
|
|
|
|
@@ -499,10 +498,10 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- protected void opTransferBlock(final DataInputStream in,
|
|
|
- final ExtendedBlock blk, final String client,
|
|
|
- final DatanodeInfo[] targets,
|
|
|
- final Token<BlockTokenIdentifier> blockToken) throws IOException {
|
|
|
+ public void transferBlock(final ExtendedBlock blk,
|
|
|
+ final Token<BlockTokenIdentifier> blockToken,
|
|
|
+ final String clientName,
|
|
|
+ final DatanodeInfo[] targets) throws IOException {
|
|
|
checkAccess(null, true, blk, blockToken,
|
|
|
Op.TRANSFER_BLOCK, BlockTokenSecretManager.AccessMode.COPY);
|
|
|
|
|
@@ -511,19 +510,16 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
|
|
|
final DataOutputStream out = new DataOutputStream(
|
|
|
NetUtils.getOutputStream(s, datanode.socketWriteTimeout));
|
|
|
try {
|
|
|
- datanode.transferReplicaForPipelineRecovery(blk, targets, client);
|
|
|
+ datanode.transferReplicaForPipelineRecovery(blk, targets, clientName);
|
|
|
writeResponse(Status.SUCCESS, out);
|
|
|
} finally {
|
|
|
IOUtils.closeStream(out);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Get block checksum (MD5 of CRC32).
|
|
|
- */
|
|
|
@Override
|
|
|
- protected void opBlockChecksum(DataInputStream in, ExtendedBlock block,
|
|
|
- Token<BlockTokenIdentifier> blockToken) throws IOException {
|
|
|
+ public void blockChecksum(final ExtendedBlock block,
|
|
|
+ final Token<BlockTokenIdentifier> blockToken) throws IOException {
|
|
|
final DataOutputStream out = new DataOutputStream(
|
|
|
NetUtils.getOutputStream(s, datanode.socketWriteTimeout));
|
|
|
checkAccess(out, true, block, blockToken,
|
|
@@ -572,12 +568,9 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
|
|
|
datanode.metrics.addBlockChecksumOp(elapsed());
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Read a block from the disk and then sends it to a destination.
|
|
|
- */
|
|
|
@Override
|
|
|
- protected void opCopyBlock(DataInputStream in, ExtendedBlock block,
|
|
|
- Token<BlockTokenIdentifier> blockToken) throws IOException {
|
|
|
+ public void copyBlock(final ExtendedBlock block,
|
|
|
+ final Token<BlockTokenIdentifier> blockToken) throws IOException {
|
|
|
updateCurrentThreadName("Copying block " + block);
|
|
|
// Read in the header
|
|
|
if (datanode.isBlockTokenEnabled) {
|
|
@@ -647,15 +640,12 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
|
|
|
datanode.metrics.addCopyBlockOp(elapsed());
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Receive a block and write it to disk, it then notifies the namenode to
|
|
|
- * remove the copy from the source.
|
|
|
- */
|
|
|
@Override
|
|
|
- protected void opReplaceBlock(DataInputStream in,
|
|
|
- ExtendedBlock block, String sourceID, DatanodeInfo proxySource,
|
|
|
- Token<BlockTokenIdentifier> blockToken) throws IOException {
|
|
|
- updateCurrentThreadName("Replacing block " + block + " from " + sourceID);
|
|
|
+ public void replaceBlock(final ExtendedBlock block,
|
|
|
+ final Token<BlockTokenIdentifier> blockToken,
|
|
|
+ final String delHint,
|
|
|
+ final DatanodeInfo proxySource) throws IOException {
|
|
|
+ updateCurrentThreadName("Replacing block " + block + " from " + delHint);
|
|
|
|
|
|
/* read header */
|
|
|
block.setNumBytes(dataXceiverServer.estimateBlockSize);
|
|
@@ -699,7 +689,7 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
|
|
|
new BufferedOutputStream(baseStream, SMALL_BUFFER_SIZE));
|
|
|
|
|
|
/* send request to the proxy */
|
|
|
- Sender.opCopyBlock(proxyOut, block, blockToken);
|
|
|
+ new Sender(proxyOut).copyBlock(block, blockToken);
|
|
|
|
|
|
// receive the response from the proxy
|
|
|
proxyReply = new DataInputStream(new BufferedInputStream(
|
|
@@ -727,7 +717,7 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
|
|
|
dataXceiverServer.balanceThrottler, null);
|
|
|
|
|
|
// notify name node
|
|
|
- datanode.notifyNamenodeReceivedBlock(block, sourceID);
|
|
|
+ datanode.notifyNamenodeReceivedBlock(block, delHint);
|
|
|
|
|
|
LOG.info("Moved block " + block +
|
|
|
" from " + s.getRemoteSocketAddress());
|