|
@@ -1126,6 +1126,7 @@ public class DataNode implements InterDatanodeProtocol, FSConstants, Runnable {
|
|
|
*/
|
|
|
private void writeBlock(DataInputStream in) throws IOException {
|
|
|
xceiverCount.incr();
|
|
|
+ DatanodeInfo srcDataNode = null;
|
|
|
LOG.debug("writeBlock receive buf size " + s.getReceiveBufferSize() +
|
|
|
" tcp no delay " + s.getTcpNoDelay());
|
|
|
//
|
|
@@ -1138,6 +1139,11 @@ public class DataNode implements InterDatanodeProtocol, FSConstants, Runnable {
|
|
|
int pipelineSize = in.readInt(); // num of datanodes in entire pipeline
|
|
|
boolean isRecovery = in.readBoolean(); // is this part of recovery?
|
|
|
String client = Text.readString(in); // working on behalf of this client
|
|
|
+ boolean hasSrcDataNode = in.readBoolean(); // is src node info present
|
|
|
+ if (hasSrcDataNode) {
|
|
|
+ srcDataNode = new DatanodeInfo();
|
|
|
+ srcDataNode.readFields(in);
|
|
|
+ }
|
|
|
int numTargets = in.readInt();
|
|
|
if (numTargets < 0) {
|
|
|
throw new IOException("Mislabelled incoming datastream.");
|
|
@@ -1159,7 +1165,7 @@ public class DataNode implements InterDatanodeProtocol, FSConstants, Runnable {
|
|
|
try {
|
|
|
// open a block receiver and check if the block does not exist
|
|
|
blockReceiver = new BlockReceiver(block, in,
|
|
|
- s.getInetAddress().toString(), isRecovery, client);
|
|
|
+ s.getInetAddress().toString(), isRecovery, client, srcDataNode);
|
|
|
|
|
|
// get a connection back to the previous target
|
|
|
replyOut = new DataOutputStream(
|
|
@@ -1196,6 +1202,10 @@ public class DataNode implements InterDatanodeProtocol, FSConstants, Runnable {
|
|
|
mirrorOut.writeInt( pipelineSize );
|
|
|
mirrorOut.writeBoolean( isRecovery );
|
|
|
Text.writeString( mirrorOut, client );
|
|
|
+ mirrorOut.writeBoolean(hasSrcDataNode);
|
|
|
+ if (hasSrcDataNode) { // pass src node information
|
|
|
+ srcDataNode.write(mirrorOut);
|
|
|
+ }
|
|
|
mirrorOut.writeInt( targets.length - 1 );
|
|
|
for ( int i = 1; i < targets.length; i++ ) {
|
|
|
targets[i].write( mirrorOut );
|
|
@@ -1419,7 +1429,7 @@ public class DataNode implements InterDatanodeProtocol, FSConstants, Runnable {
|
|
|
try {
|
|
|
// open a block receiver and check if the block does not exist
|
|
|
blockReceiver = new BlockReceiver(
|
|
|
- block, in, s.getRemoteSocketAddress().toString(), false, "");
|
|
|
+ block, in, s.getRemoteSocketAddress().toString(), false, "", null);
|
|
|
|
|
|
// receive a block
|
|
|
blockReceiver.receiveBlock(null, null, null, null, balancingThrottler, -1);
|
|
@@ -2250,10 +2260,11 @@ public class DataNode implements InterDatanodeProtocol, FSConstants, Runnable {
|
|
|
private FSDataset.BlockWriteStreams streams;
|
|
|
private boolean isRecovery = false;
|
|
|
private String clientName;
|
|
|
+ DatanodeInfo srcDataNode = null;
|
|
|
|
|
|
BlockReceiver(Block block, DataInputStream in, String inAddr,
|
|
|
- boolean isRecovery, String clientName)
|
|
|
- throws IOException {
|
|
|
+ boolean isRecovery, String clientName,
|
|
|
+ DatanodeInfo srcDataNode) throws IOException {
|
|
|
try{
|
|
|
this.block = block;
|
|
|
this.in = in;
|
|
@@ -2264,6 +2275,7 @@ public class DataNode implements InterDatanodeProtocol, FSConstants, Runnable {
|
|
|
this.checksum = DataChecksum.newDataChecksum(in);
|
|
|
this.bytesPerChecksum = checksum.getBytesPerChecksum();
|
|
|
this.checksumSize = checksum.getChecksumSize();
|
|
|
+ this.srcDataNode = srcDataNode;
|
|
|
//
|
|
|
// Open local disk out
|
|
|
//
|
|
@@ -2352,6 +2364,18 @@ public class DataNode implements InterDatanodeProtocol, FSConstants, Runnable {
|
|
|
checksum.update(dataBuf, dataOff, chunkLen);
|
|
|
|
|
|
if (!checksum.compare(checksumBuf, checksumOff)) {
|
|
|
+ if (srcDataNode != null) {
|
|
|
+ try {
|
|
|
+ LOG.info("report corrupt block " + block + " from datanode " +
|
|
|
+ srcDataNode + " to namenode");
|
|
|
+ LocatedBlock lb = new LocatedBlock(block,
|
|
|
+ new DatanodeInfo[] {srcDataNode});
|
|
|
+ namenode.reportBadBlocks(new LocatedBlock[] {lb});
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.warn("Failed to report bad block " + block +
|
|
|
+ " from datanode " + srcDataNode + " to namenode");
|
|
|
+ }
|
|
|
+ }
|
|
|
throw new IOException("Unexpected checksum mismatch " +
|
|
|
"while writing " + block + " from " + inAddr);
|
|
|
}
|
|
@@ -2770,6 +2794,7 @@ public class DataNode implements InterDatanodeProtocol, FSConstants, Runnable {
|
|
|
SMALL_BUFFER_SIZE));
|
|
|
|
|
|
blockSender = new BlockSender(b, 0, -1, false, false, false);
|
|
|
+ DatanodeInfo srcNode = new DatanodeInfo(dnRegistration);
|
|
|
|
|
|
//
|
|
|
// Header info
|
|
@@ -2781,6 +2806,8 @@ public class DataNode implements InterDatanodeProtocol, FSConstants, Runnable {
|
|
|
out.writeInt(0); // no pipelining
|
|
|
out.writeBoolean(false); // not part of recovery
|
|
|
Text.writeString(out, ""); // client
|
|
|
+ out.writeBoolean(true); // sending src node information
|
|
|
+ srcNode.write(out); // Write src node DatanodeInfo
|
|
|
// write targets
|
|
|
out.writeInt(targets.length - 1);
|
|
|
for (int i = 1; i < targets.length; i++) {
|