Author: hairong
Date: Sun Jan 24 23:57:37 2010
New Revision: 902679
URL: http://svn.apache.org/viewvc?rev=902679&view=rev
Log:
HDFS-792. DFSClient 0.20.1 is incompatible with HDFS 0.20.2. Contributed by
Todd Lipcon.
Modified:
hadoop/common/branches/branch-0.20/CHANGES.txt
hadoop/common/branches/branch-0.20/src/hdfs/org/apache/hadoop/hdfs/DFSClient.java
hadoop/common/branches/branch-0.20/src/hdfs/org/apache/hadoop/hdfs/protocol/DataTransferProtocol.java
hadoop/common/branches/branch-0.20/src/hdfs/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
hadoop/common/branches/branch-0.20/src/test/org/apache/hadoop/hdfs/TestDataTransferProtocol.java
Modified: hadoop/common/branches/branch-0.20/CHANGES.txt
URL:
http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20/CHANGES.txt?rev=902679&r1=902678&r2=902679&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20/CHANGES.txt (original)
+++ hadoop/common/branches/branch-0.20/CHANGES.txt Sun Jan 24 23:57:37 2010
@@ -89,6 +89,9 @@
MAPREDUCE-433. Use more reliable counters in TestReduceFetch. (cdouglas)
+ HDFS-792. DFSClient 0.20.1 is incompatible with HDFS 0.20.2.
+ (Tod Lipcon via hairong)
+
Release 0.20.1 - 2009-09-01
INCOMPATIBLE CHANGES
Modified:
hadoop/common/branches/branch-0.20/src/hdfs/org/apache/hadoop/hdfs/DFSClient.java
URL:
http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20/src/hdfs/org/apache/hadoop/hdfs/DFSClient.java?rev=902679&r1=902678&r2=902679&view=diff
==============================================================================
---
hadoop/common/branches/branch-0.20/src/hdfs/org/apache/hadoop/hdfs/DFSClient.java
(original)
+++
hadoop/common/branches/branch-0.20/src/hdfs/org/apache/hadoop/hdfs/DFSClient.java
Sun Jan 24 23:57:37 2010
@@ -29,6 +29,7 @@
import org.apache.hadoop.conf.*;
import org.apache.hadoop.hdfs.DistributedFileSystem.DiskStatus;
import org.apache.hadoop.hdfs.protocol.*;
+import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.PipelineAck;
import org.apache.hadoop.hdfs.server.common.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -2396,17 +2397,24 @@
public void run() {
this.setName("ResponseProcessor for block " + block);
+ PipelineAck ack = new PipelineAck();
while (!closed && clientRunning && !lastPacketInBlock) {
// process responses from datanodes.
try {
- // verify seqno from datanode
- long seqno = blockReplyStream.readLong();
- LOG.debug("DFSClient received ack for seqno " + seqno);
- if (seqno == -1) {
+ // read an ack from the pipeline
+ ack.readFields(blockReplyStream, targets.length);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("DFSClient " + ack);
+ }
+ long seqno = ack.getSeqno();
+ if (seqno == PipelineAck.HEART_BEAT.getSeqno()) {
continue;
} else if (seqno == -2) {
- // no nothing
+ // This signifies that some pipeline node failed to read
downstream
+ // and therefore has no idea what sequence number the message
corresponds
+ // to. So, we don't try to match it up with an ack.
+ assert ! ack.isSuccess();
} else {
Packet one = null;
synchronized (ackQueue) {
@@ -2422,7 +2430,7 @@
// processes response status from all datanodes.
for (int i = 0; i < targets.length && clientRunning; i++) {
- short reply = blockReplyStream.readShort();
+ short reply = ack.getReply(i);
if (reply != DataTransferProtocol.OP_STATUS_SUCCESS) {
errorIndex = i; // first bad datanode
throw new IOException("Bad response " + reply +
Modified:
hadoop/common/branches/branch-0.20/src/hdfs/org/apache/hadoop/hdfs/protocol/DataTransferProtocol.java
URL:
http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20/src/hdfs/org/apache/hadoop/hdfs/protocol/DataTransferProtocol.java?rev=902679&r1=902678&r2=902679&view=diff
==============================================================================
---
hadoop/common/branches/branch-0.20/src/hdfs/org/apache/hadoop/hdfs/protocol/DataTransferProtocol.java
(original)
+++
hadoop/common/branches/branch-0.20/src/hdfs/org/apache/hadoop/hdfs/protocol/DataTransferProtocol.java
Sun Jan 24 23:57:37 2010
@@ -17,6 +17,9 @@
*/
package org.apache.hadoop.hdfs.protocol;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
/**
*
@@ -56,6 +59,95 @@
public static final int OP_STATUS_ERROR_EXISTS = 4;
public static final int OP_STATUS_CHECKSUM_OK = 5;
+ /* seqno for a heartbeat packet */
+ public static final int HEARTBEAT_SEQNO = -1;
-
+ /** reply **/
+ public static class PipelineAck {
+ private long seqno;
+ private short replies[];
+ final public static PipelineAck HEART_BEAT =
+ new PipelineAck(HEARTBEAT_SEQNO, new short[0]);
+
+ /** default constructor **/
+ public PipelineAck() {
+ }
+
+ /**
+ * Constructor
+ * @param seqno sequence number
+ * @param replies an array of replies
+ */
+ public PipelineAck(long seqno, short[] replies) {
+ this.seqno = seqno;
+ this.replies = replies;
+ }
+
+ /**
+ * Get the sequence number
+ * @return the sequence number
+ */
+ public long getSeqno() {
+ return seqno;
+ }
+
+ /**
+ * get the ith reply
+ * @return the the ith reply
+ */
+ public short getReply(int i) {
+ return replies[i];
+ }
+
+ /**
+ * Check if this ack contains error status
+ * @return true if all statuses are SUCCESS
+ */
+ public boolean isSuccess() {
+ for (short reply : replies) {
+ if (reply != OP_STATUS_SUCCESS) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ public void readFields(DataInput in, int numRepliesExpected)
+ throws IOException {
+ assert numRepliesExpected > 0;
+
+ seqno = in.readLong();
+ if (seqno == HEARTBEAT_SEQNO) {
+ // Heartbeat doesn't forward any replies
+ replies = new short[0];
+ } else {
+ replies = new short[numRepliesExpected];
+ for (int i=0; i < replies.length; i++) {
+ replies[i] = in.readShort();
+ }
+ }
+ }
+
+ public void write(DataOutput out) throws IOException {
+ out.writeLong(seqno);
+ for(short reply : replies) {
+ out.writeShort(reply);
+ }
+ }
+
+ @Override //Object
+ public String toString() {
+ StringBuilder ack = new StringBuilder("Replies for seqno ");
+ ack.append( seqno ).append( " are" );
+ for(short reply : replies) {
+ ack.append(" ");
+ if (reply == OP_STATUS_SUCCESS) {
+ ack.append("SUCCESS");
+ } else {
+ ack.append("FAILED");
+ }
+ }
+ return ack.toString();
+ }
+ }
}
Modified:
hadoop/common/branches/branch-0.20/src/hdfs/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
URL:
http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20/src/hdfs/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java?rev=902679&r1=902678&r2=902679&view=diff
==============================================================================
---
hadoop/common/branches/branch-0.20/src/hdfs/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
(original)
+++
hadoop/common/branches/branch-0.20/src/hdfs/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
Sun Jan 24 23:57:37 2010
@@ -36,6 +36,7 @@
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.PipelineAck;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.DataChecksum;
@@ -773,8 +774,13 @@
// send a heartbeat if it is time.
now = System.currentTimeMillis();
if (now - lastHeartbeat > datanode.socketTimeout/2) {
- replyOut.writeLong(-1); // send heartbeat
+ PipelineAck.HEART_BEAT.write(replyOut); // send heart beat
replyOut.flush();
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("PacketResponder " + numTargets +
+ " for block " + block +
+ " sent a heartbeat");
+ }
lastHeartbeat = now;
}
}
@@ -814,8 +820,8 @@
lastPacket = true;
}
- replyOut.writeLong(expected);
- replyOut.writeShort(DataTransferProtocol.OP_STATUS_SUCCESS);
+ new PipelineAck(expected, new short[]{
+ DataTransferProtocol.OP_STATUS_SUCCESS}).write(replyOut);
replyOut.flush();
} catch (Exception e) {
if (running) {
@@ -845,23 +851,41 @@
while (running && datanode.shouldRun && !lastPacketInBlock) {
try {
- short op = DataTransferProtocol.OP_STATUS_SUCCESS;
boolean didRead = false;
+
+ /**
+ * Sequence number -2 is a special value that is used when
+ * a DN fails to read an ack from a downstream. In this case,
+ * it needs to tell the client that there's been an error
downstream
+ * but has no valid sequence number to use. Thus, -2 is used
+ * as an UNKNOWN value.
+ */
long expected = -2;
+
+ PipelineAck ack = new PipelineAck();
try {
- // read seqno from downstream datanode
- long seqno = mirrorIn.readLong();
+ // read an ack from downstream datanode
+ ack.readFields(mirrorIn, numTargets);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("PacketResponder " + numTargets + " got " + ack);
+ }
+ long seqno = ack.getSeqno();
didRead = true;
- if (seqno == -1) {
- replyOut.writeLong(-1); // send keepalive
+ if (seqno == PipelineAck.HEART_BEAT.getSeqno()) {
+ ack.write(replyOut); // send keepalive
replyOut.flush();
- LOG.debug("PacketResponder " + numTargets + " got -1");
continue;
} else if (seqno == -2) {
- LOG.debug("PacketResponder " + numTargets + " got -2");
+ // A downstream node must have failed to read an ack. We need
+ // to forward this on.
+ assert ! ack.isSuccess();
} else {
- LOG.debug("PacketResponder " + numTargets + " got seqno = " +
- seqno);
+ if (seqno < 0) {
+ throw new IOException("Received an invalid negative sequence
number. "
+ + "Ack = " + ack);
+ }
+ assert seqno >= 0;
+
Packet pkt = null;
synchronized (this) {
while (running && datanode.shouldRun && ackQueue.size() ==
0) {
@@ -876,7 +900,6 @@
pkt = ackQueue.removeFirst();
expected = pkt.seqno;
notifyAll();
- LOG.debug("PacketResponder " + numTargets + " seqno = " +
seqno);
if (seqno != expected) {
throw new IOException("PacketResponder " + numTargets +
" for block " + block +
@@ -909,10 +932,6 @@
continue;
}
- if (!didRead) {
- op = DataTransferProtocol.OP_STATUS_ERROR;
- }
-
// If this is the last packet in block, then close block
// file and finalize the block before responding success
if (lastPacketInBlock && !receiver.finalized) {
@@ -935,43 +954,37 @@
}
}
- // send my status back to upstream datanode
- replyOut.writeLong(expected); // send seqno upstream
- replyOut.writeShort(DataTransferProtocol.OP_STATUS_SUCCESS);
-
- LOG.debug("PacketResponder " + numTargets +
- " for block " + block +
- " responded my status " +
- " for seqno " + expected);
-
- // forward responses from downstream datanodes.
- for (int i = 0; i < numTargets && datanode.shouldRun; i++) {
- try {
- if (op == DataTransferProtocol.OP_STATUS_SUCCESS) {
- op = mirrorIn.readShort();
- if (op != DataTransferProtocol.OP_STATUS_SUCCESS) {
- LOG.debug("PacketResponder for block " + block +
- ": error code received from downstream " +
- " datanode[" + i + "] " + op);
- }
- }
- } catch (Throwable e) {
- op = DataTransferProtocol.OP_STATUS_ERROR;
+ // construct my ack message.
+ short[] replies = new short[1 + numTargets];
+ if (!didRead) { // no ack is read
+ replies[0] = DataTransferProtocol.OP_STATUS_SUCCESS;
+ // Fill all downstream nodes with ERROR - the client will
+ // eject the first node with ERROR status (our mirror)
+ for (int i = 1; i < replies.length; i++) {
+ replies[i] = DataTransferProtocol.OP_STATUS_ERROR;
+ }
+ } else {
+ replies = new short[1+numTargets];
+ replies[0] = DataTransferProtocol.OP_STATUS_SUCCESS;
+ for (int i=0; i<numTargets; i++) {
+ replies[i+1] = ack.getReply(i);
}
- replyOut.writeShort(op);
}
+ PipelineAck replyAck = new PipelineAck(expected, replies);
+
+ // send my ack back to upstream datanode
+ replyAck.write(replyOut);
replyOut.flush();
- LOG.debug("PacketResponder " + block + " " + numTargets +
- " responded other status " + " for seqno " + expected);
-
- // If we were unable to read the seqno from downstream, then stop.
- if (expected == -2) {
- running = false;
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("PacketResponder " + numTargets +
+ " for block " + block +
+ " responded an ack: " + replyAck);
}
+
// If we forwarded an error response from a downstream datanode
// and we are acting on behalf of a client, then we quit. The
// client will drive the recovery mechanism.
- if (op == DataTransferProtocol.OP_STATUS_ERROR &&
receiver.clientName.length() > 0) {
+ if (!replyAck.isSuccess() && receiver.clientName.length() > 0) {
running = false;
}
} catch (IOException e) {
Modified:
hadoop/common/branches/branch-0.20/src/test/org/apache/hadoop/hdfs/TestDataTransferProtocol.java
URL:
http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20/src/test/org/apache/hadoop/hdfs/TestDataTransferProtocol.java?rev=902679&r1=902678&r2=902679&view=diff
==============================================================================
---
hadoop/common/branches/branch-0.20/src/test/org/apache/hadoop/hdfs/TestDataTransferProtocol.java
(original)
+++
hadoop/common/branches/branch-0.20/src/test/org/apache/hadoop/hdfs/TestDataTransferProtocol.java
Sun Jan 24 23:57:37 2010
@@ -250,9 +250,9 @@
sendOut.writeInt(0); // chunk length
sendOut.writeInt(0); // zero checksum
//ok finally write a block with 0 len
- Text.writeString(recvOut, ""); // first bad node
- recvOut.writeLong(100); // sequencenumber
- recvOut.writeShort((short)DataTransferProtocol.OP_STATUS_SUCCESS);
+ Text.writeString(recvOut, "");
+ new DataTransferProtocol.PipelineAck(100,
+ new short[]{DataTransferProtocol.OP_STATUS_SUCCESS}).write(recvOut);
sendRecvData("Writing a zero len block blockid " + newBlockId, false);
/* Test OP_READ_BLOCK */