You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by vj...@apache.org on 2022/01/25 13:57:33 UTC

[hbase] branch branch-1 updated: HBASE-26660 delayed FlushRegionEntry should be removed when we need a non-delayed one (#4042)

This is an automated email from the ASF dual-hosted git repository.

vjasani pushed a commit to branch branch-1
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-1 by this push:
     new a2406f3  HBASE-26660 delayed FlushRegionEntry should be removed when we need a non-delayed one (#4042)
a2406f3 is described below

commit a2406f3764b436093bc65fd8fd7c175df6711f33
Author: Sairam Pola <sa...@gmail.com>
AuthorDate: Tue Jan 25 19:21:36 2022 +0530

    HBASE-26660 delayed FlushRegionEntry should be removed when we need a non-delayed one (#4042)
    
    Signed-off-by: Viraj Jasani <vj...@apache.org>
---
 .../apache/hadoop/hbase/regionserver/HRegion.java  |  4 ++
 .../hadoop/hbase/regionserver/MemStoreFlusher.java | 39 ++++++++---
 .../hbase/regionserver/TestMemStoreFlusher.java    | 80 ++++++++++++++++++++++
 3 files changed, 113 insertions(+), 10 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 3e752cd..59cbab9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -9227,6 +9227,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     flushesQueued.incrementAndGet();
   }
 
+  protected void decrementFlushesQueuedCount() {
+    flushesQueued.decrementAndGet();
+  }
+
   /**
    * Do not change this sequence id.
    * @return sequenceId
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
index a92a275..9e42f3c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
@@ -81,7 +81,7 @@ class MemStoreFlusher implements FlushRequester {
   // a corresponding entry in the other.
   private final BlockingQueue<FlushQueueEntry> flushQueue =
     new DelayQueue<FlushQueueEntry>();
-  private final Map<Region, FlushRegionEntry> regionsInQueue =
+  protected final Map<Region, FlushRegionEntry> regionsInQueue =
     new HashMap<Region, FlushRegionEntry>();
   private AtomicBoolean wakeupPending = new AtomicBoolean();
 
@@ -363,16 +363,28 @@ class MemStoreFlusher implements FlushRequester {
   @Override
   public boolean requestFlush(Region r, boolean forceFlushAllStores) {
     synchronized (regionsInQueue) {
-      if (!regionsInQueue.containsKey(r)) {
-        // This entry has no delay so it will be added at the top of the flush
-        // queue.  It'll come out near immediately.
-        FlushRegionEntry fqe = new FlushRegionEntry(r, forceFlushAllStores);
-        this.regionsInQueue.put(r, fqe);
-        this.flushQueue.add(fqe);
-        ((HRegion)r).incrementFlushesQueuedCount();
-        return true;
+      FlushRegionEntry existFqe = regionsInQueue.get(r);
+      if (existFqe != null) {
+        // if a delayed one exists and not reach the time to execute, just remove it
+        if (existFqe.isDelay() && existFqe.whenToExpire > EnvironmentEdgeManager.currentTime()) {
+          LOG.info("Remove the existing delayed flush entry for " + r + ", "
+            + "because we need to flush it immediately");
+          this.regionsInQueue.remove(r);
+          this.flushQueue.remove(existFqe);
+          ((HRegion)r).decrementFlushesQueuedCount();
+        } else {
+          LOG.info("Flush already requested on " + r);
+          return false;
+        }
       }
-      return false;
+
+      // This entry has no delay so it will be added at the top of the flush
+      // queue. It'll come out near immediately.
+      FlushRegionEntry fqe = new FlushRegionEntry(r, forceFlushAllStores);
+      this.regionsInQueue.put(r, fqe);
+      this.flushQueue.add(fqe);
+      ((HRegion)r).incrementFlushesQueuedCount();
+      return true;
     }
   }
 
@@ -753,6 +765,13 @@ class MemStoreFlusher implements FlushRequester {
     }
 
     /**
+     * @return True if the entry is a delay flush task
+     */
+    protected boolean isDelay() {
+      return this.whenToExpire > this.createTime;
+    }
+
+    /**
      * @return Count of times {@link #requeue(long)} was called; i.e this is
      * number of times we've been requeued.
      */
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreFlusher.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreFlusher.java
new file mode 100644
index 0000000..51b7768
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreFlusher.java
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Threads;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestMemStoreFlusher {
+  private MemStoreFlusher msf;
+
+  @Before
+  public void setUp() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set("hbase.hstore.flusher.count", "0");
+    msf = new MemStoreFlusher(conf, null);
+  }
+
+  @Test
+  public void testReplaceDelayedFlushEntry() {
+    HRegionInfo hri = new HRegionInfo(1, TableName.valueOf("TestTable"), 0);
+    HRegion r = mock(HRegion.class);
+    doReturn(hri).when(r).getRegionInfo();
+
+    // put a delayed task with 30s delay
+    msf.requestDelayedFlush(r, 30000, false);
+    assertEquals(1, msf.getFlushQueueSize());
+    assertTrue(msf.regionsInQueue.get(r).isDelay());
+
+    // put a non-delayed task, then the delayed one should be replaced
+    assertTrue(msf.requestFlush(r, false));
+    assertEquals(1, msf.getFlushQueueSize());
+    assertFalse(msf.regionsInQueue.get(r).isDelay());
+  }
+
+  @Test
+  public void testNotReplaceDelayedFlushEntryWhichExpired() {
+    HRegionInfo hri = new HRegionInfo(1, TableName.valueOf("TestTable"), 0);
+    HRegion r = mock(HRegion.class);
+    doReturn(hri).when(r).getRegionInfo();
+
+    // put a delayed task with 100ms delay
+    msf.requestDelayedFlush(r, 100, false);
+    assertEquals(1, msf.getFlushQueueSize());
+    assertTrue(msf.regionsInQueue.get(r).isDelay());
+
+    Threads.sleep(200);
+
+    // put a non-delayed task, and the delayed one is expired, so it should not be replaced
+    assertFalse(msf.requestFlush(r, false));
+    assertEquals(1, msf.getFlushQueueSize());
+    assertTrue(msf.regionsInQueue.get(r).isDelay());
+  }
+}