HDFS-9373. Erasure coding: friendly log information for write operations with some failed streamers. Contributed by Li Bo.
Change-Id: Ie8ab4ae00e9ee0eb03c32a54bea26a3524308038
This commit is contained in:
parent
4e7d32c0db
commit
5104077e1f
@ -27,6 +27,7 @@
|
|||||||
import java.util.EnumSet;
|
import java.util.EnumSet;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
|
import java.util.LinkedHashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
@ -250,6 +251,8 @@ private void flipDataBuffers() {
|
|||||||
private ExtendedBlock currentBlockGroup;
|
private ExtendedBlock currentBlockGroup;
|
||||||
private final String[] favoredNodes;
|
private final String[] favoredNodes;
|
||||||
private final List<StripedDataStreamer> failedStreamers;
|
private final List<StripedDataStreamer> failedStreamers;
|
||||||
|
private final Map<Integer, Integer> corruptBlockCountMap;
|
||||||
|
private int blockGroupIndex;
|
||||||
|
|
||||||
/** Construct a new output stream for creating a file. */
|
/** Construct a new output stream for creating a file. */
|
||||||
DFSStripedOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat,
|
DFSStripedOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat,
|
||||||
@ -268,6 +271,7 @@ private void flipDataBuffers() {
|
|||||||
numAllBlocks = numDataBlocks + numParityBlocks;
|
numAllBlocks = numDataBlocks + numParityBlocks;
|
||||||
this.favoredNodes = favoredNodes;
|
this.favoredNodes = favoredNodes;
|
||||||
failedStreamers = new ArrayList<>();
|
failedStreamers = new ArrayList<>();
|
||||||
|
corruptBlockCountMap = new LinkedHashMap<>();
|
||||||
|
|
||||||
encoder = CodecUtil.createRSRawEncoder(dfsClient.getConfiguration(),
|
encoder = CodecUtil.createRSRawEncoder(dfsClient.getConfiguration(),
|
||||||
numDataBlocks, numParityBlocks);
|
numDataBlocks, numParityBlocks);
|
||||||
@ -444,6 +448,7 @@ private void allocateNewBlock() throws IOException {
|
|||||||
}
|
}
|
||||||
// assign the new block to the current block group
|
// assign the new block to the current block group
|
||||||
currentBlockGroup = lb.getBlock();
|
currentBlockGroup = lb.getBlock();
|
||||||
|
blockGroupIndex++;
|
||||||
|
|
||||||
final LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup(
|
final LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup(
|
||||||
(LocatedStripedBlock) lb, cellSize, numDataBlocks,
|
(LocatedStripedBlock) lb, cellSize, numDataBlocks,
|
||||||
@ -590,6 +595,7 @@ private void checkStreamerFailures() throws IOException {
|
|||||||
while (newFailed.size() > 0) {
|
while (newFailed.size() > 0) {
|
||||||
failedStreamers.addAll(newFailed);
|
failedStreamers.addAll(newFailed);
|
||||||
coordinator.clearFailureStates();
|
coordinator.clearFailureStates();
|
||||||
|
corruptBlockCountMap.put(blockGroupIndex, failedStreamers.size());
|
||||||
|
|
||||||
// mark all the healthy streamers as external error
|
// mark all the healthy streamers as external error
|
||||||
Set<StripedDataStreamer> healthySet = markExternalErrorOnStreamers();
|
Set<StripedDataStreamer> healthySet = markExternalErrorOnStreamers();
|
||||||
@ -957,6 +963,7 @@ protected synchronized void closeImpl() throws IOException {
|
|||||||
dfsClient.getTracer().newScope("completeFile")) {
|
dfsClient.getTracer().newScope("completeFile")) {
|
||||||
completeFile(currentBlockGroup);
|
completeFile(currentBlockGroup);
|
||||||
}
|
}
|
||||||
|
logCorruptBlocks();
|
||||||
} catch (ClosedChannelException ignored) {
|
} catch (ClosedChannelException ignored) {
|
||||||
} finally {
|
} finally {
|
||||||
setClosed();
|
setClosed();
|
||||||
@ -1004,6 +1011,20 @@ static void sleep(long ms, String op) throws InterruptedIOException {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void logCorruptBlocks() {
|
||||||
|
for (Map.Entry<Integer, Integer> entry : corruptBlockCountMap.entrySet()) {
|
||||||
|
int bgIndex = entry.getKey();
|
||||||
|
int corruptBlockCount = entry.getValue();
|
||||||
|
StringBuilder sb = new StringBuilder();
|
||||||
|
sb.append("Block group <").append(bgIndex).append("> has ")
|
||||||
|
.append(corruptBlockCount).append(" corrupt blocks.");
|
||||||
|
if (corruptBlockCount == numAllBlocks - numDataBlocks) {
|
||||||
|
sb.append(" It's at high risk of losing data.");
|
||||||
|
}
|
||||||
|
LOG.warn(sb.toString());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
ExtendedBlock getBlock() {
|
ExtendedBlock getBlock() {
|
||||||
return currentBlockGroup;
|
return currentBlockGroup;
|
||||||
|
@ -870,6 +870,9 @@ Trunk (Unreleased)
|
|||||||
HDFS-9348. Erasure Coding: DFS GetErasureCodingPolicy API on a non-existent
|
HDFS-9348. Erasure Coding: DFS GetErasureCodingPolicy API on a non-existent
|
||||||
file should be handled properly. (Rakesh R via umamahesh)
|
file should be handled properly. (Rakesh R via umamahesh)
|
||||||
|
|
||||||
|
HDFS-9373. Erasure coding: friendly log information for write operations
|
||||||
|
with some failed streamers. (Li Bo via zhz)
|
||||||
|
|
||||||
HDFS-9451. Clean up depreated umasks and related unit tests.
|
HDFS-9451. Clean up depreated umasks and related unit tests.
|
||||||
(Wei-Chiu Chuang via wheat9)
|
(Wei-Chiu Chuang via wheat9)
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user