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 2018/12/14 23:38:20 UTC

[1/2] hbase git commit: Revert "HBASE-21504 If enable FIFOCompactionPolicy, a compaction may write a "empty" hfile whose maxTimeStamp is long max. This kind of hfile will never be archived."

Repository: hbase
Updated Branches:
  refs/heads/branch-1.3 05fed6558 -> a75a458e0


Revert "HBASE-21504 If enable FIFOCompactionPolicy, a compaction may write a "empty" hfile whose maxTimeStamp is long max. This kind of hfile will never be archived."

This reverts commit 57222ff0aa66047fc405c488f86a27b622beea64.

Repeatable TestFIFOCompactionPolicy test failure


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/d2c1deef
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/d2c1deef
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/d2c1deef

Branch: refs/heads/branch-1.3
Commit: d2c1deef1c1dd858fcf0b110753e57be466e676f
Parents: 05fed65
Author: Andrew Purtell <ap...@apache.org>
Authored: Fri Dec 14 15:35:14 2018 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Fri Dec 14 15:35:14 2018 -0800

----------------------------------------------------------------------
 .../compactions/FIFOCompactionPolicy.java       | 44 ++++++---------
 .../compactions/TestFIFOCompactionPolicy.java   | 57 --------------------
 2 files changed, 15 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d2c1deef/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/FIFOCompactionPolicy.java
----------------------------------------------------------------------
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..d339898 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,29 +97,17 @@ public class FIFOCompactionPolicy extends ExploringCompactionPolicy {
     return hasExpiredStores(storeFiles);
   }
 
-  /**
-   * 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)
-   */
-  private boolean isEmptyStoreFile(StoreFile sf) {
-    return sf.getReader().getEntries() == 0;
-  }
-
-  private boolean hasExpiredStores(Collection<StoreFile> files) {
+  private  boolean hasExpiredStores(Collection<StoreFile> files) {
     long currentTime = EnvironmentEdgeManager.currentTime();
-    for (StoreFile sf : files) {
-      if (isEmptyStoreFile(sf)) {
-        return true;
-      }
+    for(StoreFile sf: files){
       // Check MIN_VERSIONS is in HStore removeUnneededFiles
       Long maxTs = sf.getReader().getMaxTimestamp();
       long maxTtl = storeConfigInfo.getStoreFileTtl();
-      if (maxTs == null || maxTtl == Long.MAX_VALUE || (currentTime - maxTtl < maxTs)) {
-        continue;
-      } else {
+      if(maxTs == null 
+          || maxTtl == Long.MAX_VALUE
+          || (currentTime - maxTtl < maxTs)){
+        continue; 
+      } else{
         return true;
       }
     }
@@ -127,20 +115,18 @@ public class FIFOCompactionPolicy extends ExploringCompactionPolicy {
   }
 
   private  Collection<StoreFile> getExpiredStores(Collection<StoreFile> files,
-      Collection<StoreFile> filesCompacting) {
+    Collection<StoreFile> filesCompacting) {
     long currentTime = EnvironmentEdgeManager.currentTime();
-    Collection<StoreFile> expiredStores = new ArrayList<StoreFile>();
-    for (StoreFile sf : files) {
-      if (isEmptyStoreFile(sf)) {
-        expiredStores.add(sf);
-        continue;
-      }
+    Collection<StoreFile> expiredStores = new ArrayList<StoreFile>();    
+    for(StoreFile sf: files){
       // Check MIN_VERSIONS is in HStore removeUnneededFiles
       Long maxTs = sf.getReader().getMaxTimestamp();
       long maxTtl = storeConfigInfo.getStoreFileTtl();
-      if (maxTs == null || maxTtl == Long.MAX_VALUE || (currentTime - maxTtl < maxTs)) {
-        continue;
-      } else if (filesCompacting == null || filesCompacting.contains(sf) == false) {
+      if(maxTs == null 
+          || maxTtl == Long.MAX_VALUE
+          || (currentTime - maxTtl < maxTs)){
+        continue; 
+      } else if(filesCompacting == null || filesCompacting.contains(sf) == false){
         expiredStores.add(sf);
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d2c1deef/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestFIFOCompactionPolicy.java
----------------------------------------------------------------------
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..d92ef32 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
@@ -41,7 +41,6 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.Store;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdge;
@@ -49,7 +48,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.apache.hadoop.hbase.util.TimeOffsetEnvironmentEdge;
 import org.junit.AfterClass;
-import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
@@ -191,59 +189,4 @@ public class TestFIFOCompactionPolicy {
     desc.addFamily(colDesc);
     TEST_UTIL.getHBaseAdmin().createTable(desc);
   }
-
-  /**
-   * Unit test for HBASE-21504
-   */
-  @Test
-  public void testFIFOCompactionPolicyExpiredEmptyHFiles() throws Exception {
-    TableName tableName = TableName.valueOf("testFIFOCompactionPolicyExpiredEmptyHFiles");
-    HColumnDescriptor colDesc = new HColumnDescriptor(family).setTimeToLive(1);
-    HTableDescriptor desc = new HTableDescriptor(tableName)
-        .setValue(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY,
-          FIFOCompactionPolicy.class.getName())
-        .setValue(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
-          DisabledRegionSplitPolicy.class.getName())
-        .addFamily(colDesc);
-    Table table = TEST_UTIL.createTable(desc, null);
-    long ts = System.currentTimeMillis() - 10 * 1000;
-    Put put =
-        new Put(Bytes.toBytes("row1")).addColumn(family, qualifier, ts, Bytes.toBytes("value0"));
-    table.put(put);
-    TEST_UTIL.getHBaseAdmin().flush(tableName); // HFile-0
-    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);
-    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;
-      }
-    }
-    Assert.assertEquals(1, store.getStorefilesCount());
-    StoreFile sf = store.getStorefiles().iterator().next();
-    Assert.assertNotNull(sf);
-    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;
-      }
-    }
-    Assert.assertEquals(1, store.getStorefilesCount());
-    sf = store.getStorefiles().iterator().next();
-    Assert.assertNotNull(sf);
-    Assert.assertEquals(0, sf.getReader().getEntries());
-  }
 }


[2/2] hbase git commit: Update CHANGES.txt for 1.3.3RC0 again

Posted by ap...@apache.org.
Update CHANGES.txt for 1.3.3RC0 again


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/a75a458e
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/a75a458e
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/a75a458e

Branch: refs/heads/branch-1.3
Commit: a75a458e0c8f3c60db57f30ec4b06a606c9027b4
Parents: d2c1dee
Author: Andrew Purtell <ap...@apache.org>
Authored: Fri Dec 14 15:36:21 2018 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Fri Dec 14 15:36:21 2018 -0800

----------------------------------------------------------------------
 CHANGES.txt | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a75a458e/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 9377707..6aba7fb 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -120,7 +120,6 @@ Release Notes - HBase - Version 1.3.3 12/21/2018
     * [HBASE-21417] - Pre commit build is broken due to surefire plugin crashes
     * [HBASE-21439] - StochasticLoadBalancer RegionLoads aren’t being used in RegionLoad cost functions
     * [HBASE-21445] - CopyTable by bulkload will write hfile into yarn's HDFS 
-    * [HBASE-21504] - If enable FIFOCompactionPolicy, a compaction may write a "empty" hfile whose maxTimeStamp is long max. This kind of hfile will never be archived.
     * [HBASE-21546] - ConnectException in TestThriftHttpServer
     * [HBASE-21553] - schedLock not released in MasterProcedureScheduler
     * [HBASE-21582] - If call HBaseAdmin#snapshotAsync but forget call isSnapshotFinished, then SnapshotHFileCleaner will skip to run every time