You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2021/08/20 02:25:38 UTC

[GitHub] [hbase] Apache9 commented on a change in pull request #3604: HBASE-26210 HBase Write should be doomed to hang when cell size excee…

Apache9 commented on a change in pull request #3604:
URL: https://github.com/apache/hbase/pull/3604#discussion_r692611275



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java
##########
@@ -413,34 +413,61 @@ public String getFamilyName() {
   }
 
   /**
-   * Check whether anything need to be done based on the current active set size.
-   * The method is invoked upon every addition to the active set.
-   * For CompactingMemStore, flush the active set to the read-only memory if it's
-   * size is above threshold
+   * Check whether anything need to be done based on the current active set size. The method is
+   * invoked upon every addition to the active set. For CompactingMemStore, flush the active set to
+   * the read-only memory if it's size is above threshold
    * @param currActive intended segment to update
    * @param cellToAdd cell to be added to the segment
    * @param memstoreSizing object to accumulate changed size
-   * @return true if the cell can be added to the
+   * @return true if the cell can be added to the currActive
    */
-  private boolean checkAndAddToActiveSize(MutableSegment currActive, Cell cellToAdd,
+  protected boolean checkAndAddToActiveSize(MutableSegment currActive, Cell cellToAdd,
       MemStoreSizing memstoreSizing) {
-    if (shouldFlushInMemory(currActive, cellToAdd, memstoreSizing)) {
-      if (currActive.setInMemoryFlushed()) {
-        flushInMemory(currActive);
-        if (setInMemoryCompactionFlag()) {
-          // The thread is dispatched to do in-memory compaction in the background
-          InMemoryCompactionRunnable runnable = new InMemoryCompactionRunnable();
-          if (LOG.isTraceEnabled()) {
-            LOG.trace("Dispatching the MemStore in-memory flush for store " + store
-                .getColumnFamilyName());
-          }
-          getPool().execute(runnable);
+    long cellSize = MutableSegment.getCellLength(cellToAdd);
+    boolean successAdd = false;
+    while (true) {
+      long segmentDataSize = currActive.getDataSize();
+      if (!inWalReplay && segmentDataSize > inmemoryFlushSize) {
+        // when replaying edits from WAL there is no need in in-memory flush regardless the size
+        // otherwise size below flush threshold try to update atomically
+        break;
+      }
+      if (currActive.compareAndSetDataSize(segmentDataSize, segmentDataSize + cellSize)) {
+        if (memstoreSizing != null) {
+          memstoreSizing.incMemStoreSize(cellSize, 0, 0, 0);
         }
+        successAdd = true;
+        break;
       }
-      return false;
     }
-    return true;
- }
+
+    if (!inWalReplay && currActive.getDataSize() > inmemoryFlushSize) {
+      // size above flush threshold so we flush in memory
+      this.tryFlushInMemoryAndCompactingAsync(currActive);
+    }
+    return successAdd;
+  }
+
+  /**
+   * Try to flush the currActive in memory and submit the in memory compact task to
+   * {@link RegionServicesForStores#getInMemoryCompactionPool()}. Just one thread can do the actual
+   * flushing in memory.
+   * @param currActive
+   */
+  private void tryFlushInMemoryAndCompactingAsync(MutableSegment currActive) {
+    if (currActive.setInMemoryFlushed()) {

Review comment:
       So if a MutableSegment has been set to in memory flushed state, it will never go back to a normal state right? Otherwise there will be problem I suppose.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org