HBASE-25859 Reference class incorrectly parses the protobuf magic marker (#3236)

Co-authored-by: Catalin Luca <luca@adobe.com>
Signed-off-by: stack <stack@apache.org>
This commit is contained in:
Catalin Luca 2021-05-10 22:45:23 +03:00 committed by stack
parent 63d49cb7ed
commit 4e507ccda0
1 changed files with 2 additions and 4 deletions

View File

@ -25,6 +25,7 @@ import java.io.IOException;
import java.io.InputStream;
import java.util.Arrays;
import org.apache.commons.io.IOUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
@ -174,10 +175,7 @@ public class Reference {
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);
}
IOUtils.readFully(in, pbuf,0, pblen);
// WATCHOUT! Return in middle of function!!!
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.