hadoop-common-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Steve Loughran (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HADOOP-13726) Enforce that FileSystem initializes only a single instance of the requested FileSystem.
Date Wed, 12 Apr 2017 20:56:42 GMT

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

Steve Loughran commented on HADOOP-13726:
-----------------------------------------

I've realised there's another thing we can do here. We're assuming that every FS instance
has to be initialized before its added to the cache. If we just make the cache the unique
factory for an FS instance, but the init is checked for and done after, then the sync locks
across the cache would be minimal, it'd only be threads awaiting access to the specific FS
instance which would hold.

What I don't see is how you'd implement this with the current FS code (no real state model,
etc)., more specifically, no "isInitialized" property. Still, one could be added, for a sequence
like
{code}
Filesystem.get(uri, conf) {


fs = CACHE.get(...)
if (!fs.isInitialized()} {
  sychronized(fs) {
    if(!fs.isInitialized()} {
       fs.initialize(...)
       fs.setInitialized(true)
    }
   }
 }
 return fs;
}

{code}

> Enforce that FileSystem initializes only a single instance of the requested FileSystem.
> ---------------------------------------------------------------------------------------
>
>                 Key: HADOOP-13726
>                 URL: https://issues.apache.org/jira/browse/HADOOP-13726
>             Project: Hadoop Common
>          Issue Type: Improvement
>          Components: fs
>            Reporter: Chris Nauroth
>            Assignee: Manjunath Anand
>
> The {{FileSystem}} cache is intended to guarantee reuse of instances by multiple call
sites or multiple threads.  The current implementation does provide this guarantee, but there
is a brief race condition window during which multiple threads could perform redundant initialization.
 If the file system implementation has expensive initialization logic, then this is wasteful.
 This issue proposes to eliminate that race condition and guarantee initialization of only
a single instance.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


Mime
View raw message