HDFS-15192. Leaking stream when access encrypted files hit exception via webhdfs.

This commit is contained in:
Xiaoyu Yao 2020-02-25 12:09:43 -08:00
parent dda00d3ff5
commit a2ce53d8ff
2 changed files with 19 additions and 6 deletions

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.fs.Options;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSOutputStream;
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
@ -988,10 +989,14 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE) :
EnumSet.of(CreateFlag.CREATE);
fos = dfsClient.createWrappedOutputStream(
dfsClient.create(fileIdPath, permission, flag, false, replication,
blockSize, null, bufferSize, null),
null);
final DFSOutputStream dfsos = dfsClient.create(fileIdPath, permission, flag,
false, replication, blockSize, null, bufferSize, null);
try {
fos = dfsClient.createWrappedOutputStream(dfsos, null);
} catch (IOException ex) {
dfsos.close();
throw ex;
}
if ((createMode == Nfs3Constant.CREATE_UNCHECKED)
|| (createMode == Nfs3Constant.CREATE_GUARDED)) {

View File

@ -29,6 +29,7 @@ import io.netty.handler.codec.http.HttpMethod;
import io.netty.handler.codec.http.HttpRequest;
import io.netty.handler.codec.http.QueryStringDecoder;
import io.netty.handler.stream.ChunkedStream;
import org.apache.hadoop.hdfs.DFSOutputStream;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
@ -215,9 +216,16 @@ public class WebHdfsHandler extends SimpleChannelInboundHandler<HttpRequest> {
}
final DFSClient dfsClient = newDfsClient(nnId, confForCreate);
OutputStream out = dfsClient.createWrappedOutputStream(dfsClient.create(
final DFSOutputStream dfsos = dfsClient.create(
path, permission, flags, createParent, replication, blockSize, null,
bufferSize, null), null);
bufferSize, null);
OutputStream out;
try {
out = dfsClient.createWrappedOutputStream(dfsos, null);
} catch (IOException ex) {
dfsos.close();
throw ex;
}
resp = new DefaultHttpResponse(HTTP_1_1, CREATED);