[jira] [Commented] (HADOOP-13726) Enforce that FileSystem initializes only a single instance of the requested FileSystem.
[ https://issues.apache.org/jira/browse/HADOOP-13726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17803013#comment-17803013 ] Shilun Fan commented on HADOOP-13726: - Bulk update: moved all 3.4.0 non-blocker issues, please move back if it is a blocker. Retarget 3.5.0. > 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 >Priority: Major > Attachments: HADOOP-13726.001.patch > > > 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 (v8.20.10#820010) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-13726) Enforce that FileSystem initializes only a single instance of the requested FileSystem.
[ https://issues.apache.org/jira/browse/HADOOP-13726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15970660#comment-15970660 ] Manjunath Anand commented on HADOOP-13726: -- Thanks [~ste...@apache.org] for your inputs. What you proposed is a good idea. I am thinking we may need to make sure that the unitialized FS object is checked in all methods which exposes the FS object from cache which may need extra coding in other methods. Please correct me if I am wrong on this. I will wait for yours and [~cnauroth] comments to proceed further and come up with new patch as the case be. > 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 > Attachments: HADOOP-13726.001.patch > > > 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
[jira] [Commented] (HADOOP-13726) Enforce that FileSystem initializes only a single instance of the requested FileSystem.
[ https://issues.apache.org/jira/browse/HADOOP-13726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15970374#comment-15970374 ] Manjunath Anand commented on HADOOP-13726: -- Thanks [~cnauroth] for your inputs. I uploaded the patch for trunk as per the discussion in earlier comments. Although no explicit synchronization is done in the Callable while creating the FileSystem, there is no ConcurrentModificationException possibility due to the guava LocalCache implementing java ConcurrentMap. > 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 > Attachments: HADOOP-13726.001.patch > > > 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
[jira] [Commented] (HADOOP-13726) Enforce that FileSystem initializes only a single instance of the requested FileSystem.
[ https://issues.apache.org/jira/browse/HADOOP-13726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15966952#comment-15966952 ] Chris Nauroth commented on HADOOP-13726: bq. ...maybe the {{FileSystem#initialize}} call could be moved inside a {{synchronized}} block on the Cache.Key instance...? No, I'm wrong about this, because it's a different {{Key}} instance on each thread, and therefore multiple threads wouldn't be coordinating on the same lock. > 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-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-13726) Enforce that FileSystem initializes only a single instance of the requested FileSystem.
[ https://issues.apache.org/jira/browse/HADOOP-13726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15966754#comment-15966754 ] Chris Nauroth commented on HADOOP-13726: My understanding is that with use of the Guava {{Cache}}, we'd effectively achieve per-FS locking granularity already, not a lock over the whole {{Cache}} instance, and we'd achieve it without changing state tracking around the FS instances. Multiple threads attempting to retrieve the same FS would block waiting for the first thread to finish initialization, but threads retrieving a different FS could proceed concurrently. If you're trying to achieve it without using the Guava {{Cache}} at all, then maybe the {{FileSystem#initialize}} call could be moved inside a {{syncrhonized}} block on the {{Cache.Key}} instance, and it still wouldn't need state tracking changes around the FS? > 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-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-13726) Enforce that FileSystem initializes only a single instance of the requested FileSystem.
[ 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-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-13726) Enforce that FileSystem initializes only a single instance of the requested FileSystem.
[ https://issues.apache.org/jira/browse/HADOOP-13726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15963519#comment-15963519 ] Chris Nauroth commented on HADOOP-13726: Thank you, [~manju_hadoop]! Your last comment looks to me like a good way to go. Please feel free to attach a patch file as described in the [HowToContribute|https://wiki.apache.org/hadoop/HowToContribute] wiki page. bq. ...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... It's good that you remapped the {{ExecutionException}} back to {{IOException}} in your example. Typical callers are equipped to handle an {{IOException}}. I think this is acceptable, as there has never been any stated contract around {{FileSystem#get}} retrying internally. Calling code that wants to be resilient against transient failure already must have retry logic of its own. > 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-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-13726) Enforce that FileSystem initializes only a single instance of the requested FileSystem.
[ https://issues.apache.org/jira/browse/HADOOP-13726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15931610#comment-15931610 ] Manjunath Anand commented on HADOOP-13726: -- 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 feel 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 {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-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-13726) Enforce that FileSystem initializes only a single instance of the requested FileSystem.
[ https://issues.apache.org/jira/browse/HADOOP-13726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15931329#comment-15931329 ] Manjunath Anand commented on HADOOP-13726: -- Thanks [~ste...@apache.org] for praise in comments , I felt happy. Yes the array in catch block is to get around inside the closure and capture the exception. Thanks [~cnauroth] for your inputs. Yes I agree that the computeIfAbsent is subjected to locking of non equal keys having same hashbucket and usually the chances of it happening are lesser when a large initial size is given to the ConcurrentHashMap and when the hash distribution is better but it can still be a problem. After researching a little bit more on the challenge we have to pass the uri and conf to the load method, I stumbled upon [https://github.com/google/guava/wiki/CachesExplained#from-a-callable] and found a way to pass the URI and Conf to the Callable method without adding them to the Key class and am presenting the code below:- {code} private com.google.common.cache.Cache map = CacheBuilder.newBuilder().build(); private FileSystem getInternal(final URI uri, final Configuration conf, Key key) throws IOException { /** * Calling getIfPresent to avoid unnecessary creation of Callable * object everytime getInternal is called. */ FileSystem fs = map.getIfPresent(key); if(fs != null) { return fs; } if (map.size() == 0 && !ShutdownHookManager.get().isShutdownInProgress()) { ShutdownHookManager.get().addShutdownHook(clientFinalizer, SHUTDOWN_HOOK_PRIORITY); } Callable cb = new Callable() { @Override public FileSystem call() throws Exception { FileSystem fs = createFileSystem(uri, conf); if(fs!=null) throw new IOException(); fs.key = key; if (conf.getBoolean( FS_AUTOMATIC_CLOSE_KEY, FS_AUTOMATIC_CLOSE_DEFAULT)) { toAutoClose.add(key); } return fs; } }; try { fs = map.get(key,cb); } catch (ExecutionException e) { LOG.error("Exception while creating file system for key "+key,e); if(e.getCause() instanceof IOException) throw (IOException)e.getCause(); } return fs; } {code} Note that using guava Cache instead of java Map would mean additional code changes to existing method references such as entrySet, isEmpty, remove, keySet of java Map in the FileSystem and replacing it with corresponding methods from guava Cache. Please let me know your thoughts about this approach and 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-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-13726) Enforce that FileSystem initializes only a single instance of the requested FileSystem.
[ https://issues.apache.org/jira/browse/HADOOP-13726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15924000#comment-15924000 ] Steve Loughran commented on HADOOP-13726: - I was wondering why you are using an array for the return value, but it's to get changes back from a closure, isn't it? Devious. I'll have to use that trick myself. I think that looks pretty good, though I'll defer to Chris's judgement. I'm wondering how to test this. > 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
[jira] [Commented] (HADOOP-13726) Enforce that FileSystem initializes only a single instance of the requested FileSystem.
[ https://issues.apache.org/jira/browse/HADOOP-13726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15923599#comment-15923599 ] Manjunath Anand commented on HADOOP-13726: -- Thanks [~ste...@apache.org] for your inputs. Yes I agree we need to give a large enough initial table size for a better spread of keys in hash buckets and the {{computeIfAbsent}} blocks concurrent updates to the map only if the hashbcuket is same. Can you or [~cnauroth] please provide an approximate initial value for the ConcurrentHashMap to be used. The proposed {{computeIfAbsent}} implentation takes care of following things:- 1) Eliminate race condition and guarantee initialization of only a single instance. 2) Eliminate unnecessary additional synchronization compared to the earlier code 3) In case of a timeout during file initialization and in the case with multiple concurrent threads for the same URI , auto retry by the next waiting thread for file initialization. I am presenting the below code . Please review and let me know your suggestions based on which I can come up with a patch. Also please grant me contributor access to HADOOP to attach a patch to the JIRA. {code} // TODO need to provide a large initial value for this map to avoid hash collision and rehashing private final ConcurrentMap map = new ConcurrentHashMap<>(); private FileSystem getInternal(URI uri, Configuration conf, Key key) throws IOException{ FileSystem fs; fs = map.get(key); if (fs != null) { return fs; } IOException[] ie = new IOException[1]; map.computeIfAbsent(key, k -> { FileSystem fsc = null; try { fsc = createFileSystem(uri, conf); fsc.key = key; } catch (IOException e) { ie[0] = e; } return fsc; }); if(ie[0] != null) throw ie[0]; if (conf.getBoolean( FS_AUTOMATIC_CLOSE_KEY, FS_AUTOMATIC_CLOSE_DEFAULT)) { toAutoClose.add(key); } return map.get(key); } {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 > > 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
[jira] [Commented] (HADOOP-13726) Enforce that FileSystem initializes only a single instance of the requested FileSystem.
[ https://issues.apache.org/jira/browse/HADOOP-13726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15907356#comment-15907356 ] Steve Loughran commented on HADOOP-13726: - I Think the main concern of Chris (and myself) is not the operations which fail, it's those the block for a long time before failing. If it was across all filesystems, then the mechanism would be a disaster: on a shared execution environment I could submit queries tying to talk to endpoints which blocked during TCP setup, and that would kill everything, rather than just my query. If it's just per hash-key, then this can be defended against by: using a large enough initial table to reduce collision risk (though please, not another config option for now), and making sure that the hash key used in the table isn't going put everything in the same hash bucket: want a broad spread of keys > 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
[jira] [Commented] (HADOOP-13726) Enforce that FileSystem initializes only a single instance of the requested FileSystem.
[ https://issues.apache.org/jira/browse/HADOOP-13726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15906663#comment-15906663 ] Manjunath Anand commented on HADOOP-13726: -- Thanks [~cnauroth] for your inputs. After looking at the points you raised I tried few sample code to understand more about {code}computeIfAbsent{code} and found below observations (please refer to the code I shared in my above comment):- 1) I could see that if the hashcode is same for say two similar keys which are passed to computeIfAbsent concurrently then one of them waits for the other to complete, but if the hashcode of the keys are different then it doesnt block each other. 2) Inference from test results of above point is that the below point you raised should be handled fairly by the computeIfAbsent {quote} then only threads attempting to access s3a://my-bucket get blocked. Threads accessing a different FileSystem, such as hdfs://mylocalcluster can still make progress.{quote} I observed one major benefit of using {code}computeIfAbsent{code} after referring to your point which am quoting below:- {quote}FileSystem initialization is neither short nor simple, involving things like network connections and authentication, all of which can suffer problematic failure modes like timeouts. {quote} What I observed during testing is that say if multiple threads try to create FileSystem for the same key and if one thread fails then by using the computeIfAbsent the next thread which was waiting will compute and I observed that this happens until the key has a non null value in the map. So in case of a timeout for first thread attempt to create a filesystem then either the second concurrent thread or any subsequent thread will be able to retry and see if it succeeds. I tried evaluating the suggestion of using {code}LoadingCache/CacheLoader{code} however I came across a problem where in for the overloaded {code}load(K key){code} how do we pass the URI and conf from the getInternal method as the value for the map in getInternal method is not computed based on the Key but based on the URI and conf. There may be a workaround for this but it may involved more code change and redesign is my understanding. I am hoping I didnot miss out anything here but if I did please suggest. I would like to work on this (if you dont mind) but only after you are fine with the above observations and suggest me which approach to go ahead with or if you have any observations which I can explore further. > 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
[jira] [Commented] (HADOOP-13726) Enforce that FileSystem initializes only a single instance of the requested FileSystem.
[ 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
[jira] [Commented] (HADOOP-13726) Enforce that FileSystem initializes only a single instance of the requested FileSystem.
[ https://issues.apache.org/jira/browse/HADOOP-13726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15833516#comment-15833516 ] Manjunath Anand commented on HADOOP-13726: -- I saw the new method computeIfAbsent introduced in ConcurrentHashMap as part of Java 8 and felt appropriate for this scenario. Here's the implementation:- {code} private final ConcurrentMap map = new ConcurrentHashMap<>(); private FileSystem getInternal(URI uri, Configuration conf, Key key) throws IOException{ FileSystem fs; fs = map.get(key); if (fs != null) { return fs; } IOException[] ie = new IOException[1]; map.computeIfAbsent(key, k -> { FileSystem fsc = null; try { fsc = createFileSystem(uri, conf); fsc.key = key; } catch (IOException e) { ie[0] = e; } return fsc; }); if(ie[0] != null) throw ie[0]; if (conf.getBoolean( FS_AUTOMATIC_CLOSE_KEY, FS_AUTOMATIC_CLOSE_DEFAULT)) { toAutoClose.add(key); } return map.get(key); } {code} This avoids synchronization and wont stop all the threads , just those threads referring to the same key. Please let me know your thoughts. > 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.4#6332) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-13726) Enforce that FileSystem initializes only a single instance of the requested FileSystem.
[ https://issues.apache.org/jira/browse/HADOOP-13726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15584932#comment-15584932 ] Steve Loughran commented on HADOOP-13726: - Guava has some caching stuff too; it may be simpler to reuse than than repurpose some of our own 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 > > 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.4#6332) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-13726) Enforce that FileSystem initializes only a single instance of the requested FileSystem.
[ https://issues.apache.org/jira/browse/HADOOP-13726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15583706#comment-15583706 ] Chris Nauroth commented on HADOOP-13726: HADOOP-13727 describes a specific problem in S3A that prompted me to file this JIRA for the FileSystem cache. In that case, the redundant initialization caused a high number of calls to the EC2 Instance Metadata Service, which ultimately throttled clients and caused failures. > 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.4#6332) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-13726) Enforce that FileSystem initializes only a single instance of the requested FileSystem.
[ https://issues.apache.org/jira/browse/HADOOP-13726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15583695#comment-15583695 ] Chris Nauroth commented on HADOOP-13726: The file system cache employs an optimistic locking algorithm. Multiple concurrent threads might create and initialize multiple instances without lock coordination. Then, while holding the lock, the thread checks if another thread won the race and put an instance into the cache while this thread was busy initializing the file system. If so, it uses the cached instance and discards the one it just initialized itself. {code} private FileSystem getInternal(URI uri, Configuration conf, Key key) throws IOException{ FileSystem fs; synchronized (this) { fs = map.get(key); } if (fs != null) { return fs; } fs = createFileSystem(uri, conf); synchronized (this) { // refetch the lock again FileSystem oldfs = map.get(key); if (oldfs != null) { // a file system is created while lock is releasing fs.close(); // close the new file system return oldfs; // return the old file system } {code} An important consequence of this algorithm is that even though all threads ultimately get the same shared instance, it's still possible that multiple concurrent threads are attempting the {{getInternal}} operation, so they could all be calling {{FileSystem#initialize}}. Depending on the file system implementation, this can be an expensive operation. We can eliminate this race condition by using techniques similar to the NameNode RPC {{RetryCache}}. If multiple threads simultaneously try to get a {{FileSystem}} with the same cache key, then the first thread proceeds into {{FileSystem#initialize}}. All other threads enter a wait set, blocked on completion of the first thread. After the first thread completes initialization, it notifies all members of the wait set. All threads are returned the same initialized instance. The current logic traces back to HADOOP-6640. Before that change, the locking was coarser, so something like a slow NameNode RPC connection with a lot of retries could block initialization of all file systems. The change I'm proposing here would not cause a regression of HADOOP-6640. It is only intended to prevent redundant initialization. > 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.4#6332) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org