Skip to content

Commit 80976f2

Browse files
zhengzhuobinzzbpetersomogyi
authored andcommitted
HBASE-26580 The message of StoreTooBusy is confused (apache#3949)
Signed-off-by: Duo Zhang <zhangduo@apache.org> Reviewed-by: Bryan Beaudreault <bbeaudreault@hubspot.com> (cherry picked from commit 9c01d04) Change-Id: Iee521c00f2b9fa43e9942de84b285f4bd6887ed8
1 parent 66c50ac commit 80976f2

1 file changed

Lines changed: 19 additions & 14 deletions

File tree

hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/throttle/StoreHotnessProtector.java

Lines changed: 19 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -108,6 +108,8 @@ public void start(Map<byte[], List<Cell>> familyMaps) throws RegionTooBusyExcept
108108
}
109109

110110
String tooBusyStore = null;
111+
boolean aboveParallelThreadLimit = false;
112+
boolean aboveParallelPrePutLimit = false;
111113

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

120122
//we need to try to add #preparePutCount at first because preparePutToStoreMap will be
121123
//cleared when changing the configuration.
122-
preparePutToStoreMap.putIfAbsent(e.getKey(), new AtomicInteger());
123-
AtomicInteger preparePutCounter = preparePutToStoreMap.get(e.getKey());
124-
if (preparePutCounter == null) {
125-
preparePutCounter = new AtomicInteger();
126-
preparePutToStoreMap.putIfAbsent(e.getKey(), preparePutCounter);
127-
}
128-
int preparePutCount = preparePutCounter.incrementAndGet();
129-
if (store.getCurrentParallelPutCount() > this.parallelPutToStoreThreadLimit
130-
|| preparePutCount > this.parallelPreparePutToStoreThreadLimit) {
124+
int preparePutCount = preparePutToStoreMap
125+
.computeIfAbsent(e.getKey(), key -> new AtomicInteger())
126+
.incrementAndGet();
127+
boolean storeAboveThread =
128+
store.getCurrentParallelPutCount() > this.parallelPutToStoreThreadLimit;
129+
boolean storeAbovePrePut = preparePutCount > this.parallelPreparePutToStoreThreadLimit;
130+
if (storeAboveThread || storeAbovePrePut) {
131131
tooBusyStore = (tooBusyStore == null ?
132132
store.getColumnFamilyName() :
133133
tooBusyStore + "," + store.getColumnFamilyName());
134134
}
135+
aboveParallelThreadLimit |= storeAboveThread;
136+
aboveParallelPrePutLimit |= storeAbovePrePut;
135137

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

143-
if (tooBusyStore != null) {
145+
if (aboveParallelThreadLimit || aboveParallelPrePutLimit) {
144146
String msg =
145147
"StoreTooBusy," + this.region.getRegionInfo().getRegionNameAsString() + ":" + tooBusyStore
146-
+ " Above parallelPutToStoreThreadLimit(" + this.parallelPutToStoreThreadLimit + ")";
147-
if (LOG.isTraceEnabled()) {
148-
LOG.trace(msg);
149-
}
148+
+ " Above "
149+
+ (aboveParallelThreadLimit ? "parallelPutToStoreThreadLimit("
150+
+ this.parallelPutToStoreThreadLimit + ")" : "")
151+
+ (aboveParallelThreadLimit && aboveParallelPrePutLimit ? " or " : "")
152+
+ (aboveParallelPrePutLimit ? "parallelPreparePutToStoreThreadLimit("
153+
+ this.parallelPreparePutToStoreThreadLimit + ")" : "");
154+
LOG.trace(msg);
150155
throw new RegionTooBusyException(msg);
151156
}
152157
}

0 commit comments

Comments
 (0)