You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-dev@hadoop.apache.org by "Jiandan Yang (JIRA)" <ji...@apache.org> on 2017/10/31 03:24:00 UTC
[jira] [Created] (HDFS-12748) Standby NameNode memory leak when
accessing webhdfs GETHOMEDIRECTORY
Jiandan Yang created HDFS-12748:
------------------------------------
Summary: Standby NameNode memory leak when accessing webhdfs GETHOMEDIRECTORY
Key: HDFS-12748
URL: https://issues.apache.org/jira/browse/HDFS-12748
Project: Hadoop HDFS
Issue Type: Bug
Components: hdfs
Affects Versions: 2.8.2
Reporter: Jiandan Yang
In our production environment, the standby NN often do fullgc, through mat we found the largest object is FileSystem$Cache, which contains 7,844,890 DistributedFileSystem.
By view hierarchy of method FileSystem.get() , I found only NamenodeWebHdfsMethods#get call FileSystem.get(). I don't know why creating different DistributedFileSystem every time instead of get a FileSystem from cache.
{code:java}
case GETHOMEDIRECTORY: {
final String js = JsonUtil.toJsonString("Path",
FileSystem.get(conf != null ? conf : new Configuration())
.getHomeDirectory().toUri().getPath());
return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
}
{code}
When we close FileSystem when GETHOMEDIRECTORY, NN don't do fullgc.
{code:java}
case GETHOMEDIRECTORY: {
FileSystem fs = null;
try {
fs = FileSystem.get(conf != null ? conf : new Configuration());
final String js = JsonUtil.toJsonString("Path",
fs.getHomeDirectory().toUri().getPath());
return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
} finally {
if (fs != null) {
fs.close();
}
}
}
{code}
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)
---------------------------------------------------------------------
To unsubscribe, e-mail: hdfs-dev-unsubscribe@hadoop.apache.org
For additional commands, e-mail: hdfs-dev-help@hadoop.apache.org