[
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 [[email protected]] 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<Key, FileSystem> 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: [email protected]
For additional commands, e-mail: [email protected]