[ 
https://issues.apache.org/jira/browse/TEZ-4080?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16892082#comment-16892082
 ] 

Jonathan Eagles commented on TEZ-4080:
--------------------------------------

[~kgyrtkirk], took a look at the patch and description to understand the 
problem better. The design of the FileSystem cache is that there are three 
parts to the key in getting a cache hit. 1) Each new FileSystem instance has a 
unique URI, 2) is given an immutable globally unique ID, and 3) a UGI 
(UserGroupInformation) that created the filesystem. This leads me to believe 
that there are multiple UGIs at play in this scenario that prevent the cache 
from working properly.

Quick glance at createRemoteUser calls, noting that each createRemoteUser call 
is considered unique regardless of whether the user is the same as previous 
users.
{code}
// get a list of non-test createRemoteUser call
// $ git grep createRemoteUser | grep -v test
tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java:    
UserGroupInformation userUgi = 
UserGroupInformation.createRemoteUser(UserGroupInformation
tez-api/src/main/java/org/apache/tez/common/security/JobTokenIdentifier.java:   
 return UserGroupInformation.createRemoteUser(jobid.toString());
tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java:        
.createRemoteUser(jobUserName);
tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java:      dagUGI 
= UserGroupInformation.createRemoteUser(this.userName);
tez-dag/src/main/java/org/apache/tez/dag/app/web/AMWebController.java:      
callerUGI = UserGroupInformation.createRemoteUser(remoteUser);
tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java:  
  UserGroupInformation taskOwner = 
UserGroupInformation.createRemoteUser(tokenIdentifier);
tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java:  
      childUGI = UserGroupInformation.createRemoteUser(user);
{code}

Best practices for FileSystem cleanup per UGI is to use the 
FileSystem.closeAllForUGI.
{code}
// get a list of file systems that were closed per UGI, excluding tests
$ git grep FileSystem.closeAllForUGI | grep -v test
tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java:        
FileSystem.closeAllForUGI(context.getCurrentDAG().getDagUGI());
tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java:  
        FileSystem.closeAllForUGI(childUGI);
{code}

>From this, we can conclude that there is potentially some file systems left in 
>the cache for each of the UGI's that were left open (assuming some filesystems 
>were opened using this UGI). Your analysis shows the same.

Could you change from closing individual file systems to closing all file 
systems for the UGI?

> TezClient should close FileSystem objects to prevent leak
> ---------------------------------------------------------
>
>                 Key: TEZ-4080
>                 URL: https://issues.apache.org/jira/browse/TEZ-4080
>             Project: Apache Tez
>          Issue Type: Bug
>    Affects Versions: 0.9.1
>            Reporter: Zoltan Haindrich
>            Priority: Major
>         Attachments: TEZ-4080.branch-0.9.1.patch
>
>
> When opening/closing a lot of tez clients; some FileSystem object references 
> are retained even after the client is closed - due to the fact the  
> FileSystem has a "cache" which collects all open FileSystem objects - to be 
> able to close all of them from a single shutdownhook.
> Not closing these FileSystem objects causes them to pile up in the "cache" 
> which has hard references to them 
> In a simple hive test which was run with 150M of memory; these "lost" 
> filesystem objects could result in an OOM after ~170 sessions.
> A sample creation stack trace of a FileSystem object:
> {code}
>         at 
> org.apache.hadoop.hive.ql.io.ProxyLocalFileSystem.<init>(ProxyLocalFileSystem.java:49)
>         at sun.reflect.GeneratedConstructorAccessor83.newInstance(Unknown 
> Source)
>         at 
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
>         at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
>         at 
> org.apache.hadoop.util.ReflectionUtils.newInstance(ReflectionUtils.java:133)
>         at 
> org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3353)
>         at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:124)
>         at 
> org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3403)
>         at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3371)
>         at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:477)
>         at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361)
>         at 
> org.apache.tez.dag.app.DAGAppMaster.serviceInit(DAGAppMaster.java:502)
>         at 
> org.apache.hadoop.service.AbstractService.init(AbstractService.java:164)
>         at org.apache.tez.dag.app.DAGAppMaster$9.run(DAGAppMaster.java:2663)
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)

Reply via email to