You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by we...@apache.org on 2021/02/26 02:58:07 UTC
[hadoop] branch branch-3.3 updated: HDFS-15210. EC : File write
hanged when DN is shutdown by admin command. Contributed by Surendra Singh
Lilhore.
This is an automated email from the ASF dual-hosted git repository.
weichiu pushed a commit to branch branch-3.3
in repository https://gitbox.apache.org/repos/asf/hadoop.git
The following commit(s) were added to refs/heads/branch-3.3 by this push:
new a6f86af HDFS-15210. EC : File write hanged when DN is shutdown by admin command. Contributed by Surendra Singh Lilhore.
a6f86af is described below
commit a6f86af39f34f8342587feb16a56857738d3c618
Author: Surendra Singh Lilhore <su...@apache.org>
AuthorDate: Wed Apr 29 10:58:35 2020 +0530
HDFS-15210. EC : File write hanged when DN is shutdown by admin command. Contributed by Surendra Singh Lilhore.
(cherry picked from commit db6252b6c3959220c6f985f940e2e731f99d8e30)
---
.../org/apache/hadoop/hdfs/DFSStripedOutputStream.java | 15 +++++++++++++++
.../java/org/apache/hadoop/hdfs/StripedDataStreamer.java | 3 ++-
2 files changed, 17 insertions(+), 1 deletion(-)
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
index 2ed11ff..7c39656 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
@@ -283,6 +283,7 @@ public class DFSStripedOutputStream extends DFSOutputStream
private ExecutorService flushAllExecutor;
private CompletionService<Void> flushAllExecutorCompletionService;
private int blockGroupIndex;
+ private long datanodeRestartTimeout;
/** Construct a new output stream for creating a file. */
DFSStripedOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat,
@@ -322,6 +323,7 @@ public class DFSStripedOutputStream extends DFSOutputStream
streamers.add(streamer);
}
currentPackets = new DFSPacket[streamers.size()];
+ datanodeRestartTimeout = dfsClient.getConf().getDatanodeRestartTimeout();
setCurrentStreamer(0);
}
@@ -643,6 +645,11 @@ public class DFSStripedOutputStream extends DFSOutputStream
"streamer: " + streamer);
streamer.setExternalError();
healthySet.add(streamer);
+ } else if (!streamer.streamerClosed()
+ && streamer.getErrorState().hasDatanodeError()
+ && streamer.getErrorState().doWaitForRestart()) {
+ healthySet.add(streamer);
+ failedStreamers.remove(streamer);
}
}
return healthySet;
@@ -707,6 +714,14 @@ public class DFSStripedOutputStream extends DFSOutputStream
for (int i = 0; i < numAllBlocks; i++) {
coordinator.offerStreamerUpdateResult(i, newFailed.size() == 0);
}
+ //wait for get notify to failed stream
+ if (newFailed.size() != 0) {
+ try {
+ Thread.sleep(datanodeRestartTimeout);
+ } catch (InterruptedException e) {
+ // Do nothing
+ }
+ }
}
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
index e78beb1..e90e66a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
@@ -143,7 +143,8 @@ public class StripedDataStreamer extends DataStreamer {
// set up the pipeline again with the remaining nodes. when a striped
// data streamer comes here, it must be in external error state.
- assert getErrorState().hasExternalError();
+ assert getErrorState().hasExternalError()
+ || getErrorState().doWaitForRestart();
success = createBlockOutputStream(nodes, nodeStorageTypes,
nodeStorageIDs, newGS, true);
---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org