[ https://issues.apache.org/jira/browse/HADOOP-13726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15898071#comment-15898071 ]
Chris Nauroth commented on HADOOP-13726: ---------------------------------------- That's an interesting observation about {{computeIfAbsent}}, at least for the 3.x line where we can use Java 8. I am concerned about this statement from [{{computeIfAbsent}} JavaDocs|http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/ConcurrentHashMap.html#computeIfAbsent-K-java.util.function.Function-]: {quote} Some attempted update operations on this map by other threads may be blocked while computation is in progress, so the computation should be short and simple... {quote} Sometimes {{FileSystem}} initialization is neither short nor simple, involving things like network connections and authentication, all of which can suffer problematic failure modes like timeouts. The current code prevents one blocked {{FileSystem}} initialization from stalling all other threads accessing the cache. For example, if there is a blocked connection to s3a://my-bucket, then only threads attempting to access s3a://my-bucket get blocked. Threads accessing a different {{FileSystem}}, such as hdfs://mylocalcluster can still make progress. >From the JavaDocs, I don't see a clear statement of the locking granularity, >so I don't know if {{computeIfAbsent}} would preserve the current behavior. >The [code for >{{computeIfAbsent}}|http://hg.openjdk.java.net/jdk8u/jdk8u/jdk/file/8c93eb3fa1c0/src/share/classes/java/util/concurrent/ConcurrentHashMap.java#l1643] > is complex, and I don't have time right now to read it and understand the >locking granularity. (It also might be unwise to assume a particular locking >implementation is common across all possible JVMs.) This makes me skeptical of {{computeIfAbsent}} as a potential solution for this problem, but it's my first time digging this deeply into that method, so I might have more to learn here. Steve mentioned possibly Guava. I believe [{{LoadingCache}}|https://google.github.io/guava/releases/11.0.2/api/docs/com/google/common/cache/LoadingCache.html] / [{{CacheLoader}}|https://google.github.io/guava/releases/11.0.2/api/docs/com/google/common/cache/CacheLoader.html] do basically what I described in my last comment. We could potentially review that code in more detail to make sure it's a good fit. > 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 > > 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-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org