You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2021/07/31 07:00:56 UTC

[hbase] 01/02: HBASE-26144 The HStore.snapshot method is never called in main code (#3533)

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

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

commit aff880e14786082c5d5c760861199ddb0dd6a11e
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Sat Jul 31 14:33:46 2021 +0800

    HBASE-26144 The HStore.snapshot method is never called in main code (#3533)
    
    Signed-off-by: Yulin Niu <ni...@apache.org>
---
 .../apache/hadoop/hbase/regionserver/HStore.java   | 28 +++++++---------------
 .../hadoop/hbase/regionserver/TestHMobStore.java   |  6 -----
 .../hadoop/hbase/regionserver/TestHStore.java      |  5 +---
 3 files changed, 10 insertions(+), 29 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index a8cbf3d..4e1ef4c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -996,22 +996,8 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
   }
 
   /**
-   * Snapshot this stores memstore. Call before running
-   * {@link #flushCache(long, MemStoreSnapshot, MonitoredTask, ThroughputController,
-   * FlushLifeCycleTracker)}
-   *  so it has some work to do.
-   */
-  void snapshot() {
-    this.lock.writeLock().lock();
-    try {
-      this.memstore.snapshot();
-    } finally {
-      this.lock.writeLock().unlock();
-    }
-  }
-
-  /**
-   * Write out current snapshot. Presumes {@link #snapshot()} has been called previously.
+   * Write out current snapshot. Presumes {@code StoreFlusherImpl.prepare()} has been called
+   * previously.
    * @param logCacheFlushId flush sequence number
    * @return The path name of the tmp file to which the store was flushed
    * @throws IOException if exception occurs during process
@@ -1254,9 +1240,6 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
     this.lock.writeLock().lock();
     try {
       this.storeEngine.getStoreFileManager().insertNewFiles(sfs);
-      if (snapshotId > 0) {
-        this.memstore.clearSnapshot(snapshotId);
-      }
     } finally {
       // We need the lock, as long as we are updating the storeFiles
       // or changing the memstore. Let us release it before calling
@@ -1265,6 +1248,13 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
       // the lock.
       this.lock.writeLock().unlock();
     }
+    // We do not need to call clearSnapshot method inside the write lock.
+    // The clearSnapshot itself is thread safe, which can be called at the same time with other
+    // memstore operations expect snapshot and clearSnapshot. And for these two methods, in HRegion
+    // we can guarantee that there is only one onging flush, so they will be no race.
+    if (snapshotId > 0) {
+      this.memstore.clearSnapshot(snapshotId);
+    }
     // notify to be called here - only in case of flushes
     notifyChangedReadersObservers(sfs);
     if (LOG.isTraceEnabled()) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
index 5105c72..48ca166 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
@@ -461,11 +461,8 @@ public class TestHMobStore {
 
   /**
    * Flush the memstore
-   * @param storeFilesSize
-   * @throws IOException
    */
   private void flush(int storeFilesSize) throws IOException{
-    this.store.snapshot();
     flushStore(store, id++);
     Assert.assertEquals(storeFilesSize, this.store.getStorefiles().size());
     Assert.assertEquals(0, ((AbstractMemStore)this.store.memstore).getActive().getCellsCount());
@@ -473,9 +470,6 @@ public class TestHMobStore {
 
   /**
    * Flush the memstore
-   * @param store
-   * @param id
-   * @throws IOException
    */
   private static void flushStore(HMobStore store, long id) throws IOException {
     StoreFlushContext storeFlushCtx = store.createFlushContext(id, FlushLifeCycleTracker.DUMMY);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java
index 9e1896d..e024eec 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java
@@ -630,8 +630,7 @@ public class TestHStore {
     assertCheck();
   }
 
-  private void flush(int storeFilessize) throws IOException{
-    this.store.snapshot();
+  private void flush(int storeFilessize) throws IOException {
     flushStore(store, id++);
     assertEquals(storeFilessize, this.store.getStorefiles().size());
     assertEquals(0, ((AbstractMemStore)this.store.memstore).getActive().getCellsCount());
@@ -802,7 +801,6 @@ public class TestHStore {
 
   /**
    * Test to ensure correctness when using Stores with multiple timestamps
-   * @throws IOException
    */
   @Test
   public void testMultipleTimestamps() throws IOException {
@@ -817,7 +815,6 @@ public class TestHStore {
       this.store.add(kv, null);
     }
 
-    this.store.snapshot();
     flushStore(store, id++);
 
     List<Cell> kvList2 = getKeyValueSet(timestamps2,numRows, qf1, family);