HDFS-8183. Erasure Coding: Improve DFSStripedOutputStream closing of datastreamer threads. Contributed by Rakesh R.
This commit is contained in:
parent
f0628280c3
commit
1a31f1c303
|
@ -149,3 +149,6 @@
|
||||||
|
|
||||||
HDFS-8282. Erasure coding: move striped reading logic to StripedBlockUtil.
|
HDFS-8282. Erasure coding: move striped reading logic to StripedBlockUtil.
|
||||||
(Zhe Zhang)
|
(Zhe Zhang)
|
||||||
|
|
||||||
|
HDFS-8183. Erasure Coding: Improve DFSStripedOutputStream closing of
|
||||||
|
datastreamer threads. (Rakesh R via Zhe Zhang)
|
||||||
|
|
|
@ -331,18 +331,26 @@ public class DFSStripedOutputStream extends DFSOutputStream {
|
||||||
// interrupt datastreamer if force is true
|
// interrupt datastreamer if force is true
|
||||||
@Override
|
@Override
|
||||||
protected void closeThreads(boolean force) throws IOException {
|
protected void closeThreads(boolean force) throws IOException {
|
||||||
|
int index = 0;
|
||||||
|
boolean exceptionOccurred = false;
|
||||||
for (StripedDataStreamer streamer : streamers) {
|
for (StripedDataStreamer streamer : streamers) {
|
||||||
try {
|
try {
|
||||||
streamer.close(force);
|
streamer.close(force);
|
||||||
streamer.join();
|
streamer.join();
|
||||||
streamer.closeSocket();
|
streamer.closeSocket();
|
||||||
} catch (InterruptedException e) {
|
} catch (InterruptedException | IOException e) {
|
||||||
throw new IOException("Failed to shutdown streamer");
|
DFSClient.LOG.error("Failed to shutdown streamer: name="
|
||||||
|
+ streamer.getName() + ", index=" + index + ", file=" + src, e);
|
||||||
|
exceptionOccurred = true;
|
||||||
} finally {
|
} finally {
|
||||||
streamer.setSocketToNull();
|
streamer.setSocketToNull();
|
||||||
setClosed();
|
setClosed();
|
||||||
|
index++;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
if (exceptionOccurred) {
|
||||||
|
throw new IOException("Failed to shutdown streamer");
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
Loading…
Reference in New Issue