|
@@ -112,7 +112,19 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
}
|
|
|
|
|
|
public FSDataInputStream open(Path f, int bufferSize) throws IOException {
|
|
|
- return new DFSClient.DFSDataInputStream(dfs.open(getPathName(f),bufferSize));
|
|
|
+ try {
|
|
|
+ return new DFSClient.DFSDataInputStream(dfs.open(getPathName(f),bufferSize));
|
|
|
+ } catch(RemoteException e) {
|
|
|
+ if (IOException.class.getName().equals(e.getClassName()) &&
|
|
|
+ e.getMessage().startsWith(
|
|
|
+ "java.io.IOException: Cannot open filename")) {
|
|
|
+ // non-existent path
|
|
|
+ FileNotFoundException ne = new FileNotFoundException("File " + f + " does not exist.");
|
|
|
+ throw (FileNotFoundException) ne.initCause(e);
|
|
|
+ } else {
|
|
|
+ throw e; // unexpected exception
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
public FSDataOutputStream create(Path f, FsPermission permission,
|
|
@@ -338,7 +350,8 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
e.getMessage().startsWith(
|
|
|
"java.io.IOException: File does not exist: ")) {
|
|
|
// non-existent path
|
|
|
- throw new FileNotFoundException("File " + f + " does not exist.");
|
|
|
+ FileNotFoundException fe = new FileNotFoundException("File " + f + " does not exist.");
|
|
|
+ throw (FileNotFoundException) fe.initCause(e);
|
|
|
} else {
|
|
|
throw e; // unexpected exception
|
|
|
}
|