From 5630180582e5f08ef4845a104d306527bab5cd31 Mon Sep 17 00:00:00 2001 From: jayunit100 Date: Sat, 14 Sep 2013 21:51:25 -0400 Subject: [PATCH] GlusterVolume modification to wrap getPos() functionality in underlying bufferstream --- .../hadoop/fs/glusterfs/GlusterVolume.java | 34 +++++++++++++++++++ 1 file changed, 34 insertions(+) diff --git a/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterVolume.java b/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterVolume.java index 1dc1b1af..18a1e031 100644 --- a/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterVolume.java +++ b/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterVolume.java @@ -29,6 +29,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.BlockLocation; +import org.apache.hadoop.fs.BufferedFSInputStream; +import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RawLocalFileSystem; @@ -176,5 +178,37 @@ public BlockLocation[] getFileBlockLocations(FileStatus file,long start,long len public String toString(){ return "Gluster Volume mounted at: " + root; } + + + /** + * An FS DataInputStream that guards against the underlying exception + */ + @Override + public FSDataInputStream open(Path f, int bufferSize) throws IOException { + return new WrappedFSDataInputStream(super.open(f, bufferSize)); + } + + + public static class WrappedFSDataInputStream extends FSDataInputStream{ + public WrappedFSDataInputStream(FSDataInputStream in) throws IOException { + super(in); + } + long lastPos=0; + @Override + public long getPos() throws IOException { + try{ + //Try to call the underlying get pos, but if it fails, + //just return the last valid read. This should be + //correct if the underlying call fails, it will be because + //the stream has been closed (assuming that the last read ended + //at the final position in the stream, and no reads occured after that). + lastPos= ((BufferedFSInputStream)in).getPos(); + } + catch(Throwable t){ + log.error("Wrapped exception in getPos() : We will return the last valid pos." + t.getMessage()); + } + return lastPos; + } + } }