You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by GitBox <gi...@apache.org> on 2018/07/03 21:06:58 UTC

[GitHub] ilooner commented on issue #1296: DRILL-5365: Prevent plugin config from changing default fs. Make DrillFileSystem Immutable.

ilooner commented on issue #1296: DRILL-5365: Prevent plugin config from changing default fs. Make DrillFileSystem Immutable.
URL: https://github.com/apache/drill/pull/1296#issuecomment-402292010
 
 
   ## Problem
   
   @paul-rogers @vdiravka I have an update. Chun was unable to reproduce the issue, however after staring at the code for several days, I think I see the source of the issue. I believe the issue mostly stems from two bugs in the FileSystem.get method provided to us by hadoop.
   
     1. **Bug # 1** is that FileSystem.get does not actually take into account the Configuration object you give it. This means that if you ask for two different FileSystem system objects with different configurations for dfs, FileSystem.get assumes both are the same and the first dfs FileSystem and config requested wins and is the one that is always returned. That's pretty crazy to me, but you can check the equals method for **FileSystem.Cache.Key** yourself.
    2. **Bug # 2** The **fs.default.name** property is not honored when doing FileSystem.get, only **fs.defaultFS** is honored. You can see this by tracing through the code:
       1. FileSystem.get(Configuration conf) calls FileSystem.getDefaultUri(Configuration conf) 
       2. FileSystem.getDefaultUri(Configuration conf) looks up FileSystem.FS_DEFAULT_NAME_KEY in the Configuration
       3. FileSystem.FS_DEFAULT_NAME_KEY is **fs.defaultFS**.
       4. If **fs.defaultFS** is not defined then the uri for caching purposes is assumed to be **file://**
   
   These bugs would basically completely break things if hive plugin and the filesystem plugin were storing files in different places. I suspect we aren't seeing any issues because most people only have one dfs cluster to store their data for both hive and other applications.
   
   ## Solution
   
    * Bypass the caching in FileSystem.get and do our own caching in DrillFileSystem, which takes into account Configuration objects.
    * Set **fs.defaultFS**  as well as **fs.default.name** in our configuration objects.
    * Make DrillFileSystem immutable and make things cleaner and more explicit in general.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services