You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by hu...@apache.org on 2020/05/06 16:51:03 UTC

[hbase] branch branch-2 updated: HBASE-24273 HBCK's "Orphan Regions on FileSystem" reports regions with referenced HFiles (#1613) (#1661)

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

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


The following commit(s) were added to refs/heads/branch-2 by this push:
     new 04ac08d  HBASE-24273 HBCK's "Orphan Regions on FileSystem" reports regions with referenced HFiles (#1613) (#1661)
04ac08d is described below

commit 04ac08d49b8062bc0f2032d1b70a496016504b27
Author: huaxiangsun <hu...@apache.org>
AuthorDate: Wed May 6 09:50:52 2020 -0700

    HBASE-24273 HBCK's "Orphan Regions on FileSystem" reports regions with referenced HFiles (#1613) (#1661)
    
    Signed-off-by: stack <st...@apache.org>
---
 .../org/apache/hadoop/hbase/master/HbckChore.java  | 36 +++++++++++++++++++---
 .../apache/hadoop/hbase/master/TestMetaFixer.java  | 10 ++++--
 2 files changed, 39 insertions(+), 7 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HbckChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HbckChore.java
index b25bb15..6d41008 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HbckChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HbckChore.java
@@ -28,6 +28,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.ScheduledChore;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.RegionInfo;
@@ -134,7 +135,7 @@ public class HbckChore extends ScheduledChore {
       loadRegionsFromInMemoryState();
       loadRegionsFromRSReport();
       try {
-        loadRegionsFromFS();
+        loadRegionsFromFS(scanForMergedParentRegions());
       } catch (IOException e) {
         LOG.warn("Failed to load the regions from filesystem", e);
       }
@@ -187,6 +188,31 @@ public class HbckChore extends ScheduledChore {
     }
   }
 
+  /**
+   * Scan hbase:meta to get set of merged parent regions, this is a very heavy scan.
+   *
+   * @return Return generated {@link HashSet}
+   */
+  private HashSet<String> scanForMergedParentRegions() throws IOException {
+    HashSet<String> mergedParentRegions = new HashSet<>();
+    // Null tablename means scan all of meta.
+    MetaTableAccessor.scanMetaForTableRegions(this.master.getConnection(),
+      r -> {
+        List<RegionInfo> mergeParents = MetaTableAccessor.getMergeRegions(r.rawCells());
+        if (mergeParents != null) {
+          for (RegionInfo mergeRegion : mergeParents) {
+            if (mergeRegion != null) {
+              // This region is already being merged
+              mergedParentRegions.add(mergeRegion.getEncodedName());
+            }
+          }
+        }
+        return true;
+        },
+      null);
+    return mergedParentRegions;
+  }
+
   private void loadRegionsFromInMemoryState() {
     List<RegionState> regionStates =
         master.getAssignmentManager().getRegionStates().getRegionStates();
@@ -256,7 +282,7 @@ public class HbckChore extends ScheduledChore {
     }
   }
 
-  private void loadRegionsFromFS() throws IOException {
+  private void loadRegionsFromFS(final HashSet<String> mergedParentRegions) throws IOException {
     Path rootDir = master.getMasterFileSystem().getRootDir();
     FileSystem fs = master.getMasterFileSystem().getFileSystem();
 
@@ -271,12 +297,12 @@ public class HbckChore extends ScheduledChore {
           continue;
         }
         HbckRegionInfo hri = regionInfoMap.get(encodedRegionName);
-        if (hri == null) {
+        // If it is not in in-memory database and not a merged region,
+        // report it as an orphan region.
+        if (hri == null && !mergedParentRegions.contains(encodedRegionName)) {
           orphanRegionsOnFS.put(encodedRegionName, regionDir);
           continue;
         }
-        HbckRegionInfo.HdfsEntry hdfsEntry = new HbckRegionInfo.HdfsEntry(regionDir);
-        hri.setHdfsEntry(hdfsEntry);
       }
       numRegions += regionDirs.size();
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaFixer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaFixer.java
index f1531a5..ee67110 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaFixer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaFixer.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.master.assignment.GCRegionProcedure;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -147,10 +148,12 @@ public class TestMetaFixer {
   @Test
   public void testOverlap() throws Exception {
     TableName tn = TableName.valueOf(this.name.getMethodName());
-    TEST_UTIL.createMultiRegionTable(tn, HConstants.CATALOG_FAMILY);
+    Table t = TEST_UTIL.createMultiRegionTable(tn, HConstants.CATALOG_FAMILY);
+    TEST_UTIL.loadTable(t, HConstants.CATALOG_FAMILY);
     List<RegionInfo> ris = MetaTableAccessor.getTableRegions(TEST_UTIL.getConnection(), tn);
     assertTrue(ris.size() > 5);
-    MasterServices services = TEST_UTIL.getHBaseCluster().getMaster();
+    HMaster services = TEST_UTIL.getHBaseCluster().getMaster();
+    HbckChore hbckChore = services.getHbckChore();
     services.getCatalogJanitor().scan();
     CatalogJanitor.Report report = services.getCatalogJanitor().getLastReport();
     assertTrue(report.isEmpty());
@@ -174,6 +177,9 @@ public class TestMetaFixer {
         throw new RuntimeException(e);
       }
     });
+
+    hbckChore.chore();
+    assertEquals(0, hbckChore.getOrphanRegionsOnFS().size());
   }
 
   /**