You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2020/06/04 00:40:18 UTC

[hbase] branch branch-2.3 updated: HBASE-21905: [FIFOCompaction] Don't compact already inflight store files (#1829)

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

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


The following commit(s) were added to refs/heads/branch-2.3 by this push:
     new 96c5b3a  HBASE-21905: [FIFOCompaction] Don't compact already inflight store files (#1829)
96c5b3a is described below

commit 96c5b3ad8c99a154c49fc6764686124b9562fc97
Author: Bharath Vissapragada <bh...@apache.org>
AuthorDate: Wed Jun 3 16:22:28 2020 -0700

    HBASE-21905: [FIFOCompaction] Don't compact already inflight store files (#1829)
    
    This one surfaced as a flake test but turns out to be a legit bug
    in FIFOCompaction code. FifoCompaction does not check if an empty
    store file is already being compacted by an in-flight compaction
    request and still enqueues. It can potentially race with a running
    compaction (as in this test case, see jira for the exact exception).
    
    Fixes the edge case and cleans up the test code a bit.
    
    Signed-off-by: Andrew Purtell <ap...@apache.org>
---
 .../compactions/FIFOCompactionPolicy.java          | 12 ++++-----
 .../compactions/TestFIFOCompactionPolicy.java      | 31 ++++++++--------------
 2 files changed, 17 insertions(+), 26 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/FIFOCompactionPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/FIFOCompactionPolicy.java
index 5c8626b..344b90d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/FIFOCompactionPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/FIFOCompactionPolicy.java
@@ -97,11 +97,11 @@ public class FIFOCompactionPolicy extends ExploringCompactionPolicy {
   }
 
   /**
-   * The FIFOCompactionPolicy only choose those TTL expired HFiles as the compaction candidates. So
-   * if all HFiles are TTL expired, then the compaction will generate a new empty HFile. While its
-   * max timestamp will be Long.MAX_VALUE. If not considered separately, the HFile will never be
-   * archived because its TTL will be never expired. So we'll check the empty store file separately.
-   * (See HBASE-21504)
+   * The FIFOCompactionPolicy only choose the TTL expired store files as the compaction candidates.
+   * If all the store files are TTL expired, then the compaction will generate a new empty file.
+   * While its max timestamp will be Long.MAX_VALUE. If not considered separately, the store file
+   * will never be archived because its TTL will be never expired. So we'll check the empty store
+   * file separately (See HBASE-21504).
    */
   private boolean isEmptyStoreFile(HStoreFile sf) {
     return sf.getReader().getEntries() == 0;
@@ -130,7 +130,7 @@ public class FIFOCompactionPolicy extends ExploringCompactionPolicy {
     long currentTime = EnvironmentEdgeManager.currentTime();
     Collection<HStoreFile> expiredStores = new ArrayList<>();
     for (HStoreFile sf : files) {
-      if (isEmptyStoreFile(sf)) {
+      if (isEmptyStoreFile(sf) && !filesCompacting.contains(sf)) {
         expiredStores.add(sf);
         continue;
       }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestFIFOCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestFIFOCompactionPolicy.java
index 3e95181..36afd84 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestFIFOCompactionPolicy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestFIFOCompactionPolicy.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
@@ -58,6 +59,7 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.ExpectedException;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 
 @Category({ RegionServerTests.class, MediumTests.class })
 public class TestFIFOCompactionPolicy {
@@ -216,23 +218,18 @@ public class TestFIFOCompactionPolicy {
     TEST_UTIL.getAdmin().flush(tableName); // HFile-0
     put = new Put(Bytes.toBytes("row2")).addColumn(family, qualifier, ts, Bytes.toBytes("value1"));
     table.put(put);
+    final int testWaitTimeoutMs = 20000;
     TEST_UTIL.getAdmin().flush(tableName); // HFile-1
 
-    HStore store = getStoreWithName(tableName);
-    Assert.assertNotNull(store);
+    HStore store = Preconditions.checkNotNull(getStoreWithName(tableName));
     Assert.assertEquals(2, store.getStorefilesCount());
 
     TEST_UTIL.getAdmin().majorCompact(tableName);
-    for (int i = 0; i < 100; i++) {
-      if (store.getStorefilesCount() > 1) {
-        Thread.sleep(100);
-      } else {
-        break;
-      }
-    }
+    TEST_UTIL.waitFor(testWaitTimeoutMs,
+        (Waiter.Predicate<Exception>) () -> store.getStorefilesCount() == 1);
+
     Assert.assertEquals(1, store.getStorefilesCount());
-    HStoreFile sf = store.getStorefiles().iterator().next();
-    Assert.assertNotNull(sf);
+    HStoreFile sf = Preconditions.checkNotNull(store.getStorefiles().iterator().next());
     Assert.assertEquals(0, sf.getReader().getEntries());
 
     put = new Put(Bytes.toBytes("row3")).addColumn(family, qualifier, ts, Bytes.toBytes("value1"));
@@ -241,17 +238,11 @@ public class TestFIFOCompactionPolicy {
     Assert.assertEquals(2, store.getStorefilesCount());
 
     TEST_UTIL.getAdmin().majorCompact(tableName);
-    for (int i = 0; i < 100; i++) {
-      if (store.getStorefilesCount() > 1) {
-        Thread.sleep(100);
-      } else {
-        break;
-      }
-    }
+    TEST_UTIL.waitFor(testWaitTimeoutMs,
+        (Waiter.Predicate<Exception>) () -> store.getStorefilesCount() == 1);
 
     Assert.assertEquals(1, store.getStorefilesCount());
-    sf = store.getStorefiles().iterator().next();
-    Assert.assertNotNull(sf);
+    sf = Preconditions.checkNotNull(store.getStorefiles().iterator().next());
     Assert.assertEquals(0, sf.getReader().getEntries());
   }
 }