diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java index edd83875aa0..2ebef45bbaa 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java @@ -49,6 +49,7 @@ import java.util.concurrent.FutureTask; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.regex.Pattern; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.FSDataInputStream; @@ -74,7 +75,6 @@ import org.apache.hadoop.hbase.io.HFileLink; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.regionserver.StoreFileInfo; import org.apache.hadoop.hbase.security.AccessDeniedException; -import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter; import org.apache.hadoop.hdfs.DFSClient; import org.apache.hadoop.hdfs.DFSHedgedReadMetrics; import org.apache.hadoop.hdfs.DistributedFileSystem; @@ -1209,10 +1209,10 @@ public abstract class FSUtils extends CommonFSUtils { * @throws IOException When scanning the directory fails. * @throws InterruptedException */ - public static Map getTableStoreFilePathMap( - Map resultMap, + public static Map getTableStoreFilePathMap(Map resultMap, final FileSystem fs, final Path hbaseRootDir, TableName tableName, final PathFilter sfFilter, - ExecutorService executor, final ErrorReporter errors) throws IOException, InterruptedException { + ExecutorService executor, final HbckErrorReporter errors) + throws IOException, InterruptedException { final Map finalResultMap = resultMap == null ? new ConcurrentHashMap<>(128, 0.75f, 32) : resultMap; @@ -1375,7 +1375,7 @@ public abstract class FSUtils extends CommonFSUtils { */ public static Map getTableStoreFilePathMap( final FileSystem fs, final Path hbaseRootDir, PathFilter sfFilter, - ExecutorService executor, ErrorReporter errors) + ExecutorService executor, HbckErrorReporter errors) throws IOException, InterruptedException { ConcurrentHashMap map = new ConcurrentHashMap<>(1024, 0.75f, 32); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java index 8cdc9325135..731bc10a2d2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java @@ -26,7 +26,6 @@ import java.io.StringWriter; import java.net.InetAddress; import java.net.URI; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.Comparator; @@ -42,9 +41,7 @@ import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.SortedMap; -import java.util.SortedSet; import java.util.TreeMap; -import java.util.TreeSet; import java.util.Vector; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentSkipListMap; @@ -59,6 +56,7 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; + import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; @@ -77,7 +75,6 @@ import org.apache.hadoop.hbase.ClusterMetrics.Option; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.MasterNotRunningException; @@ -123,11 +120,10 @@ import org.apache.hadoop.hbase.replication.ReplicationUtils; import org.apache.hadoop.hbase.security.AccessDeniedException; import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.util.Bytes.ByteArrayComparator; -import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE; +import org.apache.hadoop.hbase.util.HbckErrorReporter.ERROR_CODE; import org.apache.hadoop.hbase.util.hbck.HFileCorruptionChecker; import org.apache.hadoop.hbase.util.hbck.ReplicationChecker; import org.apache.hadoop.hbase.util.hbck.TableIntegrityErrorHandler; -import org.apache.hadoop.hbase.util.hbck.TableIntegrityErrorHandlerImpl; import org.apache.hadoop.hbase.wal.WALSplitUtil; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; @@ -147,12 +143,8 @@ import org.slf4j.LoggerFactory; import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.base.Joiner; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; -import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; -import org.apache.hbase.thirdparty.com.google.common.collect.Multimap; -import org.apache.hbase.thirdparty.com.google.common.collect.Ordering; import org.apache.hbase.thirdparty.com.google.common.collect.Sets; -import org.apache.hbase.thirdparty.com.google.common.collect.TreeMultimap; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface; @@ -214,7 +206,7 @@ public class HBaseFsck extends Configured implements Closeable { private static boolean rsSupportsOffline = true; private static final int DEFAULT_OVERLAPS_TO_SIDELINE = 2; private static final int DEFAULT_MAX_MERGE = 5; - private static final String TO_BE_LOADED = "to_be_loaded"; + /** * Here is where hbase-1.x used to default the lock for hbck1. * It puts in place a lock when it goes to write/make changes. @@ -301,15 +293,15 @@ public class HBaseFsck extends Configured implements Closeable { /********* * State *********/ - final private ErrorReporter errors; + final private HbckErrorReporter errors; int fixes = 0; /** * This map contains the state of all hbck items. It maps from encoded region - * name to HbckInfo structure. The information contained in HbckInfo is used + * name to HbckRegionInfo structure. The information contained in HbckRegionInfo is used * to detect and correct consistency (hdfs/meta/deployment) problems. */ - private TreeMap regionInfoMap = new TreeMap<>(); + private TreeMap regionInfoMap = new TreeMap<>(); // Empty regioninfo qualifiers in hbase:meta private Set emptyRegionInfoQualifiers = new HashSet<>(); @@ -323,12 +315,12 @@ public class HBaseFsck extends Configured implements Closeable { * unless checkMetaOnly is specified, in which case, it contains only * the meta table */ - private SortedMap tablesInfo = new ConcurrentSkipListMap<>(); + private SortedMap tablesInfo = new ConcurrentSkipListMap<>(); /** * When initially looking at HDFS, we attempt to find any orphaned data. */ - private List orphanHdfsDirs = Collections.synchronizedList(new ArrayList()); + private List orphanHdfsDirs = Collections.synchronizedList(new ArrayList<>()); private Map> orphanTableDirs = new HashMap<>(); private Map tableStates = new HashMap<>(); @@ -848,7 +840,7 @@ public class HBaseFsck extends Configured implements Closeable { return errors.summarize(); } - public static byte[] keyOnly (byte[] b) { + public static byte[] keyOnly(byte[] b) { if (b == null) return b; int rowlength = Bytes.toShort(b, 0); @@ -976,8 +968,8 @@ public class HBaseFsck extends Configured implements Closeable { /** * Iterates through the list of all orphan/invalid regiondirs. */ - private void adoptHdfsOrphans(Collection orphanHdfsDirs) throws IOException { - for (HbckInfo hi : orphanHdfsDirs) { + private void adoptHdfsOrphans(Collection orphanHdfsDirs) throws IOException { + for (HbckRegionInfo hi : orphanHdfsDirs) { LOG.info("Attempting to handle orphan hdfs dir: " + hi.getHdfsRegionDir()); adoptHdfsOrphan(hi); } @@ -993,7 +985,7 @@ public class HBaseFsck extends Configured implements Closeable { * overlapping regions. */ @SuppressWarnings("deprecation") - private void adoptHdfsOrphan(HbckInfo hi) throws IOException { + private void adoptHdfsOrphan(HbckRegionInfo hi) throws IOException { Path p = hi.getHdfsRegionDir(); FileSystem fs = p.getFileSystem(getConf()); FileStatus[] dirs = fs.listStatus(p); @@ -1004,9 +996,9 @@ public class HBaseFsck extends Configured implements Closeable { } TableName tableName = hi.getTableName(); - TableInfo tableInfo = tablesInfo.get(tableName); + HbckTableInfo tableInfo = tablesInfo.get(tableName); Preconditions.checkNotNull(tableInfo, "Table '" + tableName + "' not present!"); - TableDescriptor template = tableInfo.getHTD(); + TableDescriptor template = tableInfo.getTableDescriptor(); // find min and max key values Pair orphanRegionRange = null; @@ -1283,62 +1275,24 @@ public class HBaseFsck extends Configured implements Closeable { } } - public ErrorReporter getErrors() { + public HbckErrorReporter getErrors() { return errors; } - /** - * Read the .regioninfo file from the file system. If there is no - * .regioninfo, add it to the orphan hdfs region list. - */ - private void loadHdfsRegioninfo(HbckInfo hbi) throws IOException { - Path regionDir = hbi.getHdfsRegionDir(); - if (regionDir == null) { - if (hbi.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) { - // Log warning only for default/ primary replica with no region dir - LOG.warn("No HDFS region dir found: " + hbi + " meta=" + hbi.metaEntry); - } - return; - } - - if (hbi.hdfsEntry.hri != null) { - // already loaded data - return; - } - - FileSystem fs = FileSystem.get(getConf()); - RegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir); - LOG.debug("RegionInfo read: " + hri.toString()); - hbi.hdfsEntry.hri = hri; - } - - /** - * Exception thrown when a integrity repair operation fails in an - * unresolvable way. - */ - public static class RegionRepairException extends IOException { - private static final long serialVersionUID = 1L; - final IOException ioe; - public RegionRepairException(String s, IOException ioe) { - super(s); - this.ioe = ioe; - } - } - /** * Populate hbi's from regionInfos loaded from file system. */ - private SortedMap loadHdfsRegionInfos() + private SortedMap loadHdfsRegionInfos() throws IOException, InterruptedException { tablesInfo.clear(); // regenerating the data // generate region split structure - Collection hbckInfos = regionInfoMap.values(); + Collection hbckRegionInfos = regionInfoMap.values(); // Parallelized read of .regioninfo files. - List hbis = new ArrayList<>(hbckInfos.size()); + List hbis = new ArrayList<>(hbckRegionInfos.size()); List> hbiFutures; - for (HbckInfo hbi : hbckInfos) { + for (HbckRegionInfo hbi : hbckRegionInfos) { WorkItemHdfsRegionInfo work = new WorkItemHdfsRegionInfo(hbi, this, errors); hbis.add(work); } @@ -1360,7 +1314,7 @@ public class HBaseFsck extends Configured implements Closeable { Path hbaseRoot = FSUtils.getRootDir(getConf()); FileSystem fs = hbaseRoot.getFileSystem(getConf()); // serialized table info gathering. - for (HbckInfo hbi: hbckInfos) { + for (HbckRegionInfo hbi: hbckRegionInfos) { if (hbi.getHdfsHRI() == null) { // was an orphan @@ -1376,10 +1330,10 @@ public class HBaseFsck extends Configured implements Closeable { continue; } - TableInfo modTInfo = tablesInfo.get(tableName); + HbckTableInfo modTInfo = tablesInfo.get(tableName); if (modTInfo == null) { // only executed once per table. - modTInfo = new TableInfo(tableName); + modTInfo = new HbckTableInfo(tableName, this); tablesInfo.put(tableName, modTInfo); try { TableDescriptor htd = @@ -1414,7 +1368,8 @@ public class HBaseFsck extends Configured implements Closeable { * @return a set of column families * @throws IOException */ - private Set getColumnFamilyList(Set columns, HbckInfo hbi) throws IOException { + private Set getColumnFamilyList(Set columns, HbckRegionInfo hbi) + throws IOException { Path regionDir = hbi.getHdfsRegionDir(); FileSystem fs = regionDir.getFileSystem(getConf()); FileStatus[] subDirs = fs.listStatus(regionDir, new FSUtils.FamilyDirFilter(fs)); @@ -1533,11 +1488,11 @@ public class HBaseFsck extends Configured implements Closeable { } } - private SortedMap checkHdfsIntegrity(boolean fixHoles, + private SortedMap checkHdfsIntegrity(boolean fixHoles, boolean fixOverlaps) throws IOException { LOG.info("Checking HBase region split map from HDFS data..."); logParallelMerge(); - for (TableInfo tInfo : tablesInfo.values()) { + for (HbckTableInfo tInfo : tablesInfo.values()) { TableIntegrityErrorHandler handler; if (fixHoles || fixOverlaps) { handler = tInfo.new HDFSIntegrityFixer(tInfo, errors, getConf(), @@ -1553,7 +1508,7 @@ public class HBaseFsck extends Configured implements Closeable { return tablesInfo; } - private Path getSidelineDir() throws IOException { + Path getSidelineDir() throws IOException { if (sidelineDir == null) { Path hbaseDir = FSUtils.getRootDir(getConf()); Path hbckDir = new Path(hbaseDir, HConstants.HBCK_SIDELINEDIR_NAME); @@ -1566,7 +1521,7 @@ public class HBaseFsck extends Configured implements Closeable { /** * Sideline a region dir (instead of deleting it) */ - Path sidelineRegionDir(FileSystem fs, HbckInfo hi) throws IOException { + Path sidelineRegionDir(FileSystem fs, HbckRegionInfo hi) throws IOException { return sidelineRegionDir(fs, null, hi); } @@ -1579,7 +1534,7 @@ public class HBaseFsck extends Configured implements Closeable { * on. If NULL, it is ignored. */ Path sidelineRegionDir(FileSystem fs, - String parentDir, HbckInfo hi) throws IOException { + String parentDir, HbckRegionInfo hi) throws IOException { TableName tableName = hi.getTableName(); Path regionDir = hi.getHdfsRegionDir(); @@ -1667,7 +1622,7 @@ public class HBaseFsck extends Configured implements Closeable { * Check if the specified region's table is disabled. * @param tableName table to check status of */ - private boolean isTableDisabled(TableName tableName) { + boolean isTableDisabled(TableName tableName) { return tableStates.containsKey(tableName) && tableStates.get(tableName) .inStates(TableState.State.DISABLED, TableState.State.DISABLING); @@ -1761,12 +1716,13 @@ public class HBaseFsck extends Configured implements Closeable { return false; } ServerName sn = metaLocation.getServerName(); - MetaEntry m = new MetaEntry(metaLocation.getRegionInfo(), sn, EnvironmentEdgeManager.currentTime()); - HbckInfo hbckInfo = regionInfoMap.get(metaLocation.getRegionInfo().getEncodedName()); - if (hbckInfo == null) { - regionInfoMap.put(metaLocation.getRegionInfo().getEncodedName(), new HbckInfo(m)); + HbckRegionInfo.MetaEntry m = new HbckRegionInfo.MetaEntry(metaLocation.getRegion(), sn, + EnvironmentEdgeManager.currentTime()); + HbckRegionInfo hbckRegionInfo = regionInfoMap.get(metaLocation.getRegion().getEncodedName()); + if (hbckRegionInfo == null) { + regionInfoMap.put(metaLocation.getRegion().getEncodedName(), new HbckRegionInfo(m)); } else { - hbckInfo.metaEntry = m; + hbckRegionInfo.setMetaEntry(m); } } return true; @@ -1827,7 +1783,7 @@ public class HBaseFsck extends Configured implements Closeable { // for the non-primary ones. Keeps code cleaner this way. List workItems = new ArrayList<>(regionInfoMap.size()); - for (java.util.Map.Entry e: regionInfoMap.entrySet()) { + for (java.util.Map.Entry e: regionInfoMap.entrySet()) { if (e.getValue().getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) { workItems.add(new CheckRegionConsistencyWorkItem(e.getKey(), e.getValue())); } @@ -1839,7 +1795,7 @@ public class HBaseFsck extends Configured implements Closeable { // Run a pass over the replicas and fix any assignment issues that exist on the currently // deployed/undeployed replicas. List replicaWorkItems = new ArrayList<>(regionInfoMap.size()); - for (java.util.Map.Entry e: regionInfoMap.entrySet()) { + for (java.util.Map.Entry e: regionInfoMap.entrySet()) { if (e.getValue().getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) { replicaWorkItems.add(new CheckRegionConsistencyWorkItem(e.getKey(), e.getValue())); } @@ -1894,9 +1850,9 @@ public class HBaseFsck extends Configured implements Closeable { class CheckRegionConsistencyWorkItem implements Callable { private final String key; - private final HbckInfo hbi; + private final HbckRegionInfo hbi; - CheckRegionConsistencyWorkItem(String key, HbckInfo hbi) { + CheckRegionConsistencyWorkItem(String key, HbckRegionInfo hbi) { this.key = key; this.hbi = hbi; } @@ -1920,7 +1876,7 @@ public class HBaseFsck extends Configured implements Closeable { } } - private void addSkippedRegion(final HbckInfo hbi) { + private void addSkippedRegion(final HbckRegionInfo hbi) { Set skippedRegionNames = skippedRegions.get(hbi.getTableName()); if (skippedRegionNames == null) { skippedRegionNames = new HashSet<>(); @@ -1939,7 +1895,7 @@ public class HBaseFsck extends Configured implements Closeable { for (Entry entry : tableStates.entrySet()) { TableName tableName = entry.getKey(); TableState tableState = entry.getValue(); - TableInfo tableInfo = tablesInfo.get(tableName); + HbckTableInfo tableInfo = tablesInfo.get(tableName); if (isTableIncluded(tableName) && !tableName.isSystemTable() && tableInfo == null) { @@ -2002,8 +1958,8 @@ public class HBaseFsck extends Configured implements Closeable { /** * Deletes region from meta table */ - private void deleteMetaRegion(HbckInfo hi) throws IOException { - deleteMetaRegion(hi.metaEntry.getRegionName()); + private void deleteMetaRegion(HbckRegionInfo hi) throws IOException { + deleteMetaRegion(hi.getMetaEntry().getRegionName()); } /** @@ -2018,14 +1974,14 @@ public class HBaseFsck extends Configured implements Closeable { /** * Reset the split parent region info in meta table */ - private void resetSplitParent(HbckInfo hi) throws IOException { - RowMutations mutations = new RowMutations(hi.metaEntry.getRegionName()); - Delete d = new Delete(hi.metaEntry.getRegionName()); + private void resetSplitParent(HbckRegionInfo hi) throws IOException { + RowMutations mutations = new RowMutations(hi.getMetaEntry().getRegionName()); + Delete d = new Delete(hi.getMetaEntry().getRegionName()); d.addColumn(HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER); d.addColumn(HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER); mutations.add(d); - RegionInfo hri = RegionInfoBuilder.newBuilder(hi.metaEntry) + RegionInfo hri = RegionInfoBuilder.newBuilder(hi.getMetaEntry()) .setOffline(false) .setSplit(false) .build(); @@ -2033,7 +1989,7 @@ public class HBaseFsck extends Configured implements Closeable { mutations.add(p); meta.mutateRow(mutations); - LOG.info("Reset split parent " + hi.metaEntry.getRegionNameAsString() + " in META" ); + LOG.info("Reset split parent " + hi.getMetaEntry().getRegionNameAsString() + " in META"); } /** @@ -2044,12 +2000,12 @@ public class HBaseFsck extends Configured implements Closeable { * table. The problem has to do with in-memory only master state, so * restarting the HMaster or failing over to another should fix this. */ - private void offline(byte[] regionName) throws IOException { + void offline(byte[] regionName) throws IOException { String regionString = Bytes.toStringBinary(regionName); if (!rsSupportsOffline) { - LOG.warn("Using unassign region " + regionString - + " instead of using offline method, you should" - + " restart HMaster after these repairs"); + LOG.warn( + "Using unassign region " + regionString + " instead of using offline method, you should" + + " restart HMaster after these repairs"); admin.unassign(regionName, true); return; } @@ -2060,11 +2016,11 @@ public class HBaseFsck extends Configured implements Closeable { admin.offline(regionName); } catch (IOException ioe) { String notFoundMsg = "java.lang.NoSuchMethodException: " + - "org.apache.hadoop.hbase.master.HMaster.offline([B)"; + "org.apache.hadoop.hbase.master.HMaster.offline([B)"; if (ioe.getMessage().contains(notFoundMsg)) { - LOG.warn("Using unassign region " + regionString - + " instead of using offline method, you should" - + " restart HMaster after these repairs"); + LOG.warn("Using unassign region " + regionString + + " instead of using offline method, you should" + + " restart HMaster after these repairs"); rsSupportsOffline = false; // in the future just use unassign admin.unassign(regionName, true); return; @@ -2073,40 +2029,6 @@ public class HBaseFsck extends Configured implements Closeable { } } - private void undeployRegions(HbckInfo hi) throws IOException, InterruptedException { - undeployRegionsForHbi(hi); - // undeploy replicas of the region (but only if the method is invoked for the primary) - if (hi.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) { - return; - } - int numReplicas = admin.getTableDescriptor(hi.getTableName()).getRegionReplication(); - for (int i = 1; i < numReplicas; i++) { - if (hi.getPrimaryHRIForDeployedReplica() == null) continue; - RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica( - hi.getPrimaryHRIForDeployedReplica(), i); - HbckInfo h = regionInfoMap.get(hri.getEncodedName()); - if (h != null) { - undeployRegionsForHbi(h); - //set skip checks; we undeployed it, and we don't want to evaluate this anymore - //in consistency checks - h.setSkipChecks(true); - } - } - } - - private void undeployRegionsForHbi(HbckInfo hi) throws IOException, InterruptedException { - for (OnlineEntry rse : hi.deployedEntries) { - LOG.debug("Undeploy region " + rse.hri + " from " + rse.hsa); - try { - HBaseFsckRepair.closeRegionSilentlyAndWait(connection, rse.hsa, rse.hri); - offline(rse.hri.getRegionName()); - } catch (IOException ioe) { - LOG.warn("Got exception when attempting to offline region " - + Bytes.toString(rse.hri.getRegionName()), ioe); - } - } - } - /** * Attempts to undeploy a region from a region server based in information in * META. Any operations that modify the file system should make sure that @@ -2119,8 +2041,8 @@ public class HBaseFsck extends Configured implements Closeable { * the offline ipc call exposed on the master (<0.90.5, <0.92.0) a master * restart or failover may be required. */ - private void closeRegion(HbckInfo hi) throws IOException, InterruptedException { - if (hi.metaEntry == null && hi.hdfsEntry == null) { + void closeRegion(HbckRegionInfo hi) throws IOException, InterruptedException { + if (hi.getMetaEntry() == null && hi.getHdfsEntry() == null) { undeployRegions(hi); return; } @@ -2167,7 +2089,42 @@ public class HBaseFsck extends Configured implements Closeable { } } - private void tryAssignmentRepair(HbckInfo hbi, String msg) throws IOException, + private void undeployRegions(HbckRegionInfo hi) throws IOException, InterruptedException { + undeployRegionsForHbi(hi); + // undeploy replicas of the region (but only if the method is invoked for the primary) + if (hi.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) { + return; + } + int numReplicas = admin.getDescriptor(hi.getTableName()).getRegionReplication(); + for (int i = 1; i < numReplicas; i++) { + if (hi.getPrimaryHRIForDeployedReplica() == null) continue; + RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica( + hi.getPrimaryHRIForDeployedReplica(), i); + HbckRegionInfo h = regionInfoMap.get(hri.getEncodedName()); + if (h != null) { + undeployRegionsForHbi(h); + //set skip checks; we undeployed it, and we don't want to evaluate this anymore + //in consistency checks + h.setSkipChecks(true); + } + } + } + + private void undeployRegionsForHbi(HbckRegionInfo hi) throws IOException, InterruptedException { + for (HbckRegionInfo.OnlineEntry rse : hi.getOnlineEntries()) { + LOG.debug("Undeploy region " + rse.getRegionInfo() + " from " + rse.getServerName()); + try { + HBaseFsckRepair + .closeRegionSilentlyAndWait(connection, rse.getServerName(), rse.getRegionInfo()); + offline(rse.getRegionInfo().getRegionName()); + } catch (IOException ioe) { + LOG.warn("Got exception when attempting to offline region " + + Bytes.toString(rse.getRegionInfo().getRegionName()), ioe); + } + } + } + + private void tryAssignmentRepair(HbckRegionInfo hbi, String msg) throws IOException, KeeperException, InterruptedException { // If we are trying to fix the errors if (shouldFixAssignments()) { @@ -2176,7 +2133,7 @@ public class HBaseFsck extends Configured implements Closeable { setShouldRerun(); RegionInfo hri = hbi.getHdfsHRI(); if (hri == null) { - hri = hbi.metaEntry; + hri = hbi.getMetaEntry(); } HBaseFsckRepair.fixUnassigned(admin, hri); HBaseFsckRepair.waitUntilAssigned(admin, hri); @@ -2186,7 +2143,7 @@ public class HBaseFsck extends Configured implements Closeable { int replicationCount = admin.getTableDescriptor(hri.getTable()).getRegionReplication(); for (int i = 1; i < replicationCount; i++) { hri = RegionReplicaUtil.getRegionInfoForReplica(hri, i); - HbckInfo h = regionInfoMap.get(hri.getEncodedName()); + HbckRegionInfo h = regionInfoMap.get(hri.getEncodedName()); if (h != null) { undeployRegions(h); //set skip checks; we undeploy & deploy it; we don't want to evaluate this hbi anymore @@ -2203,23 +2160,23 @@ public class HBaseFsck extends Configured implements Closeable { /** * Check a single region for consistency and correct deployment. */ - private void checkRegionConsistency(final String key, final HbckInfo hbi) - throws IOException, KeeperException, InterruptedException { + private void checkRegionConsistency(final String key, final HbckRegionInfo hbi) + throws IOException, KeeperException, InterruptedException { if (hbi.isSkipChecks()) return; String descriptiveName = hbi.toString(); - boolean inMeta = hbi.metaEntry != null; + boolean inMeta = hbi.getMetaEntry() != null; // In case not checking HDFS, assume the region is on HDFS boolean inHdfs = !shouldCheckHdfs() || hbi.getHdfsRegionDir() != null; - boolean hasMetaAssignment = inMeta && hbi.metaEntry.regionServer != null; - boolean isDeployed = !hbi.deployedOn.isEmpty(); - boolean isMultiplyDeployed = hbi.deployedOn.size() > 1; + boolean hasMetaAssignment = inMeta && hbi.getMetaEntry().regionServer != null; + boolean isDeployed = !hbi.getDeployedOn().isEmpty(); + boolean isMultiplyDeployed = hbi.getDeployedOn().size() > 1; boolean deploymentMatchesMeta = hasMetaAssignment && isDeployed && !isMultiplyDeployed && - hbi.metaEntry.regionServer.equals(hbi.deployedOn.get(0)); + hbi.getMetaEntry().regionServer.equals(hbi.getDeployedOn().get(0)); boolean splitParent = - inMeta && hbi.metaEntry.isSplit() && hbi.metaEntry.isOffline(); - boolean shouldBeDeployed = inMeta && !isTableDisabled(hbi.metaEntry.getTable()); + inMeta && hbi.getMetaEntry().isSplit() && hbi.getMetaEntry().isOffline(); + boolean shouldBeDeployed = inMeta && !isTableDisabled(hbi.getMetaEntry().getTable()); boolean recentlyModified = inHdfs && hbi.getModTime() + timelag > EnvironmentEdgeManager.currentTime(); @@ -2244,7 +2201,7 @@ public class HBaseFsck extends Configured implements Closeable { } else if (!inMeta && !inHdfs && isDeployed) { errors.reportError(ERROR_CODE.NOT_IN_META_HDFS, "Region " + descriptiveName + ", key=" + key + ", not on HDFS or in hbase:meta but " + - "deployed on " + Joiner.on(", ").join(hbi.deployedOn)); + "deployed on " + Joiner.on(", ").join(hbi.getDeployedOn())); if (shouldFixAssignments()) { undeployRegions(hbi); } @@ -2271,9 +2228,9 @@ public class HBaseFsck extends Configured implements Closeable { } RegionInfo hri = hbi.getHdfsHRI(); - TableInfo tableInfo = tablesInfo.get(hri.getTable()); + HbckTableInfo tableInfo = tablesInfo.get(hri.getTable()); - for (RegionInfo region : tableInfo.getRegionsFromMeta()) { + for (RegionInfo region : tableInfo.getRegionsFromMeta(this.regionInfoMap)) { if (Bytes.compareTo(region.getStartKey(), hri.getStartKey()) <= 0 && (region.getEndKey().length == 0 || Bytes.compareTo(region.getEndKey(), hri.getEndKey()) >= 0) @@ -2310,7 +2267,7 @@ public class HBaseFsck extends Configured implements Closeable { } else if (!inMeta && inHdfs && isDeployed) { errors.reportError(ERROR_CODE.NOT_IN_META, "Region " + descriptiveName - + " not in META, but deployed on " + Joiner.on(", ").join(hbi.deployedOn)); + + " not in META, but deployed on " + Joiner.on(", ").join(hbi.getDeployedOn())); debugLsr(hbi.getHdfsRegionDir()); if (hbi.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) { // for replicas, this means that we should undeploy the region (we would have @@ -2339,10 +2296,10 @@ public class HBaseFsck extends Configured implements Closeable { } else if (inMeta && inHdfs && !isDeployed && splitParent) { // check whether this is an actual error, or just transient state where parent // is not cleaned - if (hbi.metaEntry.splitA != null && hbi.metaEntry.splitB != null) { + if (hbi.getMetaEntry().splitA != null && hbi.getMetaEntry().splitB != null) { // check that split daughters are there - HbckInfo infoA = this.regionInfoMap.get(hbi.metaEntry.splitA.getEncodedName()); - HbckInfo infoB = this.regionInfoMap.get(hbi.metaEntry.splitB.getEncodedName()); + HbckRegionInfo infoA = this.regionInfoMap.get(hbi.getMetaEntry().splitA.getEncodedName()); + HbckRegionInfo infoB = this.regionInfoMap.get(hbi.getMetaEntry().splitB.getEncodedName()); if (infoA != null && infoB != null) { // we already processed or will process daughters. Move on, nothing to see here. hbi.setSkipChecks(true); @@ -2377,7 +2334,7 @@ public class HBaseFsck extends Configured implements Closeable { } else if (inMeta && !inHdfs && isDeployed) { errors.reportError(ERROR_CODE.NOT_IN_HDFS, "Region " + descriptiveName + " found in META, but not in HDFS, " + - "and deployed on " + Joiner.on(", ").join(hbi.deployedOn)); + "and deployed on " + Joiner.on(", ").join(hbi.getDeployedOn())); // We treat HDFS as ground truth. Any information in meta is transient // and equivalent data can be regenerated. So, lets unassign and remove // these problems from META. @@ -2396,33 +2353,33 @@ public class HBaseFsck extends Configured implements Closeable { } else if (inMeta && inHdfs && isDeployed && !shouldBeDeployed) { errors.reportError(ERROR_CODE.SHOULD_NOT_BE_DEPLOYED, "Region " + descriptiveName + " should not be deployed according " + - "to META, but is deployed on " + Joiner.on(", ").join(hbi.deployedOn)); + "to META, but is deployed on " + Joiner.on(", ").join(hbi.getDeployedOn())); if (shouldFixAssignments()) { errors.print("Trying to close the region " + descriptiveName); setShouldRerun(); - HBaseFsckRepair.fixMultiAssignment(connection, hbi.metaEntry, hbi.deployedOn); + HBaseFsckRepair.fixMultiAssignment(connection, hbi.getMetaEntry(), hbi.getDeployedOn()); } } else if (inMeta && inHdfs && isMultiplyDeployed) { errors.reportError(ERROR_CODE.MULTI_DEPLOYED, "Region " + descriptiveName - + " is listed in hbase:meta on region server " + hbi.metaEntry.regionServer + + " is listed in hbase:meta on region server " + hbi.getMetaEntry().regionServer + " but is multiply assigned to region servers " + - Joiner.on(", ").join(hbi.deployedOn)); + Joiner.on(", ").join(hbi.getDeployedOn())); // If we are trying to fix the errors if (shouldFixAssignments()) { errors.print("Trying to fix assignment error..."); setShouldRerun(); - HBaseFsckRepair.fixMultiAssignment(connection, hbi.metaEntry, hbi.deployedOn); + HBaseFsckRepair.fixMultiAssignment(connection, hbi.getMetaEntry(), hbi.getDeployedOn()); } } else if (inMeta && inHdfs && isDeployed && !deploymentMatchesMeta) { errors.reportError(ERROR_CODE.SERVER_DOES_NOT_MATCH_META, "Region " + descriptiveName + " listed in hbase:meta on region server " + - hbi.metaEntry.regionServer + " but found on region server " + - hbi.deployedOn.get(0)); + hbi.getMetaEntry().regionServer + " but found on region server " + + hbi.getDeployedOn().get(0)); // If we are trying to fix the errors if (shouldFixAssignments()) { errors.print("Trying to fix assignment error..."); setShouldRerun(); - HBaseFsckRepair.fixMultiAssignment(connection, hbi.metaEntry, hbi.deployedOn); + HBaseFsckRepair.fixMultiAssignment(connection, hbi.getMetaEntry(), hbi.getDeployedOn()); HBaseFsckRepair.waitUntilAssigned(admin, hbi.getHdfsHRI()); } } else { @@ -2443,12 +2400,12 @@ public class HBaseFsck extends Configured implements Closeable { * repeated or overlapping ones. * @throws IOException */ - SortedMap checkIntegrity() throws IOException { + SortedMap checkIntegrity() throws IOException { tablesInfo = new TreeMap<>(); LOG.debug("There are " + regionInfoMap.size() + " region info entries"); - for (HbckInfo hbi : regionInfoMap.values()) { + for (HbckRegionInfo hbi : regionInfoMap.values()) { // Check only valid, working regions - if (hbi.metaEntry == null) { + if (hbi.getMetaEntry() == null) { // this assumes that consistency check has run loadMetaEntry Path p = hbi.getHdfsRegionDir(); if (p == null) { @@ -2458,11 +2415,11 @@ public class HBaseFsck extends Configured implements Closeable { // TODO test. continue; } - if (hbi.metaEntry.regionServer == null) { + if (hbi.getMetaEntry().regionServer == null) { errors.detail("Skipping region because no region server: " + hbi); continue; } - if (hbi.metaEntry.isOffline()) { + if (hbi.getMetaEntry().isOffline()) { errors.detail("Skipping region because it is offline: " + hbi); continue; } @@ -2476,15 +2433,17 @@ public class HBaseFsck extends Configured implements Closeable { // the region chain in META //if (hbi.foundRegionDir == null) continue; //if (hbi.deployedOn.size() != 1) continue; - if (hbi.deployedOn.isEmpty()) continue; + if (hbi.getDeployedOn().isEmpty()) { + continue; + } // We should be safe here - TableName tableName = hbi.metaEntry.getTable(); - TableInfo modTInfo = tablesInfo.get(tableName); + TableName tableName = hbi.getMetaEntry().getTable(); + HbckTableInfo modTInfo = tablesInfo.get(tableName); if (modTInfo == null) { - modTInfo = new TableInfo(tableName); + modTInfo = new HbckTableInfo(tableName, this); } - for (ServerName server : hbi.deployedOn) { + for (ServerName server : hbi.getDeployedOn()) { modTInfo.addServer(server); } @@ -2498,7 +2457,7 @@ public class HBaseFsck extends Configured implements Closeable { loadTableInfosForTablesWithNoRegion(); logParallelMerge(); - for (TableInfo tInfo : tablesInfo.values()) { + for (HbckTableInfo tInfo : tablesInfo.values()) { TableIntegrityErrorHandler handler = tInfo.new IntegrityFixSuggester(tInfo, errors); if (!tInfo.checkRegionChain(handler)) { errors.report("Found inconsistency in table " + tInfo.getName()); @@ -2519,7 +2478,7 @@ public class HBaseFsck extends Configured implements Closeable { TableName tableName = htd.getTableName(); if (isTableIncluded(tableName) && !tablesInfo.containsKey(tableName)) { - TableInfo tableInfo = new TableInfo(tableName); + HbckTableInfo tableInfo = new HbckTableInfo(tableName, this); tableInfo.htds.add(htd); tablesInfo.put(htd.getTableName(), tableInfo); } @@ -2527,10 +2486,10 @@ public class HBaseFsck extends Configured implements Closeable { } /** - * Merge hdfs data by moving from contained HbckInfo into targetRegionDir. + * Merge hdfs data by moving from contained HbckRegionInfo into targetRegionDir. * @return number of file move fixes done to merge regions. */ - public int mergeRegionDirs(Path targetRegionDir, HbckInfo contained) throws IOException { + public int mergeRegionDirs(Path targetRegionDir, HbckRegionInfo contained) throws IOException { int fileMoves = 0; String thread = Thread.currentThread().getName(); LOG.debug("[" + thread + "] Contained region dir after close and pause"); @@ -2604,9 +2563,10 @@ public class HBaseFsck extends Configured implements Closeable { static class WorkItemOverlapMerge implements Callable { private TableIntegrityErrorHandler handler; - Collection overlapgroup; + Collection overlapgroup; - WorkItemOverlapMerge(Collection overlapgroup, TableIntegrityErrorHandler handler) { + WorkItemOverlapMerge(Collection overlapgroup, + TableIntegrityErrorHandler handler) { this.handler = handler; this.overlapgroup = overlapgroup; } @@ -2618,756 +2578,6 @@ public class HBaseFsck extends Configured implements Closeable { } }; - - /** - * Maintain information about a particular table. - */ - public class TableInfo { - TableName tableName; - TreeSet deployedOn; - - // backwards regions - final List backwards = new ArrayList<>(); - - // sidelined big overlapped regions - final Map sidelinedRegions = new HashMap<>(); - - // region split calculator - final RegionSplitCalculator sc = new RegionSplitCalculator<>(cmp); - - // Histogram of different TableDescriptors found. Ideally there is only one! - final Set htds = new HashSet<>(); - - // key = start split, values = set of splits in problem group - final Multimap overlapGroups = - TreeMultimap.create(RegionSplitCalculator.BYTES_COMPARATOR, cmp); - - // list of regions derived from meta entries. - private ImmutableList regionsFromMeta = null; - - TableInfo(TableName name) { - this.tableName = name; - deployedOn = new TreeSet <>(); - } - - /** - * @return descriptor common to all regions. null if are none or multiple! - */ - private TableDescriptor getHTD() { - if (htds.size() == 1) { - return (TableDescriptor)htds.toArray()[0]; - } else { - LOG.error("None/Multiple table descriptors found for table '" - + tableName + "' regions: " + htds); - } - return null; - } - - public void addRegionInfo(HbckInfo hir) { - if (Bytes.equals(hir.getEndKey(), HConstants.EMPTY_END_ROW)) { - // end key is absolute end key, just add it. - // ignore replicas other than primary for these checks - if (hir.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) sc.add(hir); - return; - } - - // if not the absolute end key, check for cycle - if (Bytes.compareTo(hir.getStartKey(), hir.getEndKey()) > 0) { - errors.reportError( - ERROR_CODE.REGION_CYCLE, - String.format("The endkey for this region comes before the " - + "startkey, startkey=%s, endkey=%s", - Bytes.toStringBinary(hir.getStartKey()), - Bytes.toStringBinary(hir.getEndKey())), this, hir); - backwards.add(hir); - return; - } - - // main case, add to split calculator - // ignore replicas other than primary for these checks - if (hir.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) sc.add(hir); - } - - public void addServer(ServerName server) { - this.deployedOn.add(server); - } - - public TableName getName() { - return tableName; - } - - public int getNumRegions() { - return sc.getStarts().size() + backwards.size(); - } - - public synchronized ImmutableList getRegionsFromMeta() { - // lazy loaded, synchronized to ensure a single load - if (regionsFromMeta == null) { - List regions = new ArrayList<>(); - for (HbckInfo h : HBaseFsck.this.regionInfoMap.values()) { - if (tableName.equals(h.getTableName())) { - if (h.metaEntry != null) { - regions.add(h.metaEntry); - } - } - } - regionsFromMeta = Ordering.from(RegionInfo.COMPARATOR).immutableSortedCopy(regions); - } - - return regionsFromMeta; - } - - private class IntegrityFixSuggester extends TableIntegrityErrorHandlerImpl { - ErrorReporter errors; - - IntegrityFixSuggester(TableInfo ti, ErrorReporter errors) { - this.errors = errors; - setTableInfo(ti); - } - - @Override - public void handleRegionStartKeyNotEmpty(HbckInfo hi) throws IOException{ - errors.reportError(ERROR_CODE.FIRST_REGION_STARTKEY_NOT_EMPTY, - "First region should start with an empty key. You need to " - + " create a new region and regioninfo in HDFS to plug the hole.", - getTableInfo(), hi); - } - - @Override - public void handleRegionEndKeyNotEmpty(byte[] curEndKey) throws IOException { - errors.reportError(ERROR_CODE.LAST_REGION_ENDKEY_NOT_EMPTY, - "Last region should end with an empty key. You need to " - + "create a new region and regioninfo in HDFS to plug the hole.", getTableInfo()); - } - - @Override - public void handleDegenerateRegion(HbckInfo hi) throws IOException{ - errors.reportError(ERROR_CODE.DEGENERATE_REGION, - "Region has the same start and end key.", getTableInfo(), hi); - } - - @Override - public void handleDuplicateStartKeys(HbckInfo r1, HbckInfo r2) throws IOException{ - byte[] key = r1.getStartKey(); - // dup start key - errors.reportError(ERROR_CODE.DUPE_STARTKEYS, - "Multiple regions have the same startkey: " - + Bytes.toStringBinary(key), getTableInfo(), r1); - errors.reportError(ERROR_CODE.DUPE_STARTKEYS, - "Multiple regions have the same startkey: " - + Bytes.toStringBinary(key), getTableInfo(), r2); - } - - @Override - public void handleSplit(HbckInfo r1, HbckInfo r2) throws IOException{ - byte[] key = r1.getStartKey(); - // dup start key - errors.reportError(ERROR_CODE.DUPE_ENDKEYS, - "Multiple regions have the same regionID: " - + Bytes.toStringBinary(key), getTableInfo(), r1); - errors.reportError(ERROR_CODE.DUPE_ENDKEYS, - "Multiple regions have the same regionID: " - + Bytes.toStringBinary(key), getTableInfo(), r2); - } - - @Override - public void handleOverlapInRegionChain(HbckInfo hi1, HbckInfo hi2) throws IOException{ - errors.reportError(ERROR_CODE.OVERLAP_IN_REGION_CHAIN, - "There is an overlap in the region chain.", - getTableInfo(), hi1, hi2); - } - - @Override - public void handleHoleInRegionChain(byte[] holeStart, byte[] holeStop) throws IOException{ - errors.reportError( - ERROR_CODE.HOLE_IN_REGION_CHAIN, - "There is a hole in the region chain between " - + Bytes.toStringBinary(holeStart) + " and " - + Bytes.toStringBinary(holeStop) - + ". You need to create a new .regioninfo and region " - + "dir in hdfs to plug the hole."); - } - }; - - /** - * This handler fixes integrity errors from hdfs information. There are - * basically three classes of integrity problems 1) holes, 2) overlaps, and - * 3) invalid regions. - * - * This class overrides methods that fix holes and the overlap group case. - * Individual cases of particular overlaps are handled by the general - * overlap group merge repair case. - * - * If hbase is online, this forces regions offline before doing merge - * operations. - */ - private class HDFSIntegrityFixer extends IntegrityFixSuggester { - Configuration conf; - - boolean fixOverlaps = true; - - HDFSIntegrityFixer(TableInfo ti, ErrorReporter errors, Configuration conf, - boolean fixHoles, boolean fixOverlaps) { - super(ti, errors); - this.conf = conf; - this.fixOverlaps = fixOverlaps; - // TODO properly use fixHoles - } - - /** - * This is a special case hole -- when the first region of a table is - * missing from META, HBase doesn't acknowledge the existance of the - * table. - */ - @Override - public void handleRegionStartKeyNotEmpty(HbckInfo next) throws IOException { - errors.reportError(ERROR_CODE.FIRST_REGION_STARTKEY_NOT_EMPTY, - "First region should start with an empty key. Creating a new " + - "region and regioninfo in HDFS to plug the hole.", - getTableInfo(), next); - TableDescriptor htd = getTableInfo().getHTD(); - // from special EMPTY_START_ROW to next region's startKey - RegionInfo newRegion = RegionInfoBuilder.newBuilder(htd.getTableName()) - .setStartKey(HConstants.EMPTY_START_ROW) - .setEndKey(next.getStartKey()) - .build(); - - // TODO test - HRegion region = HBaseFsckRepair.createHDFSRegionDir(conf, newRegion, htd); - LOG.info("Table region start key was not empty. Created new empty region: " - + newRegion + " " +region); - fixes++; - } - - @Override - public void handleRegionEndKeyNotEmpty(byte[] curEndKey) throws IOException { - errors.reportError(ERROR_CODE.LAST_REGION_ENDKEY_NOT_EMPTY, - "Last region should end with an empty key. Creating a new " - + "region and regioninfo in HDFS to plug the hole.", getTableInfo()); - TableDescriptor htd = getTableInfo().getHTD(); - // from curEndKey to EMPTY_START_ROW - RegionInfo newRegion = RegionInfoBuilder.newBuilder(htd.getTableName()) - .setStartKey(curEndKey) - .setEndKey(HConstants.EMPTY_START_ROW) - .build(); - - HRegion region = HBaseFsckRepair.createHDFSRegionDir(conf, newRegion, htd); - LOG.info("Table region end key was not empty. Created new empty region: " + newRegion - + " " + region); - fixes++; - } - - /** - * There is a hole in the hdfs regions that violates the table integrity - * rules. Create a new empty region that patches the hole. - */ - @Override - public void handleHoleInRegionChain(byte[] holeStartKey, byte[] holeStopKey) throws IOException { - errors.reportError( - ERROR_CODE.HOLE_IN_REGION_CHAIN, - "There is a hole in the region chain between " - + Bytes.toStringBinary(holeStartKey) + " and " - + Bytes.toStringBinary(holeStopKey) - + ". Creating a new regioninfo and region " - + "dir in hdfs to plug the hole."); - TableDescriptor htd = getTableInfo().getHTD(); - RegionInfo newRegion = RegionInfoBuilder.newBuilder(htd.getTableName()) - .setStartKey(holeStartKey) - .setEndKey(holeStopKey) - .build(); - HRegion region = HBaseFsckRepair.createHDFSRegionDir(conf, newRegion, htd); - LOG.info("Plugged hole by creating new empty region: "+ newRegion + " " +region); - fixes++; - } - - /** - * This takes set of overlapping regions and merges them into a single - * region. This covers cases like degenerate regions, shared start key, - * general overlaps, duplicate ranges, and partial overlapping regions. - * - * Cases: - * - Clean regions that overlap - * - Only .oldlogs regions (can't find start/stop range, or figure out) - * - * This is basically threadsafe, except for the fixer increment in mergeOverlaps. - */ - @Override - public void handleOverlapGroup(Collection overlap) - throws IOException { - Preconditions.checkNotNull(overlap); - Preconditions.checkArgument(overlap.size() >0); - - if (!this.fixOverlaps) { - LOG.warn("Not attempting to repair overlaps."); - return; - } - - if (overlap.size() > maxMerge) { - LOG.warn("Overlap group has " + overlap.size() + " overlapping " + - "regions which is greater than " + maxMerge + ", the max number of regions to merge"); - if (sidelineBigOverlaps) { - // we only sideline big overlapped groups that exceeds the max number of regions to merge - sidelineBigOverlaps(overlap); - } - return; - } - if (shouldRemoveParents()) { - removeParentsAndFixSplits(overlap); - } - mergeOverlaps(overlap); - } - - void removeParentsAndFixSplits(Collection overlap) throws IOException { - Pair range = null; - HbckInfo parent = null; - HbckInfo daughterA = null; - HbckInfo daughterB = null; - Collection daughters = new ArrayList(overlap); - - String thread = Thread.currentThread().getName(); - LOG.info("== [" + thread + "] Attempting fix splits in overlap state."); - - // we only can handle a single split per group at the time - if (overlap.size() > 3) { - LOG.info("Too many overlaps were found on this group, falling back to regular merge."); - return; - } - - for (HbckInfo hi : overlap) { - if (range == null) { - range = new Pair(hi.getStartKey(), hi.getEndKey()); - } else { - if (RegionSplitCalculator.BYTES_COMPARATOR - .compare(hi.getStartKey(), range.getFirst()) < 0) { - range.setFirst(hi.getStartKey()); - } - if (RegionSplitCalculator.BYTES_COMPARATOR - .compare(hi.getEndKey(), range.getSecond()) > 0) { - range.setSecond(hi.getEndKey()); - } - } - } - - LOG.info("This group range is [" + Bytes.toStringBinary(range.getFirst()) + ", " - + Bytes.toStringBinary(range.getSecond()) + "]"); - - // attempt to find a possible parent for the edge case of a split - for (HbckInfo hi : overlap) { - if (Bytes.compareTo(hi.getHdfsHRI().getStartKey(), range.getFirst()) == 0 - && Bytes.compareTo(hi.getHdfsHRI().getEndKey(), range.getSecond()) == 0) { - LOG.info("This is a parent for this group: " + hi.toString()); - parent = hi; - } - } - - // Remove parent regions from daughters collection - if (parent != null) { - daughters.remove(parent); - } - - // Lets verify that daughters share the regionID at split time and they - // were created after the parent - for (HbckInfo hi : daughters) { - if (Bytes.compareTo(hi.getHdfsHRI().getStartKey(), range.getFirst()) == 0) { - if (parent.getHdfsHRI().getRegionId() < hi.getHdfsHRI().getRegionId()) { - daughterA = hi; - } - } - if (Bytes.compareTo(hi.getHdfsHRI().getEndKey(), range.getSecond()) == 0) { - if (parent.getHdfsHRI().getRegionId() < hi.getHdfsHRI().getRegionId()) { - daughterB = hi; - } - } - } - - // daughters must share the same regionID and we should have a parent too - if (daughterA.getHdfsHRI().getRegionId() != daughterB.getHdfsHRI().getRegionId() || parent == null) - return; - - FileSystem fs = FileSystem.get(conf); - LOG.info("Found parent: " + parent.getRegionNameAsString()); - LOG.info("Found potential daughter a: " + daughterA.getRegionNameAsString()); - LOG.info("Found potential daughter b: " + daughterB.getRegionNameAsString()); - LOG.info("Trying to fix parent in overlap by removing the parent."); - try { - closeRegion(parent); - } catch (IOException ioe) { - LOG.warn("Parent region could not be closed, continuing with regular merge...", ioe); - return; - } catch (InterruptedException ie) { - LOG.warn("Parent region could not be closed, continuing with regular merge...", ie); - return; - } - - try { - offline(parent.getRegionName()); - } catch (IOException ioe) { - LOG.warn("Unable to offline parent region: " + parent.getRegionNameAsString() - + ". Just continuing with regular merge... ", ioe); - return; - } - - try { - HBaseFsckRepair.removeParentInMeta(conf, parent.getHdfsHRI()); - } catch (IOException ioe) { - LOG.warn("Unable to remove parent region in META: " + parent.getRegionNameAsString() - + ". Just continuing with regular merge... ", ioe); - return; - } - - sidelineRegionDir(fs, parent); - LOG.info("[" + thread + "] Sidelined parent region dir "+ parent.getHdfsRegionDir() + " into " + - getSidelineDir()); - debugLsr(parent.getHdfsRegionDir()); - - // Make sure we don't have the parents and daughters around - overlap.remove(parent); - overlap.remove(daughterA); - overlap.remove(daughterB); - - LOG.info("Done fixing split."); - - } - - void mergeOverlaps(Collection overlap) - throws IOException { - String thread = Thread.currentThread().getName(); - LOG.info("== [" + thread + "] Merging regions into one region: " - + Joiner.on(",").join(overlap)); - // get the min / max range and close all concerned regions - Pair range = null; - for (HbckInfo hi : overlap) { - if (range == null) { - range = new Pair<>(hi.getStartKey(), hi.getEndKey()); - } else { - if (RegionSplitCalculator.BYTES_COMPARATOR - .compare(hi.getStartKey(), range.getFirst()) < 0) { - range.setFirst(hi.getStartKey()); - } - if (RegionSplitCalculator.BYTES_COMPARATOR - .compare(hi.getEndKey(), range.getSecond()) > 0) { - range.setSecond(hi.getEndKey()); - } - } - // need to close files so delete can happen. - LOG.debug("[" + thread + "] Closing region before moving data around: " + hi); - LOG.debug("[" + thread + "] Contained region dir before close"); - debugLsr(hi.getHdfsRegionDir()); - try { - LOG.info("[" + thread + "] Closing region: " + hi); - closeRegion(hi); - } catch (IOException ioe) { - LOG.warn("[" + thread + "] Was unable to close region " + hi - + ". Just continuing... ", ioe); - } catch (InterruptedException e) { - LOG.warn("[" + thread + "] Was unable to close region " + hi - + ". Just continuing... ", e); - } - - try { - LOG.info("[" + thread + "] Offlining region: " + hi); - offline(hi.getRegionName()); - } catch (IOException ioe) { - LOG.warn("[" + thread + "] Unable to offline region from master: " + hi - + ". Just continuing... ", ioe); - } - } - - // create new empty container region. - TableDescriptor htd = getTableInfo().getHTD(); - // from start key to end Key - RegionInfo newRegion = RegionInfoBuilder.newBuilder(htd.getTableName()) - .setStartKey(range.getFirst()) - .setEndKey(range.getSecond()) - .build(); - HRegion region = HBaseFsckRepair.createHDFSRegionDir(conf, newRegion, htd); - LOG.info("[" + thread + "] Created new empty container region: " + - newRegion + " to contain regions: " + Joiner.on(",").join(overlap)); - debugLsr(region.getRegionFileSystem().getRegionDir()); - - // all target regions are closed, should be able to safely cleanup. - boolean didFix= false; - Path target = region.getRegionFileSystem().getRegionDir(); - for (HbckInfo contained : overlap) { - LOG.info("[" + thread + "] Merging " + contained + " into " + target ); - int merges = mergeRegionDirs(target, contained); - if (merges > 0) { - didFix = true; - } - } - if (didFix) { - fixes++; - } - } - - /** - * Sideline some regions in a big overlap group so that it - * will have fewer regions, and it is easier to merge them later on. - * - * @param bigOverlap the overlapped group with regions more than maxMerge - * @throws IOException - */ - void sidelineBigOverlaps( - Collection bigOverlap) throws IOException { - int overlapsToSideline = bigOverlap.size() - maxMerge; - if (overlapsToSideline > maxOverlapsToSideline) { - overlapsToSideline = maxOverlapsToSideline; - } - List regionsToSideline = - RegionSplitCalculator.findBigRanges(bigOverlap, overlapsToSideline); - FileSystem fs = FileSystem.get(conf); - for (HbckInfo regionToSideline: regionsToSideline) { - try { - LOG.info("Closing region: " + regionToSideline); - closeRegion(regionToSideline); - } catch (IOException ioe) { - LOG.warn("Was unable to close region " + regionToSideline - + ". Just continuing... ", ioe); - } catch (InterruptedException e) { - LOG.warn("Was unable to close region " + regionToSideline - + ". Just continuing... ", e); - } - - try { - LOG.info("Offlining region: " + regionToSideline); - offline(regionToSideline.getRegionName()); - } catch (IOException ioe) { - LOG.warn("Unable to offline region from master: " + regionToSideline - + ". Just continuing... ", ioe); - } - - LOG.info("Before sideline big overlapped region: " + regionToSideline.toString()); - Path sidelineRegionDir = sidelineRegionDir(fs, TO_BE_LOADED, regionToSideline); - if (sidelineRegionDir != null) { - sidelinedRegions.put(sidelineRegionDir, regionToSideline); - LOG.info("After sidelined big overlapped region: " - + regionToSideline.getRegionNameAsString() - + " to " + sidelineRegionDir.toString()); - fixes++; - } - } - } - } - - /** - * Check the region chain (from META) of this table. We are looking for - * holes, overlaps, and cycles. - * @return false if there are errors - * @throws IOException - */ - public boolean checkRegionChain(TableIntegrityErrorHandler handler) throws IOException { - // When table is disabled no need to check for the region chain. Some of the regions - // accidently if deployed, this below code might report some issues like missing start - // or end regions or region hole in chain and may try to fix which is unwanted. - if (isTableDisabled(this.tableName)) { - return true; - } - int originalErrorsCount = errors.getErrorList().size(); - Multimap regions = sc.calcCoverage(); - SortedSet splits = sc.getSplits(); - - byte[] prevKey = null; - byte[] problemKey = null; - - if (splits.isEmpty()) { - // no region for this table - handler.handleHoleInRegionChain(HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); - } - - for (byte[] key : splits) { - Collection ranges = regions.get(key); - if (prevKey == null && !Bytes.equals(key, HConstants.EMPTY_BYTE_ARRAY)) { - for (HbckInfo rng : ranges) { - handler.handleRegionStartKeyNotEmpty(rng); - } - } - - // check for degenerate ranges - for (HbckInfo rng : ranges) { - // special endkey case converts '' to null - byte[] endKey = rng.getEndKey(); - endKey = (endKey.length == 0) ? null : endKey; - if (Bytes.equals(rng.getStartKey(),endKey)) { - handler.handleDegenerateRegion(rng); - } - } - - if (ranges.size() == 1) { - // this split key is ok -- no overlap, not a hole. - if (problemKey != null) { - LOG.warn("reached end of problem group: " + Bytes.toStringBinary(key)); - } - problemKey = null; // fell through, no more problem. - } else if (ranges.size() > 1) { - // set the new problem key group name, if already have problem key, just - // keep using it. - if (problemKey == null) { - // only for overlap regions. - LOG.warn("Naming new problem group: " + Bytes.toStringBinary(key)); - problemKey = key; - } - overlapGroups.putAll(problemKey, ranges); - - // record errors - ArrayList subRange = new ArrayList<>(ranges); - // this dumb and n^2 but this shouldn't happen often - for (HbckInfo r1 : ranges) { - if (r1.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) continue; - subRange.remove(r1); - for (HbckInfo r2 : subRange) { - if (r2.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) continue; - // general case of same start key - if (Bytes.compareTo(r1.getStartKey(), r2.getStartKey())==0) { - handler.handleDuplicateStartKeys(r1,r2); - } else if (Bytes.compareTo(r1.getEndKey(), r2.getStartKey())==0 && - r1.getHdfsHRI().getRegionId() == r2.getHdfsHRI().getRegionId()) { - LOG.info("this is a split, log to splits"); - handler.handleSplit(r1, r2); - } else { - // overlap - handler.handleOverlapInRegionChain(r1, r2); - } - } - } - - } else if (ranges.isEmpty()) { - if (problemKey != null) { - LOG.warn("reached end of problem group: " + Bytes.toStringBinary(key)); - } - problemKey = null; - - byte[] holeStopKey = sc.getSplits().higher(key); - // if higher key is null we reached the top. - if (holeStopKey != null) { - // hole - handler.handleHoleInRegionChain(key, holeStopKey); - } - } - prevKey = key; - } - - // When the last region of a table is proper and having an empty end key, 'prevKey' - // will be null. - if (prevKey != null) { - handler.handleRegionEndKeyNotEmpty(prevKey); - } - - // TODO fold this into the TableIntegrityHandler - if (getConf().getBoolean("hbasefsck.overlap.merge.parallel", true)) { - boolean ok = handleOverlapsParallel(handler, prevKey); - if (!ok) { - return false; - } - } else { - for (Collection overlap : overlapGroups.asMap().values()) { - handler.handleOverlapGroup(overlap); - } - } - - if (details) { - // do full region split map dump - errors.print("---- Table '" + this.tableName - + "': region split map"); - dump(splits, regions); - errors.print("---- Table '" + this.tableName - + "': overlap groups"); - dumpOverlapProblems(overlapGroups); - errors.print("There are " + overlapGroups.keySet().size() - + " overlap groups with " + overlapGroups.size() - + " overlapping regions"); - } - if (!sidelinedRegions.isEmpty()) { - LOG.warn("Sidelined big overlapped regions, please bulk load them!"); - errors.print("---- Table '" + this.tableName - + "': sidelined big overlapped regions"); - dumpSidelinedRegions(sidelinedRegions); - } - return errors.getErrorList().size() == originalErrorsCount; - } - - private boolean handleOverlapsParallel(TableIntegrityErrorHandler handler, byte[] prevKey) - throws IOException { - // we parallelize overlap handler for the case we have lots of groups to fix. We can - // safely assume each group is independent. - List merges = new ArrayList<>(overlapGroups.size()); - List> rets; - for (Collection overlap : overlapGroups.asMap().values()) { - // - merges.add(new WorkItemOverlapMerge(overlap, handler)); - } - try { - rets = executor.invokeAll(merges); - } catch (InterruptedException e) { - LOG.error("Overlap merges were interrupted", e); - return false; - } - for(int i=0; i f = rets.get(i); - try { - f.get(); - } catch(ExecutionException e) { - LOG.warn("Failed to merge overlap group" + work, e.getCause()); - } catch (InterruptedException e) { - LOG.error("Waiting for overlap merges was interrupted", e); - return false; - } - } - return true; - } - - /** - * This dumps data in a visually reasonable way for visual debugging - * - * @param splits - * @param regions - */ - void dump(SortedSet splits, Multimap regions) { - // we display this way because the last end key should be displayed as well. - StringBuilder sb = new StringBuilder(); - for (byte[] k : splits) { - sb.setLength(0); // clear out existing buffer, if any. - sb.append(Bytes.toStringBinary(k) + ":\t"); - for (HbckInfo r : regions.get(k)) { - sb.append("[ "+ r.toString() + ", " - + Bytes.toStringBinary(r.getEndKey())+ "]\t"); - } - errors.print(sb.toString()); - } - } - } - - public void dumpOverlapProblems(Multimap regions) { - // we display this way because the last end key should be displayed as - // well. - for (byte[] k : regions.keySet()) { - errors.print(Bytes.toStringBinary(k) + ":"); - for (HbckInfo r : regions.get(k)) { - errors.print("[ " + r.toString() + ", " - + Bytes.toStringBinary(r.getEndKey()) + "]"); - } - errors.print("----"); - } - } - - public void dumpSidelinedRegions(Map regions) { - for (Map.Entry entry: regions.entrySet()) { - TableName tableName = entry.getValue().getTableName(); - Path path = entry.getKey(); - errors.print("This sidelined region dir should be bulk loaded: " - + path.toString()); - errors.print("Bulk load command looks like: " - + "hbase org.apache.hadoop.hbase.tool.LoadIncrementalHFiles " - + path.toUri().getPath() + " "+ tableName); - } - } - - public Multimap getOverlapGroups( - TableName table) { - TableInfo ti = tablesInfo.get(table); - return ti.overlapGroups; - } - /** * Return a list of user-space table names whose metadata have not been * modified in the last few milliseconds specified by timelag @@ -3381,8 +2591,8 @@ public class HBaseFsck extends Configured implements Closeable { List tableNames = new ArrayList<>(); long now = EnvironmentEdgeManager.currentTime(); - for (HbckInfo hbi : regionInfoMap.values()) { - MetaEntry info = hbi.metaEntry; + for (HbckRegionInfo hbi : regionInfoMap.values()) { + HbckRegionInfo.MetaEntry info = hbi.getMetaEntry(); // if the start key is zero, then we have found the first region of a table. // pick only those tables that were not modified in the last few milliseconds. @@ -3414,10 +2624,10 @@ public class HBaseFsck extends Configured implements Closeable { * region name. If the region has not been seen yet, a new entry is added * and returned. */ - private synchronized HbckInfo getOrCreateInfo(String name) { - HbckInfo hbi = regionInfoMap.get(name); + private synchronized HbckRegionInfo getOrCreateInfo(String name) { + HbckRegionInfo hbi = regionInfoMap.get(name); if (hbi == null) { - hbi = new HbckInfo(null); + hbi = new HbckRegionInfo(null); regionInfoMap.put(name, hbi); } return hbi; @@ -3443,9 +2653,9 @@ public class HBaseFsck extends Configured implements Closeable { * @throws InterruptedException */ boolean checkMetaRegion() throws IOException, KeeperException, InterruptedException { - Map metaRegions = new HashMap<>(); - for (HbckInfo value : regionInfoMap.values()) { - if (value.metaEntry != null && value.metaEntry.isMetaRegion()) { + Map metaRegions = new HashMap<>(); + for (HbckRegionInfo value : regionInfoMap.values()) { + if (value.getMetaEntry() != null && value.getMetaEntry().isMetaRegion()) { metaRegions.put(value.getReplicaId(), value); } } @@ -3455,10 +2665,10 @@ public class HBaseFsck extends Configured implements Closeable { // There will be always entries in regionInfoMap corresponding to hbase:meta & its replicas // Check the deployed servers. It should be exactly one server for each replica. for (int i = 0; i < metaReplication; i++) { - HbckInfo metaHbckInfo = metaRegions.remove(i); + HbckRegionInfo metaHbckRegionInfo = metaRegions.remove(i); List servers = new ArrayList<>(); - if (metaHbckInfo != null) { - servers = metaHbckInfo.deployedOn; + if (metaHbckRegionInfo != null) { + servers = metaHbckRegionInfo.getDeployedOn(); } if (servers.size() != 1) { noProblem = false; @@ -3467,19 +2677,20 @@ public class HBaseFsck extends Configured implements Closeable { } else if (servers.size() > 1) { errors .reportError(ERROR_CODE.MULTI_META_REGION, "hbase:meta, replicaId " + - metaHbckInfo.getReplicaId() + " is found on more than one region."); + metaHbckRegionInfo.getReplicaId() + " is found on more than one region."); if (shouldFixAssignments()) { errors.print("Trying to fix a problem with hbase:meta, replicaId " + - metaHbckInfo.getReplicaId() +".."); + metaHbckRegionInfo.getReplicaId() + ".."); setShouldRerun(); // try fix it (treat is a dupe assignment) - HBaseFsckRepair.fixMultiAssignment(connection, metaHbckInfo.metaEntry, servers); + HBaseFsckRepair + .fixMultiAssignment(connection, metaHbckRegionInfo.getMetaEntry(), servers); } } } } // unassign whatever is remaining in metaRegions. They are excess replicas. - for (Map.Entry entry : metaRegions.entrySet()) { + for (Map.Entry entry : metaRegions.entrySet()) { noProblem = false; errors.reportError(ERROR_CODE.SHOULD_NOT_BE_DEPLOYED, "hbase:meta replicas are deployed in excess. Configured " + metaReplication + @@ -3496,10 +2707,11 @@ public class HBaseFsck extends Configured implements Closeable { return noProblem; } - private void unassignMetaReplica(HbckInfo hi) throws IOException, InterruptedException, - KeeperException { + private void unassignMetaReplica(HbckRegionInfo hi) + throws IOException, InterruptedException, KeeperException { undeployRegions(hi); - ZKUtil.deleteNode(zkw, zkw.getZNodePaths().getZNodeForReplica(hi.metaEntry.getReplicaId())); + ZKUtil + .deleteNode(zkw, zkw.getZNodePaths().getZNodeForReplica(hi.getMetaEntry().getReplicaId())); } private void assignMetaReplica(int replicaId) @@ -3567,17 +2779,18 @@ public class HBaseFsck extends Configured implements Closeable { sn = h.getServerName(); hri = h.getRegionInfo(); - MetaEntry m = null; + HbckRegionInfo.MetaEntry m = null; if (hri.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) { - m = new MetaEntry(hri, sn, ts, daughters.getFirst(), daughters.getSecond()); + m = new HbckRegionInfo.MetaEntry(hri, sn, ts, daughters.getFirst(), + daughters.getSecond()); } else { - m = new MetaEntry(hri, sn, ts, null, null); + m = new HbckRegionInfo.MetaEntry(hri, sn, ts, null, null); } - HbckInfo previous = regionInfoMap.get(hri.getEncodedName()); + HbckRegionInfo previous = regionInfoMap.get(hri.getEncodedName()); if (previous == null) { - regionInfoMap.put(hri.getEncodedName(), new HbckInfo(m)); - } else if (previous.metaEntry == null) { - previous.metaEntry = m; + regionInfoMap.put(hri.getEncodedName(), new HbckRegionInfo(m)); + } else if (previous.getMetaEntry() == null) { + previous.setMetaEntry(m); } else { throw new IOException("Two entries in hbase:meta are same " + previous); } @@ -3587,7 +2800,7 @@ public class HBaseFsck extends Configured implements Closeable { mergeRegions.getFirst(), mergeRegions.getSecond() }) { if (mergeRegion != null) { // This region is already been merged - HbckInfo hbInfo = getOrCreateInfo(mergeRegion.getEncodedName()); + HbckRegionInfo hbInfo = getOrCreateInfo(mergeRegion.getEncodedName()); hbInfo.setMerged(true); } } @@ -3613,310 +2826,14 @@ public class HBaseFsck extends Configured implements Closeable { return true; } - /** - * Stores the regioninfo entries scanned from META - */ - static class MetaEntry extends HRegionInfo { - ServerName regionServer; // server hosting this region - long modTime; // timestamp of most recent modification metadata - RegionInfo splitA, splitB; //split daughters - - public MetaEntry(RegionInfo rinfo, ServerName regionServer, long modTime) { - this(rinfo, regionServer, modTime, null, null); - } - - public MetaEntry(RegionInfo rinfo, ServerName regionServer, long modTime, - RegionInfo splitA, RegionInfo splitB) { - super(rinfo); - this.regionServer = regionServer; - this.modTime = modTime; - this.splitA = splitA; - this.splitB = splitB; - } - - @Override - public boolean equals(Object o) { - boolean superEq = super.equals(o); - if (!superEq) { - return superEq; - } - - MetaEntry me = (MetaEntry) o; - if (!regionServer.equals(me.regionServer)) { - return false; - } - return (modTime == me.modTime); - } - - @Override - public int hashCode() { - int hash = Arrays.hashCode(getRegionName()); - hash = (int) (hash ^ getRegionId()); - hash ^= Arrays.hashCode(getStartKey()); - hash ^= Arrays.hashCode(getEndKey()); - hash ^= Boolean.valueOf(isOffline()).hashCode(); - hash ^= getTable().hashCode(); - if (regionServer != null) { - hash ^= regionServer.hashCode(); - } - hash = (int) (hash ^ modTime); - return hash; - } - } - - /** - * Stores the regioninfo entries from HDFS - */ - static class HdfsEntry { - RegionInfo hri; - Path hdfsRegionDir = null; - long hdfsRegionDirModTime = 0; - boolean hdfsRegioninfoFilePresent = false; - boolean hdfsOnlyEdits = false; - } - - /** - * Stores the regioninfo retrieved from Online region servers. - */ - static class OnlineEntry { - RegionInfo hri; - ServerName hsa; - - @Override - public String toString() { - return hsa.toString() + ";" + hri.getRegionNameAsString(); - } - } - - /** - * Maintain information about a particular region. It gathers information - * from three places -- HDFS, META, and region servers. - */ - public static class HbckInfo implements KeyRange { - private MetaEntry metaEntry = null; // info in META - private HdfsEntry hdfsEntry = null; // info in HDFS - private List deployedEntries = Lists.newArrayList(); // on Region Server - private List deployedOn = Lists.newArrayList(); // info on RS's - private boolean skipChecks = false; // whether to skip further checks to this region info. - private boolean isMerged = false;// whether this region has already been merged into another one - private int deployedReplicaId = RegionInfo.DEFAULT_REPLICA_ID; - private RegionInfo primaryHRIForDeployedReplica = null; - - HbckInfo(MetaEntry metaEntry) { - this.metaEntry = metaEntry; - } - - public synchronized int getReplicaId() { - return metaEntry != null? metaEntry.getReplicaId(): deployedReplicaId; - } - - public synchronized void addServer(RegionInfo hri, ServerName server) { - OnlineEntry rse = new OnlineEntry() ; - rse.hri = hri; - rse.hsa = server; - this.deployedEntries.add(rse); - this.deployedOn.add(server); - // save the replicaId that we see deployed in the cluster - this.deployedReplicaId = hri.getReplicaId(); - this.primaryHRIForDeployedReplica = - RegionReplicaUtil.getRegionInfoForDefaultReplica(hri); - } - - @Override - public synchronized String toString() { - StringBuilder sb = new StringBuilder(); - sb.append("{ meta => "); - sb.append((metaEntry != null)? metaEntry.getRegionNameAsString() : "null"); - sb.append( ", hdfs => " + getHdfsRegionDir()); - sb.append( ", deployed => " + Joiner.on(", ").join(deployedEntries)); - sb.append( ", replicaId => " + getReplicaId()); - sb.append(" }"); - return sb.toString(); - } - - @Override - public byte[] getStartKey() { - if (this.metaEntry != null) { - return this.metaEntry.getStartKey(); - } else if (this.hdfsEntry != null) { - return this.hdfsEntry.hri.getStartKey(); - } else { - LOG.error("Entry " + this + " has no meta or hdfs region start key."); - return null; - } - } - - @Override - public byte[] getEndKey() { - if (this.metaEntry != null) { - return this.metaEntry.getEndKey(); - } else if (this.hdfsEntry != null) { - return this.hdfsEntry.hri.getEndKey(); - } else { - LOG.error("Entry " + this + " has no meta or hdfs region start key."); - return null; - } - } - - public TableName getTableName() { - if (this.metaEntry != null) { - return this.metaEntry.getTable(); - } else if (this.hdfsEntry != null) { - // we are only guaranteed to have a path and not an HRI for hdfsEntry, - // so we get the name from the Path - Path tableDir = this.hdfsEntry.hdfsRegionDir.getParent(); - return FSUtils.getTableName(tableDir); - } else { - // return the info from the first online/deployed hri - for (OnlineEntry e : deployedEntries) { - return e.hri.getTable(); - } - return null; - } - } - - public String getRegionNameAsString() { - if (metaEntry != null) { - return metaEntry.getRegionNameAsString(); - } else if (hdfsEntry != null) { - if (hdfsEntry.hri != null) { - return hdfsEntry.hri.getRegionNameAsString(); - } - } else { - // return the info from the first online/deployed hri - for (OnlineEntry e : deployedEntries) { - return e.hri.getRegionNameAsString(); - } - } - return null; - } - - public byte[] getRegionName() { - if (metaEntry != null) { - return metaEntry.getRegionName(); - } else if (hdfsEntry != null) { - return hdfsEntry.hri.getRegionName(); - } else { - // return the info from the first online/deployed hri - for (OnlineEntry e : deployedEntries) { - return e.hri.getRegionName(); - } - return null; - } - } - - public RegionInfo getPrimaryHRIForDeployedReplica() { - return primaryHRIForDeployedReplica; - } - - Path getHdfsRegionDir() { - if (hdfsEntry == null) { - return null; - } - return hdfsEntry.hdfsRegionDir; - } - - boolean containsOnlyHdfsEdits() { - if (hdfsEntry == null) { - return false; - } - return hdfsEntry.hdfsOnlyEdits; - } - - boolean isHdfsRegioninfoPresent() { - if (hdfsEntry == null) { - return false; - } - return hdfsEntry.hdfsRegioninfoFilePresent; - } - - long getModTime() { - if (hdfsEntry == null) { - return 0; - } - return hdfsEntry.hdfsRegionDirModTime; - } - - RegionInfo getHdfsHRI() { - if (hdfsEntry == null) { - return null; - } - return hdfsEntry.hri; - } - - public void setSkipChecks(boolean skipChecks) { - this.skipChecks = skipChecks; - } - - public boolean isSkipChecks() { - return skipChecks; - } - - public void setMerged(boolean isMerged) { - this.isMerged = isMerged; - } - - public boolean isMerged() { - return this.isMerged; - } - } - - final static Comparator cmp = new Comparator() { - @Override - public int compare(HbckInfo l, HbckInfo r) { - if (l == r) { - // same instance - return 0; - } - - int tableCompare = l.getTableName().compareTo(r.getTableName()); - if (tableCompare != 0) { - return tableCompare; - } - - int startComparison = RegionSplitCalculator.BYTES_COMPARATOR.compare( - l.getStartKey(), r.getStartKey()); - if (startComparison != 0) { - return startComparison; - } - - // Special case for absolute endkey - byte[] endKey = r.getEndKey(); - endKey = (endKey.length == 0) ? null : endKey; - byte[] endKey2 = l.getEndKey(); - endKey2 = (endKey2.length == 0) ? null : endKey2; - int endComparison = RegionSplitCalculator.BYTES_COMPARATOR.compare( - endKey2, endKey); - - if (endComparison != 0) { - return endComparison; - } - - // use regionId as tiebreaker. - // Null is considered after all possible values so make it bigger. - if (l.hdfsEntry == null && r.hdfsEntry == null) { - return 0; - } - if (l.hdfsEntry == null && r.hdfsEntry != null) { - return 1; - } - // l.hdfsEntry must not be null - if (r.hdfsEntry == null) { - return -1; - } - // both l.hdfsEntry and r.hdfsEntry must not be null. - return Long.compare(l.hdfsEntry.hri.getRegionId(), r.hdfsEntry.hri.getRegionId()); - } - }; - /** * Prints summary of all tables found on the system. */ - private void printTableSummary(SortedMap tablesInfo) { + private void printTableSummary(SortedMap tablesInfo) { StringBuilder sb = new StringBuilder(); int numOfSkippedRegions; errors.print("Summary:"); - for (TableInfo tInfo : tablesInfo.values()) { + for (HbckTableInfo tInfo : tablesInfo.values()) { numOfSkippedRegions = (skippedRegions.containsKey(tInfo.getName())) ? skippedRegions.get(tInfo.getName()).size() : 0; @@ -3947,53 +2864,21 @@ public class HBaseFsck extends Configured implements Closeable { } } - static ErrorReporter getErrorReporter( - final Configuration conf) throws ClassNotFoundException { - Class reporter = conf.getClass("hbasefsck.errorreporter", PrintingErrorReporter.class, ErrorReporter.class); + static HbckErrorReporter getErrorReporter(final Configuration conf) + throws ClassNotFoundException { + Class reporter = + conf.getClass("hbasefsck.errorreporter", PrintingErrorReporter.class, + HbckErrorReporter.class); return ReflectionUtils.newInstance(reporter, conf); } - public interface ErrorReporter { - enum ERROR_CODE { - UNKNOWN, NO_META_REGION, NULL_META_REGION, NO_VERSION_FILE, NOT_IN_META_HDFS, NOT_IN_META, - NOT_IN_META_OR_DEPLOYED, NOT_IN_HDFS_OR_DEPLOYED, NOT_IN_HDFS, SERVER_DOES_NOT_MATCH_META, - NOT_DEPLOYED, MULTI_DEPLOYED, SHOULD_NOT_BE_DEPLOYED, MULTI_META_REGION, RS_CONNECT_FAILURE, - FIRST_REGION_STARTKEY_NOT_EMPTY, LAST_REGION_ENDKEY_NOT_EMPTY, DUPE_STARTKEYS, - HOLE_IN_REGION_CHAIN, OVERLAP_IN_REGION_CHAIN, REGION_CYCLE, DEGENERATE_REGION, - ORPHAN_HDFS_REGION, LINGERING_SPLIT_PARENT, NO_TABLEINFO_FILE, LINGERING_REFERENCE_HFILE, - LINGERING_HFILELINK, WRONG_USAGE, EMPTY_META_CELL, EXPIRED_TABLE_LOCK, BOUNDARIES_ERROR, - ORPHAN_TABLE_STATE, NO_TABLE_STATE, UNDELETED_REPLICATION_QUEUE, DUPE_ENDKEYS, - UNSUPPORTED_OPTION, INVALID_TABLE - } - void clear(); - void report(String message); - void reportError(String message); - void reportError(ERROR_CODE errorCode, String message); - void reportError(ERROR_CODE errorCode, String message, TableInfo table); - void reportError(ERROR_CODE errorCode, String message, TableInfo table, HbckInfo info); - void reportError( - ERROR_CODE errorCode, - String message, - TableInfo table, - HbckInfo info1, - HbckInfo info2 - ); - int summarize(); - void detail(String details); - ArrayList getErrorList(); - void progress(); - void print(String message); - void resetErrors(); - boolean tableHasErrors(TableInfo table); - } - - static class PrintingErrorReporter implements ErrorReporter { + static class PrintingErrorReporter implements HbckErrorReporter { public int errorCount = 0; private int showProgress; // How frequently calls to progress() will create output private static final int progressThreshold = 100; - Set errorTables = new HashSet<>(); + Set errorTables = new HashSet<>(); // for use by unit tests to verify which errors were discovered private ArrayList errorList = new ArrayList<>(); @@ -4021,22 +2906,23 @@ public class HBaseFsck extends Configured implements Closeable { } @Override - public synchronized void reportError(ERROR_CODE errorCode, String message, TableInfo table) { + public synchronized void reportError(ERROR_CODE errorCode, String message, + HbckTableInfo table) { errorTables.add(table); reportError(errorCode, message); } @Override - public synchronized void reportError(ERROR_CODE errorCode, String message, TableInfo table, - HbckInfo info) { + public synchronized void reportError(ERROR_CODE errorCode, String message, HbckTableInfo table, + HbckRegionInfo info) { errorTables.add(table); String reference = "(region " + info.getRegionNameAsString() + ")"; reportError(errorCode, reference + " " + message); } @Override - public synchronized void reportError(ERROR_CODE errorCode, String message, TableInfo table, - HbckInfo info1, HbckInfo info2) { + public synchronized void reportError(ERROR_CODE errorCode, String message, HbckTableInfo table, + HbckRegionInfo info1, HbckRegionInfo info2) { errorTables.add(table); String reference = "(regions " + info1.getRegionNameAsString() + " and " + info2.getRegionNameAsString() + ")"; @@ -4087,7 +2973,7 @@ public class HBaseFsck extends Configured implements Closeable { } @Override - public boolean tableHasErrors(TableInfo table) { + public boolean tableHasErrors(HbckTableInfo table) { return errorTables.contains(table); } @@ -4121,11 +3007,11 @@ public class HBaseFsck extends Configured implements Closeable { static class WorkItemRegion implements Callable { private final HBaseFsck hbck; private final ServerName rsinfo; - private final ErrorReporter errors; + private final HbckErrorReporter errors; private final ClusterConnection connection; - WorkItemRegion(HBaseFsck hbck, ServerName info, - ErrorReporter errors, ClusterConnection connection) { + WorkItemRegion(HBaseFsck hbck, ServerName info, HbckErrorReporter errors, + ClusterConnection connection) { this.hbck = hbck; this.rsinfo = info; this.errors = errors; @@ -4155,8 +3041,9 @@ public class HBaseFsck extends Configured implements Closeable { } // check to see if the existence of this region matches the region in META - for (RegionInfo r:regions) { - HbckInfo hbi = hbck.getOrCreateInfo(r.getEncodedName()); + + for (RegionInfo r : regions) { + HbckRegionInfo hbi = hbck.getOrCreateInfo(r.getEncodedName()); hbi.addServer(r, rsinfo); } } catch (IOException e) { // unable to connect to the region server. @@ -4185,11 +3072,10 @@ public class HBaseFsck extends Configured implements Closeable { */ class WorkItemHdfsDir implements Callable { private FileStatus tableDir; - private ErrorReporter errors; + private HbckErrorReporter errors; private FileSystem fs; - WorkItemHdfsDir(FileSystem fs, ErrorReporter errors, - FileStatus status) { + WorkItemHdfsDir(FileSystem fs, HbckErrorReporter errors, FileStatus status) { this.fs = fs; this.tableDir = status; this.errors = errors; @@ -4234,16 +3120,16 @@ public class HBaseFsck extends Configured implements Closeable { } } - HbckInfo hbi = HBaseFsck.this.getOrCreateInfo(encodedName); - HdfsEntry he = new HdfsEntry(); + HbckRegionInfo hbi = HBaseFsck.this.getOrCreateInfo(encodedName); + HbckRegionInfo.HdfsEntry he = new HbckRegionInfo.HdfsEntry(); synchronized (hbi) { if (hbi.getHdfsRegionDir() != null) { errors.print("Directory " + encodedName + " duplicate??" + hbi.getHdfsRegionDir()); } - he.hdfsRegionDir = regionDir.getPath(); - he.hdfsRegionDirModTime = regionDir.getModificationTime(); + he.regionDir = regionDir.getPath(); + he.regionDirModTime = regionDir.getModificationTime(); he.hdfsRegioninfoFilePresent = regioninfoFileExists; // we add to orphan list when we attempt to read .regioninfo @@ -4260,7 +3146,7 @@ public class HBaseFsck extends Configured implements Closeable { break; } } - hbi.hdfsEntry = he; + hbi.setHdfsEntry(he); } } catch (Exception e) { LOG.error("Could not load region dir", e); @@ -4304,11 +3190,11 @@ public class HBaseFsck extends Configured implements Closeable { * regioninfo list. */ static class WorkItemHdfsRegionInfo implements Callable { - private HbckInfo hbi; + private HbckRegionInfo hbi; private HBaseFsck hbck; - private ErrorReporter errors; + private HbckErrorReporter errors; - WorkItemHdfsRegionInfo(HbckInfo hbi, HBaseFsck hbck, ErrorReporter errors) { + WorkItemHdfsRegionInfo(HbckRegionInfo hbi, HBaseFsck hbck, HbckErrorReporter errors) { this.hbi = hbi; this.hbck = hbck; this.errors = errors; @@ -4320,7 +3206,7 @@ public class HBaseFsck extends Configured implements Closeable { if (hbi.getHdfsHRI() == null) { try { errors.progress(); - hbck.loadHdfsRegioninfo(hbi); + hbi.loadHdfsRegioninfo(hbck.getConf()); } catch (IOException ioe) { String msg = "Orphan region in HDFS: Unable to load .regioninfo from table " + hbi.getTableName() + " in hdfs dir " @@ -4350,6 +3236,10 @@ public class HBaseFsck extends Configured implements Closeable { details = true; } + public static boolean shouldDisplayFullReport() { + return details; + } + /** * Set exclusive mode. */ @@ -5079,7 +3969,7 @@ public class HBaseFsck extends Configured implements Closeable { * ls -r for debugging purposes */ public static void debugLsr(Configuration conf, - Path p, ErrorReporter errors) throws IOException { + Path p, HbckErrorReporter errors) throws IOException { if (!LOG.isDebugEnabled() || p == null) { return; } @@ -5102,4 +3992,4 @@ public class HBaseFsck extends Configured implements Closeable { } } } -} +} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HbckErrorReporter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HbckErrorReporter.java new file mode 100644 index 00000000000..500b690051b --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HbckErrorReporter.java @@ -0,0 +1,69 @@ +/** + * 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.util; + +import java.util.ArrayList; + +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.yetus.audience.InterfaceStability; + +@InterfaceAudience.Private +@InterfaceStability.Evolving +public interface HbckErrorReporter { + + enum ERROR_CODE { + UNKNOWN, NO_META_REGION, NULL_META_REGION, NO_VERSION_FILE, NOT_IN_META_HDFS, NOT_IN_META, + NOT_IN_META_OR_DEPLOYED, NOT_IN_HDFS_OR_DEPLOYED, NOT_IN_HDFS, SERVER_DOES_NOT_MATCH_META, + NOT_DEPLOYED, MULTI_DEPLOYED, SHOULD_NOT_BE_DEPLOYED, MULTI_META_REGION, RS_CONNECT_FAILURE, + FIRST_REGION_STARTKEY_NOT_EMPTY, LAST_REGION_ENDKEY_NOT_EMPTY, DUPE_STARTKEYS, + HOLE_IN_REGION_CHAIN, OVERLAP_IN_REGION_CHAIN, REGION_CYCLE, DEGENERATE_REGION, + ORPHAN_HDFS_REGION, LINGERING_SPLIT_PARENT, NO_TABLEINFO_FILE, LINGERING_REFERENCE_HFILE, + LINGERING_HFILELINK, WRONG_USAGE, EMPTY_META_CELL, EXPIRED_TABLE_LOCK, BOUNDARIES_ERROR, + ORPHAN_TABLE_STATE, NO_TABLE_STATE, UNDELETED_REPLICATION_QUEUE, DUPE_ENDKEYS, + UNSUPPORTED_OPTION, INVALID_TABLE + } + + void clear(); + + void report(String message); + + void reportError(String message); + + void reportError(ERROR_CODE errorCode, String message); + + void reportError(ERROR_CODE errorCode, String message, HbckTableInfo table); + + void reportError(ERROR_CODE errorCode, String message, HbckTableInfo table, HbckRegionInfo info); + + void reportError(ERROR_CODE errorCode, String message, HbckTableInfo table, HbckRegionInfo info1, + HbckRegionInfo info2); + + int summarize(); + + void detail(String details); + + ArrayList getErrorList(); + + void progress(); + + void print(String message); + + void resetErrors(); + + boolean tableHasErrors(HbckTableInfo table); +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HbckRegionInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HbckRegionInfo.java new file mode 100644 index 00000000000..6204071f69f --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HbckRegionInfo.java @@ -0,0 +1,412 @@ +/** + * 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.util; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionReplicaUtil; +import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.yetus.audience.InterfaceStability; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.base.Joiner; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + +/** + * Maintain information about a particular region. It gathers information + * from three places -- HDFS, META, and region servers. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public class HbckRegionInfo implements KeyRange { + private static final Logger LOG = LoggerFactory.getLogger(HbckRegionInfo.class.getName()); + + private MetaEntry metaEntry = null; // info in META + private HdfsEntry hdfsEntry = null; // info in HDFS + private List deployedEntries = Lists.newArrayList(); // on Region Server + private List deployedOn = Lists.newArrayList(); // info on RS's + private boolean skipChecks = false; // whether to skip further checks to this region info. + private boolean isMerged = false;// whether this region has already been merged into another one + private int deployedReplicaId = RegionInfo.DEFAULT_REPLICA_ID; + private RegionInfo primaryHRIForDeployedReplica = null; + + public HbckRegionInfo(MetaEntry metaEntry) { + this.metaEntry = metaEntry; + } + + public synchronized int getReplicaId() { + return metaEntry != null? metaEntry.getReplicaId(): deployedReplicaId; + } + + public synchronized void addServer(RegionInfo regionInfo, ServerName serverName) { + OnlineEntry rse = new OnlineEntry(regionInfo, serverName) ; + this.deployedEntries.add(rse); + this.deployedOn.add(serverName); + // save the replicaId that we see deployed in the cluster + this.deployedReplicaId = regionInfo.getReplicaId(); + this.primaryHRIForDeployedReplica = + RegionReplicaUtil.getRegionInfoForDefaultReplica(regionInfo); + } + + @Override + public synchronized String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("{ meta => "); + sb.append((metaEntry != null)? metaEntry.getRegionNameAsString() : "null"); + sb.append(", hdfs => " + getHdfsRegionDir()); + sb.append(", deployed => " + Joiner.on(", ").join(deployedEntries)); + sb.append(", replicaId => " + getReplicaId()); + sb.append(" }"); + return sb.toString(); + } + + @Override + public byte[] getStartKey() { + if (this.metaEntry != null) { + return this.metaEntry.getStartKey(); + } else if (this.hdfsEntry != null) { + return this.hdfsEntry.hri.getStartKey(); + } else { + LOG.error("Entry " + this + " has no meta or hdfs region start key."); + return null; + } + } + + @Override + public byte[] getEndKey() { + if (this.metaEntry != null) { + return this.metaEntry.getEndKey(); + } else if (this.hdfsEntry != null) { + return this.hdfsEntry.hri.getEndKey(); + } else { + LOG.error("Entry " + this + " has no meta or hdfs region start key."); + return null; + } + } + + public MetaEntry getMetaEntry() { + return this.metaEntry; + } + + public void setMetaEntry(MetaEntry metaEntry) { + this.metaEntry = metaEntry; + } + + public HdfsEntry getHdfsEntry() { + return this.hdfsEntry; + } + + public void setHdfsEntry(HdfsEntry hdfsEntry) { + this.hdfsEntry = hdfsEntry; + } + + public List getOnlineEntries() { + return this.deployedEntries; + } + + public List getDeployedOn() { + return this.deployedOn; + } + + /** + * Read the .regioninfo file from the file system. If there is no + * .regioninfo, add it to the orphan hdfs region list. + */ + public void loadHdfsRegioninfo(Configuration conf) throws IOException { + Path regionDir = getHdfsRegionDir(); + if (regionDir == null) { + if (getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) { + // Log warning only for default/ primary replica with no region dir + LOG.warn("No HDFS region dir found: " + this + " meta=" + metaEntry); + } + return; + } + + if (hdfsEntry.hri != null) { + // already loaded data + return; + } + + FileSystem fs = FileSystem.get(conf); + RegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir); + LOG.debug("RegionInfo read: " + hri.toString()); + hdfsEntry.hri = hri; + } + + public TableName getTableName() { + if (this.metaEntry != null) { + return this.metaEntry.getTable(); + } else if (this.hdfsEntry != null) { + // we are only guaranteed to have a path and not an HRI for hdfsEntry, + // so we get the name from the Path + Path tableDir = this.hdfsEntry.regionDir.getParent(); + return FSUtils.getTableName(tableDir); + } else { + // return the info from the first online/deployed hri + for (OnlineEntry e : deployedEntries) { + return e.getRegionInfo().getTable(); + } + return null; + } + } + + public String getRegionNameAsString() { + if (metaEntry != null) { + return metaEntry.getRegionNameAsString(); + } else if (hdfsEntry != null) { + if (hdfsEntry.hri != null) { + return hdfsEntry.hri.getRegionNameAsString(); + } + } else { + // return the info from the first online/deployed hri + for (OnlineEntry e : deployedEntries) { + return e.getRegionInfo().getRegionNameAsString(); + } + } + return null; + } + + public byte[] getRegionName() { + if (metaEntry != null) { + return metaEntry.getRegionName(); + } else if (hdfsEntry != null) { + return hdfsEntry.hri.getRegionName(); + } else { + // return the info from the first online/deployed hri + for (OnlineEntry e : deployedEntries) { + return e.getRegionInfo().getRegionName(); + } + return null; + } + } + + public RegionInfo getPrimaryHRIForDeployedReplica() { + return primaryHRIForDeployedReplica; + } + + public Path getHdfsRegionDir() { + if (hdfsEntry == null) { + return null; + } + return hdfsEntry.regionDir; + } + + public boolean containsOnlyHdfsEdits() { + if (hdfsEntry == null) { + return false; + } + return hdfsEntry.hdfsOnlyEdits; + } + + public boolean isHdfsRegioninfoPresent() { + if (hdfsEntry == null) { + return false; + } + return hdfsEntry.hdfsRegioninfoFilePresent; + } + + public long getModTime() { + if (hdfsEntry == null) { + return 0; + } + return hdfsEntry.regionDirModTime; + } + + public RegionInfo getHdfsHRI() { + if (hdfsEntry == null) { + return null; + } + return hdfsEntry.hri; + } + + public void setSkipChecks(boolean skipChecks) { + this.skipChecks = skipChecks; + } + + public boolean isSkipChecks() { + return skipChecks; + } + + public void setMerged(boolean isMerged) { + this.isMerged = isMerged; + } + + public boolean isMerged() { + return this.isMerged; + } + + /** + * Stores the regioninfo entries scanned from META + */ + public static class MetaEntry extends HRegionInfo { + ServerName regionServer; // server hosting this region + long modTime; // timestamp of most recent modification metadata + RegionInfo splitA, splitB; //split daughters + + public MetaEntry(RegionInfo rinfo, ServerName regionServer, long modTime) { + this(rinfo, regionServer, modTime, null, null); + } + + public MetaEntry(RegionInfo rinfo, ServerName regionServer, long modTime, + RegionInfo splitA, RegionInfo splitB) { + super(rinfo); + this.regionServer = regionServer; + this.modTime = modTime; + this.splitA = splitA; + this.splitB = splitB; + } + + public ServerName getRegionServer() { + return this.regionServer; + } + + @Override + public boolean equals(Object o) { + boolean superEq = super.equals(o); + if (!superEq) { + return superEq; + } + + MetaEntry me = (MetaEntry) o; + if (!regionServer.equals(me.regionServer)) { + return false; + } + return (modTime == me.modTime); + } + + @Override + public int hashCode() { + int hash = Arrays.hashCode(getRegionName()); + hash = (int) (hash ^ getRegionId()); + hash ^= Arrays.hashCode(getStartKey()); + hash ^= Arrays.hashCode(getEndKey()); + hash ^= Boolean.valueOf(isOffline()).hashCode(); + hash ^= getTable().hashCode(); + if (regionServer != null) { + hash ^= regionServer.hashCode(); + } + hash = (int) (hash ^ modTime); + return hash; + } + } + + /** + * Stores the regioninfo entries from HDFS + */ + public static class HdfsEntry { + RegionInfo hri; + Path regionDir = null; + long regionDirModTime = 0; + boolean hdfsRegioninfoFilePresent = false; + boolean hdfsOnlyEdits = false; + + HdfsEntry() { + } + + public HdfsEntry(Path regionDir, long regionDirModTime) { + this.regionDir = regionDir; + this.regionDirModTime = regionDirModTime; + } + } + + /** + * Stores the regioninfo retrieved from Online region servers. + */ + static class OnlineEntry { + private RegionInfo regionInfo; + private ServerName serverName; + + OnlineEntry(RegionInfo regionInfo, ServerName serverName) { + this.regionInfo = regionInfo; + this.serverName = serverName; + } + + public RegionInfo getRegionInfo() { + return regionInfo; + } + + public ServerName getServerName() { + return serverName; + } + + @Override + public String toString() { + return serverName.toString() + ";" + regionInfo.getRegionNameAsString(); + } + } + + final static Comparator COMPARATOR = new Comparator() { + @Override + public int compare(HbckRegionInfo l, HbckRegionInfo r) { + if (l == r) { + // same instance + return 0; + } + + int tableCompare = l.getTableName().compareTo(r.getTableName()); + if (tableCompare != 0) { + return tableCompare; + } + + int startComparison = RegionSplitCalculator.BYTES_COMPARATOR.compare( + l.getStartKey(), r.getStartKey()); + if (startComparison != 0) { + return startComparison; + } + + // Special case for absolute endkey + byte[] endKey = r.getEndKey(); + endKey = (endKey.length == 0) ? null : endKey; + byte[] endKey2 = l.getEndKey(); + endKey2 = (endKey2.length == 0) ? null : endKey2; + int endComparison = RegionSplitCalculator.BYTES_COMPARATOR.compare( + endKey2, endKey); + + if (endComparison != 0) { + return endComparison; + } + + // use regionId as tiebreaker. + // Null is considered after all possible values so make it bigger. + if (l.getHdfsEntry() == null && r.getHdfsEntry() == null) { + return 0; + } + if (l.getHdfsEntry() == null && r.getHdfsEntry() != null) { + return 1; + } + // l.hdfsEntry must not be null + if (r.getHdfsEntry() == null) { + return -1; + } + // both l.hdfsEntry and r.hdfsEntry must not be null. + return Long.compare(l.getHdfsEntry().hri.getRegionId(), r.getHdfsEntry().hri.getRegionId()); + } + }; +} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HbckTableInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HbckTableInfo.java new file mode 100644 index 00000000000..7cedd2e8d9c --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HbckTableInfo.java @@ -0,0 +1,810 @@ +/* + * 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.util; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.ServerName; +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.TableDescriptor; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.tool.BulkLoadHFilesTool; +import org.apache.hadoop.hbase.util.hbck.TableIntegrityErrorHandler; +import org.apache.hadoop.hbase.util.hbck.TableIntegrityErrorHandlerImpl; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.yetus.audience.InterfaceStability; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.base.Joiner; +import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; +import org.apache.hbase.thirdparty.com.google.common.collect.Multimap; +import org.apache.hbase.thirdparty.com.google.common.collect.Ordering; +import org.apache.hbase.thirdparty.com.google.common.collect.TreeMultimap; + +/** + * Maintain information about a particular table. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public class HbckTableInfo { + private static final Logger LOG = LoggerFactory.getLogger(HbckTableInfo.class.getName()); + + private static final String TO_BE_LOADED = "to_be_loaded"; + + TableName tableName; + TreeSet deployedOn; + + // backwards regions + final List backwards = new ArrayList<>(); + + // sidelined big overlapped regions + final Map sidelinedRegions = new HashMap<>(); + + // region split calculator + final RegionSplitCalculator sc = + new RegionSplitCalculator<>(HbckRegionInfo.COMPARATOR); + + // Histogram of different TableDescriptors found. Ideally there is only one! + final Set htds = new HashSet<>(); + + // key = start split, values = set of splits in problem group + final Multimap overlapGroups = + TreeMultimap.create(RegionSplitCalculator.BYTES_COMPARATOR, HbckRegionInfo.COMPARATOR); + + // list of regions derived from meta entries. + private ImmutableList regionsFromMeta = null; + + HBaseFsck hbck; + + HbckTableInfo(TableName name, HBaseFsck hbck) { + this.tableName = name; + this.hbck = hbck; + deployedOn = new TreeSet<>(); + } + + /** + * @return descriptor common to all regions. null if are none or multiple! + */ + TableDescriptor getTableDescriptor() { + if (htds.size() == 1) { + return (TableDescriptor)htds.toArray()[0]; + } else { + LOG.error("None/Multiple table descriptors found for table '" + + tableName + "' regions: " + htds); + } + return null; + } + + public void addRegionInfo(HbckRegionInfo hir) { + if (Bytes.equals(hir.getEndKey(), HConstants.EMPTY_END_ROW)) { + // end key is absolute end key, just add it. + // ignore replicas other than primary for these checks + if (hir.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) { + sc.add(hir); + } + return; + } + + // if not the absolute end key, check for cycle + if (Bytes.compareTo(hir.getStartKey(), hir.getEndKey()) > 0) { + hbck.getErrors().reportError(HbckErrorReporter.ERROR_CODE.REGION_CYCLE, String.format( + "The endkey for this region comes before the " + "startkey, startkey=%s, endkey=%s", + Bytes.toStringBinary(hir.getStartKey()), Bytes.toStringBinary(hir.getEndKey())), this, + hir); + backwards.add(hir); + return; + } + + // main case, add to split calculator + // ignore replicas other than primary for these checks + if (hir.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) { + sc.add(hir); + } + } + + public void addServer(ServerName server) { + this.deployedOn.add(server); + } + + public TableName getName() { + return tableName; + } + + public int getNumRegions() { + return sc.getStarts().size() + backwards.size(); + } + + public synchronized ImmutableList getRegionsFromMeta( + TreeMap regionInfoMap) { + // lazy loaded, synchronized to ensure a single load + if (regionsFromMeta == null) { + List regions = new ArrayList<>(); + for (HbckRegionInfo h : regionInfoMap.values()) { + if (tableName.equals(h.getTableName())) { + if (h.getMetaEntry() != null) { + regions.add(h.getMetaEntry()); + } + } + } + regionsFromMeta = Ordering.from(RegionInfo.COMPARATOR).immutableSortedCopy(regions); + } + + return regionsFromMeta; + } + + class IntegrityFixSuggester extends TableIntegrityErrorHandlerImpl { + HbckErrorReporter errors; + + IntegrityFixSuggester(HbckTableInfo ti, HbckErrorReporter errors) { + this.errors = errors; + setTableInfo(ti); + } + + @Override + public void handleRegionStartKeyNotEmpty(HbckRegionInfo hi) throws IOException { + errors.reportError(HbckErrorReporter.ERROR_CODE.FIRST_REGION_STARTKEY_NOT_EMPTY, + "First region should start with an empty key. You need to " + + " create a new region and regioninfo in HDFS to plug the hole.", + getTableInfo(), hi); + } + + @Override + public void handleRegionEndKeyNotEmpty(byte[] curEndKey) throws IOException { + errors.reportError(HbckErrorReporter.ERROR_CODE.LAST_REGION_ENDKEY_NOT_EMPTY, + "Last region should end with an empty key. You need to " + + "create a new region and regioninfo in HDFS to plug the hole.", getTableInfo()); + } + + @Override + public void handleDegenerateRegion(HbckRegionInfo hi) throws IOException{ + errors.reportError(HbckErrorReporter.ERROR_CODE.DEGENERATE_REGION, + "Region has the same start and end key.", getTableInfo(), hi); + } + + @Override + public void handleDuplicateStartKeys(HbckRegionInfo r1, HbckRegionInfo r2) throws IOException { + byte[] key = r1.getStartKey(); + // dup start key + errors.reportError(HbckErrorReporter.ERROR_CODE.DUPE_STARTKEYS, + "Multiple regions have the same startkey: " + Bytes.toStringBinary(key), getTableInfo(), + r1); + errors.reportError(HbckErrorReporter.ERROR_CODE.DUPE_STARTKEYS, + "Multiple regions have the same startkey: " + Bytes.toStringBinary(key), getTableInfo(), + r2); + } + + @Override + public void handleSplit(HbckRegionInfo r1, HbckRegionInfo r2) throws IOException{ + byte[] key = r1.getStartKey(); + // dup start key + errors.reportError(HbckErrorReporter.ERROR_CODE.DUPE_ENDKEYS, + "Multiple regions have the same regionID: " + + Bytes.toStringBinary(key), getTableInfo(), r1); + errors.reportError(HbckErrorReporter.ERROR_CODE.DUPE_ENDKEYS, + "Multiple regions have the same regionID: " + + Bytes.toStringBinary(key), getTableInfo(), r2); + } + + @Override + public void handleOverlapInRegionChain(HbckRegionInfo hi1, HbckRegionInfo hi2) + throws IOException { + errors.reportError(HbckErrorReporter.ERROR_CODE.OVERLAP_IN_REGION_CHAIN, + "There is an overlap in the region chain.", getTableInfo(), hi1, hi2); + } + + @Override + public void handleHoleInRegionChain(byte[] holeStart, byte[] holeStop) throws IOException { + errors.reportError( + HbckErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN, + "There is a hole in the region chain between " + + Bytes.toStringBinary(holeStart) + " and " + + Bytes.toStringBinary(holeStop) + + ". You need to create a new .regioninfo and region " + + "dir in hdfs to plug the hole."); + } + } + + /** + * This handler fixes integrity errors from hdfs information. There are + * basically three classes of integrity problems 1) holes, 2) overlaps, and + * 3) invalid regions. + * + * This class overrides methods that fix holes and the overlap group case. + * Individual cases of particular overlaps are handled by the general + * overlap group merge repair case. + * + * If hbase is online, this forces regions offline before doing merge + * operations. + */ + class HDFSIntegrityFixer extends IntegrityFixSuggester { + Configuration conf; + + boolean fixOverlaps = true; + + HDFSIntegrityFixer(HbckTableInfo ti, HbckErrorReporter errors, Configuration conf, + boolean fixHoles, boolean fixOverlaps) { + super(ti, errors); + this.conf = conf; + this.fixOverlaps = fixOverlaps; + // TODO properly use fixHoles + } + + /** + * This is a special case hole -- when the first region of a table is + * missing from META, HBase doesn't acknowledge the existance of the + * table. + */ + @Override + public void handleRegionStartKeyNotEmpty(HbckRegionInfo next) throws IOException { + errors.reportError(HbckErrorReporter.ERROR_CODE.FIRST_REGION_STARTKEY_NOT_EMPTY, + "First region should start with an empty key. Creating a new " + + "region and regioninfo in HDFS to plug the hole.", + getTableInfo(), next); + TableDescriptor htd = getTableInfo().getTableDescriptor(); + // from special EMPTY_START_ROW to next region's startKey + RegionInfo newRegion = RegionInfoBuilder.newBuilder(htd.getTableName()) + .setStartKey(HConstants.EMPTY_START_ROW) + .setEndKey(next.getStartKey()) + .build(); + + // TODO test + HRegion region = HBaseFsckRepair.createHDFSRegionDir(conf, newRegion, htd); + LOG.info("Table region start key was not empty. Created new empty region: " + + newRegion + " " +region); + hbck.fixes++; + } + + @Override + public void handleRegionEndKeyNotEmpty(byte[] curEndKey) throws IOException { + errors.reportError(HbckErrorReporter.ERROR_CODE.LAST_REGION_ENDKEY_NOT_EMPTY, + "Last region should end with an empty key. Creating a new " + + "region and regioninfo in HDFS to plug the hole.", getTableInfo()); + TableDescriptor htd = getTableInfo().getTableDescriptor(); + // from curEndKey to EMPTY_START_ROW + RegionInfo newRegion = RegionInfoBuilder.newBuilder(htd.getTableName()) + .setStartKey(curEndKey) + .setEndKey(HConstants.EMPTY_START_ROW) + .build(); + + HRegion region = HBaseFsckRepair.createHDFSRegionDir(conf, newRegion, htd); + LOG.info("Table region end key was not empty. Created new empty region: " + newRegion + + " " + region); + hbck.fixes++; + } + + /** + * There is a hole in the hdfs regions that violates the table integrity + * rules. Create a new empty region that patches the hole. + */ + @Override + public void handleHoleInRegionChain(byte[] holeStartKey, byte[] holeStopKey) + throws IOException { + errors.reportError(HbckErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN, + "There is a hole in the region chain between " + Bytes.toStringBinary(holeStartKey) + + " and " + Bytes.toStringBinary(holeStopKey) + + ". Creating a new regioninfo and region " + "dir in hdfs to plug the hole."); + TableDescriptor htd = getTableInfo().getTableDescriptor(); + RegionInfo newRegion = + RegionInfoBuilder.newBuilder(htd.getTableName()).setStartKey(holeStartKey) + .setEndKey(holeStopKey).build(); + HRegion region = HBaseFsckRepair.createHDFSRegionDir(conf, newRegion, htd); + LOG.info("Plugged hole by creating new empty region: " + newRegion + " " + region); + hbck.fixes++; + } + + /** + * This takes set of overlapping regions and merges them into a single + * region. This covers cases like degenerate regions, shared start key, + * general overlaps, duplicate ranges, and partial overlapping regions. + * + * Cases: + * - Clean regions that overlap + * - Only .oldlogs regions (can't find start/stop range, or figure out) + * + * This is basically threadsafe, except for the fixer increment in mergeOverlaps. + */ + @Override + public void handleOverlapGroup(Collection overlap) + throws IOException { + Preconditions.checkNotNull(overlap); + Preconditions.checkArgument(overlap.size() >0); + + if (!this.fixOverlaps) { + LOG.warn("Not attempting to repair overlaps."); + return; + } + + if (overlap.size() > hbck.getMaxMerge()) { + LOG.warn("Overlap group has " + overlap.size() + " overlapping " + + "regions which is greater than " + hbck.getMaxMerge() + + ", the max number of regions to merge"); + if (hbck.shouldSidelineBigOverlaps()) { + // we only sideline big overlapped groups that exceeds the max number of regions to merge + sidelineBigOverlaps(overlap); + } + return; + } + if (hbck.shouldRemoveParents()) { + removeParentsAndFixSplits(overlap); + } + mergeOverlaps(overlap); + } + + void removeParentsAndFixSplits(Collection overlap) throws IOException { + Pair range = null; + HbckRegionInfo parent = null; + HbckRegionInfo daughterA = null; + HbckRegionInfo daughterB = null; + Collection daughters = new ArrayList(overlap); + + String thread = Thread.currentThread().getName(); + LOG.info("== [" + thread + "] Attempting fix splits in overlap state."); + + // we only can handle a single split per group at the time + if (overlap.size() > 3) { + LOG.info("Too many overlaps were found on this group, falling back to regular merge."); + return; + } + + for (HbckRegionInfo hi : overlap) { + if (range == null) { + range = new Pair(hi.getStartKey(), hi.getEndKey()); + } else { + if (RegionSplitCalculator.BYTES_COMPARATOR + .compare(hi.getStartKey(), range.getFirst()) < 0) { + range.setFirst(hi.getStartKey()); + } + if (RegionSplitCalculator.BYTES_COMPARATOR + .compare(hi.getEndKey(), range.getSecond()) > 0) { + range.setSecond(hi.getEndKey()); + } + } + } + + LOG.info("This group range is [" + Bytes.toStringBinary(range.getFirst()) + ", " + + Bytes.toStringBinary(range.getSecond()) + "]"); + + // attempt to find a possible parent for the edge case of a split + for (HbckRegionInfo hi : overlap) { + if (Bytes.compareTo(hi.getHdfsHRI().getStartKey(), range.getFirst()) == 0 + && Bytes.compareTo(hi.getHdfsHRI().getEndKey(), range.getSecond()) == 0) { + LOG.info("This is a parent for this group: " + hi.toString()); + parent = hi; + } + } + + // Remove parent regions from daughters collection + if (parent != null) { + daughters.remove(parent); + } + + // Lets verify that daughters share the regionID at split time and they + // were created after the parent + for (HbckRegionInfo hi : daughters) { + if (Bytes.compareTo(hi.getHdfsHRI().getStartKey(), range.getFirst()) == 0) { + if (parent.getHdfsHRI().getRegionId() < hi.getHdfsHRI().getRegionId()) { + daughterA = hi; + } + } + if (Bytes.compareTo(hi.getHdfsHRI().getEndKey(), range.getSecond()) == 0) { + if (parent.getHdfsHRI().getRegionId() < hi.getHdfsHRI().getRegionId()) { + daughterB = hi; + } + } + } + + // daughters must share the same regionID and we should have a parent too + if (daughterA.getHdfsHRI().getRegionId() != daughterB.getHdfsHRI().getRegionId() || + parent == null) { + return; + } + + FileSystem fs = FileSystem.get(conf); + LOG.info("Found parent: " + parent.getRegionNameAsString()); + LOG.info("Found potential daughter a: " + daughterA.getRegionNameAsString()); + LOG.info("Found potential daughter b: " + daughterB.getRegionNameAsString()); + LOG.info("Trying to fix parent in overlap by removing the parent."); + try { + hbck.closeRegion(parent); + } catch (IOException ioe) { + LOG.warn("Parent region could not be closed, continuing with regular merge...", ioe); + return; + } catch (InterruptedException ie) { + LOG.warn("Parent region could not be closed, continuing with regular merge...", ie); + return; + } + + try { + hbck.offline(parent.getRegionName()); + } catch (IOException ioe) { + LOG.warn("Unable to offline parent region: " + parent.getRegionNameAsString() + + ". Just continuing with regular merge... ", ioe); + return; + } + + try { + HBaseFsckRepair.removeParentInMeta(conf, parent.getHdfsHRI()); + } catch (IOException ioe) { + LOG.warn("Unable to remove parent region in META: " + parent.getRegionNameAsString() + + ". Just continuing with regular merge... ", ioe); + return; + } + + hbck.sidelineRegionDir(fs, parent); + LOG.info( + "[" + thread + "] Sidelined parent region dir " + parent.getHdfsRegionDir() + " into " + + hbck.getSidelineDir()); + hbck.debugLsr(parent.getHdfsRegionDir()); + + // Make sure we don't have the parents and daughters around + overlap.remove(parent); + overlap.remove(daughterA); + overlap.remove(daughterB); + + LOG.info("Done fixing split."); + + } + + void mergeOverlaps(Collection overlap) + throws IOException { + String thread = Thread.currentThread().getName(); + LOG.info("== [" + thread + "] Merging regions into one region: " + + Joiner.on(",").join(overlap)); + // get the min / max range and close all concerned regions + Pair range = null; + for (HbckRegionInfo hi : overlap) { + if (range == null) { + range = new Pair<>(hi.getStartKey(), hi.getEndKey()); + } else { + if (RegionSplitCalculator.BYTES_COMPARATOR + .compare(hi.getStartKey(), range.getFirst()) < 0) { + range.setFirst(hi.getStartKey()); + } + if (RegionSplitCalculator.BYTES_COMPARATOR + .compare(hi.getEndKey(), range.getSecond()) > 0) { + range.setSecond(hi.getEndKey()); + } + } + // need to close files so delete can happen. + LOG.debug("[" + thread + "] Closing region before moving data around: " + hi); + LOG.debug("[" + thread + "] Contained region dir before close"); + hbck.debugLsr(hi.getHdfsRegionDir()); + try { + LOG.info("[" + thread + "] Closing region: " + hi); + hbck.closeRegion(hi); + } catch (IOException ioe) { + LOG.warn("[" + thread + "] Was unable to close region " + hi + + ". Just continuing... ", ioe); + } catch (InterruptedException e) { + LOG.warn("[" + thread + "] Was unable to close region " + hi + + ". Just continuing... ", e); + } + + try { + LOG.info("[" + thread + "] Offlining region: " + hi); + hbck.offline(hi.getRegionName()); + } catch (IOException ioe) { + LOG.warn("[" + thread + "] Unable to offline region from master: " + hi + + ". Just continuing... ", ioe); + } + } + + // create new empty container region. + TableDescriptor htd = getTableInfo().getTableDescriptor(); + // from start key to end Key + RegionInfo newRegion = RegionInfoBuilder.newBuilder(htd.getTableName()) + .setStartKey(range.getFirst()) + .setEndKey(range.getSecond()) + .build(); + HRegion region = HBaseFsckRepair.createHDFSRegionDir(conf, newRegion, htd); + LOG.info("[" + thread + "] Created new empty container region: " + + newRegion + " to contain regions: " + Joiner.on(",").join(overlap)); + hbck.debugLsr(region.getRegionFileSystem().getRegionDir()); + + // all target regions are closed, should be able to safely cleanup. + boolean didFix= false; + Path target = region.getRegionFileSystem().getRegionDir(); + for (HbckRegionInfo contained : overlap) { + LOG.info("[" + thread + "] Merging " + contained + " into " + target); + int merges = hbck.mergeRegionDirs(target, contained); + if (merges > 0) { + didFix = true; + } + } + if (didFix) { + hbck.fixes++; + } + } + + /** + * Sideline some regions in a big overlap group so that it + * will have fewer regions, and it is easier to merge them later on. + * + * @param bigOverlap the overlapped group with regions more than maxMerge + */ + void sidelineBigOverlaps(Collection bigOverlap) throws IOException { + int overlapsToSideline = bigOverlap.size() - hbck.getMaxMerge(); + if (overlapsToSideline > hbck.getMaxOverlapsToSideline()) { + overlapsToSideline = hbck.getMaxOverlapsToSideline(); + } + List regionsToSideline = + RegionSplitCalculator.findBigRanges(bigOverlap, overlapsToSideline); + FileSystem fs = FileSystem.get(conf); + for (HbckRegionInfo regionToSideline: regionsToSideline) { + try { + LOG.info("Closing region: " + regionToSideline); + hbck.closeRegion(regionToSideline); + } catch (IOException ioe) { + LOG.warn("Was unable to close region " + regionToSideline + + ". Just continuing... ", ioe); + } catch (InterruptedException e) { + LOG.warn("Was unable to close region " + regionToSideline + + ". Just continuing... ", e); + } + + try { + LOG.info("Offlining region: " + regionToSideline); + hbck.offline(regionToSideline.getRegionName()); + } catch (IOException ioe) { + LOG.warn("Unable to offline region from master: " + regionToSideline + + ". Just continuing... ", ioe); + } + + LOG.info("Before sideline big overlapped region: " + regionToSideline.toString()); + Path sidelineRegionDir = hbck.sidelineRegionDir(fs, TO_BE_LOADED, regionToSideline); + if (sidelineRegionDir != null) { + sidelinedRegions.put(sidelineRegionDir, regionToSideline); + LOG.info("After sidelined big overlapped region: " + + regionToSideline.getRegionNameAsString() + + " to " + sidelineRegionDir.toString()); + hbck.fixes++; + } + } + } + } + + /** + * Check the region chain (from META) of this table. We are looking for + * holes, overlaps, and cycles. + * @return false if there are errors + */ + public boolean checkRegionChain(TableIntegrityErrorHandler handler) throws IOException { + // When table is disabled no need to check for the region chain. Some of the regions + // accidently if deployed, this below code might report some issues like missing start + // or end regions or region hole in chain and may try to fix which is unwanted. + if (hbck.isTableDisabled(this.tableName)) { + return true; + } + int originalErrorsCount = hbck.getErrors().getErrorList().size(); + Multimap regions = sc.calcCoverage(); + SortedSet splits = sc.getSplits(); + + byte[] prevKey = null; + byte[] problemKey = null; + + if (splits.isEmpty()) { + // no region for this table + handler.handleHoleInRegionChain(HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); + } + + for (byte[] key : splits) { + Collection ranges = regions.get(key); + if (prevKey == null && !Bytes.equals(key, HConstants.EMPTY_BYTE_ARRAY)) { + for (HbckRegionInfo rng : ranges) { + handler.handleRegionStartKeyNotEmpty(rng); + } + } + + // check for degenerate ranges + for (HbckRegionInfo rng : ranges) { + // special endkey case converts '' to null + byte[] endKey = rng.getEndKey(); + endKey = (endKey.length == 0) ? null : endKey; + if (Bytes.equals(rng.getStartKey(),endKey)) { + handler.handleDegenerateRegion(rng); + } + } + + if (ranges.size() == 1) { + // this split key is ok -- no overlap, not a hole. + if (problemKey != null) { + LOG.warn("reached end of problem group: " + Bytes.toStringBinary(key)); + } + problemKey = null; // fell through, no more problem. + } else if (ranges.size() > 1) { + // set the new problem key group name, if already have problem key, just + // keep using it. + if (problemKey == null) { + // only for overlap regions. + LOG.warn("Naming new problem group: " + Bytes.toStringBinary(key)); + problemKey = key; + } + overlapGroups.putAll(problemKey, ranges); + + // record errors + ArrayList subRange = new ArrayList<>(ranges); + // this dumb and n^2 but this shouldn't happen often + for (HbckRegionInfo r1 : ranges) { + if (r1.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) { + continue; + } + subRange.remove(r1); + for (HbckRegionInfo r2 : subRange) { + if (r2.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) { + continue; + } + // general case of same start key + if (Bytes.compareTo(r1.getStartKey(), r2.getStartKey())==0) { + handler.handleDuplicateStartKeys(r1,r2); + } else if (Bytes.compareTo(r1.getEndKey(), r2.getStartKey())==0 && + r1.getHdfsHRI().getRegionId() == r2.getHdfsHRI().getRegionId()) { + LOG.info("this is a split, log to splits"); + handler.handleSplit(r1, r2); + } else { + // overlap + handler.handleOverlapInRegionChain(r1, r2); + } + } + } + + } else if (ranges.isEmpty()) { + if (problemKey != null) { + LOG.warn("reached end of problem group: " + Bytes.toStringBinary(key)); + } + problemKey = null; + + byte[] holeStopKey = sc.getSplits().higher(key); + // if higher key is null we reached the top. + if (holeStopKey != null) { + // hole + handler.handleHoleInRegionChain(key, holeStopKey); + } + } + prevKey = key; + } + + // When the last region of a table is proper and having an empty end key, 'prevKey' + // will be null. + if (prevKey != null) { + handler.handleRegionEndKeyNotEmpty(prevKey); + } + + // TODO fold this into the TableIntegrityHandler + if (hbck.getConf().getBoolean("hbasefsck.overlap.merge.parallel", true)) { + boolean ok = handleOverlapsParallel(handler, prevKey); + if (!ok) { + return false; + } + } else { + for (Collection overlap : overlapGroups.asMap().values()) { + handler.handleOverlapGroup(overlap); + } + } + + if (hbck.shouldDisplayFullReport()) { + // do full region split map dump + hbck.getErrors().print("---- Table '" + this.tableName + + "': region split map"); + dump(splits, regions); + hbck.getErrors().print("---- Table '" + this.tableName + + "': overlap groups"); + dumpOverlapProblems(overlapGroups); + hbck.getErrors().print("There are " + overlapGroups.keySet().size() + + " overlap groups with " + overlapGroups.size() + + " overlapping regions"); + } + if (!sidelinedRegions.isEmpty()) { + LOG.warn("Sidelined big overlapped regions, please bulk load them!"); + hbck.getErrors().print("---- Table '" + this.tableName + + "': sidelined big overlapped regions"); + dumpSidelinedRegions(sidelinedRegions); + } + return hbck.getErrors().getErrorList().size() == originalErrorsCount; + } + + private boolean handleOverlapsParallel(TableIntegrityErrorHandler handler, byte[] prevKey) + throws IOException { + // we parallelize overlap handler for the case we have lots of groups to fix. We can + // safely assume each group is independent. + List merges = new ArrayList<>(overlapGroups.size()); + List> rets; + for (Collection overlap : overlapGroups.asMap().values()) { + // + merges.add(new HBaseFsck.WorkItemOverlapMerge(overlap, handler)); + } + try { + rets = hbck.executor.invokeAll(merges); + } catch (InterruptedException e) { + LOG.error("Overlap merges were interrupted", e); + return false; + } + for(int i=0; i f = rets.get(i); + try { + f.get(); + } catch(ExecutionException e) { + LOG.warn("Failed to merge overlap group" + work, e.getCause()); + } catch (InterruptedException e) { + LOG.error("Waiting for overlap merges was interrupted", e); + return false; + } + } + return true; + } + + /** + * This dumps data in a visually reasonable way for visual debugging + */ + private void dump(SortedSet splits, Multimap regions) { + // we display this way because the last end key should be displayed as well. + StringBuilder sb = new StringBuilder(); + for (byte[] k : splits) { + sb.setLength(0); // clear out existing buffer, if any. + sb.append(Bytes.toStringBinary(k) + ":\t"); + for (HbckRegionInfo r : regions.get(k)) { + sb.append("[ "+ r.toString() + ", " + + Bytes.toStringBinary(r.getEndKey())+ "]\t"); + } + hbck.getErrors().print(sb.toString()); + } + } + + private void dumpOverlapProblems(Multimap regions) { + // we display this way because the last end key should be displayed as + // well. + for (byte[] k : regions.keySet()) { + hbck.getErrors().print(Bytes.toStringBinary(k) + ":"); + for (HbckRegionInfo r : regions.get(k)) { + hbck.getErrors().print("[ " + r.toString() + ", " + + Bytes.toStringBinary(r.getEndKey()) + "]"); + } + hbck.getErrors().print("----"); + } + } + + private void dumpSidelinedRegions(Map regions) { + for (Map.Entry entry : regions.entrySet()) { + TableName tableName = entry.getValue().getTableName(); + Path path = entry.getKey(); + hbck.getErrors().print("This sidelined region dir should be bulk loaded: " + path.toString()); + hbck.getErrors().print("Bulk load command looks like: " + BulkLoadHFilesTool.NAME + " " + + path.toUri().getPath() + " " + tableName); + } + } +} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/HFileCorruptionChecker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/HFileCorruptionChecker.java index 41f3cde67ec..f64f9912f89 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/HFileCorruptionChecker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/HFileCorruptionChecker.java @@ -31,6 +31,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.hadoop.hbase.util.HbckErrorReporter; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,7 +49,6 @@ import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.FSUtils.FamilyDirFilter; import org.apache.hadoop.hbase.util.FSUtils.HFileFilter; import org.apache.hadoop.hbase.util.FSUtils.RegionDirFilter; -import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter; /** * This class marches through all of the region's hfiles and verifies that @@ -523,7 +523,7 @@ public class HFileCorruptionChecker { * Print a human readable summary of hfile quarantining operations. * @param out */ - public void report(ErrorReporter out) { + public void report(HbckErrorReporter out) { out.print("Checked " + hfilesChecked.get() + " hfile for corruption"); out.print(" HFiles corrupted: " + corrupted.size()); if (inQuarantineMode) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java index c08c6546426..7203fd103bb 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java @@ -30,8 +30,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerStorage; import org.apache.hadoop.hbase.replication.ReplicationQueueInfo; import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; -import org.apache.hadoop.hbase.util.HBaseFsck; -import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter; +import org.apache.hadoop.hbase.util.HbckErrorReporter; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; @@ -45,7 +44,7 @@ public class ReplicationChecker { private static final Logger LOG = LoggerFactory.getLogger(ReplicationChecker.class); - private final ErrorReporter errorReporter; + private final HbckErrorReporter errorReporter; // replicator with its queueIds for removed peers private Map> undeletedQueueIds = new HashMap<>(); // replicator with its undeleted queueIds for removed peers in hfile-refs queue @@ -54,7 +53,7 @@ public class ReplicationChecker { private final ReplicationPeerStorage peerStorage; private final ReplicationQueueStorage queueStorage; - public ReplicationChecker(Configuration conf, ZKWatcher zkw, ErrorReporter errorReporter) { + public ReplicationChecker(Configuration conf, ZKWatcher zkw, HbckErrorReporter errorReporter) { this.peerStorage = ReplicationStorageFactory.getReplicationPeerStorage(zkw, conf); this.queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zkw, conf); this.errorReporter = errorReporter; @@ -62,7 +61,7 @@ public class ReplicationChecker { public boolean hasUnDeletedQueues() { return errorReporter.getErrorList() - .contains(HBaseFsck.ErrorReporter.ERROR_CODE.UNDELETED_REPLICATION_QUEUE); + .contains(HbckErrorReporter.ERROR_CODE.UNDELETED_REPLICATION_QUEUE); } private Map> getUnDeletedQueues() throws ReplicationException { @@ -104,16 +103,14 @@ public class ReplicationChecker { String msg = "Undeleted replication queue for removed peer found: " + String.format("[removedPeerId=%s, replicator=%s, queueId=%s]", queueInfo.getPeerId(), replicator, queueId); - errorReporter.reportError(HBaseFsck.ErrorReporter.ERROR_CODE.UNDELETED_REPLICATION_QUEUE, - msg); + errorReporter.reportError(HbckErrorReporter.ERROR_CODE.UNDELETED_REPLICATION_QUEUE, msg); }); }); undeletedHFileRefsPeerIds = getUndeletedHFileRefsPeers(); - undeletedHFileRefsPeerIds.stream() - .map( - peerId -> "Undeleted replication hfile-refs queue for removed peer " + peerId + " found") + undeletedHFileRefsPeerIds.stream().map( + peerId -> "Undeleted replication hfile-refs queue for removed peer " + peerId + " found") .forEach(msg -> errorReporter - .reportError(HBaseFsck.ErrorReporter.ERROR_CODE.UNDELETED_REPLICATION_QUEUE, msg)); + .reportError(HbckErrorReporter.ERROR_CODE.UNDELETED_REPLICATION_QUEUE, msg)); } public void fixUnDeletedQueues() throws ReplicationException { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/TableIntegrityErrorHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/TableIntegrityErrorHandler.java index d5f5f5320c8..e4b4a814e2d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/TableIntegrityErrorHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/TableIntegrityErrorHandler.java @@ -20,8 +20,8 @@ package org.apache.hadoop.hbase.util.hbck; import java.io.IOException; import java.util.Collection; -import org.apache.hadoop.hbase.util.HBaseFsck.HbckInfo; -import org.apache.hadoop.hbase.util.HBaseFsck.TableInfo; +import org.apache.hadoop.hbase.util.HbckRegionInfo; +import org.apache.hadoop.hbase.util.HbckTableInfo; import org.apache.yetus.audience.InterfaceAudience; /** @@ -33,22 +33,22 @@ import org.apache.yetus.audience.InterfaceAudience; @InterfaceAudience.Private public interface TableIntegrityErrorHandler { - TableInfo getTableInfo(); + HbckTableInfo getTableInfo(); /** * Set the TableInfo used by all HRegionInfos fabricated by other callbacks */ - void setTableInfo(TableInfo ti); + void setTableInfo(HbckTableInfo ti); /** * Callback for handling case where a Table has a first region that does not * have an empty start key. * - * @param hi An HbckInfo of the second region in a table. This should have + * @param hi An HbckRegionInfo of the second region in a table. This should have * a non-empty startkey, and can be used to fabricate a first region that * has an empty start key. */ - void handleRegionStartKeyNotEmpty(HbckInfo hi) throws IOException; + void handleRegionStartKeyNotEmpty(HbckRegionInfo hi) throws IOException; /** * Callback for handling case where a Table has a last region that does not @@ -62,35 +62,35 @@ public interface TableIntegrityErrorHandler { /** * Callback for handling a region that has the same start and end key. * - * @param hi An HbckInfo for a degenerate key. + * @param hi An HbckRegionInfo for a degenerate key. */ - void handleDegenerateRegion(HbckInfo hi) throws IOException; + void handleDegenerateRegion(HbckRegionInfo hi) throws IOException; /** * Callback for handling two regions that have the same start key. This is * a specific case of a region overlap. - * @param hi1 one of the overlapping HbckInfo - * @param hi2 the other overlapping HbckInfo + * @param hi1 one of the overlapping HbckRegionInfo + * @param hi2 the other overlapping HbckRegionInfo */ - void handleDuplicateStartKeys(HbckInfo hi1, HbckInfo hi2) throws IOException; + void handleDuplicateStartKeys(HbckRegionInfo hi1, HbckRegionInfo hi2) throws IOException; /** * Callback for handling two regions that have the same regionID * a specific case of a split - * @param hi1 one of the overlapping HbckInfo - * @param hi2 the other overlapping HbckInfo + * @param hi1 one of the overlapping HbckRegionInfo + * @param hi2 the other overlapping HbckRegionInfo */ - void handleSplit(HbckInfo hi1, HbckInfo hi2) throws IOException; + void handleSplit(HbckRegionInfo hi1, HbckRegionInfo hi2) throws IOException; /** * Callback for handling two reigons that overlap in some arbitrary way. * This is a specific case of region overlap, and called for each possible * pair. If two regions have the same start key, the handleDuplicateStartKeys * method is called. - * @param hi1 one of the overlapping HbckInfo - * @param hi2 the other overlapping HbckInfo + * @param hi1 one of the overlapping HbckRegionInfo + * @param hi2 the other overlapping HbckRegionInfo */ - void handleOverlapInRegionChain(HbckInfo hi1, HbckInfo hi2) + void handleOverlapInRegionChain(HbckRegionInfo hi1, HbckRegionInfo hi2) throws IOException; /** @@ -106,5 +106,5 @@ public interface TableIntegrityErrorHandler { * Callback for handling an group of regions that overlap. * @param overlap Collection of overlapping regions. */ - void handleOverlapGroup(Collection overlap) throws IOException; + void handleOverlapGroup(Collection overlap) throws IOException; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/TableIntegrityErrorHandlerImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/TableIntegrityErrorHandlerImpl.java index 96039b5f42b..f39c623aa46 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/TableIntegrityErrorHandlerImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/TableIntegrityErrorHandlerImpl.java @@ -19,8 +19,8 @@ package org.apache.hadoop.hbase.util.hbck; import java.io.IOException; import java.util.Collection; -import org.apache.hadoop.hbase.util.HBaseFsck.HbckInfo; -import org.apache.hadoop.hbase.util.HBaseFsck.TableInfo; +import org.apache.hadoop.hbase.util.HbckRegionInfo; +import org.apache.hadoop.hbase.util.HbckTableInfo; import org.apache.yetus.audience.InterfaceAudience; /** @@ -30,13 +30,13 @@ import org.apache.yetus.audience.InterfaceAudience; @InterfaceAudience.Private abstract public class TableIntegrityErrorHandlerImpl implements TableIntegrityErrorHandler { - TableInfo ti; + HbckTableInfo ti; /** * {@inheritDoc} */ @Override - public TableInfo getTableInfo() { + public HbckTableInfo getTableInfo() { return ti; } @@ -44,7 +44,7 @@ abstract public class TableIntegrityErrorHandlerImpl implements * {@inheritDoc} */ @Override - public void setTableInfo(TableInfo ti2) { + public void setTableInfo(HbckTableInfo ti2) { this.ti = ti2; } @@ -52,7 +52,7 @@ abstract public class TableIntegrityErrorHandlerImpl implements * {@inheritDoc} */ @Override - public void handleRegionStartKeyNotEmpty(HbckInfo hi) throws IOException { + public void handleRegionStartKeyNotEmpty(HbckRegionInfo hi) throws IOException { } /** @@ -66,14 +66,14 @@ abstract public class TableIntegrityErrorHandlerImpl implements * {@inheritDoc} */ @Override - public void handleDegenerateRegion(HbckInfo hi) throws IOException { + public void handleDegenerateRegion(HbckRegionInfo hi) throws IOException { } /** * {@inheritDoc} */ @Override - public void handleDuplicateStartKeys(HbckInfo hi1, HbckInfo hi2) + public void handleDuplicateStartKeys(HbckRegionInfo hi1, HbckRegionInfo hi2) throws IOException { } @@ -81,7 +81,7 @@ abstract public class TableIntegrityErrorHandlerImpl implements * {@inheritDoc} */ @Override - public void handleOverlapInRegionChain(HbckInfo hi1, HbckInfo hi2) + public void handleOverlapInRegionChain(HbckRegionInfo hi1, HbckRegionInfo hi2) throws IOException { } @@ -97,7 +97,7 @@ abstract public class TableIntegrityErrorHandlerImpl implements * {@inheritDoc} */ @Override - public void handleOverlapGroup(Collection overlap) + public void handleOverlapGroup(Collection overlap) throws IOException { } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java index 40ce1c553ca..cc1805a5609 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java @@ -65,9 +65,6 @@ import org.apache.hadoop.hbase.master.assignment.RegionStates; import org.apache.hadoop.hbase.mob.MobFileName; import org.apache.hadoop.hbase.mob.MobUtils; import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; -import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter; -import org.apache.hadoop.hbase.util.HBaseFsck.HbckInfo; -import org.apache.hadoop.hbase.util.HBaseFsck.TableInfo; import org.apache.hadoop.hbase.util.hbck.HFileCorruptionChecker; import org.apache.zookeeper.KeeperException; import org.junit.rules.TestName; @@ -462,7 +459,7 @@ public class BaseTestHBaseFsck { } - static class MockErrorReporter implements ErrorReporter { + static class MockErrorReporter implements HbckErrorReporter { static int calledCount = 0; @Override @@ -486,19 +483,19 @@ public class BaseTestHBaseFsck { } @Override - public void reportError(ERROR_CODE errorCode, String message, TableInfo table) { + public void reportError(ERROR_CODE errorCode, String message, HbckTableInfo table) { calledCount++; } @Override public void reportError(ERROR_CODE errorCode, - String message, TableInfo table, HbckInfo info) { + String message, HbckTableInfo table, HbckRegionInfo info) { calledCount++; } @Override public void reportError(ERROR_CODE errorCode, String message, - TableInfo table, HbckInfo info1, HbckInfo info2) { + HbckTableInfo table, HbckRegionInfo info1, HbckRegionInfo info2) { calledCount++; } @@ -534,7 +531,7 @@ public class BaseTestHBaseFsck { } @Override - public boolean tableHasErrors(TableInfo table) { + public boolean tableHasErrors(HbckTableInfo table) { calledCount++; return false; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckComparator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckComparator.java index 99c8e236955..7da5c49e94c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckComparator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckComparator.java @@ -25,8 +25,7 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.util.HBaseFsck.HbckInfo; -import org.apache.hadoop.hbase.util.HBaseFsck.MetaEntry; +import org.apache.hadoop.hbase.util.HbckRegionInfo.MetaEntry; import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -51,58 +50,57 @@ public class TestHBaseFsckComparator { byte[] keyC = Bytes.toBytes("C"); byte[] keyEnd = Bytes.toBytes(""); - static HbckInfo genHbckInfo(TableName table, byte[] start, byte[] end, int time) { - return new HbckInfo(new MetaEntry(new HRegionInfo(table, start, end), null, + static HbckRegionInfo genHbckInfo(TableName table, byte[] start, byte[] end, int time) { + return new HbckRegionInfo(new MetaEntry(new HRegionInfo(table, start, end), null, time)); } @Test public void testEquals() { - HbckInfo hi1 = genHbckInfo(table, keyA, keyB, 0); - HbckInfo hi2 = genHbckInfo(table, keyA, keyB, 0); - assertEquals(0, HBaseFsck.cmp.compare(hi1, hi2)); - assertEquals(0, HBaseFsck.cmp.compare(hi2, hi1)); + HbckRegionInfo hi1 = genHbckInfo(table, keyA, keyB, 0); + HbckRegionInfo hi2 = genHbckInfo(table, keyA, keyB, 0); + assertEquals(0, HbckRegionInfo.COMPARATOR.compare(hi1, hi2)); + assertEquals(0, HbckRegionInfo.COMPARATOR.compare(hi2, hi1)); } @Test public void testEqualsInstance() { - HbckInfo hi1 = genHbckInfo(table, keyA, keyB, 0); - HbckInfo hi2 = hi1; - assertEquals(0, HBaseFsck.cmp.compare(hi1, hi2)); - assertEquals(0, HBaseFsck.cmp.compare(hi2, hi1)); + HbckRegionInfo hi1 = genHbckInfo(table, keyA, keyB, 0); + HbckRegionInfo hi2 = hi1; + assertEquals(0, HbckRegionInfo.COMPARATOR.compare(hi1, hi2)); + assertEquals(0, HbckRegionInfo.COMPARATOR.compare(hi2, hi1)); } @Test public void testDiffTable() { - HbckInfo hi1 = genHbckInfo(table, keyA, keyC, 0); - HbckInfo hi2 = genHbckInfo(table2, keyA, keyC, 0); - assertTrue(HBaseFsck.cmp.compare(hi1, hi2) < 0); - assertTrue(HBaseFsck.cmp.compare(hi2, hi1) > 0); + HbckRegionInfo hi1 = genHbckInfo(table, keyA, keyC, 0); + HbckRegionInfo hi2 = genHbckInfo(table2, keyA, keyC, 0); + assertTrue(HbckRegionInfo.COMPARATOR.compare(hi1, hi2) < 0); + assertTrue(HbckRegionInfo.COMPARATOR.compare(hi2, hi1) > 0); } @Test public void testDiffStartKey() { - HbckInfo hi1 = genHbckInfo(table, keyStart, keyC, 0); - HbckInfo hi2 = genHbckInfo(table, keyA, keyC, 0); - assertTrue(HBaseFsck.cmp.compare(hi1, hi2) < 0); - assertTrue(HBaseFsck.cmp.compare(hi2, hi1) > 0); + HbckRegionInfo hi1 = genHbckInfo(table, keyStart, keyC, 0); + HbckRegionInfo hi2 = genHbckInfo(table, keyA, keyC, 0); + assertTrue(HbckRegionInfo.COMPARATOR.compare(hi1, hi2) < 0); + assertTrue(HbckRegionInfo.COMPARATOR.compare(hi2, hi1) > 0); } @Test public void testDiffEndKey() { - HbckInfo hi1 = genHbckInfo(table, keyA, keyB, 0); - HbckInfo hi2 = genHbckInfo(table, keyA, keyC, 0); - assertTrue(HBaseFsck.cmp.compare(hi1, hi2) < 0); - assertTrue(HBaseFsck.cmp.compare(hi2, hi1) > 0); + HbckRegionInfo hi1 = genHbckInfo(table, keyA, keyB, 0); + HbckRegionInfo hi2 = genHbckInfo(table, keyA, keyC, 0); + assertTrue(HbckRegionInfo.COMPARATOR.compare(hi1, hi2) < 0); + assertTrue(HbckRegionInfo.COMPARATOR.compare(hi2, hi1) > 0); } @Test public void testAbsEndKey() { - HbckInfo hi1 = genHbckInfo(table, keyA, keyC, 0); - HbckInfo hi2 = genHbckInfo(table, keyA, keyEnd, 0); - assertTrue(HBaseFsck.cmp.compare(hi1, hi2) < 0); - assertTrue(HBaseFsck.cmp.compare(hi2, hi1) > 0); + HbckRegionInfo hi1 = genHbckInfo(table, keyA, keyC, 0); + HbckRegionInfo hi2 = genHbckInfo(table, keyA, keyEnd, 0); + assertTrue(HbckRegionInfo.COMPARATOR.compare(hi1, hi2) < 0); + assertTrue(HbckRegionInfo.COMPARATOR.compare(hi2, hi1) > 0); } - } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java index 3ae09f6b5fa..72512eed157 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java @@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; -import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE; +import org.apache.hadoop.hbase.util.HbckErrorReporter.ERROR_CODE; import org.apache.hadoop.hbase.util.hbck.HbckTestingUtil; import org.junit.AfterClass; import org.junit.BeforeClass; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java index 1808b5e658f..d04c3514f36 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java @@ -30,7 +30,7 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.util.HBaseFsck; -import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE; +import org.apache.hadoop.hbase.util.HbckErrorReporter.ERROR_CODE; public class HbckTestingUtil { private static ExecutorService exec = new ScheduledThreadPoolExecutor(10);