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

Zoltan Haindrich commented on TEZ-4080:
---------------------------------------

Thank you [~jeagles] for taking a look!
sorry; but I was busy doing other things - always wanted to get back to this; 
now it seems I can get back to this....

I've tried to use this cloaseAllForUGI methods and the robustness test I've put 
together pass with these changes as well...

Somehow I don't feel that using those methods will lead to 100% correct 
solutions in every cases - In Hive we may have multiple sessions - even for the 
same user; if I understand this correctly - issuing a closeall for one session 
will kill the filesystem objects for the other as well....anyway... I will 
create some testcases for that - and see how wrong I am with the these 
assumptions :D

> 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.02.patch, 
> 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
(v8.3.2#803003)

Reply via email to