diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/Reference.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/Reference.java index 695a2a9..875c71d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/Reference.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/Reference.java @@ -20,12 +20,12 @@ package org.apache.hadoop.hbase.io; import java.io.BufferedInputStream; import java.io.DataInput; +import java.io.DataInputStream; import java.io.DataOutput; import java.io.IOException; import java.io.InputStream; import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -175,9 +175,29 @@ public class Reference implements Writable { */ public static Reference read(final FileSystem fs, final Path p) throws IOException { - FSDataInputStream in = fs.open(p); + InputStream in = fs.open(p); try { - return parseFrom(in); + // I need to be able to move back in the stream if this is not a pb serialization so I can + // do the Writable decoding instead. + in = in.markSupported()? in: new BufferedInputStream(in); + int pblen = ProtobufUtil.lengthOfPBMagic(); + in.mark(pblen); + byte [] pbuf = new byte[pblen]; + int read = in.read(pbuf); + if (read != pblen) throw new IOException("read=" + read + ", wanted=" + pblen); + if (ProtobufUtil.isPBMagicPrefix(pbuf)) { + return convert(FSProtos.Reference.parseFrom(in)); + } else { + // Presume Writables. Need to reset the stream since it didn't start w/ pb. + // We won't bother rewriting thie Reference as a pb since Reference is transitory. + in.reset(); + Reference r = new Reference(); + DataInputStream dis = new DataInputStream(in); + // Set in = dis so it gets the close below in the finally on our way out. + in = dis; + r.readFields(dis); + return r; + } } finally { in.close(); } @@ -208,32 +228,4 @@ public class Reference implements Writable { byte [] toByteArray() throws IOException { return ProtobufUtil.prependPBMagic(convert().toByteArray()); } - - /** - * Parses an {@link Reference} instance from the passed in stream. Presumes the - * Reference was serialized to the stream with {@link #toDelimitedByteArray()} - * @param in - * @return An instance of Reference. - * @throws IOException - */ - static Reference parseFrom(final FSDataInputStream in) throws IOException { - // I need to be able to move back in the stream if this is not a pb serialization so I can - // do the Writable decoding instead. - InputStream is = in.markSupported()? in: new BufferedInputStream(in); - int pblen = ProtobufUtil.lengthOfPBMagic(); - is.mark(pblen); - byte [] pbuf = new byte[pblen]; - int read = is.read(pbuf); - if (read != pblen) throw new IOException("read=" + read + ", wanted=" + pblen); - if (ProtobufUtil.isPBMagicPrefix(pbuf)) { - return convert(FSProtos.Reference.parseFrom(is)); - } else { - // Presume Writables. Need to reset the stream since it didn't start w/ pb. - // We won't bother rewriting thie Reference as a pb since Reference is transitory. - in.reset(); - Reference r = new Reference(); - r.readFields(in); - return r; - } - } }