HDFS-15398. EC: hdfs client hangs due to exception during addBlock. Contributed by Hongbing Wang.
This commit is contained in:
parent
a7526ba9f7
commit
043628dcf1
@ -501,8 +501,14 @@ private void allocateNewBlock() throws IOException {
|
|||||||
|
|
||||||
LOG.debug("Allocating new block group. The previous block group: "
|
LOG.debug("Allocating new block group. The previous block group: "
|
||||||
+ prevBlockGroup);
|
+ prevBlockGroup);
|
||||||
final LocatedBlock lb = addBlock(excludedNodes, dfsClient, src,
|
final LocatedBlock lb;
|
||||||
prevBlockGroup, fileId, favoredNodes, getAddBlockFlags());
|
try {
|
||||||
|
lb = addBlock(excludedNodes, dfsClient, src,
|
||||||
|
prevBlockGroup, fileId, favoredNodes, getAddBlockFlags());
|
||||||
|
} catch (IOException ioe) {
|
||||||
|
closeAllStreamers();
|
||||||
|
throw ioe;
|
||||||
|
}
|
||||||
assert lb.isStriped();
|
assert lb.isStriped();
|
||||||
// assign the new block to the current block group
|
// assign the new block to the current block group
|
||||||
currentBlockGroup = lb.getBlock();
|
currentBlockGroup = lb.getBlock();
|
||||||
|
@ -61,4 +61,40 @@ public void testDFSStripedOutputStreamUpdatePipeline() throws Exception {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test writing ec file hang when applying the second block group occurs
|
||||||
|
* an addBlock exception (e.g. quota exception).
|
||||||
|
*/
|
||||||
|
@Test(timeout = 90000)
|
||||||
|
public void testECWriteHangWhenAddBlockWithException() throws Exception {
|
||||||
|
Configuration conf = new HdfsConfiguration();
|
||||||
|
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1 * 1024 * 1024);
|
||||||
|
try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
|
||||||
|
.numDataNodes(3).build()) {
|
||||||
|
cluster.waitActive();
|
||||||
|
final DistributedFileSystem dfs = cluster.getFileSystem();
|
||||||
|
// Create a file with EC policy
|
||||||
|
Path dir = new Path("/test");
|
||||||
|
dfs.mkdirs(dir);
|
||||||
|
dfs.enableErasureCodingPolicy("XOR-2-1-1024k");
|
||||||
|
dfs.setErasureCodingPolicy(dir, "XOR-2-1-1024k");
|
||||||
|
Path filePath = new Path("/test/file");
|
||||||
|
FSDataOutputStream out = dfs.create(filePath);
|
||||||
|
for (int i = 0; i < 1024 * 1024 * 2; i++) {
|
||||||
|
out.write(i);
|
||||||
|
}
|
||||||
|
dfs.setQuota(dir, 5, 0);
|
||||||
|
try {
|
||||||
|
for (int i = 0; i < 1024 * 1024 * 2; i++) {
|
||||||
|
out.write(i);
|
||||||
|
}
|
||||||
|
} catch (Exception e) {
|
||||||
|
dfs.delete(filePath, true);
|
||||||
|
} finally {
|
||||||
|
// The close should be success, shouldn't get stuck.
|
||||||
|
IOUtils.closeStream(out);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
Loading…
Reference in New Issue
Block a user