Thanks Harsh and Daryn !
Daryn, I tried the option you suggested and changed getFileSystem(String user)
implementation as following...
private static FileSystem getFileSystem(String user) throws Exception {
final Configuration conf = new Configuration();
conf.set("fs.default.name", "file:///");
UserGroupInformation ugi = UserGroupInformation.createRemoteUser(user);
return ugi.doAs(new PrivilegedAction<FileSystem>() {
@Override
public FileSystem run() {
try {
return FileSystem.get(new URI("file:///"),conf);
} catch(Exception ex) {
ex.printStackTrace();
return null;
}
}
});
}
but with this implementation also multiple FileSystem instances are created.
Did I miss something?
As per as my usecase is concerned, we are developing a oozie like application
where clients can give a workflow and their hadoop user and also store data on
hdfs through us. We are using FileSystem.get(..) api to get handle to
FileSystem impl specific to the client's hadoop user.
Thanks
----- Original Message -----
From: Daryn Sharp <[email protected]>
To: "[email protected]" <[email protected]>
Cc: Himanshu Gupta <[email protected]>
Sent: Tuesday, 11 September 2012 12:49 AM
Subject: Re: FileSystem.get(Uri,Configuration,String) caching issue
Yes, the sample code demonstrates what happens when you use a new UGI for every
FileSystem.get. If possible you should avoid the variant of fs.get that
accepts the user as a string since it may create another UGI from the string
user on every call. The cache will fill with instances for every new UGI.
If you need a filesystem for your kerberos or unix user, don't bother with
passing the user or manipulating the UGI since the default for fs.get uses the
current user ugi. If you are trying to be another user, use
UGI.createRemoteUser and use it to execute your code within a doAs block. All
the fs gets will return the same cached object as long as you don't pass in the
user string.
Normally only daemons that accept requests for arbitrary users have to deal
with the ugi. If this is simple app code then I'd suggest leaving the ugi
alone. If you have a complex use case, please describe it to us so better
advice can be offered. I hope this helps!
Daryn
On Sep 10, 2012, at 10:14 AM, Harsh J wrote:
> What you're seeing is genuine.
>
> You seem to be hitting the abuse scenario described by Daryn here:
> https://issues.apache.org/jira/browse/HDFS-3545?focusedCommentId=13398502&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13398502
>
> You can instead choose to skip passing a username for local FS
> instances, as I think they don't make much sense when done locally -
> as a workaround.
>
> On Mon, Sep 10, 2012 at 4:59 PM, Himanshu Gupta <[email protected]> wrote:
>> I am using FileSystem.get(URI uri, Configuration conf, String user) to
>> create FileSystem implementation(LocalFileSystem in this case) instances.
>> From what I know, FileSystem internally has a cache to retain the objects
>> based on uri and user. So if I call FileSystem.get(..) method multiple times
>> with same uri and user, then only one instance of LocalFileSystem needs to
>> be created and cached. However, I observed(with hadoop-core-1.0.0) that each
>> call creates a new instance of LocalFileSystem and puts it in the cache
>> leading to memory issues.
>>
>> Please see the code below and let me know if I am doing something wrong.
>>
>> Thanks
>>
>>
>> import java.net.URI;
>>
>> import org.apache.hadoop.conf.Configuration;
>> import org.apache.hadoop.fs.FileSystem;
>>
>> public class FileSystemCacheIssue {
>>
>> private static FileSystem getFileSystem(String user) throws Exception {
>> Configuration conf = new Configuration();
>> conf.set("fs.default.name", "file:///");
>> return FileSystem.get(new URI("file:///"),conf,user);
>> }
>>
>> public static void main(String[] args) throws Exception {
>> for(int i = 0; i < 1000; i++) {
>> getFileSystem("himanshg");
>> }
>>
>> FileSystem fs = getFileSystem("himanshg");
>> System.out.println(fs.getClass().getCanonicalName());
>>
>> //put a breakpoint here and look at the heap dump for number of
>> LocalFileSystem
>> //instances, Ideally I expect it to be 1, but there are 1001
>> System.out.println("Keep your debugger here and check.");
>> }
>> }
>
>
>
> --
> Harsh J