HBASE-25876 Add retry if we fail to read all bytes of the protobuf magic marker (#3251)
Signed-off-by: Duo Zhang <zhangduo@apache.org> Signed-off-by Anoop Sam John <anoopsamjohn@apache.org>
This commit is contained in:
parent
61d9b46aab
commit
075545731a
@ -964,8 +964,7 @@ public class HRegionInfo implements RegionInfo {
|
||||
}
|
||||
|
||||
//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 {
|
||||
|
@ -49,6 +49,7 @@ import java.util.concurrent.locks.Lock;
|
||||
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;
|
||||
@ -1067,11 +1068,7 @@ public class BucketCache implements BlockCache, HeapSize {
|
||||
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
|
||||
|
Loading…
x
Reference in New Issue
Block a user