HDFS-2494. Close the streams and DFSClient in DatanodeWebHdfsMethods. Contributed by Uma Maheswara Rao G

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1189360 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Tsz-wo Sze 2011-10-26 18:24:29 +00:00
parent 2ce817aecf
commit 758194382c
2 changed files with 60 additions and 19 deletions

View File

@ -159,6 +159,9 @@ Trunk (unreleased changes)
HDFS-2411. The the auth to local mappings are not being respected, with
webhdfs enabled. (jitendra)
HDFS-2494. Close the streams and DFSClient in DatanodeWebHdfsMethods.
(Uma Maheswara Rao G via szetszwo)
Release 0.23.0 - Unreleased
INCOMPATIBLE CHANGES

View File

@ -152,17 +152,23 @@ public Response run() throws IOException, URISyntaxException {
{
final Configuration conf = new Configuration(datanode.getConf());
final InetSocketAddress nnRpcAddr = NameNode.getAddress(conf);
final DFSClient dfsclient = new DFSClient(nnRpcAddr, conf);
final int b = bufferSize.getValue(conf);
final FSDataOutputStream out = new FSDataOutputStream(dfsclient.create(
fullpath, permission.getFsPermission(),
overwrite.getValue() ? EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)
: EnumSet.of(CreateFlag.CREATE),
replication.getValue(conf), blockSize.getValue(conf), null, b), null);
DFSClient dfsclient = new DFSClient(nnRpcAddr, conf);
FSDataOutputStream out = null;
try {
out = new FSDataOutputStream(dfsclient.create(
fullpath, permission.getFsPermission(),
overwrite.getValue() ? EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)
: EnumSet.of(CreateFlag.CREATE),
replication.getValue(conf), blockSize.getValue(conf), null, b), null);
IOUtils.copyBytes(in, out, b);
} finally {
out.close();
out = null;
dfsclient.close();
dfsclient = null;
} finally {
IOUtils.cleanup(LOG, out);
IOUtils.cleanup(LOG, dfsclient);
}
final InetSocketAddress nnHttpAddr = NameNode.getHttpAddress(conf);
final URI uri = new URI(WebHdfsFileSystem.SCHEME, null,
@ -227,13 +233,19 @@ public Response run() throws IOException {
{
final Configuration conf = new Configuration(datanode.getConf());
final InetSocketAddress nnRpcAddr = NameNode.getAddress(conf);
final DFSClient dfsclient = new DFSClient(nnRpcAddr, conf);
final int b = bufferSize.getValue(conf);
final FSDataOutputStream out = dfsclient.append(fullpath, b, null, null);
DFSClient dfsclient = new DFSClient(nnRpcAddr, conf);
FSDataOutputStream out = null;
try {
out = dfsclient.append(fullpath, b, null, null);
IOUtils.copyBytes(in, out, b);
} finally {
out.close();
out = null;
dfsclient.close();
dfsclient = null;
} finally {
IOUtils.cleanup(LOG, out);
IOUtils.cleanup(LOG, dfsclient);
}
return Response.ok().type(MediaType.APPLICATION_JSON).build();
}
@ -301,18 +313,36 @@ public Response run() throws IOException {
case OPEN:
{
final int b = bufferSize.getValue(conf);
final DFSDataInputStream in = new DFSClient.DFSDataInputStream(
dfsclient.open(fullpath, b, true));
in.seek(offset.getValue());
DFSDataInputStream in = null;
try {
in = new DFSClient.DFSDataInputStream(
dfsclient.open(fullpath, b, true));
in.seek(offset.getValue());
} catch(IOException ioe) {
IOUtils.cleanup(LOG, in);
IOUtils.cleanup(LOG, dfsclient);
throw ioe;
}
final DFSDataInputStream dis = in;
final StreamingOutput streaming = new StreamingOutput() {
@Override
public void write(final OutputStream out) throws IOException {
final Long n = length.getValue();
if (n == null) {
IOUtils.copyBytes(in, out, b);
} else {
IOUtils.copyBytes(in, out, n, false);
DFSDataInputStream dfsin = dis;
DFSClient client = dfsclient;
try {
if (n == null) {
IOUtils.copyBytes(dfsin, out, b);
} else {
IOUtils.copyBytes(dfsin, out, n, false);
}
dfsin.close();
dfsin = null;
dfsclient.close();
client = null;
} finally {
IOUtils.cleanup(LOG, dfsin);
IOUtils.cleanup(LOG, client);
}
}
};
@ -324,7 +354,15 @@ public void write(final OutputStream out) throws IOException {
}
case GETFILECHECKSUM:
{
final MD5MD5CRC32FileChecksum checksum = dfsclient.getFileChecksum(fullpath);
MD5MD5CRC32FileChecksum checksum = null;
DFSClient client = dfsclient;
try {
checksum = client.getFileChecksum(fullpath);
client.close();
client = null;
} finally {
IOUtils.cleanup(LOG, client);
}
final String js = JsonUtil.toJsonString(checksum);
return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
}