Skip to content

Commit c5275f0

Browse files
committed
HBASE-28839: Handle all types of exceptions during retrieval of bucket-cache from persistence.
In certain scenarios, where there is discrepancy between the number of chunks persisted in the file and the number of chunks stored in the persistence file. This is because the bucket cache may be operated upon in parallel. During the retrievel of bucket-cache from persistence, it was observed that, if an exception, other than the IOException occurs, the exception is not logged and also the retrieval thread exits leaving the bucket cache in an uninitialised state, leaving it unusable. With this change, the retrieval code does not rely on the metadata information (number of chunks) and instead, it reads from the file-stream as long as the data is available to be read. This change, enables the retrieval thread to print all types of exceptions and also reinitialises the bucket cache and makes it reusable. Change-Id: I81b7f5fe06945702bbc59df96d054f95f03de499
1 parent 2531f93 commit c5275f0

File tree

2 files changed

+26
-47
lines changed

2 files changed

+26
-47
lines changed

hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java

Lines changed: 17 additions & 35 deletions
Original file line numberDiff line numberDiff line change
@@ -390,17 +390,17 @@ private void startPersistenceRetriever(int[] bucketSizes, long capacity) {
390390
try {
391391
retrieveFromFile(bucketSizes);
392392
LOG.info("Persistent bucket cache recovery from {} is complete.", persistencePath);
393-
} catch (IOException ioex) {
394-
LOG.error("Can't restore from file[{}] because of ", persistencePath, ioex);
393+
} catch (Throwable ex) {
394+
LOG.error("Can't restore from file[{}] because of ", persistencePath, ex);
395395
backingMap.clear();
396396
fullyCachedFiles.clear();
397397
backingMapValidated.set(true);
398+
regionCachedSize.clear();
398399
try {
399400
bucketAllocator = new BucketAllocator(capacity, bucketSizes);
400-
} catch (BucketAllocatorException ex) {
401-
LOG.error("Exception during Bucket Allocation", ex);
401+
} catch (BucketAllocatorException allocatorException) {
402+
LOG.error("Exception during Bucket Allocation", allocatorException);
402403
}
403-
regionCachedSize.clear();
404404
} finally {
405405
this.cacheState = CacheState.ENABLED;
406406
startWriterThreads();
@@ -951,7 +951,8 @@ public void logStats() {
951951
: (StringUtils.formatPercent(cacheStats.getHitCachingRatio(), 2) + ", "))
952952
+ "evictions=" + cacheStats.getEvictionCount() + ", " + "evicted="
953953
+ cacheStats.getEvictedCount() + ", " + "evictedPerRun=" + cacheStats.evictedPerEviction()
954-
+ ", " + "allocationFailCount=" + cacheStats.getAllocationFailCount());
954+
+ ", " + "allocationFailCount=" + cacheStats.getAllocationFailCount() + ", blocksCount="
955+
+ backingMap.size());
955956
cacheStats.reset();
956957

957958
bucketAllocator.logDebugStatistics();
@@ -1496,7 +1497,7 @@ private void retrieveFromFile(int[] bucketSizes) throws IOException {
14961497
} else if (Arrays.equals(pbuf, BucketProtoUtils.PB_MAGIC_V2)) {
14971498
// The new persistence format of chunked persistence.
14981499
LOG.info("Reading new chunked format of persistence.");
1499-
retrieveChunkedBackingMap(in, bucketSizes);
1500+
retrieveChunkedBackingMap(in);
15001501
} else {
15011502
// In 3.0 we have enough flexibility to dump the old cache data.
15021503
// TODO: In 2.x line, this might need to be filled in to support reading the old format
@@ -1626,52 +1627,33 @@ private void parsePB(BucketCacheProtos.BucketCacheEntry proto) throws IOExceptio
16261627
}
16271628

16281629
private void persistChunkedBackingMap(FileOutputStream fos) throws IOException {
1629-
long numChunks = backingMap.size() / persistenceChunkSize;
1630-
if (backingMap.size() % persistenceChunkSize != 0) {
1631-
numChunks += 1;
1632-
}
1633-
16341630
LOG.debug(
16351631
"persistToFile: before persisting backing map size: {}, "
16361632
+ "fullycachedFiles size: {}, chunkSize: {}, numberofChunks: {}",
1637-
backingMap.size(), fullyCachedFiles.size(), persistenceChunkSize, numChunks);
1633+
backingMap.size(), fullyCachedFiles.size(), persistenceChunkSize);
16381634

1639-
BucketProtoUtils.serializeAsPB(this, fos, persistenceChunkSize, numChunks);
1635+
BucketProtoUtils.serializeAsPB(this, fos, persistenceChunkSize);
16401636

16411637
LOG.debug(
1642-
"persistToFile: after persisting backing map size: {}, "
1643-
+ "fullycachedFiles size: {}, numChunksPersisteed: {}",
1644-
backingMap.size(), fullyCachedFiles.size(), numChunks);
1638+
"persistToFile: after persisting backing map size: {}, " + "fullycachedFiles size: {}",
1639+
backingMap.size(), fullyCachedFiles.size());
16451640
}
16461641

1647-
private void retrieveChunkedBackingMap(FileInputStream in, int[] bucketSizes) throws IOException {
1648-
byte[] bytes = new byte[Long.BYTES];
1649-
int readSize = in.read(bytes);
1650-
if (readSize != Long.BYTES) {
1651-
throw new IOException("Invalid size of chunk-size read from persistence: " + readSize);
1652-
}
1653-
long batchSize = Bytes.toLong(bytes, 0);
1654-
1655-
readSize = in.read(bytes);
1656-
if (readSize != Long.BYTES) {
1657-
throw new IOException("Invalid size for number of chunks read from persistence: " + readSize);
1658-
}
1659-
long numChunks = Bytes.toLong(bytes, 0);
1660-
1661-
LOG.info("Number of chunks: {}, chunk size: {}", numChunks, batchSize);
1642+
private void retrieveChunkedBackingMap(FileInputStream in) throws IOException {
16621643

16631644
// Read the first chunk that has all the details.
16641645
BucketCacheProtos.BucketCacheEntry firstChunk =
16651646
BucketCacheProtos.BucketCacheEntry.parseDelimitedFrom(in);
16661647
parseFirstChunk(firstChunk);
16671648

16681649
// Subsequent chunks have the backingMap entries.
1669-
for (int i = 1; i < numChunks; i++) {
1670-
LOG.info("Reading chunk no: {}", i + 1);
1650+
int numChunks = 0;
1651+
while (in.available() > 0) {
16711652
parseChunkPB(BucketCacheProtos.BackingMap.parseDelimitedFrom(in),
16721653
firstChunk.getDeserializersMap());
1673-
LOG.info("Retrieved chunk: {}", i + 1);
1654+
numChunks++;
16741655
}
1656+
LOG.info("Retrieved {} of chunks with blockCount = {}.", numChunks, backingMap.size());
16751657
verifyFileIntegrity(firstChunk);
16761658
verifyCapacityAndClasses(firstChunk.getCacheCapacity(), firstChunk.getIoClass(),
16771659
firstChunk.getMapClass());

hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketProtoUtils.java

Lines changed: 9 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -33,7 +33,6 @@
3333
import org.apache.hadoop.hbase.io.hfile.BlockType;
3434
import org.apache.hadoop.hbase.io.hfile.CacheableDeserializerIdManager;
3535
import org.apache.hadoop.hbase.io.hfile.HFileBlock;
36-
import org.apache.hadoop.hbase.util.Bytes;
3736
import org.apache.hadoop.hbase.util.Pair;
3837
import org.apache.yetus.audience.InterfaceAudience;
3938

@@ -62,31 +61,29 @@ static BucketCacheProtos.BucketCacheEntry toPB(BucketCache cache,
6261
.build();
6362
}
6463

65-
public static void serializeAsPB(BucketCache cache, FileOutputStream fos, long chunkSize,
66-
long numChunks) throws IOException {
64+
public static void serializeAsPB(BucketCache cache, FileOutputStream fos, long chunkSize)
65+
throws IOException {
66+
// Write the new version of magic number.
67+
fos.write(PB_MAGIC_V2);
68+
6769
int blockCount = 0;
68-
int chunkCount = 0;
6970
int backingMapSize = cache.backingMap.size();
7071
BucketCacheProtos.BackingMap.Builder builder = BucketCacheProtos.BackingMap.newBuilder();
71-
72-
fos.write(PB_MAGIC_V2);
73-
fos.write(Bytes.toBytes(chunkSize));
74-
fos.write(Bytes.toBytes(numChunks));
75-
72+
boolean firstChunkPersisted = false;
7673
BucketCacheProtos.BackingMapEntry.Builder entryBuilder =
7774
BucketCacheProtos.BackingMapEntry.newBuilder();
75+
7876
for (Map.Entry<BlockCacheKey, BucketEntry> entry : cache.backingMap.entrySet()) {
7977
blockCount++;
8078
entryBuilder.clear();
8179
entryBuilder.setKey(BucketProtoUtils.toPB(entry.getKey()));
8280
entryBuilder.setValue(BucketProtoUtils.toPB(entry.getValue()));
8381
builder.addEntry(entryBuilder.build());
84-
8582
if (blockCount % chunkSize == 0 || (blockCount == backingMapSize)) {
86-
chunkCount++;
87-
if (chunkCount == 1) {
83+
if (firstChunkPersisted == false) {
8884
// Persist all details along with the first chunk into BucketCacheEntry
8985
BucketProtoUtils.toPB(cache, builder.build()).writeDelimitedTo(fos);
86+
firstChunkPersisted = true;
9087
} else {
9188
// Directly persist subsequent backing-map chunks.
9289
builder.build().writeDelimitedTo(fos);

0 commit comments

Comments
 (0)