You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2019/08/06 05:40:12 UTC

[hbase] branch branch-2 updated (d7af4f9 -> c03bf73)

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

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


    from d7af4f9  HBASE-22785 Fixed Checkstyle issues in exceptions and enhanced Javadoc
     new 2bc9526  HBASE-22771 [HBCK2] fixMeta method and server-side support
     new 5fa8668  HBASE-22771 [HBCK2] fixMeta method and server-side support
     new c03bf73    HBASE-22771 [HBCK2] fixMeta method and server-side support; AMENDMENT remove mistaken .rej commit

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../org/apache/hadoop/hbase/MetaTableAccessor.java |   3 +-
 .../org/apache/hadoop/hbase/client/HBaseHbck.java  |  10 ++
 .../java/org/apache/hadoop/hbase/client/Hbck.java  |   5 +
 .../src/main/protobuf/Master.proto                 |   8 +
 .../apache/hadoop/hbase/master/CatalogJanitor.java | 100 +++++--------
 .../hadoop/hbase/master/MasterRpcServices.java     |  14 ++
 .../org/apache/hadoop/hbase/master/MetaFixer.java  | 163 +++++++++++++++++++++
 .../apache/hadoop/hbase/regionserver/HRegion.java  |  21 ++-
 .../hbase/regionserver/HRegionFileSystem.java      |  15 +-
 .../main/resources/hbase-webapps/master/hbck.jsp   |  30 ++--
 .../hbase/master/TestCatalogJanitorCluster.java    |   6 +-
 .../apache/hadoop/hbase/master/TestMetaFixer.java  | 127 ++++++++++++++++
 12 files changed, 403 insertions(+), 99 deletions(-)
 create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaFixer.java
 create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaFixer.java


[hbase] 01/03: HBASE-22771 [HBCK2] fixMeta method and server-side support

Posted by st...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 2bc9526080065d61aaff50d2d41c3e27e61b04cd
Author: stack <st...@apache.org>
AuthorDate: Tue Jul 30 17:07:48 2019 -0700

    HBASE-22771 [HBCK2] fixMeta method and server-side support
    
    This is a first cut at this patch. Implements hold fixing only
    currently.
    
    Add a fixMeta method to Hbck Interface.
    
    M hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
     Bug fix. If hole is on end of last table, I wasn't seeing it.
    
    A hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaFixer.java
     Add a general meta fixer class. Explains up top why this stuff doesn't
     belong inside MetaTableAccessor.
    
    M hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
     Break out the filesystem messing so don't have to copy it nor do more
     than is needed doing fixup for Region holes.
    
    M hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
     Change behavious slightly. If directory exists, don't fail as we did
     but try and keep going and create .regioninfo file if missing (or
     overwrite if in place). This should make it idempotent. Can rerun
     command. Lets see if any repercussions in test suite.
    
    A hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaFixer.java
     Add test.
    
    Signed-off-by: Zheng Hu <op...@gmail.com>
    Signed-off-by: Guanghao Zhang <zg...@apache.org>
---
 .../org/apache/hadoop/hbase/MetaTableAccessor.java |   3 +-
 .../org/apache/hadoop/hbase/client/HBaseHbck.java  |  10 ++
 .../apache/hadoop/hbase/client/HBaseHbck.java.rej  |  17 +++
 .../java/org/apache/hadoop/hbase/client/Hbck.java  |   5 +
 .../src/main/protobuf/Master.proto                 |   8 +
 .../apache/hadoop/hbase/master/CatalogJanitor.java | 100 +++++--------
 .../hadoop/hbase/master/MasterRpcServices.java     |  14 ++
 .../org/apache/hadoop/hbase/master/MetaFixer.java  | 163 +++++++++++++++++++++
 .../apache/hadoop/hbase/regionserver/HRegion.java  |  21 ++-
 .../hbase/regionserver/HRegionFileSystem.java      |  15 +-
 .../main/resources/hbase-webapps/master/hbck.jsp   |  30 ++--
 .../hbase/master/TestCatalogJanitorCluster.java    |   6 +-
 .../apache/hadoop/hbase/master/TestMetaFixer.java  | 129 ++++++++++++++++
 13 files changed, 422 insertions(+), 99 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
index 34ec9b3..c2ca02d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
@@ -1842,9 +1842,8 @@ public class MetaTableAccessor {
    */
   public static void deleteRegionInfo(Connection connection, RegionInfo regionInfo)
       throws IOException {
-    long time = EnvironmentEdgeManager.currentTime();
     Delete delete = new Delete(regionInfo.getRegionName());
-    delete.addFamily(getCatalogFamily(), time);
+    delete.addFamily(getCatalogFamily(), HConstants.LATEST_TIMESTAMP);
     deleteFromMetaTable(connection, delete);
     LOG.info("Deleted " + regionInfo.getRegionNameAsString());
   }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java
index 07736b8..d322333 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignsResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BypassProcedureRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BypassProcedureResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FixMetaRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.HbckService.BlockingInterface;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunHbckChoreRequest;
@@ -186,4 +187,13 @@ public class HBaseHbck implements Hbck {
       throw new IOException(se);
     }
   }
+
+  @Override
+  public void fixMeta() throws IOException {
+    try {
+      this.hbck.fixMeta(rpcControllerFactory.newController(), FixMetaRequest.newBuilder().build());
+    } catch (ServiceException se) {
+      throw new IOException(se);
+    }
+  }
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java.rej b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java.rej
new file mode 100644
index 0000000..28cb26d
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java.rej
@@ -0,0 +1,17 @@
+diff a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java	(rejected hunks)
+@@ -186,4 +187,13 @@ public class HBaseHbck implements Hbck {
+       throw new IOException(se);
+     }
+   }
+-}
+\ No newline at end of file
++
++  @Override
++  public void fixMeta() throws IOException {
++    try {
++      this.hbck.fixMeta(rpcControllerFactory.newController(), FixMetaRequest.newBuilder().build());
++    } catch (ServiceException se) {
++      throw new IOException(se);
++    }
++  }
++}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Hbck.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Hbck.java
index 0d29fec..5fe2c8e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Hbck.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Hbck.java
@@ -130,4 +130,9 @@ public interface Hbck extends Abortable, Closeable {
    * @throws IOException if a remote or network exception occurs
    */
   boolean runHbckChore() throws IOException;
+
+  /**
+   * Fix Meta.
+   */
+  void fixMeta() throws IOException;
 }
diff --git a/hbase-protocol-shaded/src/main/protobuf/Master.proto b/hbase-protocol-shaded/src/main/protobuf/Master.proto
index 003447a..15783b5 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Master.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto
@@ -1111,6 +1111,10 @@ message ScheduleServerCrashProcedureResponse {
   repeated uint64 pid = 1;
 }
 
+message FixMetaRequest {}
+
+message FixMetaResponse {}
+
 service HbckService {
   /** Update state of the table in meta only*/
   rpc SetTableStateInMeta(SetTableStateInMetaRequest)
@@ -1147,4 +1151,8 @@ service HbckService {
    */
   rpc RunHbckChore(RunHbckChoreRequest)
     returns(RunHbckChoreResponse);
+
+  /** Schedule a fix meta run. */
+  rpc FixMeta(FixMetaRequest)
+    returns(FixMetaResponse);
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
index e6564a5..192bc84 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
@@ -458,9 +458,9 @@ public class CatalogJanitor extends ScheduledChore {
     final Map<RegionInfo, Result> mergedRegions = new TreeMap<>(RegionInfo.COMPARATOR);
     int count = 0;
 
-    private final List<Pair<MetaRow, MetaRow>> holes = new ArrayList<>();
-    private final List<Pair<MetaRow, MetaRow>> overlaps = new ArrayList<>();
-    private final List<Pair<MetaRow, ServerName>> unknownServers = new ArrayList<>();
+    private final List<Pair<RegionInfo, RegionInfo>> holes = new ArrayList<>();
+    private final List<Pair<RegionInfo, RegionInfo>> overlaps = new ArrayList<>();
+    private final List<Pair<RegionInfo, ServerName>> unknownServers = new ArrayList<>();
     private final List<byte []> emptyRegionInfo = new ArrayList<>();
 
     @VisibleForTesting
@@ -470,15 +470,15 @@ public class CatalogJanitor extends ScheduledChore {
       return this.now;
     }
 
-    public List<Pair<MetaRow, MetaRow>> getHoles() {
+    public List<Pair<RegionInfo, RegionInfo>> getHoles() {
       return this.holes;
     }
 
-    public List<Pair<MetaRow, MetaRow>> getOverlaps() {
+    public List<Pair<RegionInfo, RegionInfo>> getOverlaps() {
       return this.overlaps;
     }
 
-    public List<Pair<MetaRow, ServerName>> getUnknownServers() {
+    public List<Pair<RegionInfo, ServerName>> getUnknownServers() {
       return unknownServers;
     }
 
@@ -497,19 +497,19 @@ public class CatalogJanitor extends ScheduledChore {
     @Override
     public String toString() {
       StringBuffer sb = new StringBuffer();
-      for (Pair<MetaRow, MetaRow> p: this.holes) {
+      for (Pair<RegionInfo, RegionInfo> p: this.holes) {
         if (sb.length() > 0) {
           sb.append(", ");
         }
-        sb.append("hole=" + Bytes.toStringBinary(p.getFirst().metaRow) + "/" +
-            Bytes.toStringBinary(p.getSecond().metaRow));
+        sb.append("hole=" + p.getFirst().getRegionNameAsString() + "/" +
+            p.getSecond().getRegionNameAsString());
       }
-      for (Pair<MetaRow, MetaRow> p: this.overlaps) {
+      for (Pair<RegionInfo, RegionInfo> p: this.overlaps) {
         if (sb.length() > 0) {
           sb.append(", ");
         }
-        sb.append("overlap=").append(Bytes.toStringBinary(p.getFirst().metaRow)).append("/").
-            append(Bytes.toStringBinary(p.getSecond().metaRow));
+        sb.append("overlap=" + p.getFirst().getRegionNameAsString() + "/" +
+            p.getSecond().getRegionNameAsString());
       }
       for (byte [] r: this.emptyRegionInfo) {
         if (sb.length() > 0) {
@@ -517,53 +517,18 @@ public class CatalogJanitor extends ScheduledChore {
         }
         sb.append("empty=").append(Bytes.toStringBinary(r));
       }
-      for (Pair<MetaRow, ServerName> p: this.unknownServers) {
+      for (Pair<RegionInfo, ServerName> p: this.unknownServers) {
         if (sb.length() > 0) {
           sb.append(", ");
         }
         sb.append("unknown_server=").append(p.getSecond()).append("/").
-            append(Bytes.toStringBinary(p.getFirst().metaRow));
+            append(p.getFirst().getRegionNameAsString());
       }
       return sb.toString();
     }
   }
 
   /**
-   * Simple datastructure to hold a MetaRow content.
-   */
-  public static class MetaRow {
-    /**
-     * A marker for use in case where there is a hole at the very
-     * first row in hbase:meta. Should never happen.
-     */
-    private static final MetaRow UNDEFINED =
-        new MetaRow(HConstants.EMPTY_START_ROW, RegionInfo.UNDEFINED);
-
-    /**
-     * Row from hbase:meta table.
-     */
-    private final byte [] metaRow;
-
-    /**
-     * The decoded RegionInfo gotten from hbase:meta.
-     */
-    private final RegionInfo regionInfo;
-
-    MetaRow(byte [] metaRow, RegionInfo regionInfo) {
-      this.metaRow = metaRow;
-      this.regionInfo = regionInfo;
-    }
-
-    public RegionInfo getRegionInfo() {
-      return regionInfo;
-    }
-
-    public byte[] getMetaRow() {
-      return metaRow;
-    }
-  }
-
-  /**
    * Visitor we use in here in CatalogJanitor to go against hbase:meta table.
    * Generates a Report made of a collection of split parents and counts of rows
    * in the hbase:meta table. Also runs hbase:meta consistency checks to
@@ -584,7 +549,7 @@ public class CatalogJanitor extends ScheduledChore {
     /**
      * RegionInfo from previous row.
      */
-    private MetaRow previous = null;
+    private RegionInfo previous = null;
 
     ReportMakingVisitor(MasterServices services) {
       this.services = services;
@@ -647,7 +612,12 @@ public class CatalogJanitor extends ScheduledChore {
         return ri;
       }
 
-      MetaRow mrri = new MetaRow(metaTableRow.getRow(), ri);
+      if (!Bytes.equals(metaTableRow.getRow(), ri.getRegionName())) {
+        LOG.warn("INCONSISTENCY: Row name is not equal to serialized info:regioninfo content; " +
+                "row={} {}; See if RegionInfo is referenced in another hbase:meta row? Delete?",
+            Bytes.toStringBinary(metaTableRow.getRow()), ri.getRegionNameAsString());
+        return null;
+      }
       // If table is disabled, skip integrity check.
       if (!isTableDisabled(ri)) {
         if (isTableTransition(ri)) {
@@ -655,28 +625,28 @@ public class CatalogJanitor extends ScheduledChore {
           // and if this is the first. Report 'hole' if neither is true.
           // HBCK1 used to have a special category for missing start or end keys.
           // We'll just lump them in as 'holes'.
-          if ((this.previous != null && !this.previous.regionInfo.isLast()) || !ri.isFirst()) {
-            addHole(this.previous == null? MetaRow.UNDEFINED: this.previous, mrri);
+          if ((this.previous != null && !this.previous.isLast()) || !ri.isFirst()) {
+            addHole(this.previous == null? RegionInfo.UNDEFINED: this.previous, ri);
           }
         } else {
-          if (!this.previous.regionInfo.isNext(ri)) {
-            if (this.previous.regionInfo.isOverlap(ri)) {
-              addOverlap(this.previous, mrri);
+          if (!this.previous.isNext(ri)) {
+            if (this.previous.isOverlap(ri)) {
+              addOverlap(this.previous, ri);
             } else {
-              addHole(this.previous, mrri);
+              addHole(this.previous, ri);
             }
           }
         }
       }
-      this.previous = mrri;
+      this.previous = ri;
       return ri;
     }
 
-    private void addOverlap(MetaRow a, MetaRow b) {
+    private void addOverlap(RegionInfo a, RegionInfo b) {
       this.report.overlaps.add(new Pair<>(a, b));
     }
 
-    private void addHole(MetaRow a, MetaRow b) {
+    private void addHole(RegionInfo a, RegionInfo b) {
       this.report.holes.add(new Pair<>(a, b));
     }
 
@@ -723,8 +693,7 @@ public class CatalogJanitor extends ScheduledChore {
             isServerKnownAndOnline(sn);
         switch (state) {
           case UNKNOWN:
-            this.report.unknownServers.add(
-                new Pair(new MetaRow(metaTableRow, location.getRegion()), sn));
+            this.report.unknownServers.add(new Pair(location.getRegion(), sn));
             break;
 
           default:
@@ -738,11 +707,16 @@ public class CatalogJanitor extends ScheduledChore {
      */
     private boolean isTableTransition(RegionInfo ri) {
       return this.previous == null ||
-          !this.previous.regionInfo.getTable().equals(ri.getTable());
+          !this.previous.getTable().equals(ri.getTable());
     }
 
     @Override
     public void close() throws IOException {
+      // This is a table transition... after the last region. Check previous.
+      // Should be last region. If not, its a hole on end of laster table.
+      if (this.previous != null && !this.previous.isLast()) {
+        addHole(this.previous, RegionInfo.UNDEFINED);
+      }
       this.closed = true;
     }
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 1c9004b..84402c3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -182,6 +182,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTabl
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FixMetaResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FixMetaRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
@@ -2511,6 +2513,18 @@ public class MasterRpcServices extends RSRpcServices
   }
 
   @Override
+  public FixMetaResponse fixMeta(RpcController controller, FixMetaRequest request)
+      throws ServiceException {
+    try {
+      MetaFixer mf = new MetaFixer(this.master);
+      mf.fix();
+      return FixMetaResponse.newBuilder().build();
+    } catch (IOException ioe) {
+      throw new ServiceException(ioe);
+    }
+  }
+
+  @Override
   public SwitchRpcThrottleResponse switchRpcThrottle(RpcController controller,
       SwitchRpcThrottleRequest request) throws ServiceException {
     try {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaFixer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaFixer.java
new file mode 100644
index 0000000..d94e4bd
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaFixer.java
@@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Server-side fixing of bad or inconsistent state in hbase:meta.
+ * Distinct from MetaTableAccessor because {@link MetaTableAccessor} is about low-level
+ * manipulations driven by the Master. This class MetaFixer is
+ * employed by the Master and it 'knows' about holes and orphan
+ * and encapsulates their fixing on behalf of the Master.
+ */
+@InterfaceAudience.Private
+class MetaFixer {
+  private static final Logger LOG = LoggerFactory.getLogger(MetaFixer.class);
+  private final MasterServices masterServices;
+
+  MetaFixer(MasterServices masterServices) {
+    this.masterServices = masterServices;
+  }
+
+  void fix() throws IOException {
+    CatalogJanitor.Report report = this.masterServices.getCatalogJanitor().getLastReport();
+    if (report == null) {
+      LOG.info("CatalogJanitor has not generated a report yet; run 'catalogjanitor_run' in " +
+          "shell or wait until CatalogJanitor chore runs.");
+      return;
+    }
+    fixHoles(report);
+    fixOverlaps(report);
+  }
+
+  /**
+   * If hole, it papers it over by adding a region in the filesystem and to hbase:meta.
+   * Does not assign.
+   * @return True if we fixed any 'holes'.
+   */
+  boolean fixHoles(CatalogJanitor.Report report) throws IOException {
+    boolean result = false;
+    List<Pair<RegionInfo, RegionInfo>> holes = report.getHoles();
+    if (holes.isEmpty()) {
+      LOG.debug("No holes.");
+      return result;
+    }
+    for (Pair<RegionInfo, RegionInfo> p: holes) {
+      RegionInfo ri = getHoleCover(p);
+      if (ri == null) {
+        continue;
+      }
+      Configuration configuration = this.masterServices.getConfiguration();
+      HRegion.createRegionDir(configuration, ri, FSUtils.getRootDir(configuration));
+      // If an error here, then we'll have a region in the filesystem but not
+      // in hbase:meta (if the below fails). Should be able to rerun the fix.
+      // The second call to createRegionDir will just go through. Idempotent.
+      Put put = MetaTableAccessor.makePutFromRegionInfo(ri, HConstants.LATEST_TIMESTAMP);
+      MetaTableAccessor.putsToMetaTable(this.masterServices.getConnection(), Arrays.asList(put));
+      LOG.info("Fixed hole by adding {}; region is NOT assigned (assign to online).", ri);
+      result = true;
+    }
+    return result;
+  }
+
+  /**
+   * @return Calculated RegionInfo that covers the hole <code>hole</code>
+   */
+  private RegionInfo getHoleCover(Pair<RegionInfo, RegionInfo> hole) {
+    RegionInfo holeCover = null;
+    RegionInfo left = hole.getFirst();
+    RegionInfo right = hole.getSecond();
+    if (left.getTable().equals(right.getTable())) {
+      // Simple case.
+      if (Bytes.compareTo(left.getEndKey(), right.getStartKey()) >= 0) {
+        LOG.warn("Skipping hole fix; left-side endKey is not less than right-side startKey; " +
+            "left=<{}>, right=<{}>", left, right);
+        return holeCover;
+      }
+      holeCover = buildRegionInfo(left.getTable(), left.getEndKey(), right.getStartKey());
+    } else {
+      boolean leftUndefined = left.equals(RegionInfo.UNDEFINED);
+      boolean rightUnefined = right.equals(RegionInfo.UNDEFINED);
+      boolean last = left.isLast();
+      boolean first = right.isFirst();
+      if (leftUndefined && rightUnefined) {
+        LOG.warn("Skipping hole fix; both the hole left-side and right-side RegionInfos are " +
+            "UNDEFINED; left=<{}>, right=<{}>", left, right);
+        return holeCover;
+      }
+      if (leftUndefined || last) {
+        holeCover = buildRegionInfo(right.getTable(), HConstants.EMPTY_START_ROW,
+            right.getStartKey());
+      } else if (rightUnefined || first) {
+        holeCover = buildRegionInfo(left.getTable(), left.getEndKey(), HConstants.EMPTY_END_ROW);
+      } else {
+        LOG.warn("Skipping hole fix; don't know what to do with left=<{}>, right=<{}>",
+            left, right);
+        return holeCover;
+      }
+    }
+    return holeCover;
+  }
+
+  private RegionInfo buildRegionInfo(TableName tn, byte [] start, byte [] end) {
+    return RegionInfoBuilder.newBuilder(tn).setStartKey(start).setEndKey(end).build();
+  }
+
+  boolean fixOverlaps(CatalogJanitor.Report report) throws IOException {
+    boolean result = false;
+    List<Pair<RegionInfo, RegionInfo>> overlaps = report.getOverlaps();
+    if (overlaps.isEmpty()) {
+      LOG.debug("No overlaps.");
+      return result;
+    }
+    for (Pair<RegionInfo, RegionInfo> p: overlaps) {
+      RegionInfo ri = getHoleCover(p);
+      if (ri == null) {
+        continue;
+      }
+      Configuration configuration = this.masterServices.getConfiguration();
+      HRegion.createRegionDir(configuration, ri, FSUtils.getRootDir(configuration));
+      // If an error here, then we'll have a region in the filesystem but not
+      // in hbase:meta (if the below fails). Should be able to rerun the fix.
+      // The second call to createRegionDir will just go through. Idempotent.
+      Put put = MetaTableAccessor.makePutFromRegionInfo(ri, HConstants.LATEST_TIMESTAMP);
+      MetaTableAccessor.putsToMetaTable(this.masterServices.getConnection(), Arrays.asList(put));
+      LOG.info("Fixed hole by adding {}; region is NOT assigned (assign to online).", ri);
+      result = true;
+    }
+    return result;
+  }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 246ffea..d695fef 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -7073,12 +7073,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         final Configuration conf, final TableDescriptor hTableDescriptor,
         final WAL wal, final boolean initialize)
   throws IOException {
-    LOG.info("creating HRegion " + info.getTable().getNameAsString()
-        + " HTD == " + hTableDescriptor + " RootDir = " + rootDir +
-        " Table name == " + info.getTable().getNameAsString());
+    LOG.info("creating " + info
+        + ", tableDescriptor=" + (hTableDescriptor == null? "null": hTableDescriptor) +
+        ", regionDir=" + rootDir);
+    createRegionDir(conf, info, rootDir);
     FileSystem fs = rootDir.getFileSystem(conf);
     Path tableDir = FSUtils.getTableDir(rootDir, info.getTable());
-    HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir, info);
     HRegion region = HRegion.newHRegion(tableDir, wal, fs, conf, info, hTableDescriptor, null);
     if (initialize) {
       region.initialize(null);
@@ -7086,6 +7086,19 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     return region;
   }
 
+  /**
+   * Create the region directory in the filesystem.
+   */
+  public static HRegionFileSystem createRegionDir(Configuration configuration, RegionInfo ri,
+        Path rootDir)
+      throws IOException {
+    FileSystem fs = rootDir.getFileSystem(configuration);
+    Path tableDir = FSUtils.getTableDir(rootDir, ri.getTable());
+    // If directory already exists, will log warning and keep going. Will try to create
+    // .regioninfo. If one exists, will overwrite.
+    return HRegionFileSystem.createRegionOnFileSystem(configuration, fs, tableDir, ri);
+  }
+
   public static HRegion createHRegion(final RegionInfo info, final Path rootDir,
                                       final Configuration conf,
                                       final TableDescriptor hTableDescriptor,
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
index 24026e6..45a2627 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
@@ -26,6 +26,7 @@ import java.util.List;
 import java.util.Objects;
 import java.util.Optional;
 import java.util.UUID;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -867,6 +868,7 @@ public class HRegionFileSystem {
 
   /**
    * Write the .regioninfo file on-disk.
+   * Overwrites if exists already.
    */
   private static void writeRegionInfoFileContent(final Configuration conf, final FileSystem fs,
       final Path regionInfoFile, final byte[] content) throws IOException {
@@ -989,13 +991,12 @@ public class HRegionFileSystem {
       Path regionDir = regionFs.getRegionDir();
       if (fs.exists(regionDir)) {
         LOG.warn("Trying to create a region that already exists on disk: " + regionDir);
-        throw new IOException("The specified region already exists on disk: " + regionDir);
-      }
-
-      // Create the region directory
-      if (!createDirOnFileSystem(fs, conf, regionDir)) {
-        LOG.warn("Unable to create the region directory: " + regionDir);
-        throw new IOException("Unable to create region directory: " + regionDir);
+      } else {
+        // Create the region directory
+        if (!createDirOnFileSystem(fs, conf, regionDir)) {
+          LOG.warn("Unable to create the region directory: " + regionDir);
+          throw new IOException("Unable to create region directory: " + regionDir);
+        }
       }
 
       // Write HRI to a file in case we need to recover hbase:meta
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/hbck.jsp b/hbase-server/src/main/resources/hbase-webapps/master/hbck.jsp
index fc212e8..fcc0732 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/hbck.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/hbck.jsp
@@ -27,6 +27,7 @@
          import="java.time.ZonedDateTime"
          import="java.time.format.DateTimeFormatter"
 %>
+<%@ page import="org.apache.hadoop.hbase.client.RegionInfo" %>
 <%@ page import="org.apache.hadoop.hbase.master.HbckChore" %>
 <%@ page import="org.apache.hadoop.hbase.master.HMaster" %>
 <%@ page import="org.apache.hadoop.hbase.ServerName" %>
@@ -34,7 +35,6 @@
 <%@ page import="org.apache.hadoop.hbase.util.Pair" %>
 <%@ page import="org.apache.hadoop.hbase.master.CatalogJanitor" %>
 <%@ page import="org.apache.hadoop.hbase.master.CatalogJanitor.Report" %>
-<%@ page import="org.apache.hadoop.hbase.master.CatalogJanitor.MetaRow" %>
 <%
   HMaster master = (HMaster) getServletContext().getAttribute(HMaster.MASTER);
   pageContext.setAttribute("pageTitle", "HBase Master HBCK Report: " + master.getServerName());
@@ -188,17 +188,13 @@
           </div>
           <table class="table table-striped">
             <tr>
-              <th>Row before hole</th>
               <th>RegionInfo</th>
-              <th>Row after hole</th>
               <th>RegionInfo</th>
             </tr>
-            <% for (Pair<MetaRow, MetaRow> p : report.getHoles()) { %>
+            <% for (Pair<RegionInfo, RegionInfo> p : report.getHoles()) { %>
             <tr>
-              <td><%= Bytes.toStringBinary(p.getFirst().getMetaRow()) %></td>
-              <td><%= p.getFirst().getRegionInfo() %></td>
-              <td><%= Bytes.toStringBinary(p.getSecond().getMetaRow()) %></td>
-              <td><%= p.getSecond().getRegionInfo() %></td>
+              <td><%= p.getFirst() %></td>
+              <td><%= p.getSecond() %></td>
             </tr>
             <% } %>
 
@@ -213,17 +209,13 @@
             </div>
             <table class="table table-striped">
               <tr>
-                <th>Row</th>
                 <th>RegionInfo</th>
-                <th>Other Row</th>
                 <th>Other RegionInfo</th>
               </tr>
-              <% for (Pair<MetaRow, MetaRow> p : report.getOverlaps()) { %>
+              <% for (Pair<RegionInfo, RegionInfo> p : report.getOverlaps()) { %>
               <tr>
-                <td><%= Bytes.toStringBinary(p.getFirst().getMetaRow()) %></td>
-                <td><%= p.getFirst().getRegionInfo() %></td>
-                <td><%= Bytes.toStringBinary(p.getSecond().getMetaRow()) %></td>
-                <td><%= p.getSecond().getRegionInfo() %></td>
+                <td><%= p.getFirst() %></td>
+                <td><%= p.getSecond() %></td>
               </tr>
               <% } %>
 
@@ -238,15 +230,13 @@
             </div>
             <table class="table table-striped">
               <tr>
-                <th>Row</th>
-                <th>ServerName</th>
                 <th>RegionInfo</th>
+                <th>ServerName</th>
               </tr>
-              <% for (Pair<MetaRow, ServerName> p: report.getUnknownServers()) { %>
+              <% for (Pair<RegionInfo, ServerName> p: report.getUnknownServers()) { %>
               <tr>
-                <td><%= Bytes.toStringBinary(p.getFirst().getMetaRow()) %></td>
+                <td><%= p.getFirst() %></td>
                 <td><%= p.getSecond() %></td>
-                <td><%= p.getFirst().getRegionInfo() %></td>
               </tr>
               <% } %>
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitorCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitorCluster.java
index 090690b..2bbdaea 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitorCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitorCluster.java
@@ -93,9 +93,9 @@ public class TestCatalogJanitorCluster {
     report = janitor.getLastReport();
     assertFalse(report.isEmpty());
     assertEquals(1, report.getHoles().size());
-    assertTrue(report.getHoles().get(0).getFirst().getRegionInfo().getTable().equals(T1));
-    assertTrue(report.getHoles().get(0).getFirst().getRegionInfo().isLast());
-    assertTrue(report.getHoles().get(0).getSecond().getRegionInfo().getTable().equals(T2));
+    assertTrue(report.getHoles().get(0).getFirst().getTable().equals(T1));
+    assertTrue(report.getHoles().get(0).getFirst().isLast());
+    assertTrue(report.getHoles().get(0).getSecond().getTable().equals(T2));
     assertEquals(0, report.getOverlaps().size());
     // Next, add overlaps to first row in t3
     List<RegionInfo> t3Ris = MetaTableAccessor.getTableRegions(TEST_UTIL.getConnection(), T3);
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
new file mode 100644
index 0000000..6a24f86
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaFixer.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+@Category({MasterTests.class, LargeTests.class})
+public class TestMetaFixer {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestMetaFixer.class);
+  @Rule
+  public TestName name = new TestName();
+  private static final Logger LOG = LoggerFactory.getLogger(TestMetaFixer.class);
+
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  @BeforeClass
+  public static void setupBeforeClass() throws Exception {
+    TEST_UTIL.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  private void deleteRegion(MasterServices services, RegionInfo ri) throws IOException {
+    MetaTableAccessor.deleteRegionInfo(TEST_UTIL.getConnection(), ri);
+    // Delete it from Master context too else it sticks around.
+    services.getAssignmentManager().getRegionStates().deleteRegion(ri);
+  }
+
+  @Test
+  public void testPlugsHoles() throws IOException {
+    TableName tn = TableName.valueOf(this.name.getMethodName());
+    TEST_UTIL.createMultiRegionTable(tn, HConstants.CATALOG_FAMILY);
+    List<RegionInfo> ris = MetaTableAccessor.getTableRegions(TEST_UTIL.getConnection(), tn);
+    MasterServices services = TEST_UTIL.getHBaseCluster().getMaster();
+    services.getCatalogJanitor().scan();
+    CatalogJanitor.Report report = services.getCatalogJanitor().getLastReport();
+    Assert.assertTrue(report.isEmpty());
+    int originalCount = ris.size();
+    // Remove first, last and middle region. See if hole gets plugged. Table has 26 regions.
+    deleteRegion(services, ris.get(ris.size() -1));
+    deleteRegion(services, ris.get(3));
+    deleteRegion(services, ris.get(0));
+    ris = MetaTableAccessor.getTableRegions(TEST_UTIL.getConnection(), tn);
+    services.getCatalogJanitor().scan();
+    report = services.getCatalogJanitor().getLastReport();
+    Assert.assertEquals(report.toString(), 3, report.getHoles().size());
+    MetaFixer fixer = new MetaFixer(services);
+    Assert.assertTrue(fixer.fixHoles(report));
+    services.getCatalogJanitor().scan();
+    report = services.getCatalogJanitor().getLastReport();
+    Assert.assertTrue(report.toString(), report.isEmpty());
+    // Disable and reenable so the added regions get reassigned.
+    TEST_UTIL.getAdmin().disableTable(tn);
+    TEST_UTIL.getAdmin().enableTable(tn);
+    ris = MetaTableAccessor.getTableRegions(TEST_UTIL.getConnection(), tn);
+    Assert.assertEquals(originalCount, ris.size());
+  }
+
+  /**
+   * Just make sure running fixMeta does right thing for the case
+   * of a single-region Table where the region gets dropped.
+   * There is nothing much we can do. We can't restore what
+   * we don't know about (at least from a read of hbase:meta).
+   */
+  @Test
+  public void testOneRegionTable() throws IOException {
+    TableName tn = TableName.valueOf(this.name.getMethodName());
+    TEST_UTIL.createTable(tn, HConstants.CATALOG_FAMILY);
+    List<RegionInfo> ris = MetaTableAccessor.getTableRegions(TEST_UTIL.getConnection(), tn);
+    MasterServices services = TEST_UTIL.getHBaseCluster().getMaster();
+    services.getCatalogJanitor().scan();
+    CatalogJanitor.Report report = services.getCatalogJanitor().getLastReport();
+    int originalCount = ris.size();
+    deleteRegion(services, ris.get(0));
+    services.getCatalogJanitor().scan();
+    report = services.getCatalogJanitor().getLastReport();
+    ris = MetaTableAccessor.getTableRegions(TEST_UTIL.getConnection(), tn);
+    Assert.assertTrue(ris.isEmpty());
+    MetaFixer fixer = new MetaFixer(services);
+    Assert.assertFalse(fixer.fixHoles(report));
+    report = services.getCatalogJanitor().getLastReport();
+    Assert.assertTrue(report.isEmpty());
+    ris = MetaTableAccessor.getTableRegions(TEST_UTIL.getConnection(), tn);
+    Assert.assertEquals(0, ris.size());
+  }
+}


[hbase] 03/03: HBASE-22771 [HBCK2] fixMeta method and server-side support; AMENDMENT remove mistaken .rej commit

Posted by st...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit c03bf73e2487fd8ba55548652588a810e3faf45c
Author: stack <st...@apache.org>
AuthorDate: Mon Aug 5 22:39:54 2019 -0700

      HBASE-22771 [HBCK2] fixMeta method and server-side support; AMENDMENT remove mistaken .rej commit
---
 .../org/apache/hadoop/hbase/client/HBaseHbck.java.rej   | 17 -----------------
 1 file changed, 17 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java.rej b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java.rej
deleted file mode 100644
index 28cb26d..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java.rej
+++ /dev/null
@@ -1,17 +0,0 @@
-diff a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java	(rejected hunks)
-@@ -186,4 +187,13 @@ public class HBaseHbck implements Hbck {
-       throw new IOException(se);
-     }
-   }
--}
-\ No newline at end of file
-+
-+  @Override
-+  public void fixMeta() throws IOException {
-+    try {
-+      this.hbck.fixMeta(rpcControllerFactory.newController(), FixMetaRequest.newBuilder().build());
-+    } catch (ServiceException se) {
-+      throw new IOException(se);
-+    }
-+  }
-+}


[hbase] 02/03: HBASE-22771 [HBCK2] fixMeta method and server-side support

Posted by st...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 5fa86682026b1ce2f84cffe51ef81e49556fa4c8
Author: stack <st...@apache.org>
AuthorDate: Tue Jul 30 17:07:48 2019 -0700

    HBASE-22771 [HBCK2] fixMeta method and server-side support
    
    This is a first cut at this patch. Implements hold fixing only
    currently.
    
    Add a fixMeta method to Hbck Interface.
    
    M hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
     Bug fix. If hole is on end of last table, I wasn't seeing it.
    
    A hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaFixer.java
     Add a general meta fixer class. Explains up top why this stuff doesn't
     belong inside MetaTableAccessor.
    
    M hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
     Break out the filesystem messing so don't have to copy it nor do more
     than is needed doing fixup for Region holes.
    
    M hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
     Change behavious slightly. If directory exists, don't fail as we did
     but try and keep going and create .regioninfo file if missing (or
     overwrite if in place). This should make it idempotent. Can rerun
     command. Lets see if any repercussions in test suite.
    
    A hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaFixer.java
     Add test.
    
    Signed-off-by: Zheng Hu <op...@gmail.com>
    Signed-off-by: Guanghao Zhang <zg...@apache.org>
---
 .../src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java | 2 +-
 .../src/test/java/org/apache/hadoop/hbase/master/TestMetaFixer.java     | 2 --
 2 files changed, 1 insertion(+), 3 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 84402c3..9be2045 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -182,8 +182,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTabl
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FixMetaResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FixMetaRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FixMetaResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
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 6a24f86..2bffe31 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
@@ -25,9 +25,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.junit.AfterClass;