|
@@ -27,6 +27,7 @@ import java.io.IOException;
|
|
|
import java.io.InputStream;
|
|
|
import java.io.InterruptedIOException;
|
|
|
import java.io.OutputStream;
|
|
|
+import java.net.InetAddress;
|
|
|
import java.net.InetSocketAddress;
|
|
|
import java.net.Socket;
|
|
|
import java.nio.BufferOverflowException;
|
|
@@ -335,6 +336,8 @@ public class DFSOutputStream extends FSOutputSummer
|
|
|
private String[] favoredNodes;
|
|
|
volatile boolean hasError = false;
|
|
|
volatile int errorIndex = -1;
|
|
|
+ volatile int restartingNodeIndex = -1; // Restarting node index
|
|
|
+ private long restartDeadline = 0; // Deadline of DN restart
|
|
|
private BlockConstructionStage stage; // block construction stage
|
|
|
private long bytesSent = 0; // number of bytes that've been sent
|
|
|
|
|
@@ -471,7 +474,7 @@ public class DFSOutputStream extends FSOutputSummer
|
|
|
try {
|
|
|
// process datanode IO errors if any
|
|
|
boolean doSleep = false;
|
|
|
- if (hasError && errorIndex>=0) {
|
|
|
+ if (hasError && (errorIndex >= 0 || restartingNodeIndex >= 0)) {
|
|
|
doSleep = processDatanodeError();
|
|
|
}
|
|
|
|
|
@@ -571,8 +574,12 @@ public class DFSOutputStream extends FSOutputSummer
|
|
|
blockStream.flush();
|
|
|
} catch (IOException e) {
|
|
|
// HDFS-3398 treat primary DN is down since client is unable to
|
|
|
- // write to primary DN
|
|
|
- errorIndex = 0;
|
|
|
+ // write to primary DN. If a failed or restarting node has already
|
|
|
+ // been recorded by the responder, the following call will have no
|
|
|
+ // effect. Pipeline recovery can handle only one node error at a
|
|
|
+ // time. If the primary node fails again during the recovery, it
|
|
|
+ // will be taken out then.
|
|
|
+ tryMarkPrimaryDatanodeFailed();
|
|
|
throw e;
|
|
|
}
|
|
|
lastPacket = Time.now();
|
|
@@ -609,12 +616,16 @@ public class DFSOutputStream extends FSOutputSummer
|
|
|
Thread.sleep(artificialSlowdown);
|
|
|
}
|
|
|
} catch (Throwable e) {
|
|
|
- DFSClient.LOG.warn("DataStreamer Exception", e);
|
|
|
+ // Log warning if there was a real error.
|
|
|
+ if (restartingNodeIndex == -1) {
|
|
|
+ DFSClient.LOG.warn("DataStreamer Exception", e);
|
|
|
+ }
|
|
|
if (e instanceof IOException) {
|
|
|
setLastException((IOException)e);
|
|
|
}
|
|
|
hasError = true;
|
|
|
- if (errorIndex == -1) { // not a datanode error
|
|
|
+ if (errorIndex == -1 && restartingNodeIndex == -1) {
|
|
|
+ // Not a datanode issue
|
|
|
streamerClosed = true;
|
|
|
}
|
|
|
}
|
|
@@ -694,6 +705,65 @@ public class DFSOutputStream extends FSOutputSummer
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ // The following synchronized methods are used whenever
|
|
|
+ // errorIndex or restartingNodeIndex is set. This is because
|
|
|
+ // check & set needs to be atomic. Simply reading variables
|
|
|
+ // does not require a synchronization. When responder is
|
|
|
+ // not running (e.g. during pipeline recovery), there is no
|
|
|
+ // need to use these methods.
|
|
|
+
|
|
|
+ /** Set the error node index. Called by responder */
|
|
|
+ synchronized void setErrorIndex(int idx) {
|
|
|
+ errorIndex = idx;
|
|
|
+ }
|
|
|
+
|
|
|
+ /** Set the restarting node index. Called by responder */
|
|
|
+ synchronized void setRestartingNodeIndex(int idx) {
|
|
|
+ restartingNodeIndex = idx;
|
|
|
+ // If the data streamer has already set the primary node
|
|
|
+ // bad, clear it. It is likely that the write failed due to
|
|
|
+ // the DN shutdown. Even if it was a real failure, the pipeline
|
|
|
+ // recovery will take care of it.
|
|
|
+ errorIndex = -1;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * This method is used when no explicit error report was received,
|
|
|
+ * but something failed. When the primary node is a suspect or
|
|
|
+ * unsure about the cause, the primary node is marked as failed.
|
|
|
+ */
|
|
|
+ synchronized void tryMarkPrimaryDatanodeFailed() {
|
|
|
+ // There should be no existing error and no ongoing restart.
|
|
|
+ if ((errorIndex == -1) && (restartingNodeIndex == -1)) {
|
|
|
+ errorIndex = 0;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Examine whether it is worth waiting for a node to restart.
|
|
|
+ * @param index the node index
|
|
|
+ */
|
|
|
+ boolean shouldWaitForRestart(int index) {
|
|
|
+ // Only one node in the pipeline.
|
|
|
+ if (nodes.length == 1) {
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+
|
|
|
+ // Is it a local node?
|
|
|
+ InetAddress addr = null;
|
|
|
+ try {
|
|
|
+ addr = InetAddress.getByName(nodes[index].getIpAddr());
|
|
|
+ } catch (java.net.UnknownHostException e) {
|
|
|
+ // we are passing an ip address. this should not happen.
|
|
|
+ assert false;
|
|
|
+ }
|
|
|
+
|
|
|
+ if (addr != null && NetUtils.isLocalAddress(addr)) {
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
//
|
|
|
// Processes responses from the datanodes. A packet is removed
|
|
|
// from the ackQueue when its response arrives.
|
|
@@ -727,8 +797,20 @@ public class DFSOutputStream extends FSOutputSummer
|
|
|
// processes response status from datanodes.
|
|
|
for (int i = ack.getNumOfReplies()-1; i >=0 && dfsClient.clientRunning; i--) {
|
|
|
final Status reply = ack.getReply(i);
|
|
|
+ // Restart will not be treated differently unless it is
|
|
|
+ // the local node or the only one in the pipeline.
|
|
|
+ if (PipelineAck.isRestartOOBStatus(reply) &&
|
|
|
+ shouldWaitForRestart(i)) {
|
|
|
+ restartDeadline = dfsClient.getConf().datanodeRestartTimeout +
|
|
|
+ Time.now();
|
|
|
+ setRestartingNodeIndex(i);
|
|
|
+ String message = "A datanode is restarting: " + targets[i];
|
|
|
+ DFSClient.LOG.info(message);
|
|
|
+ throw new IOException(message);
|
|
|
+ }
|
|
|
+ // node error
|
|
|
if (reply != SUCCESS) {
|
|
|
- errorIndex = i; // first bad datanode
|
|
|
+ setErrorIndex(i); // first bad datanode
|
|
|
throw new IOException("Bad response " + reply +
|
|
|
" for block " + block +
|
|
|
" from datanode " +
|
|
@@ -777,12 +859,16 @@ public class DFSOutputStream extends FSOutputSummer
|
|
|
setLastException((IOException)e);
|
|
|
}
|
|
|
hasError = true;
|
|
|
- errorIndex = errorIndex==-1 ? 0 : errorIndex;
|
|
|
+ // If no explicit error report was received, mark the primary
|
|
|
+ // node as failed.
|
|
|
+ tryMarkPrimaryDatanodeFailed();
|
|
|
synchronized (dataQueue) {
|
|
|
dataQueue.notifyAll();
|
|
|
}
|
|
|
- DFSClient.LOG.warn("DFSOutputStream ResponseProcessor exception "
|
|
|
- + " for block " + block, e);
|
|
|
+ if (restartingNodeIndex == -1) {
|
|
|
+ DFSClient.LOG.warn("DFSOutputStream ResponseProcessor exception "
|
|
|
+ + " for block " + block, e);
|
|
|
+ }
|
|
|
responderClosed = true;
|
|
|
}
|
|
|
}
|
|
@@ -1001,6 +1087,24 @@ public class DFSOutputStream extends FSOutputSummer
|
|
|
boolean success = false;
|
|
|
long newGS = 0L;
|
|
|
while (!success && !streamerClosed && dfsClient.clientRunning) {
|
|
|
+ // Sleep before reconnect if a dn is restarting.
|
|
|
+ // This process will be repeated until the deadline or the datanode
|
|
|
+ // starts back up.
|
|
|
+ if (restartingNodeIndex >= 0) {
|
|
|
+ // 4 seconds or the configured deadline period, whichever is shorter.
|
|
|
+ // This is the retry interval and recovery will be retried in this
|
|
|
+ // interval until timeout or success.
|
|
|
+ long delay = Math.min(dfsClient.getConf().datanodeRestartTimeout,
|
|
|
+ 4000L);
|
|
|
+ try {
|
|
|
+ Thread.sleep(delay);
|
|
|
+ } catch (InterruptedException ie) {
|
|
|
+ lastException.set(new IOException("Interrupted while waiting for " +
|
|
|
+ "datanode to restart. " + nodes[restartingNodeIndex]));
|
|
|
+ streamerClosed = true;
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ }
|
|
|
boolean isRecovery = hasError;
|
|
|
// remove bad datanode from list of datanodes.
|
|
|
// If errorIndex was not set (i.e. appends), then do not remove
|
|
@@ -1037,7 +1141,24 @@ public class DFSOutputStream extends FSOutputSummer
|
|
|
|
|
|
setPipeline(newnodes, newStorageIDs);
|
|
|
|
|
|
- hasError = false;
|
|
|
+ // Just took care of a node error while waiting for a node restart
|
|
|
+ if (restartingNodeIndex >= 0) {
|
|
|
+ // If the error came from a node further away than the restarting
|
|
|
+ // node, the restart must have been complete.
|
|
|
+ if (errorIndex > restartingNodeIndex) {
|
|
|
+ restartingNodeIndex = -1;
|
|
|
+ } else if (errorIndex < restartingNodeIndex) {
|
|
|
+ // the node index has shifted.
|
|
|
+ restartingNodeIndex--;
|
|
|
+ } else {
|
|
|
+ // this shouldn't happen...
|
|
|
+ assert false;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ if (restartingNodeIndex == -1) {
|
|
|
+ hasError = false;
|
|
|
+ }
|
|
|
lastException.set(null);
|
|
|
errorIndex = -1;
|
|
|
}
|
|
@@ -1066,7 +1187,34 @@ public class DFSOutputStream extends FSOutputSummer
|
|
|
} else {
|
|
|
success = createBlockOutputStream(nodes, newGS, isRecovery);
|
|
|
}
|
|
|
- }
|
|
|
+
|
|
|
+ if (restartingNodeIndex >= 0) {
|
|
|
+ assert hasError == true;
|
|
|
+ // check errorIndex set above
|
|
|
+ if (errorIndex == restartingNodeIndex) {
|
|
|
+ // ignore, if came from the restarting node
|
|
|
+ errorIndex = -1;
|
|
|
+ }
|
|
|
+ // still within the deadline
|
|
|
+ if (Time.now() < restartDeadline) {
|
|
|
+ continue; // with in the deadline
|
|
|
+ }
|
|
|
+ // expired. declare the restarting node dead
|
|
|
+ restartDeadline = 0;
|
|
|
+ int expiredNodeIndex = restartingNodeIndex;
|
|
|
+ restartingNodeIndex = -1;
|
|
|
+ DFSClient.LOG.warn("Datanode did not restart in time: " +
|
|
|
+ nodes[expiredNodeIndex]);
|
|
|
+ // Mark the restarting node as failed. If there is any other failed
|
|
|
+ // node during the last pipeline construction attempt, it will not be
|
|
|
+ // overwritten/dropped. In this case, the restarting node will get
|
|
|
+ // excluded in the following attempt, if it still does not come up.
|
|
|
+ if (errorIndex == -1) {
|
|
|
+ errorIndex = expiredNodeIndex;
|
|
|
+ }
|
|
|
+ // From this point on, normal pipeline recovery applies.
|
|
|
+ }
|
|
|
+ } // while
|
|
|
|
|
|
if (success) {
|
|
|
// update pipeline at the namenode
|
|
@@ -1144,6 +1292,7 @@ public class DFSOutputStream extends FSOutputSummer
|
|
|
}
|
|
|
Status pipelineStatus = SUCCESS;
|
|
|
String firstBadLink = "";
|
|
|
+ boolean checkRestart = false;
|
|
|
if (DFSClient.LOG.isDebugEnabled()) {
|
|
|
for (int i = 0; i < nodes.length; i++) {
|
|
|
DFSClient.LOG.debug("pipeline = " + nodes[i]);
|
|
@@ -1192,6 +1341,16 @@ public class DFSOutputStream extends FSOutputSummer
|
|
|
pipelineStatus = resp.getStatus();
|
|
|
firstBadLink = resp.getFirstBadLink();
|
|
|
|
|
|
+ // Got an restart OOB ack.
|
|
|
+ // If a node is already restarting, this status is not likely from
|
|
|
+ // the same node. If it is from a different node, it is not
|
|
|
+ // from the local datanode. Thus it is safe to treat this as a
|
|
|
+ // regular node error.
|
|
|
+ if (PipelineAck.isRestartOOBStatus(pipelineStatus) &&
|
|
|
+ restartingNodeIndex == -1) {
|
|
|
+ checkRestart = true;
|
|
|
+ throw new IOException("A datanode is restarting.");
|
|
|
+ }
|
|
|
if (pipelineStatus != SUCCESS) {
|
|
|
if (pipelineStatus == Status.ERROR_ACCESS_TOKEN) {
|
|
|
throw new InvalidBlockTokenException(
|
|
@@ -1205,9 +1364,12 @@ public class DFSOutputStream extends FSOutputSummer
|
|
|
assert null == blockStream : "Previous blockStream unclosed";
|
|
|
blockStream = out;
|
|
|
result = true; // success
|
|
|
-
|
|
|
+ restartingNodeIndex = -1;
|
|
|
+ hasError = false;
|
|
|
} catch (IOException ie) {
|
|
|
- DFSClient.LOG.info("Exception in createBlockOutputStream", ie);
|
|
|
+ if (restartingNodeIndex == -1) {
|
|
|
+ DFSClient.LOG.info("Exception in createBlockOutputStream", ie);
|
|
|
+ }
|
|
|
if (ie instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
|
|
|
DFSClient.LOG.info("Will fetch a new encryption key and retry, "
|
|
|
+ "encryption key was invalid when connecting to "
|
|
@@ -1230,8 +1392,18 @@ public class DFSOutputStream extends FSOutputSummer
|
|
|
}
|
|
|
}
|
|
|
} else {
|
|
|
+ assert checkRestart == false;
|
|
|
errorIndex = 0;
|
|
|
}
|
|
|
+ // Check whether there is a restart worth waiting for.
|
|
|
+ if (checkRestart && shouldWaitForRestart(errorIndex)) {
|
|
|
+ restartDeadline = dfsClient.getConf().datanodeRestartTimeout +
|
|
|
+ Time.now();
|
|
|
+ restartingNodeIndex = errorIndex;
|
|
|
+ errorIndex = -1;
|
|
|
+ DFSClient.LOG.info("Waiting for the datanode to be restarted: " +
|
|
|
+ nodes[restartingNodeIndex]);
|
|
|
+ }
|
|
|
hasError = true;
|
|
|
setLastException(ie);
|
|
|
result = false; // error
|