Skip to content

Commit

Permalink
HBASE-25876 Add retry if we fail to read all bytes of the protobuf ma…
Browse files Browse the repository at this point in the history
…gic marker (#3251)

Signed-off-by: Duo Zhang <zhangduo@apache.org>
Signed-off-by Anoop Sam John <anoopsamjohn@apache.org>
  • Loading branch information
saintstack authored May 12, 2021
1 parent 61d9b46 commit 0755457
Show file tree
Hide file tree
Showing 2 changed files with 3 additions and 7 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -964,8 +964,7 @@ public static HRegionInfo parseFrom(final DataInputStream in) throws IOException
}

//assumption: if Writable serialization, it should be longer than pblen.
int read = in.read(pbuf);
if (read != pblen) throw new IOException("read=" + read + ", wanted=" + pblen);
in.readFully(pbuf, 0, pblen);
if (ProtobufUtil.isPBMagicPrefix(pbuf)) {
return convert(HBaseProtos.RegionInfo.parseDelimitedFrom(in));
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@
import java.util.concurrent.locks.ReentrantLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.function.Consumer;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.TableName;
Expand Down Expand Up @@ -1067,11 +1068,7 @@ private void retrieveFromFile(int[] bucketSizes) throws IOException {
try (FileInputStream in = deleteFileOnClose(persistenceFile)) {
int pblen = ProtobufMagic.lengthOfPBMagic();
byte[] pbuf = new byte[pblen];
int read = in.read(pbuf);
if (read != pblen) {
throw new IOException("Incorrect number of bytes read while checking for protobuf magic "
+ "number. Requested=" + pblen + ", Received= " + read + ", File=" + persistencePath);
}
IOUtils.readFully(in, pbuf, 0, pblen);
if (! ProtobufMagic.isPBMagicPrefix(pbuf)) {
// In 3.0 we have enough flexibility to dump the old cache data.
// TODO: In 2.x line, this might need to be filled in to support reading the old format
Expand Down

0 comments on commit 0755457

Please sign in to comment.