001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.util; 019 020import java.io.Closeable; 021import java.io.FileNotFoundException; 022import java.io.IOException; 023import java.io.InterruptedIOException; 024import java.io.PrintWriter; 025import java.io.StringWriter; 026import java.net.InetAddress; 027import java.net.URI; 028import java.util.ArrayList; 029import java.util.Arrays; 030import java.util.Collection; 031import java.util.Collections; 032import java.util.Comparator; 033import java.util.EnumSet; 034import java.util.HashMap; 035import java.util.HashSet; 036import java.util.Iterator; 037import java.util.List; 038import java.util.Locale; 039import java.util.Map; 040import java.util.Map.Entry; 041import java.util.Objects; 042import java.util.Optional; 043import java.util.Set; 044import java.util.SortedMap; 045import java.util.SortedSet; 046import java.util.TreeMap; 047import java.util.TreeSet; 048import java.util.Vector; 049import java.util.concurrent.Callable; 050import java.util.concurrent.ConcurrentSkipListMap; 051import java.util.concurrent.ExecutionException; 052import java.util.concurrent.ExecutorService; 053import java.util.concurrent.Executors; 054import java.util.concurrent.Future; 055import java.util.concurrent.FutureTask; 056import java.util.concurrent.ScheduledThreadPoolExecutor; 057import java.util.concurrent.TimeUnit; 058import java.util.concurrent.TimeoutException; 059import java.util.concurrent.atomic.AtomicBoolean; 060import java.util.concurrent.atomic.AtomicInteger; 061import java.util.stream.Collectors; 062import org.apache.commons.io.IOUtils; 063import org.apache.commons.lang3.RandomStringUtils; 064import org.apache.commons.lang3.StringUtils; 065import org.apache.hadoop.conf.Configuration; 066import org.apache.hadoop.conf.Configured; 067import org.apache.hadoop.fs.FSDataOutputStream; 068import org.apache.hadoop.fs.FileStatus; 069import org.apache.hadoop.fs.FileSystem; 070import org.apache.hadoop.fs.Path; 071import org.apache.hadoop.fs.permission.FsAction; 072import org.apache.hadoop.fs.permission.FsPermission; 073import org.apache.hadoop.hbase.Abortable; 074import org.apache.hadoop.hbase.Cell; 075import org.apache.hadoop.hbase.CellUtil; 076import org.apache.hadoop.hbase.ClusterMetrics; 077import org.apache.hadoop.hbase.ClusterMetrics.Option; 078import org.apache.hadoop.hbase.HBaseConfiguration; 079import org.apache.hadoop.hbase.HBaseInterfaceAudience; 080import org.apache.hadoop.hbase.HConstants; 081import org.apache.hadoop.hbase.HRegionInfo; 082import org.apache.hadoop.hbase.HRegionLocation; 083import org.apache.hadoop.hbase.KeyValue; 084import org.apache.hadoop.hbase.MasterNotRunningException; 085import org.apache.hadoop.hbase.MetaTableAccessor; 086import org.apache.hadoop.hbase.RegionLocations; 087import org.apache.hadoop.hbase.ServerName; 088import org.apache.hadoop.hbase.TableName; 089import org.apache.hadoop.hbase.TableNotFoundException; 090import org.apache.hadoop.hbase.ZooKeeperConnectionException; 091import org.apache.hadoop.hbase.client.Admin; 092import org.apache.hadoop.hbase.client.ClusterConnection; 093import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; 094import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; 095import org.apache.hadoop.hbase.client.Connection; 096import org.apache.hadoop.hbase.client.ConnectionFactory; 097import org.apache.hadoop.hbase.client.Delete; 098import org.apache.hadoop.hbase.client.Get; 099import org.apache.hadoop.hbase.client.Put; 100import org.apache.hadoop.hbase.client.RegionInfo; 101import org.apache.hadoop.hbase.client.RegionInfoBuilder; 102import org.apache.hadoop.hbase.client.RegionReplicaUtil; 103import org.apache.hadoop.hbase.client.Result; 104import org.apache.hadoop.hbase.client.ResultScanner; 105import org.apache.hadoop.hbase.client.RowMutations; 106import org.apache.hadoop.hbase.client.Scan; 107import org.apache.hadoop.hbase.client.Table; 108import org.apache.hadoop.hbase.client.TableDescriptor; 109import org.apache.hadoop.hbase.client.TableDescriptorBuilder; 110import org.apache.hadoop.hbase.client.TableState; 111import org.apache.hadoop.hbase.io.FileLink; 112import org.apache.hadoop.hbase.io.HFileLink; 113import org.apache.hadoop.hbase.io.hfile.CacheConfig; 114import org.apache.hadoop.hbase.io.hfile.HFile; 115import org.apache.hadoop.hbase.log.HBaseMarkers; 116import org.apache.hadoop.hbase.master.MasterFileSystem; 117import org.apache.hadoop.hbase.master.RegionState; 118import org.apache.hadoop.hbase.regionserver.HRegion; 119import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; 120import org.apache.hadoop.hbase.regionserver.StoreFileInfo; 121import org.apache.hadoop.hbase.replication.ReplicationException; 122import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; 123import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; 124import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; 125import org.apache.hadoop.hbase.replication.ReplicationUtils; 126import org.apache.hadoop.hbase.security.AccessDeniedException; 127import org.apache.hadoop.hbase.security.UserProvider; 128import org.apache.hadoop.hbase.util.Bytes.ByteArrayComparator; 129import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE; 130import org.apache.hadoop.hbase.util.hbck.HFileCorruptionChecker; 131import org.apache.hadoop.hbase.util.hbck.ReplicationChecker; 132import org.apache.hadoop.hbase.util.hbck.TableIntegrityErrorHandler; 133import org.apache.hadoop.hbase.util.hbck.TableIntegrityErrorHandlerImpl; 134import org.apache.hadoop.hbase.wal.WAL; 135import org.apache.hadoop.hbase.wal.WALFactory; 136import org.apache.hadoop.hbase.wal.WALSplitter; 137import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; 138import org.apache.hadoop.hbase.zookeeper.ZKUtil; 139import org.apache.hadoop.hbase.zookeeper.ZKWatcher; 140import org.apache.hadoop.hbase.zookeeper.ZNodePaths; 141import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException; 142import org.apache.hadoop.ipc.RemoteException; 143import org.apache.hadoop.security.UserGroupInformation; 144import org.apache.hadoop.util.ReflectionUtils; 145import org.apache.hadoop.util.Tool; 146import org.apache.hadoop.util.ToolRunner; 147import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; 148import org.apache.hbase.thirdparty.com.google.common.collect.Sets; 149import org.apache.yetus.audience.InterfaceAudience; 150import org.apache.yetus.audience.InterfaceStability; 151import org.apache.zookeeper.KeeperException; 152import org.slf4j.Logger; 153import org.slf4j.LoggerFactory; 154 155import org.apache.hbase.thirdparty.com.google.common.base.Joiner; 156import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; 157import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; 158import org.apache.hbase.thirdparty.com.google.common.collect.Lists; 159import org.apache.hbase.thirdparty.com.google.common.collect.Multimap; 160import org.apache.hbase.thirdparty.com.google.common.collect.Ordering; 161import org.apache.hbase.thirdparty.com.google.common.collect.TreeMultimap; 162 163import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 164import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface; 165 166/** 167 * HBaseFsck (hbck) is a tool for checking and repairing region consistency and 168 * table integrity problems in a corrupted HBase. This tool was written for hbase-1.x. It does not 169 * work with hbase-2.x; it can read state but is not allowed to change state; i.e. effect 'repair'. 170 * See hbck2 (HBASE-19121) for a hbck tool for hbase2. 171 * 172 * <p> 173 * Region consistency checks verify that hbase:meta, region deployment on region 174 * servers and the state of data in HDFS (.regioninfo files) all are in 175 * accordance. 176 * <p> 177 * Table integrity checks verify that all possible row keys resolve to exactly 178 * one region of a table. This means there are no individual degenerate 179 * or backwards regions; no holes between regions; and that there are no 180 * overlapping regions. 181 * <p> 182 * The general repair strategy works in two phases: 183 * <ol> 184 * <li> Repair Table Integrity on HDFS. (merge or fabricate regions) 185 * <li> Repair Region Consistency with hbase:meta and assignments 186 * </ol> 187 * <p> 188 * For table integrity repairs, the tables' region directories are scanned 189 * for .regioninfo files. Each table's integrity is then verified. If there 190 * are any orphan regions (regions with no .regioninfo files) or holes, new 191 * regions are fabricated. Backwards regions are sidelined as well as empty 192 * degenerate (endkey==startkey) regions. If there are any overlapping regions, 193 * a new region is created and all data is merged into the new region. 194 * <p> 195 * Table integrity repairs deal solely with HDFS and could potentially be done 196 * offline -- the hbase region servers or master do not need to be running. 197 * This phase can eventually be used to completely reconstruct the hbase:meta table in 198 * an offline fashion. 199 * <p> 200 * Region consistency requires three conditions -- 1) valid .regioninfo file 201 * present in an HDFS region dir, 2) valid row with .regioninfo data in META, 202 * and 3) a region is deployed only at the regionserver that was assigned to 203 * with proper state in the master. 204 * <p> 205 * Region consistency repairs require hbase to be online so that hbck can 206 * contact the HBase master and region servers. The hbck#connect() method must 207 * first be called successfully. Much of the region consistency information 208 * is transient and less risky to repair. 209 * <p> 210 * If hbck is run from the command line, there are a handful of arguments that 211 * can be used to limit the kinds of repairs hbck will do. See the code in 212 * {@link #printUsageAndExit()} for more details. 213 */ 214@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS) 215@InterfaceStability.Evolving 216public class HBaseFsck extends Configured implements Closeable { 217 public static final long DEFAULT_TIME_LAG = 60000; // default value of 1 minute 218 public static final long DEFAULT_SLEEP_BEFORE_RERUN = 10000; 219 private static final int MAX_NUM_THREADS = 50; // #threads to contact regions 220 private static boolean rsSupportsOffline = true; 221 private static final int DEFAULT_OVERLAPS_TO_SIDELINE = 2; 222 private static final int DEFAULT_MAX_MERGE = 5; 223 private static final String TO_BE_LOADED = "to_be_loaded"; 224 /** 225 * Here is where hbase-1.x used to default the lock for hbck1. 226 * It puts in place a lock when it goes to write/make changes. 227 */ 228 @VisibleForTesting 229 public static final String HBCK_LOCK_FILE = "hbase-hbck.lock"; 230 private static final int DEFAULT_MAX_LOCK_FILE_ATTEMPTS = 5; 231 private static final int DEFAULT_LOCK_FILE_ATTEMPT_SLEEP_INTERVAL = 200; // milliseconds 232 private static final int DEFAULT_LOCK_FILE_ATTEMPT_MAX_SLEEP_TIME = 5000; // milliseconds 233 // We have to set the timeout value > HdfsConstants.LEASE_SOFTLIMIT_PERIOD. 234 // In HADOOP-2.6 and later, the Namenode proxy now created with custom RetryPolicy for 235 // AlreadyBeingCreatedException which is implies timeout on this operations up to 236 // HdfsConstants.LEASE_SOFTLIMIT_PERIOD (60 seconds). 237 private static final int DEFAULT_WAIT_FOR_LOCK_TIMEOUT = 80; // seconds 238 private static final int DEFAULT_MAX_CREATE_ZNODE_ATTEMPTS = 5; 239 private static final int DEFAULT_CREATE_ZNODE_ATTEMPT_SLEEP_INTERVAL = 200; // milliseconds 240 private static final int DEFAULT_CREATE_ZNODE_ATTEMPT_MAX_SLEEP_TIME = 5000; // milliseconds 241 242 /********************** 243 * Internal resources 244 **********************/ 245 private static final Logger LOG = LoggerFactory.getLogger(HBaseFsck.class.getName()); 246 private ClusterMetrics status; 247 private ClusterConnection connection; 248 private Admin admin; 249 private Table meta; 250 // threads to do ||izable tasks: retrieve data from regionservers, handle overlapping regions 251 protected ExecutorService executor; 252 private long startMillis = EnvironmentEdgeManager.currentTime(); 253 private HFileCorruptionChecker hfcc; 254 private int retcode = 0; 255 private Path HBCK_LOCK_PATH; 256 private FSDataOutputStream hbckOutFd; 257 // This lock is to prevent cleanup of balancer resources twice between 258 // ShutdownHook and the main code. We cleanup only if the connect() is 259 // successful 260 private final AtomicBoolean hbckLockCleanup = new AtomicBoolean(false); 261 262 // Unsupported options in HBase 2.0+ 263 private static final Set<String> unsupportedOptionsInV2 = Sets.newHashSet("-fix", 264 "-fixAssignments", "-fixMeta", "-fixHdfsHoles", "-fixHdfsOrphans", "-fixTableOrphans", 265 "-fixHdfsOverlaps", "-sidelineBigOverlaps", "-fixSplitParents", "-removeParents", 266 "-fixEmptyMetaCells", "-repair", "-repairHoles", "-maxOverlapsToSideline", "-maxMerge"); 267 268 /*********** 269 * Options 270 ***********/ 271 private static boolean details = false; // do we display the full report 272 private long timelag = DEFAULT_TIME_LAG; // tables whose modtime is older 273 private static boolean forceExclusive = false; // only this hbck can modify HBase 274 private boolean fixAssignments = false; // fix assignment errors? 275 private boolean fixMeta = false; // fix meta errors? 276 private boolean checkHdfs = true; // load and check fs consistency? 277 private boolean fixHdfsHoles = false; // fix fs holes? 278 private boolean fixHdfsOverlaps = false; // fix fs overlaps (risky) 279 private boolean fixHdfsOrphans = false; // fix fs holes (missing .regioninfo) 280 private boolean fixTableOrphans = false; // fix fs holes (missing .tableinfo) 281 private boolean fixVersionFile = false; // fix missing hbase.version file in hdfs 282 private boolean fixSplitParents = false; // fix lingering split parents 283 private boolean removeParents = false; // remove split parents 284 private boolean fixReferenceFiles = false; // fix lingering reference store file 285 private boolean fixHFileLinks = false; // fix lingering HFileLinks 286 private boolean fixEmptyMetaCells = false; // fix (remove) empty REGIONINFO_QUALIFIER rows 287 private boolean fixReplication = false; // fix undeleted replication queues for removed peer 288 private boolean cleanReplicationBarrier = false; // clean replication barriers of a table 289 private boolean fixAny = false; // Set to true if any of the fix is required. 290 291 // limit checking/fixes to listed tables, if empty attempt to check/fix all 292 // hbase:meta are always checked 293 private Set<TableName> tablesIncluded = new HashSet<>(); 294 private TableName cleanReplicationBarrierTable; 295 private int maxMerge = DEFAULT_MAX_MERGE; // maximum number of overlapping regions to merge 296 // maximum number of overlapping regions to sideline 297 private int maxOverlapsToSideline = DEFAULT_OVERLAPS_TO_SIDELINE; 298 private boolean sidelineBigOverlaps = false; // sideline overlaps with >maxMerge regions 299 private Path sidelineDir = null; 300 301 private boolean rerun = false; // if we tried to fix something, rerun hbck 302 private static boolean summary = false; // if we want to print less output 303 private boolean checkMetaOnly = false; 304 private boolean checkRegionBoundaries = false; 305 private boolean ignorePreCheckPermission = false; // if pre-check permission 306 307 /********* 308 * State 309 *********/ 310 final private ErrorReporter errors; 311 int fixes = 0; 312 313 /** 314 * This map contains the state of all hbck items. It maps from encoded region 315 * name to HbckInfo structure. The information contained in HbckInfo is used 316 * to detect and correct consistency (hdfs/meta/deployment) problems. 317 */ 318 private TreeMap<String, HbckInfo> regionInfoMap = new TreeMap<>(); 319 // Empty regioninfo qualifiers in hbase:meta 320 private Set<Result> emptyRegionInfoQualifiers = new HashSet<>(); 321 322 /** 323 * This map from Tablename -> TableInfo contains the structures necessary to 324 * detect table consistency problems (holes, dupes, overlaps). It is sorted 325 * to prevent dupes. 326 * 327 * If tablesIncluded is empty, this map contains all tables. 328 * Otherwise, it contains only meta tables and tables in tablesIncluded, 329 * unless checkMetaOnly is specified, in which case, it contains only 330 * the meta table 331 */ 332 private SortedMap<TableName, TableInfo> tablesInfo = new ConcurrentSkipListMap<>(); 333 334 /** 335 * When initially looking at HDFS, we attempt to find any orphaned data. 336 */ 337 private List<HbckInfo> orphanHdfsDirs = Collections.synchronizedList(new ArrayList<HbckInfo>()); 338 339 private Map<TableName, Set<String>> orphanTableDirs = new HashMap<>(); 340 private Map<TableName, TableState> tableStates = new HashMap<>(); 341 private final RetryCounterFactory lockFileRetryCounterFactory; 342 private final RetryCounterFactory createZNodeRetryCounterFactory; 343 344 private Map<TableName, Set<String>> skippedRegions = new HashMap<>(); 345 346 private ZKWatcher zkw = null; 347 private String hbckEphemeralNodePath = null; 348 private boolean hbckZodeCreated = false; 349 350 /** 351 * Constructor 352 * 353 * @param conf Configuration object 354 * @throws MasterNotRunningException if the master is not running 355 * @throws ZooKeeperConnectionException if unable to connect to ZooKeeper 356 */ 357 public HBaseFsck(Configuration conf) throws IOException, ClassNotFoundException { 358 this(conf, createThreadPool(conf)); 359 } 360 361 private static ExecutorService createThreadPool(Configuration conf) { 362 int numThreads = conf.getInt("hbasefsck.numthreads", MAX_NUM_THREADS); 363 return new ScheduledThreadPoolExecutor(numThreads, Threads.newDaemonThreadFactory("hbasefsck")); 364 } 365 366 /** 367 * Constructor 368 * 369 * @param conf 370 * Configuration object 371 * @throws MasterNotRunningException 372 * if the master is not running 373 * @throws ZooKeeperConnectionException 374 * if unable to connect to ZooKeeper 375 */ 376 public HBaseFsck(Configuration conf, ExecutorService exec) throws MasterNotRunningException, 377 ZooKeeperConnectionException, IOException, ClassNotFoundException { 378 super(conf); 379 errors = getErrorReporter(getConf()); 380 this.executor = exec; 381 lockFileRetryCounterFactory = createLockRetryCounterFactory(getConf()); 382 createZNodeRetryCounterFactory = createZnodeRetryCounterFactory(getConf()); 383 zkw = createZooKeeperWatcher(); 384 } 385 386 /** 387 * @return A retry counter factory configured for retrying lock file creation. 388 */ 389 public static RetryCounterFactory createLockRetryCounterFactory(Configuration conf) { 390 return new RetryCounterFactory( 391 conf.getInt("hbase.hbck.lockfile.attempts", DEFAULT_MAX_LOCK_FILE_ATTEMPTS), 392 conf.getInt("hbase.hbck.lockfile.attempt.sleep.interval", 393 DEFAULT_LOCK_FILE_ATTEMPT_SLEEP_INTERVAL), 394 conf.getInt("hbase.hbck.lockfile.attempt.maxsleeptime", 395 DEFAULT_LOCK_FILE_ATTEMPT_MAX_SLEEP_TIME)); 396 } 397 398 /** 399 * @return A retry counter factory configured for retrying znode creation. 400 */ 401 private static RetryCounterFactory createZnodeRetryCounterFactory(Configuration conf) { 402 return new RetryCounterFactory( 403 conf.getInt("hbase.hbck.createznode.attempts", DEFAULT_MAX_CREATE_ZNODE_ATTEMPTS), 404 conf.getInt("hbase.hbck.createznode.attempt.sleep.interval", 405 DEFAULT_CREATE_ZNODE_ATTEMPT_SLEEP_INTERVAL), 406 conf.getInt("hbase.hbck.createznode.attempt.maxsleeptime", 407 DEFAULT_CREATE_ZNODE_ATTEMPT_MAX_SLEEP_TIME)); 408 } 409 410 /** 411 * @return Return the tmp dir this tool writes too. 412 */ 413 @VisibleForTesting 414 public static Path getTmpDir(Configuration conf) throws IOException { 415 return new Path(FSUtils.getRootDir(conf), HConstants.HBASE_TEMP_DIRECTORY); 416 } 417 418 private static class FileLockCallable implements Callable<FSDataOutputStream> { 419 RetryCounter retryCounter; 420 private final Configuration conf; 421 private Path hbckLockPath = null; 422 423 public FileLockCallable(Configuration conf, RetryCounter retryCounter) { 424 this.retryCounter = retryCounter; 425 this.conf = conf; 426 } 427 428 /** 429 * @return Will be <code>null</code> unless you call {@link #call()} 430 */ 431 Path getHbckLockPath() { 432 return this.hbckLockPath; 433 } 434 435 @Override 436 public FSDataOutputStream call() throws IOException { 437 try { 438 FileSystem fs = FSUtils.getCurrentFileSystem(this.conf); 439 FsPermission defaultPerms = FSUtils.getFilePermissions(fs, this.conf, 440 HConstants.DATA_FILE_UMASK_KEY); 441 Path tmpDir = getTmpDir(conf); 442 this.hbckLockPath = new Path(tmpDir, HBCK_LOCK_FILE); 443 fs.mkdirs(tmpDir); 444 final FSDataOutputStream out = createFileWithRetries(fs, this.hbckLockPath, defaultPerms); 445 out.writeBytes(InetAddress.getLocalHost().toString()); 446 // Add a note into the file we write on why hbase2 is writing out an hbck1 lock file. 447 out.writeBytes(" Written by an hbase-2.x Master to block an " + 448 "attempt by an hbase-1.x HBCK tool making modification to state. " + 449 "See 'HBCK must match HBase server version' in the hbase refguide."); 450 out.flush(); 451 return out; 452 } catch(RemoteException e) { 453 if(AlreadyBeingCreatedException.class.getName().equals(e.getClassName())){ 454 return null; 455 } else { 456 throw e; 457 } 458 } 459 } 460 461 private FSDataOutputStream createFileWithRetries(final FileSystem fs, 462 final Path hbckLockFilePath, final FsPermission defaultPerms) 463 throws IOException { 464 IOException exception = null; 465 do { 466 try { 467 return FSUtils.create(fs, hbckLockFilePath, defaultPerms, false); 468 } catch (IOException ioe) { 469 LOG.info("Failed to create lock file " + hbckLockFilePath.getName() 470 + ", try=" + (retryCounter.getAttemptTimes() + 1) + " of " 471 + retryCounter.getMaxAttempts()); 472 LOG.debug("Failed to create lock file " + hbckLockFilePath.getName(), 473 ioe); 474 try { 475 exception = ioe; 476 retryCounter.sleepUntilNextRetry(); 477 } catch (InterruptedException ie) { 478 throw (InterruptedIOException) new InterruptedIOException( 479 "Can't create lock file " + hbckLockFilePath.getName()) 480 .initCause(ie); 481 } 482 } 483 } while (retryCounter.shouldRetry()); 484 485 throw exception; 486 } 487 } 488 489 /** 490 * This method maintains a lock using a file. If the creation fails we return null 491 * 492 * @return FSDataOutputStream object corresponding to the newly opened lock file 493 * @throws IOException if IO failure occurs 494 */ 495 public static Pair<Path, FSDataOutputStream> checkAndMarkRunningHbck(Configuration conf, 496 RetryCounter retryCounter) throws IOException { 497 FileLockCallable callable = new FileLockCallable(conf, retryCounter); 498 ExecutorService executor = Executors.newFixedThreadPool(1); 499 FutureTask<FSDataOutputStream> futureTask = new FutureTask<>(callable); 500 executor.execute(futureTask); 501 final int timeoutInSeconds = conf.getInt( 502 "hbase.hbck.lockfile.maxwaittime", DEFAULT_WAIT_FOR_LOCK_TIMEOUT); 503 FSDataOutputStream stream = null; 504 try { 505 stream = futureTask.get(timeoutInSeconds, TimeUnit.SECONDS); 506 } catch (ExecutionException ee) { 507 LOG.warn("Encountered exception when opening lock file", ee); 508 } catch (InterruptedException ie) { 509 LOG.warn("Interrupted when opening lock file", ie); 510 Thread.currentThread().interrupt(); 511 } catch (TimeoutException exception) { 512 // took too long to obtain lock 513 LOG.warn("Took more than " + timeoutInSeconds + " seconds in obtaining lock"); 514 futureTask.cancel(true); 515 } finally { 516 executor.shutdownNow(); 517 } 518 return new Pair<Path, FSDataOutputStream>(callable.getHbckLockPath(), stream); 519 } 520 521 private void unlockHbck() { 522 if (isExclusive() && hbckLockCleanup.compareAndSet(true, false)) { 523 RetryCounter retryCounter = lockFileRetryCounterFactory.create(); 524 do { 525 try { 526 IOUtils.closeQuietly(hbckOutFd); 527 FSUtils.delete(FSUtils.getCurrentFileSystem(getConf()), HBCK_LOCK_PATH, true); 528 LOG.info("Finishing hbck"); 529 return; 530 } catch (IOException ioe) { 531 LOG.info("Failed to delete " + HBCK_LOCK_PATH + ", try=" 532 + (retryCounter.getAttemptTimes() + 1) + " of " 533 + retryCounter.getMaxAttempts()); 534 LOG.debug("Failed to delete " + HBCK_LOCK_PATH, ioe); 535 try { 536 retryCounter.sleepUntilNextRetry(); 537 } catch (InterruptedException ie) { 538 Thread.currentThread().interrupt(); 539 LOG.warn("Interrupted while deleting lock file" + 540 HBCK_LOCK_PATH); 541 return; 542 } 543 } 544 } while (retryCounter.shouldRetry()); 545 } 546 } 547 548 /** 549 * To repair region consistency, one must call connect() in order to repair 550 * online state. 551 */ 552 public void connect() throws IOException { 553 554 if (isExclusive()) { 555 // Grab the lock 556 Pair<Path, FSDataOutputStream> pair = 557 checkAndMarkRunningHbck(getConf(), this.lockFileRetryCounterFactory.create()); 558 HBCK_LOCK_PATH = pair.getFirst(); 559 this.hbckOutFd = pair.getSecond(); 560 if (hbckOutFd == null) { 561 setRetCode(-1); 562 LOG.error("Another instance of hbck is fixing HBase, exiting this instance. " + 563 "[If you are sure no other instance is running, delete the lock file " + 564 HBCK_LOCK_PATH + " and rerun the tool]"); 565 throw new IOException("Duplicate hbck - Abort"); 566 } 567 568 // Make sure to cleanup the lock 569 hbckLockCleanup.set(true); 570 } 571 572 573 // Add a shutdown hook to this thread, in case user tries to 574 // kill the hbck with a ctrl-c, we want to cleanup the lock so that 575 // it is available for further calls 576 Runtime.getRuntime().addShutdownHook(new Thread() { 577 @Override 578 public void run() { 579 IOUtils.closeQuietly(HBaseFsck.this); 580 cleanupHbckZnode(); 581 unlockHbck(); 582 } 583 }); 584 585 LOG.info("Launching hbck"); 586 587 connection = (ClusterConnection)ConnectionFactory.createConnection(getConf()); 588 admin = connection.getAdmin(); 589 meta = connection.getTable(TableName.META_TABLE_NAME); 590 status = admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS, 591 Option.DEAD_SERVERS, Option.MASTER, Option.BACKUP_MASTERS, 592 Option.REGIONS_IN_TRANSITION, Option.HBASE_VERSION)); 593 } 594 595 /** 596 * Get deployed regions according to the region servers. 597 */ 598 private void loadDeployedRegions() throws IOException, InterruptedException { 599 // From the master, get a list of all known live region servers 600 Collection<ServerName> regionServers = status.getLiveServerMetrics().keySet(); 601 errors.print("Number of live region servers: " + regionServers.size()); 602 if (details) { 603 for (ServerName rsinfo: regionServers) { 604 errors.print(" " + rsinfo.getServerName()); 605 } 606 } 607 608 // From the master, get a list of all dead region servers 609 Collection<ServerName> deadRegionServers = status.getDeadServerNames(); 610 errors.print("Number of dead region servers: " + deadRegionServers.size()); 611 if (details) { 612 for (ServerName name: deadRegionServers) { 613 errors.print(" " + name); 614 } 615 } 616 617 // Print the current master name and state 618 errors.print("Master: " + status.getMasterName()); 619 620 // Print the list of all backup masters 621 Collection<ServerName> backupMasters = status.getBackupMasterNames(); 622 errors.print("Number of backup masters: " + backupMasters.size()); 623 if (details) { 624 for (ServerName name: backupMasters) { 625 errors.print(" " + name); 626 } 627 } 628 629 errors.print("Average load: " + status.getAverageLoad()); 630 errors.print("Number of requests: " + status.getRequestCount()); 631 errors.print("Number of regions: " + status.getRegionCount()); 632 633 List<RegionState> rits = status.getRegionStatesInTransition(); 634 errors.print("Number of regions in transition: " + rits.size()); 635 if (details) { 636 for (RegionState state: rits) { 637 errors.print(" " + state.toDescriptiveString()); 638 } 639 } 640 641 // Determine what's deployed 642 processRegionServers(regionServers); 643 } 644 645 /** 646 * Clear the current state of hbck. 647 */ 648 private void clearState() { 649 // Make sure regionInfo is empty before starting 650 fixes = 0; 651 regionInfoMap.clear(); 652 emptyRegionInfoQualifiers.clear(); 653 tableStates.clear(); 654 errors.clear(); 655 tablesInfo.clear(); 656 orphanHdfsDirs.clear(); 657 skippedRegions.clear(); 658 } 659 660 /** 661 * This repair method analyzes hbase data in hdfs and repairs it to satisfy 662 * the table integrity rules. HBase doesn't need to be online for this 663 * operation to work. 664 */ 665 public void offlineHdfsIntegrityRepair() throws IOException, InterruptedException { 666 // Initial pass to fix orphans. 667 if (shouldCheckHdfs() && (shouldFixHdfsOrphans() || shouldFixHdfsHoles() 668 || shouldFixHdfsOverlaps() || shouldFixTableOrphans())) { 669 LOG.info("Loading regioninfos HDFS"); 670 // if nothing is happening this should always complete in two iterations. 671 int maxIterations = getConf().getInt("hbase.hbck.integrityrepair.iterations.max", 3); 672 int curIter = 0; 673 do { 674 clearState(); // clears hbck state and reset fixes to 0 and. 675 // repair what's on HDFS 676 restoreHdfsIntegrity(); 677 curIter++;// limit the number of iterations. 678 } while (fixes > 0 && curIter <= maxIterations); 679 680 // Repairs should be done in the first iteration and verification in the second. 681 // If there are more than 2 passes, something funny has happened. 682 if (curIter > 2) { 683 if (curIter == maxIterations) { 684 LOG.warn("Exiting integrity repairs after max " + curIter + " iterations. " 685 + "Tables integrity may not be fully repaired!"); 686 } else { 687 LOG.info("Successfully exiting integrity repairs after " + curIter + " iterations"); 688 } 689 } 690 } 691 } 692 693 /** 694 * This repair method requires the cluster to be online since it contacts 695 * region servers and the masters. It makes each region's state in HDFS, in 696 * hbase:meta, and deployments consistent. 697 * 698 * @return If > 0 , number of errors detected, if < 0 there was an unrecoverable 699 * error. If 0, we have a clean hbase. 700 */ 701 public int onlineConsistencyRepair() throws IOException, KeeperException, 702 InterruptedException { 703 704 // get regions according to what is online on each RegionServer 705 loadDeployedRegions(); 706 // check whether hbase:meta is deployed and online 707 recordMetaRegion(); 708 // Check if hbase:meta is found only once and in the right place 709 if (!checkMetaRegion()) { 710 String errorMsg = "hbase:meta table is not consistent. "; 711 if (shouldFixAssignments()) { 712 errorMsg += "HBCK will try fixing it. Rerun once hbase:meta is back to consistent state."; 713 } else { 714 errorMsg += "Run HBCK with proper fix options to fix hbase:meta inconsistency."; 715 } 716 errors.reportError(errorMsg + " Exiting..."); 717 return -2; 718 } 719 // Not going with further consistency check for tables when hbase:meta itself is not consistent. 720 LOG.info("Loading regionsinfo from the hbase:meta table"); 721 boolean success = loadMetaEntries(); 722 if (!success) return -1; 723 724 // Empty cells in hbase:meta? 725 reportEmptyMetaCells(); 726 727 // Check if we have to cleanup empty REGIONINFO_QUALIFIER rows from hbase:meta 728 if (shouldFixEmptyMetaCells()) { 729 fixEmptyMetaCells(); 730 } 731 732 // get a list of all tables that have not changed recently. 733 if (!checkMetaOnly) { 734 reportTablesInFlux(); 735 } 736 737 // Get disabled tables states 738 loadTableStates(); 739 740 // load regiondirs and regioninfos from HDFS 741 if (shouldCheckHdfs()) { 742 LOG.info("Loading region directories from HDFS"); 743 loadHdfsRegionDirs(); 744 LOG.info("Loading region information from HDFS"); 745 loadHdfsRegionInfos(); 746 } 747 748 // fix the orphan tables 749 fixOrphanTables(); 750 751 LOG.info("Checking and fixing region consistency"); 752 // Check and fix consistency 753 checkAndFixConsistency(); 754 755 // Check integrity (does not fix) 756 checkIntegrity(); 757 return errors.getErrorList().size(); 758 } 759 760 /** 761 * This method maintains an ephemeral znode. If the creation fails we return false or throw 762 * exception 763 * 764 * @return true if creating znode succeeds; false otherwise 765 * @throws IOException if IO failure occurs 766 */ 767 private boolean setMasterInMaintenanceMode() throws IOException { 768 RetryCounter retryCounter = createZNodeRetryCounterFactory.create(); 769 hbckEphemeralNodePath = ZNodePaths.joinZNode( 770 zkw.getZNodePaths().masterMaintZNode, 771 "hbck-" + Long.toString(EnvironmentEdgeManager.currentTime())); 772 do { 773 try { 774 hbckZodeCreated = ZKUtil.createEphemeralNodeAndWatch(zkw, hbckEphemeralNodePath, null); 775 if (hbckZodeCreated) { 776 break; 777 } 778 } catch (KeeperException e) { 779 if (retryCounter.getAttemptTimes() >= retryCounter.getMaxAttempts()) { 780 throw new IOException("Can't create znode " + hbckEphemeralNodePath, e); 781 } 782 // fall through and retry 783 } 784 785 LOG.warn("Fail to create znode " + hbckEphemeralNodePath + ", try=" + 786 (retryCounter.getAttemptTimes() + 1) + " of " + retryCounter.getMaxAttempts()); 787 788 try { 789 retryCounter.sleepUntilNextRetry(); 790 } catch (InterruptedException ie) { 791 throw (InterruptedIOException) new InterruptedIOException( 792 "Can't create znode " + hbckEphemeralNodePath).initCause(ie); 793 } 794 } while (retryCounter.shouldRetry()); 795 return hbckZodeCreated; 796 } 797 798 private void cleanupHbckZnode() { 799 try { 800 if (zkw != null && hbckZodeCreated) { 801 ZKUtil.deleteNode(zkw, hbckEphemeralNodePath); 802 hbckZodeCreated = false; 803 } 804 } catch (KeeperException e) { 805 // Ignore 806 if (!e.code().equals(KeeperException.Code.NONODE)) { 807 LOG.warn("Delete HBCK znode " + hbckEphemeralNodePath + " failed ", e); 808 } 809 } 810 } 811 812 /** 813 * Contacts the master and prints out cluster-wide information 814 * @return 0 on success, non-zero on failure 815 */ 816 public int onlineHbck() 817 throws IOException, KeeperException, InterruptedException, ReplicationException { 818 // print hbase server version 819 errors.print("Version: " + status.getHBaseVersion()); 820 821 // Clean start 822 clearState(); 823 // Do offline check and repair first 824 offlineHdfsIntegrityRepair(); 825 offlineReferenceFileRepair(); 826 offlineHLinkFileRepair(); 827 // If Master runs maintenance tasks (such as balancer, catalog janitor, etc) during online 828 // hbck, it is likely that hbck would be misled and report transient errors. Therefore, it 829 // is better to set Master into maintenance mode during online hbck. 830 // 831 if (!setMasterInMaintenanceMode()) { 832 LOG.warn("HBCK is running while master is not in maintenance mode, you might see transient " 833 + "error. Please run HBCK multiple times to reduce the chance of transient error."); 834 } 835 836 onlineConsistencyRepair(); 837 838 if (checkRegionBoundaries) { 839 checkRegionBoundaries(); 840 } 841 842 checkAndFixReplication(); 843 844 cleanReplicationBarrier(); 845 846 // Remove the hbck znode 847 cleanupHbckZnode(); 848 849 // Remove the hbck lock 850 unlockHbck(); 851 852 // Print table summary 853 printTableSummary(tablesInfo); 854 return errors.summarize(); 855 } 856 857 public static byte[] keyOnly (byte[] b) { 858 if (b == null) 859 return b; 860 int rowlength = Bytes.toShort(b, 0); 861 byte[] result = new byte[rowlength]; 862 System.arraycopy(b, Bytes.SIZEOF_SHORT, result, 0, rowlength); 863 return result; 864 } 865 866 @Override 867 public void close() throws IOException { 868 try { 869 cleanupHbckZnode(); 870 unlockHbck(); 871 } catch (Exception io) { 872 LOG.warn(io.toString(), io); 873 } finally { 874 if (zkw != null) { 875 zkw.close(); 876 zkw = null; 877 } 878 IOUtils.closeQuietly(admin); 879 IOUtils.closeQuietly(meta); 880 IOUtils.closeQuietly(connection); 881 } 882 } 883 884 private static class RegionBoundariesInformation { 885 public byte [] regionName; 886 public byte [] metaFirstKey; 887 public byte [] metaLastKey; 888 public byte [] storesFirstKey; 889 public byte [] storesLastKey; 890 @Override 891 public String toString () { 892 return "regionName=" + Bytes.toStringBinary(regionName) + 893 "\nmetaFirstKey=" + Bytes.toStringBinary(metaFirstKey) + 894 "\nmetaLastKey=" + Bytes.toStringBinary(metaLastKey) + 895 "\nstoresFirstKey=" + Bytes.toStringBinary(storesFirstKey) + 896 "\nstoresLastKey=" + Bytes.toStringBinary(storesLastKey); 897 } 898 } 899 900 public void checkRegionBoundaries() { 901 try { 902 ByteArrayComparator comparator = new ByteArrayComparator(); 903 List<RegionInfo> regions = MetaTableAccessor.getAllRegions(connection, true); 904 final RegionBoundariesInformation currentRegionBoundariesInformation = 905 new RegionBoundariesInformation(); 906 Path hbaseRoot = FSUtils.getRootDir(getConf()); 907 for (RegionInfo regionInfo : regions) { 908 Path tableDir = FSUtils.getTableDir(hbaseRoot, regionInfo.getTable()); 909 currentRegionBoundariesInformation.regionName = regionInfo.getRegionName(); 910 // For each region, get the start and stop key from the META and compare them to the 911 // same information from the Stores. 912 Path path = new Path(tableDir, regionInfo.getEncodedName()); 913 FileSystem fs = path.getFileSystem(getConf()); 914 FileStatus[] files = fs.listStatus(path); 915 // For all the column families in this region... 916 byte[] storeFirstKey = null; 917 byte[] storeLastKey = null; 918 for (FileStatus file : files) { 919 String fileName = file.getPath().toString(); 920 fileName = fileName.substring(fileName.lastIndexOf("/") + 1); 921 if (!fileName.startsWith(".") && !fileName.endsWith("recovered.edits")) { 922 FileStatus[] storeFiles = fs.listStatus(file.getPath()); 923 // For all the stores in this column family. 924 for (FileStatus storeFile : storeFiles) { 925 HFile.Reader reader = HFile.createReader(fs, storeFile.getPath(), 926 CacheConfig.DISABLED, true, getConf()); 927 if ((reader.getFirstKey() != null) 928 && ((storeFirstKey == null) || (comparator.compare(storeFirstKey, 929 ((KeyValue.KeyOnlyKeyValue) reader.getFirstKey().get()).getKey()) > 0))) { 930 storeFirstKey = ((KeyValue.KeyOnlyKeyValue)reader.getFirstKey().get()).getKey(); 931 } 932 if ((reader.getLastKey() != null) 933 && ((storeLastKey == null) || (comparator.compare(storeLastKey, 934 ((KeyValue.KeyOnlyKeyValue)reader.getLastKey().get()).getKey())) < 0)) { 935 storeLastKey = ((KeyValue.KeyOnlyKeyValue)reader.getLastKey().get()).getKey(); 936 } 937 reader.close(); 938 } 939 } 940 } 941 currentRegionBoundariesInformation.metaFirstKey = regionInfo.getStartKey(); 942 currentRegionBoundariesInformation.metaLastKey = regionInfo.getEndKey(); 943 currentRegionBoundariesInformation.storesFirstKey = keyOnly(storeFirstKey); 944 currentRegionBoundariesInformation.storesLastKey = keyOnly(storeLastKey); 945 if (currentRegionBoundariesInformation.metaFirstKey.length == 0) 946 currentRegionBoundariesInformation.metaFirstKey = null; 947 if (currentRegionBoundariesInformation.metaLastKey.length == 0) 948 currentRegionBoundariesInformation.metaLastKey = null; 949 950 // For a region to be correct, we need the META start key to be smaller or equal to the 951 // smallest start key from all the stores, and the start key from the next META entry to 952 // be bigger than the last key from all the current stores. First region start key is null; 953 // Last region end key is null; some regions can be empty and not have any store. 954 955 boolean valid = true; 956 // Checking start key. 957 if ((currentRegionBoundariesInformation.storesFirstKey != null) 958 && (currentRegionBoundariesInformation.metaFirstKey != null)) { 959 valid = valid 960 && comparator.compare(currentRegionBoundariesInformation.storesFirstKey, 961 currentRegionBoundariesInformation.metaFirstKey) >= 0; 962 } 963 // Checking stop key. 964 if ((currentRegionBoundariesInformation.storesLastKey != null) 965 && (currentRegionBoundariesInformation.metaLastKey != null)) { 966 valid = valid 967 && comparator.compare(currentRegionBoundariesInformation.storesLastKey, 968 currentRegionBoundariesInformation.metaLastKey) < 0; 969 } 970 if (!valid) { 971 errors.reportError(ERROR_CODE.BOUNDARIES_ERROR, "Found issues with regions boundaries", 972 tablesInfo.get(regionInfo.getTable())); 973 LOG.warn("Region's boundaries not aligned between stores and META for:"); 974 LOG.warn(Objects.toString(currentRegionBoundariesInformation)); 975 } 976 } 977 } catch (IOException e) { 978 LOG.error(e.toString(), e); 979 } 980 } 981 982 /** 983 * Iterates through the list of all orphan/invalid regiondirs. 984 */ 985 private void adoptHdfsOrphans(Collection<HbckInfo> orphanHdfsDirs) throws IOException { 986 for (HbckInfo hi : orphanHdfsDirs) { 987 LOG.info("Attempting to handle orphan hdfs dir: " + hi.getHdfsRegionDir()); 988 adoptHdfsOrphan(hi); 989 } 990 } 991 992 /** 993 * Orphaned regions are regions without a .regioninfo file in them. We "adopt" 994 * these orphans by creating a new region, and moving the column families, 995 * recovered edits, WALs, into the new region dir. We determine the region 996 * startkey and endkeys by looking at all of the hfiles inside the column 997 * families to identify the min and max keys. The resulting region will 998 * likely violate table integrity but will be dealt with by merging 999 * overlapping regions. 1000 */ 1001 @SuppressWarnings("deprecation") 1002 private void adoptHdfsOrphan(HbckInfo hi) throws IOException { 1003 Path p = hi.getHdfsRegionDir(); 1004 FileSystem fs = p.getFileSystem(getConf()); 1005 FileStatus[] dirs = fs.listStatus(p); 1006 if (dirs == null) { 1007 LOG.warn("Attempt to adopt orphan hdfs region skipped because no files present in " + 1008 p + ". This dir could probably be deleted."); 1009 return ; 1010 } 1011 1012 TableName tableName = hi.getTableName(); 1013 TableInfo tableInfo = tablesInfo.get(tableName); 1014 Preconditions.checkNotNull(tableInfo, "Table '" + tableName + "' not present!"); 1015 TableDescriptor template = tableInfo.getHTD(); 1016 1017 // find min and max key values 1018 Pair<byte[],byte[]> orphanRegionRange = null; 1019 for (FileStatus cf : dirs) { 1020 String cfName= cf.getPath().getName(); 1021 // TODO Figure out what the special dirs are 1022 if (cfName.startsWith(".") || cfName.equals(HConstants.SPLIT_LOGDIR_NAME)) continue; 1023 1024 FileStatus[] hfiles = fs.listStatus(cf.getPath()); 1025 for (FileStatus hfile : hfiles) { 1026 byte[] start, end; 1027 HFile.Reader hf = null; 1028 try { 1029 hf = HFile.createReader(fs, hfile.getPath(), CacheConfig.DISABLED, true, getConf()); 1030 hf.loadFileInfo(); 1031 Optional<Cell> startKv = hf.getFirstKey(); 1032 start = CellUtil.cloneRow(startKv.get()); 1033 Optional<Cell> endKv = hf.getLastKey(); 1034 end = CellUtil.cloneRow(endKv.get()); 1035 } catch (IOException ioe) { 1036 LOG.warn("Problem reading orphan file " + hfile + ", skipping"); 1037 continue; 1038 } catch (NullPointerException ioe) { 1039 LOG.warn("Orphan file " + hfile + " is possibly corrupted HFile, skipping"); 1040 continue; 1041 } finally { 1042 if (hf != null) { 1043 hf.close(); 1044 } 1045 } 1046 1047 // expand the range to include the range of all hfiles 1048 if (orphanRegionRange == null) { 1049 // first range 1050 orphanRegionRange = new Pair<>(start, end); 1051 } else { 1052 // TODO add test 1053 1054 // expand range only if the hfile is wider. 1055 if (Bytes.compareTo(orphanRegionRange.getFirst(), start) > 0) { 1056 orphanRegionRange.setFirst(start); 1057 } 1058 if (Bytes.compareTo(orphanRegionRange.getSecond(), end) < 0 ) { 1059 orphanRegionRange.setSecond(end); 1060 } 1061 } 1062 } 1063 } 1064 if (orphanRegionRange == null) { 1065 LOG.warn("No data in dir " + p + ", sidelining data"); 1066 fixes++; 1067 sidelineRegionDir(fs, hi); 1068 return; 1069 } 1070 LOG.info("Min max keys are : [" + Bytes.toString(orphanRegionRange.getFirst()) + ", " + 1071 Bytes.toString(orphanRegionRange.getSecond()) + ")"); 1072 1073 // create new region on hdfs. move data into place. 1074 RegionInfo regionInfo = RegionInfoBuilder.newBuilder(template.getTableName()) 1075 .setStartKey(orphanRegionRange.getFirst()) 1076 .setEndKey(Bytes.add(orphanRegionRange.getSecond(), new byte[1])) 1077 .build(); 1078 LOG.info("Creating new region : " + regionInfo); 1079 HRegion region = HBaseFsckRepair.createHDFSRegionDir(getConf(), regionInfo, template); 1080 Path target = region.getRegionFileSystem().getRegionDir(); 1081 1082 // rename all the data to new region 1083 mergeRegionDirs(target, hi); 1084 fixes++; 1085 } 1086 1087 /** 1088 * This method determines if there are table integrity errors in HDFS. If 1089 * there are errors and the appropriate "fix" options are enabled, the method 1090 * will first correct orphan regions making them into legit regiondirs, and 1091 * then reload to merge potentially overlapping regions. 1092 * 1093 * @return number of table integrity errors found 1094 */ 1095 private int restoreHdfsIntegrity() throws IOException, InterruptedException { 1096 // Determine what's on HDFS 1097 LOG.info("Loading HBase regioninfo from HDFS..."); 1098 loadHdfsRegionDirs(); // populating regioninfo table. 1099 1100 int errs = errors.getErrorList().size(); 1101 // First time just get suggestions. 1102 tablesInfo = loadHdfsRegionInfos(); // update tableInfos based on region info in fs. 1103 checkHdfsIntegrity(false, false); 1104 1105 if (errors.getErrorList().size() == errs) { 1106 LOG.info("No integrity errors. We are done with this phase. Glorious."); 1107 return 0; 1108 } 1109 1110 if (shouldFixHdfsOrphans() && orphanHdfsDirs.size() > 0) { 1111 adoptHdfsOrphans(orphanHdfsDirs); 1112 // TODO optimize by incrementally adding instead of reloading. 1113 } 1114 1115 // Make sure there are no holes now. 1116 if (shouldFixHdfsHoles()) { 1117 clearState(); // this also resets # fixes. 1118 loadHdfsRegionDirs(); 1119 tablesInfo = loadHdfsRegionInfos(); // update tableInfos based on region info in fs. 1120 tablesInfo = checkHdfsIntegrity(shouldFixHdfsHoles(), false); 1121 } 1122 1123 // Now we fix overlaps 1124 if (shouldFixHdfsOverlaps()) { 1125 // second pass we fix overlaps. 1126 clearState(); // this also resets # fixes. 1127 loadHdfsRegionDirs(); 1128 tablesInfo = loadHdfsRegionInfos(); // update tableInfos based on region info in fs. 1129 tablesInfo = checkHdfsIntegrity(false, shouldFixHdfsOverlaps()); 1130 } 1131 1132 return errors.getErrorList().size(); 1133 } 1134 1135 /** 1136 * Scan all the store file names to find any lingering reference files, 1137 * which refer to some none-exiting files. If "fix" option is enabled, 1138 * any lingering reference file will be sidelined if found. 1139 * <p> 1140 * Lingering reference file prevents a region from opening. It has to 1141 * be fixed before a cluster can start properly. 1142 */ 1143 private void offlineReferenceFileRepair() throws IOException, InterruptedException { 1144 clearState(); 1145 Configuration conf = getConf(); 1146 Path hbaseRoot = FSUtils.getRootDir(conf); 1147 FileSystem fs = hbaseRoot.getFileSystem(conf); 1148 LOG.info("Computing mapping of all store files"); 1149 Map<String, Path> allFiles = FSUtils.getTableStoreFilePathMap(fs, hbaseRoot, 1150 new FSUtils.ReferenceFileFilter(fs), executor, errors); 1151 errors.print(""); 1152 LOG.info("Validating mapping using HDFS state"); 1153 for (Path path: allFiles.values()) { 1154 Path referredToFile = StoreFileInfo.getReferredToFile(path); 1155 if (fs.exists(referredToFile)) continue; // good, expected 1156 1157 // Found a lingering reference file 1158 errors.reportError(ERROR_CODE.LINGERING_REFERENCE_HFILE, 1159 "Found lingering reference file " + path); 1160 if (!shouldFixReferenceFiles()) continue; 1161 1162 // Now, trying to fix it since requested 1163 boolean success = false; 1164 String pathStr = path.toString(); 1165 1166 // A reference file path should be like 1167 // ${hbase.rootdir}/data/namespace/table_name/region_id/family_name/referred_file.region_name 1168 // Up 5 directories to get the root folder. 1169 // So the file will be sidelined to a similar folder structure. 1170 int index = pathStr.lastIndexOf(Path.SEPARATOR_CHAR); 1171 for (int i = 0; index > 0 && i < 5; i++) { 1172 index = pathStr.lastIndexOf(Path.SEPARATOR_CHAR, index - 1); 1173 } 1174 if (index > 0) { 1175 Path rootDir = getSidelineDir(); 1176 Path dst = new Path(rootDir, pathStr.substring(index + 1)); 1177 fs.mkdirs(dst.getParent()); 1178 LOG.info("Trying to sideline reference file " 1179 + path + " to " + dst); 1180 setShouldRerun(); 1181 1182 success = fs.rename(path, dst); 1183 debugLsr(dst); 1184 1185 } 1186 if (!success) { 1187 LOG.error("Failed to sideline reference file " + path); 1188 } 1189 } 1190 } 1191 1192 /** 1193 * Scan all the store file names to find any lingering HFileLink files, 1194 * which refer to some none-exiting files. If "fix" option is enabled, 1195 * any lingering HFileLink file will be sidelined if found. 1196 */ 1197 private void offlineHLinkFileRepair() throws IOException, InterruptedException { 1198 Configuration conf = getConf(); 1199 Path hbaseRoot = FSUtils.getRootDir(conf); 1200 FileSystem fs = hbaseRoot.getFileSystem(conf); 1201 LOG.info("Computing mapping of all link files"); 1202 Map<String, Path> allFiles = FSUtils 1203 .getTableStoreFilePathMap(fs, hbaseRoot, new FSUtils.HFileLinkFilter(), executor, errors); 1204 errors.print(""); 1205 1206 LOG.info("Validating mapping using HDFS state"); 1207 for (Path path : allFiles.values()) { 1208 // building HFileLink object to gather locations 1209 HFileLink actualLink = HFileLink.buildFromHFileLinkPattern(conf, path); 1210 if (actualLink.exists(fs)) continue; // good, expected 1211 1212 // Found a lingering HFileLink 1213 errors.reportError(ERROR_CODE.LINGERING_HFILELINK, "Found lingering HFileLink " + path); 1214 if (!shouldFixHFileLinks()) continue; 1215 1216 // Now, trying to fix it since requested 1217 setShouldRerun(); 1218 1219 // An HFileLink path should be like 1220 // ${hbase.rootdir}/data/namespace/table_name/region_id/family_name/linkedtable=linkedregionname-linkedhfilename 1221 // sidelineing will happen in the ${hbase.rootdir}/${sidelinedir} directory with the same folder structure. 1222 boolean success = sidelineFile(fs, hbaseRoot, path); 1223 1224 if (!success) { 1225 LOG.error("Failed to sideline HFileLink file " + path); 1226 } 1227 1228 // An HFileLink backreference path should be like 1229 // ${hbase.rootdir}/archive/data/namespace/table_name/region_id/family_name/.links-linkedhfilename 1230 // sidelineing will happen in the ${hbase.rootdir}/${sidelinedir} directory with the same folder structure. 1231 Path backRefPath = FileLink.getBackReferencesDir(HFileArchiveUtil 1232 .getStoreArchivePath(conf, HFileLink.getReferencedTableName(path.getName().toString()), 1233 HFileLink.getReferencedRegionName(path.getName().toString()), 1234 path.getParent().getName()), 1235 HFileLink.getReferencedHFileName(path.getName().toString())); 1236 success = sidelineFile(fs, hbaseRoot, backRefPath); 1237 1238 if (!success) { 1239 LOG.error("Failed to sideline HFileLink backreference file " + path); 1240 } 1241 } 1242 } 1243 1244 private boolean sidelineFile(FileSystem fs, Path hbaseRoot, Path path) throws IOException { 1245 URI uri = hbaseRoot.toUri().relativize(path.toUri()); 1246 if (uri.isAbsolute()) return false; 1247 String relativePath = uri.getPath(); 1248 Path rootDir = getSidelineDir(); 1249 Path dst = new Path(rootDir, relativePath); 1250 boolean pathCreated = fs.mkdirs(dst.getParent()); 1251 if (!pathCreated) { 1252 LOG.error("Failed to create path: " + dst.getParent()); 1253 return false; 1254 } 1255 LOG.info("Trying to sideline file " + path + " to " + dst); 1256 return fs.rename(path, dst); 1257 } 1258 1259 /** 1260 * TODO -- need to add tests for this. 1261 */ 1262 private void reportEmptyMetaCells() { 1263 errors.print("Number of empty REGIONINFO_QUALIFIER rows in hbase:meta: " + 1264 emptyRegionInfoQualifiers.size()); 1265 if (details) { 1266 for (Result r: emptyRegionInfoQualifiers) { 1267 errors.print(" " + r); 1268 } 1269 } 1270 } 1271 1272 /** 1273 * TODO -- need to add tests for this. 1274 */ 1275 private void reportTablesInFlux() { 1276 AtomicInteger numSkipped = new AtomicInteger(0); 1277 TableDescriptor[] allTables = getTables(numSkipped); 1278 errors.print("Number of Tables: " + allTables.length); 1279 if (details) { 1280 if (numSkipped.get() > 0) { 1281 errors.detail("Number of Tables in flux: " + numSkipped.get()); 1282 } 1283 for (TableDescriptor td : allTables) { 1284 errors.detail(" Table: " + td.getTableName() + "\t" + 1285 (td.isReadOnly() ? "ro" : "rw") + "\t" + 1286 (td.isMetaRegion() ? "META" : " ") + "\t" + 1287 " families: " + td.getColumnFamilyCount()); 1288 } 1289 } 1290 } 1291 1292 public ErrorReporter getErrors() { 1293 return errors; 1294 } 1295 1296 /** 1297 * Read the .regioninfo file from the file system. If there is no 1298 * .regioninfo, add it to the orphan hdfs region list. 1299 */ 1300 private void loadHdfsRegioninfo(HbckInfo hbi) throws IOException { 1301 Path regionDir = hbi.getHdfsRegionDir(); 1302 if (regionDir == null) { 1303 if (hbi.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) { 1304 // Log warning only for default/ primary replica with no region dir 1305 LOG.warn("No HDFS region dir found: " + hbi + " meta=" + hbi.metaEntry); 1306 } 1307 return; 1308 } 1309 1310 if (hbi.hdfsEntry.hri != null) { 1311 // already loaded data 1312 return; 1313 } 1314 1315 FileSystem fs = FileSystem.get(getConf()); 1316 RegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir); 1317 LOG.debug("RegionInfo read: " + hri.toString()); 1318 hbi.hdfsEntry.hri = hri; 1319 } 1320 1321 /** 1322 * Exception thrown when a integrity repair operation fails in an 1323 * unresolvable way. 1324 */ 1325 public static class RegionRepairException extends IOException { 1326 private static final long serialVersionUID = 1L; 1327 final IOException ioe; 1328 public RegionRepairException(String s, IOException ioe) { 1329 super(s); 1330 this.ioe = ioe; 1331 } 1332 } 1333 1334 /** 1335 * Populate hbi's from regionInfos loaded from file system. 1336 */ 1337 private SortedMap<TableName, TableInfo> loadHdfsRegionInfos() 1338 throws IOException, InterruptedException { 1339 tablesInfo.clear(); // regenerating the data 1340 // generate region split structure 1341 Collection<HbckInfo> hbckInfos = regionInfoMap.values(); 1342 1343 // Parallelized read of .regioninfo files. 1344 List<WorkItemHdfsRegionInfo> hbis = new ArrayList<>(hbckInfos.size()); 1345 List<Future<Void>> hbiFutures; 1346 1347 for (HbckInfo hbi : hbckInfos) { 1348 WorkItemHdfsRegionInfo work = new WorkItemHdfsRegionInfo(hbi, this, errors); 1349 hbis.add(work); 1350 } 1351 1352 // Submit and wait for completion 1353 hbiFutures = executor.invokeAll(hbis); 1354 1355 for(int i=0; i<hbiFutures.size(); i++) { 1356 WorkItemHdfsRegionInfo work = hbis.get(i); 1357 Future<Void> f = hbiFutures.get(i); 1358 try { 1359 f.get(); 1360 } catch(ExecutionException e) { 1361 LOG.warn("Failed to read .regioninfo file for region " + 1362 work.hbi.getRegionNameAsString(), e.getCause()); 1363 } 1364 } 1365 1366 Path hbaseRoot = FSUtils.getRootDir(getConf()); 1367 FileSystem fs = hbaseRoot.getFileSystem(getConf()); 1368 // serialized table info gathering. 1369 for (HbckInfo hbi: hbckInfos) { 1370 1371 if (hbi.getHdfsHRI() == null) { 1372 // was an orphan 1373 continue; 1374 } 1375 1376 1377 // get table name from hdfs, populate various HBaseFsck tables. 1378 TableName tableName = hbi.getTableName(); 1379 if (tableName == null) { 1380 // There was an entry in hbase:meta not in the HDFS? 1381 LOG.warn("tableName was null for: " + hbi); 1382 continue; 1383 } 1384 1385 TableInfo modTInfo = tablesInfo.get(tableName); 1386 if (modTInfo == null) { 1387 // only executed once per table. 1388 modTInfo = new TableInfo(tableName); 1389 tablesInfo.put(tableName, modTInfo); 1390 try { 1391 TableDescriptor htd = 1392 FSTableDescriptors.getTableDescriptorFromFs(fs, hbaseRoot, tableName); 1393 modTInfo.htds.add(htd); 1394 } catch (IOException ioe) { 1395 if (!orphanTableDirs.containsKey(tableName)) { 1396 LOG.warn("Unable to read .tableinfo from " + hbaseRoot, ioe); 1397 //should only report once for each table 1398 errors.reportError(ERROR_CODE.NO_TABLEINFO_FILE, 1399 "Unable to read .tableinfo from " + hbaseRoot + "/" + tableName); 1400 Set<String> columns = new HashSet<>(); 1401 orphanTableDirs.put(tableName, getColumnFamilyList(columns, hbi)); 1402 } 1403 } 1404 } 1405 if (!hbi.isSkipChecks()) { 1406 modTInfo.addRegionInfo(hbi); 1407 } 1408 } 1409 1410 loadTableInfosForTablesWithNoRegion(); 1411 errors.print(""); 1412 1413 return tablesInfo; 1414 } 1415 1416 /** 1417 * To get the column family list according to the column family dirs 1418 * @param columns 1419 * @param hbi 1420 * @return a set of column families 1421 * @throws IOException 1422 */ 1423 private Set<String> getColumnFamilyList(Set<String> columns, HbckInfo hbi) throws IOException { 1424 Path regionDir = hbi.getHdfsRegionDir(); 1425 FileSystem fs = regionDir.getFileSystem(getConf()); 1426 FileStatus[] subDirs = fs.listStatus(regionDir, new FSUtils.FamilyDirFilter(fs)); 1427 for (FileStatus subdir : subDirs) { 1428 String columnfamily = subdir.getPath().getName(); 1429 columns.add(columnfamily); 1430 } 1431 return columns; 1432 } 1433 1434 /** 1435 * To fabricate a .tableinfo file with following contents<br> 1436 * 1. the correct tablename <br> 1437 * 2. the correct colfamily list<br> 1438 * 3. the default properties for both {@link TableDescriptor} and {@link ColumnFamilyDescriptor}<br> 1439 * @throws IOException 1440 */ 1441 private boolean fabricateTableInfo(FSTableDescriptors fstd, TableName tableName, 1442 Set<String> columns) throws IOException { 1443 if (columns ==null || columns.isEmpty()) return false; 1444 TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName); 1445 for (String columnfamimly : columns) { 1446 builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(columnfamimly)); 1447 } 1448 fstd.createTableDescriptor(builder.build(), true); 1449 return true; 1450 } 1451 1452 /** 1453 * To fix the empty REGIONINFO_QUALIFIER rows from hbase:meta <br> 1454 * @throws IOException 1455 */ 1456 public void fixEmptyMetaCells() throws IOException { 1457 if (shouldFixEmptyMetaCells() && !emptyRegionInfoQualifiers.isEmpty()) { 1458 LOG.info("Trying to fix empty REGIONINFO_QUALIFIER hbase:meta rows."); 1459 for (Result region : emptyRegionInfoQualifiers) { 1460 deleteMetaRegion(region.getRow()); 1461 errors.getErrorList().remove(ERROR_CODE.EMPTY_META_CELL); 1462 } 1463 emptyRegionInfoQualifiers.clear(); 1464 } 1465 } 1466 1467 /** 1468 * To fix orphan table by creating a .tableinfo file under tableDir <br> 1469 * 1. if TableInfo is cached, to recover the .tableinfo accordingly <br> 1470 * 2. else create a default .tableinfo file with following items<br> 1471 * 2.1 the correct tablename <br> 1472 * 2.2 the correct colfamily list<br> 1473 * 2.3 the default properties for both {@link TableDescriptor} and {@link ColumnFamilyDescriptor}<br> 1474 * @throws IOException 1475 */ 1476 public void fixOrphanTables() throws IOException { 1477 if (shouldFixTableOrphans() && !orphanTableDirs.isEmpty()) { 1478 1479 List<TableName> tmpList = new ArrayList<>(orphanTableDirs.keySet().size()); 1480 tmpList.addAll(orphanTableDirs.keySet()); 1481 TableDescriptor[] htds = getTableDescriptors(tmpList); 1482 Iterator<Entry<TableName, Set<String>>> iter = 1483 orphanTableDirs.entrySet().iterator(); 1484 int j = 0; 1485 int numFailedCase = 0; 1486 FSTableDescriptors fstd = new FSTableDescriptors(getConf()); 1487 while (iter.hasNext()) { 1488 Entry<TableName, Set<String>> entry = 1489 iter.next(); 1490 TableName tableName = entry.getKey(); 1491 LOG.info("Trying to fix orphan table error: " + tableName); 1492 if (j < htds.length) { 1493 if (tableName.equals(htds[j].getTableName())) { 1494 TableDescriptor htd = htds[j]; 1495 LOG.info("fixing orphan table: " + tableName + " from cache"); 1496 fstd.createTableDescriptor(htd, true); 1497 j++; 1498 iter.remove(); 1499 } 1500 } else { 1501 if (fabricateTableInfo(fstd, tableName, entry.getValue())) { 1502 LOG.warn("fixing orphan table: " + tableName + " with a default .tableinfo file"); 1503 LOG.warn("Strongly recommend to modify the TableDescriptor if necessary for: " + tableName); 1504 iter.remove(); 1505 } else { 1506 LOG.error("Unable to create default .tableinfo for " + tableName + " while missing column family information"); 1507 numFailedCase++; 1508 } 1509 } 1510 fixes++; 1511 } 1512 1513 if (orphanTableDirs.isEmpty()) { 1514 // all orphanTableDirs are luckily recovered 1515 // re-run doFsck after recovering the .tableinfo file 1516 setShouldRerun(); 1517 LOG.warn("Strongly recommend to re-run manually hfsck after all orphanTableDirs being fixed"); 1518 } else if (numFailedCase > 0) { 1519 LOG.error("Failed to fix " + numFailedCase 1520 + " OrphanTables with default .tableinfo files"); 1521 } 1522 1523 } 1524 //cleanup the list 1525 orphanTableDirs.clear(); 1526 1527 } 1528 1529 /** 1530 * This borrows code from MasterFileSystem.bootstrap(). Explicitly creates it's own WAL, so be 1531 * sure to close it as well as the region when you're finished. 1532 * @param walFactoryID A unique identifier for WAL factory. Filesystem implementations will use 1533 * this ID to make a directory inside WAL directory path. 1534 * @return an open hbase:meta HRegion 1535 */ 1536 private HRegion createNewMeta(String walFactoryID) throws IOException { 1537 Path rootdir = FSUtils.getRootDir(getConf()); 1538 Configuration c = getConf(); 1539 RegionInfo metaHRI = RegionInfoBuilder.FIRST_META_REGIONINFO; 1540 TableDescriptor metaDescriptor = new FSTableDescriptors(c).get(TableName.META_TABLE_NAME); 1541 MasterFileSystem.setInfoFamilyCachingForMeta(metaDescriptor, false); 1542 // The WAL subsystem will use the default rootDir rather than the passed in rootDir 1543 // unless I pass along via the conf. 1544 Configuration confForWAL = new Configuration(c); 1545 confForWAL.set(HConstants.HBASE_DIR, rootdir.toString()); 1546 WAL wal = new WALFactory(confForWAL, walFactoryID).getWAL(metaHRI); 1547 HRegion meta = HRegion.createHRegion(metaHRI, rootdir, c, metaDescriptor, wal); 1548 MasterFileSystem.setInfoFamilyCachingForMeta(metaDescriptor, true); 1549 return meta; 1550 } 1551 1552 /** 1553 * Generate set of puts to add to new meta. This expects the tables to be 1554 * clean with no overlaps or holes. If there are any problems it returns null. 1555 * 1556 * @return An array list of puts to do in bulk, null if tables have problems 1557 */ 1558 private ArrayList<Put> generatePuts(SortedMap<TableName, TableInfo> tablesInfo) 1559 throws IOException { 1560 ArrayList<Put> puts = new ArrayList<>(); 1561 boolean hasProblems = false; 1562 for (Entry<TableName, TableInfo> e : tablesInfo.entrySet()) { 1563 TableName name = e.getKey(); 1564 1565 // skip "hbase:meta" 1566 if (name.compareTo(TableName.META_TABLE_NAME) == 0) { 1567 continue; 1568 } 1569 1570 TableInfo ti = e.getValue(); 1571 puts.add(MetaTableAccessor.makePutFromTableState( 1572 new TableState(ti.tableName, TableState.State.ENABLED), 1573 EnvironmentEdgeManager.currentTime())); 1574 for (Entry<byte[], Collection<HbckInfo>> spl : ti.sc.getStarts().asMap() 1575 .entrySet()) { 1576 Collection<HbckInfo> his = spl.getValue(); 1577 int sz = his.size(); 1578 if (sz != 1) { 1579 // problem 1580 LOG.error("Split starting at " + Bytes.toStringBinary(spl.getKey()) 1581 + " had " + sz + " regions instead of exactly 1." ); 1582 hasProblems = true; 1583 continue; 1584 } 1585 1586 // add the row directly to meta. 1587 HbckInfo hi = his.iterator().next(); 1588 RegionInfo hri = hi.getHdfsHRI(); // hi.metaEntry; 1589 Put p = MetaTableAccessor.makePutFromRegionInfo(hri, EnvironmentEdgeManager.currentTime()); 1590 puts.add(p); 1591 } 1592 } 1593 return hasProblems ? null : puts; 1594 } 1595 1596 /** 1597 * Suggest fixes for each table 1598 */ 1599 private void suggestFixes( 1600 SortedMap<TableName, TableInfo> tablesInfo) throws IOException { 1601 logParallelMerge(); 1602 for (TableInfo tInfo : tablesInfo.values()) { 1603 TableIntegrityErrorHandler handler = tInfo.new IntegrityFixSuggester(tInfo, errors); 1604 tInfo.checkRegionChain(handler); 1605 } 1606 } 1607 1608 /** 1609 * Rebuilds meta from information in hdfs/fs. Depends on configuration settings passed into 1610 * hbck constructor to point to a particular fs/dir. Assumes HBase is OFFLINE. 1611 * 1612 * @param fix flag that determines if method should attempt to fix holes 1613 * @return true if successful, false if attempt failed. 1614 */ 1615 public boolean rebuildMeta(boolean fix) throws IOException, 1616 InterruptedException { 1617 1618 // TODO check to make sure hbase is offline. (or at least the table 1619 // currently being worked on is off line) 1620 1621 // Determine what's on HDFS 1622 LOG.info("Loading HBase regioninfo from HDFS..."); 1623 loadHdfsRegionDirs(); // populating regioninfo table. 1624 1625 int errs = errors.getErrorList().size(); 1626 tablesInfo = loadHdfsRegionInfos(); // update tableInfos based on region info in fs. 1627 checkHdfsIntegrity(false, false); 1628 1629 // make sure ok. 1630 if (errors.getErrorList().size() != errs) { 1631 // While in error state, iterate until no more fixes possible 1632 while(true) { 1633 fixes = 0; 1634 suggestFixes(tablesInfo); 1635 errors.clear(); 1636 loadHdfsRegionInfos(); // update tableInfos based on region info in fs. 1637 checkHdfsIntegrity(shouldFixHdfsHoles(), shouldFixHdfsOverlaps()); 1638 1639 int errCount = errors.getErrorList().size(); 1640 1641 if (fixes == 0) { 1642 if (errCount > 0) { 1643 return false; // failed to fix problems. 1644 } else { 1645 break; // no fixes and no problems? drop out and fix stuff! 1646 } 1647 } 1648 } 1649 } 1650 1651 // we can rebuild, move old meta out of the way and start 1652 LOG.info("HDFS regioninfo's seems good. Sidelining old hbase:meta"); 1653 Path backupDir = sidelineOldMeta(); 1654 1655 LOG.info("Creating new hbase:meta"); 1656 String walFactoryId = "hbck-meta-recovery-" + RandomStringUtils.randomNumeric(8); 1657 HRegion meta = createNewMeta(walFactoryId); 1658 1659 // populate meta 1660 List<Put> puts = generatePuts(tablesInfo); 1661 if (puts == null) { 1662 LOG.error(HBaseMarkers.FATAL, "Problem encountered when creating new hbase:meta " 1663 + "entries. You may need to restore the previously sidelined hbase:meta"); 1664 return false; 1665 } 1666 meta.batchMutate(puts.toArray(new Put[puts.size()]), HConstants.NO_NONCE, HConstants.NO_NONCE); 1667 meta.close(); 1668 if (meta.getWAL() != null) { 1669 meta.getWAL().close(); 1670 } 1671 // clean up the temporary hbck meta recovery WAL directory 1672 removeHBCKMetaRecoveryWALDir(walFactoryId); 1673 LOG.info("Success! hbase:meta table rebuilt."); 1674 LOG.info("Old hbase:meta is moved into " + backupDir); 1675 return true; 1676 } 1677 1678 /** 1679 * Removes the empty Meta recovery WAL directory. 1680 * @param walFactoryId A unique identifier for WAL factory which was used by Filesystem to make a 1681 * Meta recovery WAL directory inside WAL directory path. 1682 */ 1683 private void removeHBCKMetaRecoveryWALDir(String walFactoryId) throws IOException { 1684 Path rootdir = FSUtils.getRootDir(getConf()); 1685 Path walLogDir = new Path(new Path(rootdir, HConstants.HREGION_LOGDIR_NAME), walFactoryId); 1686 FileSystem fs = FSUtils.getCurrentFileSystem(getConf()); 1687 FileStatus[] walFiles = FSUtils.listStatus(fs, walLogDir, null); 1688 if (walFiles == null || walFiles.length == 0) { 1689 LOG.info("HBCK meta recovery WAL directory is empty, removing it now."); 1690 if (!FSUtils.deleteDirectory(fs, walLogDir)) { 1691 LOG.warn("Couldn't clear the HBCK Meta recovery WAL directory " + walLogDir); 1692 } 1693 } 1694 } 1695 1696 /** 1697 * Log an appropriate message about whether or not overlapping merges are computed in parallel. 1698 */ 1699 private void logParallelMerge() { 1700 if (getConf().getBoolean("hbasefsck.overlap.merge.parallel", true)) { 1701 LOG.info("Handling overlap merges in parallel. set hbasefsck.overlap.merge.parallel to" + 1702 " false to run serially."); 1703 } else { 1704 LOG.info("Handling overlap merges serially. set hbasefsck.overlap.merge.parallel to" + 1705 " true to run in parallel."); 1706 } 1707 } 1708 1709 private SortedMap<TableName, TableInfo> checkHdfsIntegrity(boolean fixHoles, 1710 boolean fixOverlaps) throws IOException { 1711 LOG.info("Checking HBase region split map from HDFS data..."); 1712 logParallelMerge(); 1713 for (TableInfo tInfo : tablesInfo.values()) { 1714 TableIntegrityErrorHandler handler; 1715 if (fixHoles || fixOverlaps) { 1716 handler = tInfo.new HDFSIntegrityFixer(tInfo, errors, getConf(), 1717 fixHoles, fixOverlaps); 1718 } else { 1719 handler = tInfo.new IntegrityFixSuggester(tInfo, errors); 1720 } 1721 if (!tInfo.checkRegionChain(handler)) { 1722 // should dump info as well. 1723 errors.report("Found inconsistency in table " + tInfo.getName()); 1724 } 1725 } 1726 return tablesInfo; 1727 } 1728 1729 private Path getSidelineDir() throws IOException { 1730 if (sidelineDir == null) { 1731 Path hbaseDir = FSUtils.getRootDir(getConf()); 1732 Path hbckDir = new Path(hbaseDir, HConstants.HBCK_SIDELINEDIR_NAME); 1733 sidelineDir = new Path(hbckDir, hbaseDir.getName() + "-" 1734 + startMillis); 1735 } 1736 return sidelineDir; 1737 } 1738 1739 /** 1740 * Sideline a region dir (instead of deleting it) 1741 */ 1742 Path sidelineRegionDir(FileSystem fs, HbckInfo hi) throws IOException { 1743 return sidelineRegionDir(fs, null, hi); 1744 } 1745 1746 /** 1747 * Sideline a region dir (instead of deleting it) 1748 * 1749 * @param parentDir if specified, the region will be sidelined to folder like 1750 * {@literal .../parentDir/<table name>/<region name>}. The purpose is to group together 1751 * similar regions sidelined, for example, those regions should be bulk loaded back later 1752 * on. If NULL, it is ignored. 1753 */ 1754 Path sidelineRegionDir(FileSystem fs, 1755 String parentDir, HbckInfo hi) throws IOException { 1756 TableName tableName = hi.getTableName(); 1757 Path regionDir = hi.getHdfsRegionDir(); 1758 1759 if (!fs.exists(regionDir)) { 1760 LOG.warn("No previous " + regionDir + " exists. Continuing."); 1761 return null; 1762 } 1763 1764 Path rootDir = getSidelineDir(); 1765 if (parentDir != null) { 1766 rootDir = new Path(rootDir, parentDir); 1767 } 1768 Path sidelineTableDir= FSUtils.getTableDir(rootDir, tableName); 1769 Path sidelineRegionDir = new Path(sidelineTableDir, regionDir.getName()); 1770 fs.mkdirs(sidelineRegionDir); 1771 boolean success = false; 1772 FileStatus[] cfs = fs.listStatus(regionDir); 1773 if (cfs == null) { 1774 LOG.info("Region dir is empty: " + regionDir); 1775 } else { 1776 for (FileStatus cf : cfs) { 1777 Path src = cf.getPath(); 1778 Path dst = new Path(sidelineRegionDir, src.getName()); 1779 if (fs.isFile(src)) { 1780 // simple file 1781 success = fs.rename(src, dst); 1782 if (!success) { 1783 String msg = "Unable to rename file " + src + " to " + dst; 1784 LOG.error(msg); 1785 throw new IOException(msg); 1786 } 1787 continue; 1788 } 1789 1790 // is a directory. 1791 fs.mkdirs(dst); 1792 1793 LOG.info("Sidelining files from " + src + " into containing region " + dst); 1794 // FileSystem.rename is inconsistent with directories -- if the 1795 // dst (foo/a) exists and is a dir, and the src (foo/b) is a dir, 1796 // it moves the src into the dst dir resulting in (foo/a/b). If 1797 // the dst does not exist, and the src a dir, src becomes dst. (foo/b) 1798 FileStatus[] hfiles = fs.listStatus(src); 1799 if (hfiles != null && hfiles.length > 0) { 1800 for (FileStatus hfile : hfiles) { 1801 success = fs.rename(hfile.getPath(), dst); 1802 if (!success) { 1803 String msg = "Unable to rename file " + src + " to " + dst; 1804 LOG.error(msg); 1805 throw new IOException(msg); 1806 } 1807 } 1808 } 1809 LOG.debug("Sideline directory contents:"); 1810 debugLsr(sidelineRegionDir); 1811 } 1812 } 1813 1814 LOG.info("Removing old region dir: " + regionDir); 1815 success = fs.delete(regionDir, true); 1816 if (!success) { 1817 String msg = "Unable to delete dir " + regionDir; 1818 LOG.error(msg); 1819 throw new IOException(msg); 1820 } 1821 return sidelineRegionDir; 1822 } 1823 1824 /** 1825 * Side line an entire table. 1826 */ 1827 void sidelineTable(FileSystem fs, TableName tableName, Path hbaseDir, 1828 Path backupHbaseDir) throws IOException { 1829 Path tableDir = FSUtils.getTableDir(hbaseDir, tableName); 1830 if (fs.exists(tableDir)) { 1831 Path backupTableDir= FSUtils.getTableDir(backupHbaseDir, tableName); 1832 fs.mkdirs(backupTableDir.getParent()); 1833 boolean success = fs.rename(tableDir, backupTableDir); 1834 if (!success) { 1835 throw new IOException("Failed to move " + tableName + " from " 1836 + tableDir + " to " + backupTableDir); 1837 } 1838 } else { 1839 LOG.info("No previous " + tableName + " exists. Continuing."); 1840 } 1841 } 1842 1843 /** 1844 * @return Path to backup of original directory 1845 */ 1846 Path sidelineOldMeta() throws IOException { 1847 // put current hbase:meta aside. 1848 Path hbaseDir = FSUtils.getRootDir(getConf()); 1849 FileSystem fs = hbaseDir.getFileSystem(getConf()); 1850 Path backupDir = getSidelineDir(); 1851 fs.mkdirs(backupDir); 1852 1853 try { 1854 sidelineTable(fs, TableName.META_TABLE_NAME, hbaseDir, backupDir); 1855 } catch (IOException e) { 1856 LOG.error(HBaseMarkers.FATAL, "... failed to sideline meta. Currently in " 1857 + "inconsistent state. To restore try to rename hbase:meta in " + 1858 backupDir.getName() + " to " + hbaseDir.getName() + ".", e); 1859 throw e; // throw original exception 1860 } 1861 return backupDir; 1862 } 1863 1864 /** 1865 * Load the list of disabled tables in ZK into local set. 1866 * @throws ZooKeeperConnectionException 1867 * @throws IOException 1868 */ 1869 private void loadTableStates() 1870 throws IOException { 1871 tableStates = MetaTableAccessor.getTableStates(connection); 1872 // Add hbase:meta so this tool keeps working. In hbase2, meta is always enabled though it 1873 // has no entry in the table states. HBCK doesn't work right w/ hbase2 but just do this in 1874 // meantime. 1875 this.tableStates.put(TableName.META_TABLE_NAME, 1876 new TableState(TableName.META_TABLE_NAME, TableState.State.ENABLED)); 1877 } 1878 1879 /** 1880 * Check if the specified region's table is disabled. 1881 * @param tableName table to check status of 1882 */ 1883 private boolean isTableDisabled(TableName tableName) { 1884 return tableStates.containsKey(tableName) 1885 && tableStates.get(tableName) 1886 .inStates(TableState.State.DISABLED, TableState.State.DISABLING); 1887 } 1888 1889 /** 1890 * Scan HDFS for all regions, recording their information into 1891 * regionInfoMap 1892 */ 1893 public void loadHdfsRegionDirs() throws IOException, InterruptedException { 1894 Path rootDir = FSUtils.getRootDir(getConf()); 1895 FileSystem fs = rootDir.getFileSystem(getConf()); 1896 1897 // list all tables from HDFS 1898 List<FileStatus> tableDirs = Lists.newArrayList(); 1899 1900 boolean foundVersionFile = fs.exists(new Path(rootDir, HConstants.VERSION_FILE_NAME)); 1901 1902 List<Path> paths = FSUtils.getTableDirs(fs, rootDir); 1903 for (Path path : paths) { 1904 TableName tableName = FSUtils.getTableName(path); 1905 if ((!checkMetaOnly && 1906 isTableIncluded(tableName)) || 1907 tableName.equals(TableName.META_TABLE_NAME)) { 1908 tableDirs.add(fs.getFileStatus(path)); 1909 } 1910 } 1911 1912 // verify that version file exists 1913 if (!foundVersionFile) { 1914 errors.reportError(ERROR_CODE.NO_VERSION_FILE, 1915 "Version file does not exist in root dir " + rootDir); 1916 if (shouldFixVersionFile()) { 1917 LOG.info("Trying to create a new " + HConstants.VERSION_FILE_NAME 1918 + " file."); 1919 setShouldRerun(); 1920 FSUtils.setVersion(fs, rootDir, getConf().getInt( 1921 HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000), getConf().getInt( 1922 HConstants.VERSION_FILE_WRITE_ATTEMPTS, 1923 HConstants.DEFAULT_VERSION_FILE_WRITE_ATTEMPTS)); 1924 } 1925 } 1926 1927 // Avoid multithreading at table-level because already multithreaded internally at 1928 // region-level. Additionally multithreading at table-level can lead to deadlock 1929 // if there are many tables in the cluster. Since there are a limited # of threads 1930 // in the executor's thread pool and if we multithread at the table-level by putting 1931 // WorkItemHdfsDir callables into the executor, then we will have some threads in the 1932 // executor tied up solely in waiting for the tables' region-level calls to complete. 1933 // If there are enough tables then there will be no actual threads in the pool left 1934 // for the region-level callables to be serviced. 1935 for (FileStatus tableDir : tableDirs) { 1936 LOG.debug("Loading region dirs from " +tableDir.getPath()); 1937 WorkItemHdfsDir item = new WorkItemHdfsDir(fs, errors, tableDir); 1938 try { 1939 item.call(); 1940 } catch (ExecutionException e) { 1941 LOG.warn("Could not completely load table dir " + 1942 tableDir.getPath(), e.getCause()); 1943 } 1944 } 1945 errors.print(""); 1946 } 1947 1948 /** 1949 * Record the location of the hbase:meta region as found in ZooKeeper. 1950 */ 1951 private boolean recordMetaRegion() throws IOException { 1952 RegionLocations rl = connection.locateRegion(TableName.META_TABLE_NAME, 1953 HConstants.EMPTY_START_ROW, false, false); 1954 if (rl == null) { 1955 errors.reportError(ERROR_CODE.NULL_META_REGION, 1956 "META region was not found in ZooKeeper"); 1957 return false; 1958 } 1959 for (HRegionLocation metaLocation : rl.getRegionLocations()) { 1960 // Check if Meta region is valid and existing 1961 if (metaLocation == null ) { 1962 errors.reportError(ERROR_CODE.NULL_META_REGION, 1963 "META region location is null"); 1964 return false; 1965 } 1966 if (metaLocation.getRegionInfo() == null) { 1967 errors.reportError(ERROR_CODE.NULL_META_REGION, 1968 "META location regionInfo is null"); 1969 return false; 1970 } 1971 if (metaLocation.getHostname() == null) { 1972 errors.reportError(ERROR_CODE.NULL_META_REGION, 1973 "META location hostName is null"); 1974 return false; 1975 } 1976 ServerName sn = metaLocation.getServerName(); 1977 MetaEntry m = new MetaEntry(metaLocation.getRegionInfo(), sn, EnvironmentEdgeManager.currentTime()); 1978 HbckInfo hbckInfo = regionInfoMap.get(metaLocation.getRegionInfo().getEncodedName()); 1979 if (hbckInfo == null) { 1980 regionInfoMap.put(metaLocation.getRegionInfo().getEncodedName(), new HbckInfo(m)); 1981 } else { 1982 hbckInfo.metaEntry = m; 1983 } 1984 } 1985 return true; 1986 } 1987 1988 private ZKWatcher createZooKeeperWatcher() throws IOException { 1989 return new ZKWatcher(getConf(), "hbase Fsck", new Abortable() { 1990 @Override 1991 public void abort(String why, Throwable e) { 1992 LOG.error(why, e); 1993 System.exit(1); 1994 } 1995 1996 @Override 1997 public boolean isAborted() { 1998 return false; 1999 } 2000 2001 }); 2002 } 2003 2004 private ServerName getMetaRegionServerName(int replicaId) 2005 throws IOException, KeeperException { 2006 return new MetaTableLocator().getMetaRegionLocation(zkw, replicaId); 2007 } 2008 2009 /** 2010 * Contacts each regionserver and fetches metadata about regions. 2011 * @param regionServerList - the list of region servers to connect to 2012 * @throws IOException if a remote or network exception occurs 2013 */ 2014 void processRegionServers(Collection<ServerName> regionServerList) 2015 throws IOException, InterruptedException { 2016 2017 List<WorkItemRegion> workItems = new ArrayList<>(regionServerList.size()); 2018 List<Future<Void>> workFutures; 2019 2020 // loop to contact each region server in parallel 2021 for (ServerName rsinfo: regionServerList) { 2022 workItems.add(new WorkItemRegion(this, rsinfo, errors, connection)); 2023 } 2024 2025 workFutures = executor.invokeAll(workItems); 2026 2027 for(int i=0; i<workFutures.size(); i++) { 2028 WorkItemRegion item = workItems.get(i); 2029 Future<Void> f = workFutures.get(i); 2030 try { 2031 f.get(); 2032 } catch(ExecutionException e) { 2033 LOG.warn("Could not process regionserver " + item.rsinfo.getHostAndPort(), 2034 e.getCause()); 2035 } 2036 } 2037 } 2038 2039 /** 2040 * Check consistency of all regions that have been found in previous phases. 2041 */ 2042 private void checkAndFixConsistency() 2043 throws IOException, KeeperException, InterruptedException { 2044 // Divide the checks in two phases. One for default/primary replicas and another 2045 // for the non-primary ones. Keeps code cleaner this way. 2046 2047 List<CheckRegionConsistencyWorkItem> workItems = new ArrayList<>(regionInfoMap.size()); 2048 for (java.util.Map.Entry<String, HbckInfo> e: regionInfoMap.entrySet()) { 2049 if (e.getValue().getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) { 2050 workItems.add(new CheckRegionConsistencyWorkItem(e.getKey(), e.getValue())); 2051 } 2052 } 2053 checkRegionConsistencyConcurrently(workItems); 2054 2055 boolean prevHdfsCheck = shouldCheckHdfs(); 2056 setCheckHdfs(false); //replicas don't have any hdfs data 2057 // Run a pass over the replicas and fix any assignment issues that exist on the currently 2058 // deployed/undeployed replicas. 2059 List<CheckRegionConsistencyWorkItem> replicaWorkItems = new ArrayList<>(regionInfoMap.size()); 2060 for (java.util.Map.Entry<String, HbckInfo> e: regionInfoMap.entrySet()) { 2061 if (e.getValue().getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) { 2062 replicaWorkItems.add(new CheckRegionConsistencyWorkItem(e.getKey(), e.getValue())); 2063 } 2064 } 2065 checkRegionConsistencyConcurrently(replicaWorkItems); 2066 setCheckHdfs(prevHdfsCheck); 2067 2068 // If some regions is skipped during checkRegionConsistencyConcurrently() phase, we might 2069 // not get accurate state of the hbase if continuing. The config here allows users to tune 2070 // the tolerance of number of skipped region. 2071 // TODO: evaluate the consequence to continue the hbck operation without config. 2072 int terminateThreshold = getConf().getInt("hbase.hbck.skipped.regions.limit", 0); 2073 int numOfSkippedRegions = skippedRegions.size(); 2074 if (numOfSkippedRegions > 0 && numOfSkippedRegions > terminateThreshold) { 2075 throw new IOException(numOfSkippedRegions 2076 + " region(s) could not be checked or repaired. See logs for detail."); 2077 } 2078 2079 if (shouldCheckHdfs()) { 2080 checkAndFixTableStates(); 2081 } 2082 } 2083 2084 /** 2085 * Check consistency of all regions using mulitple threads concurrently. 2086 */ 2087 private void checkRegionConsistencyConcurrently( 2088 final List<CheckRegionConsistencyWorkItem> workItems) 2089 throws IOException, KeeperException, InterruptedException { 2090 if (workItems.isEmpty()) { 2091 return; // nothing to check 2092 } 2093 2094 List<Future<Void>> workFutures = executor.invokeAll(workItems); 2095 for(Future<Void> f: workFutures) { 2096 try { 2097 f.get(); 2098 } catch(ExecutionException e1) { 2099 LOG.warn("Could not check region consistency " , e1.getCause()); 2100 if (e1.getCause() instanceof IOException) { 2101 throw (IOException)e1.getCause(); 2102 } else if (e1.getCause() instanceof KeeperException) { 2103 throw (KeeperException)e1.getCause(); 2104 } else if (e1.getCause() instanceof InterruptedException) { 2105 throw (InterruptedException)e1.getCause(); 2106 } else { 2107 throw new IOException(e1.getCause()); 2108 } 2109 } 2110 } 2111 } 2112 2113 class CheckRegionConsistencyWorkItem implements Callable<Void> { 2114 private final String key; 2115 private final HbckInfo hbi; 2116 2117 CheckRegionConsistencyWorkItem(String key, HbckInfo hbi) { 2118 this.key = key; 2119 this.hbi = hbi; 2120 } 2121 2122 @Override 2123 public synchronized Void call() throws Exception { 2124 try { 2125 checkRegionConsistency(key, hbi); 2126 } catch (Exception e) { 2127 // If the region is non-META region, skip this region and send warning/error message; if 2128 // the region is META region, we should not continue. 2129 LOG.warn("Unable to complete check or repair the region '" + hbi.getRegionNameAsString() 2130 + "'.", e); 2131 if (hbi.getHdfsHRI().isMetaRegion()) { 2132 throw e; 2133 } 2134 LOG.warn("Skip region '" + hbi.getRegionNameAsString() + "'"); 2135 addSkippedRegion(hbi); 2136 } 2137 return null; 2138 } 2139 } 2140 2141 private void addSkippedRegion(final HbckInfo hbi) { 2142 Set<String> skippedRegionNames = skippedRegions.get(hbi.getTableName()); 2143 if (skippedRegionNames == null) { 2144 skippedRegionNames = new HashSet<>(); 2145 } 2146 skippedRegionNames.add(hbi.getRegionNameAsString()); 2147 skippedRegions.put(hbi.getTableName(), skippedRegionNames); 2148 } 2149 2150 /** 2151 * Check and fix table states, assumes full info available: 2152 * - tableInfos 2153 * - empty tables loaded 2154 */ 2155 private void checkAndFixTableStates() throws IOException { 2156 // first check dangling states 2157 for (Entry<TableName, TableState> entry : tableStates.entrySet()) { 2158 TableName tableName = entry.getKey(); 2159 TableState tableState = entry.getValue(); 2160 TableInfo tableInfo = tablesInfo.get(tableName); 2161 if (isTableIncluded(tableName) 2162 && !tableName.isSystemTable() 2163 && tableInfo == null) { 2164 if (fixMeta) { 2165 MetaTableAccessor.deleteTableState(connection, tableName); 2166 TableState state = MetaTableAccessor.getTableState(connection, tableName); 2167 if (state != null) { 2168 errors.reportError(ERROR_CODE.ORPHAN_TABLE_STATE, 2169 tableName + " unable to delete dangling table state " + tableState); 2170 } 2171 } else if (!checkMetaOnly) { 2172 // dangling table state in meta if checkMetaOnly is false. If checkMetaOnly is 2173 // true, tableInfo will be null as tablesInfo are not polulated for all tables from hdfs 2174 errors.reportError(ERROR_CODE.ORPHAN_TABLE_STATE, 2175 tableName + " has dangling table state " + tableState); 2176 } 2177 } 2178 } 2179 // check that all tables have states 2180 for (TableName tableName : tablesInfo.keySet()) { 2181 if (isTableIncluded(tableName) && !tableStates.containsKey(tableName)) { 2182 if (fixMeta) { 2183 MetaTableAccessor.updateTableState(connection, tableName, TableState.State.ENABLED); 2184 TableState newState = MetaTableAccessor.getTableState(connection, tableName); 2185 if (newState == null) { 2186 errors.reportError(ERROR_CODE.NO_TABLE_STATE, 2187 "Unable to change state for table " + tableName + " in meta "); 2188 } 2189 } else { 2190 errors.reportError(ERROR_CODE.NO_TABLE_STATE, 2191 tableName + " has no state in meta "); 2192 } 2193 } 2194 } 2195 } 2196 2197 private void preCheckPermission() throws IOException, AccessDeniedException { 2198 if (shouldIgnorePreCheckPermission()) { 2199 return; 2200 } 2201 2202 Path hbaseDir = FSUtils.getRootDir(getConf()); 2203 FileSystem fs = hbaseDir.getFileSystem(getConf()); 2204 UserProvider userProvider = UserProvider.instantiate(getConf()); 2205 UserGroupInformation ugi = userProvider.getCurrent().getUGI(); 2206 FileStatus[] files = fs.listStatus(hbaseDir); 2207 for (FileStatus file : files) { 2208 try { 2209 FSUtils.checkAccess(ugi, file, FsAction.WRITE); 2210 } catch (AccessDeniedException ace) { 2211 LOG.warn("Got AccessDeniedException when preCheckPermission ", ace); 2212 errors.reportError(ERROR_CODE.WRONG_USAGE, "Current user " + ugi.getUserName() 2213 + " does not have write perms to " + file.getPath() 2214 + ". Please rerun hbck as hdfs user " + file.getOwner()); 2215 throw ace; 2216 } 2217 } 2218 } 2219 2220 /** 2221 * Deletes region from meta table 2222 */ 2223 private void deleteMetaRegion(HbckInfo hi) throws IOException { 2224 deleteMetaRegion(hi.metaEntry.getRegionName()); 2225 } 2226 2227 /** 2228 * Deletes region from meta table 2229 */ 2230 private void deleteMetaRegion(byte[] metaKey) throws IOException { 2231 Delete d = new Delete(metaKey); 2232 meta.delete(d); 2233 LOG.info("Deleted " + Bytes.toString(metaKey) + " from META" ); 2234 } 2235 2236 /** 2237 * Reset the split parent region info in meta table 2238 */ 2239 private void resetSplitParent(HbckInfo hi) throws IOException { 2240 RowMutations mutations = new RowMutations(hi.metaEntry.getRegionName()); 2241 Delete d = new Delete(hi.metaEntry.getRegionName()); 2242 d.addColumn(HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER); 2243 d.addColumn(HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER); 2244 mutations.add(d); 2245 2246 RegionInfo hri = RegionInfoBuilder.newBuilder(hi.metaEntry) 2247 .setOffline(false) 2248 .setSplit(false) 2249 .build(); 2250 Put p = MetaTableAccessor.makePutFromRegionInfo(hri, EnvironmentEdgeManager.currentTime()); 2251 mutations.add(p); 2252 2253 meta.mutateRow(mutations); 2254 LOG.info("Reset split parent " + hi.metaEntry.getRegionNameAsString() + " in META" ); 2255 } 2256 2257 /** 2258 * This backwards-compatibility wrapper for permanently offlining a region 2259 * that should not be alive. If the region server does not support the 2260 * "offline" method, it will use the closest unassign method instead. This 2261 * will basically work until one attempts to disable or delete the affected 2262 * table. The problem has to do with in-memory only master state, so 2263 * restarting the HMaster or failing over to another should fix this. 2264 */ 2265 private void offline(byte[] regionName) throws IOException { 2266 String regionString = Bytes.toStringBinary(regionName); 2267 if (!rsSupportsOffline) { 2268 LOG.warn("Using unassign region " + regionString 2269 + " instead of using offline method, you should" 2270 + " restart HMaster after these repairs"); 2271 admin.unassign(regionName, true); 2272 return; 2273 } 2274 2275 // first time we assume the rs's supports #offline. 2276 try { 2277 LOG.info("Offlining region " + regionString); 2278 admin.offline(regionName); 2279 } catch (IOException ioe) { 2280 String notFoundMsg = "java.lang.NoSuchMethodException: " + 2281 "org.apache.hadoop.hbase.master.HMaster.offline([B)"; 2282 if (ioe.getMessage().contains(notFoundMsg)) { 2283 LOG.warn("Using unassign region " + regionString 2284 + " instead of using offline method, you should" 2285 + " restart HMaster after these repairs"); 2286 rsSupportsOffline = false; // in the future just use unassign 2287 admin.unassign(regionName, true); 2288 return; 2289 } 2290 throw ioe; 2291 } 2292 } 2293 2294 private void undeployRegions(HbckInfo hi) throws IOException, InterruptedException { 2295 undeployRegionsForHbi(hi); 2296 // undeploy replicas of the region (but only if the method is invoked for the primary) 2297 if (hi.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) { 2298 return; 2299 } 2300 int numReplicas = admin.getTableDescriptor(hi.getTableName()).getRegionReplication(); 2301 for (int i = 1; i < numReplicas; i++) { 2302 if (hi.getPrimaryHRIForDeployedReplica() == null) continue; 2303 RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica( 2304 hi.getPrimaryHRIForDeployedReplica(), i); 2305 HbckInfo h = regionInfoMap.get(hri.getEncodedName()); 2306 if (h != null) { 2307 undeployRegionsForHbi(h); 2308 //set skip checks; we undeployed it, and we don't want to evaluate this anymore 2309 //in consistency checks 2310 h.setSkipChecks(true); 2311 } 2312 } 2313 } 2314 2315 private void undeployRegionsForHbi(HbckInfo hi) throws IOException, InterruptedException { 2316 for (OnlineEntry rse : hi.deployedEntries) { 2317 LOG.debug("Undeploy region " + rse.hri + " from " + rse.hsa); 2318 try { 2319 HBaseFsckRepair.closeRegionSilentlyAndWait(connection, rse.hsa, rse.hri); 2320 offline(rse.hri.getRegionName()); 2321 } catch (IOException ioe) { 2322 LOG.warn("Got exception when attempting to offline region " 2323 + Bytes.toString(rse.hri.getRegionName()), ioe); 2324 } 2325 } 2326 } 2327 2328 /** 2329 * Attempts to undeploy a region from a region server based in information in 2330 * META. Any operations that modify the file system should make sure that 2331 * its corresponding region is not deployed to prevent data races. 2332 * 2333 * A separate call is required to update the master in-memory region state 2334 * kept in the AssignementManager. Because disable uses this state instead of 2335 * that found in META, we can't seem to cleanly disable/delete tables that 2336 * have been hbck fixed. When used on a version of HBase that does not have 2337 * the offline ipc call exposed on the master (<0.90.5, <0.92.0) a master 2338 * restart or failover may be required. 2339 */ 2340 private void closeRegion(HbckInfo hi) throws IOException, InterruptedException { 2341 if (hi.metaEntry == null && hi.hdfsEntry == null) { 2342 undeployRegions(hi); 2343 return; 2344 } 2345 2346 // get assignment info and hregioninfo from meta. 2347 Get get = new Get(hi.getRegionName()); 2348 get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); 2349 get.addColumn(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER); 2350 get.addColumn(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER); 2351 // also get the locations of the replicas to close if the primary region is being closed 2352 if (hi.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) { 2353 int numReplicas = admin.getTableDescriptor(hi.getTableName()).getRegionReplication(); 2354 for (int i = 0; i < numReplicas; i++) { 2355 get.addColumn(HConstants.CATALOG_FAMILY, MetaTableAccessor.getServerColumn(i)); 2356 get.addColumn(HConstants.CATALOG_FAMILY, MetaTableAccessor.getStartCodeColumn(i)); 2357 } 2358 } 2359 Result r = meta.get(get); 2360 RegionLocations rl = MetaTableAccessor.getRegionLocations(r); 2361 if (rl == null) { 2362 LOG.warn("Unable to close region " + hi.getRegionNameAsString() + 2363 " since meta does not have handle to reach it"); 2364 return; 2365 } 2366 for (HRegionLocation h : rl.getRegionLocations()) { 2367 ServerName serverName = h.getServerName(); 2368 if (serverName == null) { 2369 errors.reportError("Unable to close region " 2370 + hi.getRegionNameAsString() + " because meta does not " 2371 + "have handle to reach it."); 2372 continue; 2373 } 2374 RegionInfo hri = h.getRegionInfo(); 2375 if (hri == null) { 2376 LOG.warn("Unable to close region " + hi.getRegionNameAsString() 2377 + " because hbase:meta had invalid or missing " 2378 + HConstants.CATALOG_FAMILY_STR + ":" 2379 + Bytes.toString(HConstants.REGIONINFO_QUALIFIER) 2380 + " qualifier value."); 2381 continue; 2382 } 2383 // close the region -- close files and remove assignment 2384 HBaseFsckRepair.closeRegionSilentlyAndWait(connection, serverName, hri); 2385 } 2386 } 2387 2388 private void tryAssignmentRepair(HbckInfo hbi, String msg) throws IOException, 2389 KeeperException, InterruptedException { 2390 // If we are trying to fix the errors 2391 if (shouldFixAssignments()) { 2392 errors.print(msg); 2393 undeployRegions(hbi); 2394 setShouldRerun(); 2395 RegionInfo hri = hbi.getHdfsHRI(); 2396 if (hri == null) { 2397 hri = hbi.metaEntry; 2398 } 2399 HBaseFsckRepair.fixUnassigned(admin, hri); 2400 HBaseFsckRepair.waitUntilAssigned(admin, hri); 2401 2402 // also assign replicas if needed (do it only when this call operates on a primary replica) 2403 if (hbi.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) return; 2404 int replicationCount = admin.getTableDescriptor(hri.getTable()).getRegionReplication(); 2405 for (int i = 1; i < replicationCount; i++) { 2406 hri = RegionReplicaUtil.getRegionInfoForReplica(hri, i); 2407 HbckInfo h = regionInfoMap.get(hri.getEncodedName()); 2408 if (h != null) { 2409 undeployRegions(h); 2410 //set skip checks; we undeploy & deploy it; we don't want to evaluate this hbi anymore 2411 //in consistency checks 2412 h.setSkipChecks(true); 2413 } 2414 HBaseFsckRepair.fixUnassigned(admin, hri); 2415 HBaseFsckRepair.waitUntilAssigned(admin, hri); 2416 } 2417 2418 } 2419 } 2420 2421 /** 2422 * Check a single region for consistency and correct deployment. 2423 */ 2424 private void checkRegionConsistency(final String key, final HbckInfo hbi) 2425 throws IOException, KeeperException, InterruptedException { 2426 2427 if (hbi.isSkipChecks()) return; 2428 String descriptiveName = hbi.toString(); 2429 boolean inMeta = hbi.metaEntry != null; 2430 // In case not checking HDFS, assume the region is on HDFS 2431 boolean inHdfs = !shouldCheckHdfs() || hbi.getHdfsRegionDir() != null; 2432 boolean hasMetaAssignment = inMeta && hbi.metaEntry.regionServer != null; 2433 boolean isDeployed = !hbi.deployedOn.isEmpty(); 2434 boolean isMultiplyDeployed = hbi.deployedOn.size() > 1; 2435 boolean deploymentMatchesMeta = 2436 hasMetaAssignment && isDeployed && !isMultiplyDeployed && 2437 hbi.metaEntry.regionServer.equals(hbi.deployedOn.get(0)); 2438 boolean splitParent = 2439 inMeta && hbi.metaEntry.isSplit() && hbi.metaEntry.isOffline(); 2440 boolean shouldBeDeployed = inMeta && !isTableDisabled(hbi.metaEntry.getTable()); 2441 boolean recentlyModified = inHdfs && 2442 hbi.getModTime() + timelag > EnvironmentEdgeManager.currentTime(); 2443 2444 // ========== First the healthy cases ============= 2445 if (hbi.containsOnlyHdfsEdits()) { 2446 return; 2447 } 2448 if (inMeta && inHdfs && isDeployed && deploymentMatchesMeta && shouldBeDeployed) { 2449 return; 2450 } else if (inMeta && inHdfs && !shouldBeDeployed && !isDeployed) { 2451 LOG.info("Region " + descriptiveName + " is in META, and in a disabled " + 2452 "tabled that is not deployed"); 2453 return; 2454 } else if (recentlyModified) { 2455 LOG.warn("Region " + descriptiveName + " was recently modified -- skipping"); 2456 return; 2457 } 2458 // ========== Cases where the region is not in hbase:meta ============= 2459 else if (!inMeta && !inHdfs && !isDeployed) { 2460 // We shouldn't have record of this region at all then! 2461 assert false : "Entry for region with no data"; 2462 } else if (!inMeta && !inHdfs && isDeployed) { 2463 errors.reportError(ERROR_CODE.NOT_IN_META_HDFS, "Region " 2464 + descriptiveName + ", key=" + key + ", not on HDFS or in hbase:meta but " + 2465 "deployed on " + Joiner.on(", ").join(hbi.deployedOn)); 2466 if (shouldFixAssignments()) { 2467 undeployRegions(hbi); 2468 } 2469 2470 } else if (!inMeta && inHdfs && !isDeployed) { 2471 if (hbi.isMerged()) { 2472 // This region has already been merged, the remaining hdfs file will be 2473 // cleaned by CatalogJanitor later 2474 hbi.setSkipChecks(true); 2475 LOG.info("Region " + descriptiveName 2476 + " got merge recently, its file(s) will be cleaned by CatalogJanitor later"); 2477 return; 2478 } 2479 errors.reportError(ERROR_CODE.NOT_IN_META_OR_DEPLOYED, "Region " 2480 + descriptiveName + " on HDFS, but not listed in hbase:meta " + 2481 "or deployed on any region server"); 2482 // restore region consistency of an adopted orphan 2483 if (shouldFixMeta()) { 2484 if (!hbi.isHdfsRegioninfoPresent()) { 2485 LOG.error("Region " + hbi.getHdfsHRI() + " could have been repaired" 2486 + " in table integrity repair phase if -fixHdfsOrphans was" + 2487 " used."); 2488 return; 2489 } 2490 2491 RegionInfo hri = hbi.getHdfsHRI(); 2492 TableInfo tableInfo = tablesInfo.get(hri.getTable()); 2493 2494 for (RegionInfo region : tableInfo.getRegionsFromMeta()) { 2495 if (Bytes.compareTo(region.getStartKey(), hri.getStartKey()) <= 0 2496 && (region.getEndKey().length == 0 || Bytes.compareTo(region.getEndKey(), 2497 hri.getEndKey()) >= 0) 2498 && Bytes.compareTo(region.getStartKey(), hri.getEndKey()) <= 0) { 2499 if(region.isSplit() || region.isOffline()) continue; 2500 Path regionDir = hbi.getHdfsRegionDir(); 2501 FileSystem fs = regionDir.getFileSystem(getConf()); 2502 List<Path> familyDirs = FSUtils.getFamilyDirs(fs, regionDir); 2503 for (Path familyDir : familyDirs) { 2504 List<Path> referenceFilePaths = FSUtils.getReferenceFilePaths(fs, familyDir); 2505 for (Path referenceFilePath : referenceFilePaths) { 2506 Path parentRegionDir = 2507 StoreFileInfo.getReferredToFile(referenceFilePath).getParent().getParent(); 2508 if (parentRegionDir.toString().endsWith(region.getEncodedName())) { 2509 LOG.warn(hri + " start and stop keys are in the range of " + region 2510 + ". The region might not be cleaned up from hdfs when region " + region 2511 + " split failed. Hence deleting from hdfs."); 2512 HRegionFileSystem.deleteRegionFromFileSystem(getConf(), fs, 2513 regionDir.getParent(), hri); 2514 return; 2515 } 2516 } 2517 } 2518 } 2519 } 2520 LOG.info("Patching hbase:meta with .regioninfo: " + hbi.getHdfsHRI()); 2521 int numReplicas = admin.getTableDescriptor(hbi.getTableName()).getRegionReplication(); 2522 HBaseFsckRepair.fixMetaHoleOnlineAndAddReplicas(getConf(), hbi.getHdfsHRI(), 2523 admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)) 2524 .getLiveServerMetrics().keySet(), numReplicas); 2525 2526 tryAssignmentRepair(hbi, "Trying to reassign region..."); 2527 } 2528 2529 } else if (!inMeta && inHdfs && isDeployed) { 2530 errors.reportError(ERROR_CODE.NOT_IN_META, "Region " + descriptiveName 2531 + " not in META, but deployed on " + Joiner.on(", ").join(hbi.deployedOn)); 2532 debugLsr(hbi.getHdfsRegionDir()); 2533 if (hbi.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) { 2534 // for replicas, this means that we should undeploy the region (we would have 2535 // gone over the primaries and fixed meta holes in first phase under 2536 // checkAndFixConsistency; we shouldn't get the condition !inMeta at 2537 // this stage unless unwanted replica) 2538 if (shouldFixAssignments()) { 2539 undeployRegionsForHbi(hbi); 2540 } 2541 } 2542 if (shouldFixMeta() && hbi.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) { 2543 if (!hbi.isHdfsRegioninfoPresent()) { 2544 LOG.error("This should have been repaired in table integrity repair phase"); 2545 return; 2546 } 2547 2548 LOG.info("Patching hbase:meta with with .regioninfo: " + hbi.getHdfsHRI()); 2549 int numReplicas = admin.getTableDescriptor(hbi.getTableName()).getRegionReplication(); 2550 HBaseFsckRepair.fixMetaHoleOnlineAndAddReplicas(getConf(), hbi.getHdfsHRI(), 2551 admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)) 2552 .getLiveServerMetrics().keySet(), numReplicas); 2553 tryAssignmentRepair(hbi, "Trying to fix unassigned region..."); 2554 } 2555 2556 // ========== Cases where the region is in hbase:meta ============= 2557 } else if (inMeta && inHdfs && !isDeployed && splitParent) { 2558 // check whether this is an actual error, or just transient state where parent 2559 // is not cleaned 2560 if (hbi.metaEntry.splitA != null && hbi.metaEntry.splitB != null) { 2561 // check that split daughters are there 2562 HbckInfo infoA = this.regionInfoMap.get(hbi.metaEntry.splitA.getEncodedName()); 2563 HbckInfo infoB = this.regionInfoMap.get(hbi.metaEntry.splitB.getEncodedName()); 2564 if (infoA != null && infoB != null) { 2565 // we already processed or will process daughters. Move on, nothing to see here. 2566 hbi.setSkipChecks(true); 2567 return; 2568 } 2569 } 2570 2571 // For Replica region, we need to do a similar check. If replica is not split successfully, 2572 // error is going to be reported against primary daughter region. 2573 if (hbi.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) { 2574 LOG.info("Region " + descriptiveName + " is a split parent in META, in HDFS, " 2575 + "and not deployed on any region server. This may be transient."); 2576 hbi.setSkipChecks(true); 2577 return; 2578 } 2579 2580 errors.reportError(ERROR_CODE.LINGERING_SPLIT_PARENT, "Region " 2581 + descriptiveName + " is a split parent in META, in HDFS, " 2582 + "and not deployed on any region server. This could be transient, " 2583 + "consider to run the catalog janitor first!"); 2584 if (shouldFixSplitParents()) { 2585 setShouldRerun(); 2586 resetSplitParent(hbi); 2587 } 2588 } else if (inMeta && !inHdfs && !isDeployed) { 2589 errors.reportError(ERROR_CODE.NOT_IN_HDFS_OR_DEPLOYED, "Region " 2590 + descriptiveName + " found in META, but not in HDFS " 2591 + "or deployed on any region server."); 2592 if (shouldFixMeta()) { 2593 deleteMetaRegion(hbi); 2594 } 2595 } else if (inMeta && !inHdfs && isDeployed) { 2596 errors.reportError(ERROR_CODE.NOT_IN_HDFS, "Region " + descriptiveName 2597 + " found in META, but not in HDFS, " + 2598 "and deployed on " + Joiner.on(", ").join(hbi.deployedOn)); 2599 // We treat HDFS as ground truth. Any information in meta is transient 2600 // and equivalent data can be regenerated. So, lets unassign and remove 2601 // these problems from META. 2602 if (shouldFixAssignments()) { 2603 errors.print("Trying to fix unassigned region..."); 2604 undeployRegions(hbi); 2605 } 2606 if (shouldFixMeta()) { 2607 // wait for it to complete 2608 deleteMetaRegion(hbi); 2609 } 2610 } else if (inMeta && inHdfs && !isDeployed && shouldBeDeployed) { 2611 errors.reportError(ERROR_CODE.NOT_DEPLOYED, "Region " + descriptiveName 2612 + " not deployed on any region server."); 2613 tryAssignmentRepair(hbi, "Trying to fix unassigned region..."); 2614 } else if (inMeta && inHdfs && isDeployed && !shouldBeDeployed) { 2615 errors.reportError(ERROR_CODE.SHOULD_NOT_BE_DEPLOYED, 2616 "Region " + descriptiveName + " should not be deployed according " + 2617 "to META, but is deployed on " + Joiner.on(", ").join(hbi.deployedOn)); 2618 if (shouldFixAssignments()) { 2619 errors.print("Trying to close the region " + descriptiveName); 2620 setShouldRerun(); 2621 HBaseFsckRepair.fixMultiAssignment(connection, hbi.metaEntry, hbi.deployedOn); 2622 } 2623 } else if (inMeta && inHdfs && isMultiplyDeployed) { 2624 errors.reportError(ERROR_CODE.MULTI_DEPLOYED, "Region " + descriptiveName 2625 + " is listed in hbase:meta on region server " + hbi.metaEntry.regionServer 2626 + " but is multiply assigned to region servers " + 2627 Joiner.on(", ").join(hbi.deployedOn)); 2628 // If we are trying to fix the errors 2629 if (shouldFixAssignments()) { 2630 errors.print("Trying to fix assignment error..."); 2631 setShouldRerun(); 2632 HBaseFsckRepair.fixMultiAssignment(connection, hbi.metaEntry, hbi.deployedOn); 2633 } 2634 } else if (inMeta && inHdfs && isDeployed && !deploymentMatchesMeta) { 2635 errors.reportError(ERROR_CODE.SERVER_DOES_NOT_MATCH_META, "Region " 2636 + descriptiveName + " listed in hbase:meta on region server " + 2637 hbi.metaEntry.regionServer + " but found on region server " + 2638 hbi.deployedOn.get(0)); 2639 // If we are trying to fix the errors 2640 if (shouldFixAssignments()) { 2641 errors.print("Trying to fix assignment error..."); 2642 setShouldRerun(); 2643 HBaseFsckRepair.fixMultiAssignment(connection, hbi.metaEntry, hbi.deployedOn); 2644 HBaseFsckRepair.waitUntilAssigned(admin, hbi.getHdfsHRI()); 2645 } 2646 } else { 2647 errors.reportError(ERROR_CODE.UNKNOWN, "Region " + descriptiveName + 2648 " is in an unforeseen state:" + 2649 " inMeta=" + inMeta + 2650 " inHdfs=" + inHdfs + 2651 " isDeployed=" + isDeployed + 2652 " isMultiplyDeployed=" + isMultiplyDeployed + 2653 " deploymentMatchesMeta=" + deploymentMatchesMeta + 2654 " shouldBeDeployed=" + shouldBeDeployed); 2655 } 2656 } 2657 2658 /** 2659 * Checks tables integrity. Goes over all regions and scans the tables. 2660 * Collects all the pieces for each table and checks if there are missing, 2661 * repeated or overlapping ones. 2662 * @throws IOException 2663 */ 2664 SortedMap<TableName, TableInfo> checkIntegrity() throws IOException { 2665 tablesInfo = new TreeMap<>(); 2666 LOG.debug("There are " + regionInfoMap.size() + " region info entries"); 2667 for (HbckInfo hbi : regionInfoMap.values()) { 2668 // Check only valid, working regions 2669 if (hbi.metaEntry == null) { 2670 // this assumes that consistency check has run loadMetaEntry 2671 Path p = hbi.getHdfsRegionDir(); 2672 if (p == null) { 2673 errors.report("No regioninfo in Meta or HDFS. " + hbi); 2674 } 2675 2676 // TODO test. 2677 continue; 2678 } 2679 if (hbi.metaEntry.regionServer == null) { 2680 errors.detail("Skipping region because no region server: " + hbi); 2681 continue; 2682 } 2683 if (hbi.metaEntry.isOffline()) { 2684 errors.detail("Skipping region because it is offline: " + hbi); 2685 continue; 2686 } 2687 if (hbi.containsOnlyHdfsEdits()) { 2688 errors.detail("Skipping region because it only contains edits" + hbi); 2689 continue; 2690 } 2691 2692 // Missing regionDir or over-deployment is checked elsewhere. Include 2693 // these cases in modTInfo, so we can evaluate those regions as part of 2694 // the region chain in META 2695 //if (hbi.foundRegionDir == null) continue; 2696 //if (hbi.deployedOn.size() != 1) continue; 2697 if (hbi.deployedOn.isEmpty()) continue; 2698 2699 // We should be safe here 2700 TableName tableName = hbi.metaEntry.getTable(); 2701 TableInfo modTInfo = tablesInfo.get(tableName); 2702 if (modTInfo == null) { 2703 modTInfo = new TableInfo(tableName); 2704 } 2705 for (ServerName server : hbi.deployedOn) { 2706 modTInfo.addServer(server); 2707 } 2708 2709 if (!hbi.isSkipChecks()) { 2710 modTInfo.addRegionInfo(hbi); 2711 } 2712 2713 tablesInfo.put(tableName, modTInfo); 2714 } 2715 2716 loadTableInfosForTablesWithNoRegion(); 2717 2718 logParallelMerge(); 2719 for (TableInfo tInfo : tablesInfo.values()) { 2720 TableIntegrityErrorHandler handler = tInfo.new IntegrityFixSuggester(tInfo, errors); 2721 if (!tInfo.checkRegionChain(handler)) { 2722 errors.report("Found inconsistency in table " + tInfo.getName()); 2723 } 2724 } 2725 return tablesInfo; 2726 } 2727 2728 /** Loads table info's for tables that may not have been included, since there are no 2729 * regions reported for the table, but table dir is there in hdfs 2730 */ 2731 private void loadTableInfosForTablesWithNoRegion() throws IOException { 2732 Map<String, TableDescriptor> allTables = new FSTableDescriptors(getConf()).getAll(); 2733 for (TableDescriptor htd : allTables.values()) { 2734 if (checkMetaOnly && !htd.isMetaTable()) { 2735 continue; 2736 } 2737 2738 TableName tableName = htd.getTableName(); 2739 if (isTableIncluded(tableName) && !tablesInfo.containsKey(tableName)) { 2740 TableInfo tableInfo = new TableInfo(tableName); 2741 tableInfo.htds.add(htd); 2742 tablesInfo.put(htd.getTableName(), tableInfo); 2743 } 2744 } 2745 } 2746 2747 /** 2748 * Merge hdfs data by moving from contained HbckInfo into targetRegionDir. 2749 * @return number of file move fixes done to merge regions. 2750 */ 2751 public int mergeRegionDirs(Path targetRegionDir, HbckInfo contained) throws IOException { 2752 int fileMoves = 0; 2753 String thread = Thread.currentThread().getName(); 2754 LOG.debug("[" + thread + "] Contained region dir after close and pause"); 2755 debugLsr(contained.getHdfsRegionDir()); 2756 2757 // rename the contained into the container. 2758 FileSystem fs = targetRegionDir.getFileSystem(getConf()); 2759 FileStatus[] dirs = null; 2760 try { 2761 dirs = fs.listStatus(contained.getHdfsRegionDir()); 2762 } catch (FileNotFoundException fnfe) { 2763 // region we are attempting to merge in is not present! Since this is a merge, there is 2764 // no harm skipping this region if it does not exist. 2765 if (!fs.exists(contained.getHdfsRegionDir())) { 2766 LOG.warn("[" + thread + "] HDFS region dir " + contained.getHdfsRegionDir() 2767 + " is missing. Assuming already sidelined or moved."); 2768 } else { 2769 sidelineRegionDir(fs, contained); 2770 } 2771 return fileMoves; 2772 } 2773 2774 if (dirs == null) { 2775 if (!fs.exists(contained.getHdfsRegionDir())) { 2776 LOG.warn("[" + thread + "] HDFS region dir " + contained.getHdfsRegionDir() 2777 + " already sidelined."); 2778 } else { 2779 sidelineRegionDir(fs, contained); 2780 } 2781 return fileMoves; 2782 } 2783 2784 for (FileStatus cf : dirs) { 2785 Path src = cf.getPath(); 2786 Path dst = new Path(targetRegionDir, src.getName()); 2787 2788 if (src.getName().equals(HRegionFileSystem.REGION_INFO_FILE)) { 2789 // do not copy the old .regioninfo file. 2790 continue; 2791 } 2792 2793 if (src.getName().equals(HConstants.HREGION_OLDLOGDIR_NAME)) { 2794 // do not copy the .oldlogs files 2795 continue; 2796 } 2797 2798 LOG.info("[" + thread + "] Moving files from " + src + " into containing region " + dst); 2799 // FileSystem.rename is inconsistent with directories -- if the 2800 // dst (foo/a) exists and is a dir, and the src (foo/b) is a dir, 2801 // it moves the src into the dst dir resulting in (foo/a/b). If 2802 // the dst does not exist, and the src a dir, src becomes dst. (foo/b) 2803 for (FileStatus hfile : fs.listStatus(src)) { 2804 boolean success = fs.rename(hfile.getPath(), dst); 2805 if (success) { 2806 fileMoves++; 2807 } 2808 } 2809 LOG.debug("[" + thread + "] Sideline directory contents:"); 2810 debugLsr(targetRegionDir); 2811 } 2812 2813 // if all success. 2814 sidelineRegionDir(fs, contained); 2815 LOG.info("[" + thread + "] Sidelined region dir "+ contained.getHdfsRegionDir() + " into " + 2816 getSidelineDir()); 2817 debugLsr(contained.getHdfsRegionDir()); 2818 2819 return fileMoves; 2820 } 2821 2822 2823 static class WorkItemOverlapMerge implements Callable<Void> { 2824 private TableIntegrityErrorHandler handler; 2825 Collection<HbckInfo> overlapgroup; 2826 2827 WorkItemOverlapMerge(Collection<HbckInfo> overlapgroup, TableIntegrityErrorHandler handler) { 2828 this.handler = handler; 2829 this.overlapgroup = overlapgroup; 2830 } 2831 2832 @Override 2833 public Void call() throws Exception { 2834 handler.handleOverlapGroup(overlapgroup); 2835 return null; 2836 } 2837 }; 2838 2839 2840 /** 2841 * Maintain information about a particular table. 2842 */ 2843 public class TableInfo { 2844 TableName tableName; 2845 TreeSet <ServerName> deployedOn; 2846 2847 // backwards regions 2848 final List<HbckInfo> backwards = new ArrayList<>(); 2849 2850 // sidelined big overlapped regions 2851 final Map<Path, HbckInfo> sidelinedRegions = new HashMap<>(); 2852 2853 // region split calculator 2854 final RegionSplitCalculator<HbckInfo> sc = new RegionSplitCalculator<>(cmp); 2855 2856 // Histogram of different TableDescriptors found. Ideally there is only one! 2857 final Set<TableDescriptor> htds = new HashSet<>(); 2858 2859 // key = start split, values = set of splits in problem group 2860 final Multimap<byte[], HbckInfo> overlapGroups = 2861 TreeMultimap.create(RegionSplitCalculator.BYTES_COMPARATOR, cmp); 2862 2863 // list of regions derived from meta entries. 2864 private ImmutableList<RegionInfo> regionsFromMeta = null; 2865 2866 TableInfo(TableName name) { 2867 this.tableName = name; 2868 deployedOn = new TreeSet <>(); 2869 } 2870 2871 /** 2872 * @return descriptor common to all regions. null if are none or multiple! 2873 */ 2874 private TableDescriptor getHTD() { 2875 if (htds.size() == 1) { 2876 return (TableDescriptor)htds.toArray()[0]; 2877 } else { 2878 LOG.error("None/Multiple table descriptors found for table '" 2879 + tableName + "' regions: " + htds); 2880 } 2881 return null; 2882 } 2883 2884 public void addRegionInfo(HbckInfo hir) { 2885 if (Bytes.equals(hir.getEndKey(), HConstants.EMPTY_END_ROW)) { 2886 // end key is absolute end key, just add it. 2887 // ignore replicas other than primary for these checks 2888 if (hir.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) sc.add(hir); 2889 return; 2890 } 2891 2892 // if not the absolute end key, check for cycle 2893 if (Bytes.compareTo(hir.getStartKey(), hir.getEndKey()) > 0) { 2894 errors.reportError( 2895 ERROR_CODE.REGION_CYCLE, 2896 String.format("The endkey for this region comes before the " 2897 + "startkey, startkey=%s, endkey=%s", 2898 Bytes.toStringBinary(hir.getStartKey()), 2899 Bytes.toStringBinary(hir.getEndKey())), this, hir); 2900 backwards.add(hir); 2901 return; 2902 } 2903 2904 // main case, add to split calculator 2905 // ignore replicas other than primary for these checks 2906 if (hir.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) sc.add(hir); 2907 } 2908 2909 public void addServer(ServerName server) { 2910 this.deployedOn.add(server); 2911 } 2912 2913 public TableName getName() { 2914 return tableName; 2915 } 2916 2917 public int getNumRegions() { 2918 return sc.getStarts().size() + backwards.size(); 2919 } 2920 2921 public synchronized ImmutableList<RegionInfo> getRegionsFromMeta() { 2922 // lazy loaded, synchronized to ensure a single load 2923 if (regionsFromMeta == null) { 2924 List<RegionInfo> regions = new ArrayList<>(); 2925 for (HbckInfo h : HBaseFsck.this.regionInfoMap.values()) { 2926 if (tableName.equals(h.getTableName())) { 2927 if (h.metaEntry != null) { 2928 regions.add(h.metaEntry); 2929 } 2930 } 2931 } 2932 regionsFromMeta = Ordering.from(RegionInfo.COMPARATOR).immutableSortedCopy(regions); 2933 } 2934 2935 return regionsFromMeta; 2936 } 2937 2938 private class IntegrityFixSuggester extends TableIntegrityErrorHandlerImpl { 2939 ErrorReporter errors; 2940 2941 IntegrityFixSuggester(TableInfo ti, ErrorReporter errors) { 2942 this.errors = errors; 2943 setTableInfo(ti); 2944 } 2945 2946 @Override 2947 public void handleRegionStartKeyNotEmpty(HbckInfo hi) throws IOException{ 2948 errors.reportError(ERROR_CODE.FIRST_REGION_STARTKEY_NOT_EMPTY, 2949 "First region should start with an empty key. You need to " 2950 + " create a new region and regioninfo in HDFS to plug the hole.", 2951 getTableInfo(), hi); 2952 } 2953 2954 @Override 2955 public void handleRegionEndKeyNotEmpty(byte[] curEndKey) throws IOException { 2956 errors.reportError(ERROR_CODE.LAST_REGION_ENDKEY_NOT_EMPTY, 2957 "Last region should end with an empty key. You need to " 2958 + "create a new region and regioninfo in HDFS to plug the hole.", getTableInfo()); 2959 } 2960 2961 @Override 2962 public void handleDegenerateRegion(HbckInfo hi) throws IOException{ 2963 errors.reportError(ERROR_CODE.DEGENERATE_REGION, 2964 "Region has the same start and end key.", getTableInfo(), hi); 2965 } 2966 2967 @Override 2968 public void handleDuplicateStartKeys(HbckInfo r1, HbckInfo r2) throws IOException{ 2969 byte[] key = r1.getStartKey(); 2970 // dup start key 2971 errors.reportError(ERROR_CODE.DUPE_STARTKEYS, 2972 "Multiple regions have the same startkey: " 2973 + Bytes.toStringBinary(key), getTableInfo(), r1); 2974 errors.reportError(ERROR_CODE.DUPE_STARTKEYS, 2975 "Multiple regions have the same startkey: " 2976 + Bytes.toStringBinary(key), getTableInfo(), r2); 2977 } 2978 2979 @Override 2980 public void handleSplit(HbckInfo r1, HbckInfo r2) throws IOException{ 2981 byte[] key = r1.getStartKey(); 2982 // dup start key 2983 errors.reportError(ERROR_CODE.DUPE_ENDKEYS, 2984 "Multiple regions have the same regionID: " 2985 + Bytes.toStringBinary(key), getTableInfo(), r1); 2986 errors.reportError(ERROR_CODE.DUPE_ENDKEYS, 2987 "Multiple regions have the same regionID: " 2988 + Bytes.toStringBinary(key), getTableInfo(), r2); 2989 } 2990 2991 @Override 2992 public void handleOverlapInRegionChain(HbckInfo hi1, HbckInfo hi2) throws IOException{ 2993 errors.reportError(ERROR_CODE.OVERLAP_IN_REGION_CHAIN, 2994 "There is an overlap in the region chain.", 2995 getTableInfo(), hi1, hi2); 2996 } 2997 2998 @Override 2999 public void handleHoleInRegionChain(byte[] holeStart, byte[] holeStop) throws IOException{ 3000 errors.reportError( 3001 ERROR_CODE.HOLE_IN_REGION_CHAIN, 3002 "There is a hole in the region chain between " 3003 + Bytes.toStringBinary(holeStart) + " and " 3004 + Bytes.toStringBinary(holeStop) 3005 + ". You need to create a new .regioninfo and region " 3006 + "dir in hdfs to plug the hole."); 3007 } 3008 }; 3009 3010 /** 3011 * This handler fixes integrity errors from hdfs information. There are 3012 * basically three classes of integrity problems 1) holes, 2) overlaps, and 3013 * 3) invalid regions. 3014 * 3015 * This class overrides methods that fix holes and the overlap group case. 3016 * Individual cases of particular overlaps are handled by the general 3017 * overlap group merge repair case. 3018 * 3019 * If hbase is online, this forces regions offline before doing merge 3020 * operations. 3021 */ 3022 private class HDFSIntegrityFixer extends IntegrityFixSuggester { 3023 Configuration conf; 3024 3025 boolean fixOverlaps = true; 3026 3027 HDFSIntegrityFixer(TableInfo ti, ErrorReporter errors, Configuration conf, 3028 boolean fixHoles, boolean fixOverlaps) { 3029 super(ti, errors); 3030 this.conf = conf; 3031 this.fixOverlaps = fixOverlaps; 3032 // TODO properly use fixHoles 3033 } 3034 3035 /** 3036 * This is a special case hole -- when the first region of a table is 3037 * missing from META, HBase doesn't acknowledge the existance of the 3038 * table. 3039 */ 3040 @Override 3041 public void handleRegionStartKeyNotEmpty(HbckInfo next) throws IOException { 3042 errors.reportError(ERROR_CODE.FIRST_REGION_STARTKEY_NOT_EMPTY, 3043 "First region should start with an empty key. Creating a new " + 3044 "region and regioninfo in HDFS to plug the hole.", 3045 getTableInfo(), next); 3046 TableDescriptor htd = getTableInfo().getHTD(); 3047 // from special EMPTY_START_ROW to next region's startKey 3048 RegionInfo newRegion = RegionInfoBuilder.newBuilder(htd.getTableName()) 3049 .setStartKey(HConstants.EMPTY_START_ROW) 3050 .setEndKey(next.getStartKey()) 3051 .build(); 3052 3053 // TODO test 3054 HRegion region = HBaseFsckRepair.createHDFSRegionDir(conf, newRegion, htd); 3055 LOG.info("Table region start key was not empty. Created new empty region: " 3056 + newRegion + " " +region); 3057 fixes++; 3058 } 3059 3060 @Override 3061 public void handleRegionEndKeyNotEmpty(byte[] curEndKey) throws IOException { 3062 errors.reportError(ERROR_CODE.LAST_REGION_ENDKEY_NOT_EMPTY, 3063 "Last region should end with an empty key. Creating a new " 3064 + "region and regioninfo in HDFS to plug the hole.", getTableInfo()); 3065 TableDescriptor htd = getTableInfo().getHTD(); 3066 // from curEndKey to EMPTY_START_ROW 3067 RegionInfo newRegion = RegionInfoBuilder.newBuilder(htd.getTableName()) 3068 .setStartKey(curEndKey) 3069 .setEndKey(HConstants.EMPTY_START_ROW) 3070 .build(); 3071 3072 HRegion region = HBaseFsckRepair.createHDFSRegionDir(conf, newRegion, htd); 3073 LOG.info("Table region end key was not empty. Created new empty region: " + newRegion 3074 + " " + region); 3075 fixes++; 3076 } 3077 3078 /** 3079 * There is a hole in the hdfs regions that violates the table integrity 3080 * rules. Create a new empty region that patches the hole. 3081 */ 3082 @Override 3083 public void handleHoleInRegionChain(byte[] holeStartKey, byte[] holeStopKey) throws IOException { 3084 errors.reportError( 3085 ERROR_CODE.HOLE_IN_REGION_CHAIN, 3086 "There is a hole in the region chain between " 3087 + Bytes.toStringBinary(holeStartKey) + " and " 3088 + Bytes.toStringBinary(holeStopKey) 3089 + ". Creating a new regioninfo and region " 3090 + "dir in hdfs to plug the hole."); 3091 TableDescriptor htd = getTableInfo().getHTD(); 3092 RegionInfo newRegion = RegionInfoBuilder.newBuilder(htd.getTableName()) 3093 .setStartKey(holeStartKey) 3094 .setEndKey(holeStopKey) 3095 .build(); 3096 HRegion region = HBaseFsckRepair.createHDFSRegionDir(conf, newRegion, htd); 3097 LOG.info("Plugged hole by creating new empty region: "+ newRegion + " " +region); 3098 fixes++; 3099 } 3100 3101 /** 3102 * This takes set of overlapping regions and merges them into a single 3103 * region. This covers cases like degenerate regions, shared start key, 3104 * general overlaps, duplicate ranges, and partial overlapping regions. 3105 * 3106 * Cases: 3107 * - Clean regions that overlap 3108 * - Only .oldlogs regions (can't find start/stop range, or figure out) 3109 * 3110 * This is basically threadsafe, except for the fixer increment in mergeOverlaps. 3111 */ 3112 @Override 3113 public void handleOverlapGroup(Collection<HbckInfo> overlap) 3114 throws IOException { 3115 Preconditions.checkNotNull(overlap); 3116 Preconditions.checkArgument(overlap.size() >0); 3117 3118 if (!this.fixOverlaps) { 3119 LOG.warn("Not attempting to repair overlaps."); 3120 return; 3121 } 3122 3123 if (overlap.size() > maxMerge) { 3124 LOG.warn("Overlap group has " + overlap.size() + " overlapping " + 3125 "regions which is greater than " + maxMerge + ", the max number of regions to merge"); 3126 if (sidelineBigOverlaps) { 3127 // we only sideline big overlapped groups that exceeds the max number of regions to merge 3128 sidelineBigOverlaps(overlap); 3129 } 3130 return; 3131 } 3132 if (shouldRemoveParents()) { 3133 removeParentsAndFixSplits(overlap); 3134 } 3135 mergeOverlaps(overlap); 3136 } 3137 3138 void removeParentsAndFixSplits(Collection<HbckInfo> overlap) throws IOException { 3139 Pair<byte[], byte[]> range = null; 3140 HbckInfo parent = null; 3141 HbckInfo daughterA = null; 3142 HbckInfo daughterB = null; 3143 Collection<HbckInfo> daughters = new ArrayList<HbckInfo>(overlap); 3144 3145 String thread = Thread.currentThread().getName(); 3146 LOG.info("== [" + thread + "] Attempting fix splits in overlap state."); 3147 3148 // we only can handle a single split per group at the time 3149 if (overlap.size() > 3) { 3150 LOG.info("Too many overlaps were found on this group, falling back to regular merge."); 3151 return; 3152 } 3153 3154 for (HbckInfo hi : overlap) { 3155 if (range == null) { 3156 range = new Pair<byte[], byte[]>(hi.getStartKey(), hi.getEndKey()); 3157 } else { 3158 if (RegionSplitCalculator.BYTES_COMPARATOR 3159 .compare(hi.getStartKey(), range.getFirst()) < 0) { 3160 range.setFirst(hi.getStartKey()); 3161 } 3162 if (RegionSplitCalculator.BYTES_COMPARATOR 3163 .compare(hi.getEndKey(), range.getSecond()) > 0) { 3164 range.setSecond(hi.getEndKey()); 3165 } 3166 } 3167 } 3168 3169 LOG.info("This group range is [" + Bytes.toStringBinary(range.getFirst()) + ", " 3170 + Bytes.toStringBinary(range.getSecond()) + "]"); 3171 3172 // attempt to find a possible parent for the edge case of a split 3173 for (HbckInfo hi : overlap) { 3174 if (Bytes.compareTo(hi.getHdfsHRI().getStartKey(), range.getFirst()) == 0 3175 && Bytes.compareTo(hi.getHdfsHRI().getEndKey(), range.getSecond()) == 0) { 3176 LOG.info("This is a parent for this group: " + hi.toString()); 3177 parent = hi; 3178 } 3179 } 3180 3181 // Remove parent regions from daughters collection 3182 if (parent != null) { 3183 daughters.remove(parent); 3184 } 3185 3186 // Lets verify that daughters share the regionID at split time and they 3187 // were created after the parent 3188 for (HbckInfo hi : daughters) { 3189 if (Bytes.compareTo(hi.getHdfsHRI().getStartKey(), range.getFirst()) == 0) { 3190 if (parent.getHdfsHRI().getRegionId() < hi.getHdfsHRI().getRegionId()) { 3191 daughterA = hi; 3192 } 3193 } 3194 if (Bytes.compareTo(hi.getHdfsHRI().getEndKey(), range.getSecond()) == 0) { 3195 if (parent.getHdfsHRI().getRegionId() < hi.getHdfsHRI().getRegionId()) { 3196 daughterB = hi; 3197 } 3198 } 3199 } 3200 3201 // daughters must share the same regionID and we should have a parent too 3202 if (daughterA.getHdfsHRI().getRegionId() != daughterB.getHdfsHRI().getRegionId() || parent == null) 3203 return; 3204 3205 FileSystem fs = FileSystem.get(conf); 3206 LOG.info("Found parent: " + parent.getRegionNameAsString()); 3207 LOG.info("Found potential daughter a: " + daughterA.getRegionNameAsString()); 3208 LOG.info("Found potential daughter b: " + daughterB.getRegionNameAsString()); 3209 LOG.info("Trying to fix parent in overlap by removing the parent."); 3210 try { 3211 closeRegion(parent); 3212 } catch (IOException ioe) { 3213 LOG.warn("Parent region could not be closed, continuing with regular merge...", ioe); 3214 return; 3215 } catch (InterruptedException ie) { 3216 LOG.warn("Parent region could not be closed, continuing with regular merge...", ie); 3217 return; 3218 } 3219 3220 try { 3221 offline(parent.getRegionName()); 3222 } catch (IOException ioe) { 3223 LOG.warn("Unable to offline parent region: " + parent.getRegionNameAsString() 3224 + ". Just continuing with regular merge... ", ioe); 3225 return; 3226 } 3227 3228 try { 3229 HBaseFsckRepair.removeParentInMeta(conf, parent.getHdfsHRI()); 3230 } catch (IOException ioe) { 3231 LOG.warn("Unable to remove parent region in META: " + parent.getRegionNameAsString() 3232 + ". Just continuing with regular merge... ", ioe); 3233 return; 3234 } 3235 3236 sidelineRegionDir(fs, parent); 3237 LOG.info("[" + thread + "] Sidelined parent region dir "+ parent.getHdfsRegionDir() + " into " + 3238 getSidelineDir()); 3239 debugLsr(parent.getHdfsRegionDir()); 3240 3241 // Make sure we don't have the parents and daughters around 3242 overlap.remove(parent); 3243 overlap.remove(daughterA); 3244 overlap.remove(daughterB); 3245 3246 LOG.info("Done fixing split."); 3247 3248 } 3249 3250 void mergeOverlaps(Collection<HbckInfo> overlap) 3251 throws IOException { 3252 String thread = Thread.currentThread().getName(); 3253 LOG.info("== [" + thread + "] Merging regions into one region: " 3254 + Joiner.on(",").join(overlap)); 3255 // get the min / max range and close all concerned regions 3256 Pair<byte[], byte[]> range = null; 3257 for (HbckInfo hi : overlap) { 3258 if (range == null) { 3259 range = new Pair<>(hi.getStartKey(), hi.getEndKey()); 3260 } else { 3261 if (RegionSplitCalculator.BYTES_COMPARATOR 3262 .compare(hi.getStartKey(), range.getFirst()) < 0) { 3263 range.setFirst(hi.getStartKey()); 3264 } 3265 if (RegionSplitCalculator.BYTES_COMPARATOR 3266 .compare(hi.getEndKey(), range.getSecond()) > 0) { 3267 range.setSecond(hi.getEndKey()); 3268 } 3269 } 3270 // need to close files so delete can happen. 3271 LOG.debug("[" + thread + "] Closing region before moving data around: " + hi); 3272 LOG.debug("[" + thread + "] Contained region dir before close"); 3273 debugLsr(hi.getHdfsRegionDir()); 3274 try { 3275 LOG.info("[" + thread + "] Closing region: " + hi); 3276 closeRegion(hi); 3277 } catch (IOException ioe) { 3278 LOG.warn("[" + thread + "] Was unable to close region " + hi 3279 + ". Just continuing... ", ioe); 3280 } catch (InterruptedException e) { 3281 LOG.warn("[" + thread + "] Was unable to close region " + hi 3282 + ". Just continuing... ", e); 3283 } 3284 3285 try { 3286 LOG.info("[" + thread + "] Offlining region: " + hi); 3287 offline(hi.getRegionName()); 3288 } catch (IOException ioe) { 3289 LOG.warn("[" + thread + "] Unable to offline region from master: " + hi 3290 + ". Just continuing... ", ioe); 3291 } 3292 } 3293 3294 // create new empty container region. 3295 TableDescriptor htd = getTableInfo().getHTD(); 3296 // from start key to end Key 3297 RegionInfo newRegion = RegionInfoBuilder.newBuilder(htd.getTableName()) 3298 .setStartKey(range.getFirst()) 3299 .setEndKey(range.getSecond()) 3300 .build(); 3301 HRegion region = HBaseFsckRepair.createHDFSRegionDir(conf, newRegion, htd); 3302 LOG.info("[" + thread + "] Created new empty container region: " + 3303 newRegion + " to contain regions: " + Joiner.on(",").join(overlap)); 3304 debugLsr(region.getRegionFileSystem().getRegionDir()); 3305 3306 // all target regions are closed, should be able to safely cleanup. 3307 boolean didFix= false; 3308 Path target = region.getRegionFileSystem().getRegionDir(); 3309 for (HbckInfo contained : overlap) { 3310 LOG.info("[" + thread + "] Merging " + contained + " into " + target ); 3311 int merges = mergeRegionDirs(target, contained); 3312 if (merges > 0) { 3313 didFix = true; 3314 } 3315 } 3316 if (didFix) { 3317 fixes++; 3318 } 3319 } 3320 3321 /** 3322 * Sideline some regions in a big overlap group so that it 3323 * will have fewer regions, and it is easier to merge them later on. 3324 * 3325 * @param bigOverlap the overlapped group with regions more than maxMerge 3326 * @throws IOException 3327 */ 3328 void sidelineBigOverlaps( 3329 Collection<HbckInfo> bigOverlap) throws IOException { 3330 int overlapsToSideline = bigOverlap.size() - maxMerge; 3331 if (overlapsToSideline > maxOverlapsToSideline) { 3332 overlapsToSideline = maxOverlapsToSideline; 3333 } 3334 List<HbckInfo> regionsToSideline = 3335 RegionSplitCalculator.findBigRanges(bigOverlap, overlapsToSideline); 3336 FileSystem fs = FileSystem.get(conf); 3337 for (HbckInfo regionToSideline: regionsToSideline) { 3338 try { 3339 LOG.info("Closing region: " + regionToSideline); 3340 closeRegion(regionToSideline); 3341 } catch (IOException ioe) { 3342 LOG.warn("Was unable to close region " + regionToSideline 3343 + ". Just continuing... ", ioe); 3344 } catch (InterruptedException e) { 3345 LOG.warn("Was unable to close region " + regionToSideline 3346 + ". Just continuing... ", e); 3347 } 3348 3349 try { 3350 LOG.info("Offlining region: " + regionToSideline); 3351 offline(regionToSideline.getRegionName()); 3352 } catch (IOException ioe) { 3353 LOG.warn("Unable to offline region from master: " + regionToSideline 3354 + ". Just continuing... ", ioe); 3355 } 3356 3357 LOG.info("Before sideline big overlapped region: " + regionToSideline.toString()); 3358 Path sidelineRegionDir = sidelineRegionDir(fs, TO_BE_LOADED, regionToSideline); 3359 if (sidelineRegionDir != null) { 3360 sidelinedRegions.put(sidelineRegionDir, regionToSideline); 3361 LOG.info("After sidelined big overlapped region: " 3362 + regionToSideline.getRegionNameAsString() 3363 + " to " + sidelineRegionDir.toString()); 3364 fixes++; 3365 } 3366 } 3367 } 3368 } 3369 3370 /** 3371 * Check the region chain (from META) of this table. We are looking for 3372 * holes, overlaps, and cycles. 3373 * @return false if there are errors 3374 * @throws IOException 3375 */ 3376 public boolean checkRegionChain(TableIntegrityErrorHandler handler) throws IOException { 3377 // When table is disabled no need to check for the region chain. Some of the regions 3378 // accidently if deployed, this below code might report some issues like missing start 3379 // or end regions or region hole in chain and may try to fix which is unwanted. 3380 if (isTableDisabled(this.tableName)) { 3381 return true; 3382 } 3383 int originalErrorsCount = errors.getErrorList().size(); 3384 Multimap<byte[], HbckInfo> regions = sc.calcCoverage(); 3385 SortedSet<byte[]> splits = sc.getSplits(); 3386 3387 byte[] prevKey = null; 3388 byte[] problemKey = null; 3389 3390 if (splits.isEmpty()) { 3391 // no region for this table 3392 handler.handleHoleInRegionChain(HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); 3393 } 3394 3395 for (byte[] key : splits) { 3396 Collection<HbckInfo> ranges = regions.get(key); 3397 if (prevKey == null && !Bytes.equals(key, HConstants.EMPTY_BYTE_ARRAY)) { 3398 for (HbckInfo rng : ranges) { 3399 handler.handleRegionStartKeyNotEmpty(rng); 3400 } 3401 } 3402 3403 // check for degenerate ranges 3404 for (HbckInfo rng : ranges) { 3405 // special endkey case converts '' to null 3406 byte[] endKey = rng.getEndKey(); 3407 endKey = (endKey.length == 0) ? null : endKey; 3408 if (Bytes.equals(rng.getStartKey(),endKey)) { 3409 handler.handleDegenerateRegion(rng); 3410 } 3411 } 3412 3413 if (ranges.size() == 1) { 3414 // this split key is ok -- no overlap, not a hole. 3415 if (problemKey != null) { 3416 LOG.warn("reached end of problem group: " + Bytes.toStringBinary(key)); 3417 } 3418 problemKey = null; // fell through, no more problem. 3419 } else if (ranges.size() > 1) { 3420 // set the new problem key group name, if already have problem key, just 3421 // keep using it. 3422 if (problemKey == null) { 3423 // only for overlap regions. 3424 LOG.warn("Naming new problem group: " + Bytes.toStringBinary(key)); 3425 problemKey = key; 3426 } 3427 overlapGroups.putAll(problemKey, ranges); 3428 3429 // record errors 3430 ArrayList<HbckInfo> subRange = new ArrayList<>(ranges); 3431 // this dumb and n^2 but this shouldn't happen often 3432 for (HbckInfo r1 : ranges) { 3433 if (r1.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) continue; 3434 subRange.remove(r1); 3435 for (HbckInfo r2 : subRange) { 3436 if (r2.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) continue; 3437 // general case of same start key 3438 if (Bytes.compareTo(r1.getStartKey(), r2.getStartKey())==0) { 3439 handler.handleDuplicateStartKeys(r1,r2); 3440 } else if (Bytes.compareTo(r1.getEndKey(), r2.getStartKey())==0 && 3441 r1.getHdfsHRI().getRegionId() == r2.getHdfsHRI().getRegionId()) { 3442 LOG.info("this is a split, log to splits"); 3443 handler.handleSplit(r1, r2); 3444 } else { 3445 // overlap 3446 handler.handleOverlapInRegionChain(r1, r2); 3447 } 3448 } 3449 } 3450 3451 } else if (ranges.isEmpty()) { 3452 if (problemKey != null) { 3453 LOG.warn("reached end of problem group: " + Bytes.toStringBinary(key)); 3454 } 3455 problemKey = null; 3456 3457 byte[] holeStopKey = sc.getSplits().higher(key); 3458 // if higher key is null we reached the top. 3459 if (holeStopKey != null) { 3460 // hole 3461 handler.handleHoleInRegionChain(key, holeStopKey); 3462 } 3463 } 3464 prevKey = key; 3465 } 3466 3467 // When the last region of a table is proper and having an empty end key, 'prevKey' 3468 // will be null. 3469 if (prevKey != null) { 3470 handler.handleRegionEndKeyNotEmpty(prevKey); 3471 } 3472 3473 // TODO fold this into the TableIntegrityHandler 3474 if (getConf().getBoolean("hbasefsck.overlap.merge.parallel", true)) { 3475 boolean ok = handleOverlapsParallel(handler, prevKey); 3476 if (!ok) { 3477 return false; 3478 } 3479 } else { 3480 for (Collection<HbckInfo> overlap : overlapGroups.asMap().values()) { 3481 handler.handleOverlapGroup(overlap); 3482 } 3483 } 3484 3485 if (details) { 3486 // do full region split map dump 3487 errors.print("---- Table '" + this.tableName 3488 + "': region split map"); 3489 dump(splits, regions); 3490 errors.print("---- Table '" + this.tableName 3491 + "': overlap groups"); 3492 dumpOverlapProblems(overlapGroups); 3493 errors.print("There are " + overlapGroups.keySet().size() 3494 + " overlap groups with " + overlapGroups.size() 3495 + " overlapping regions"); 3496 } 3497 if (!sidelinedRegions.isEmpty()) { 3498 LOG.warn("Sidelined big overlapped regions, please bulk load them!"); 3499 errors.print("---- Table '" + this.tableName 3500 + "': sidelined big overlapped regions"); 3501 dumpSidelinedRegions(sidelinedRegions); 3502 } 3503 return errors.getErrorList().size() == originalErrorsCount; 3504 } 3505 3506 private boolean handleOverlapsParallel(TableIntegrityErrorHandler handler, byte[] prevKey) 3507 throws IOException { 3508 // we parallelize overlap handler for the case we have lots of groups to fix. We can 3509 // safely assume each group is independent. 3510 List<WorkItemOverlapMerge> merges = new ArrayList<>(overlapGroups.size()); 3511 List<Future<Void>> rets; 3512 for (Collection<HbckInfo> overlap : overlapGroups.asMap().values()) { 3513 // 3514 merges.add(new WorkItemOverlapMerge(overlap, handler)); 3515 } 3516 try { 3517 rets = executor.invokeAll(merges); 3518 } catch (InterruptedException e) { 3519 LOG.error("Overlap merges were interrupted", e); 3520 return false; 3521 } 3522 for(int i=0; i<merges.size(); i++) { 3523 WorkItemOverlapMerge work = merges.get(i); 3524 Future<Void> f = rets.get(i); 3525 try { 3526 f.get(); 3527 } catch(ExecutionException e) { 3528 LOG.warn("Failed to merge overlap group" + work, e.getCause()); 3529 } catch (InterruptedException e) { 3530 LOG.error("Waiting for overlap merges was interrupted", e); 3531 return false; 3532 } 3533 } 3534 return true; 3535 } 3536 3537 /** 3538 * This dumps data in a visually reasonable way for visual debugging 3539 * 3540 * @param splits 3541 * @param regions 3542 */ 3543 void dump(SortedSet<byte[]> splits, Multimap<byte[], HbckInfo> regions) { 3544 // we display this way because the last end key should be displayed as well. 3545 StringBuilder sb = new StringBuilder(); 3546 for (byte[] k : splits) { 3547 sb.setLength(0); // clear out existing buffer, if any. 3548 sb.append(Bytes.toStringBinary(k) + ":\t"); 3549 for (HbckInfo r : regions.get(k)) { 3550 sb.append("[ "+ r.toString() + ", " 3551 + Bytes.toStringBinary(r.getEndKey())+ "]\t"); 3552 } 3553 errors.print(sb.toString()); 3554 } 3555 } 3556 } 3557 3558 public void dumpOverlapProblems(Multimap<byte[], HbckInfo> regions) { 3559 // we display this way because the last end key should be displayed as 3560 // well. 3561 for (byte[] k : regions.keySet()) { 3562 errors.print(Bytes.toStringBinary(k) + ":"); 3563 for (HbckInfo r : regions.get(k)) { 3564 errors.print("[ " + r.toString() + ", " 3565 + Bytes.toStringBinary(r.getEndKey()) + "]"); 3566 } 3567 errors.print("----"); 3568 } 3569 } 3570 3571 public void dumpSidelinedRegions(Map<Path, HbckInfo> regions) { 3572 for (Map.Entry<Path, HbckInfo> entry: regions.entrySet()) { 3573 TableName tableName = entry.getValue().getTableName(); 3574 Path path = entry.getKey(); 3575 errors.print("This sidelined region dir should be bulk loaded: " 3576 + path.toString()); 3577 errors.print("Bulk load command looks like: " 3578 + "hbase org.apache.hadoop.hbase.tool.LoadIncrementalHFiles " 3579 + path.toUri().getPath() + " "+ tableName); 3580 } 3581 } 3582 3583 public Multimap<byte[], HbckInfo> getOverlapGroups( 3584 TableName table) { 3585 TableInfo ti = tablesInfo.get(table); 3586 return ti.overlapGroups; 3587 } 3588 3589 /** 3590 * Return a list of user-space table names whose metadata have not been 3591 * modified in the last few milliseconds specified by timelag 3592 * if any of the REGIONINFO_QUALIFIER, SERVER_QUALIFIER, STARTCODE_QUALIFIER, 3593 * SPLITA_QUALIFIER, SPLITB_QUALIFIER have not changed in the last 3594 * milliseconds specified by timelag, then the table is a candidate to be returned. 3595 * @return tables that have not been modified recently 3596 * @throws IOException if an error is encountered 3597 */ 3598 TableDescriptor[] getTables(AtomicInteger numSkipped) { 3599 List<TableName> tableNames = new ArrayList<>(); 3600 long now = EnvironmentEdgeManager.currentTime(); 3601 3602 for (HbckInfo hbi : regionInfoMap.values()) { 3603 MetaEntry info = hbi.metaEntry; 3604 3605 // if the start key is zero, then we have found the first region of a table. 3606 // pick only those tables that were not modified in the last few milliseconds. 3607 if (info != null && info.getStartKey().length == 0 && !info.isMetaRegion()) { 3608 if (info.modTime + timelag < now) { 3609 tableNames.add(info.getTable()); 3610 } else { 3611 numSkipped.incrementAndGet(); // one more in-flux table 3612 } 3613 } 3614 } 3615 return getTableDescriptors(tableNames); 3616 } 3617 3618 TableDescriptor[] getTableDescriptors(List<TableName> tableNames) { 3619 LOG.info("getTableDescriptors == tableNames => " + tableNames); 3620 try (Connection conn = ConnectionFactory.createConnection(getConf()); 3621 Admin admin = conn.getAdmin()) { 3622 List<TableDescriptor> tds = admin.listTableDescriptors(tableNames); 3623 return tds.toArray(new TableDescriptor[tds.size()]); 3624 } catch (IOException e) { 3625 LOG.debug("Exception getting table descriptors", e); 3626 } 3627 return new TableDescriptor[0]; 3628 } 3629 3630 /** 3631 * Gets the entry in regionInfo corresponding to the the given encoded 3632 * region name. If the region has not been seen yet, a new entry is added 3633 * and returned. 3634 */ 3635 private synchronized HbckInfo getOrCreateInfo(String name) { 3636 HbckInfo hbi = regionInfoMap.get(name); 3637 if (hbi == null) { 3638 hbi = new HbckInfo(null); 3639 regionInfoMap.put(name, hbi); 3640 } 3641 return hbi; 3642 } 3643 3644 private void checkAndFixReplication() throws ReplicationException { 3645 ReplicationChecker checker = new ReplicationChecker(getConf(), zkw, errors); 3646 checker.checkUnDeletedQueues(); 3647 3648 if (checker.hasUnDeletedQueues() && this.fixReplication) { 3649 checker.fixUnDeletedQueues(); 3650 setShouldRerun(); 3651 } 3652 } 3653 3654 /** 3655 * Check values in regionInfo for hbase:meta 3656 * Check if zero or more than one regions with hbase:meta are found. 3657 * If there are inconsistencies (i.e. zero or more than one regions 3658 * pretend to be holding the hbase:meta) try to fix that and report an error. 3659 * @throws IOException from HBaseFsckRepair functions 3660 * @throws KeeperException 3661 * @throws InterruptedException 3662 */ 3663 boolean checkMetaRegion() throws IOException, KeeperException, InterruptedException { 3664 Map<Integer, HbckInfo> metaRegions = new HashMap<>(); 3665 for (HbckInfo value : regionInfoMap.values()) { 3666 if (value.metaEntry != null && value.metaEntry.isMetaRegion()) { 3667 metaRegions.put(value.getReplicaId(), value); 3668 } 3669 } 3670 int metaReplication = admin.getTableDescriptor(TableName.META_TABLE_NAME) 3671 .getRegionReplication(); 3672 boolean noProblem = true; 3673 // There will be always entries in regionInfoMap corresponding to hbase:meta & its replicas 3674 // Check the deployed servers. It should be exactly one server for each replica. 3675 for (int i = 0; i < metaReplication; i++) { 3676 HbckInfo metaHbckInfo = metaRegions.remove(i); 3677 List<ServerName> servers = new ArrayList<>(); 3678 if (metaHbckInfo != null) { 3679 servers = metaHbckInfo.deployedOn; 3680 } 3681 if (servers.size() != 1) { 3682 noProblem = false; 3683 if (servers.isEmpty()) { 3684 assignMetaReplica(i); 3685 } else if (servers.size() > 1) { 3686 errors 3687 .reportError(ERROR_CODE.MULTI_META_REGION, "hbase:meta, replicaId " + 3688 metaHbckInfo.getReplicaId() + " is found on more than one region."); 3689 if (shouldFixAssignments()) { 3690 errors.print("Trying to fix a problem with hbase:meta, replicaId " + 3691 metaHbckInfo.getReplicaId() +".."); 3692 setShouldRerun(); 3693 // try fix it (treat is a dupe assignment) 3694 HBaseFsckRepair.fixMultiAssignment(connection, metaHbckInfo.metaEntry, servers); 3695 } 3696 } 3697 } 3698 } 3699 // unassign whatever is remaining in metaRegions. They are excess replicas. 3700 for (Map.Entry<Integer, HbckInfo> entry : metaRegions.entrySet()) { 3701 noProblem = false; 3702 errors.reportError(ERROR_CODE.SHOULD_NOT_BE_DEPLOYED, 3703 "hbase:meta replicas are deployed in excess. Configured " + metaReplication + 3704 ", deployed " + metaRegions.size()); 3705 if (shouldFixAssignments()) { 3706 errors.print("Trying to undeploy excess replica, replicaId: " + entry.getKey() + 3707 " of hbase:meta.."); 3708 setShouldRerun(); 3709 unassignMetaReplica(entry.getValue()); 3710 } 3711 } 3712 // if noProblem is false, rerun hbck with hopefully fixed META 3713 // if noProblem is true, no errors, so continue normally 3714 return noProblem; 3715 } 3716 3717 private void unassignMetaReplica(HbckInfo hi) throws IOException, InterruptedException, 3718 KeeperException { 3719 undeployRegions(hi); 3720 ZKUtil.deleteNode(zkw, zkw.getZNodePaths().getZNodeForReplica(hi.metaEntry.getReplicaId())); 3721 } 3722 3723 private void assignMetaReplica(int replicaId) 3724 throws IOException, KeeperException, InterruptedException { 3725 errors.reportError(ERROR_CODE.NO_META_REGION, "hbase:meta, replicaId " + 3726 replicaId +" is not found on any region."); 3727 if (shouldFixAssignments()) { 3728 errors.print("Trying to fix a problem with hbase:meta.."); 3729 setShouldRerun(); 3730 // try to fix it (treat it as unassigned region) 3731 RegionInfo h = RegionReplicaUtil.getRegionInfoForReplica( 3732 RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId); 3733 HBaseFsckRepair.fixUnassigned(admin, h); 3734 HBaseFsckRepair.waitUntilAssigned(admin, h); 3735 } 3736 } 3737 3738 /** 3739 * Scan hbase:meta, adding all regions found to the regionInfo map. 3740 * @throws IOException if an error is encountered 3741 */ 3742 boolean loadMetaEntries() throws IOException { 3743 MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() { 3744 int countRecord = 1; 3745 3746 // comparator to sort KeyValues with latest modtime 3747 final Comparator<Cell> comp = new Comparator<Cell>() { 3748 @Override 3749 public int compare(Cell k1, Cell k2) { 3750 return Long.compare(k1.getTimestamp(), k2.getTimestamp()); 3751 } 3752 }; 3753 3754 @Override 3755 public boolean visit(Result result) throws IOException { 3756 try { 3757 3758 // record the latest modification of this META record 3759 long ts = Collections.max(result.listCells(), comp).getTimestamp(); 3760 RegionLocations rl = MetaTableAccessor.getRegionLocations(result); 3761 if (rl == null) { 3762 emptyRegionInfoQualifiers.add(result); 3763 errors.reportError(ERROR_CODE.EMPTY_META_CELL, 3764 "Empty REGIONINFO_QUALIFIER found in hbase:meta"); 3765 return true; 3766 } 3767 ServerName sn = null; 3768 if (rl.getRegionLocation(RegionInfo.DEFAULT_REPLICA_ID) == null || 3769 rl.getRegionLocation(RegionInfo.DEFAULT_REPLICA_ID).getRegionInfo() == null) { 3770 emptyRegionInfoQualifiers.add(result); 3771 errors.reportError(ERROR_CODE.EMPTY_META_CELL, 3772 "Empty REGIONINFO_QUALIFIER found in hbase:meta"); 3773 return true; 3774 } 3775 RegionInfo hri = rl.getRegionLocation(RegionInfo.DEFAULT_REPLICA_ID).getRegionInfo(); 3776 if (!(isTableIncluded(hri.getTable()) 3777 || hri.isMetaRegion())) { 3778 return true; 3779 } 3780 PairOfSameType<RegionInfo> daughters = MetaTableAccessor.getDaughterRegions(result); 3781 for (HRegionLocation h : rl.getRegionLocations()) { 3782 if (h == null || h.getRegionInfo() == null) { 3783 continue; 3784 } 3785 sn = h.getServerName(); 3786 hri = h.getRegionInfo(); 3787 3788 MetaEntry m = null; 3789 if (hri.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) { 3790 m = new MetaEntry(hri, sn, ts, daughters.getFirst(), daughters.getSecond()); 3791 } else { 3792 m = new MetaEntry(hri, sn, ts, null, null); 3793 } 3794 HbckInfo previous = regionInfoMap.get(hri.getEncodedName()); 3795 if (previous == null) { 3796 regionInfoMap.put(hri.getEncodedName(), new HbckInfo(m)); 3797 } else if (previous.metaEntry == null) { 3798 previous.metaEntry = m; 3799 } else { 3800 throw new IOException("Two entries in hbase:meta are same " + previous); 3801 } 3802 } 3803 PairOfSameType<RegionInfo> mergeRegions = MetaTableAccessor.getMergeRegions(result); 3804 for (RegionInfo mergeRegion : new RegionInfo[] { 3805 mergeRegions.getFirst(), mergeRegions.getSecond() }) { 3806 if (mergeRegion != null) { 3807 // This region is already been merged 3808 HbckInfo hbInfo = getOrCreateInfo(mergeRegion.getEncodedName()); 3809 hbInfo.setMerged(true); 3810 } 3811 } 3812 3813 // show proof of progress to the user, once for every 100 records. 3814 if (countRecord % 100 == 0) { 3815 errors.progress(); 3816 } 3817 countRecord++; 3818 return true; 3819 } catch (RuntimeException e) { 3820 LOG.error("Result=" + result); 3821 throw e; 3822 } 3823 } 3824 }; 3825 if (!checkMetaOnly) { 3826 // Scan hbase:meta to pick up user regions 3827 MetaTableAccessor.fullScanRegions(connection, visitor); 3828 } 3829 3830 errors.print(""); 3831 return true; 3832 } 3833 3834 /** 3835 * Stores the regioninfo entries scanned from META 3836 */ 3837 static class MetaEntry extends HRegionInfo { 3838 ServerName regionServer; // server hosting this region 3839 long modTime; // timestamp of most recent modification metadata 3840 RegionInfo splitA, splitB; //split daughters 3841 3842 public MetaEntry(RegionInfo rinfo, ServerName regionServer, long modTime) { 3843 this(rinfo, regionServer, modTime, null, null); 3844 } 3845 3846 public MetaEntry(RegionInfo rinfo, ServerName regionServer, long modTime, 3847 RegionInfo splitA, RegionInfo splitB) { 3848 super(rinfo); 3849 this.regionServer = regionServer; 3850 this.modTime = modTime; 3851 this.splitA = splitA; 3852 this.splitB = splitB; 3853 } 3854 3855 @Override 3856 public boolean equals(Object o) { 3857 boolean superEq = super.equals(o); 3858 if (!superEq) { 3859 return superEq; 3860 } 3861 3862 MetaEntry me = (MetaEntry) o; 3863 if (!regionServer.equals(me.regionServer)) { 3864 return false; 3865 } 3866 return (modTime == me.modTime); 3867 } 3868 3869 @Override 3870 public int hashCode() { 3871 int hash = Arrays.hashCode(getRegionName()); 3872 hash = (int) (hash ^ getRegionId()); 3873 hash ^= Arrays.hashCode(getStartKey()); 3874 hash ^= Arrays.hashCode(getEndKey()); 3875 hash ^= Boolean.valueOf(isOffline()).hashCode(); 3876 hash ^= getTable().hashCode(); 3877 if (regionServer != null) { 3878 hash ^= regionServer.hashCode(); 3879 } 3880 hash = (int) (hash ^ modTime); 3881 return hash; 3882 } 3883 } 3884 3885 /** 3886 * Stores the regioninfo entries from HDFS 3887 */ 3888 static class HdfsEntry { 3889 RegionInfo hri; 3890 Path hdfsRegionDir = null; 3891 long hdfsRegionDirModTime = 0; 3892 boolean hdfsRegioninfoFilePresent = false; 3893 boolean hdfsOnlyEdits = false; 3894 } 3895 3896 /** 3897 * Stores the regioninfo retrieved from Online region servers. 3898 */ 3899 static class OnlineEntry { 3900 RegionInfo hri; 3901 ServerName hsa; 3902 3903 @Override 3904 public String toString() { 3905 return hsa.toString() + ";" + hri.getRegionNameAsString(); 3906 } 3907 } 3908 3909 /** 3910 * Maintain information about a particular region. It gathers information 3911 * from three places -- HDFS, META, and region servers. 3912 */ 3913 public static class HbckInfo implements KeyRange { 3914 private MetaEntry metaEntry = null; // info in META 3915 private HdfsEntry hdfsEntry = null; // info in HDFS 3916 private List<OnlineEntry> deployedEntries = Lists.newArrayList(); // on Region Server 3917 private List<ServerName> deployedOn = Lists.newArrayList(); // info on RS's 3918 private boolean skipChecks = false; // whether to skip further checks to this region info. 3919 private boolean isMerged = false;// whether this region has already been merged into another one 3920 private int deployedReplicaId = RegionInfo.DEFAULT_REPLICA_ID; 3921 private RegionInfo primaryHRIForDeployedReplica = null; 3922 3923 HbckInfo(MetaEntry metaEntry) { 3924 this.metaEntry = metaEntry; 3925 } 3926 3927 public synchronized int getReplicaId() { 3928 return metaEntry != null? metaEntry.getReplicaId(): deployedReplicaId; 3929 } 3930 3931 public synchronized void addServer(RegionInfo hri, ServerName server) { 3932 OnlineEntry rse = new OnlineEntry() ; 3933 rse.hri = hri; 3934 rse.hsa = server; 3935 this.deployedEntries.add(rse); 3936 this.deployedOn.add(server); 3937 // save the replicaId that we see deployed in the cluster 3938 this.deployedReplicaId = hri.getReplicaId(); 3939 this.primaryHRIForDeployedReplica = 3940 RegionReplicaUtil.getRegionInfoForDefaultReplica(hri); 3941 } 3942 3943 @Override 3944 public synchronized String toString() { 3945 StringBuilder sb = new StringBuilder(); 3946 sb.append("{ meta => "); 3947 sb.append((metaEntry != null)? metaEntry.getRegionNameAsString() : "null"); 3948 sb.append( ", hdfs => " + getHdfsRegionDir()); 3949 sb.append( ", deployed => " + Joiner.on(", ").join(deployedEntries)); 3950 sb.append( ", replicaId => " + getReplicaId()); 3951 sb.append(" }"); 3952 return sb.toString(); 3953 } 3954 3955 @Override 3956 public byte[] getStartKey() { 3957 if (this.metaEntry != null) { 3958 return this.metaEntry.getStartKey(); 3959 } else if (this.hdfsEntry != null) { 3960 return this.hdfsEntry.hri.getStartKey(); 3961 } else { 3962 LOG.error("Entry " + this + " has no meta or hdfs region start key."); 3963 return null; 3964 } 3965 } 3966 3967 @Override 3968 public byte[] getEndKey() { 3969 if (this.metaEntry != null) { 3970 return this.metaEntry.getEndKey(); 3971 } else if (this.hdfsEntry != null) { 3972 return this.hdfsEntry.hri.getEndKey(); 3973 } else { 3974 LOG.error("Entry " + this + " has no meta or hdfs region start key."); 3975 return null; 3976 } 3977 } 3978 3979 public TableName getTableName() { 3980 if (this.metaEntry != null) { 3981 return this.metaEntry.getTable(); 3982 } else if (this.hdfsEntry != null) { 3983 // we are only guaranteed to have a path and not an HRI for hdfsEntry, 3984 // so we get the name from the Path 3985 Path tableDir = this.hdfsEntry.hdfsRegionDir.getParent(); 3986 return FSUtils.getTableName(tableDir); 3987 } else { 3988 // return the info from the first online/deployed hri 3989 for (OnlineEntry e : deployedEntries) { 3990 return e.hri.getTable(); 3991 } 3992 return null; 3993 } 3994 } 3995 3996 public String getRegionNameAsString() { 3997 if (metaEntry != null) { 3998 return metaEntry.getRegionNameAsString(); 3999 } else if (hdfsEntry != null) { 4000 if (hdfsEntry.hri != null) { 4001 return hdfsEntry.hri.getRegionNameAsString(); 4002 } 4003 } else { 4004 // return the info from the first online/deployed hri 4005 for (OnlineEntry e : deployedEntries) { 4006 return e.hri.getRegionNameAsString(); 4007 } 4008 } 4009 return null; 4010 } 4011 4012 public byte[] getRegionName() { 4013 if (metaEntry != null) { 4014 return metaEntry.getRegionName(); 4015 } else if (hdfsEntry != null) { 4016 return hdfsEntry.hri.getRegionName(); 4017 } else { 4018 // return the info from the first online/deployed hri 4019 for (OnlineEntry e : deployedEntries) { 4020 return e.hri.getRegionName(); 4021 } 4022 return null; 4023 } 4024 } 4025 4026 public RegionInfo getPrimaryHRIForDeployedReplica() { 4027 return primaryHRIForDeployedReplica; 4028 } 4029 4030 Path getHdfsRegionDir() { 4031 if (hdfsEntry == null) { 4032 return null; 4033 } 4034 return hdfsEntry.hdfsRegionDir; 4035 } 4036 4037 boolean containsOnlyHdfsEdits() { 4038 if (hdfsEntry == null) { 4039 return false; 4040 } 4041 return hdfsEntry.hdfsOnlyEdits; 4042 } 4043 4044 boolean isHdfsRegioninfoPresent() { 4045 if (hdfsEntry == null) { 4046 return false; 4047 } 4048 return hdfsEntry.hdfsRegioninfoFilePresent; 4049 } 4050 4051 long getModTime() { 4052 if (hdfsEntry == null) { 4053 return 0; 4054 } 4055 return hdfsEntry.hdfsRegionDirModTime; 4056 } 4057 4058 RegionInfo getHdfsHRI() { 4059 if (hdfsEntry == null) { 4060 return null; 4061 } 4062 return hdfsEntry.hri; 4063 } 4064 4065 public void setSkipChecks(boolean skipChecks) { 4066 this.skipChecks = skipChecks; 4067 } 4068 4069 public boolean isSkipChecks() { 4070 return skipChecks; 4071 } 4072 4073 public void setMerged(boolean isMerged) { 4074 this.isMerged = isMerged; 4075 } 4076 4077 public boolean isMerged() { 4078 return this.isMerged; 4079 } 4080 } 4081 4082 final static Comparator<HbckInfo> cmp = new Comparator<HbckInfo>() { 4083 @Override 4084 public int compare(HbckInfo l, HbckInfo r) { 4085 if (l == r) { 4086 // same instance 4087 return 0; 4088 } 4089 4090 int tableCompare = l.getTableName().compareTo(r.getTableName()); 4091 if (tableCompare != 0) { 4092 return tableCompare; 4093 } 4094 4095 int startComparison = RegionSplitCalculator.BYTES_COMPARATOR.compare( 4096 l.getStartKey(), r.getStartKey()); 4097 if (startComparison != 0) { 4098 return startComparison; 4099 } 4100 4101 // Special case for absolute endkey 4102 byte[] endKey = r.getEndKey(); 4103 endKey = (endKey.length == 0) ? null : endKey; 4104 byte[] endKey2 = l.getEndKey(); 4105 endKey2 = (endKey2.length == 0) ? null : endKey2; 4106 int endComparison = RegionSplitCalculator.BYTES_COMPARATOR.compare( 4107 endKey2, endKey); 4108 4109 if (endComparison != 0) { 4110 return endComparison; 4111 } 4112 4113 // use regionId as tiebreaker. 4114 // Null is considered after all possible values so make it bigger. 4115 if (l.hdfsEntry == null && r.hdfsEntry == null) { 4116 return 0; 4117 } 4118 if (l.hdfsEntry == null && r.hdfsEntry != null) { 4119 return 1; 4120 } 4121 // l.hdfsEntry must not be null 4122 if (r.hdfsEntry == null) { 4123 return -1; 4124 } 4125 // both l.hdfsEntry and r.hdfsEntry must not be null. 4126 return Long.compare(l.hdfsEntry.hri.getRegionId(), r.hdfsEntry.hri.getRegionId()); 4127 } 4128 }; 4129 4130 /** 4131 * Prints summary of all tables found on the system. 4132 */ 4133 private void printTableSummary(SortedMap<TableName, TableInfo> tablesInfo) { 4134 StringBuilder sb = new StringBuilder(); 4135 int numOfSkippedRegions; 4136 errors.print("Summary:"); 4137 for (TableInfo tInfo : tablesInfo.values()) { 4138 numOfSkippedRegions = (skippedRegions.containsKey(tInfo.getName())) ? 4139 skippedRegions.get(tInfo.getName()).size() : 0; 4140 4141 if (errors.tableHasErrors(tInfo)) { 4142 errors.print("Table " + tInfo.getName() + " is inconsistent."); 4143 } else if (numOfSkippedRegions > 0){ 4144 errors.print("Table " + tInfo.getName() + " is okay (with " 4145 + numOfSkippedRegions + " skipped regions)."); 4146 } 4147 else { 4148 errors.print("Table " + tInfo.getName() + " is okay."); 4149 } 4150 errors.print(" Number of regions: " + tInfo.getNumRegions()); 4151 if (numOfSkippedRegions > 0) { 4152 Set<String> skippedRegionStrings = skippedRegions.get(tInfo.getName()); 4153 System.out.println(" Number of skipped regions: " + numOfSkippedRegions); 4154 System.out.println(" List of skipped regions:"); 4155 for(String sr : skippedRegionStrings) { 4156 System.out.println(" " + sr); 4157 } 4158 } 4159 sb.setLength(0); // clear out existing buffer, if any. 4160 sb.append(" Deployed on: "); 4161 for (ServerName server : tInfo.deployedOn) { 4162 sb.append(" " + server.toString()); 4163 } 4164 errors.print(sb.toString()); 4165 } 4166 } 4167 4168 static ErrorReporter getErrorReporter( 4169 final Configuration conf) throws ClassNotFoundException { 4170 Class<? extends ErrorReporter> reporter = conf.getClass("hbasefsck.errorreporter", PrintingErrorReporter.class, ErrorReporter.class); 4171 return ReflectionUtils.newInstance(reporter, conf); 4172 } 4173 4174 public interface ErrorReporter { 4175 enum ERROR_CODE { 4176 UNKNOWN, NO_META_REGION, NULL_META_REGION, NO_VERSION_FILE, NOT_IN_META_HDFS, NOT_IN_META, 4177 NOT_IN_META_OR_DEPLOYED, NOT_IN_HDFS_OR_DEPLOYED, NOT_IN_HDFS, SERVER_DOES_NOT_MATCH_META, 4178 NOT_DEPLOYED, MULTI_DEPLOYED, SHOULD_NOT_BE_DEPLOYED, MULTI_META_REGION, RS_CONNECT_FAILURE, 4179 FIRST_REGION_STARTKEY_NOT_EMPTY, LAST_REGION_ENDKEY_NOT_EMPTY, DUPE_STARTKEYS, 4180 HOLE_IN_REGION_CHAIN, OVERLAP_IN_REGION_CHAIN, REGION_CYCLE, DEGENERATE_REGION, 4181 ORPHAN_HDFS_REGION, LINGERING_SPLIT_PARENT, NO_TABLEINFO_FILE, LINGERING_REFERENCE_HFILE, 4182 LINGERING_HFILELINK, WRONG_USAGE, EMPTY_META_CELL, EXPIRED_TABLE_LOCK, BOUNDARIES_ERROR, 4183 ORPHAN_TABLE_STATE, NO_TABLE_STATE, UNDELETED_REPLICATION_QUEUE, DUPE_ENDKEYS, 4184 UNSUPPORTED_OPTION, INVALID_TABLE 4185 } 4186 void clear(); 4187 void report(String message); 4188 void reportError(String message); 4189 void reportError(ERROR_CODE errorCode, String message); 4190 void reportError(ERROR_CODE errorCode, String message, TableInfo table); 4191 void reportError(ERROR_CODE errorCode, String message, TableInfo table, HbckInfo info); 4192 void reportError( 4193 ERROR_CODE errorCode, 4194 String message, 4195 TableInfo table, 4196 HbckInfo info1, 4197 HbckInfo info2 4198 ); 4199 int summarize(); 4200 void detail(String details); 4201 ArrayList<ERROR_CODE> getErrorList(); 4202 void progress(); 4203 void print(String message); 4204 void resetErrors(); 4205 boolean tableHasErrors(TableInfo table); 4206 } 4207 4208 static class PrintingErrorReporter implements ErrorReporter { 4209 public int errorCount = 0; 4210 private int showProgress; 4211 // How frequently calls to progress() will create output 4212 private static final int progressThreshold = 100; 4213 4214 Set<TableInfo> errorTables = new HashSet<>(); 4215 4216 // for use by unit tests to verify which errors were discovered 4217 private ArrayList<ERROR_CODE> errorList = new ArrayList<>(); 4218 4219 @Override 4220 public void clear() { 4221 errorTables.clear(); 4222 errorList.clear(); 4223 errorCount = 0; 4224 } 4225 4226 @Override 4227 public synchronized void reportError(ERROR_CODE errorCode, String message) { 4228 if (errorCode == ERROR_CODE.WRONG_USAGE) { 4229 System.err.println(message); 4230 return; 4231 } 4232 4233 errorList.add(errorCode); 4234 if (!summary) { 4235 System.out.println("ERROR: " + message); 4236 } 4237 errorCount++; 4238 showProgress = 0; 4239 } 4240 4241 @Override 4242 public synchronized void reportError(ERROR_CODE errorCode, String message, TableInfo table) { 4243 errorTables.add(table); 4244 reportError(errorCode, message); 4245 } 4246 4247 @Override 4248 public synchronized void reportError(ERROR_CODE errorCode, String message, TableInfo table, 4249 HbckInfo info) { 4250 errorTables.add(table); 4251 String reference = "(region " + info.getRegionNameAsString() + ")"; 4252 reportError(errorCode, reference + " " + message); 4253 } 4254 4255 @Override 4256 public synchronized void reportError(ERROR_CODE errorCode, String message, TableInfo table, 4257 HbckInfo info1, HbckInfo info2) { 4258 errorTables.add(table); 4259 String reference = "(regions " + info1.getRegionNameAsString() 4260 + " and " + info2.getRegionNameAsString() + ")"; 4261 reportError(errorCode, reference + " " + message); 4262 } 4263 4264 @Override 4265 public synchronized void reportError(String message) { 4266 reportError(ERROR_CODE.UNKNOWN, message); 4267 } 4268 4269 /** 4270 * Report error information, but do not increment the error count. Intended for cases 4271 * where the actual error would have been reported previously. 4272 * @param message 4273 */ 4274 @Override 4275 public synchronized void report(String message) { 4276 if (! summary) { 4277 System.out.println("ERROR: " + message); 4278 } 4279 showProgress = 0; 4280 } 4281 4282 @Override 4283 public synchronized int summarize() { 4284 System.out.println(Integer.toString(errorCount) + 4285 " inconsistencies detected."); 4286 if (errorCount == 0) { 4287 System.out.println("Status: OK"); 4288 return 0; 4289 } else { 4290 System.out.println("Status: INCONSISTENT"); 4291 return -1; 4292 } 4293 } 4294 4295 @Override 4296 public ArrayList<ERROR_CODE> getErrorList() { 4297 return errorList; 4298 } 4299 4300 @Override 4301 public synchronized void print(String message) { 4302 if (!summary) { 4303 System.out.println(message); 4304 } 4305 } 4306 4307 @Override 4308 public boolean tableHasErrors(TableInfo table) { 4309 return errorTables.contains(table); 4310 } 4311 4312 @Override 4313 public void resetErrors() { 4314 errorCount = 0; 4315 } 4316 4317 @Override 4318 public synchronized void detail(String message) { 4319 if (details) { 4320 System.out.println(message); 4321 } 4322 showProgress = 0; 4323 } 4324 4325 @Override 4326 public synchronized void progress() { 4327 if (showProgress++ == progressThreshold) { 4328 if (!summary) { 4329 System.out.print("."); 4330 } 4331 showProgress = 0; 4332 } 4333 } 4334 } 4335 4336 /** 4337 * Contact a region server and get all information from it 4338 */ 4339 static class WorkItemRegion implements Callable<Void> { 4340 private final HBaseFsck hbck; 4341 private final ServerName rsinfo; 4342 private final ErrorReporter errors; 4343 private final ClusterConnection connection; 4344 4345 WorkItemRegion(HBaseFsck hbck, ServerName info, 4346 ErrorReporter errors, ClusterConnection connection) { 4347 this.hbck = hbck; 4348 this.rsinfo = info; 4349 this.errors = errors; 4350 this.connection = connection; 4351 } 4352 4353 @Override 4354 public synchronized Void call() throws IOException { 4355 errors.progress(); 4356 try { 4357 BlockingInterface server = connection.getAdmin(rsinfo); 4358 4359 // list all online regions from this region server 4360 List<RegionInfo> regions = ProtobufUtil.getOnlineRegions(server); 4361 regions = filterRegions(regions); 4362 4363 if (details) { 4364 errors.detail("RegionServer: " + rsinfo.getServerName() + 4365 " number of regions: " + regions.size()); 4366 for (RegionInfo rinfo: regions) { 4367 errors.detail(" " + rinfo.getRegionNameAsString() + 4368 " id: " + rinfo.getRegionId() + 4369 " encoded_name: " + rinfo.getEncodedName() + 4370 " start: " + Bytes.toStringBinary(rinfo.getStartKey()) + 4371 " end: " + Bytes.toStringBinary(rinfo.getEndKey())); 4372 } 4373 } 4374 4375 // check to see if the existence of this region matches the region in META 4376 for (RegionInfo r:regions) { 4377 HbckInfo hbi = hbck.getOrCreateInfo(r.getEncodedName()); 4378 hbi.addServer(r, rsinfo); 4379 } 4380 } catch (IOException e) { // unable to connect to the region server. 4381 errors.reportError(ERROR_CODE.RS_CONNECT_FAILURE, "RegionServer: " + rsinfo.getServerName() + 4382 " Unable to fetch region information. " + e); 4383 throw e; 4384 } 4385 return null; 4386 } 4387 4388 private List<RegionInfo> filterRegions(List<RegionInfo> regions) { 4389 List<RegionInfo> ret = Lists.newArrayList(); 4390 for (RegionInfo hri : regions) { 4391 if (hri.isMetaRegion() || (!hbck.checkMetaOnly 4392 && hbck.isTableIncluded(hri.getTable()))) { 4393 ret.add(hri); 4394 } 4395 } 4396 return ret; 4397 } 4398 } 4399 4400 /** 4401 * Contact hdfs and get all information about specified table directory into 4402 * regioninfo list. 4403 */ 4404 class WorkItemHdfsDir implements Callable<Void> { 4405 private FileStatus tableDir; 4406 private ErrorReporter errors; 4407 private FileSystem fs; 4408 4409 WorkItemHdfsDir(FileSystem fs, ErrorReporter errors, 4410 FileStatus status) { 4411 this.fs = fs; 4412 this.tableDir = status; 4413 this.errors = errors; 4414 } 4415 4416 @Override 4417 public synchronized Void call() throws InterruptedException, ExecutionException { 4418 final Vector<Exception> exceptions = new Vector<>(); 4419 4420 try { 4421 final FileStatus[] regionDirs = fs.listStatus(tableDir.getPath()); 4422 final List<Future<?>> futures = new ArrayList<>(regionDirs.length); 4423 4424 for (final FileStatus regionDir : regionDirs) { 4425 errors.progress(); 4426 final String encodedName = regionDir.getPath().getName(); 4427 // ignore directories that aren't hexadecimal 4428 if (!encodedName.toLowerCase(Locale.ROOT).matches("[0-9a-f]+")) { 4429 continue; 4430 } 4431 4432 if (!exceptions.isEmpty()) { 4433 break; 4434 } 4435 4436 futures.add(executor.submit(new Runnable() { 4437 @Override 4438 public void run() { 4439 try { 4440 LOG.debug("Loading region info from hdfs:"+ regionDir.getPath()); 4441 4442 Path regioninfoFile = new Path(regionDir.getPath(), HRegionFileSystem.REGION_INFO_FILE); 4443 boolean regioninfoFileExists = fs.exists(regioninfoFile); 4444 4445 if (!regioninfoFileExists) { 4446 // As tables become larger it is more and more likely that by the time you 4447 // reach a given region that it will be gone due to region splits/merges. 4448 if (!fs.exists(regionDir.getPath())) { 4449 LOG.warn("By the time we tried to process this region dir it was already gone: " 4450 + regionDir.getPath()); 4451 return; 4452 } 4453 } 4454 4455 HbckInfo hbi = HBaseFsck.this.getOrCreateInfo(encodedName); 4456 HdfsEntry he = new HdfsEntry(); 4457 synchronized (hbi) { 4458 if (hbi.getHdfsRegionDir() != null) { 4459 errors.print("Directory " + encodedName + " duplicate??" + 4460 hbi.getHdfsRegionDir()); 4461 } 4462 4463 he.hdfsRegionDir = regionDir.getPath(); 4464 he.hdfsRegionDirModTime = regionDir.getModificationTime(); 4465 he.hdfsRegioninfoFilePresent = regioninfoFileExists; 4466 // we add to orphan list when we attempt to read .regioninfo 4467 4468 // Set a flag if this region contains only edits 4469 // This is special case if a region is left after split 4470 he.hdfsOnlyEdits = true; 4471 FileStatus[] subDirs = fs.listStatus(regionDir.getPath()); 4472 Path ePath = WALSplitter.getRegionDirRecoveredEditsDir(regionDir.getPath()); 4473 for (FileStatus subDir : subDirs) { 4474 errors.progress(); 4475 String sdName = subDir.getPath().getName(); 4476 if (!sdName.startsWith(".") && !sdName.equals(ePath.getName())) { 4477 he.hdfsOnlyEdits = false; 4478 break; 4479 } 4480 } 4481 hbi.hdfsEntry = he; 4482 } 4483 } catch (Exception e) { 4484 LOG.error("Could not load region dir", e); 4485 exceptions.add(e); 4486 } 4487 } 4488 })); 4489 } 4490 4491 // Ensure all pending tasks are complete (or that we run into an exception) 4492 for (Future<?> f : futures) { 4493 if (!exceptions.isEmpty()) { 4494 break; 4495 } 4496 try { 4497 f.get(); 4498 } catch (ExecutionException e) { 4499 LOG.error("Unexpected exec exception! Should've been caught already. (Bug?)", e); 4500 // Shouldn't happen, we already logged/caught any exceptions in the Runnable 4501 }; 4502 } 4503 } catch (IOException e) { 4504 LOG.error("Cannot execute WorkItemHdfsDir for " + tableDir, e); 4505 exceptions.add(e); 4506 } finally { 4507 if (!exceptions.isEmpty()) { 4508 errors.reportError(ERROR_CODE.RS_CONNECT_FAILURE, "Table Directory: " 4509 + tableDir.getPath().getName() 4510 + " Unable to fetch all HDFS region information. "); 4511 // Just throw the first exception as an indication something bad happened 4512 // Don't need to propagate all the exceptions, we already logged them all anyway 4513 throw new ExecutionException("First exception in WorkItemHdfsDir", exceptions.firstElement()); 4514 } 4515 } 4516 return null; 4517 } 4518 } 4519 4520 /** 4521 * Contact hdfs and get all information about specified table directory into 4522 * regioninfo list. 4523 */ 4524 static class WorkItemHdfsRegionInfo implements Callable<Void> { 4525 private HbckInfo hbi; 4526 private HBaseFsck hbck; 4527 private ErrorReporter errors; 4528 4529 WorkItemHdfsRegionInfo(HbckInfo hbi, HBaseFsck hbck, ErrorReporter errors) { 4530 this.hbi = hbi; 4531 this.hbck = hbck; 4532 this.errors = errors; 4533 } 4534 4535 @Override 4536 public synchronized Void call() throws IOException { 4537 // only load entries that haven't been loaded yet. 4538 if (hbi.getHdfsHRI() == null) { 4539 try { 4540 errors.progress(); 4541 hbck.loadHdfsRegioninfo(hbi); 4542 } catch (IOException ioe) { 4543 String msg = "Orphan region in HDFS: Unable to load .regioninfo from table " 4544 + hbi.getTableName() + " in hdfs dir " 4545 + hbi.getHdfsRegionDir() 4546 + "! It may be an invalid format or version file. Treating as " 4547 + "an orphaned regiondir."; 4548 errors.reportError(ERROR_CODE.ORPHAN_HDFS_REGION, msg); 4549 try { 4550 hbck.debugLsr(hbi.getHdfsRegionDir()); 4551 } catch (IOException ioe2) { 4552 LOG.error("Unable to read directory " + hbi.getHdfsRegionDir(), ioe2); 4553 throw ioe2; 4554 } 4555 hbck.orphanHdfsDirs.add(hbi); 4556 throw ioe; 4557 } 4558 } 4559 return null; 4560 } 4561 }; 4562 4563 /** 4564 * Display the full report from fsck. This displays all live and dead region 4565 * servers, and all known regions. 4566 */ 4567 public static void setDisplayFullReport() { 4568 details = true; 4569 } 4570 4571 /** 4572 * Set exclusive mode. 4573 */ 4574 public static void setForceExclusive() { 4575 forceExclusive = true; 4576 } 4577 4578 /** 4579 * Only one instance of hbck can modify HBase at a time. 4580 */ 4581 public boolean isExclusive() { 4582 return fixAny || forceExclusive; 4583 } 4584 4585 /** 4586 * Set summary mode. 4587 * Print only summary of the tables and status (OK or INCONSISTENT) 4588 */ 4589 static void setSummary() { 4590 summary = true; 4591 } 4592 4593 /** 4594 * Set hbase:meta check mode. 4595 * Print only info about hbase:meta table deployment/state 4596 */ 4597 void setCheckMetaOnly() { 4598 checkMetaOnly = true; 4599 } 4600 4601 /** 4602 * Set region boundaries check mode. 4603 */ 4604 void setRegionBoundariesCheck() { 4605 checkRegionBoundaries = true; 4606 } 4607 4608 /** 4609 * Set replication fix mode. 4610 */ 4611 public void setFixReplication(boolean shouldFix) { 4612 fixReplication = shouldFix; 4613 fixAny |= shouldFix; 4614 } 4615 4616 public void setCleanReplicationBarrier(boolean shouldClean) { 4617 cleanReplicationBarrier = shouldClean; 4618 } 4619 4620 /** 4621 * Check if we should rerun fsck again. This checks if we've tried to 4622 * fix something and we should rerun fsck tool again. 4623 * Display the full report from fsck. This displays all live and dead 4624 * region servers, and all known regions. 4625 */ 4626 void setShouldRerun() { 4627 rerun = true; 4628 } 4629 4630 public boolean shouldRerun() { 4631 return rerun; 4632 } 4633 4634 /** 4635 * Fix inconsistencies found by fsck. This should try to fix errors (if any) 4636 * found by fsck utility. 4637 */ 4638 public void setFixAssignments(boolean shouldFix) { 4639 fixAssignments = shouldFix; 4640 fixAny |= shouldFix; 4641 } 4642 4643 boolean shouldFixAssignments() { 4644 return fixAssignments; 4645 } 4646 4647 public void setFixMeta(boolean shouldFix) { 4648 fixMeta = shouldFix; 4649 fixAny |= shouldFix; 4650 } 4651 4652 boolean shouldFixMeta() { 4653 return fixMeta; 4654 } 4655 4656 public void setFixEmptyMetaCells(boolean shouldFix) { 4657 fixEmptyMetaCells = shouldFix; 4658 fixAny |= shouldFix; 4659 } 4660 4661 boolean shouldFixEmptyMetaCells() { 4662 return fixEmptyMetaCells; 4663 } 4664 4665 public void setCheckHdfs(boolean checking) { 4666 checkHdfs = checking; 4667 } 4668 4669 boolean shouldCheckHdfs() { 4670 return checkHdfs; 4671 } 4672 4673 public void setFixHdfsHoles(boolean shouldFix) { 4674 fixHdfsHoles = shouldFix; 4675 fixAny |= shouldFix; 4676 } 4677 4678 boolean shouldFixHdfsHoles() { 4679 return fixHdfsHoles; 4680 } 4681 4682 public void setFixTableOrphans(boolean shouldFix) { 4683 fixTableOrphans = shouldFix; 4684 fixAny |= shouldFix; 4685 } 4686 4687 boolean shouldFixTableOrphans() { 4688 return fixTableOrphans; 4689 } 4690 4691 public void setFixHdfsOverlaps(boolean shouldFix) { 4692 fixHdfsOverlaps = shouldFix; 4693 fixAny |= shouldFix; 4694 } 4695 4696 boolean shouldFixHdfsOverlaps() { 4697 return fixHdfsOverlaps; 4698 } 4699 4700 public void setFixHdfsOrphans(boolean shouldFix) { 4701 fixHdfsOrphans = shouldFix; 4702 fixAny |= shouldFix; 4703 } 4704 4705 boolean shouldFixHdfsOrphans() { 4706 return fixHdfsOrphans; 4707 } 4708 4709 public void setFixVersionFile(boolean shouldFix) { 4710 fixVersionFile = shouldFix; 4711 fixAny |= shouldFix; 4712 } 4713 4714 public boolean shouldFixVersionFile() { 4715 return fixVersionFile; 4716 } 4717 4718 public void setSidelineBigOverlaps(boolean sbo) { 4719 this.sidelineBigOverlaps = sbo; 4720 } 4721 4722 public boolean shouldSidelineBigOverlaps() { 4723 return sidelineBigOverlaps; 4724 } 4725 4726 public void setFixSplitParents(boolean shouldFix) { 4727 fixSplitParents = shouldFix; 4728 fixAny |= shouldFix; 4729 } 4730 4731 public void setRemoveParents(boolean shouldFix) { 4732 removeParents = shouldFix; 4733 fixAny |= shouldFix; 4734 } 4735 4736 boolean shouldFixSplitParents() { 4737 return fixSplitParents; 4738 } 4739 4740 boolean shouldRemoveParents() { 4741 return removeParents; 4742 } 4743 4744 public void setFixReferenceFiles(boolean shouldFix) { 4745 fixReferenceFiles = shouldFix; 4746 fixAny |= shouldFix; 4747 } 4748 4749 boolean shouldFixReferenceFiles() { 4750 return fixReferenceFiles; 4751 } 4752 4753 public void setFixHFileLinks(boolean shouldFix) { 4754 fixHFileLinks = shouldFix; 4755 fixAny |= shouldFix; 4756 } 4757 4758 boolean shouldFixHFileLinks() { 4759 return fixHFileLinks; 4760 } 4761 4762 public boolean shouldIgnorePreCheckPermission() { 4763 return !fixAny || ignorePreCheckPermission; 4764 } 4765 4766 public void setIgnorePreCheckPermission(boolean ignorePreCheckPermission) { 4767 this.ignorePreCheckPermission = ignorePreCheckPermission; 4768 } 4769 4770 /** 4771 * @param mm maximum number of regions to merge into a single region. 4772 */ 4773 public void setMaxMerge(int mm) { 4774 this.maxMerge = mm; 4775 } 4776 4777 public int getMaxMerge() { 4778 return maxMerge; 4779 } 4780 4781 public void setMaxOverlapsToSideline(int mo) { 4782 this.maxOverlapsToSideline = mo; 4783 } 4784 4785 public int getMaxOverlapsToSideline() { 4786 return maxOverlapsToSideline; 4787 } 4788 4789 /** 4790 * Only check/fix tables specified by the list, 4791 * Empty list means all tables are included. 4792 */ 4793 boolean isTableIncluded(TableName table) { 4794 return (tablesIncluded.isEmpty()) || tablesIncluded.contains(table); 4795 } 4796 4797 public void includeTable(TableName table) { 4798 tablesIncluded.add(table); 4799 } 4800 4801 Set<TableName> getIncludedTables() { 4802 return new HashSet<>(tablesIncluded); 4803 } 4804 4805 /** 4806 * We are interested in only those tables that have not changed their state in 4807 * hbase:meta during the last few seconds specified by hbase.admin.fsck.timelag 4808 * @param seconds - the time in seconds 4809 */ 4810 public void setTimeLag(long seconds) { 4811 timelag = seconds * 1000; // convert to milliseconds 4812 } 4813 4814 /** 4815 * 4816 * @param sidelineDir - HDFS path to sideline data 4817 */ 4818 public void setSidelineDir(String sidelineDir) { 4819 this.sidelineDir = new Path(sidelineDir); 4820 } 4821 4822 protected HFileCorruptionChecker createHFileCorruptionChecker(boolean sidelineCorruptHFiles) throws IOException { 4823 return new HFileCorruptionChecker(getConf(), executor, sidelineCorruptHFiles); 4824 } 4825 4826 public HFileCorruptionChecker getHFilecorruptionChecker() { 4827 return hfcc; 4828 } 4829 4830 public void setHFileCorruptionChecker(HFileCorruptionChecker hfcc) { 4831 this.hfcc = hfcc; 4832 } 4833 4834 public void setRetCode(int code) { 4835 this.retcode = code; 4836 } 4837 4838 public int getRetCode() { 4839 return retcode; 4840 } 4841 4842 protected HBaseFsck printUsageAndExit() { 4843 StringWriter sw = new StringWriter(2048); 4844 PrintWriter out = new PrintWriter(sw); 4845 out.println(""); 4846 out.println("-----------------------------------------------------------------------"); 4847 out.println("NOTE: As of HBase version 2.0, the hbck tool is significantly changed."); 4848 out.println("In general, all Read-Only options are supported and can be be used"); 4849 out.println("safely. Most -fix/ -repair options are NOT supported. Please see usage"); 4850 out.println("below for details on which options are not supported."); 4851 out.println("-----------------------------------------------------------------------"); 4852 out.println(""); 4853 out.println("Usage: fsck [opts] {only tables}"); 4854 out.println(" where [opts] are:"); 4855 out.println(" -help Display help options (this)"); 4856 out.println(" -details Display full report of all regions."); 4857 out.println(" -timelag <timeInSeconds> Process only regions that " + 4858 " have not experienced any metadata updates in the last " + 4859 " <timeInSeconds> seconds."); 4860 out.println(" -sleepBeforeRerun <timeInSeconds> Sleep this many seconds" + 4861 " before checking if the fix worked if run with -fix"); 4862 out.println(" -summary Print only summary of the tables and status."); 4863 out.println(" -metaonly Only check the state of the hbase:meta table."); 4864 out.println(" -sidelineDir <hdfs://> HDFS path to backup existing meta."); 4865 out.println(" -boundaries Verify that regions boundaries are the same between META and store files."); 4866 out.println(" -exclusive Abort if another hbck is exclusive or fixing."); 4867 4868 out.println(""); 4869 out.println(" Datafile Repair options: (expert features, use with caution!)"); 4870 out.println(" -checkCorruptHFiles Check all Hfiles by opening them to make sure they are valid"); 4871 out.println(" -sidelineCorruptHFiles Quarantine corrupted HFiles. implies -checkCorruptHFiles"); 4872 4873 out.println(""); 4874 out.println(" Replication options"); 4875 out.println(" -fixReplication Deletes replication queues for removed peers"); 4876 4877 out.println(""); 4878 out.println(" Metadata Repair options supported as of version 2.0: (expert features, use with caution!)"); 4879 out.println(" -fixVersionFile Try to fix missing hbase.version file in hdfs."); 4880 out.println(" -fixReferenceFiles Try to offline lingering reference store files"); 4881 out.println(" -fixHFileLinks Try to offline lingering HFileLinks"); 4882 out.println(" -noHdfsChecking Don't load/check region info from HDFS." 4883 + " Assumes hbase:meta region info is good. Won't check/fix any HDFS issue, e.g. hole, orphan, or overlap"); 4884 out.println(" -ignorePreCheckPermission ignore filesystem permission pre-check"); 4885 4886 out.println(""); 4887 out.println("NOTE: Following options are NOT supported as of HBase version 2.0+."); 4888 out.println(""); 4889 out.println(" UNSUPPORTED Metadata Repair options: (expert features, use with caution!)"); 4890 out.println(" -fix Try to fix region assignments. This is for backwards compatiblity"); 4891 out.println(" -fixAssignments Try to fix region assignments. Replaces the old -fix"); 4892 out.println(" -fixMeta Try to fix meta problems. This assumes HDFS region info is good."); 4893 out.println(" -fixHdfsHoles Try to fix region holes in hdfs."); 4894 out.println(" -fixHdfsOrphans Try to fix region dirs with no .regioninfo file in hdfs"); 4895 out.println(" -fixTableOrphans Try to fix table dirs with no .tableinfo file in hdfs (online mode only)"); 4896 out.println(" -fixHdfsOverlaps Try to fix region overlaps in hdfs."); 4897 out.println(" -maxMerge <n> When fixing region overlaps, allow at most <n> regions to merge. (n=" + DEFAULT_MAX_MERGE +" by default)"); 4898 out.println(" -sidelineBigOverlaps When fixing region overlaps, allow to sideline big overlaps"); 4899 out.println(" -maxOverlapsToSideline <n> When fixing region overlaps, allow at most <n> regions to sideline per group. (n=" + DEFAULT_OVERLAPS_TO_SIDELINE +" by default)"); 4900 out.println(" -fixSplitParents Try to force offline split parents to be online."); 4901 out.println(" -removeParents Try to offline and sideline lingering parents and keep daughter regions."); 4902 out.println(" -fixEmptyMetaCells Try to fix hbase:meta entries not referencing any region" 4903 + " (empty REGIONINFO_QUALIFIER rows)"); 4904 4905 out.println(""); 4906 out.println(" UNSUPPORTED Metadata Repair shortcuts"); 4907 out.println(" -repair Shortcut for -fixAssignments -fixMeta -fixHdfsHoles " + 4908 "-fixHdfsOrphans -fixHdfsOverlaps -fixVersionFile -sidelineBigOverlaps -fixReferenceFiles" + 4909 "-fixHFileLinks"); 4910 out.println(" -repairHoles Shortcut for -fixAssignments -fixMeta -fixHdfsHoles"); 4911 out.println(""); 4912 out.println(" Replication options"); 4913 out.println(" -fixReplication Deletes replication queues for removed peers"); 4914 out.println(" -cleanReplicationBrarier [tableName] clean the replication barriers " + 4915 "of a specified table, tableName is required"); 4916 out.flush(); 4917 errors.reportError(ERROR_CODE.WRONG_USAGE, sw.toString()); 4918 4919 setRetCode(-2); 4920 return this; 4921 } 4922 4923 /** 4924 * Main program 4925 * 4926 * @param args 4927 * @throws Exception 4928 */ 4929 public static void main(String[] args) throws Exception { 4930 // create a fsck object 4931 Configuration conf = HBaseConfiguration.create(); 4932 Path hbasedir = FSUtils.getRootDir(conf); 4933 URI defaultFs = hbasedir.getFileSystem(conf).getUri(); 4934 FSUtils.setFsDefault(conf, new Path(defaultFs)); 4935 int ret = ToolRunner.run(new HBaseFsckTool(conf), args); 4936 System.exit(ret); 4937 } 4938 4939 /** 4940 * This is a Tool wrapper that gathers -Dxxx=yyy configuration settings from the command line. 4941 */ 4942 static class HBaseFsckTool extends Configured implements Tool { 4943 HBaseFsckTool(Configuration conf) { super(conf); } 4944 @Override 4945 public int run(String[] args) throws Exception { 4946 HBaseFsck hbck = new HBaseFsck(getConf()); 4947 hbck.exec(hbck.executor, args); 4948 hbck.close(); 4949 return hbck.getRetCode(); 4950 } 4951 }; 4952 4953 4954 public HBaseFsck exec(ExecutorService exec, String[] args) 4955 throws KeeperException, IOException, InterruptedException, ReplicationException { 4956 long sleepBeforeRerun = DEFAULT_SLEEP_BEFORE_RERUN; 4957 4958 boolean checkCorruptHFiles = false; 4959 boolean sidelineCorruptHFiles = false; 4960 4961 // Process command-line args. 4962 for (int i = 0; i < args.length; i++) { 4963 String cmd = args[i]; 4964 if (cmd.equals("-help") || cmd.equals("-h")) { 4965 return printUsageAndExit(); 4966 } else if (cmd.equals("-details")) { 4967 setDisplayFullReport(); 4968 } else if (cmd.equals("-exclusive")) { 4969 setForceExclusive(); 4970 } else if (cmd.equals("-timelag")) { 4971 if (i == args.length - 1) { 4972 errors.reportError(ERROR_CODE.WRONG_USAGE, "HBaseFsck: -timelag needs a value."); 4973 return printUsageAndExit(); 4974 } 4975 try { 4976 long timelag = Long.parseLong(args[++i]); 4977 setTimeLag(timelag); 4978 } catch (NumberFormatException e) { 4979 errors.reportError(ERROR_CODE.WRONG_USAGE, "-timelag needs a numeric value."); 4980 return printUsageAndExit(); 4981 } 4982 } else if (cmd.equals("-sleepBeforeRerun")) { 4983 if (i == args.length - 1) { 4984 errors.reportError(ERROR_CODE.WRONG_USAGE, 4985 "HBaseFsck: -sleepBeforeRerun needs a value."); 4986 return printUsageAndExit(); 4987 } 4988 try { 4989 sleepBeforeRerun = Long.parseLong(args[++i]); 4990 } catch (NumberFormatException e) { 4991 errors.reportError(ERROR_CODE.WRONG_USAGE, "-sleepBeforeRerun needs a numeric value."); 4992 return printUsageAndExit(); 4993 } 4994 } else if (cmd.equals("-sidelineDir")) { 4995 if (i == args.length - 1) { 4996 errors.reportError(ERROR_CODE.WRONG_USAGE, "HBaseFsck: -sidelineDir needs a value."); 4997 return printUsageAndExit(); 4998 } 4999 setSidelineDir(args[++i]); 5000 } else if (cmd.equals("-fix")) { 5001 errors.reportError(ERROR_CODE.WRONG_USAGE, 5002 "This option is deprecated, please use -fixAssignments instead."); 5003 setFixAssignments(true); 5004 } else if (cmd.equals("-fixAssignments")) { 5005 setFixAssignments(true); 5006 } else if (cmd.equals("-fixMeta")) { 5007 setFixMeta(true); 5008 } else if (cmd.equals("-noHdfsChecking")) { 5009 setCheckHdfs(false); 5010 } else if (cmd.equals("-fixHdfsHoles")) { 5011 setFixHdfsHoles(true); 5012 } else if (cmd.equals("-fixHdfsOrphans")) { 5013 setFixHdfsOrphans(true); 5014 } else if (cmd.equals("-fixTableOrphans")) { 5015 setFixTableOrphans(true); 5016 } else if (cmd.equals("-fixHdfsOverlaps")) { 5017 setFixHdfsOverlaps(true); 5018 } else if (cmd.equals("-fixVersionFile")) { 5019 setFixVersionFile(true); 5020 } else if (cmd.equals("-sidelineBigOverlaps")) { 5021 setSidelineBigOverlaps(true); 5022 } else if (cmd.equals("-fixSplitParents")) { 5023 setFixSplitParents(true); 5024 } else if (cmd.equals("-removeParents")) { 5025 setRemoveParents(true); 5026 } else if (cmd.equals("-ignorePreCheckPermission")) { 5027 setIgnorePreCheckPermission(true); 5028 } else if (cmd.equals("-checkCorruptHFiles")) { 5029 checkCorruptHFiles = true; 5030 } else if (cmd.equals("-sidelineCorruptHFiles")) { 5031 sidelineCorruptHFiles = true; 5032 } else if (cmd.equals("-fixReferenceFiles")) { 5033 setFixReferenceFiles(true); 5034 } else if (cmd.equals("-fixHFileLinks")) { 5035 setFixHFileLinks(true); 5036 } else if (cmd.equals("-fixEmptyMetaCells")) { 5037 setFixEmptyMetaCells(true); 5038 } else if (cmd.equals("-repair")) { 5039 // this attempts to merge overlapping hdfs regions, needs testing 5040 // under load 5041 setFixHdfsHoles(true); 5042 setFixHdfsOrphans(true); 5043 setFixMeta(true); 5044 setFixAssignments(true); 5045 setFixHdfsOverlaps(true); 5046 setFixVersionFile(true); 5047 setSidelineBigOverlaps(true); 5048 setFixSplitParents(false); 5049 setCheckHdfs(true); 5050 setFixReferenceFiles(true); 5051 setFixHFileLinks(true); 5052 } else if (cmd.equals("-repairHoles")) { 5053 // this will make all missing hdfs regions available but may lose data 5054 setFixHdfsHoles(true); 5055 setFixHdfsOrphans(false); 5056 setFixMeta(true); 5057 setFixAssignments(true); 5058 setFixHdfsOverlaps(false); 5059 setSidelineBigOverlaps(false); 5060 setFixSplitParents(false); 5061 setCheckHdfs(true); 5062 } else if (cmd.equals("-maxOverlapsToSideline")) { 5063 if (i == args.length - 1) { 5064 errors.reportError(ERROR_CODE.WRONG_USAGE, 5065 "-maxOverlapsToSideline needs a numeric value argument."); 5066 return printUsageAndExit(); 5067 } 5068 try { 5069 int maxOverlapsToSideline = Integer.parseInt(args[++i]); 5070 setMaxOverlapsToSideline(maxOverlapsToSideline); 5071 } catch (NumberFormatException e) { 5072 errors.reportError(ERROR_CODE.WRONG_USAGE, 5073 "-maxOverlapsToSideline needs a numeric value argument."); 5074 return printUsageAndExit(); 5075 } 5076 } else if (cmd.equals("-maxMerge")) { 5077 if (i == args.length - 1) { 5078 errors.reportError(ERROR_CODE.WRONG_USAGE, 5079 "-maxMerge needs a numeric value argument."); 5080 return printUsageAndExit(); 5081 } 5082 try { 5083 int maxMerge = Integer.parseInt(args[++i]); 5084 setMaxMerge(maxMerge); 5085 } catch (NumberFormatException e) { 5086 errors.reportError(ERROR_CODE.WRONG_USAGE, 5087 "-maxMerge needs a numeric value argument."); 5088 return printUsageAndExit(); 5089 } 5090 } else if (cmd.equals("-summary")) { 5091 setSummary(); 5092 } else if (cmd.equals("-metaonly")) { 5093 setCheckMetaOnly(); 5094 } else if (cmd.equals("-boundaries")) { 5095 setRegionBoundariesCheck(); 5096 } else if (cmd.equals("-fixReplication")) { 5097 setFixReplication(true); 5098 } else if (cmd.equals("-cleanReplicationBarrier")) { 5099 setCleanReplicationBarrier(true); 5100 if(args[++i].startsWith("-")){ 5101 printUsageAndExit(); 5102 } 5103 setCleanReplicationBarrierTable(args[i]); 5104 } else if (cmd.startsWith("-")) { 5105 errors.reportError(ERROR_CODE.WRONG_USAGE, "Unrecognized option:" + cmd); 5106 return printUsageAndExit(); 5107 } else { 5108 includeTable(TableName.valueOf(cmd)); 5109 errors.print("Allow checking/fixes for table: " + cmd); 5110 } 5111 } 5112 5113 errors.print("HBaseFsck command line options: " + StringUtils.join(args, " ")); 5114 5115 // pre-check current user has FS write permission or not 5116 try { 5117 preCheckPermission(); 5118 } catch (AccessDeniedException ace) { 5119 Runtime.getRuntime().exit(-1); 5120 } catch (IOException ioe) { 5121 Runtime.getRuntime().exit(-1); 5122 } 5123 5124 // do the real work of hbck 5125 connect(); 5126 5127 // after connecting to server above, we have server version 5128 // check if unsupported option is specified based on server version 5129 if (!isOptionsSupported(args)) { 5130 return printUsageAndExit(); 5131 } 5132 5133 try { 5134 // if corrupt file mode is on, first fix them since they may be opened later 5135 if (checkCorruptHFiles || sidelineCorruptHFiles) { 5136 LOG.info("Checking all hfiles for corruption"); 5137 HFileCorruptionChecker hfcc = createHFileCorruptionChecker(sidelineCorruptHFiles); 5138 setHFileCorruptionChecker(hfcc); // so we can get result 5139 Collection<TableName> tables = getIncludedTables(); 5140 Collection<Path> tableDirs = new ArrayList<>(); 5141 Path rootdir = FSUtils.getRootDir(getConf()); 5142 if (tables.size() > 0) { 5143 for (TableName t : tables) { 5144 tableDirs.add(FSUtils.getTableDir(rootdir, t)); 5145 } 5146 } else { 5147 tableDirs = FSUtils.getTableDirs(FSUtils.getCurrentFileSystem(getConf()), rootdir); 5148 } 5149 hfcc.checkTables(tableDirs); 5150 hfcc.report(errors); 5151 } 5152 5153 // check and fix table integrity, region consistency. 5154 int code = onlineHbck(); 5155 setRetCode(code); 5156 // If we have changed the HBase state it is better to run hbck again 5157 // to see if we haven't broken something else in the process. 5158 // We run it only once more because otherwise we can easily fall into 5159 // an infinite loop. 5160 if (shouldRerun()) { 5161 try { 5162 LOG.info("Sleeping " + sleepBeforeRerun + "ms before re-checking after fix..."); 5163 Thread.sleep(sleepBeforeRerun); 5164 } catch (InterruptedException ie) { 5165 LOG.warn("Interrupted while sleeping"); 5166 return this; 5167 } 5168 // Just report 5169 setFixAssignments(false); 5170 setFixMeta(false); 5171 setFixHdfsHoles(false); 5172 setFixHdfsOverlaps(false); 5173 setFixVersionFile(false); 5174 setFixTableOrphans(false); 5175 errors.resetErrors(); 5176 code = onlineHbck(); 5177 setRetCode(code); 5178 } 5179 } finally { 5180 IOUtils.closeQuietly(this); 5181 } 5182 return this; 5183 } 5184 5185 private boolean isOptionsSupported(String[] args) { 5186 boolean result = true; 5187 String hbaseServerVersion = status.getHBaseVersion(); 5188 if (VersionInfo.compareVersion("2.any.any", hbaseServerVersion) < 0) { 5189 // Process command-line args. 5190 for (String arg : args) { 5191 if (unsupportedOptionsInV2.contains(arg)) { 5192 errors.reportError(ERROR_CODE.UNSUPPORTED_OPTION, 5193 "option '" + arg + "' is not " + "supportted!"); 5194 result = false; 5195 break; 5196 } 5197 } 5198 } 5199 return result; 5200 } 5201 5202 public void setCleanReplicationBarrierTable(String cleanReplicationBarrierTable) { 5203 this.cleanReplicationBarrierTable = TableName.valueOf(cleanReplicationBarrierTable); 5204 } 5205 5206 public void cleanReplicationBarrier() throws IOException { 5207 if (!cleanReplicationBarrier || cleanReplicationBarrierTable == null) { 5208 return; 5209 } 5210 if (cleanReplicationBarrierTable.isSystemTable()) { 5211 errors.reportError(ERROR_CODE.INVALID_TABLE, 5212 "invalid table: " + cleanReplicationBarrierTable); 5213 return; 5214 } 5215 5216 boolean isGlobalScope = false; 5217 try { 5218 isGlobalScope = admin.getDescriptor(cleanReplicationBarrierTable).hasGlobalReplicationScope(); 5219 } catch (TableNotFoundException e) { 5220 LOG.info("we may need to clean some erroneous data due to bugs"); 5221 } 5222 5223 if (isGlobalScope) { 5224 errors.reportError(ERROR_CODE.INVALID_TABLE, 5225 "table's replication scope is global: " + cleanReplicationBarrierTable); 5226 return; 5227 } 5228 List<byte[]> regionNames = new ArrayList<>(); 5229 Scan barrierScan = new Scan(); 5230 barrierScan.setCaching(100); 5231 barrierScan.addFamily(HConstants.REPLICATION_BARRIER_FAMILY); 5232 barrierScan 5233 .withStartRow(MetaTableAccessor.getTableStartRowForMeta(cleanReplicationBarrierTable, 5234 MetaTableAccessor.QueryType.REGION)) 5235 .withStopRow(MetaTableAccessor.getTableStopRowForMeta(cleanReplicationBarrierTable, 5236 MetaTableAccessor.QueryType.REGION)); 5237 Result result; 5238 try (ResultScanner scanner = meta.getScanner(barrierScan)) { 5239 while ((result = scanner.next()) != null) { 5240 regionNames.add(result.getRow()); 5241 } 5242 } 5243 if (regionNames.size() <= 0) { 5244 errors.reportError(ERROR_CODE.INVALID_TABLE, 5245 "there is no barriers of this table: " + cleanReplicationBarrierTable); 5246 return; 5247 } 5248 ReplicationQueueStorage queueStorage = 5249 ReplicationStorageFactory.getReplicationQueueStorage(zkw, getConf()); 5250 List<ReplicationPeerDescription> peerDescriptions = admin.listReplicationPeers(); 5251 if (peerDescriptions != null && peerDescriptions.size() > 0) { 5252 List<String> peers = peerDescriptions.stream() 5253 .filter(peerConfig -> ReplicationUtils.contains(peerConfig.getPeerConfig(), 5254 cleanReplicationBarrierTable)) 5255 .map(peerConfig -> peerConfig.getPeerId()).collect(Collectors.toList()); 5256 try { 5257 List<String> batch = new ArrayList<>(); 5258 for (String peer : peers) { 5259 for (byte[] regionName : regionNames) { 5260 batch.add(RegionInfo.encodeRegionName(regionName)); 5261 if (batch.size() % 100 == 0) { 5262 queueStorage.removeLastSequenceIds(peer, batch); 5263 batch.clear(); 5264 } 5265 } 5266 if (batch.size() > 0) { 5267 queueStorage.removeLastSequenceIds(peer, batch); 5268 batch.clear(); 5269 } 5270 } 5271 } catch (ReplicationException re) { 5272 throw new IOException(re); 5273 } 5274 } 5275 for (byte[] regionName : regionNames) { 5276 meta.delete(new Delete(regionName).addFamily(HConstants.REPLICATION_BARRIER_FAMILY)); 5277 } 5278 setShouldRerun(); 5279 } 5280 5281 /** 5282 * ls -r for debugging purposes 5283 */ 5284 void debugLsr(Path p) throws IOException { 5285 debugLsr(getConf(), p, errors); 5286 } 5287 5288 /** 5289 * ls -r for debugging purposes 5290 */ 5291 public static void debugLsr(Configuration conf, 5292 Path p) throws IOException { 5293 debugLsr(conf, p, new PrintingErrorReporter()); 5294 } 5295 5296 /** 5297 * ls -r for debugging purposes 5298 */ 5299 public static void debugLsr(Configuration conf, 5300 Path p, ErrorReporter errors) throws IOException { 5301 if (!LOG.isDebugEnabled() || p == null) { 5302 return; 5303 } 5304 FileSystem fs = p.getFileSystem(conf); 5305 5306 if (!fs.exists(p)) { 5307 // nothing 5308 return; 5309 } 5310 errors.print(p.toString()); 5311 5312 if (fs.isFile(p)) { 5313 return; 5314 } 5315 5316 if (fs.getFileStatus(p).isDirectory()) { 5317 FileStatus[] fss= fs.listStatus(p); 5318 for (FileStatus status : fss) { 5319 debugLsr(conf, status.getPath(), errors); 5320 } 5321 } 5322 } 5323}