@Deprecated @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
Deprecated.
|
private static class |
HBaseFsck.FileLockCallable
Deprecated.
|
(package private) static class |
HBaseFsck.HBaseFsckTool
Deprecated.
This is a Tool wrapper that gathers -Dxxx=yyy configuration settings from the command line.
|
(package private) static class |
HBaseFsck.PrintingErrorReporter
Deprecated.
|
private static class |
HBaseFsck.RegionBoundariesInformation
Deprecated.
|
(package private) class |
HBaseFsck.WorkItemHdfsDir
Deprecated.
Contact hdfs and get all information about specified table directory into regioninfo list.
|
(package private) static class |
HBaseFsck.WorkItemHdfsRegionInfo
Deprecated.
Contact hdfs and get all information about specified table directory into regioninfo list.
|
(package private) static class |
HBaseFsck.WorkItemOverlapMerge
Deprecated.
|
(package private) static class |
HBaseFsck.WorkItemRegion
Deprecated.
Contact a region server and get all information from it
|
Modifier and Type | Field and Description |
---|---|
private Admin |
admin
Deprecated.
|
private boolean |
checkHdfs
Deprecated.
|
private boolean |
checkMetaOnly
Deprecated.
|
private boolean |
checkRegionBoundaries
Deprecated.
|
private boolean |
cleanReplicationBarrier
Deprecated.
|
private TableName |
cleanReplicationBarrierTable
Deprecated.
|
private ClusterConnection |
connection
Deprecated.
|
private RetryCounterFactory |
createZNodeRetryCounterFactory
Deprecated.
|
private static int |
DEFAULT_CREATE_ZNODE_ATTEMPT_MAX_SLEEP_TIME
Deprecated.
|
private static int |
DEFAULT_CREATE_ZNODE_ATTEMPT_SLEEP_INTERVAL
Deprecated.
|
private static int |
DEFAULT_LOCK_FILE_ATTEMPT_MAX_SLEEP_TIME
Deprecated.
|
private static int |
DEFAULT_LOCK_FILE_ATTEMPT_SLEEP_INTERVAL
Deprecated.
|
private static int |
DEFAULT_MAX_CREATE_ZNODE_ATTEMPTS
Deprecated.
|
private static int |
DEFAULT_MAX_LOCK_FILE_ATTEMPTS
Deprecated.
|
private static int |
DEFAULT_MAX_MERGE
Deprecated.
|
private static int |
DEFAULT_OVERLAPS_TO_SIDELINE
Deprecated.
|
static long |
DEFAULT_SLEEP_BEFORE_RERUN
Deprecated.
|
static long |
DEFAULT_TIME_LAG
Deprecated.
|
private static int |
DEFAULT_WAIT_FOR_LOCK_TIMEOUT
Deprecated.
|
private static boolean |
details
Deprecated.
Options
|
private Set<Result> |
emptyRegionInfoQualifiers
Deprecated.
|
private HbckErrorReporter |
errors
Deprecated.
State
|
protected ExecutorService |
executor
Deprecated.
|
private boolean |
fixAny
Deprecated.
|
private boolean |
fixAssignments
Deprecated.
|
private boolean |
fixEmptyMetaCells
Deprecated.
|
(package private) int |
fixes
Deprecated.
|
private boolean |
fixHdfsHoles
Deprecated.
|
private boolean |
fixHdfsOrphans
Deprecated.
|
private boolean |
fixHdfsOverlaps
Deprecated.
|
private boolean |
fixHFileLinks
Deprecated.
|
private boolean |
fixMeta
Deprecated.
|
private boolean |
fixReferenceFiles
Deprecated.
|
private boolean |
fixReplication
Deprecated.
|
private boolean |
fixSplitParents
Deprecated.
|
private boolean |
fixTableOrphans
Deprecated.
|
private boolean |
fixVersionFile
Deprecated.
|
private static boolean |
forceExclusive
Deprecated.
|
static String |
HBCK_LOCK_FILE
Deprecated.
Here is where hbase-1.x used to default the lock for hbck1.
|
private org.apache.hadoop.fs.Path |
HBCK_LOCK_PATH
Deprecated.
|
private String |
hbckEphemeralNodePath
Deprecated.
|
private AtomicBoolean |
hbckLockCleanup
Deprecated.
|
private org.apache.hadoop.fs.FSDataOutputStream |
hbckOutFd
Deprecated.
|
private boolean |
hbckZodeCreated
Deprecated.
|
private HFileCorruptionChecker |
hfcc
Deprecated.
|
private boolean |
ignorePreCheckPermission
Deprecated.
|
private RetryCounterFactory |
lockFileRetryCounterFactory
Deprecated.
|
private static org.slf4j.Logger |
LOG
Deprecated.
Internal resources
|
private static int |
MAX_NUM_THREADS
Deprecated.
|
private int |
maxMerge
Deprecated.
|
private int |
maxOverlapsToSideline
Deprecated.
|
private Table |
meta
Deprecated.
|
private List<HbckRegionInfo> |
orphanHdfsDirs
Deprecated.
When initially looking at HDFS, we attempt to find any orphaned data.
|
private Map<TableName,Set<String>> |
orphanTableDirs
Deprecated.
|
private TreeMap<String,HbckRegionInfo> |
regionInfoMap
Deprecated.
This map contains the state of all hbck items.
|
private boolean |
removeParents
Deprecated.
|
private boolean |
rerun
Deprecated.
|
private int |
retcode
Deprecated.
|
private static boolean |
rsSupportsOffline
Deprecated.
|
private boolean |
sidelineBigOverlaps
Deprecated.
|
private org.apache.hadoop.fs.Path |
sidelineDir
Deprecated.
|
private Map<TableName,Set<String>> |
skippedRegions
Deprecated.
|
private long |
startMillis
Deprecated.
|
private ClusterMetrics |
status
Deprecated.
|
private static boolean |
summary
Deprecated.
|
private Set<TableName> |
tablesIncluded
Deprecated.
|
private SortedMap<TableName,HbckTableInfo> |
tablesInfo
Deprecated.
This map from Tablename -> TableInfo contains the structures necessary to detect table
consistency problems (holes, dupes, overlaps).
|
private Map<TableName,TableState> |
tableStates
Deprecated.
|
private long |
timelag
Deprecated.
|
private static Set<String> |
unsupportedOptionsInV2
Deprecated.
|
private ZKWatcher |
zkw
Deprecated.
|
Constructor and Description |
---|
HBaseFsck(org.apache.hadoop.conf.Configuration conf)
Deprecated.
Constructor
|
HBaseFsck(org.apache.hadoop.conf.Configuration conf,
ExecutorService exec)
Deprecated.
Constructor n * Configuration object n * if the master is not running n * if unable to connect
to ZooKeeper
|
Modifier and Type | Method and Description |
---|---|
private void |
addSkippedRegion(HbckRegionInfo hbi)
Deprecated.
|
private void |
adoptHdfsOrphan(HbckRegionInfo hi)
Deprecated.
Orphaned regions are regions without a .regioninfo file in them.
|
private void |
adoptHdfsOrphans(Collection<HbckRegionInfo> orphanHdfsDirs)
Deprecated.
Iterates through the list of all orphan/invalid regiondirs.
|
private void |
assignMetaReplica(int replicaId)
Deprecated.
|
private void |
checkAndFixConsistency()
Deprecated.
Check consistency of all regions that have been found in previous phases.
|
private void |
checkAndFixReplication()
Deprecated.
|
private void |
checkAndFixTableStates()
Deprecated.
Check and fix table states, assumes full info available: - tableInfos - empty tables loaded
|
static Pair<org.apache.hadoop.fs.Path,org.apache.hadoop.fs.FSDataOutputStream> |
checkAndMarkRunningHbck(org.apache.hadoop.conf.Configuration conf,
RetryCounter retryCounter)
Deprecated.
This method maintains a lock using a file.
|
private SortedMap<TableName,HbckTableInfo> |
checkHdfsIntegrity(boolean fixHoles,
boolean fixOverlaps)
Deprecated.
|
(package private) SortedMap<TableName,HbckTableInfo> |
checkIntegrity()
Deprecated.
Checks tables integrity.
|
(package private) boolean |
checkMetaRegion()
Deprecated.
Check values in regionInfo for hbase:meta Check if zero or more than one regions with
hbase:meta are found.
|
void |
checkRegionBoundaries()
Deprecated.
|
private void |
checkRegionConsistency(String key,
HbckRegionInfo hbi)
Deprecated.
Check a single region for consistency and correct deployment.
|
private void |
checkRegionConsistencyConcurrently(List<HBaseFsck.CheckRegionConsistencyWorkItem> workItems)
Deprecated.
Check consistency of all regions using multiple threads concurrently.
|
void |
cleanReplicationBarrier()
Deprecated.
|
private void |
cleanupHbckZnode()
Deprecated.
|
private void |
clearState()
Deprecated.
Clear the current state of hbck.
|
void |
close()
Deprecated.
|
(package private) void |
closeRegion(HbckRegionInfo hi)
Deprecated.
Attempts to undeploy a region from a region server based in information in META.
|
void |
connect()
Deprecated.
To repair region consistency, one must call connect() in order to repair online state.
|
protected HFileCorruptionChecker |
createHFileCorruptionChecker(boolean sidelineCorruptHFiles)
Deprecated.
|
static RetryCounterFactory |
createLockRetryCounterFactory(org.apache.hadoop.conf.Configuration conf)
Deprecated.
Returns A retry counter factory configured for retrying lock file creation.
|
private static ExecutorService |
createThreadPool(org.apache.hadoop.conf.Configuration conf)
Deprecated.
|
private static RetryCounterFactory |
createZnodeRetryCounterFactory(org.apache.hadoop.conf.Configuration conf)
Deprecated.
Returns A retry counter factory configured for retrying znode creation.
|
private ZKWatcher |
createZooKeeperWatcher()
Deprecated.
|
static void |
debugLsr(org.apache.hadoop.conf.Configuration conf,
org.apache.hadoop.fs.Path p)
Deprecated.
ls -r for debugging purposes
|
static void |
debugLsr(org.apache.hadoop.conf.Configuration conf,
org.apache.hadoop.fs.Path p,
HbckErrorReporter errors)
Deprecated.
ls -r for debugging purposes
|
(package private) void |
debugLsr(org.apache.hadoop.fs.Path p)
Deprecated.
ls -r for debugging purposes
|
private void |
deleteMetaRegion(byte[] metaKey)
Deprecated.
Deletes region from meta table
|
private void |
deleteMetaRegion(HbckRegionInfo hi)
Deprecated.
Deletes region from meta table
|
HBaseFsck |
exec(ExecutorService exec,
String[] args)
Deprecated.
|
private boolean |
fabricateTableInfo(FSTableDescriptors fstd,
TableName tableName,
Set<String> columns)
Deprecated.
To fabricate a .tableinfo file with following contents
1. |
void |
fixEmptyMetaCells()
Deprecated.
To fix the empty REGIONINFO_QUALIFIER rows from hbase:meta
n |
void |
fixOrphanTables()
Deprecated.
To fix orphan table by creating a .tableinfo file under tableDir
1. |
private Set<String> |
getColumnFamilyList(Set<String> columns,
HbckRegionInfo hbi)
Deprecated.
To get the column family list according to the column family dirs nn * @return a set of column
families n
|
(package private) static HbckErrorReporter |
getErrorReporter(org.apache.hadoop.conf.Configuration conf)
Deprecated.
|
HbckErrorReporter |
getErrors()
Deprecated.
|
HFileCorruptionChecker |
getHFilecorruptionChecker()
Deprecated.
|
(package private) Set<TableName> |
getIncludedTables()
Deprecated.
|
int |
getMaxMerge()
Deprecated.
|
int |
getMaxOverlapsToSideline()
Deprecated.
|
private HbckRegionInfo |
getOrCreateInfo(String name)
Deprecated.
Gets the entry in regionInfo corresponding to the the given encoded region name.
|
int |
getRetCode()
Deprecated.
|
(package private) org.apache.hadoop.fs.Path |
getSidelineDir()
Deprecated.
|
(package private) TableDescriptor[] |
getTableDescriptors(List<TableName> tableNames)
Deprecated.
|
(package private) TableDescriptor[] |
getTables(AtomicInteger numSkipped)
Deprecated.
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.
|
static org.apache.hadoop.fs.Path |
getTmpDir(org.apache.hadoop.conf.Configuration conf)
Deprecated.
Returns Return the tmp dir this tool writes too.
|
void |
includeTable(TableName table)
Deprecated.
|
boolean |
isExclusive()
Deprecated.
Only one instance of hbck can modify HBase at a time.
|
private boolean |
isOptionsSupported(String[] args)
Deprecated.
|
(package private) boolean |
isTableDisabled(TableName tableName)
Deprecated.
Check if the specified region's table is disabled.
|
(package private) boolean |
isTableIncluded(TableName table)
Deprecated.
Only check/fix tables specified by the list, Empty list means all tables are included.
|
static byte[] |
keyOnly(byte[] b)
Deprecated.
|
private void |
loadDeployedRegions()
Deprecated.
Get deployed regions according to the region servers.
|
void |
loadHdfsRegionDirs()
Deprecated.
Scan HDFS for all regions, recording their information into regionInfoMap
|
private SortedMap<TableName,HbckTableInfo> |
loadHdfsRegionInfos()
Deprecated.
Populate hbi's from regionInfos loaded from file system.
|
(package private) boolean |
loadMetaEntries()
Deprecated.
Scan hbase:meta, adding all regions found to the regionInfo map.
|
private void |
loadTableInfosForTablesWithNoRegion()
Deprecated.
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 |
loadTableStates()
Deprecated.
Load the list of disabled tables in ZK into local set.
|
private void |
logParallelMerge()
Deprecated.
Log an appropriate message about whether or not overlapping merges are computed in parallel.
|
static void |
main(String[] args)
Deprecated.
Main program nn
|
int |
mergeRegionDirs(org.apache.hadoop.fs.Path targetRegionDir,
HbckRegionInfo contained)
Deprecated.
Merge hdfs data by moving from contained HbckRegionInfo into targetRegionDir.
|
(package private) void |
offline(byte[] regionName)
Deprecated.
This backwards-compatibility wrapper for permanently offlining a region that should not be
alive.
|
void |
offlineHdfsIntegrityRepair()
Deprecated.
This repair method analyzes hbase data in hdfs and repairs it to satisfy the table integrity
rules.
|
private void |
offlineHLinkFileRepair()
Deprecated.
Scan all the store file names to find any lingering HFileLink files, which refer to some
none-exiting files.
|
private void |
offlineReferenceFileRepair()
Deprecated.
Scan all the store file names to find any lingering reference files, which refer to some
none-exiting files.
|
int |
onlineConsistencyRepair()
Deprecated.
This repair method requires the cluster to be online since it contacts region servers and the
masters.
|
int |
onlineHbck()
Deprecated.
Contacts the master and prints out cluster-wide information
|
private void |
preCheckPermission()
Deprecated.
|
private void |
printTableSummary(SortedMap<TableName,HbckTableInfo> tablesInfo)
Deprecated.
Prints summary of all tables found on the system.
|
protected HBaseFsck |
printUsageAndExit()
Deprecated.
|
(package private) void |
processRegionServers(Collection<ServerName> regionServerList)
Deprecated.
Contacts each regionserver and fetches metadata about regions.
|
private boolean |
recordMetaRegion()
Deprecated.
Record the location of the hbase:meta region as found in ZooKeeper.
|
private void |
reportEmptyMetaCells()
Deprecated.
TODO -- need to add tests for this.
|
private void |
reportTablesInFlux()
Deprecated.
TODO -- need to add tests for this.
|
private void |
resetSplitParent(HbckRegionInfo hi)
Deprecated.
Reset the split parent region info in meta table
|
private int |
restoreHdfsIntegrity()
Deprecated.
This method determines if there are table integrity errors in HDFS.
|
void |
setCheckHdfs(boolean checking)
Deprecated.
|
(package private) void |
setCheckMetaOnly()
Deprecated.
Set hbase:meta check mode.
|
void |
setCleanReplicationBarrier(boolean shouldClean)
Deprecated.
|
void |
setCleanReplicationBarrierTable(String cleanReplicationBarrierTable)
Deprecated.
|
static void |
setDisplayFullReport()
Deprecated.
Display the full report from fsck.
|
void |
setFixAssignments(boolean shouldFix)
Deprecated.
Fix inconsistencies found by fsck.
|
void |
setFixEmptyMetaCells(boolean shouldFix)
Deprecated.
|
void |
setFixHdfsHoles(boolean shouldFix)
Deprecated.
|
void |
setFixHdfsOrphans(boolean shouldFix)
Deprecated.
|
void |
setFixHdfsOverlaps(boolean shouldFix)
Deprecated.
|
void |
setFixHFileLinks(boolean shouldFix)
Deprecated.
|
void |
setFixMeta(boolean shouldFix)
Deprecated.
|
void |
setFixReferenceFiles(boolean shouldFix)
Deprecated.
|
void |
setFixReplication(boolean shouldFix)
Deprecated.
Set replication fix mode.
|
void |
setFixSplitParents(boolean shouldFix)
Deprecated.
|
void |
setFixTableOrphans(boolean shouldFix)
Deprecated.
|
void |
setFixVersionFile(boolean shouldFix)
Deprecated.
|
static void |
setForceExclusive()
Deprecated.
Set exclusive mode.
|
void |
setHFileCorruptionChecker(HFileCorruptionChecker hfcc)
Deprecated.
|
void |
setIgnorePreCheckPermission(boolean ignorePreCheckPermission)
Deprecated.
|
private boolean |
setMasterInMaintenanceMode()
Deprecated.
This method maintains an ephemeral znode.
|
void |
setMaxMerge(int mm)
Deprecated.
|
void |
setMaxOverlapsToSideline(int mo)
Deprecated.
|
(package private) void |
setRegionBoundariesCheck()
Deprecated.
Set region boundaries check mode.
|
void |
setRemoveParents(boolean shouldFix)
Deprecated.
|
void |
setRetCode(int code)
Deprecated.
|
(package private) void |
setShouldRerun()
Deprecated.
Check if we should rerun fsck again.
|
void |
setSidelineBigOverlaps(boolean sbo)
Deprecated.
|
void |
setSidelineDir(String sidelineDir)
Deprecated.
|
(package private) static void |
setSummary()
Deprecated.
Set summary mode.
|
void |
setTimeLag(long seconds)
Deprecated.
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()
Deprecated.
|
static boolean |
shouldDisplayFullReport()
Deprecated.
|
(package private) boolean |
shouldFixAssignments()
Deprecated.
|
(package private) boolean |
shouldFixEmptyMetaCells()
Deprecated.
|
(package private) boolean |
shouldFixHdfsHoles()
Deprecated.
|
(package private) boolean |
shouldFixHdfsOrphans()
Deprecated.
|
(package private) boolean |
shouldFixHdfsOverlaps()
Deprecated.
|
(package private) boolean |
shouldFixHFileLinks()
Deprecated.
|
(package private) boolean |
shouldFixMeta()
Deprecated.
|
(package private) boolean |
shouldFixReferenceFiles()
Deprecated.
|
(package private) boolean |
shouldFixSplitParents()
Deprecated.
|
(package private) boolean |
shouldFixTableOrphans()
Deprecated.
|
boolean |
shouldFixVersionFile()
Deprecated.
|
boolean |
shouldIgnorePreCheckPermission()
Deprecated.
|
(package private) boolean |
shouldRemoveParents()
Deprecated.
|
boolean |
shouldRerun()
Deprecated.
|
boolean |
shouldSidelineBigOverlaps()
Deprecated.
|
private boolean |
sidelineFile(org.apache.hadoop.fs.FileSystem fs,
org.apache.hadoop.fs.Path hbaseRoot,
org.apache.hadoop.fs.Path path)
Deprecated.
|
(package private) org.apache.hadoop.fs.Path |
sidelineRegionDir(org.apache.hadoop.fs.FileSystem fs,
HbckRegionInfo hi)
Deprecated.
Sideline a region dir (instead of deleting it)
|
(package private) org.apache.hadoop.fs.Path |
sidelineRegionDir(org.apache.hadoop.fs.FileSystem fs,
String parentDir,
HbckRegionInfo hi)
Deprecated.
Sideline a region dir (instead of deleting it)
|
private void |
tryAssignmentRepair(HbckRegionInfo hbi,
String msg)
Deprecated.
|
private void |
unassignMetaReplica(HbckRegionInfo hi)
Deprecated.
|
private void |
undeployRegions(HbckRegionInfo hi)
Deprecated.
|
private void |
undeployRegionsForHbi(HbckRegionInfo hi)
Deprecated.
|
private void |
unlockHbck()
Deprecated.
|
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
@InterfaceAudience.Private public 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 int DEFAULT_MAX_CREATE_ZNODE_ATTEMPTS
private static final int DEFAULT_CREATE_ZNODE_ATTEMPT_SLEEP_INTERVAL
private static final int DEFAULT_CREATE_ZNODE_ATTEMPT_MAX_SLEEP_TIME
private static final org.slf4j.Logger LOG
private ClusterMetrics status
private ClusterConnection connection
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 final Set<String> unsupportedOptionsInV2
private static boolean details
private long timelag
private static boolean forceExclusive
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 fixHFileLinks
private boolean fixEmptyMetaCells
private boolean fixReplication
private boolean cleanReplicationBarrier
private boolean fixAny
private Set<TableName> tablesIncluded
private TableName cleanReplicationBarrierTable
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 HbckErrorReporter errors
int fixes
private TreeMap<String,HbckRegionInfo> regionInfoMap
private Set<Result> emptyRegionInfoQualifiers
private SortedMap<TableName,HbckTableInfo> tablesInfo
private List<HbckRegionInfo> orphanHdfsDirs
private Map<TableName,Set<String>> orphanTableDirs
private Map<TableName,TableState> tableStates
private final RetryCounterFactory lockFileRetryCounterFactory
private final RetryCounterFactory createZNodeRetryCounterFactory
private Map<TableName,Set<String>> skippedRegions
private String hbckEphemeralNodePath
private boolean hbckZodeCreated
public HBaseFsck(org.apache.hadoop.conf.Configuration conf) throws 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
private static ExecutorService createThreadPool(org.apache.hadoop.conf.Configuration conf)
public static RetryCounterFactory createLockRetryCounterFactory(org.apache.hadoop.conf.Configuration conf)
private static RetryCounterFactory createZnodeRetryCounterFactory(org.apache.hadoop.conf.Configuration conf)
@InterfaceAudience.Private public static org.apache.hadoop.fs.Path getTmpDir(org.apache.hadoop.conf.Configuration conf) throws IOException
IOException
public static Pair<org.apache.hadoop.fs.Path,org.apache.hadoop.fs.FSDataOutputStream> checkAndMarkRunningHbck(org.apache.hadoop.conf.Configuration conf, RetryCounter retryCounter) throws IOException
IOException
- if IO failure occursprivate 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
private boolean setMasterInMaintenanceMode() throws IOException
IOException
- if IO failure occursprivate void cleanupHbckZnode()
public int onlineHbck() throws IOException, org.apache.zookeeper.KeeperException, InterruptedException, ReplicationException
IOException
org.apache.zookeeper.KeeperException
InterruptedException
ReplicationException
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<HbckRegionInfo> orphanHdfsDirs) throws IOException
IOException
private void adoptHdfsOrphan(HbckRegionInfo hi) throws IOException
IOException
private int restoreHdfsIntegrity() throws IOException, InterruptedException
IOException
InterruptedException
private void offlineReferenceFileRepair() throws IOException, InterruptedException
Lingering reference file prevents a region from opening. It has to be fixed before a cluster can start properly.
IOException
InterruptedException
private void offlineHLinkFileRepair() throws IOException, InterruptedException
IOException
InterruptedException
private boolean sidelineFile(org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path hbaseRoot, org.apache.hadoop.fs.Path path) throws IOException
IOException
private void reportEmptyMetaCells()
private void reportTablesInFlux()
public HbckErrorReporter getErrors()
private SortedMap<TableName,HbckTableInfo> loadHdfsRegionInfos() throws IOException, InterruptedException
IOException
InterruptedException
private Set<String> getColumnFamilyList(Set<String> columns, HbckRegionInfo hbi) throws IOException
IOException
private boolean fabricateTableInfo(FSTableDescriptors fstd, TableName tableName, Set<String> columns) throws IOException
TableDescriptor
and
ColumnFamilyDescriptor
IOException
public void fixEmptyMetaCells() throws IOException
IOException
public void fixOrphanTables() throws IOException
TableDescriptor
and
ColumnFamilyDescriptor
IOException
private void logParallelMerge()
private SortedMap<TableName,HbckTableInfo> checkHdfsIntegrity(boolean fixHoles, boolean fixOverlaps) throws IOException
IOException
org.apache.hadoop.fs.Path getSidelineDir() throws IOException
IOException
org.apache.hadoop.fs.Path sidelineRegionDir(org.apache.hadoop.fs.FileSystem fs, HbckRegionInfo hi) throws IOException
IOException
org.apache.hadoop.fs.Path sidelineRegionDir(org.apache.hadoop.fs.FileSystem fs, String parentDir, HbckRegionInfo hi) throws IOException
parentDir
- if specified, the region will be sidelined to folder like
.../parentDir/<table name>/<region name>. The purpose is to group
together similar regions sidelined, for example, those regions should be bulk
loaded back later on. If NULL, it is ignored.IOException
private void loadTableStates() throws IOException
IOException
boolean isTableDisabled(TableName tableName)
tableName
- table to check status ofpublic void loadHdfsRegionDirs() throws IOException, InterruptedException
IOException
InterruptedException
private boolean recordMetaRegion() throws IOException
IOException
private ZKWatcher createZooKeeperWatcher() throws IOException
IOException
void processRegionServers(Collection<ServerName> regionServerList) throws IOException, InterruptedException
regionServerList
- - the list of region servers to connect toIOException
- if a remote or network exception occursInterruptedException
private void checkAndFixConsistency() throws IOException, org.apache.zookeeper.KeeperException, InterruptedException
IOException
org.apache.zookeeper.KeeperException
InterruptedException
private void checkRegionConsistencyConcurrently(List<HBaseFsck.CheckRegionConsistencyWorkItem> workItems) throws IOException, org.apache.zookeeper.KeeperException, InterruptedException
IOException
org.apache.zookeeper.KeeperException
InterruptedException
private void addSkippedRegion(HbckRegionInfo hbi)
private void checkAndFixTableStates() throws IOException
IOException
private void preCheckPermission() throws IOException
IOException
private void deleteMetaRegion(HbckRegionInfo hi) throws IOException
IOException
private void deleteMetaRegion(byte[] metaKey) throws IOException
IOException
private void resetSplitParent(HbckRegionInfo hi) throws IOException
IOException
void offline(byte[] regionName) throws IOException
IOException
void closeRegion(HbckRegionInfo hi) throws IOException, InterruptedException
IOException
InterruptedException
private void undeployRegions(HbckRegionInfo hi) throws IOException, InterruptedException
IOException
InterruptedException
private void undeployRegionsForHbi(HbckRegionInfo hi) throws IOException, InterruptedException
IOException
InterruptedException
private void tryAssignmentRepair(HbckRegionInfo hbi, String msg) throws IOException, org.apache.zookeeper.KeeperException, InterruptedException
IOException
org.apache.zookeeper.KeeperException
InterruptedException
private void checkRegionConsistency(String key, HbckRegionInfo hbi) throws IOException, org.apache.zookeeper.KeeperException, InterruptedException
IOException
org.apache.zookeeper.KeeperException
InterruptedException
SortedMap<TableName,HbckTableInfo> checkIntegrity() throws IOException
IOException
private void loadTableInfosForTablesWithNoRegion() throws IOException
IOException
public int mergeRegionDirs(org.apache.hadoop.fs.Path targetRegionDir, HbckRegionInfo contained) throws IOException
IOException
TableDescriptor[] getTables(AtomicInteger numSkipped)
IOException
- if an error is encounteredTableDescriptor[] getTableDescriptors(List<TableName> tableNames)
private HbckRegionInfo getOrCreateInfo(String name)
private void checkAndFixReplication() throws ReplicationException
ReplicationException
boolean checkMetaRegion() throws IOException, org.apache.zookeeper.KeeperException, InterruptedException
IOException
- from HBaseFsckRepair functions nnorg.apache.zookeeper.KeeperException
InterruptedException
private void unassignMetaReplica(HbckRegionInfo hi) throws IOException, InterruptedException, org.apache.zookeeper.KeeperException
IOException
InterruptedException
org.apache.zookeeper.KeeperException
private void assignMetaReplica(int replicaId) throws IOException, org.apache.zookeeper.KeeperException, InterruptedException
IOException
org.apache.zookeeper.KeeperException
InterruptedException
boolean loadMetaEntries() throws IOException
IOException
- if an error is encounteredprivate void printTableSummary(SortedMap<TableName,HbckTableInfo> tablesInfo)
static HbckErrorReporter getErrorReporter(org.apache.hadoop.conf.Configuration conf) throws ClassNotFoundException
ClassNotFoundException
public static void setDisplayFullReport()
public static boolean shouldDisplayFullReport()
public static void setForceExclusive()
public boolean isExclusive()
static void setSummary()
void setCheckMetaOnly()
void setRegionBoundariesCheck()
public void setFixReplication(boolean shouldFix)
public void setCleanReplicationBarrier(boolean shouldClean)
void setShouldRerun()
public boolean shouldRerun()
public void setFixAssignments(boolean shouldFix)
boolean shouldFixAssignments()
public void setFixMeta(boolean shouldFix)
boolean shouldFixMeta()
public void setFixEmptyMetaCells(boolean shouldFix)
boolean shouldFixEmptyMetaCells()
public void setCheckHdfs(boolean checking)
boolean shouldCheckHdfs()
public void setFixHdfsHoles(boolean shouldFix)
boolean shouldFixHdfsHoles()
public void setFixTableOrphans(boolean shouldFix)
boolean shouldFixTableOrphans()
public void setFixHdfsOverlaps(boolean shouldFix)
boolean shouldFixHdfsOverlaps()
public void setFixHdfsOrphans(boolean shouldFix)
boolean shouldFixHdfsOrphans()
public void setFixVersionFile(boolean shouldFix)
public boolean shouldFixVersionFile()
public void setSidelineBigOverlaps(boolean sbo)
public boolean shouldSidelineBigOverlaps()
public void setFixSplitParents(boolean shouldFix)
public void setRemoveParents(boolean shouldFix)
boolean shouldFixSplitParents()
boolean shouldRemoveParents()
public void setFixReferenceFiles(boolean shouldFix)
boolean shouldFixReferenceFiles()
public void setFixHFileLinks(boolean shouldFix)
boolean shouldFixHFileLinks()
public boolean shouldIgnorePreCheckPermission()
public void setIgnorePreCheckPermission(boolean ignorePreCheckPermission)
public void setMaxMerge(int mm)
mm
- maximum number of regions to merge into a single region.public int getMaxMerge()
public void setMaxOverlapsToSideline(int mo)
public int getMaxOverlapsToSideline()
boolean isTableIncluded(TableName table)
public void includeTable(TableName table)
Set<TableName> getIncludedTables()
public void setTimeLag(long seconds)
seconds
- - the time in secondspublic void setSidelineDir(String sidelineDir)
sidelineDir
- - HDFS path to sideline dataprotected HFileCorruptionChecker createHFileCorruptionChecker(boolean sidelineCorruptHFiles) throws IOException
IOException
public HFileCorruptionChecker getHFilecorruptionChecker()
public void setHFileCorruptionChecker(HFileCorruptionChecker hfcc)
public void setRetCode(int code)
public int getRetCode()
protected HBaseFsck printUsageAndExit()
public static void main(String[] args) throws Exception
Exception
public HBaseFsck exec(ExecutorService exec, String[] args) throws org.apache.zookeeper.KeeperException, IOException, InterruptedException, ReplicationException
org.apache.zookeeper.KeeperException
IOException
InterruptedException
ReplicationException
private boolean isOptionsSupported(String[] args)
public void setCleanReplicationBarrierTable(String cleanReplicationBarrierTable)
public void cleanReplicationBarrier() throws IOException
IOException
void debugLsr(org.apache.hadoop.fs.Path p) throws IOException
IOException
public static void debugLsr(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.Path p) throws IOException
IOException
public static void debugLsr(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.Path p, HbckErrorReporter errors) throws IOException
IOException
Copyright © 2007–2020 The Apache Software Foundation. All rights reserved.