HADOOP-6640. FileSystem.get() does RPC retries within a static synchronized block. Contributed by Hairong Kuang.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@930096 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Hairong Kuang 2010-04-01 20:58:10 +00:00
parent 6f35c10aab
commit 7b1ac5ac13
3 changed files with 68 additions and 7 deletions

View File

@ -306,6 +306,9 @@ Trunk (unreleased changes)
HADOOP-6654. Fix code example in WritableComparable javadoc. (Tom White
via szetszwo)
HADOOP-6640. FileSystem.get() does RPC retries within a static
synchronized block. (hairong)
Release 0.21.0 - Unreleased
INCOMPATIBLE CHANGES

View File

@ -1758,33 +1758,46 @@ 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) {
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;
}
}
synchronized void remove(Key key, FileSystem fs) {
if (map.containsKey(key) && fs == map.get(key)) {

View File

@ -21,7 +21,9 @@ package org.apache.hadoop.fs;
import static junit.framework.Assert.assertSame;
import static junit.framework.Assert.assertNotSame;
import java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.UserGroupInformation;
@ -46,6 +48,49 @@ public class TestFileSystemCaching {
assertSame(fs1, fs2);
}
public static class InitializeForeverFileSystem extends LocalFileSystem {
public void initialize(URI uri, Configuration conf) throws IOException {
// notify that InitializeForeverFileSystem started initialization
synchronized (conf) {
conf.notify();
}
try {
while (true) {
Thread.sleep(1000);
}
} catch (InterruptedException e) {
return;
}
}
}
@Test
public void testCacheEnabledWithInitializeForeverFS() throws Exception {
final Configuration conf = new Configuration();
Thread t = new Thread() {
public void run() {
conf.set("fs.localfs1.impl", "org.apache.hadoop.fs." +
"TestFileSystemCaching$InitializeForeverFileSystem");
try {
FileSystem.get(new URI("localfs1://a"), conf);
} catch (IOException e) {
e.printStackTrace();
} catch (URISyntaxException e) {
e.printStackTrace();
}
}
};
t.start();
// wait for InitializeForeverFileSystem to start initialization
synchronized (conf) {
conf.wait();
}
conf.set("fs.cachedfile.impl", conf.get("fs.file.impl"));
FileSystem.get(new URI("cachedfile://a"), conf);
t.interrupt();
t.join();
}
@Test
public void testCacheDisabled() throws Exception {
Configuration conf = new Configuration();