From 907e114004c8c9417497616bc00797db47690791 Mon Sep 17 00:00:00 2001 From: stack Date: Mon, 10 May 2021 15:16:58 -0500 Subject: [PATCH] HBASE-25876 Add retry if we fail to read all bytes of the protobuf magic marker --- .../src/main/java/org/apache/hadoop/hbase/HRegionInfo.java | 3 +-- .../apache/hadoop/hbase/io/hfile/bucket/BucketCache.java | 7 ++----- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java index 6edd3fa3973b..87544308ecee 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java @@ -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 { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java index 1f9aac425e29..80f683379f4f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java @@ -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; @@ -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