|
@@ -1758,32 +1758,45 @@ public abstract class FileSystem extends Configured implements Closeable {
|
|
|
/** A variable that makes all objects in the cache unique */
|
|
|
private static AtomicLong unique = new AtomicLong(1);
|
|
|
|
|
|
- synchronized FileSystem get(URI uri, Configuration conf) throws IOException{
|
|
|
+ FileSystem get(URI uri, Configuration conf) throws IOException{
|
|
|
Key key = new Key(uri, conf);
|
|
|
return getInternal(uri, conf, key);
|
|
|
}
|
|
|
|
|
|
/** The objects inserted into the cache using this method are all unique */
|
|
|
- synchronized FileSystem getUnique(URI uri, Configuration conf) throws IOException{
|
|
|
+ FileSystem getUnique(URI uri, Configuration conf) throws IOException{
|
|
|
Key key = new Key(uri, conf, unique.getAndIncrement());
|
|
|
return getInternal(uri, conf, key);
|
|
|
}
|
|
|
|
|
|
private FileSystem getInternal(URI uri, Configuration conf, Key key) throws IOException{
|
|
|
- FileSystem fs = map.get(key);
|
|
|
- if (fs == null) {
|
|
|
- fs = createFileSystem(uri, conf);
|
|
|
+ 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
|
|
|
+ }
|
|
|
+
|
|
|
+ // now insert the new file system into the map
|
|
|
if (map.isEmpty() && !clientFinalizer.isAlive()) {
|
|
|
Runtime.getRuntime().addShutdownHook(clientFinalizer);
|
|
|
}
|
|
|
fs.key = key;
|
|
|
map.put(key, fs);
|
|
|
-
|
|
|
if (conf.getBoolean("fs.automatic.close", true)) {
|
|
|
toAutoClose.add(key);
|
|
|
}
|
|
|
+ return fs;
|
|
|
}
|
|
|
- return fs;
|
|
|
}
|
|
|
|
|
|
synchronized void remove(Key key, FileSystem fs) {
|