HDFS-11195. Return error when appending files by webhdfs rest api fails. Contributed by Yuanbo Liu.
(cherry picked from commit 5b7acdd206
)
This commit is contained in:
parent
7a92345f16
commit
1ab279c311
|
@ -55,9 +55,13 @@ class HdfsWriter extends SimpleChannelInboundHandler<HttpContent> {
|
||||||
throws IOException {
|
throws IOException {
|
||||||
chunk.content().readBytes(out, chunk.content().readableBytes());
|
chunk.content().readBytes(out, chunk.content().readableBytes());
|
||||||
if (chunk instanceof LastHttpContent) {
|
if (chunk instanceof LastHttpContent) {
|
||||||
|
try {
|
||||||
|
releaseDfsResourcesAndThrow();
|
||||||
response.headers().set(CONNECTION, CLOSE);
|
response.headers().set(CONNECTION, CLOSE);
|
||||||
ctx.write(response).addListener(ChannelFutureListener.CLOSE);
|
ctx.write(response).addListener(ChannelFutureListener.CLOSE);
|
||||||
releaseDfsResources();
|
} catch (Exception cause) {
|
||||||
|
exceptionCaught(ctx, cause);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -71,7 +75,10 @@ class HdfsWriter extends SimpleChannelInboundHandler<HttpContent> {
|
||||||
releaseDfsResources();
|
releaseDfsResources();
|
||||||
DefaultHttpResponse resp = ExceptionHandler.exceptionCaught(cause);
|
DefaultHttpResponse resp = ExceptionHandler.exceptionCaught(cause);
|
||||||
resp.headers().set(CONNECTION, CLOSE);
|
resp.headers().set(CONNECTION, CLOSE);
|
||||||
ctx.writeAndFlush(response).addListener(ChannelFutureListener.CLOSE);
|
ctx.writeAndFlush(resp).addListener(ChannelFutureListener.CLOSE);
|
||||||
|
if (LOG != null && LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("Exception in channel handler ", cause);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void releaseDfsResources() {
|
private void releaseDfsResources() {
|
||||||
|
@ -79,4 +86,8 @@ class HdfsWriter extends SimpleChannelInboundHandler<HttpContent> {
|
||||||
IOUtils.cleanup(LOG, client);
|
IOUtils.cleanup(LOG, client);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void releaseDfsResourcesAndThrow() throws Exception {
|
||||||
|
out.close();
|
||||||
|
client.close();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -1835,12 +1835,19 @@ public class MiniDFSCluster implements AutoCloseable {
|
||||||
*/
|
*/
|
||||||
public void shutdownDataNodes() {
|
public void shutdownDataNodes() {
|
||||||
for (int i = dataNodes.size()-1; i >= 0; i--) {
|
for (int i = dataNodes.size()-1; i >= 0; i--) {
|
||||||
LOG.info("Shutting down DataNode " + i);
|
shutdownDataNode(i);
|
||||||
DataNode dn = dataNodes.remove(i).datanode;
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Shutdown the datanode at a given index.
|
||||||
|
*/
|
||||||
|
public void shutdownDataNode(int dnIndex) {
|
||||||
|
LOG.info("Shutting down DataNode " + dnIndex);
|
||||||
|
DataNode dn = dataNodes.remove(dnIndex).datanode;
|
||||||
dn.shutdown();
|
dn.shutdown();
|
||||||
numDataNodes--;
|
numDataNodes--;
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Shutdown all the namenodes.
|
* Shutdown all the namenodes.
|
||||||
|
|
|
@ -1173,4 +1173,43 @@ public class TestWebHDFS {
|
||||||
cluster.shutdown();
|
cluster.shutdown();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testWebHdfsAppend() throws Exception {
|
||||||
|
MiniDFSCluster cluster = null;
|
||||||
|
final Configuration conf = WebHdfsTestUtil.createConf();
|
||||||
|
final int dnNumber = 3;
|
||||||
|
try {
|
||||||
|
|
||||||
|
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(dnNumber).build();
|
||||||
|
|
||||||
|
final WebHdfsFileSystem webFS = WebHdfsTestUtil.getWebHdfsFileSystem(
|
||||||
|
conf, WebHdfsConstants.WEBHDFS_SCHEME);
|
||||||
|
|
||||||
|
final DistributedFileSystem fs = cluster.getFileSystem();
|
||||||
|
|
||||||
|
final Path appendFile = new Path("/testAppend.txt");
|
||||||
|
final String content = "hello world";
|
||||||
|
DFSTestUtil.writeFile(fs, appendFile, content);
|
||||||
|
|
||||||
|
for (int index = 0; index < dnNumber - 1; index++){
|
||||||
|
cluster.shutdownDataNode(index);
|
||||||
|
}
|
||||||
|
cluster.restartNameNodes();
|
||||||
|
cluster.waitActive();
|
||||||
|
|
||||||
|
try {
|
||||||
|
DFSTestUtil.appendFile(webFS, appendFile, content);
|
||||||
|
fail("Should fail to append file since "
|
||||||
|
+ "datanode number is 1 and replication is 3");
|
||||||
|
} catch (IOException ignored) {
|
||||||
|
String resultContent = DFSTestUtil.readFile(fs, appendFile);
|
||||||
|
assertTrue(resultContent.equals(content));
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
if (cluster != null) {
|
||||||
|
cluster.shutdown(true);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue