You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2018/10/26 23:44:43 UTC

hbase git commit: HBASE-21175 Partially initialized SnapshotHFileCleaner leads to NPE during TestHFileArchiving

Repository: hbase
Updated Branches:
  refs/heads/master e5ba79816 -> 7cdb52519


HBASE-21175 Partially initialized SnapshotHFileCleaner leads to NPE during TestHFileArchiving

Signed-off-by: tedyu <yu...@gmail.com>


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

Branch: refs/heads/master
Commit: 7cdb52519236966a7cb6dff7fbd0609c87545f75
Parents: e5ba798
Author: Artem Ervits <ge...@gmail.com>
Authored: Fri Oct 26 13:39:05 2018 -0400
Committer: tedyu <yu...@gmail.com>
Committed: Fri Oct 26 16:44:33 2018 -0700

----------------------------------------------------------------------
 .../hbase/master/cleaner/HFileCleaner.java      |  9 +++
 .../hadoop/hbase/backup/TestHFileArchiving.java | 58 ++++++++++++++------
 2 files changed, 50 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7cdb5251/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
index 47b0228..7ad6177 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
@@ -32,6 +32,8 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.hadoop.hbase.util.StealJobQueue;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -44,6 +46,7 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
  */
 @InterfaceAudience.Private
 public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> {
+  private MasterServices master;
 
   public static final String MASTER_HFILE_CLEANER_PLUGINS = "hbase.master.hfilecleaner.plugins";
 
@@ -496,4 +499,10 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> {
       t.interrupt();
     }
   }
+
+  public void init(Map<String, Object> params) {
+    if (params != null && params.containsKey(HMaster.MASTER)) {
+      this.master = (MasterServices) params.get(HMaster.MASTER);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/7cdb5251/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java
index dfebd38..2d32f9e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java
@@ -19,26 +19,33 @@ package org.apache.hadoop.hbase.backup;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
+import java.util.Objects;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
-import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.ChoreService;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
-import org.apache.hadoop.hbase.regionserver.CompactingMemStore;
 import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -177,10 +184,11 @@ public class TestHFileArchiving {
   /**
    * Test that the region directory is removed when we archive a region without store files, but
    * still has hidden files.
-   * @throws Exception
+   * @throws IOException throws an IOException if there's problem creating a table
+   *   or if there's an issue with accessing FileSystem.
    */
   @Test
-  public void testDeleteRegionWithNoStoreFiles() throws Exception {
+  public void testDeleteRegionWithNoStoreFiles() throws IOException {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     UTIL.createTable(tableName, TEST_FAM);
 
@@ -209,7 +217,7 @@ public class TestHFileArchiving {
     PathFilter nonHidden = new PathFilter() {
       @Override
       public boolean accept(Path file) {
-        return dirFilter.accept(file) && !file.getName().toString().startsWith(".");
+        return dirFilter.accept(file) && !file.getName().startsWith(".");
       }
     };
     FileStatus[] storeDirs = FSUtils.listStatus(fs, regionDir, nonHidden);
@@ -271,12 +279,14 @@ public class TestHFileArchiving {
     assertArchiveFiles(fs, storeFiles, 30000);
   }
 
-  private void assertArchiveFiles(FileSystem fs, List<String> storeFiles, long timeout) throws IOException {
+  private void assertArchiveFiles(FileSystem fs, List<String> storeFiles, long timeout)
+          throws IOException {
     long end = System.currentTimeMillis() + timeout;
     Path archiveDir = HFileArchiveUtil.getArchivePath(UTIL.getConfiguration());
     List<String> archivedFiles = new ArrayList<>();
 
-    // We have to ensure that the DeleteTableHandler is finished. HBaseAdmin.deleteXXX() can return before all files
+    // We have to ensure that the DeleteTableHandler is finished. HBaseAdmin.deleteXXX()
+    // can return before all files
     // are archived. We should fix HBASE-5487 and fix synchronous operations from admin.
     while (System.currentTimeMillis() < end) {
       archivedFiles = getAllFileNames(fs, archiveDir);
@@ -304,10 +314,11 @@ public class TestHFileArchiving {
 
   /**
    * Test that the store files are archived when a column family is removed.
-   * @throws Exception
+   * @throws java.io.IOException if there's a problem creating a table.
+   * @throws java.lang.InterruptedException problem getting a RegionServer.
    */
   @Test
-  public void testArchiveOnTableFamilyDelete() throws Exception {
+  public void testArchiveOnTableFamilyDelete() throws IOException, InterruptedException {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     UTIL.createTable(tableName, new byte[][] {TEST_FAM, Bytes.toBytes("fam2")});
 
@@ -374,11 +385,10 @@ public class TestHFileArchiving {
     Stoppable stoppable = new StoppableImplementation();
 
     // The cleaner should be looping without long pauses to reproduce the race condition.
-    HFileCleaner cleaner = new HFileCleaner(1, stoppable, conf, fs, archiveDir);
-    assertFalse("cleaner should not be null", cleaner == null);
+    HFileCleaner cleaner = getHFileCleaner(stoppable, conf, fs, archiveDir);
+    assertNotNull("cleaner should not be null", cleaner);
     try {
       choreService.scheduleChore(cleaner);
-
       // Keep creating/archiving new files while the cleaner is running in the other thread
       long startTime = System.currentTimeMillis();
       for (long fid = 0; (System.currentTimeMillis() - startTime) < TEST_TIME; ++fid) {
@@ -418,6 +428,16 @@ public class TestHFileArchiving {
     }
   }
 
+  // Avoid passing a null master to CleanerChore, see HBASE-21175
+  private HFileCleaner getHFileCleaner(Stoppable stoppable, Configuration conf,
+        FileSystem fs, Path archiveDir) throws IOException {
+    Map<String, Object> params = new HashMap<>();
+    params.put(HMaster.MASTER, UTIL.getMiniHBaseCluster().getMaster());
+    HFileCleaner cleaner = new HFileCleaner(1, stoppable, conf, fs, archiveDir);
+    cleaner.init(params);
+    return Objects.requireNonNull(cleaner);
+  }
+
   private void clearArchiveDirectory() throws IOException {
     UTIL.getTestFileSystem().delete(
       new Path(UTIL.getDefaultRootDirPath(), HConstants.HFILE_ARCHIVE_DIRECTORY), true);
@@ -428,9 +448,9 @@ public class TestHFileArchiving {
    * @param fs the file system to inspect
    * @param archiveDir the directory in which to look
    * @return a list of all files in the directory and sub-directories
-   * @throws IOException
+   * @throws java.io.IOException throws IOException in case FS is unavailable
    */
-  private List<String> getAllFileNames(final FileSystem fs, Path archiveDir) throws IOException {
+  private List<String> getAllFileNames(final FileSystem fs, Path archiveDir) throws IOException  {
     FileStatus[] files = FSUtils.listStatus(fs, archiveDir, new PathFilter() {
       @Override
       public boolean accept(Path p) {
@@ -446,12 +466,16 @@ public class TestHFileArchiving {
   /** Recursively lookup all the file names under the file[] array **/
   private List<String> recurseOnFiles(FileSystem fs, FileStatus[] files, List<String> fileNames)
       throws IOException {
-    if (files == null || files.length == 0) return fileNames;
+    if (files == null || files.length == 0) {
+      return fileNames;
+    }
 
     for (FileStatus file : files) {
       if (file.isDirectory()) {
         recurseOnFiles(fs, FSUtils.listStatus(fs, file.getPath(), null), fileNames);
-      } else fileNames.add(file.getPath().getName());
+      } else {
+        fileNames.add(file.getPath().getName());
+      }
     }
     return fileNames;
   }