|
@@ -328,6 +328,7 @@ class DataStreamer extends Daemon {
|
|
static class ErrorState {
|
|
static class ErrorState {
|
|
private boolean error = false;
|
|
private boolean error = false;
|
|
private int badNodeIndex = -1;
|
|
private int badNodeIndex = -1;
|
|
|
|
+ private boolean waitForRestart = true;
|
|
private int restartingNodeIndex = -1;
|
|
private int restartingNodeIndex = -1;
|
|
private long restartingNodeDeadline = 0;
|
|
private long restartingNodeDeadline = 0;
|
|
private final long datanodeRestartTimeout;
|
|
private final long datanodeRestartTimeout;
|
|
@@ -341,6 +342,7 @@ class DataStreamer extends Daemon {
|
|
badNodeIndex = -1;
|
|
badNodeIndex = -1;
|
|
restartingNodeIndex = -1;
|
|
restartingNodeIndex = -1;
|
|
restartingNodeDeadline = 0;
|
|
restartingNodeDeadline = 0;
|
|
|
|
+ waitForRestart = true;
|
|
}
|
|
}
|
|
|
|
|
|
synchronized boolean hasError() {
|
|
synchronized boolean hasError() {
|
|
@@ -367,14 +369,19 @@ class DataStreamer extends Daemon {
|
|
return restartingNodeIndex;
|
|
return restartingNodeIndex;
|
|
}
|
|
}
|
|
|
|
|
|
- synchronized void initRestartingNode(int i, String message) {
|
|
|
|
|
|
+ synchronized void initRestartingNode(int i, String message,
|
|
|
|
+ boolean shouldWait) {
|
|
restartingNodeIndex = i;
|
|
restartingNodeIndex = i;
|
|
- restartingNodeDeadline = Time.monotonicNow() + datanodeRestartTimeout;
|
|
|
|
- // 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.
|
|
|
|
- badNodeIndex = -1;
|
|
|
|
|
|
+ if (shouldWait) {
|
|
|
|
+ restartingNodeDeadline = Time.monotonicNow() + datanodeRestartTimeout;
|
|
|
|
+ // 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.
|
|
|
|
+ badNodeIndex = -1;
|
|
|
|
+ } else {
|
|
|
|
+ this.waitForRestart = false;
|
|
|
|
+ }
|
|
LOG.info(message);
|
|
LOG.info(message);
|
|
}
|
|
}
|
|
|
|
|
|
@@ -383,7 +390,7 @@ class DataStreamer extends Daemon {
|
|
}
|
|
}
|
|
|
|
|
|
synchronized boolean isNodeMarked() {
|
|
synchronized boolean isNodeMarked() {
|
|
- return badNodeIndex >= 0 || isRestartingNode();
|
|
|
|
|
|
+ return badNodeIndex >= 0 || (isRestartingNode() && doWaitForRestart());
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -408,7 +415,7 @@ class DataStreamer extends Daemon {
|
|
} else if (badNodeIndex < restartingNodeIndex) {
|
|
} else if (badNodeIndex < restartingNodeIndex) {
|
|
// the node index has shifted.
|
|
// the node index has shifted.
|
|
restartingNodeIndex--;
|
|
restartingNodeIndex--;
|
|
- } else {
|
|
|
|
|
|
+ } else if (waitForRestart) {
|
|
throw new IllegalStateException("badNodeIndex = " + badNodeIndex
|
|
throw new IllegalStateException("badNodeIndex = " + badNodeIndex
|
|
+ " = restartingNodeIndex = " + restartingNodeIndex);
|
|
+ " = restartingNodeIndex = " + restartingNodeIndex);
|
|
}
|
|
}
|
|
@@ -450,6 +457,10 @@ class DataStreamer extends Daemon {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ boolean doWaitForRestart() {
|
|
|
|
+ return waitForRestart;
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
private volatile boolean streamerClosed = false;
|
|
private volatile boolean streamerClosed = false;
|
|
@@ -469,6 +480,8 @@ class DataStreamer extends Daemon {
|
|
|
|
|
|
/** Nodes have been used in the pipeline before and have failed. */
|
|
/** Nodes have been used in the pipeline before and have failed. */
|
|
private final List<DatanodeInfo> failed = new ArrayList<>();
|
|
private final List<DatanodeInfo> failed = new ArrayList<>();
|
|
|
|
+ /** Restarting Nodes */
|
|
|
|
+ private List<DatanodeInfo> restartingNodes = new ArrayList<>();
|
|
/** The times have retried to recover pipeline, for the same packet. */
|
|
/** The times have retried to recover pipeline, for the same packet. */
|
|
private volatile int pipelineRecoveryCount = 0;
|
|
private volatile int pipelineRecoveryCount = 0;
|
|
/** Has the current block been hflushed? */
|
|
/** Has the current block been hflushed? */
|
|
@@ -1020,6 +1033,13 @@ class DataStreamer extends Daemon {
|
|
return true;
|
|
return true;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /*
|
|
|
|
+ * Treat all nodes as remote for test when skip enabled.
|
|
|
|
+ */
|
|
|
|
+ if (DFSClientFaultInjector.get().skipRollingRestartWait()) {
|
|
|
|
+ return false;
|
|
|
|
+ }
|
|
|
|
+
|
|
// Is it a local node?
|
|
// Is it a local node?
|
|
InetAddress addr = null;
|
|
InetAddress addr = null;
|
|
try {
|
|
try {
|
|
@@ -1087,11 +1107,11 @@ class DataStreamer extends Daemon {
|
|
}
|
|
}
|
|
// Restart will not be treated differently unless it is
|
|
// Restart will not be treated differently unless it is
|
|
// the local node or the only one in the pipeline.
|
|
// the local node or the only one in the pipeline.
|
|
- if (PipelineAck.isRestartOOBStatus(reply) &&
|
|
|
|
- shouldWaitForRestart(i)) {
|
|
|
|
|
|
+ if (PipelineAck.isRestartOOBStatus(reply)) {
|
|
final String message = "Datanode " + i + " is restarting: "
|
|
final String message = "Datanode " + i + " is restarting: "
|
|
+ targets[i];
|
|
+ targets[i];
|
|
- errorState.initRestartingNode(i, message);
|
|
|
|
|
|
+ errorState.initRestartingNode(i, message,
|
|
|
|
+ shouldWaitForRestart(i));
|
|
throw new IOException(message);
|
|
throw new IOException(message);
|
|
}
|
|
}
|
|
// node error
|
|
// node error
|
|
@@ -1452,6 +1472,14 @@ class DataStreamer extends Daemon {
|
|
*/
|
|
*/
|
|
private boolean handleRestartingDatanode() {
|
|
private boolean handleRestartingDatanode() {
|
|
if (errorState.isRestartingNode()) {
|
|
if (errorState.isRestartingNode()) {
|
|
|
|
+ if (!errorState.doWaitForRestart()) {
|
|
|
|
+ // If node is restarting and not worth to wait for restart then can go
|
|
|
|
+ // ahead with error recovery considering it as bad node for now. Later
|
|
|
|
+ // it should be able to re-consider the same node for future pipeline
|
|
|
|
+ // updates.
|
|
|
|
+ errorState.setBadNodeIndex(errorState.getRestartingNodeIndex());
|
|
|
|
+ return true;
|
|
|
|
+ }
|
|
// 4 seconds or the configured deadline period, whichever is shorter.
|
|
// 4 seconds or the configured deadline period, whichever is shorter.
|
|
// This is the retry interval and recovery will be retried in this
|
|
// This is the retry interval and recovery will be retried in this
|
|
// interval until timeout or success.
|
|
// interval until timeout or success.
|
|
@@ -1483,9 +1511,14 @@ class DataStreamer extends Daemon {
|
|
return false;
|
|
return false;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ String reason = "bad.";
|
|
|
|
+ if (errorState.getRestartingNodeIndex() == badNodeIndex) {
|
|
|
|
+ reason = "restarting.";
|
|
|
|
+ restartingNodes.add(nodes[badNodeIndex]);
|
|
|
|
+ }
|
|
LOG.warn("Error Recovery for " + block + " in pipeline "
|
|
LOG.warn("Error Recovery for " + block + " in pipeline "
|
|
+ Arrays.toString(nodes) + ": datanode " + badNodeIndex
|
|
+ Arrays.toString(nodes) + ": datanode " + badNodeIndex
|
|
- + "("+ nodes[badNodeIndex] + ") is bad.");
|
|
|
|
|
|
+ + "("+ nodes[badNodeIndex] + ") is " + reason);
|
|
failed.add(nodes[badNodeIndex]);
|
|
failed.add(nodes[badNodeIndex]);
|
|
|
|
|
|
DatanodeInfo[] newnodes = new DatanodeInfo[nodes.length-1];
|
|
DatanodeInfo[] newnodes = new DatanodeInfo[nodes.length-1];
|
|
@@ -1691,6 +1724,9 @@ class DataStreamer extends Daemon {
|
|
blockStream = out;
|
|
blockStream = out;
|
|
result = true; // success
|
|
result = true; // success
|
|
errorState.reset();
|
|
errorState.reset();
|
|
|
|
+ // remove all restarting nodes from failed nodes list
|
|
|
|
+ failed.removeAll(restartingNodes);
|
|
|
|
+ restartingNodes.clear();
|
|
} catch (IOException ie) {
|
|
} catch (IOException ie) {
|
|
if (!errorState.isRestartingNode()) {
|
|
if (!errorState.isRestartingNode()) {
|
|
LOG.info("Exception in createBlockOutputStream", ie);
|
|
LOG.info("Exception in createBlockOutputStream", ie);
|
|
@@ -1724,9 +1760,10 @@ class DataStreamer extends Daemon {
|
|
|
|
|
|
final int i = errorState.getBadNodeIndex();
|
|
final int i = errorState.getBadNodeIndex();
|
|
// Check whether there is a restart worth waiting for.
|
|
// Check whether there is a restart worth waiting for.
|
|
- if (checkRestart && shouldWaitForRestart(i)) {
|
|
|
|
- errorState.initRestartingNode(i, "Datanode " + i + " is restarting: "
|
|
|
|
- + nodes[i]);
|
|
|
|
|
|
+ if (checkRestart) {
|
|
|
|
+ errorState.initRestartingNode(i,
|
|
|
|
+ "Datanode " + i + " is restarting: " + nodes[i],
|
|
|
|
+ shouldWaitForRestart(i));
|
|
}
|
|
}
|
|
errorState.setError(true);
|
|
errorState.setError(true);
|
|
lastException.set(ie);
|
|
lastException.set(ie);
|