Skip to content

Commit

Permalink
HBASE-26580 The message of StoreTooBusy is confused (#3949)
Browse files Browse the repository at this point in the history
Signed-off-by: Duo Zhang <zhangduo@apache.org>
Reviewed-by: Bryan Beaudreault <bbeaudreault@hubspot.com>
  • Loading branch information
zhengzhuobinzzb authored and Apache9 committed Dec 18, 2021
1 parent 12a286c commit 68d8573
Showing 1 changed file with 19 additions and 14 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -108,6 +108,8 @@ public void start(Map<byte[], List<Cell>> familyMaps) throws RegionTooBusyExcept
}

String tooBusyStore = null;
boolean aboveParallelThreadLimit = false;
boolean aboveParallelPrePutLimit = false;

for (Map.Entry<byte[], List<Cell>> e : familyMaps.entrySet()) {
Store store = this.region.getStore(e.getKey());
Expand All @@ -119,19 +121,19 @@ public void start(Map<byte[], List<Cell>> familyMaps) throws RegionTooBusyExcept

//we need to try to add #preparePutCount at first because preparePutToStoreMap will be
//cleared when changing the configuration.
preparePutToStoreMap.putIfAbsent(e.getKey(), new AtomicInteger());
AtomicInteger preparePutCounter = preparePutToStoreMap.get(e.getKey());
if (preparePutCounter == null) {
preparePutCounter = new AtomicInteger();
preparePutToStoreMap.putIfAbsent(e.getKey(), preparePutCounter);
}
int preparePutCount = preparePutCounter.incrementAndGet();
if (store.getCurrentParallelPutCount() > this.parallelPutToStoreThreadLimit
|| preparePutCount > this.parallelPreparePutToStoreThreadLimit) {
int preparePutCount = preparePutToStoreMap
.computeIfAbsent(e.getKey(), key -> new AtomicInteger())
.incrementAndGet();
boolean storeAboveThread =
store.getCurrentParallelPutCount() > this.parallelPutToStoreThreadLimit;
boolean storeAbovePrePut = preparePutCount > this.parallelPreparePutToStoreThreadLimit;
if (storeAboveThread || storeAbovePrePut) {
tooBusyStore = (tooBusyStore == null ?
store.getColumnFamilyName() :
tooBusyStore + "," + store.getColumnFamilyName());
}
aboveParallelThreadLimit |= storeAboveThread;
aboveParallelPrePutLimit |= storeAbovePrePut;

if (LOG.isTraceEnabled()) {
LOG.trace(store.getColumnFamilyName() + ": preparePutCount=" + preparePutCount
Expand All @@ -140,13 +142,16 @@ public void start(Map<byte[], List<Cell>> familyMaps) throws RegionTooBusyExcept
}
}

if (tooBusyStore != null) {
if (aboveParallelThreadLimit || aboveParallelPrePutLimit) {
String msg =
"StoreTooBusy," + this.region.getRegionInfo().getRegionNameAsString() + ":" + tooBusyStore
+ " Above parallelPutToStoreThreadLimit(" + this.parallelPutToStoreThreadLimit + ")";
if (LOG.isTraceEnabled()) {
LOG.trace(msg);
}
+ " Above "
+ (aboveParallelThreadLimit ? "parallelPutToStoreThreadLimit("
+ this.parallelPutToStoreThreadLimit + ")" : "")
+ (aboveParallelThreadLimit && aboveParallelPrePutLimit ? " or " : "")
+ (aboveParallelPrePutLimit ? "parallelPreparePutToStoreThreadLimit("
+ this.parallelPreparePutToStoreThreadLimit + ")" : "");
LOG.trace(msg);
throw new RegionTooBusyException(msg);
}
}
Expand Down

0 comments on commit 68d8573

Please sign in to comment.