hadoop-common-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Manjunath Anand (JIRA)" <j...@apache.org>
Subject [jira] [Comment Edited] (HADOOP-13726) Enforce that FileSystem initializes only a single instance of the requested FileSystem.
Date Sun, 19 Mar 2017 08:15:41 GMT

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

Manjunath Anand edited comment on HADOOP-13726 at 3/19/17 8:14 AM:
-------------------------------------------------------------------

I looked at the code which is called by both [LoadingCache#get|http://google.github.io/guava/releases/snapshot/api/docs/com/google/common/cache/LoadingCache.html#get-K-]
and [Cache#get(key,Callable)|http://google.github.io/guava/releases/snapshot/api/docs/com/google/common/cache/Cache.html#get-K-java.util.concurrent.Callable-]
and they both call the same code so we should be good with the locking granularity as mentioned
by [~cnauroth] in his comments and also as per my understanding I dont see any issue with
referring to state (uri and conf) other than what is in the key unlike what is mentioned in
the [Cache#get(key,Callable)|http://google.github.io/guava/releases/snapshot/api/docs/com/google/common/cache/Cache.html#get-K-java.util.concurrent.Callable-]
 doc which says:- {quote} Or use LoadingCache.get(K), which lacks the ability to refer to
state other than that in the key. {quote}

The only concern I saw during testing of the code in the above comment versus the {{computeIfAbsent}}
code approach presented in earlier comments is that if there are multiple threads trying to
initialize concurrently same FileSystem and if the thread which succeeded in getting the lock
throws an exception during FileSystem initialization, then all other threads waiting for the
result will get ExecutionException and wouldnot retry serially unlike the {{computeIfAbsent}}
wherein upon one thread throwing exception during initialization, the other concurrent thread
which was waiting would proceed with the initialization which effectively means a cool auto
retry feature with {{computeIfAbsent}} approach during concurrent threads for same FileSystem



was (Author: manju_hadoop):
I looked at the code which is called by both [LoadingCache#get|http://google.github.io/guava/releases/snapshot/api/docs/com/google/common/cache/LoadingCache.html#get-K-]
and [Cache#get(key,Callable)|http://google.github.io/guava/releases/snapshot/api/docs/com/google/common/cache/Cache.html#get-K-java.util.concurrent.Callable-]
and they both call the same code so we should be good with the locking granularity as mentioned
by [~cnauroth] in his comments and also as per my understanding I dont see any issue with
referring to state (uri and conf) other than what is in the key unlike what is mentioned in
the [Cache#get(key,Callable)|http://google.github.io/guava/releases/snapshot/api/docs/com/google/common/cache/Cache.html#get-K-java.util.concurrent.Callable-]
 doc which says:- {quote} Or use LoadingCache.get(K), which lacks the ability to refer to
state other than that in the key. {quote}

The only concern I saw during testing of the code in the above comment versus the {{computeIfAbsent}}
code approach presented in earlier comments is that if there are multiple threads trying to
initialize concurrently same FileSystem and if the thread which succeeded in getting the lock
throws an exception then all other threads waiting for the result will get ExecutionException
and wouldnot retry serially unlike the {{computeIfAbsent}} wherein upon one thread throwing
exception during initialization, the other concurrent thread which was waiting would proceed
with the initialization which effectively means a cool auto retry feature with {{computeIfAbsent}}
approach during concurrent threads for same FileSystem


> 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