@InterfaceAudience.LimitedPrivate(value="Tools") @InterfaceStability.Evolving public class HBaseFsck extends org.apache.hadoop.conf.Configured implements Closeable
Region consistency checks verify that hbase:meta, region deployment on region servers and the state of data in HDFS (.regioninfo files) all are in accordance.
Table integrity checks verify that all possible row keys resolve to exactly one region of a table. This means there are no individual degenerate or backwards regions; no holes between regions; and that there are no overlapping regions.
The general repair strategy works in two phases:
For table integrity repairs, the tables' region directories are scanned for .regioninfo files. Each table's integrity is then verified. If there are any orphan regions (regions with no .regioninfo files) or holes, new regions are fabricated. Backwards regions are sidelined as well as empty degenerate (endkey==startkey) regions. If there are any overlapping regions, a new region is created and all data is merged into the new region.
Table integrity repairs deal solely with HDFS and could potentially be done offline -- the hbase region servers or master do not need to be running. This phase can eventually be used to completely reconstruct the hbase:meta table in an offline fashion.
Region consistency requires three conditions -- 1) valid .regioninfo file present in an HDFS region dir, 2) valid row with .regioninfo data in META, and 3) a region is deployed only at the regionserver that was assigned to with proper state in the master.
Region consistency repairs require hbase to be online so that hbck can contact the HBase master and region servers. The hbck#connect() method must first be called successfully. Much of the region consistency information is transient and less risky to repair.
If hbck is run from the command line, there are a handful of arguments that
can be used to limit the kinds of repairs hbck will do. See the code in
printUsageAndExit()
for more details.
Modifier and Type | Class and Description |
---|---|
(package private) class |
HBaseFsck.CheckRegionConsistencyWorkItem |
static interface |
HBaseFsck.ErrorReporter |
private class |
HBaseFsck.FileLockCallable |
(package private) static class |
HBaseFsck.HBaseFsckTool
This is a Tool wrapper that gathers -Dxxx=yyy configuration settings from the command line.
|
static class |
HBaseFsck.HbckInfo
Maintain information about a particular region.
|
(package private) static class |
HBaseFsck.HdfsEntry
Stores the regioninfo entries from HDFS
|
(package private) static class |
HBaseFsck.MetaEntry
Stores the regioninfo entries scanned from META
|
(package private) static class |
HBaseFsck.OnlineEntry
Stores the regioninfo retrieved from Online region servers.
|
(package private) static class |
HBaseFsck.PrintingErrorReporter |
private static class |
HBaseFsck.RegionBoundariesInformation |
static class |
HBaseFsck.RegionRepairException
Exception thrown when a integrity repair operation fails in an
unresolvable way.
|
class |
HBaseFsck.TableInfo
Maintain information about a particular table.
|
(package private) static class |
HBaseFsck.WorkItemHdfsDir
Contact hdfs and get all information about specified table directory into
regioninfo list.
|
(package private) static class |
HBaseFsck.WorkItemHdfsRegionInfo
Contact hdfs and get all information about specified table directory into
regioninfo list.
|
(package private) static class |
HBaseFsck.WorkItemOverlapMerge |
(package private) static class |
HBaseFsck.WorkItemRegion
Contact a region server and get all information from it
|
Modifier and Type | Field and Description |
---|---|
private Admin |
admin |
private boolean |
checkHdfs |
private boolean |
checkMetaOnly |
private boolean |
checkRegionBoundaries |
(package private) static Comparator<HBaseFsck.HbckInfo> |
cmp |
private ClusterConnection |
connection |
private static int |
DEFAULT_LOCK_FILE_ATTEMPT_MAX_SLEEP_TIME |
private static int |
DEFAULT_LOCK_FILE_ATTEMPT_SLEEP_INTERVAL |
private static int |
DEFAULT_MAX_LOCK_FILE_ATTEMPTS |
private static int |
DEFAULT_MAX_MERGE |
private static int |
DEFAULT_OVERLAPS_TO_SIDELINE |
static long |
DEFAULT_SLEEP_BEFORE_RERUN |
static long |
DEFAULT_TIME_LAG |
private static int |
DEFAULT_WAIT_FOR_LOCK_TIMEOUT |
private static boolean |
details
Options
|
private static boolean |
disableBalancer |
private TreeSet<TableName> |
disabledTables |
private Set<Result> |
emptyRegionInfoQualifiers |
private HBaseFsck.ErrorReporter |
errors
State
|
protected ExecutorService |
executor |
private boolean |
fixAny |
private boolean |
fixAssignments |
private boolean |
fixEmptyMetaCells |
(package private) int |
fixes |
private boolean |
fixHdfsHoles |
private boolean |
fixHdfsOrphans |
private boolean |
fixHdfsOverlaps |
private boolean |
fixMeta |
private boolean |
fixReferenceFiles |
private boolean |
fixSplitParents |
private boolean |
fixTableLocks |
private boolean |
fixTableOrphans |
private boolean |
fixTableZNodes |
private boolean |
fixVersionFile |
private static boolean |
forceExclusive |
private static String |
HBCK_LOCK_FILE |
private org.apache.hadoop.fs.Path |
HBCK_LOCK_PATH |
private AtomicBoolean |
hbckLockCleanup |
private org.apache.hadoop.fs.FSDataOutputStream |
hbckOutFd |
private HFileCorruptionChecker |
hfcc |
private boolean |
ignorePreCheckPermission |
private RetryCounterFactory |
lockFileRetryCounterFactory |
private static org.apache.commons.logging.Log |
LOG
Internal resources
|
private static int |
MAX_NUM_THREADS |
private int |
maxMerge |
private int |
maxOverlapsToSideline |
private Table |
meta |
private Set<TableName> |
orphanedTableZNodes
List of orphaned table ZNodes
|
private List<HBaseFsck.HbckInfo> |
orphanHdfsDirs
When initially looking at HDFS, we attempt to find any orphaned data.
|
private Map<TableName,Set<String>> |
orphanTableDirs |
private TreeMap<String,HBaseFsck.HbckInfo> |
regionInfoMap
This map contains the state of all hbck items.
|
private boolean |
removeParents |
private boolean |
rerun |
private int |
retcode |
private static boolean |
rsSupportsOffline |
private boolean |
sidelineBigOverlaps |
private org.apache.hadoop.fs.Path |
sidelineDir |
private Map<TableName,Set<String>> |
skippedRegions |
private long |
startMillis |
private ClusterStatus |
status |
private static boolean |
SUMMARY |
private Set<TableName> |
tablesIncluded |
private SortedMap<TableName,HBaseFsck.TableInfo> |
tablesInfo
This map from Tablename -> TableInfo contains the structures necessary to
detect table consistency problems (holes, dupes, overlaps).
|
private long |
timelag |
private static String |
TO_BE_LOADED |
Constructor and Description |
---|
HBaseFsck(org.apache.hadoop.conf.Configuration conf)
Constructor
|
HBaseFsck(org.apache.hadoop.conf.Configuration conf,
ExecutorService exec)
Constructor
|
Modifier and Type | Method and Description |
---|---|
private void |
addSkippedRegion(HBaseFsck.HbckInfo hbi) |
private void |
adoptHdfsOrphan(HBaseFsck.HbckInfo hi)
Orphaned regions are regions without a .regioninfo file in them.
|
private void |
adoptHdfsOrphans(Collection<HBaseFsck.HbckInfo> orphanHdfsDirs)
Iterates through the list of all orphan/invalid regiondirs.
|
private void |
assignMetaReplica(int replicaId) |
private void |
checkAndFixConsistency()
Check consistency of all regions that have been found in previous phases.
|
private void |
checkAndFixOrphanedTableZNodes()
Check whether a orphaned table ZNode exists and fix it if requested.
|
private void |
checkAndFixTableLocks() |
private org.apache.hadoop.fs.FSDataOutputStream |
checkAndMarkRunningHbck()
This method maintains a lock using a file.
|
private SortedMap<TableName,HBaseFsck.TableInfo> |
checkHdfsIntegrity(boolean fixHoles,
boolean fixOverlaps) |
(package private) SortedMap<TableName,HBaseFsck.TableInfo> |
checkIntegrity()
Checks tables integrity.
|
(package private) boolean |
checkMetaRegion()
Check values in regionInfo for hbase:meta
Check if zero or more than one regions with hbase:meta are found.
|
void |
checkRegionBoundaries() |
private void |
checkRegionConsistency(String key,
HBaseFsck.HbckInfo hbi)
Check a single region for consistency and correct deployment.
|
private void |
checkRegionConsistencyConcurrently(List<HBaseFsck.CheckRegionConsistencyWorkItem> workItems)
Check consistency of all regions using mulitple threads concurrently.
|
private void |
clearState()
Clear the current state of hbck.
|
void |
close() |
private void |
closeRegion(HBaseFsck.HbckInfo hi)
Attempts to undeploy a region from a region server based in information in
META.
|
void |
connect()
To repair region consistency, one must call connect() in order to repair
online state.
|
protected HFileCorruptionChecker |
createHFileCorruptionChecker(boolean sidelineCorruptHFiles) |
private HRegion |
createNewMeta()
This borrows code from MasterFileSystem.bootstrap()
|
private static ExecutorService |
createThreadPool(org.apache.hadoop.conf.Configuration conf) |
private ZooKeeperWatcher |
createZooKeeperWatcher() |
static void |
debugLsr(org.apache.hadoop.conf.Configuration conf,
org.apache.hadoop.fs.Path p)
ls -r for debugging purposes
|
static void |
debugLsr(org.apache.hadoop.conf.Configuration conf,
org.apache.hadoop.fs.Path p,
HBaseFsck.ErrorReporter errors)
ls -r for debugging purposes
|
(package private) void |
debugLsr(org.apache.hadoop.fs.Path p)
ls -r for debugging purposes
|
private void |
deleteMetaRegion(byte[] metaKey)
Deletes region from meta table
|
private void |
deleteMetaRegion(HBaseFsck.HbckInfo hi)
Deletes region from meta table
|
void |
dumpOverlapProblems(com.google.common.collect.Multimap<byte[],HBaseFsck.HbckInfo> regions) |
void |
dumpSidelinedRegions(Map<org.apache.hadoop.fs.Path,HBaseFsck.HbckInfo> regions) |
HBaseFsck |
exec(ExecutorService exec,
String[] args) |
private boolean |
fabricateTableInfo(FSTableDescriptors fstd,
TableName tableName,
Set<String> columns)
To fabricate a .tableinfo file with following contents
1. |
void |
fixEmptyMetaCells()
To fix the empty REGIONINFO_QUALIFIER rows from hbase:meta
|
void |
fixOrphanTables()
To fix orphan table by creating a .tableinfo file under tableDir
1. |
private ArrayList<Put> |
generatePuts(SortedMap<TableName,HBaseFsck.TableInfo> tablesInfo)
Generate set of puts to add to new meta.
|
private Set<String> |
getColumnFamilyList(Set<String> columns,
HBaseFsck.HbckInfo hbi)
To get the column family list according to the column family dirs
|
(package private) static HBaseFsck.ErrorReporter |
getErrorReporter(org.apache.hadoop.conf.Configuration conf) |
HBaseFsck.ErrorReporter |
getErrors() |
HFileCorruptionChecker |
getHFilecorruptionChecker() |
(package private) HTableDescriptor[] |
getHTableDescriptors(List<TableName> tableNames) |
(package private) Set<TableName> |
getIncludedTables() |
int |
getMaxMerge() |
int |
getMaxOverlapsToSideline() |
private ServerName |
getMetaRegionServerName(int replicaId) |
private HBaseFsck.HbckInfo |
getOrCreateInfo(String name)
Gets the entry in regionInfo corresponding to the the given encoded
region name.
|
com.google.common.collect.Multimap<byte[],HBaseFsck.HbckInfo> |
getOverlapGroups(TableName table) |
int |
getRetCode() |
private org.apache.hadoop.fs.Path |
getSidelineDir() |
(package private) HTableDescriptor[] |
getTables(AtomicInteger numSkipped)
Return a list of user-space table names whose metadata have not been
modified in the last few milliseconds specified by timelag
if any of the REGIONINFO_QUALIFIER, SERVER_QUALIFIER, STARTCODE_QUALIFIER,
SPLITA_QUALIFIER, SPLITB_QUALIFIER have not changed in the last
milliseconds specified by timelag, then the table is a candidate to be returned.
|
void |
includeTable(TableName table) |
boolean |
isExclusive()
Only one instance of hbck can modify HBase at a time.
|
private boolean |
isTableDisabled(HRegionInfo regionInfo)
Check if the specified region's table is disabled.
|
(package private) boolean |
isTableIncluded(TableName table)
Only check/fix tables specified by the list,
Empty list means all tables are included.
|
static byte[] |
keyOnly(byte[] b) |
private void |
loadDeployedRegions()
Get deployed regions according to the region servers.
|
private void |
loadDisabledTables()
Load the list of disabled tables in ZK into local set.
|
void |
loadHdfsRegionDirs()
Scan HDFS for all regions, recording their information into
regionInfoMap
|
private void |
loadHdfsRegioninfo(HBaseFsck.HbckInfo hbi)
Read the .regioninfo file from the file system.
|
private SortedMap<TableName,HBaseFsck.TableInfo> |
loadHdfsRegionInfos()
Populate hbi's from regionInfos loaded from file system.
|
(package private) boolean |
loadMetaEntries()
Scan hbase:meta, adding all regions found to the regionInfo map.
|
private void |
loadTableInfosForTablesWithNoRegion()
Loads table info's for tables that may not have been included, since there are no
regions reported for the table, but table dir is there in hdfs
|
private void |
logParallelMerge()
Log an appropriate message about whether or not overlapping merges are computed in parallel.
|
static void |
main(String[] args)
Main program
|
int |
mergeRegionDirs(org.apache.hadoop.fs.Path targetRegionDir,
HBaseFsck.HbckInfo contained)
Merge hdfs data by moving from contained HbckInfo into targetRegionDir.
|
private void |
offline(byte[] regionName)
This backwards-compatibility wrapper for permanently offlining a region
that should not be alive.
|
void |
offlineHdfsIntegrityRepair()
This repair method analyzes hbase data in hdfs and repairs it to satisfy
the table integrity rules.
|
private void |
offlineReferenceFileRepair()
Scan all the store file names to find any lingering reference files,
which refer to some none-exiting files.
|
int |
onlineConsistencyRepair()
This repair method requires the cluster to be online since it contacts
region servers and the masters.
|
int |
onlineHbck()
Contacts the master and prints out cluster-wide information
|
private void |
preCheckPermission() |
private void |
printTableSummary(SortedMap<TableName,HBaseFsck.TableInfo> tablesInfo)
Prints summary of all tables found on the system.
|
protected HBaseFsck |
printUsageAndExit() |
(package private) void |
processRegionServers(Collection<ServerName> regionServerList)
Contacts each regionserver and fetches metadata about regions.
|
boolean |
rebuildMeta(boolean fix)
Rebuilds meta from information in hdfs/fs.
|
private boolean |
recordMetaRegion()
Record the location of the hbase:meta region as found in ZooKeeper.
|
private void |
reportEmptyMetaCells()
TODO -- need to add tests for this.
|
private void |
reportTablesInFlux()
TODO -- need to add tests for this.
|
private void |
resetSplitParent(HBaseFsck.HbckInfo hi)
Reset the split parent region info in meta table
|
private int |
restoreHdfsIntegrity()
This method determines if there are table integrity errors in HDFS.
|
void |
setCheckHdfs(boolean checking) |
(package private) void |
setCheckMetaOnly()
Set hbase:meta check mode.
|
static void |
setDisableBalancer()
Disable the load balancer.
|
static void |
setDisplayFullReport()
Display the full report from fsck.
|
void |
setFixAssignments(boolean shouldFix)
Fix inconsistencies found by fsck.
|
void |
setFixEmptyMetaCells(boolean shouldFix) |
void |
setFixHdfsHoles(boolean shouldFix) |
void |
setFixHdfsOrphans(boolean shouldFix) |
void |
setFixHdfsOverlaps(boolean shouldFix) |
void |
setFixMeta(boolean shouldFix) |
void |
setFixReferenceFiles(boolean shouldFix) |
void |
setFixSplitParents(boolean shouldFix) |
void |
setFixTableLocks(boolean shouldFix)
Set table locks fix mode.
|
void |
setFixTableOrphans(boolean shouldFix) |
void |
setFixTableZNodes(boolean shouldFix)
Set orphaned table ZNodes fix mode.
|
void |
setFixVersionFile(boolean shouldFix) |
static void |
setForceExclusive()
Set exclusive mode.
|
void |
setHFileCorruptionChecker(HFileCorruptionChecker hfcc) |
void |
setIgnorePreCheckPermission(boolean ignorePreCheckPermission) |
void |
setMaxMerge(int mm) |
void |
setMaxOverlapsToSideline(int mo) |
(package private) void |
setRegionBoundariesCheck()
Set region boundaries check mode.
|
void |
setRemoveParents(boolean shouldFix) |
void |
setRetCode(int code) |
(package private) void |
setShouldRerun()
Check if we should rerun fsck again.
|
void |
setSidelineBigOverlaps(boolean sbo) |
void |
setSidelineDir(String sidelineDir) |
(package private) static void |
setSummary()
Set summary mode.
|
void |
setTimeLag(long seconds)
We are interested in only those tables that have not changed their state in
hbase:meta during the last few seconds specified by hbase.admin.fsck.timelag
|
(package private) boolean |
shouldCheckHdfs() |
boolean |
shouldDisableBalancer()
The balancer should be disabled if we are modifying HBase.
|
(package private) boolean |
shouldFixAssignments() |
(package private) boolean |
shouldFixEmptyMetaCells() |
(package private) boolean |
shouldFixHdfsHoles() |
(package private) boolean |
shouldFixHdfsOrphans() |
(package private) boolean |
shouldFixHdfsOverlaps() |
(package private) boolean |
shouldFixMeta() |
(package private) boolean |
shouldFixReferenceFiles() |
(package private) boolean |
shouldFixSplitParents() |
(package private) boolean |
shouldFixTableOrphans() |
boolean |
shouldFixVersionFile() |
boolean |
shouldIgnorePreCheckPermission() |
(package private) boolean |
shouldRemoveParents() |
(package private) boolean |
shouldRerun() |
boolean |
shouldSidelineBigOverlaps() |
(package private) org.apache.hadoop.fs.Path |
sidelineOldMeta() |
(package private) org.apache.hadoop.fs.Path |
sidelineRegionDir(org.apache.hadoop.fs.FileSystem fs,
HBaseFsck.HbckInfo hi)
Sideline a region dir (instead of deleting it)
|
(package private) org.apache.hadoop.fs.Path |
sidelineRegionDir(org.apache.hadoop.fs.FileSystem fs,
String parentDir,
HBaseFsck.HbckInfo hi)
Sideline a region dir (instead of deleting it)
|
(package private) void |
sidelineTable(org.apache.hadoop.fs.FileSystem fs,
TableName tableName,
org.apache.hadoop.fs.Path hbaseDir,
org.apache.hadoop.fs.Path backupHbaseDir)
Side line an entire table.
|
private void |
suggestFixes(SortedMap<TableName,HBaseFsck.TableInfo> tablesInfo)
Suggest fixes for each table
|
private void |
tryAssignmentRepair(HBaseFsck.HbckInfo hbi,
String msg) |
private void |
unassignMetaReplica(HBaseFsck.HbckInfo hi) |
private void |
undeployRegions(HBaseFsck.HbckInfo hi) |
private void |
undeployRegionsForHbi(HBaseFsck.HbckInfo hi) |
private void |
unlockHbck() |
public static final long DEFAULT_TIME_LAG
public static final long DEFAULT_SLEEP_BEFORE_RERUN
private static final int MAX_NUM_THREADS
private static boolean rsSupportsOffline
private static final int DEFAULT_OVERLAPS_TO_SIDELINE
private static final int DEFAULT_MAX_MERGE
private static final String TO_BE_LOADED
private static final String HBCK_LOCK_FILE
private static final int DEFAULT_MAX_LOCK_FILE_ATTEMPTS
private static final int DEFAULT_LOCK_FILE_ATTEMPT_SLEEP_INTERVAL
private static final int DEFAULT_LOCK_FILE_ATTEMPT_MAX_SLEEP_TIME
private static final int DEFAULT_WAIT_FOR_LOCK_TIMEOUT
private static final org.apache.commons.logging.Log LOG
private ClusterStatus status
private ClusterConnection connection
private Admin admin
private Table meta
protected ExecutorService executor
private long startMillis
private HFileCorruptionChecker hfcc
private int retcode
private org.apache.hadoop.fs.Path HBCK_LOCK_PATH
private org.apache.hadoop.fs.FSDataOutputStream hbckOutFd
private final AtomicBoolean hbckLockCleanup
private static boolean details
private long timelag
private static boolean forceExclusive
private static boolean disableBalancer
private boolean fixAssignments
private boolean fixMeta
private boolean checkHdfs
private boolean fixHdfsHoles
private boolean fixHdfsOverlaps
private boolean fixHdfsOrphans
private boolean fixTableOrphans
private boolean fixVersionFile
private boolean fixSplitParents
private boolean removeParents
private boolean fixReferenceFiles
private boolean fixEmptyMetaCells
private boolean fixTableLocks
private boolean fixTableZNodes
private boolean fixAny
private int maxMerge
private int maxOverlapsToSideline
private boolean sidelineBigOverlaps
private org.apache.hadoop.fs.Path sidelineDir
private boolean rerun
private static boolean SUMMARY
private boolean checkMetaOnly
private boolean checkRegionBoundaries
private boolean ignorePreCheckPermission
private final HBaseFsck.ErrorReporter errors
int fixes
private TreeMap<String,HBaseFsck.HbckInfo> regionInfoMap
private SortedMap<TableName,HBaseFsck.TableInfo> tablesInfo
private List<HBaseFsck.HbckInfo> orphanHdfsDirs
private final RetryCounterFactory lockFileRetryCounterFactory
static final Comparator<HBaseFsck.HbckInfo> cmp
public HBaseFsck(org.apache.hadoop.conf.Configuration conf) throws MasterNotRunningException, ZooKeeperConnectionException, IOException, ClassNotFoundException
conf
- Configuration objectMasterNotRunningException
- if the master is not runningZooKeeperConnectionException
- if unable to connect to ZooKeeperIOException
ClassNotFoundException
public HBaseFsck(org.apache.hadoop.conf.Configuration conf, ExecutorService exec) throws MasterNotRunningException, ZooKeeperConnectionException, IOException, ClassNotFoundException
conf
- Configuration objectMasterNotRunningException
- if the master is not runningZooKeeperConnectionException
- if unable to connect to ZooKeeperIOException
ClassNotFoundException
private static ExecutorService createThreadPool(org.apache.hadoop.conf.Configuration conf)
private org.apache.hadoop.fs.FSDataOutputStream checkAndMarkRunningHbck() throws IOException
IOException
private void unlockHbck()
public void connect() throws IOException
IOException
private void loadDeployedRegions() throws IOException, InterruptedException
IOException
InterruptedException
private void clearState()
public void offlineHdfsIntegrityRepair() throws IOException, InterruptedException
IOException
InterruptedException
public int onlineConsistencyRepair() throws IOException, org.apache.zookeeper.KeeperException, InterruptedException
IOException
org.apache.zookeeper.KeeperException
InterruptedException
public int onlineHbck() throws IOException, org.apache.zookeeper.KeeperException, InterruptedException, com.google.protobuf.ServiceException
IOException
org.apache.zookeeper.KeeperException
InterruptedException
com.google.protobuf.ServiceException
public static byte[] keyOnly(byte[] b)
public void close() throws IOException
close
in interface Closeable
close
in interface AutoCloseable
IOException
public void checkRegionBoundaries()
private void adoptHdfsOrphans(Collection<HBaseFsck.HbckInfo> orphanHdfsDirs) throws IOException
IOException
private void adoptHdfsOrphan(HBaseFsck.HbckInfo hi) throws IOException
IOException
private int restoreHdfsIntegrity() throws IOException, InterruptedException
IOException
InterruptedException
private void offlineReferenceFileRepair() throws IOException
Lingering reference file prevents a region from opening. It has to be fixed before a cluster can start properly.
IOException
private void reportEmptyMetaCells()
private void reportTablesInFlux()
public HBaseFsck.ErrorReporter getErrors()
private void loadHdfsRegioninfo(HBaseFsck.HbckInfo hbi) throws IOException
IOException
private SortedMap<TableName,HBaseFsck.TableInfo> loadHdfsRegionInfos() throws IOException, InterruptedException
IOException
InterruptedException
private Set<String> getColumnFamilyList(Set<String> columns, HBaseFsck.HbckInfo hbi) throws IOException
columns
- hbi
- IOException
private boolean fabricateTableInfo(FSTableDescriptors fstd, TableName tableName, Set<String> columns) throws IOException
HTableDescriptor
and HColumnDescriptor
IOException
public void fixEmptyMetaCells() throws IOException
IOException
public void fixOrphanTables() throws IOException
HTableDescriptor
and HColumnDescriptor
IOException
private HRegion createNewMeta() throws IOException
IOException
private ArrayList<Put> generatePuts(SortedMap<TableName,HBaseFsck.TableInfo> tablesInfo) throws IOException
IOException
private void suggestFixes(SortedMap<TableName,HBaseFsck.TableInfo> tablesInfo) throws IOException
IOException
public boolean rebuildMeta(boolean fix) throws IOException, InterruptedException
fix
- flag that determines if method should attempt to fix holesIOException
InterruptedException
private void logParallelMerge()
private SortedMap<TableName,HBaseFsck.TableInfo> checkHdfsIntegrity(boolean fixHoles, boolean fixOverlaps) throws IOException
IOException
private org.apache.hadoop.fs.Path getSidelineDir() throws IOException
IOException
org.apache.hadoop.fs.Path sidelineRegionDir(org.apache.hadoop.fs.FileSystem fs, HBaseFsck.HbckInfo hi) throws IOException
IOException
org.apache.hadoop.fs.Path sidelineRegionDir(org.apache.hadoop.fs.FileSystem fs, String parentDir, HBaseFsck.HbckInfo hi) throws IOException
parentDir
- if specified, the region will be sidelined to
folder like .../parentDir/