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

bharathv 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 26828ac  HBASE-21905: [FIFOCompaction] Don't compact already inflight 
store files (#1829)
26828ac is described below

commit 26828ac615391d321cde76a9e4ffba2d0c0be278
Author: Bharath Vissapragada <bhara...@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 <apurt...@apache.org>
---
 .../compactions/FIFOCompactionPolicy.java          | 22 ++++++-------
 .../compactions/TestFIFOCompactionPolicy.java      | 37 ++++++++++------------
 2 files changed, 28 insertions(+), 31 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 52e1b72..7769f6f 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
@@ -98,11 +98,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(StoreFile sf) {
     return sf.getReader().getEntries() == 0;
@@ -115,9 +115,9 @@ public class FIFOCompactionPolicy extends 
ExploringCompactionPolicy {
         return true;
       }
       // Check MIN_VERSIONS is in HStore removeUnneededFiles
-      Long maxTs = sf.getReader().getMaxTimestamp();
+      long maxTs = sf.getReader().getMaxTimestamp();
       long maxTtl = storeConfigInfo.getStoreFileTtl();
-      if (maxTs == null || maxTtl == Long.MAX_VALUE || (currentTime - maxTtl < 
maxTs)) {
+      if (maxTtl == Long.MAX_VALUE || currentTime - maxTtl < maxTs) {
         continue;
       } else {
         return true;
@@ -131,16 +131,16 @@ public class FIFOCompactionPolicy extends 
ExploringCompactionPolicy {
     long currentTime = EnvironmentEdgeManager.currentTime();
     Collection<StoreFile> expiredStores = new ArrayList<StoreFile>();
     for (StoreFile sf : files) {
-      if (isEmptyStoreFile(sf)) {
+      if (isEmptyStoreFile(sf) && !filesCompacting.contains(sf)) {
         expiredStores.add(sf);
         continue;
       }
       // Check MIN_VERSIONS is in HStore removeUnneededFiles
-      Long maxTs = sf.getReader().getMaxTimestamp();
+      long maxTs = sf.getReader().getMaxTimestamp();
       long maxTtl = storeConfigInfo.getStoreFileTtl();
-      if (maxTs == null || maxTtl == Long.MAX_VALUE || (currentTime - maxTtl < 
maxTs)) {
+      if (maxTtl == Long.MAX_VALUE || (currentTime - maxTtl < maxTs)) {
         continue;
-      } else if (filesCompacting == null || filesCompacting.contains(sf) == 
false) {
+      } else if (filesCompacting == null || !filesCompacting.contains(sf)) {
         expiredStores.add(sf);
       }
     }
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 41832e9..3f9bc65 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
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.regionserver.compactions;
 
 import static org.junit.Assert.assertEquals;
 
+import com.google.common.base.Preconditions;
 import java.io.IOException;
 import java.util.List;
 import java.util.concurrent.ThreadLocalRandom;
@@ -31,6 +32,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 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.HBaseAdmin;
 import org.apache.hadoop.hbase.client.Put;
@@ -214,36 +216,31 @@ public class TestFIFOCompactionPolicy {
     put = new Put(Bytes.toBytes("row2")).addColumn(family, qualifier, ts, 
Bytes.toBytes("value1"));
     table.put(put);
     TEST_UTIL.getHBaseAdmin().flush(tableName); // HFile-1
-    Store store = getStoreWithName(tableName);
-    Assert.assertNotNull(store);
+    final Store store = 
Preconditions.checkNotNull(getStoreWithName(tableName));
     Assert.assertEquals(2, store.getStorefilesCount());
     TEST_UTIL.getHBaseAdmin().majorCompact(tableName);
-    for (int i = 0; i < 100; i++) {
-      if (store.getStorefilesCount() > 1) {
-        Thread.sleep(100);
-      } else {
-        break;
+    final int testWaitTimeoutMs = 20000;
+    TEST_UTIL.waitFor(testWaitTimeoutMs, new Waiter.Predicate<Exception>() {
+      @Override
+      public boolean evaluate() {
+        return store.getStorefilesCount() == 1;
       }
-    }
-    Assert.assertEquals(1, store.getStorefilesCount());
-    StoreFile sf = store.getStorefiles().iterator().next();
-    Assert.assertNotNull(sf);
+    });
+    StoreFile sf = 
Preconditions.checkNotNull(store.getStorefiles().iterator().next());
+    // Empty store file generated.
     Assert.assertEquals(0, sf.getReader().getEntries());
     put = new Put(Bytes.toBytes("row3")).addColumn(family, qualifier, ts, 
Bytes.toBytes("value1"));
     table.put(put);
     TEST_UTIL.getHBaseAdmin().flush(tableName); // HFile-2
     Assert.assertEquals(2, store.getStorefilesCount());
     TEST_UTIL.getHBaseAdmin().majorCompact(tableName);
-    for (int i = 0; i < 100; i++) {
-      if (store.getStorefilesCount() > 1) {
-        Thread.sleep(100);
-      } else {
-        break;
+    TEST_UTIL.waitFor(testWaitTimeoutMs, new Waiter.Predicate<Exception>() {
+      @Override
+      public boolean evaluate() {
+        return 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());
   }
 }

Reply via email to