1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.util;
19
20 import java.io.Closeable;
21 import java.io.FileNotFoundException;
22 import java.io.IOException;
23 import java.io.InterruptedIOException;
24 import java.io.PrintWriter;
25 import java.io.StringWriter;
26 import java.net.InetAddress;
27 import java.net.URI;
28 import java.util.ArrayList;
29 import java.util.Arrays;
30 import java.util.Collection;
31 import java.util.Collections;
32 import java.util.Comparator;
33 import java.util.HashMap;
34 import java.util.HashSet;
35 import java.util.Iterator;
36 import java.util.List;
37 import java.util.Map;
38 import java.util.Map.Entry;
39 import java.util.Set;
40 import java.util.SortedMap;
41 import java.util.SortedSet;
42 import java.util.TreeMap;
43 import java.util.TreeSet;
44 import java.util.concurrent.Callable;
45 import java.util.concurrent.ConcurrentSkipListMap;
46 import java.util.concurrent.ExecutionException;
47 import java.util.concurrent.ExecutorService;
48 import java.util.concurrent.Executors;
49 import java.util.concurrent.Future;
50 import java.util.concurrent.FutureTask;
51 import java.util.concurrent.ScheduledThreadPoolExecutor;
52 import java.util.concurrent.TimeUnit;
53 import java.util.concurrent.TimeoutException;
54 import java.util.concurrent.atomic.AtomicBoolean;
55 import java.util.concurrent.atomic.AtomicInteger;
56
57 import org.apache.commons.io.IOUtils;
58 import org.apache.commons.lang.StringUtils;
59 import org.apache.commons.logging.Log;
60 import org.apache.commons.logging.LogFactory;
61 import org.apache.hadoop.conf.Configuration;
62 import org.apache.hadoop.conf.Configured;
63 import org.apache.hadoop.fs.FSDataOutputStream;
64 import org.apache.hadoop.fs.FileStatus;
65 import org.apache.hadoop.fs.FileSystem;
66 import org.apache.hadoop.fs.Path;
67 import org.apache.hadoop.fs.permission.FsAction;
68 import org.apache.hadoop.fs.permission.FsPermission;
69 import org.apache.hadoop.hbase.Abortable;
70 import org.apache.hadoop.hbase.Cell;
71 import org.apache.hadoop.hbase.ClusterStatus;
72 import org.apache.hadoop.hbase.CoordinatedStateException;
73 import org.apache.hadoop.hbase.HBaseConfiguration;
74 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
75 import org.apache.hadoop.hbase.HColumnDescriptor;
76 import org.apache.hadoop.hbase.HConstants;
77 import org.apache.hadoop.hbase.HRegionInfo;
78 import org.apache.hadoop.hbase.HRegionLocation;
79 import org.apache.hadoop.hbase.HTableDescriptor;
80 import org.apache.hadoop.hbase.KeyValue;
81 import org.apache.hadoop.hbase.MasterNotRunningException;
82 import org.apache.hadoop.hbase.MetaTableAccessor;
83 import org.apache.hadoop.hbase.RegionLocations;
84 import org.apache.hadoop.hbase.ServerName;
85 import org.apache.hadoop.hbase.TableName;
86 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
87 import org.apache.hadoop.hbase.classification.InterfaceAudience;
88 import org.apache.hadoop.hbase.classification.InterfaceStability;
89 import org.apache.hadoop.hbase.client.Admin;
90 import org.apache.hadoop.hbase.client.ClusterConnection;
91 import org.apache.hadoop.hbase.client.ConnectionFactory;
92 import org.apache.hadoop.hbase.client.Delete;
93 import org.apache.hadoop.hbase.client.Get;
94 import org.apache.hadoop.hbase.client.HBaseAdmin;
95 import org.apache.hadoop.hbase.client.HConnectable;
96 import org.apache.hadoop.hbase.client.HConnection;
97 import org.apache.hadoop.hbase.client.HConnectionManager;
98 import org.apache.hadoop.hbase.client.MetaScanner;
99 import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
100 import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
101 import org.apache.hadoop.hbase.client.Put;
102 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
103 import org.apache.hadoop.hbase.client.Result;
104 import org.apache.hadoop.hbase.client.RowMutations;
105 import org.apache.hadoop.hbase.client.Table;
106 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
107 import org.apache.hadoop.hbase.io.hfile.HFile;
108 import org.apache.hadoop.hbase.master.MasterFileSystem;
109 import org.apache.hadoop.hbase.master.RegionState;
110 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
111 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
112 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
113 import org.apache.hadoop.hbase.regionserver.HRegion;
114 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
115 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
116 import org.apache.hadoop.hbase.security.AccessDeniedException;
117 import org.apache.hadoop.hbase.security.UserProvider;
118 import org.apache.hadoop.hbase.util.Bytes.ByteArrayComparator;
119 import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
120 import org.apache.hadoop.hbase.util.hbck.HFileCorruptionChecker;
121 import org.apache.hadoop.hbase.util.hbck.TableIntegrityErrorHandler;
122 import org.apache.hadoop.hbase.util.hbck.TableIntegrityErrorHandlerImpl;
123 import org.apache.hadoop.hbase.util.hbck.TableLockChecker;
124 import org.apache.hadoop.hbase.wal.WALSplitter;
125 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
126 import org.apache.hadoop.hbase.zookeeper.ZKTableStateClientSideReader;
127 import org.apache.hadoop.hbase.zookeeper.ZKTableStateManager;
128 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
129 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
130 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
131 import org.apache.hadoop.ipc.RemoteException;
132 import org.apache.hadoop.security.UserGroupInformation;
133 import org.apache.hadoop.util.ReflectionUtils;
134 import org.apache.hadoop.util.Tool;
135 import org.apache.hadoop.util.ToolRunner;
136 import org.apache.zookeeper.KeeperException;
137
138 import com.google.common.base.Joiner;
139 import com.google.common.base.Preconditions;
140 import com.google.common.collect.ImmutableList;
141 import com.google.common.collect.Lists;
142 import com.google.common.collect.Multimap;
143 import com.google.common.collect.Ordering;
144 import com.google.common.collect.TreeMultimap;
145 import com.google.protobuf.ServiceException;
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
193 @InterfaceStability.Evolving
194 public class HBaseFsck extends Configured implements Closeable {
195 public static final long DEFAULT_TIME_LAG = 60000;
196 public static final long DEFAULT_SLEEP_BEFORE_RERUN = 10000;
197 private static final int MAX_NUM_THREADS = 50;
198 private static boolean rsSupportsOffline = true;
199 private static final int DEFAULT_OVERLAPS_TO_SIDELINE = 2;
200 private static final int DEFAULT_MAX_MERGE = 5;
201 private static final String TO_BE_LOADED = "to_be_loaded";
202 private static final String HBCK_LOCK_FILE = "hbase-hbck.lock";
203 private static final int DEFAULT_MAX_LOCK_FILE_ATTEMPTS = 5;
204 private static final int DEFAULT_LOCK_FILE_ATTEMPT_SLEEP_INTERVAL = 200;
205 private static final int DEFAULT_LOCK_FILE_ATTEMPT_MAX_SLEEP_TIME = 5000;
206
207
208
209
210 private static final int DEFAULT_WAIT_FOR_LOCK_TIMEOUT = 80;
211
212
213
214
215 private static final Log LOG = LogFactory.getLog(HBaseFsck.class.getName());
216 private ClusterStatus status;
217 private ClusterConnection connection;
218 private Admin admin;
219 private Table meta;
220
221 protected ExecutorService executor;
222 private long startMillis = EnvironmentEdgeManager.currentTime();
223 private HFileCorruptionChecker hfcc;
224 private int retcode = 0;
225 private Path HBCK_LOCK_PATH;
226 private FSDataOutputStream hbckOutFd;
227
228
229
230 private final AtomicBoolean hbckLockCleanup = new AtomicBoolean(false);
231
232
233
234
235 private static boolean details = false;
236 private long timelag = DEFAULT_TIME_LAG;
237 private static boolean forceExclusive = false;
238 private static boolean disableBalancer = false;
239 private boolean fixAssignments = false;
240 private boolean fixMeta = false;
241 private boolean checkHdfs = true;
242 private boolean fixHdfsHoles = false;
243 private boolean fixHdfsOverlaps = false;
244 private boolean fixHdfsOrphans = false;
245 private boolean fixTableOrphans = false;
246 private boolean fixVersionFile = false;
247 private boolean fixSplitParents = false;
248 private boolean removeParents = false;
249 private boolean fixReferenceFiles = false;
250 private boolean fixEmptyMetaCells = false;
251 private boolean fixTableLocks = false;
252 private boolean fixTableZNodes = false;
253 private boolean fixAny = false;
254
255
256
257 private Set<TableName> tablesIncluded = new HashSet<TableName>();
258 private int maxMerge = DEFAULT_MAX_MERGE;
259 private int maxOverlapsToSideline = DEFAULT_OVERLAPS_TO_SIDELINE;
260 private boolean sidelineBigOverlaps = false;
261 private Path sidelineDir = null;
262
263 private boolean rerun = false;
264 private static boolean SUMMARY = false;
265 private boolean checkMetaOnly = false;
266 private boolean checkRegionBoundaries = false;
267 private boolean ignorePreCheckPermission = false;
268
269
270
271
272 final private ErrorReporter errors;
273 int fixes = 0;
274
275
276
277
278
279
280 private TreeMap<String, HbckInfo> regionInfoMap = new TreeMap<String, HbckInfo>();
281 private TreeSet<TableName> disabledTables =
282 new TreeSet<TableName>();
283
284 private Set<Result> emptyRegionInfoQualifiers = new HashSet<Result>();
285
286
287
288
289
290
291
292
293
294
295
296 private SortedMap<TableName, TableInfo> tablesInfo =
297 new ConcurrentSkipListMap<TableName, TableInfo>();
298
299
300
301
302 private List<HbckInfo> orphanHdfsDirs = Collections.synchronizedList(new ArrayList<HbckInfo>());
303
304 private Map<TableName, Set<String>> orphanTableDirs =
305 new HashMap<TableName, Set<String>>();
306
307 private Map<TableName, Set<String>> skippedRegions = new HashMap<TableName, Set<String>>();
308
309
310
311
312 private Set<TableName> orphanedTableZNodes = new HashSet<TableName>();
313 private final RetryCounterFactory lockFileRetryCounterFactory;
314
315
316
317
318
319
320
321
322
323 public HBaseFsck(Configuration conf) throws MasterNotRunningException,
324 ZooKeeperConnectionException, IOException, ClassNotFoundException {
325 this(conf, createThreadPool(conf));
326 }
327
328 private static ExecutorService createThreadPool(Configuration conf) {
329 int numThreads = conf.getInt("hbasefsck.numthreads", MAX_NUM_THREADS);
330 return new ScheduledThreadPoolExecutor(numThreads, Threads.newDaemonThreadFactory("hbasefsck"));
331 }
332
333
334
335
336
337
338
339
340
341
342
343 public HBaseFsck(Configuration conf, ExecutorService exec) throws MasterNotRunningException,
344 ZooKeeperConnectionException, IOException, ClassNotFoundException {
345 super(conf);
346 errors = getErrorReporter(getConf());
347 this.executor = exec;
348 lockFileRetryCounterFactory = new RetryCounterFactory(
349 getConf().getInt("hbase.hbck.lockfile.attempts", DEFAULT_MAX_LOCK_FILE_ATTEMPTS),
350 getConf().getInt(
351 "hbase.hbck.lockfile.attempt.sleep.interval", DEFAULT_LOCK_FILE_ATTEMPT_SLEEP_INTERVAL),
352 getConf().getInt(
353 "hbase.hbck.lockfile.attempt.maxsleeptime", DEFAULT_LOCK_FILE_ATTEMPT_MAX_SLEEP_TIME));
354 }
355
356 private class FileLockCallable implements Callable<FSDataOutputStream> {
357 RetryCounter retryCounter;
358
359 public FileLockCallable(RetryCounter retryCounter) {
360 this.retryCounter = retryCounter;
361 }
362 @Override
363 public FSDataOutputStream call() throws IOException {
364 try {
365 FileSystem fs = FSUtils.getCurrentFileSystem(getConf());
366 FsPermission defaultPerms = FSUtils.getFilePermissions(fs, getConf(),
367 HConstants.DATA_FILE_UMASK_KEY);
368 Path tmpDir = new Path(FSUtils.getRootDir(getConf()), HConstants.HBASE_TEMP_DIRECTORY);
369 fs.mkdirs(tmpDir);
370 HBCK_LOCK_PATH = new Path(tmpDir, HBCK_LOCK_FILE);
371 final FSDataOutputStream out = createFileWithRetries(fs, HBCK_LOCK_PATH, defaultPerms);
372 out.writeBytes(InetAddress.getLocalHost().toString());
373 out.flush();
374 return out;
375 } catch(RemoteException e) {
376 if(AlreadyBeingCreatedException.class.getName().equals(e.getClassName())){
377 return null;
378 } else {
379 throw e;
380 }
381 }
382 }
383
384 private FSDataOutputStream createFileWithRetries(final FileSystem fs,
385 final Path hbckLockFilePath, final FsPermission defaultPerms)
386 throws IOException {
387
388 IOException exception = null;
389 do {
390 try {
391 return FSUtils.create(fs, hbckLockFilePath, defaultPerms, false);
392 } catch (IOException ioe) {
393 LOG.info("Failed to create lock file " + hbckLockFilePath.getName()
394 + ", try=" + (retryCounter.getAttemptTimes() + 1) + " of "
395 + retryCounter.getMaxAttempts());
396 LOG.debug("Failed to create lock file " + hbckLockFilePath.getName(),
397 ioe);
398 try {
399 exception = ioe;
400 retryCounter.sleepUntilNextRetry();
401 } catch (InterruptedException ie) {
402 throw (InterruptedIOException) new InterruptedIOException(
403 "Can't create lock file " + hbckLockFilePath.getName())
404 .initCause(ie);
405 }
406 }
407 } while (retryCounter.shouldRetry());
408
409 throw exception;
410 }
411 }
412
413
414
415
416
417
418
419 private FSDataOutputStream checkAndMarkRunningHbck() throws IOException {
420 RetryCounter retryCounter = lockFileRetryCounterFactory.create();
421 FileLockCallable callable = new FileLockCallable(retryCounter);
422 ExecutorService executor = Executors.newFixedThreadPool(1);
423 FutureTask<FSDataOutputStream> futureTask = new FutureTask<FSDataOutputStream>(callable);
424 executor.execute(futureTask);
425 final int timeoutInSeconds = getConf().getInt(
426 "hbase.hbck.lockfile.maxwaittime", DEFAULT_WAIT_FOR_LOCK_TIMEOUT);
427 FSDataOutputStream stream = null;
428 try {
429 stream = futureTask.get(timeoutInSeconds, TimeUnit.SECONDS);
430 } catch (ExecutionException ee) {
431 LOG.warn("Encountered exception when opening lock file", ee);
432 } catch (InterruptedException ie) {
433 LOG.warn("Interrupted when opening lock file", ie);
434 Thread.currentThread().interrupt();
435 } catch (TimeoutException exception) {
436
437 LOG.warn("Took more than " + timeoutInSeconds + " seconds in obtaining lock");
438 futureTask.cancel(true);
439 } finally {
440 executor.shutdownNow();
441 }
442 return stream;
443 }
444
445 private void unlockHbck() {
446 if (isExclusive() && hbckLockCleanup.compareAndSet(true, false)) {
447 RetryCounter retryCounter = lockFileRetryCounterFactory.create();
448 do {
449 try {
450 IOUtils.closeQuietly(hbckOutFd);
451 FSUtils.delete(FSUtils.getCurrentFileSystem(getConf()),
452 HBCK_LOCK_PATH, true);
453 LOG.info("Finishing hbck");
454 return;
455 } catch (IOException ioe) {
456 LOG.info("Failed to delete " + HBCK_LOCK_PATH + ", try="
457 + (retryCounter.getAttemptTimes() + 1) + " of "
458 + retryCounter.getMaxAttempts());
459 LOG.debug("Failed to delete " + HBCK_LOCK_PATH, ioe);
460 try {
461 retryCounter.sleepUntilNextRetry();
462 } catch (InterruptedException ie) {
463 Thread.currentThread().interrupt();
464 LOG.warn("Interrupted while deleting lock file" +
465 HBCK_LOCK_PATH);
466 return;
467 }
468 }
469 } while (retryCounter.shouldRetry());
470 }
471 }
472
473
474
475
476
477 public void connect() throws IOException {
478
479 if (isExclusive()) {
480
481 hbckOutFd = checkAndMarkRunningHbck();
482 if (hbckOutFd == null) {
483 setRetCode(-1);
484 LOG.error("Another instance of hbck is fixing HBase, exiting this instance. " +
485 "[If you are sure no other instance is running, delete the lock file " +
486 HBCK_LOCK_PATH + " and rerun the tool]");
487 throw new IOException("Duplicate hbck - Abort");
488 }
489
490
491 hbckLockCleanup.set(true);
492 }
493
494
495
496
497
498 Runtime.getRuntime().addShutdownHook(new Thread() {
499 @Override
500 public void run() {
501 IOUtils.closeQuietly(HBaseFsck.this);
502 unlockHbck();
503 }
504 });
505
506 LOG.info("Launching hbck");
507
508 connection = (ClusterConnection)ConnectionFactory.createConnection(getConf());
509 admin = connection.getAdmin();
510 meta = connection.getTable(TableName.META_TABLE_NAME);
511 status = admin.getClusterStatus();
512 }
513
514
515
516
517 private void loadDeployedRegions() throws IOException, InterruptedException {
518
519 Collection<ServerName> regionServers = status.getServers();
520 errors.print("Number of live region servers: " + regionServers.size());
521 if (details) {
522 for (ServerName rsinfo: regionServers) {
523 errors.print(" " + rsinfo.getServerName());
524 }
525 }
526
527
528 Collection<ServerName> deadRegionServers = status.getDeadServerNames();
529 errors.print("Number of dead region servers: " + deadRegionServers.size());
530 if (details) {
531 for (ServerName name: deadRegionServers) {
532 errors.print(" " + name);
533 }
534 }
535
536
537 errors.print("Master: " + status.getMaster());
538
539
540 Collection<ServerName> backupMasters = status.getBackupMasters();
541 errors.print("Number of backup masters: " + backupMasters.size());
542 if (details) {
543 for (ServerName name: backupMasters) {
544 errors.print(" " + name);
545 }
546 }
547
548 errors.print("Average load: " + status.getAverageLoad());
549 errors.print("Number of requests: " + status.getRequestsCount());
550 errors.print("Number of regions: " + status.getRegionsCount());
551
552 Map<String, RegionState> rits = status.getRegionsInTransition();
553 errors.print("Number of regions in transition: " + rits.size());
554 if (details) {
555 for (RegionState state: rits.values()) {
556 errors.print(" " + state.toDescriptiveString());
557 }
558 }
559
560
561 processRegionServers(regionServers);
562 }
563
564
565
566
567 private void clearState() {
568
569 fixes = 0;
570 regionInfoMap.clear();
571 emptyRegionInfoQualifiers.clear();
572 disabledTables.clear();
573 errors.clear();
574 tablesInfo.clear();
575 orphanHdfsDirs.clear();
576 skippedRegions.clear();
577 }
578
579
580
581
582
583
584 public void offlineHdfsIntegrityRepair() throws IOException, InterruptedException {
585
586 if (shouldCheckHdfs() && (shouldFixHdfsOrphans() || shouldFixHdfsHoles()
587 || shouldFixHdfsOverlaps() || shouldFixTableOrphans())) {
588 LOG.info("Loading regioninfos HDFS");
589
590 int maxIterations = getConf().getInt("hbase.hbck.integrityrepair.iterations.max", 3);
591 int curIter = 0;
592 do {
593 clearState();
594
595 restoreHdfsIntegrity();
596 curIter++;
597 } while (fixes > 0 && curIter <= maxIterations);
598
599
600
601 if (curIter > 2) {
602 if (curIter == maxIterations) {
603 LOG.warn("Exiting integrity repairs after max " + curIter + " iterations. "
604 + "Tables integrity may not be fully repaired!");
605 } else {
606 LOG.info("Successfully exiting integrity repairs after " + curIter + " iterations");
607 }
608 }
609 }
610 }
611
612
613
614
615
616
617
618
619
620 public int onlineConsistencyRepair() throws IOException, KeeperException,
621 InterruptedException {
622 clearState();
623
624
625 loadDeployedRegions();
626
627 recordMetaRegion();
628
629 if (!checkMetaRegion()) {
630 String errorMsg = "hbase:meta table is not consistent. ";
631 if (shouldFixAssignments()) {
632 errorMsg += "HBCK will try fixing it. Rerun once hbase:meta is back to consistent state.";
633 } else {
634 errorMsg += "Run HBCK with proper fix options to fix hbase:meta inconsistency.";
635 }
636 errors.reportError(errorMsg + " Exiting...");
637 return -2;
638 }
639
640 LOG.info("Loading regionsinfo from the hbase:meta table");
641 boolean success = loadMetaEntries();
642 if (!success) return -1;
643
644
645 reportEmptyMetaCells();
646
647
648 if (shouldFixEmptyMetaCells()) {
649 fixEmptyMetaCells();
650 }
651
652
653 if (!checkMetaOnly) {
654 reportTablesInFlux();
655 }
656
657
658 if (shouldCheckHdfs()) {
659 LOG.info("Loading region directories from HDFS");
660 loadHdfsRegionDirs();
661 LOG.info("Loading region information from HDFS");
662 loadHdfsRegionInfos();
663 }
664
665
666 loadDisabledTables();
667
668
669 fixOrphanTables();
670
671 LOG.info("Checking and fixing region consistency");
672
673 checkAndFixConsistency();
674
675
676 checkIntegrity();
677 return errors.getErrorList().size();
678 }
679
680
681
682
683
684 public int onlineHbck() throws IOException, KeeperException, InterruptedException, ServiceException {
685
686 errors.print("Version: " + status.getHBaseVersion());
687 offlineHdfsIntegrityRepair();
688
689 boolean oldBalancer = false;
690 if (shouldDisableBalancer()) {
691 oldBalancer = admin.setBalancerRunning(false, true);
692 }
693
694 try {
695 onlineConsistencyRepair();
696 }
697 finally {
698
699
700
701 if (shouldDisableBalancer() && oldBalancer) {
702 admin.setBalancerRunning(oldBalancer, false);
703 }
704 }
705
706 if (checkRegionBoundaries) {
707 checkRegionBoundaries();
708 }
709
710 offlineReferenceFileRepair();
711
712 checkAndFixTableLocks();
713
714
715 checkAndFixOrphanedTableZNodes();
716
717
718 unlockHbck();
719
720
721 printTableSummary(tablesInfo);
722 return errors.summarize();
723 }
724
725 public static byte[] keyOnly (byte[] b) {
726 if (b == null)
727 return b;
728 int rowlength = Bytes.toShort(b, 0);
729 byte[] result = new byte[rowlength];
730 System.arraycopy(b, Bytes.SIZEOF_SHORT, result, 0, rowlength);
731 return result;
732 }
733
734 @Override
735 public void close() throws IOException {
736 IOUtils.closeQuietly(admin);
737 IOUtils.closeQuietly(meta);
738 IOUtils.closeQuietly(connection);
739 }
740
741 private static class RegionBoundariesInformation {
742 public byte [] regionName;
743 public byte [] metaFirstKey;
744 public byte [] metaLastKey;
745 public byte [] storesFirstKey;
746 public byte [] storesLastKey;
747 @Override
748 public String toString () {
749 return "regionName=" + Bytes.toStringBinary(regionName) +
750 "\nmetaFirstKey=" + Bytes.toStringBinary(metaFirstKey) +
751 "\nmetaLastKey=" + Bytes.toStringBinary(metaLastKey) +
752 "\nstoresFirstKey=" + Bytes.toStringBinary(storesFirstKey) +
753 "\nstoresLastKey=" + Bytes.toStringBinary(storesLastKey);
754 }
755 }
756
757 public void checkRegionBoundaries() {
758 try {
759 ByteArrayComparator comparator = new ByteArrayComparator();
760 List<HRegionInfo> regions = MetaScanner.listAllRegions(getConf(), connection, false);
761 final RegionBoundariesInformation currentRegionBoundariesInformation =
762 new RegionBoundariesInformation();
763 Path hbaseRoot = FSUtils.getRootDir(getConf());
764 for (HRegionInfo regionInfo : regions) {
765 Path tableDir = FSUtils.getTableDir(hbaseRoot, regionInfo.getTable());
766 currentRegionBoundariesInformation.regionName = regionInfo.getRegionName();
767
768
769 Path path = new Path(tableDir, regionInfo.getEncodedName());
770 FileSystem fs = path.getFileSystem(getConf());
771 FileStatus[] files = fs.listStatus(path);
772
773 byte[] storeFirstKey = null;
774 byte[] storeLastKey = null;
775 for (FileStatus file : files) {
776 String fileName = file.getPath().toString();
777 fileName = fileName.substring(fileName.lastIndexOf("/") + 1);
778 if (!fileName.startsWith(".") && !fileName.endsWith("recovered.edits")) {
779 FileStatus[] storeFiles = fs.listStatus(file.getPath());
780
781 for (FileStatus storeFile : storeFiles) {
782 HFile.Reader reader = HFile.createReader(fs, storeFile.getPath(), new CacheConfig(
783 getConf()), getConf());
784 if ((reader.getFirstKey() != null)
785 && ((storeFirstKey == null) || (comparator.compare(storeFirstKey,
786 reader.getFirstKey()) > 0))) {
787 storeFirstKey = reader.getFirstKey();
788 }
789 if ((reader.getLastKey() != null)
790 && ((storeLastKey == null) || (comparator.compare(storeLastKey,
791 reader.getLastKey())) < 0)) {
792 storeLastKey = reader.getLastKey();
793 }
794 reader.close();
795 }
796 }
797 }
798 currentRegionBoundariesInformation.metaFirstKey = regionInfo.getStartKey();
799 currentRegionBoundariesInformation.metaLastKey = regionInfo.getEndKey();
800 currentRegionBoundariesInformation.storesFirstKey = keyOnly(storeFirstKey);
801 currentRegionBoundariesInformation.storesLastKey = keyOnly(storeLastKey);
802 if (currentRegionBoundariesInformation.metaFirstKey.length == 0)
803 currentRegionBoundariesInformation.metaFirstKey = null;
804 if (currentRegionBoundariesInformation.metaLastKey.length == 0)
805 currentRegionBoundariesInformation.metaLastKey = null;
806
807
808
809
810
811
812 boolean valid = true;
813
814 if ((currentRegionBoundariesInformation.storesFirstKey != null)
815 && (currentRegionBoundariesInformation.metaFirstKey != null)) {
816 valid = valid
817 && comparator.compare(currentRegionBoundariesInformation.storesFirstKey,
818 currentRegionBoundariesInformation.metaFirstKey) >= 0;
819 }
820
821 if ((currentRegionBoundariesInformation.storesLastKey != null)
822 && (currentRegionBoundariesInformation.metaLastKey != null)) {
823 valid = valid
824 && comparator.compare(currentRegionBoundariesInformation.storesLastKey,
825 currentRegionBoundariesInformation.metaLastKey) < 0;
826 }
827 if (!valid) {
828 errors.reportError(ERROR_CODE.BOUNDARIES_ERROR, "Found issues with regions boundaries",
829 tablesInfo.get(regionInfo.getTable()));
830 LOG.warn("Region's boundaries not alligned between stores and META for:");
831 LOG.warn(currentRegionBoundariesInformation);
832 }
833 }
834 } catch (IOException e) {
835 LOG.error(e);
836 }
837 }
838
839
840
841
842 private void adoptHdfsOrphans(Collection<HbckInfo> orphanHdfsDirs) throws IOException {
843 for (HbckInfo hi : orphanHdfsDirs) {
844 LOG.info("Attempting to handle orphan hdfs dir: " + hi.getHdfsRegionDir());
845 adoptHdfsOrphan(hi);
846 }
847 }
848
849
850
851
852
853
854
855
856
857
858 @SuppressWarnings("deprecation")
859 private void adoptHdfsOrphan(HbckInfo hi) throws IOException {
860 Path p = hi.getHdfsRegionDir();
861 FileSystem fs = p.getFileSystem(getConf());
862 FileStatus[] dirs = fs.listStatus(p);
863 if (dirs == null) {
864 LOG.warn("Attempt to adopt ophan hdfs region skipped becuase no files present in " +
865 p + ". This dir could probably be deleted.");
866 return ;
867 }
868
869 TableName tableName = hi.getTableName();
870 TableInfo tableInfo = tablesInfo.get(tableName);
871 Preconditions.checkNotNull(tableInfo, "Table '" + tableName + "' not present!");
872 HTableDescriptor template = tableInfo.getHTD();
873
874
875 Pair<byte[],byte[]> orphanRegionRange = null;
876 for (FileStatus cf : dirs) {
877 String cfName= cf.getPath().getName();
878
879 if (cfName.startsWith(".") || cfName.equals(HConstants.SPLIT_LOGDIR_NAME)) continue;
880
881 FileStatus[] hfiles = fs.listStatus(cf.getPath());
882 for (FileStatus hfile : hfiles) {
883 byte[] start, end;
884 HFile.Reader hf = null;
885 try {
886 CacheConfig cacheConf = new CacheConfig(getConf());
887 hf = HFile.createReader(fs, hfile.getPath(), cacheConf, getConf());
888 hf.loadFileInfo();
889 KeyValue startKv = KeyValue.createKeyValueFromKey(hf.getFirstKey());
890 start = startKv.getRow();
891 KeyValue endKv = KeyValue.createKeyValueFromKey(hf.getLastKey());
892 end = endKv.getRow();
893 } catch (IOException ioe) {
894 LOG.warn("Problem reading orphan file " + hfile + ", skipping");
895 continue;
896 } catch (NullPointerException ioe) {
897 LOG.warn("Orphan file " + hfile + " is possibly corrupted HFile, skipping");
898 continue;
899 } finally {
900 if (hf != null) {
901 hf.close();
902 }
903 }
904
905
906 if (orphanRegionRange == null) {
907
908 orphanRegionRange = new Pair<byte[], byte[]>(start, end);
909 } else {
910
911
912
913 if (Bytes.compareTo(orphanRegionRange.getFirst(), start) > 0) {
914 orphanRegionRange.setFirst(start);
915 }
916 if (Bytes.compareTo(orphanRegionRange.getSecond(), end) < 0 ) {
917 orphanRegionRange.setSecond(end);
918 }
919 }
920 }
921 }
922 if (orphanRegionRange == null) {
923 LOG.warn("No data in dir " + p + ", sidelining data");
924 fixes++;
925 sidelineRegionDir(fs, hi);
926 return;
927 }
928 LOG.info("Min max keys are : [" + Bytes.toString(orphanRegionRange.getFirst()) + ", " +
929 Bytes.toString(orphanRegionRange.getSecond()) + ")");
930
931
932 HRegionInfo hri = new HRegionInfo(template.getTableName(), orphanRegionRange.getFirst(),
933 Bytes.add(orphanRegionRange.getSecond(), new byte[1]));
934 LOG.info("Creating new region : " + hri);
935 HRegion region = HBaseFsckRepair.createHDFSRegionDir(getConf(), hri, template);
936 Path target = region.getRegionFileSystem().getRegionDir();
937
938
939 mergeRegionDirs(target, hi);
940 fixes++;
941 }
942
943
944
945
946
947
948
949
950
951 private int restoreHdfsIntegrity() throws IOException, InterruptedException {
952
953 LOG.info("Loading HBase regioninfo from HDFS...");
954 loadHdfsRegionDirs();
955
956 int errs = errors.getErrorList().size();
957
958 tablesInfo = loadHdfsRegionInfos();
959 checkHdfsIntegrity(false, false);
960
961 if (errors.getErrorList().size() == errs) {
962 LOG.info("No integrity errors. We are done with this phase. Glorious.");
963 return 0;
964 }
965
966 if (shouldFixHdfsOrphans() && orphanHdfsDirs.size() > 0) {
967 adoptHdfsOrphans(orphanHdfsDirs);
968
969 }
970
971
972 if (shouldFixHdfsHoles()) {
973 clearState();
974 loadHdfsRegionDirs();
975 tablesInfo = loadHdfsRegionInfos();
976 tablesInfo = checkHdfsIntegrity(shouldFixHdfsHoles(), false);
977 }
978
979
980 if (shouldFixHdfsOverlaps()) {
981
982 clearState();
983 loadHdfsRegionDirs();
984 tablesInfo = loadHdfsRegionInfos();
985 tablesInfo = checkHdfsIntegrity(false, shouldFixHdfsOverlaps());
986 }
987
988 return errors.getErrorList().size();
989 }
990
991
992
993
994
995
996
997
998
999 private void offlineReferenceFileRepair() throws IOException {
1000 Configuration conf = getConf();
1001 Path hbaseRoot = FSUtils.getRootDir(conf);
1002 FileSystem fs = hbaseRoot.getFileSystem(conf);
1003 LOG.info("Computing mapping of all store files");
1004 Map<String, Path> allFiles = FSUtils.getTableStoreFilePathMap(fs, hbaseRoot, errors);
1005 errors.print("");
1006 LOG.info("Validating mapping using HDFS state");
1007 for (Path path: allFiles.values()) {
1008 boolean isReference = false;
1009 try {
1010 isReference = StoreFileInfo.isReference(path);
1011 } catch (Throwable t) {
1012
1013
1014
1015
1016 }
1017 if (!isReference) continue;
1018
1019 Path referredToFile = StoreFileInfo.getReferredToFile(path);
1020 if (fs.exists(referredToFile)) continue;
1021
1022
1023 errors.reportError(ERROR_CODE.LINGERING_REFERENCE_HFILE,
1024 "Found lingering reference file " + path);
1025 if (!shouldFixReferenceFiles()) continue;
1026
1027
1028 boolean success = false;
1029 String pathStr = path.toString();
1030
1031
1032
1033
1034
1035 int index = pathStr.lastIndexOf(Path.SEPARATOR_CHAR);
1036 for (int i = 0; index > 0 && i < 5; i++) {
1037 index = pathStr.lastIndexOf(Path.SEPARATOR_CHAR, index - 1);
1038 }
1039 if (index > 0) {
1040 Path rootDir = getSidelineDir();
1041 Path dst = new Path(rootDir, pathStr.substring(index + 1));
1042 fs.mkdirs(dst.getParent());
1043 LOG.info("Trying to sildeline reference file "
1044 + path + " to " + dst);
1045 setShouldRerun();
1046
1047 success = fs.rename(path, dst);
1048 debugLsr(dst);
1049
1050 }
1051 if (!success) {
1052 LOG.error("Failed to sideline reference file " + path);
1053 }
1054 }
1055 }
1056
1057
1058
1059
1060 private void reportEmptyMetaCells() {
1061 errors.print("Number of empty REGIONINFO_QUALIFIER rows in hbase:meta: " +
1062 emptyRegionInfoQualifiers.size());
1063 if (details) {
1064 for (Result r: emptyRegionInfoQualifiers) {
1065 errors.print(" " + r);
1066 }
1067 }
1068 }
1069
1070
1071
1072
1073 private void reportTablesInFlux() {
1074 AtomicInteger numSkipped = new AtomicInteger(0);
1075 HTableDescriptor[] allTables = getTables(numSkipped);
1076 errors.print("Number of Tables: " + allTables.length);
1077 if (details) {
1078 if (numSkipped.get() > 0) {
1079 errors.detail("Number of Tables in flux: " + numSkipped.get());
1080 }
1081 for (HTableDescriptor td : allTables) {
1082 errors.detail(" Table: " + td.getTableName() + "\t" +
1083 (td.isReadOnly() ? "ro" : "rw") + "\t" +
1084 (td.isMetaRegion() ? "META" : " ") + "\t" +
1085 " families: " + td.getFamilies().size());
1086 }
1087 }
1088 }
1089
1090 public ErrorReporter getErrors() {
1091 return errors;
1092 }
1093
1094
1095
1096
1097
1098 private void loadHdfsRegioninfo(HbckInfo hbi) throws IOException {
1099 Path regionDir = hbi.getHdfsRegionDir();
1100 if (regionDir == null) {
1101 if (hbi.getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
1102
1103 LOG.warn("No HDFS region dir found: " + hbi + " meta=" + hbi.metaEntry);
1104 }
1105 return;
1106 }
1107
1108 if (hbi.hdfsEntry.hri != null) {
1109
1110 return;
1111 }
1112
1113 FileSystem fs = FileSystem.get(getConf());
1114 HRegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir);
1115 LOG.debug("HRegionInfo read: " + hri.toString());
1116 hbi.hdfsEntry.hri = hri;
1117 }
1118
1119
1120
1121
1122
1123 public static class RegionRepairException extends IOException {
1124 private static final long serialVersionUID = 1L;
1125 final IOException ioe;
1126 public RegionRepairException(String s, IOException ioe) {
1127 super(s);
1128 this.ioe = ioe;
1129 }
1130 }
1131
1132
1133
1134
1135 private SortedMap<TableName, TableInfo> loadHdfsRegionInfos()
1136 throws IOException, InterruptedException {
1137 tablesInfo.clear();
1138
1139 Collection<HbckInfo> hbckInfos = regionInfoMap.values();
1140
1141
1142 List<WorkItemHdfsRegionInfo> hbis = new ArrayList<WorkItemHdfsRegionInfo>(hbckInfos.size());
1143 List<Future<Void>> hbiFutures;
1144
1145 for (HbckInfo hbi : hbckInfos) {
1146 WorkItemHdfsRegionInfo work = new WorkItemHdfsRegionInfo(hbi, this, errors);
1147 hbis.add(work);
1148 }
1149
1150
1151 hbiFutures = executor.invokeAll(hbis);
1152
1153 for(int i=0; i<hbiFutures.size(); i++) {
1154 WorkItemHdfsRegionInfo work = hbis.get(i);
1155 Future<Void> f = hbiFutures.get(i);
1156 try {
1157 f.get();
1158 } catch(ExecutionException e) {
1159 LOG.warn("Failed to read .regioninfo file for region " +
1160 work.hbi.getRegionNameAsString(), e.getCause());
1161 }
1162 }
1163
1164 Path hbaseRoot = FSUtils.getRootDir(getConf());
1165 FileSystem fs = hbaseRoot.getFileSystem(getConf());
1166
1167 for (HbckInfo hbi: hbckInfos) {
1168
1169 if (hbi.getHdfsHRI() == null) {
1170
1171 continue;
1172 }
1173
1174
1175
1176 TableName tableName = hbi.getTableName();
1177 if (tableName == null) {
1178
1179 LOG.warn("tableName was null for: " + hbi);
1180 continue;
1181 }
1182
1183 TableInfo modTInfo = tablesInfo.get(tableName);
1184 if (modTInfo == null) {
1185
1186 modTInfo = new TableInfo(tableName);
1187 tablesInfo.put(tableName, modTInfo);
1188 try {
1189 HTableDescriptor htd =
1190 FSTableDescriptors.getTableDescriptorFromFs(fs, hbaseRoot, tableName);
1191 modTInfo.htds.add(htd);
1192 } catch (IOException ioe) {
1193 if (!orphanTableDirs.containsKey(tableName)) {
1194 LOG.warn("Unable to read .tableinfo from " + hbaseRoot, ioe);
1195
1196 errors.reportError(ERROR_CODE.NO_TABLEINFO_FILE,
1197 "Unable to read .tableinfo from " + hbaseRoot + "/" + tableName);
1198 Set<String> columns = new HashSet<String>();
1199 orphanTableDirs.put(tableName, getColumnFamilyList(columns, hbi));
1200 }
1201 }
1202 }
1203 if (!hbi.isSkipChecks()) {
1204 modTInfo.addRegionInfo(hbi);
1205 }
1206 }
1207
1208 loadTableInfosForTablesWithNoRegion();
1209 errors.print("");
1210
1211 return tablesInfo;
1212 }
1213
1214
1215
1216
1217
1218
1219
1220
1221 private Set<String> getColumnFamilyList(Set<String> columns, HbckInfo hbi) throws IOException {
1222 Path regionDir = hbi.getHdfsRegionDir();
1223 FileSystem fs = regionDir.getFileSystem(getConf());
1224 FileStatus[] subDirs = fs.listStatus(regionDir, new FSUtils.FamilyDirFilter(fs));
1225 for (FileStatus subdir : subDirs) {
1226 String columnfamily = subdir.getPath().getName();
1227 columns.add(columnfamily);
1228 }
1229 return columns;
1230 }
1231
1232
1233
1234
1235
1236
1237
1238
1239 private boolean fabricateTableInfo(FSTableDescriptors fstd, TableName tableName,
1240 Set<String> columns) throws IOException {
1241 if (columns ==null || columns.isEmpty()) return false;
1242 HTableDescriptor htd = new HTableDescriptor(tableName);
1243 for (String columnfamimly : columns) {
1244 htd.addFamily(new HColumnDescriptor(columnfamimly));
1245 }
1246 fstd.createTableDescriptor(htd, true);
1247 return true;
1248 }
1249
1250
1251
1252
1253
1254 public void fixEmptyMetaCells() throws IOException {
1255 if (shouldFixEmptyMetaCells() && !emptyRegionInfoQualifiers.isEmpty()) {
1256 LOG.info("Trying to fix empty REGIONINFO_QUALIFIER hbase:meta rows.");
1257 for (Result region : emptyRegionInfoQualifiers) {
1258 deleteMetaRegion(region.getRow());
1259 errors.getErrorList().remove(ERROR_CODE.EMPTY_META_CELL);
1260 }
1261 emptyRegionInfoQualifiers.clear();
1262 }
1263 }
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274 public void fixOrphanTables() throws IOException {
1275 if (shouldFixTableOrphans() && !orphanTableDirs.isEmpty()) {
1276
1277 List<TableName> tmpList = new ArrayList<TableName>();
1278 tmpList.addAll(orphanTableDirs.keySet());
1279 HTableDescriptor[] htds = getHTableDescriptors(tmpList);
1280 Iterator<Entry<TableName, Set<String>>> iter =
1281 orphanTableDirs.entrySet().iterator();
1282 int j = 0;
1283 int numFailedCase = 0;
1284 FSTableDescriptors fstd = new FSTableDescriptors(getConf());
1285 while (iter.hasNext()) {
1286 Entry<TableName, Set<String>> entry =
1287 iter.next();
1288 TableName tableName = entry.getKey();
1289 LOG.info("Trying to fix orphan table error: " + tableName);
1290 if (j < htds.length) {
1291 if (tableName.equals(htds[j].getTableName())) {
1292 HTableDescriptor htd = htds[j];
1293 LOG.info("fixing orphan table: " + tableName + " from cache");
1294 fstd.createTableDescriptor(htd, true);
1295 j++;
1296 iter.remove();
1297 }
1298 } else {
1299 if (fabricateTableInfo(fstd, tableName, entry.getValue())) {
1300 LOG.warn("fixing orphan table: " + tableName + " with a default .tableinfo file");
1301 LOG.warn("Strongly recommend to modify the HTableDescriptor if necessary for: " + tableName);
1302 iter.remove();
1303 } else {
1304 LOG.error("Unable to create default .tableinfo for " + tableName + " while missing column family information");
1305 numFailedCase++;
1306 }
1307 }
1308 fixes++;
1309 }
1310
1311 if (orphanTableDirs.isEmpty()) {
1312
1313
1314 setShouldRerun();
1315 LOG.warn("Strongly recommend to re-run manually hfsck after all orphanTableDirs being fixed");
1316 } else if (numFailedCase > 0) {
1317 LOG.error("Failed to fix " + numFailedCase
1318 + " OrphanTables with default .tableinfo files");
1319 }
1320
1321 }
1322
1323 orphanTableDirs.clear();
1324
1325 }
1326
1327
1328
1329
1330
1331
1332 private HRegion createNewMeta() throws IOException {
1333 Path rootdir = FSUtils.getRootDir(getConf());
1334 Configuration c = getConf();
1335 HRegionInfo metaHRI = new HRegionInfo(HRegionInfo.FIRST_META_REGIONINFO);
1336 HTableDescriptor metaDescriptor = new FSTableDescriptors(c).get(TableName.META_TABLE_NAME);
1337 MasterFileSystem.setInfoFamilyCachingForMeta(metaDescriptor, false);
1338 HRegion meta = HRegion.createHRegion(metaHRI, rootdir, c, metaDescriptor);
1339 MasterFileSystem.setInfoFamilyCachingForMeta(metaDescriptor, true);
1340 return meta;
1341 }
1342
1343
1344
1345
1346
1347
1348
1349 private ArrayList<Put> generatePuts(
1350 SortedMap<TableName, TableInfo> tablesInfo) throws IOException {
1351 ArrayList<Put> puts = new ArrayList<Put>();
1352 boolean hasProblems = false;
1353 for (Entry<TableName, TableInfo> e : tablesInfo.entrySet()) {
1354 TableName name = e.getKey();
1355
1356
1357 if (name.compareTo(TableName.META_TABLE_NAME) == 0) {
1358 continue;
1359 }
1360
1361 TableInfo ti = e.getValue();
1362 for (Entry<byte[], Collection<HbckInfo>> spl : ti.sc.getStarts().asMap()
1363 .entrySet()) {
1364 Collection<HbckInfo> his = spl.getValue();
1365 int sz = his.size();
1366 if (sz != 1) {
1367
1368 LOG.error("Split starting at " + Bytes.toStringBinary(spl.getKey())
1369 + " had " + sz + " regions instead of exactly 1." );
1370 hasProblems = true;
1371 continue;
1372 }
1373
1374
1375 HbckInfo hi = his.iterator().next();
1376 HRegionInfo hri = hi.getHdfsHRI();
1377 Put p = MetaTableAccessor.makePutFromRegionInfo(hri);
1378 puts.add(p);
1379 }
1380 }
1381 return hasProblems ? null : puts;
1382 }
1383
1384
1385
1386
1387 private void suggestFixes(
1388 SortedMap<TableName, TableInfo> tablesInfo) throws IOException {
1389 logParallelMerge();
1390 for (TableInfo tInfo : tablesInfo.values()) {
1391 TableIntegrityErrorHandler handler = tInfo.new IntegrityFixSuggester(tInfo, errors);
1392 tInfo.checkRegionChain(handler);
1393 }
1394 }
1395
1396
1397
1398
1399
1400
1401
1402
1403 public boolean rebuildMeta(boolean fix) throws IOException,
1404 InterruptedException {
1405
1406
1407
1408
1409
1410 LOG.info("Loading HBase regioninfo from HDFS...");
1411 loadHdfsRegionDirs();
1412
1413 int errs = errors.getErrorList().size();
1414 tablesInfo = loadHdfsRegionInfos();
1415 checkHdfsIntegrity(false, false);
1416
1417
1418 if (errors.getErrorList().size() != errs) {
1419
1420 while(true) {
1421 fixes = 0;
1422 suggestFixes(tablesInfo);
1423 errors.clear();
1424 loadHdfsRegionInfos();
1425 checkHdfsIntegrity(shouldFixHdfsHoles(), shouldFixHdfsOverlaps());
1426
1427 int errCount = errors.getErrorList().size();
1428
1429 if (fixes == 0) {
1430 if (errCount > 0) {
1431 return false;
1432 } else {
1433 break;
1434 }
1435 }
1436 }
1437 }
1438
1439
1440 LOG.info("HDFS regioninfo's seems good. Sidelining old hbase:meta");
1441 Path backupDir = sidelineOldMeta();
1442
1443 LOG.info("Creating new hbase:meta");
1444 HRegion meta = createNewMeta();
1445
1446
1447 List<Put> puts = generatePuts(tablesInfo);
1448 if (puts == null) {
1449 LOG.fatal("Problem encountered when creating new hbase:meta entries. " +
1450 "You may need to restore the previously sidelined hbase:meta");
1451 return false;
1452 }
1453 meta.batchMutate(puts.toArray(new Put[puts.size()]));
1454 HRegion.closeHRegion(meta);
1455 LOG.info("Success! hbase:meta table rebuilt.");
1456 LOG.info("Old hbase:meta is moved into " + backupDir);
1457 return true;
1458 }
1459
1460
1461
1462
1463 private void logParallelMerge() {
1464 if (getConf().getBoolean("hbasefsck.overlap.merge.parallel", true)) {
1465 LOG.info("Handling overlap merges in parallel. set hbasefsck.overlap.merge.parallel to" +
1466 " false to run serially.");
1467 } else {
1468 LOG.info("Handling overlap merges serially. set hbasefsck.overlap.merge.parallel to" +
1469 " true to run in parallel.");
1470 }
1471 }
1472
1473 private SortedMap<TableName, TableInfo> checkHdfsIntegrity(boolean fixHoles,
1474 boolean fixOverlaps) throws IOException {
1475 LOG.info("Checking HBase region split map from HDFS data...");
1476 logParallelMerge();
1477 for (TableInfo tInfo : tablesInfo.values()) {
1478 TableIntegrityErrorHandler handler;
1479 if (fixHoles || fixOverlaps) {
1480 handler = tInfo.new HDFSIntegrityFixer(tInfo, errors, getConf(),
1481 fixHoles, fixOverlaps);
1482 } else {
1483 handler = tInfo.new IntegrityFixSuggester(tInfo, errors);
1484 }
1485 if (!tInfo.checkRegionChain(handler)) {
1486
1487 errors.report("Found inconsistency in table " + tInfo.getName());
1488 }
1489 }
1490 return tablesInfo;
1491 }
1492
1493 private Path getSidelineDir() throws IOException {
1494 if (sidelineDir == null) {
1495 Path hbaseDir = FSUtils.getRootDir(getConf());
1496 Path hbckDir = new Path(hbaseDir, HConstants.HBCK_SIDELINEDIR_NAME);
1497 sidelineDir = new Path(hbckDir, hbaseDir.getName() + "-"
1498 + startMillis);
1499 }
1500 return sidelineDir;
1501 }
1502
1503
1504
1505
1506 Path sidelineRegionDir(FileSystem fs, HbckInfo hi) throws IOException {
1507 return sidelineRegionDir(fs, null, hi);
1508 }
1509
1510
1511
1512
1513
1514
1515
1516
1517
1518 Path sidelineRegionDir(FileSystem fs,
1519 String parentDir, HbckInfo hi) throws IOException {
1520 TableName tableName = hi.getTableName();
1521 Path regionDir = hi.getHdfsRegionDir();
1522
1523 if (!fs.exists(regionDir)) {
1524 LOG.warn("No previous " + regionDir + " exists. Continuing.");
1525 return null;
1526 }
1527
1528 Path rootDir = getSidelineDir();
1529 if (parentDir != null) {
1530 rootDir = new Path(rootDir, parentDir);
1531 }
1532 Path sidelineTableDir= FSUtils.getTableDir(rootDir, tableName);
1533 Path sidelineRegionDir = new Path(sidelineTableDir, regionDir.getName());
1534 fs.mkdirs(sidelineRegionDir);
1535 boolean success = false;
1536 FileStatus[] cfs = fs.listStatus(regionDir);
1537 if (cfs == null) {
1538 LOG.info("Region dir is empty: " + regionDir);
1539 } else {
1540 for (FileStatus cf : cfs) {
1541 Path src = cf.getPath();
1542 Path dst = new Path(sidelineRegionDir, src.getName());
1543 if (fs.isFile(src)) {
1544
1545 success = fs.rename(src, dst);
1546 if (!success) {
1547 String msg = "Unable to rename file " + src + " to " + dst;
1548 LOG.error(msg);
1549 throw new IOException(msg);
1550 }
1551 continue;
1552 }
1553
1554
1555 fs.mkdirs(dst);
1556
1557 LOG.info("Sidelining files from " + src + " into containing region " + dst);
1558
1559
1560
1561
1562 FileStatus[] hfiles = fs.listStatus(src);
1563 if (hfiles != null && hfiles.length > 0) {
1564 for (FileStatus hfile : hfiles) {
1565 success = fs.rename(hfile.getPath(), dst);
1566 if (!success) {
1567 String msg = "Unable to rename file " + src + " to " + dst;
1568 LOG.error(msg);
1569 throw new IOException(msg);
1570 }
1571 }
1572 }
1573 LOG.debug("Sideline directory contents:");
1574 debugLsr(sidelineRegionDir);
1575 }
1576 }
1577
1578 LOG.info("Removing old region dir: " + regionDir);
1579 success = fs.delete(regionDir, true);
1580 if (!success) {
1581 String msg = "Unable to delete dir " + regionDir;
1582 LOG.error(msg);
1583 throw new IOException(msg);
1584 }
1585 return sidelineRegionDir;
1586 }
1587
1588
1589
1590
1591 void sidelineTable(FileSystem fs, TableName tableName, Path hbaseDir,
1592 Path backupHbaseDir) throws IOException {
1593 Path tableDir = FSUtils.getTableDir(hbaseDir, tableName);
1594 if (fs.exists(tableDir)) {
1595 Path backupTableDir= FSUtils.getTableDir(backupHbaseDir, tableName);
1596 fs.mkdirs(backupTableDir.getParent());
1597 boolean success = fs.rename(tableDir, backupTableDir);
1598 if (!success) {
1599 throw new IOException("Failed to move " + tableName + " from "
1600 + tableDir + " to " + backupTableDir);
1601 }
1602 } else {
1603 LOG.info("No previous " + tableName + " exists. Continuing.");
1604 }
1605 }
1606
1607
1608
1609
1610 Path sidelineOldMeta() throws IOException {
1611
1612 Path hbaseDir = FSUtils.getRootDir(getConf());
1613 FileSystem fs = hbaseDir.getFileSystem(getConf());
1614 Path backupDir = getSidelineDir();
1615 fs.mkdirs(backupDir);
1616
1617 try {
1618 sidelineTable(fs, TableName.META_TABLE_NAME, hbaseDir, backupDir);
1619 } catch (IOException e) {
1620 LOG.fatal("... failed to sideline meta. Currently in inconsistent state. To restore "
1621 + "try to rename hbase:meta in " + backupDir.getName() + " to "
1622 + hbaseDir.getName() + ".", e);
1623 throw e;
1624 }
1625 return backupDir;
1626 }
1627
1628
1629
1630
1631
1632
1633 private void loadDisabledTables()
1634 throws ZooKeeperConnectionException, IOException {
1635 HConnectionManager.execute(new HConnectable<Void>(getConf()) {
1636 @Override
1637 public Void connect(HConnection connection) throws IOException {
1638 ZooKeeperWatcher zkw = createZooKeeperWatcher();
1639 try {
1640 for (TableName tableName :
1641 ZKTableStateClientSideReader.getDisabledOrDisablingTables(zkw)) {
1642 disabledTables.add(tableName);
1643 }
1644 } catch (KeeperException ke) {
1645 throw new IOException(ke);
1646 } catch (InterruptedException e) {
1647 throw new InterruptedIOException();
1648 } finally {
1649 zkw.close();
1650 }
1651 return null;
1652 }
1653 });
1654 }
1655
1656
1657
1658
1659 private boolean isTableDisabled(HRegionInfo regionInfo) {
1660 return disabledTables.contains(regionInfo.getTable());
1661 }
1662
1663
1664
1665
1666
1667 public void loadHdfsRegionDirs() throws IOException, InterruptedException {
1668 Path rootDir = FSUtils.getRootDir(getConf());
1669 FileSystem fs = rootDir.getFileSystem(getConf());
1670
1671
1672 List<FileStatus> tableDirs = Lists.newArrayList();
1673
1674 boolean foundVersionFile = fs.exists(new Path(rootDir, HConstants.VERSION_FILE_NAME));
1675
1676 List<Path> paths = FSUtils.getTableDirs(fs, rootDir);
1677 for (Path path : paths) {
1678 TableName tableName = FSUtils.getTableName(path);
1679 if ((!checkMetaOnly &&
1680 isTableIncluded(tableName)) ||
1681 tableName.equals(TableName.META_TABLE_NAME)) {
1682 tableDirs.add(fs.getFileStatus(path));
1683 }
1684 }
1685
1686
1687 if (!foundVersionFile) {
1688 errors.reportError(ERROR_CODE.NO_VERSION_FILE,
1689 "Version file does not exist in root dir " + rootDir);
1690 if (shouldFixVersionFile()) {
1691 LOG.info("Trying to create a new " + HConstants.VERSION_FILE_NAME
1692 + " file.");
1693 setShouldRerun();
1694 FSUtils.setVersion(fs, rootDir, getConf().getInt(
1695 HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000), getConf().getInt(
1696 HConstants.VERSION_FILE_WRITE_ATTEMPTS,
1697 HConstants.DEFAULT_VERSION_FILE_WRITE_ATTEMPTS));
1698 }
1699 }
1700
1701
1702
1703
1704
1705
1706
1707
1708
1709
1710
1711
1712
1713
1714
1715
1716
1717
1718
1719
1720
1721
1722
1723
1724
1725
1726 private boolean recordMetaRegion() throws IOException {
1727 RegionLocations rl = ((ClusterConnection)connection).locateRegion(TableName.META_TABLE_NAME,
1728 HConstants.EMPTY_START_ROW, false, false);
1729 if (rl == null) {
1730 errors.reportError(ERROR_CODE.NULL_META_REGION,
1731 "META region or some of its attributes are null.");
1732 return false;
1733 }
1734 for (HRegionLocation metaLocation : rl.getRegionLocations()) {
1735
1736 if (metaLocation == null || metaLocation.getRegionInfo() == null ||
1737 metaLocation.getHostname() == null) {
1738 errors.reportError(ERROR_CODE.NULL_META_REGION,
1739 "META region or some of its attributes are null.");
1740 return false;
1741 }
1742 ServerName sn = metaLocation.getServerName();
1743 MetaEntry m = new MetaEntry(metaLocation.getRegionInfo(), sn, EnvironmentEdgeManager.currentTime());
1744 HbckInfo hbckInfo = regionInfoMap.get(metaLocation.getRegionInfo().getEncodedName());
1745 if (hbckInfo == null) {
1746 regionInfoMap.put(metaLocation.getRegionInfo().getEncodedName(), new HbckInfo(m));
1747 } else {
1748 hbckInfo.metaEntry = m;
1749 }
1750 }
1751 return true;
1752 }
1753
1754 private ZooKeeperWatcher createZooKeeperWatcher() throws IOException {
1755 return new ZooKeeperWatcher(getConf(), "hbase Fsck", new Abortable() {
1756 @Override
1757 public void abort(String why, Throwable e) {
1758 LOG.error(why, e);
1759 System.exit(1);
1760 }
1761
1762 @Override
1763 public boolean isAborted() {
1764 return false;
1765 }
1766
1767 });
1768 }
1769
1770 private ServerName getMetaRegionServerName(int replicaId)
1771 throws IOException, KeeperException {
1772 ZooKeeperWatcher zkw = createZooKeeperWatcher();
1773 ServerName sn = null;
1774 try {
1775 sn = new MetaTableLocator().getMetaRegionLocation(zkw, replicaId);
1776 } finally {
1777 zkw.close();
1778 }
1779 return sn;
1780 }
1781
1782
1783
1784
1785
1786
1787 void processRegionServers(Collection<ServerName> regionServerList)
1788 throws IOException, InterruptedException {
1789
1790 List<WorkItemRegion> workItems = new ArrayList<WorkItemRegion>(regionServerList.size());
1791 List<Future<Void>> workFutures;
1792
1793
1794 for (ServerName rsinfo: regionServerList) {
1795 workItems.add(new WorkItemRegion(this, rsinfo, errors, connection));
1796 }
1797
1798 workFutures = executor.invokeAll(workItems);
1799
1800 for(int i=0; i<workFutures.size(); i++) {
1801 WorkItemRegion item = workItems.get(i);
1802 Future<Void> f = workFutures.get(i);
1803 try {
1804 f.get();
1805 } catch(ExecutionException e) {
1806 LOG.warn("Could not process regionserver " + item.rsinfo.getHostAndPort(),
1807 e.getCause());
1808 }
1809 }
1810 }
1811
1812
1813
1814
1815 private void checkAndFixConsistency()
1816 throws IOException, KeeperException, InterruptedException {
1817
1818
1819 List<CheckRegionConsistencyWorkItem> workItems =
1820 new ArrayList<CheckRegionConsistencyWorkItem>(regionInfoMap.size());
1821 for (java.util.Map.Entry<String, HbckInfo> e: regionInfoMap.entrySet()) {
1822 if (e.getValue().getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
1823 workItems.add(new CheckRegionConsistencyWorkItem(e.getKey(), e.getValue()));
1824 }
1825 }
1826 checkRegionConsistencyConcurrently(workItems);
1827
1828 boolean prevHdfsCheck = shouldCheckHdfs();
1829 setCheckHdfs(false);
1830
1831
1832 List<CheckRegionConsistencyWorkItem> replicaWorkItems =
1833 new ArrayList<CheckRegionConsistencyWorkItem>(regionInfoMap.size());
1834 for (java.util.Map.Entry<String, HbckInfo> e: regionInfoMap.entrySet()) {
1835 if (e.getValue().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
1836 replicaWorkItems.add(new CheckRegionConsistencyWorkItem(e.getKey(), e.getValue()));
1837 }
1838 }
1839 checkRegionConsistencyConcurrently(replicaWorkItems);
1840 setCheckHdfs(prevHdfsCheck);
1841
1842
1843
1844
1845
1846 int terminateThreshold = getConf().getInt("hbase.hbck.skipped.regions.limit", 0);
1847 int numOfSkippedRegions = skippedRegions.size();
1848 if (numOfSkippedRegions > 0 && numOfSkippedRegions > terminateThreshold) {
1849 throw new IOException(numOfSkippedRegions
1850 + " region(s) could not be checked or repaired. See logs for detail.");
1851 }
1852 }
1853
1854
1855
1856
1857 private void checkRegionConsistencyConcurrently(
1858 final List<CheckRegionConsistencyWorkItem> workItems)
1859 throws IOException, KeeperException, InterruptedException {
1860 if (workItems.isEmpty()) {
1861 return;
1862 }
1863
1864 List<Future<Void>> workFutures = executor.invokeAll(workItems);
1865 for(Future<Void> f: workFutures) {
1866 try {
1867 f.get();
1868 } catch(ExecutionException e1) {
1869 LOG.warn("Could not check region consistency " , e1.getCause());
1870 if (e1.getCause() instanceof IOException) {
1871 throw (IOException)e1.getCause();
1872 } else if (e1.getCause() instanceof KeeperException) {
1873 throw (KeeperException)e1.getCause();
1874 } else if (e1.getCause() instanceof InterruptedException) {
1875 throw (InterruptedException)e1.getCause();
1876 } else {
1877 throw new IOException(e1.getCause());
1878 }
1879 }
1880 }
1881 }
1882
1883 class CheckRegionConsistencyWorkItem implements Callable<Void> {
1884 private final String key;
1885 private final HbckInfo hbi;
1886
1887 CheckRegionConsistencyWorkItem(String key, HbckInfo hbi) {
1888 this.key = key;
1889 this.hbi = hbi;
1890 }
1891
1892 @Override
1893 public synchronized Void call() throws Exception {
1894 try {
1895 checkRegionConsistency(key, hbi);
1896 } catch (Exception e) {
1897
1898
1899 LOG.warn("Unable to complete check or repair the region '" + hbi.getRegionNameAsString()
1900 + "'.", e);
1901 if (hbi.getHdfsHRI().isMetaRegion()) {
1902 throw e;
1903 }
1904 LOG.warn("Skip region '" + hbi.getRegionNameAsString() + "'");
1905 addSkippedRegion(hbi);
1906 }
1907 return null;
1908 }
1909 }
1910
1911 private void addSkippedRegion(final HbckInfo hbi) {
1912 Set<String> skippedRegionNames = skippedRegions.get(hbi.getTableName());
1913 if (skippedRegionNames == null) {
1914 skippedRegionNames = new HashSet<String>();
1915 }
1916 skippedRegionNames.add(hbi.getRegionNameAsString());
1917 skippedRegions.put(hbi.getTableName(), skippedRegionNames);
1918 }
1919
1920 private void preCheckPermission() throws IOException, AccessDeniedException {
1921 if (shouldIgnorePreCheckPermission()) {
1922 return;
1923 }
1924
1925 Path hbaseDir = FSUtils.getRootDir(getConf());
1926 FileSystem fs = hbaseDir.getFileSystem(getConf());
1927 UserProvider userProvider = UserProvider.instantiate(getConf());
1928 UserGroupInformation ugi = userProvider.getCurrent().getUGI();
1929 FileStatus[] files = fs.listStatus(hbaseDir);
1930 for (FileStatus file : files) {
1931 try {
1932 FSUtils.checkAccess(ugi, file, FsAction.WRITE);
1933 } catch (AccessDeniedException ace) {
1934 LOG.warn("Got AccessDeniedException when preCheckPermission ", ace);
1935 errors.reportError(ERROR_CODE.WRONG_USAGE, "Current user " + ugi.getUserName()
1936 + " does not have write perms to " + file.getPath()
1937 + ". Please rerun hbck as hdfs user " + file.getOwner());
1938 throw ace;
1939 }
1940 }
1941 }
1942
1943
1944
1945
1946 private void deleteMetaRegion(HbckInfo hi) throws IOException {
1947 deleteMetaRegion(hi.metaEntry.getRegionName());
1948 }
1949
1950
1951
1952
1953 private void deleteMetaRegion(byte[] metaKey) throws IOException {
1954 Delete d = new Delete(metaKey);
1955 meta.delete(d);
1956 LOG.info("Deleted " + Bytes.toString(metaKey) + " from META" );
1957 }
1958
1959
1960
1961
1962 private void resetSplitParent(HbckInfo hi) throws IOException {
1963 RowMutations mutations = new RowMutations(hi.metaEntry.getRegionName());
1964 Delete d = new Delete(hi.metaEntry.getRegionName());
1965 d.deleteColumn(HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER);
1966 d.deleteColumn(HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER);
1967 mutations.add(d);
1968
1969 HRegionInfo hri = new HRegionInfo(hi.metaEntry);
1970 hri.setOffline(false);
1971 hri.setSplit(false);
1972 Put p = MetaTableAccessor.makePutFromRegionInfo(hri);
1973 mutations.add(p);
1974
1975 meta.mutateRow(mutations);
1976 LOG.info("Reset split parent " + hi.metaEntry.getRegionNameAsString() + " in META" );
1977 }
1978
1979
1980
1981
1982
1983
1984
1985
1986
1987 private void offline(byte[] regionName) throws IOException {
1988 String regionString = Bytes.toStringBinary(regionName);
1989 if (!rsSupportsOffline) {
1990 LOG.warn("Using unassign region " + regionString
1991 + " instead of using offline method, you should"
1992 + " restart HMaster after these repairs");
1993 admin.unassign(regionName, true);
1994 return;
1995 }
1996
1997
1998 try {
1999 LOG.info("Offlining region " + regionString);
2000 admin.offline(regionName);
2001 } catch (IOException ioe) {
2002 String notFoundMsg = "java.lang.NoSuchMethodException: " +
2003 "org.apache.hadoop.hbase.master.HMaster.offline([B)";
2004 if (ioe.getMessage().contains(notFoundMsg)) {
2005 LOG.warn("Using unassign region " + regionString
2006 + " instead of using offline method, you should"
2007 + " restart HMaster after these repairs");
2008 rsSupportsOffline = false;
2009 admin.unassign(regionName, true);
2010 return;
2011 }
2012 throw ioe;
2013 }
2014 }
2015
2016 private void undeployRegions(HbckInfo hi) throws IOException, InterruptedException {
2017 undeployRegionsForHbi(hi);
2018
2019 if (hi.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
2020 return;
2021 }
2022 int numReplicas = admin.getTableDescriptor(hi.getTableName()).getRegionReplication();
2023 for (int i = 1; i < numReplicas; i++) {
2024 if (hi.getPrimaryHRIForDeployedReplica() == null) continue;
2025 HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(
2026 hi.getPrimaryHRIForDeployedReplica(), i);
2027 HbckInfo h = regionInfoMap.get(hri.getEncodedName());
2028 if (h != null) {
2029 undeployRegionsForHbi(h);
2030
2031
2032 h.setSkipChecks(true);
2033 }
2034 }
2035 }
2036
2037 private void undeployRegionsForHbi(HbckInfo hi) throws IOException, InterruptedException {
2038 for (OnlineEntry rse : hi.deployedEntries) {
2039 LOG.debug("Undeploy region " + rse.hri + " from " + rse.hsa);
2040 try {
2041 HBaseFsckRepair.closeRegionSilentlyAndWait(connection, rse.hsa, rse.hri);
2042 offline(rse.hri.getRegionName());
2043 } catch (IOException ioe) {
2044 LOG.warn("Got exception when attempting to offline region "
2045 + Bytes.toString(rse.hri.getRegionName()), ioe);
2046 }
2047 }
2048 }
2049
2050
2051
2052
2053
2054
2055
2056
2057
2058
2059
2060
2061
2062 private void closeRegion(HbckInfo hi) throws IOException, InterruptedException {
2063 if (hi.metaEntry == null && hi.hdfsEntry == null) {
2064 undeployRegions(hi);
2065 return;
2066 }
2067
2068
2069 Get get = new Get(hi.getRegionName());
2070 get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
2071 get.addColumn(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
2072 get.addColumn(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER);
2073
2074 if (hi.getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
2075 int numReplicas = admin.getTableDescriptor(hi.getTableName()).getRegionReplication();
2076 for (int i = 0; i < numReplicas; i++) {
2077 get.addColumn(HConstants.CATALOG_FAMILY, MetaTableAccessor.getServerColumn(i));
2078 get.addColumn(HConstants.CATALOG_FAMILY, MetaTableAccessor.getStartCodeColumn(i));
2079 }
2080 }
2081 Result r = meta.get(get);
2082 RegionLocations rl = MetaTableAccessor.getRegionLocations(r);
2083 if (rl == null) {
2084 LOG.warn("Unable to close region " + hi.getRegionNameAsString() +
2085 " since meta does not have handle to reach it");
2086 return;
2087 }
2088 for (HRegionLocation h : rl.getRegionLocations()) {
2089 ServerName serverName = h.getServerName();
2090 if (serverName == null) {
2091 errors.reportError("Unable to close region "
2092 + hi.getRegionNameAsString() + " because meta does not "
2093 + "have handle to reach it.");
2094 continue;
2095 }
2096 HRegionInfo hri = h.getRegionInfo();
2097 if (hri == null) {
2098 LOG.warn("Unable to close region " + hi.getRegionNameAsString()
2099 + " because hbase:meta had invalid or missing "
2100 + HConstants.CATALOG_FAMILY_STR + ":"
2101 + Bytes.toString(HConstants.REGIONINFO_QUALIFIER)
2102 + " qualifier value.");
2103 continue;
2104 }
2105
2106 HBaseFsckRepair.closeRegionSilentlyAndWait(connection, serverName, hri);
2107 }
2108 }
2109
2110 private void tryAssignmentRepair(HbckInfo hbi, String msg) throws IOException,
2111 KeeperException, InterruptedException {
2112
2113 if (shouldFixAssignments()) {
2114 errors.print(msg);
2115 undeployRegions(hbi);
2116 setShouldRerun();
2117 HRegionInfo hri = hbi.getHdfsHRI();
2118 if (hri == null) {
2119 hri = hbi.metaEntry;
2120 }
2121 HBaseFsckRepair.fixUnassigned(admin, hri);
2122 HBaseFsckRepair.waitUntilAssigned(admin, hri);
2123
2124
2125 if (hbi.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) return;
2126 int replicationCount = admin.getTableDescriptor(hri.getTable()).getRegionReplication();
2127 for (int i = 1; i < replicationCount; i++) {
2128 hri = RegionReplicaUtil.getRegionInfoForReplica(hri, i);
2129 HbckInfo h = regionInfoMap.get(hri.getEncodedName());
2130 if (h != null) {
2131 undeployRegions(h);
2132
2133
2134 h.setSkipChecks(true);
2135 }
2136 HBaseFsckRepair.fixUnassigned(admin, hri);
2137 HBaseFsckRepair.waitUntilAssigned(admin, hri);
2138 }
2139
2140 }
2141 }
2142
2143
2144
2145
2146 private void checkRegionConsistency(final String key, final HbckInfo hbi)
2147 throws IOException, KeeperException, InterruptedException {
2148
2149 if (hbi.isSkipChecks()) return;
2150 String descriptiveName = hbi.toString();
2151 boolean inMeta = hbi.metaEntry != null;
2152
2153 boolean inHdfs = !shouldCheckHdfs() || hbi.getHdfsRegionDir() != null;
2154 boolean hasMetaAssignment = inMeta && hbi.metaEntry.regionServer != null;
2155 boolean isDeployed = !hbi.deployedOn.isEmpty();
2156 boolean isMultiplyDeployed = hbi.deployedOn.size() > 1;
2157 boolean deploymentMatchesMeta =
2158 hasMetaAssignment && isDeployed && !isMultiplyDeployed &&
2159 hbi.metaEntry.regionServer.equals(hbi.deployedOn.get(0));
2160 boolean splitParent =
2161 (hbi.metaEntry == null)? false: hbi.metaEntry.isSplit() && hbi.metaEntry.isOffline();
2162 boolean shouldBeDeployed = inMeta && !isTableDisabled(hbi.metaEntry);
2163 boolean recentlyModified = inHdfs &&
2164 hbi.getModTime() + timelag > EnvironmentEdgeManager.currentTime();
2165
2166
2167 if (hbi.containsOnlyHdfsEdits()) {
2168 return;
2169 }
2170 if (inMeta && inHdfs && isDeployed && deploymentMatchesMeta && shouldBeDeployed) {
2171 return;
2172 } else if (inMeta && inHdfs && !shouldBeDeployed && !isDeployed) {
2173 LOG.info("Region " + descriptiveName + " is in META, and in a disabled " +
2174 "tabled that is not deployed");
2175 return;
2176 } else if (recentlyModified) {
2177 LOG.warn("Region " + descriptiveName + " was recently modified -- skipping");
2178 return;
2179 }
2180
2181 else if (!inMeta && !inHdfs && !isDeployed) {
2182
2183 assert false : "Entry for region with no data";
2184 } else if (!inMeta && !inHdfs && isDeployed) {
2185 errors.reportError(ERROR_CODE.NOT_IN_META_HDFS, "Region "
2186 + descriptiveName + ", key=" + key + ", not on HDFS or in hbase:meta but " +
2187 "deployed on " + Joiner.on(", ").join(hbi.deployedOn));
2188 if (shouldFixAssignments()) {
2189 undeployRegions(hbi);
2190 }
2191
2192 } else if (!inMeta && inHdfs && !isDeployed) {
2193 if (hbi.isMerged()) {
2194
2195
2196 hbi.setSkipChecks(true);
2197 LOG.info("Region " + descriptiveName
2198 + " got merge recently, its file(s) will be cleaned by CatalogJanitor later");
2199 return;
2200 }
2201 errors.reportError(ERROR_CODE.NOT_IN_META_OR_DEPLOYED, "Region "
2202 + descriptiveName + " on HDFS, but not listed in hbase:meta " +
2203 "or deployed on any region server");
2204
2205 if (shouldFixMeta()) {
2206 if (!hbi.isHdfsRegioninfoPresent()) {
2207 LOG.error("Region " + hbi.getHdfsHRI() + " could have been repaired"
2208 + " in table integrity repair phase if -fixHdfsOrphans was" +
2209 " used.");
2210 return;
2211 }
2212
2213 HRegionInfo hri = hbi.getHdfsHRI();
2214 TableInfo tableInfo = tablesInfo.get(hri.getTable());
2215
2216 for (HRegionInfo region : tableInfo.getRegionsFromMeta()) {
2217 if (Bytes.compareTo(region.getStartKey(), hri.getStartKey()) <= 0
2218 && (region.getEndKey().length == 0 || Bytes.compareTo(region.getEndKey(),
2219 hri.getEndKey()) >= 0)
2220 && Bytes.compareTo(region.getStartKey(), hri.getEndKey()) <= 0) {
2221 if(region.isSplit() || region.isOffline()) continue;
2222 Path regionDir = hbi.getHdfsRegionDir();
2223 FileSystem fs = regionDir.getFileSystem(getConf());
2224 List<Path> familyDirs = FSUtils.getFamilyDirs(fs, regionDir);
2225 for (Path familyDir : familyDirs) {
2226 List<Path> referenceFilePaths = FSUtils.getReferenceFilePaths(fs, familyDir);
2227 for (Path referenceFilePath : referenceFilePaths) {
2228 Path parentRegionDir =
2229 StoreFileInfo.getReferredToFile(referenceFilePath).getParent().getParent();
2230 if (parentRegionDir.toString().endsWith(region.getEncodedName())) {
2231 LOG.warn(hri + " start and stop keys are in the range of " + region
2232 + ". The region might not be cleaned up from hdfs when region " + region
2233 + " split failed. Hence deleting from hdfs.");
2234 HRegionFileSystem.deleteRegionFromFileSystem(getConf(), fs,
2235 regionDir.getParent(), hri);
2236 return;
2237 }
2238 }
2239 }
2240 }
2241 }
2242
2243 LOG.info("Patching hbase:meta with .regioninfo: " + hbi.getHdfsHRI());
2244 int numReplicas = admin.getTableDescriptor(hbi.getTableName()).getRegionReplication();
2245 HBaseFsckRepair.fixMetaHoleOnlineAndAddReplicas(getConf(), hbi.getHdfsHRI(),
2246 admin.getClusterStatus().getServers(), numReplicas);
2247
2248 tryAssignmentRepair(hbi, "Trying to reassign region...");
2249 }
2250
2251 } else if (!inMeta && inHdfs && isDeployed) {
2252 errors.reportError(ERROR_CODE.NOT_IN_META, "Region " + descriptiveName
2253 + " not in META, but deployed on " + Joiner.on(", ").join(hbi.deployedOn));
2254 debugLsr(hbi.getHdfsRegionDir());
2255 if (hbi.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
2256
2257
2258
2259
2260 if (shouldFixAssignments()) {
2261 undeployRegionsForHbi(hbi);
2262 }
2263 }
2264 if (shouldFixMeta() && hbi.getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
2265 if (!hbi.isHdfsRegioninfoPresent()) {
2266 LOG.error("This should have been repaired in table integrity repair phase");
2267 return;
2268 }
2269
2270 LOG.info("Patching hbase:meta with with .regioninfo: " + hbi.getHdfsHRI());
2271 int numReplicas = admin.getTableDescriptor(hbi.getTableName()).getRegionReplication();
2272 HBaseFsckRepair.fixMetaHoleOnlineAndAddReplicas(getConf(), hbi.getHdfsHRI(),
2273 admin.getClusterStatus().getServers(), numReplicas);
2274 tryAssignmentRepair(hbi, "Trying to fix unassigned region...");
2275 }
2276
2277
2278 } else if (inMeta && inHdfs && !isDeployed && splitParent) {
2279
2280
2281 if (hbi.metaEntry.splitA != null && hbi.metaEntry.splitB != null) {
2282
2283 HbckInfo infoA = this.regionInfoMap.get(hbi.metaEntry.splitA.getEncodedName());
2284 HbckInfo infoB = this.regionInfoMap.get(hbi.metaEntry.splitB.getEncodedName());
2285 if (infoA != null && infoB != null) {
2286
2287 hbi.setSkipChecks(true);
2288 return;
2289 }
2290 }
2291
2292
2293
2294 if (hbi.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
2295 LOG.info("Region " + descriptiveName + " is a split parent in META, in HDFS, "
2296 + "and not deployed on any region server. This may be transient.");
2297 hbi.setSkipChecks(true);
2298 return;
2299 }
2300
2301 errors.reportError(ERROR_CODE.LINGERING_SPLIT_PARENT, "Region "
2302 + descriptiveName + " is a split parent in META, in HDFS, "
2303 + "and not deployed on any region server. This could be transient, "
2304 + "consider to run the catalog janitor first!");
2305 if (shouldFixSplitParents()) {
2306 setShouldRerun();
2307 resetSplitParent(hbi);
2308 }
2309 } else if (inMeta && !inHdfs && !isDeployed) {
2310 errors.reportError(ERROR_CODE.NOT_IN_HDFS_OR_DEPLOYED, "Region "
2311 + descriptiveName + " found in META, but not in HDFS "
2312 + "or deployed on any region server.");
2313 if (shouldFixMeta()) {
2314 deleteMetaRegion(hbi);
2315 }
2316 } else if (inMeta && !inHdfs && isDeployed) {
2317 errors.reportError(ERROR_CODE.NOT_IN_HDFS, "Region " + descriptiveName
2318 + " found in META, but not in HDFS, " +
2319 "and deployed on " + Joiner.on(", ").join(hbi.deployedOn));
2320
2321
2322
2323 if (shouldFixAssignments()) {
2324 errors.print("Trying to fix unassigned region...");
2325 undeployRegions(hbi);
2326 }
2327 if (shouldFixMeta()) {
2328
2329 deleteMetaRegion(hbi);
2330 }
2331 } else if (inMeta && inHdfs && !isDeployed && shouldBeDeployed) {
2332 errors.reportError(ERROR_CODE.NOT_DEPLOYED, "Region " + descriptiveName
2333 + " not deployed on any region server.");
2334 tryAssignmentRepair(hbi, "Trying to fix unassigned region...");
2335 } else if (inMeta && inHdfs && isDeployed && !shouldBeDeployed) {
2336 errors.reportError(ERROR_CODE.SHOULD_NOT_BE_DEPLOYED,
2337 "Region " + descriptiveName + " should not be deployed according " +
2338 "to META, but is deployed on " + Joiner.on(", ").join(hbi.deployedOn));
2339 if (shouldFixAssignments()) {
2340 errors.print("Trying to close the region " + descriptiveName);
2341 setShouldRerun();
2342 HBaseFsckRepair.fixMultiAssignment(connection, hbi.metaEntry, hbi.deployedOn);
2343 }
2344 } else if (inMeta && inHdfs && isMultiplyDeployed) {
2345 errors.reportError(ERROR_CODE.MULTI_DEPLOYED, "Region " + descriptiveName
2346 + " is listed in hbase:meta on region server " + hbi.metaEntry.regionServer
2347 + " but is multiply assigned to region servers " +
2348 Joiner.on(", ").join(hbi.deployedOn));
2349
2350 if (shouldFixAssignments()) {
2351 errors.print("Trying to fix assignment error...");
2352 setShouldRerun();
2353 HBaseFsckRepair.fixMultiAssignment(connection, hbi.metaEntry, hbi.deployedOn);
2354 }
2355 } else if (inMeta && inHdfs && isDeployed && !deploymentMatchesMeta) {
2356 errors.reportError(ERROR_CODE.SERVER_DOES_NOT_MATCH_META, "Region "
2357 + descriptiveName + " listed in hbase:meta on region server " +
2358 hbi.metaEntry.regionServer + " but found on region server " +
2359 hbi.deployedOn.get(0));
2360
2361 if (shouldFixAssignments()) {
2362 errors.print("Trying to fix assignment error...");
2363 setShouldRerun();
2364 HBaseFsckRepair.fixMultiAssignment(connection, hbi.metaEntry, hbi.deployedOn);
2365 HBaseFsckRepair.waitUntilAssigned(admin, hbi.getHdfsHRI());
2366 }
2367 } else {
2368 errors.reportError(ERROR_CODE.UNKNOWN, "Region " + descriptiveName +
2369 " is in an unforeseen state:" +
2370 " inMeta=" + inMeta +
2371 " inHdfs=" + inHdfs +
2372 " isDeployed=" + isDeployed +
2373 " isMultiplyDeployed=" + isMultiplyDeployed +
2374 " deploymentMatchesMeta=" + deploymentMatchesMeta +
2375 " shouldBeDeployed=" + shouldBeDeployed);
2376 }
2377 }
2378
2379
2380
2381
2382
2383
2384
2385 SortedMap<TableName, TableInfo> checkIntegrity() throws IOException {
2386 tablesInfo = new TreeMap<TableName,TableInfo> ();
2387 LOG.debug("There are " + regionInfoMap.size() + " region info entries");
2388 for (HbckInfo hbi : regionInfoMap.values()) {
2389
2390 if (hbi.metaEntry == null) {
2391
2392 Path p = hbi.getHdfsRegionDir();
2393 if (p == null) {
2394 errors.report("No regioninfo in Meta or HDFS. " + hbi);
2395 }
2396
2397
2398 continue;
2399 }
2400 if (hbi.metaEntry.regionServer == null) {
2401 errors.detail("Skipping region because no region server: " + hbi);
2402 continue;
2403 }
2404 if (hbi.metaEntry.isOffline()) {
2405 errors.detail("Skipping region because it is offline: " + hbi);
2406 continue;
2407 }
2408 if (hbi.containsOnlyHdfsEdits()) {
2409 errors.detail("Skipping region because it only contains edits" + hbi);
2410 continue;
2411 }
2412
2413
2414
2415
2416
2417
2418 if (hbi.deployedOn.size() == 0) continue;
2419
2420
2421 TableName tableName = hbi.metaEntry.getTable();
2422 TableInfo modTInfo = tablesInfo.get(tableName);
2423 if (modTInfo == null) {
2424 modTInfo = new TableInfo(tableName);
2425 }
2426 for (ServerName server : hbi.deployedOn) {
2427 modTInfo.addServer(server);
2428 }
2429
2430 if (!hbi.isSkipChecks()) {
2431 modTInfo.addRegionInfo(hbi);
2432 }
2433
2434 tablesInfo.put(tableName, modTInfo);
2435 }
2436
2437 loadTableInfosForTablesWithNoRegion();
2438
2439 logParallelMerge();
2440 for (TableInfo tInfo : tablesInfo.values()) {
2441 TableIntegrityErrorHandler handler = tInfo.new IntegrityFixSuggester(tInfo, errors);
2442 if (!tInfo.checkRegionChain(handler)) {
2443 errors.report("Found inconsistency in table " + tInfo.getName());
2444 }
2445 }
2446 return tablesInfo;
2447 }
2448
2449
2450
2451
2452 private void loadTableInfosForTablesWithNoRegion() throws IOException {
2453 Map<String, HTableDescriptor> allTables = new FSTableDescriptors(getConf()).getAll();
2454 for (HTableDescriptor htd : allTables.values()) {
2455 if (checkMetaOnly && !htd.isMetaTable()) {
2456 continue;
2457 }
2458
2459 TableName tableName = htd.getTableName();
2460 if (isTableIncluded(tableName) && !tablesInfo.containsKey(tableName)) {
2461 TableInfo tableInfo = new TableInfo(tableName);
2462 tableInfo.htds.add(htd);
2463 tablesInfo.put(htd.getTableName(), tableInfo);
2464 }
2465 }
2466 }
2467
2468
2469
2470
2471
2472 public int mergeRegionDirs(Path targetRegionDir, HbckInfo contained) throws IOException {
2473 int fileMoves = 0;
2474 String thread = Thread.currentThread().getName();
2475 LOG.debug("[" + thread + "] Contained region dir after close and pause");
2476 debugLsr(contained.getHdfsRegionDir());
2477
2478
2479 FileSystem fs = targetRegionDir.getFileSystem(getConf());
2480 FileStatus[] dirs = null;
2481 try {
2482 dirs = fs.listStatus(contained.getHdfsRegionDir());
2483 } catch (FileNotFoundException fnfe) {
2484
2485
2486 if (!fs.exists(contained.getHdfsRegionDir())) {
2487 LOG.warn("[" + thread + "] HDFS region dir " + contained.getHdfsRegionDir()
2488 + " is missing. Assuming already sidelined or moved.");
2489 } else {
2490 sidelineRegionDir(fs, contained);
2491 }
2492 return fileMoves;
2493 }
2494
2495 if (dirs == null) {
2496 if (!fs.exists(contained.getHdfsRegionDir())) {
2497 LOG.warn("[" + thread + "] HDFS region dir " + contained.getHdfsRegionDir()
2498 + " already sidelined.");
2499 } else {
2500 sidelineRegionDir(fs, contained);
2501 }
2502 return fileMoves;
2503 }
2504
2505 for (FileStatus cf : dirs) {
2506 Path src = cf.getPath();
2507 Path dst = new Path(targetRegionDir, src.getName());
2508
2509 if (src.getName().equals(HRegionFileSystem.REGION_INFO_FILE)) {
2510
2511 continue;
2512 }
2513
2514 if (src.getName().equals(HConstants.HREGION_OLDLOGDIR_NAME)) {
2515
2516 continue;
2517 }
2518
2519 LOG.info("[" + thread + "] Moving files from " + src + " into containing region " + dst);
2520
2521
2522
2523
2524 for (FileStatus hfile : fs.listStatus(src)) {
2525 boolean success = fs.rename(hfile.getPath(), dst);
2526 if (success) {
2527 fileMoves++;
2528 }
2529 }
2530 LOG.debug("[" + thread + "] Sideline directory contents:");
2531 debugLsr(targetRegionDir);
2532 }
2533
2534
2535 sidelineRegionDir(fs, contained);
2536 LOG.info("[" + thread + "] Sidelined region dir "+ contained.getHdfsRegionDir() + " into " +
2537 getSidelineDir());
2538 debugLsr(contained.getHdfsRegionDir());
2539
2540 return fileMoves;
2541 }
2542
2543
2544 static class WorkItemOverlapMerge implements Callable<Void> {
2545 private TableIntegrityErrorHandler handler;
2546 Collection<HbckInfo> overlapgroup;
2547
2548 WorkItemOverlapMerge(Collection<HbckInfo> overlapgroup, TableIntegrityErrorHandler handler) {
2549 this.handler = handler;
2550 this.overlapgroup = overlapgroup;
2551 }
2552
2553 @Override
2554 public Void call() throws Exception {
2555 handler.handleOverlapGroup(overlapgroup);
2556 return null;
2557 }
2558 };
2559
2560
2561
2562
2563
2564 public class TableInfo {
2565 TableName tableName;
2566 TreeSet <ServerName> deployedOn;
2567
2568
2569 final List<HbckInfo> backwards = new ArrayList<HbckInfo>();
2570
2571
2572 final Map<Path, HbckInfo> sidelinedRegions = new HashMap<Path, HbckInfo>();
2573
2574
2575 final RegionSplitCalculator<HbckInfo> sc = new RegionSplitCalculator<HbckInfo>(cmp);
2576
2577
2578 final Set<HTableDescriptor> htds = new HashSet<HTableDescriptor>();
2579
2580
2581 final Multimap<byte[], HbckInfo> overlapGroups =
2582 TreeMultimap.create(RegionSplitCalculator.BYTES_COMPARATOR, cmp);
2583
2584
2585 private ImmutableList<HRegionInfo> regionsFromMeta = null;
2586
2587 TableInfo(TableName name) {
2588 this.tableName = name;
2589 deployedOn = new TreeSet <ServerName>();
2590 }
2591
2592
2593
2594
2595 private HTableDescriptor getHTD() {
2596 if (htds.size() == 1) {
2597 return (HTableDescriptor)htds.toArray()[0];
2598 } else {
2599 LOG.error("None/Multiple table descriptors found for table '"
2600 + tableName + "' regions: " + htds);
2601 }
2602 return null;
2603 }
2604
2605 public void addRegionInfo(HbckInfo hir) {
2606 if (Bytes.equals(hir.getEndKey(), HConstants.EMPTY_END_ROW)) {
2607
2608
2609 if (hir.getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) sc.add(hir);
2610 return;
2611 }
2612
2613
2614 if (Bytes.compareTo(hir.getStartKey(), hir.getEndKey()) > 0) {
2615 errors.reportError(
2616 ERROR_CODE.REGION_CYCLE,
2617 String.format("The endkey for this region comes before the "
2618 + "startkey, startkey=%s, endkey=%s",
2619 Bytes.toStringBinary(hir.getStartKey()),
2620 Bytes.toStringBinary(hir.getEndKey())), this, hir);
2621 backwards.add(hir);
2622 return;
2623 }
2624
2625
2626
2627 if (hir.getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) sc.add(hir);
2628 }
2629
2630 public void addServer(ServerName server) {
2631 this.deployedOn.add(server);
2632 }
2633
2634 public TableName getName() {
2635 return tableName;
2636 }
2637
2638 public int getNumRegions() {
2639 return sc.getStarts().size() + backwards.size();
2640 }
2641
2642 public synchronized ImmutableList<HRegionInfo> getRegionsFromMeta() {
2643
2644 if (regionsFromMeta == null) {
2645 List<HRegionInfo> regions = new ArrayList<HRegionInfo>();
2646 for (HbckInfo h : HBaseFsck.this.regionInfoMap.values()) {
2647 if (tableName.equals(h.getTableName())) {
2648 if (h.metaEntry != null) {
2649 regions.add((HRegionInfo) h.metaEntry);
2650 }
2651 }
2652 }
2653 regionsFromMeta = Ordering.natural().immutableSortedCopy(regions);
2654 }
2655
2656 return regionsFromMeta;
2657 }
2658
2659
2660 private class IntegrityFixSuggester extends TableIntegrityErrorHandlerImpl {
2661 ErrorReporter errors;
2662
2663 IntegrityFixSuggester(TableInfo ti, ErrorReporter errors) {
2664 this.errors = errors;
2665 setTableInfo(ti);
2666 }
2667
2668 @Override
2669 public void handleRegionStartKeyNotEmpty(HbckInfo hi) throws IOException{
2670 errors.reportError(ERROR_CODE.FIRST_REGION_STARTKEY_NOT_EMPTY,
2671 "First region should start with an empty key. You need to "
2672 + " create a new region and regioninfo in HDFS to plug the hole.",
2673 getTableInfo(), hi);
2674 }
2675
2676 @Override
2677 public void handleRegionEndKeyNotEmpty(byte[] curEndKey) throws IOException {
2678 errors.reportError(ERROR_CODE.LAST_REGION_ENDKEY_NOT_EMPTY,
2679 "Last region should end with an empty key. You need to "
2680 + "create a new region and regioninfo in HDFS to plug the hole.", getTableInfo());
2681 }
2682
2683 @Override
2684 public void handleDegenerateRegion(HbckInfo hi) throws IOException{
2685 errors.reportError(ERROR_CODE.DEGENERATE_REGION,
2686 "Region has the same start and end key.", getTableInfo(), hi);
2687 }
2688
2689 @Override
2690 public void handleDuplicateStartKeys(HbckInfo r1, HbckInfo r2) throws IOException{
2691 byte[] key = r1.getStartKey();
2692
2693 errors.reportError(ERROR_CODE.DUPE_STARTKEYS,
2694 "Multiple regions have the same startkey: "
2695 + Bytes.toStringBinary(key), getTableInfo(), r1);
2696 errors.reportError(ERROR_CODE.DUPE_STARTKEYS,
2697 "Multiple regions have the same startkey: "
2698 + Bytes.toStringBinary(key), getTableInfo(), r2);
2699 }
2700
2701 @Override
2702 public void handleSplit(HbckInfo r1, HbckInfo r2) throws IOException{
2703 byte[] key = r1.getStartKey();
2704
2705 errors.reportError(ERROR_CODE.DUPE_ENDKEYS,
2706 "Multiple regions have the same regionID: "
2707 + Bytes.toStringBinary(key), getTableInfo(), r1);
2708 errors.reportError(ERROR_CODE.DUPE_ENDKEYS,
2709 "Multiple regions have the same regionID: "
2710 + Bytes.toStringBinary(key), getTableInfo(), r2);
2711 }
2712
2713 @Override
2714 public void handleOverlapInRegionChain(HbckInfo hi1, HbckInfo hi2) throws IOException{
2715 errors.reportError(ERROR_CODE.OVERLAP_IN_REGION_CHAIN,
2716 "There is an overlap in the region chain.",
2717 getTableInfo(), hi1, hi2);
2718 }
2719
2720 @Override
2721 public void handleHoleInRegionChain(byte[] holeStart, byte[] holeStop) throws IOException{
2722 errors.reportError(
2723 ERROR_CODE.HOLE_IN_REGION_CHAIN,
2724 "There is a hole in the region chain between "
2725 + Bytes.toStringBinary(holeStart) + " and "
2726 + Bytes.toStringBinary(holeStop)
2727 + ". You need to create a new .regioninfo and region "
2728 + "dir in hdfs to plug the hole.");
2729 }
2730 };
2731
2732
2733
2734
2735
2736
2737
2738
2739
2740
2741
2742
2743
2744 private class HDFSIntegrityFixer extends IntegrityFixSuggester {
2745 Configuration conf;
2746
2747 boolean fixOverlaps = true;
2748
2749 HDFSIntegrityFixer(TableInfo ti, ErrorReporter errors, Configuration conf,
2750 boolean fixHoles, boolean fixOverlaps) {
2751 super(ti, errors);
2752 this.conf = conf;
2753 this.fixOverlaps = fixOverlaps;
2754
2755 }
2756
2757
2758
2759
2760
2761
2762 @Override
2763 public void handleRegionStartKeyNotEmpty(HbckInfo next) throws IOException {
2764 errors.reportError(ERROR_CODE.FIRST_REGION_STARTKEY_NOT_EMPTY,
2765 "First region should start with an empty key. Creating a new " +
2766 "region and regioninfo in HDFS to plug the hole.",
2767 getTableInfo(), next);
2768 HTableDescriptor htd = getTableInfo().getHTD();
2769
2770 HRegionInfo newRegion = new HRegionInfo(htd.getTableName(),
2771 HConstants.EMPTY_START_ROW, next.getStartKey());
2772
2773
2774 HRegion region = HBaseFsckRepair.createHDFSRegionDir(conf, newRegion, htd);
2775 LOG.info("Table region start key was not empty. Created new empty region: "
2776 + newRegion + " " +region);
2777 fixes++;
2778 }
2779
2780 @Override
2781 public void handleRegionEndKeyNotEmpty(byte[] curEndKey) throws IOException {
2782 errors.reportError(ERROR_CODE.LAST_REGION_ENDKEY_NOT_EMPTY,
2783 "Last region should end with an empty key. Creating a new "
2784 + "region and regioninfo in HDFS to plug the hole.", getTableInfo());
2785 HTableDescriptor htd = getTableInfo().getHTD();
2786
2787 HRegionInfo newRegion = new HRegionInfo(htd.getTableName(), curEndKey,
2788 HConstants.EMPTY_START_ROW);
2789
2790 HRegion region = HBaseFsckRepair.createHDFSRegionDir(conf, newRegion, htd);
2791 LOG.info("Table region end key was not empty. Created new empty region: " + newRegion
2792 + " " + region);
2793 fixes++;
2794 }
2795
2796
2797
2798
2799
2800 @Override
2801 public void handleHoleInRegionChain(byte[] holeStartKey, byte[] holeStopKey) throws IOException {
2802 errors.reportError(
2803 ERROR_CODE.HOLE_IN_REGION_CHAIN,
2804 "There is a hole in the region chain between "
2805 + Bytes.toStringBinary(holeStartKey) + " and "
2806 + Bytes.toStringBinary(holeStopKey)
2807 + ". Creating a new regioninfo and region "
2808 + "dir in hdfs to plug the hole.");
2809 HTableDescriptor htd = getTableInfo().getHTD();
2810 HRegionInfo newRegion = new HRegionInfo(htd.getTableName(), holeStartKey, holeStopKey);
2811 HRegion region = HBaseFsckRepair.createHDFSRegionDir(conf, newRegion, htd);
2812 LOG.info("Plugged hole by creating new empty region: "+ newRegion + " " +region);
2813 fixes++;
2814 }
2815
2816
2817
2818
2819
2820
2821
2822
2823
2824
2825
2826
2827 @Override
2828 public void handleOverlapGroup(Collection<HbckInfo> overlap)
2829 throws IOException {
2830 Preconditions.checkNotNull(overlap);
2831 Preconditions.checkArgument(overlap.size() >0);
2832
2833 if (!this.fixOverlaps) {
2834 LOG.warn("Not attempting to repair overlaps.");
2835 return;
2836 }
2837
2838 if (overlap.size() > maxMerge) {
2839 LOG.warn("Overlap group has " + overlap.size() + " overlapping " +
2840 "regions which is greater than " + maxMerge + ", the max number of regions to merge");
2841 if (sidelineBigOverlaps) {
2842
2843 sidelineBigOverlaps(overlap);
2844 }
2845 return;
2846 }
2847 if (shouldRemoveParents()) {
2848 removeParentsAndFixSplits(overlap);
2849 }
2850 mergeOverlaps(overlap);
2851 }
2852
2853 void removeParentsAndFixSplits(Collection<HbckInfo> overlap) throws IOException {
2854 Pair<byte[], byte[]> range = null;
2855 HbckInfo parent = null;
2856 HbckInfo daughterA = null;
2857 HbckInfo daughterB = null;
2858 Collection<HbckInfo> daughters = new ArrayList<HbckInfo>(overlap);
2859
2860 String thread = Thread.currentThread().getName();
2861 LOG.info("== [" + thread + "] Attempting fix splits in overlap state.");
2862
2863
2864 if (overlap.size() > 3) {
2865 LOG.info("Too many overlaps were found on this group, falling back to regular merge.");
2866 return;
2867 }
2868
2869 for (HbckInfo hi : overlap) {
2870 if (range == null) {
2871 range = new Pair<byte[], byte[]>(hi.getStartKey(), hi.getEndKey());
2872 } else {
2873 if (RegionSplitCalculator.BYTES_COMPARATOR
2874 .compare(hi.getStartKey(), range.getFirst()) < 0) {
2875 range.setFirst(hi.getStartKey());
2876 }
2877 if (RegionSplitCalculator.BYTES_COMPARATOR
2878 .compare(hi.getEndKey(), range.getSecond()) > 0) {
2879 range.setSecond(hi.getEndKey());
2880 }
2881 }
2882 }
2883
2884 LOG.info("This group range is [" + Bytes.toStringBinary(range.getFirst()) + ", "
2885 + Bytes.toStringBinary(range.getSecond()) + "]");
2886
2887
2888 for (HbckInfo hi : overlap) {
2889 if (Bytes.compareTo(hi.getHdfsHRI().getStartKey(), range.getFirst()) == 0
2890 && Bytes.compareTo(hi.getHdfsHRI().getEndKey(), range.getSecond()) == 0) {
2891 LOG.info("This is a parent for this group: " + hi.toString());
2892 parent = hi;
2893 }
2894 }
2895
2896
2897 if (parent != null) {
2898 daughters.remove(parent);
2899 }
2900
2901
2902
2903 for (HbckInfo hi : daughters) {
2904 if (Bytes.compareTo(hi.getHdfsHRI().getStartKey(), range.getFirst()) == 0) {
2905 if (parent.getHdfsHRI().getRegionId() < hi.getHdfsHRI().getRegionId()) {
2906 daughterA = hi;
2907 }
2908 }
2909 if (Bytes.compareTo(hi.getHdfsHRI().getEndKey(), range.getSecond()) == 0) {
2910 if (parent.getHdfsHRI().getRegionId() < hi.getHdfsHRI().getRegionId()) {
2911 daughterB = hi;
2912 }
2913 }
2914 }
2915
2916
2917 if (daughterA.getHdfsHRI().getRegionId() != daughterB.getHdfsHRI().getRegionId() || parent == null)
2918 return;
2919
2920 FileSystem fs = FileSystem.get(conf);
2921 LOG.info("Found parent: " + parent.getRegionNameAsString());
2922 LOG.info("Found potential daughter a: " + daughterA.getRegionNameAsString());
2923 LOG.info("Found potential daughter b: " + daughterB.getRegionNameAsString());
2924 LOG.info("Trying to fix parent in overlap by removing the parent.");
2925 try {
2926 closeRegion(parent);
2927 } catch (IOException ioe) {
2928 LOG.warn("Parent region could not be closed, continuing with regular merge...", ioe);
2929 return;
2930 } catch (InterruptedException ie) {
2931 LOG.warn("Parent region could not be closed, continuing with regular merge...", ie);
2932 return;
2933 }
2934
2935 try {
2936 offline(parent.getRegionName());
2937 } catch (IOException ioe) {
2938 LOG.warn("Unable to offline parent region: " + parent.getRegionNameAsString()
2939 + ". Just continuing with regular merge... ", ioe);
2940 return;
2941 }
2942
2943 try {
2944 HBaseFsckRepair.removeParentInMeta(conf, parent.getHdfsHRI());
2945 } catch (IOException ioe) {
2946 LOG.warn("Unable to remove parent region in META: " + parent.getRegionNameAsString()
2947 + ". Just continuing with regular merge... ", ioe);
2948 return;
2949 }
2950
2951 sidelineRegionDir(fs, parent);
2952 LOG.info("[" + thread + "] Sidelined parent region dir "+ parent.getHdfsRegionDir() + " into " +
2953 getSidelineDir());
2954 debugLsr(parent.getHdfsRegionDir());
2955
2956
2957 overlap.remove(parent);
2958 overlap.remove(daughterA);
2959 overlap.remove(daughterB);
2960
2961 LOG.info("Done fixing split.");
2962
2963 }
2964
2965 void mergeOverlaps(Collection<HbckInfo> overlap)
2966 throws IOException {
2967 String thread = Thread.currentThread().getName();
2968 LOG.info("== [" + thread + "] Merging regions into one region: "
2969 + Joiner.on(",").join(overlap));
2970
2971 Pair<byte[], byte[]> range = null;
2972 for (HbckInfo hi : overlap) {
2973 if (range == null) {
2974 range = new Pair<byte[], byte[]>(hi.getStartKey(), hi.getEndKey());
2975 } else {
2976 if (RegionSplitCalculator.BYTES_COMPARATOR
2977 .compare(hi.getStartKey(), range.getFirst()) < 0) {
2978 range.setFirst(hi.getStartKey());
2979 }
2980 if (RegionSplitCalculator.BYTES_COMPARATOR
2981 .compare(hi.getEndKey(), range.getSecond()) > 0) {
2982 range.setSecond(hi.getEndKey());
2983 }
2984 }
2985
2986 LOG.debug("[" + thread + "] Closing region before moving data around: " + hi);
2987 LOG.debug("[" + thread + "] Contained region dir before close");
2988 debugLsr(hi.getHdfsRegionDir());
2989 try {
2990 LOG.info("[" + thread + "] Closing region: " + hi);
2991 closeRegion(hi);
2992 } catch (IOException ioe) {
2993 LOG.warn("[" + thread + "] Was unable to close region " + hi
2994 + ". Just continuing... ", ioe);
2995 } catch (InterruptedException e) {
2996 LOG.warn("[" + thread + "] Was unable to close region " + hi
2997 + ". Just continuing... ", e);
2998 }
2999
3000 try {
3001 LOG.info("[" + thread + "] Offlining region: " + hi);
3002 offline(hi.getRegionName());
3003 } catch (IOException ioe) {
3004 LOG.warn("[" + thread + "] Unable to offline region from master: " + hi
3005 + ". Just continuing... ", ioe);
3006 }
3007 }
3008
3009
3010 HTableDescriptor htd = getTableInfo().getHTD();
3011
3012 HRegionInfo newRegion = new HRegionInfo(htd.getTableName(), range.getFirst(),
3013 range.getSecond());
3014 HRegion region = HBaseFsckRepair.createHDFSRegionDir(conf, newRegion, htd);
3015 LOG.info("[" + thread + "] Created new empty container region: " +
3016 newRegion + " to contain regions: " + Joiner.on(",").join(overlap));
3017 debugLsr(region.getRegionFileSystem().getRegionDir());
3018
3019
3020 boolean didFix= false;
3021 Path target = region.getRegionFileSystem().getRegionDir();
3022 for (HbckInfo contained : overlap) {
3023 LOG.info("[" + thread + "] Merging " + contained + " into " + target );
3024 int merges = mergeRegionDirs(target, contained);
3025 if (merges > 0) {
3026 didFix = true;
3027 }
3028 }
3029 if (didFix) {
3030 fixes++;
3031 }
3032 }
3033
3034
3035
3036
3037
3038
3039
3040
3041 void sidelineBigOverlaps(
3042 Collection<HbckInfo> bigOverlap) throws IOException {
3043 int overlapsToSideline = bigOverlap.size() - maxMerge;
3044 if (overlapsToSideline > maxOverlapsToSideline) {
3045 overlapsToSideline = maxOverlapsToSideline;
3046 }
3047 List<HbckInfo> regionsToSideline =
3048 RegionSplitCalculator.findBigRanges(bigOverlap, overlapsToSideline);
3049 FileSystem fs = FileSystem.get(conf);
3050 for (HbckInfo regionToSideline: regionsToSideline) {
3051 try {
3052 LOG.info("Closing region: " + regionToSideline);
3053 closeRegion(regionToSideline);
3054 } catch (IOException ioe) {
3055 LOG.warn("Was unable to close region " + regionToSideline
3056 + ". Just continuing... ", ioe);
3057 } catch (InterruptedException e) {
3058 LOG.warn("Was unable to close region " + regionToSideline
3059 + ". Just continuing... ", e);
3060 }
3061
3062 try {
3063 LOG.info("Offlining region: " + regionToSideline);
3064 offline(regionToSideline.getRegionName());
3065 } catch (IOException ioe) {
3066 LOG.warn("Unable to offline region from master: " + regionToSideline
3067 + ". Just continuing... ", ioe);
3068 }
3069
3070 LOG.info("Before sideline big overlapped region: " + regionToSideline.toString());
3071 Path sidelineRegionDir = sidelineRegionDir(fs, TO_BE_LOADED, regionToSideline);
3072 if (sidelineRegionDir != null) {
3073 sidelinedRegions.put(sidelineRegionDir, regionToSideline);
3074 LOG.info("After sidelined big overlapped region: "
3075 + regionToSideline.getRegionNameAsString()
3076 + " to " + sidelineRegionDir.toString());
3077 fixes++;
3078 }
3079 }
3080 }
3081 }
3082
3083
3084
3085
3086
3087
3088
3089 public boolean checkRegionChain(TableIntegrityErrorHandler handler) throws IOException {
3090
3091
3092
3093 if (disabledTables.contains(this.tableName)) {
3094 return true;
3095 }
3096 int originalErrorsCount = errors.getErrorList().size();
3097 Multimap<byte[], HbckInfo> regions = sc.calcCoverage();
3098 SortedSet<byte[]> splits = sc.getSplits();
3099
3100 byte[] prevKey = null;
3101 byte[] problemKey = null;
3102
3103 if (splits.size() == 0) {
3104
3105 handler.handleHoleInRegionChain(HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
3106 }
3107
3108 for (byte[] key : splits) {
3109 Collection<HbckInfo> ranges = regions.get(key);
3110 if (prevKey == null && !Bytes.equals(key, HConstants.EMPTY_BYTE_ARRAY)) {
3111 for (HbckInfo rng : ranges) {
3112 handler.handleRegionStartKeyNotEmpty(rng);
3113 }
3114 }
3115
3116
3117 for (HbckInfo rng : ranges) {
3118
3119 byte[] endKey = rng.getEndKey();
3120 endKey = (endKey.length == 0) ? null : endKey;
3121 if (Bytes.equals(rng.getStartKey(),endKey)) {
3122 handler.handleDegenerateRegion(rng);
3123 }
3124 }
3125
3126 if (ranges.size() == 1) {
3127
3128 if (problemKey != null) {
3129 LOG.warn("reached end of problem group: " + Bytes.toStringBinary(key));
3130 }
3131 problemKey = null;
3132 } else if (ranges.size() > 1) {
3133
3134
3135 if (problemKey == null) {
3136
3137 LOG.warn("Naming new problem group: " + Bytes.toStringBinary(key));
3138 problemKey = key;
3139 }
3140 overlapGroups.putAll(problemKey, ranges);
3141
3142
3143 ArrayList<HbckInfo> subRange = new ArrayList<HbckInfo>(ranges);
3144
3145 for (HbckInfo r1 : ranges) {
3146 if (r1.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) continue;
3147 subRange.remove(r1);
3148 for (HbckInfo r2 : subRange) {
3149 if (r2.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) continue;
3150
3151 if (Bytes.compareTo(r1.getStartKey(), r2.getStartKey())==0) {
3152 handler.handleDuplicateStartKeys(r1,r2);
3153 } else if (Bytes.compareTo(r1.getEndKey(), r2.getStartKey())==0 &&
3154 r1.getHdfsHRI().getRegionId() == r2.getHdfsHRI().getRegionId()) {
3155 LOG.info("this is a split, log to splits");
3156 handler.handleSplit(r1, r2);
3157 } else {
3158
3159 handler.handleOverlapInRegionChain(r1, r2);
3160 }
3161 }
3162 }
3163
3164 } else if (ranges.size() == 0) {
3165 if (problemKey != null) {
3166 LOG.warn("reached end of problem group: " + Bytes.toStringBinary(key));
3167 }
3168 problemKey = null;
3169
3170 byte[] holeStopKey = sc.getSplits().higher(key);
3171
3172 if (holeStopKey != null) {
3173
3174 handler.handleHoleInRegionChain(key, holeStopKey);
3175 }
3176 }
3177 prevKey = key;
3178 }
3179
3180
3181
3182 if (prevKey != null) {
3183 handler.handleRegionEndKeyNotEmpty(prevKey);
3184 }
3185
3186
3187 if (getConf().getBoolean("hbasefsck.overlap.merge.parallel", true)) {
3188 boolean ok = handleOverlapsParallel(handler, prevKey);
3189 if (!ok) {
3190 return false;
3191 }
3192 } else {
3193 for (Collection<HbckInfo> overlap : overlapGroups.asMap().values()) {
3194 handler.handleOverlapGroup(overlap);
3195 }
3196 }
3197
3198 if (details) {
3199
3200 errors.print("---- Table '" + this.tableName
3201 + "': region split map");
3202 dump(splits, regions);
3203 errors.print("---- Table '" + this.tableName
3204 + "': overlap groups");
3205 dumpOverlapProblems(overlapGroups);
3206 errors.print("There are " + overlapGroups.keySet().size()
3207 + " overlap groups with " + overlapGroups.size()
3208 + " overlapping regions");
3209 }
3210 if (!sidelinedRegions.isEmpty()) {
3211 LOG.warn("Sidelined big overlapped regions, please bulk load them!");
3212 errors.print("---- Table '" + this.tableName
3213 + "': sidelined big overlapped regions");
3214 dumpSidelinedRegions(sidelinedRegions);
3215 }
3216 return errors.getErrorList().size() == originalErrorsCount;
3217 }
3218
3219 private boolean handleOverlapsParallel(TableIntegrityErrorHandler handler, byte[] prevKey)
3220 throws IOException {
3221
3222
3223 List<WorkItemOverlapMerge> merges = new ArrayList<WorkItemOverlapMerge>(overlapGroups.size());
3224 List<Future<Void>> rets;
3225 for (Collection<HbckInfo> overlap : overlapGroups.asMap().values()) {
3226
3227 merges.add(new WorkItemOverlapMerge(overlap, handler));
3228 }
3229 try {
3230 rets = executor.invokeAll(merges);
3231 } catch (InterruptedException e) {
3232 LOG.error("Overlap merges were interrupted", e);
3233 return false;
3234 }
3235 for(int i=0; i<merges.size(); i++) {
3236 WorkItemOverlapMerge work = merges.get(i);
3237 Future<Void> f = rets.get(i);
3238 try {
3239 f.get();
3240 } catch(ExecutionException e) {
3241 LOG.warn("Failed to merge overlap group" + work, e.getCause());
3242 } catch (InterruptedException e) {
3243 LOG.error("Waiting for overlap merges was interrupted", e);
3244 return false;
3245 }
3246 }
3247 return true;
3248 }
3249
3250
3251
3252
3253
3254
3255
3256 void dump(SortedSet<byte[]> splits, Multimap<byte[], HbckInfo> regions) {
3257
3258 StringBuilder sb = new StringBuilder();
3259 for (byte[] k : splits) {
3260 sb.setLength(0);
3261 sb.append(Bytes.toStringBinary(k) + ":\t");
3262 for (HbckInfo r : regions.get(k)) {
3263 sb.append("[ "+ r.toString() + ", "
3264 + Bytes.toStringBinary(r.getEndKey())+ "]\t");
3265 }
3266 errors.print(sb.toString());
3267 }
3268 }
3269 }
3270
3271 public void dumpOverlapProblems(Multimap<byte[], HbckInfo> regions) {
3272
3273
3274 for (byte[] k : regions.keySet()) {
3275 errors.print(Bytes.toStringBinary(k) + ":");
3276 for (HbckInfo r : regions.get(k)) {
3277 errors.print("[ " + r.toString() + ", "
3278 + Bytes.toStringBinary(r.getEndKey()) + "]");
3279 }
3280 errors.print("----");
3281 }
3282 }
3283
3284 public void dumpSidelinedRegions(Map<Path, HbckInfo> regions) {
3285 for (Map.Entry<Path, HbckInfo> entry: regions.entrySet()) {
3286 TableName tableName = entry.getValue().getTableName();
3287 Path path = entry.getKey();
3288 errors.print("This sidelined region dir should be bulk loaded: "
3289 + path.toString());
3290 errors.print("Bulk load command looks like: "
3291 + "hbase org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles "
3292 + path.toUri().getPath() + " "+ tableName);
3293 }
3294 }
3295
3296 public Multimap<byte[], HbckInfo> getOverlapGroups(
3297 TableName table) {
3298 TableInfo ti = tablesInfo.get(table);
3299 return ti.overlapGroups;
3300 }
3301
3302
3303
3304
3305
3306
3307
3308
3309
3310
3311 HTableDescriptor[] getTables(AtomicInteger numSkipped) {
3312 List<TableName> tableNames = new ArrayList<TableName>();
3313 long now = EnvironmentEdgeManager.currentTime();
3314
3315 for (HbckInfo hbi : regionInfoMap.values()) {
3316 MetaEntry info = hbi.metaEntry;
3317
3318
3319
3320 if (info != null && info.getStartKey().length == 0 && !info.isMetaRegion()) {
3321 if (info.modTime + timelag < now) {
3322 tableNames.add(info.getTable());
3323 } else {
3324 numSkipped.incrementAndGet();
3325 }
3326 }
3327 }
3328 return getHTableDescriptors(tableNames);
3329 }
3330
3331 HTableDescriptor[] getHTableDescriptors(List<TableName> tableNames) {
3332 HTableDescriptor[] htd = new HTableDescriptor[0];
3333 Admin admin = null;
3334 try {
3335 LOG.info("getHTableDescriptors == tableNames => " + tableNames);
3336 admin = new HBaseAdmin(getConf());
3337 htd = admin.getTableDescriptorsByTableName(tableNames);
3338 } catch (IOException e) {
3339 LOG.debug("Exception getting table descriptors", e);
3340 } finally {
3341 if (admin != null) {
3342 try {
3343 admin.close();
3344 } catch (IOException e) {
3345 LOG.debug("Exception closing HBaseAdmin", e);
3346 }
3347 }
3348 }
3349 return htd;
3350 }
3351
3352
3353
3354
3355
3356
3357 private synchronized HbckInfo getOrCreateInfo(String name) {
3358 HbckInfo hbi = regionInfoMap.get(name);
3359 if (hbi == null) {
3360 hbi = new HbckInfo(null);
3361 regionInfoMap.put(name, hbi);
3362 }
3363 return hbi;
3364 }
3365
3366 private void checkAndFixTableLocks() throws IOException {
3367 ZooKeeperWatcher zkw = createZooKeeperWatcher();
3368
3369 try {
3370 TableLockChecker checker = new TableLockChecker(zkw, errors);
3371 checker.checkTableLocks();
3372
3373 if (this.fixTableLocks) {
3374 checker.fixExpiredTableLocks();
3375 }
3376 } finally {
3377 zkw.close();
3378 }
3379 }
3380
3381
3382
3383
3384
3385
3386
3387 private void checkAndFixOrphanedTableZNodes()
3388 throws IOException, KeeperException, InterruptedException {
3389 ZooKeeperWatcher zkw = createZooKeeperWatcher();
3390
3391 try {
3392 Set<TableName> enablingTables = ZKTableStateClientSideReader.getEnablingTables(zkw);
3393 String msg;
3394 TableInfo tableInfo;
3395
3396 for (TableName tableName : enablingTables) {
3397
3398 tableInfo = tablesInfo.get(tableName);
3399 if (tableInfo != null) {
3400
3401 continue;
3402 }
3403
3404 msg = "Table " + tableName + " not found in hbase:meta. Orphaned table ZNode found.";
3405 LOG.warn(msg);
3406 orphanedTableZNodes.add(tableName);
3407 errors.reportError(ERROR_CODE.ORPHANED_ZK_TABLE_ENTRY, msg);
3408 }
3409
3410 if (orphanedTableZNodes.size() > 0 && this.fixTableZNodes) {
3411 ZKTableStateManager zkTableStateMgr = new ZKTableStateManager(zkw);
3412
3413 for (TableName tableName : orphanedTableZNodes) {
3414 try {
3415
3416
3417
3418
3419 zkTableStateMgr.setTableState(tableName, ZooKeeperProtos.Table.State.DISABLED);
3420 } catch (CoordinatedStateException e) {
3421
3422 LOG.error(
3423 "Got a CoordinatedStateException while fixing the ENABLING table znode " + tableName,
3424 e);
3425 }
3426 }
3427 }
3428 } finally {
3429 zkw.close();
3430 }
3431 }
3432
3433
3434
3435
3436
3437
3438
3439
3440
3441
3442 boolean checkMetaRegion() throws IOException, KeeperException, InterruptedException {
3443 Map<Integer, HbckInfo> metaRegions = new HashMap<Integer, HbckInfo>();
3444 for (HbckInfo value : regionInfoMap.values()) {
3445 if (value.metaEntry != null && value.metaEntry.isMetaRegion()) {
3446 metaRegions.put(value.getReplicaId(), value);
3447 }
3448 }
3449 int metaReplication = admin.getTableDescriptor(TableName.META_TABLE_NAME)
3450 .getRegionReplication();
3451 boolean noProblem = true;
3452
3453
3454 for (int i = 0; i < metaReplication; i++) {
3455 HbckInfo metaHbckInfo = metaRegions.remove(i);
3456 List<ServerName> servers = new ArrayList<ServerName>();
3457 if (metaHbckInfo != null) {
3458 servers = metaHbckInfo.deployedOn;
3459 }
3460 if (servers.size() != 1) {
3461 noProblem = false;
3462 if (servers.size() == 0) {
3463 assignMetaReplica(i);
3464 } else if (servers.size() > 1) {
3465 errors
3466 .reportError(ERROR_CODE.MULTI_META_REGION, "hbase:meta, replicaId " +
3467 metaHbckInfo.getReplicaId() + " is found on more than one region.");
3468 if (shouldFixAssignments()) {
3469 errors.print("Trying to fix a problem with hbase:meta, replicaId " +
3470 metaHbckInfo.getReplicaId() +"..");
3471 setShouldRerun();
3472
3473 HBaseFsckRepair.fixMultiAssignment(connection, metaHbckInfo.metaEntry, servers);
3474 }
3475 }
3476 }
3477 }
3478
3479 for (Map.Entry<Integer, HbckInfo> entry : metaRegions.entrySet()) {
3480 noProblem = false;
3481 errors.reportError(ERROR_CODE.SHOULD_NOT_BE_DEPLOYED,
3482 "hbase:meta replicas are deployed in excess. Configured " + metaReplication +
3483 ", deployed " + metaRegions.size());
3484 if (shouldFixAssignments()) {
3485 errors.print("Trying to undeploy excess replica, replicaId: " + entry.getKey() +
3486 " of hbase:meta..");
3487 setShouldRerun();
3488 unassignMetaReplica(entry.getValue());
3489 }
3490 }
3491
3492
3493 return noProblem;
3494 }
3495
3496 private void unassignMetaReplica(HbckInfo hi) throws IOException, InterruptedException,
3497 KeeperException {
3498 undeployRegions(hi);
3499 ZooKeeperWatcher zkw = createZooKeeperWatcher();
3500 try {
3501 ZKUtil.deleteNode(zkw, zkw.getZNodeForReplica(hi.metaEntry.getReplicaId()));
3502 } finally {
3503 zkw.close();
3504 }
3505 }
3506
3507 private void assignMetaReplica(int replicaId)
3508 throws IOException, KeeperException, InterruptedException {
3509 errors.reportError(ERROR_CODE.NO_META_REGION, "hbase:meta, replicaId " +
3510 replicaId +" is not found on any region.");
3511 if (shouldFixAssignments()) {
3512 errors.print("Trying to fix a problem with hbase:meta..");
3513 setShouldRerun();
3514
3515 HRegionInfo h = RegionReplicaUtil.getRegionInfoForReplica(
3516 HRegionInfo.FIRST_META_REGIONINFO, replicaId);
3517 HBaseFsckRepair.fixUnassigned(admin, h);
3518 HBaseFsckRepair.waitUntilAssigned(admin, h);
3519 }
3520 }
3521
3522
3523
3524
3525
3526 boolean loadMetaEntries() throws IOException {
3527 MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
3528 int countRecord = 1;
3529
3530
3531 final Comparator<Cell> comp = new Comparator<Cell>() {
3532 @Override
3533 public int compare(Cell k1, Cell k2) {
3534 return Long.compare(k1.getTimestamp(), k2.getTimestamp());
3535 }
3536 };
3537
3538 @Override
3539 public boolean processRow(Result result) throws IOException {
3540 try {
3541
3542
3543 long ts = Collections.max(result.listCells(), comp).getTimestamp();
3544 RegionLocations rl = MetaTableAccessor.getRegionLocations(result);
3545 if (rl == null) {
3546 emptyRegionInfoQualifiers.add(result);
3547 errors.reportError(ERROR_CODE.EMPTY_META_CELL,
3548 "Empty REGIONINFO_QUALIFIER found in hbase:meta");
3549 return true;
3550 }
3551 ServerName sn = null;
3552 if (rl.getRegionLocation(HRegionInfo.DEFAULT_REPLICA_ID) == null ||
3553 rl.getRegionLocation(HRegionInfo.DEFAULT_REPLICA_ID).getRegionInfo() == null) {
3554 emptyRegionInfoQualifiers.add(result);
3555 errors.reportError(ERROR_CODE.EMPTY_META_CELL,
3556 "Empty REGIONINFO_QUALIFIER found in hbase:meta");
3557 return true;
3558 }
3559 HRegionInfo hri = rl.getRegionLocation(HRegionInfo.DEFAULT_REPLICA_ID).getRegionInfo();
3560 if (!(isTableIncluded(hri.getTable())
3561 || hri.isMetaRegion())) {
3562 return true;
3563 }
3564 PairOfSameType<HRegionInfo> daughters = HRegionInfo.getDaughterRegions(result);
3565 for (HRegionLocation h : rl.getRegionLocations()) {
3566 if (h == null || h.getRegionInfo() == null) {
3567 continue;
3568 }
3569 sn = h.getServerName();
3570 hri = h.getRegionInfo();
3571
3572 MetaEntry m = null;
3573 if (hri.getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
3574 m = new MetaEntry(hri, sn, ts, daughters.getFirst(), daughters.getSecond());
3575 } else {
3576 m = new MetaEntry(hri, sn, ts, null, null);
3577 }
3578 HbckInfo previous = regionInfoMap.get(hri.getEncodedName());
3579 if (previous == null) {
3580 regionInfoMap.put(hri.getEncodedName(), new HbckInfo(m));
3581 } else if (previous.metaEntry == null) {
3582 previous.metaEntry = m;
3583 } else {
3584 throw new IOException("Two entries in hbase:meta are same " + previous);
3585 }
3586 }
3587 PairOfSameType<HRegionInfo> mergeRegions = HRegionInfo.getMergeRegions(result);
3588 for (HRegionInfo mergeRegion : new HRegionInfo[] {
3589 mergeRegions.getFirst(), mergeRegions.getSecond() }) {
3590 if (mergeRegion != null) {
3591
3592 HbckInfo hbInfo = getOrCreateInfo(mergeRegion.getEncodedName());
3593 hbInfo.setMerged(true);
3594 }
3595 }
3596
3597
3598 if (countRecord % 100 == 0) {
3599 errors.progress();
3600 }
3601 countRecord++;
3602 return true;
3603 } catch (RuntimeException e) {
3604 LOG.error("Result=" + result);
3605 throw e;
3606 }
3607 }
3608 };
3609 if (!checkMetaOnly) {
3610
3611 MetaScanner.metaScan(connection, visitor);
3612 }
3613
3614 errors.print("");
3615 return true;
3616 }
3617
3618
3619
3620
3621 static class MetaEntry extends HRegionInfo {
3622 ServerName regionServer;
3623 long modTime;
3624 HRegionInfo splitA, splitB;
3625
3626 public MetaEntry(HRegionInfo rinfo, ServerName regionServer, long modTime) {
3627 this(rinfo, regionServer, modTime, null, null);
3628 }
3629
3630 public MetaEntry(HRegionInfo rinfo, ServerName regionServer, long modTime,
3631 HRegionInfo splitA, HRegionInfo splitB) {
3632 super(rinfo);
3633 this.regionServer = regionServer;
3634 this.modTime = modTime;
3635 this.splitA = splitA;
3636 this.splitB = splitB;
3637 }
3638
3639 @Override
3640 public boolean equals(Object o) {
3641 boolean superEq = super.equals(o);
3642 if (!superEq) {
3643 return superEq;
3644 }
3645
3646 MetaEntry me = (MetaEntry) o;
3647 if (!regionServer.equals(me.regionServer)) {
3648 return false;
3649 }
3650 return (modTime == me.modTime);
3651 }
3652
3653 @Override
3654 public int hashCode() {
3655 int hash = Arrays.hashCode(getRegionName());
3656 hash ^= getRegionId();
3657 hash ^= Arrays.hashCode(getStartKey());
3658 hash ^= Arrays.hashCode(getEndKey());
3659 hash ^= Boolean.valueOf(isOffline()).hashCode();
3660 hash ^= getTable().hashCode();
3661 if (regionServer != null) {
3662 hash ^= regionServer.hashCode();
3663 }
3664 hash ^= modTime;
3665 return hash;
3666 }
3667 }
3668
3669
3670
3671
3672 static class HdfsEntry {
3673 HRegionInfo hri;
3674 Path hdfsRegionDir = null;
3675 long hdfsRegionDirModTime = 0;
3676 boolean hdfsRegioninfoFilePresent = false;
3677 boolean hdfsOnlyEdits = false;
3678 }
3679
3680
3681
3682
3683 static class OnlineEntry {
3684 HRegionInfo hri;
3685 ServerName hsa;
3686
3687 @Override
3688 public String toString() {
3689 return hsa.toString() + ";" + hri.getRegionNameAsString();
3690 }
3691 }
3692
3693
3694
3695
3696
3697 public static class HbckInfo implements KeyRange {
3698 private MetaEntry metaEntry = null;
3699 private HdfsEntry hdfsEntry = null;
3700 private List<OnlineEntry> deployedEntries = Lists.newArrayList();
3701 private List<ServerName> deployedOn = Lists.newArrayList();
3702 private boolean skipChecks = false;
3703 private boolean isMerged = false;
3704 private int deployedReplicaId = HRegionInfo.DEFAULT_REPLICA_ID;
3705 private HRegionInfo primaryHRIForDeployedReplica = null;
3706
3707 HbckInfo(MetaEntry metaEntry) {
3708 this.metaEntry = metaEntry;
3709 }
3710
3711 public synchronized int getReplicaId() {
3712 return metaEntry != null? metaEntry.getReplicaId(): deployedReplicaId;
3713 }
3714
3715 public synchronized void addServer(HRegionInfo hri, ServerName server) {
3716 OnlineEntry rse = new OnlineEntry() ;
3717 rse.hri = hri;
3718 rse.hsa = server;
3719 this.deployedEntries.add(rse);
3720 this.deployedOn.add(server);
3721
3722 this.deployedReplicaId = hri.getReplicaId();
3723 this.primaryHRIForDeployedReplica =
3724 RegionReplicaUtil.getRegionInfoForDefaultReplica(hri);
3725 }
3726
3727 @Override
3728 public synchronized String toString() {
3729 StringBuilder sb = new StringBuilder();
3730 sb.append("{ meta => ");
3731 sb.append((metaEntry != null)? metaEntry.getRegionNameAsString() : "null");
3732 sb.append( ", hdfs => " + getHdfsRegionDir());
3733 sb.append( ", deployed => " + Joiner.on(", ").join(deployedEntries));
3734 sb.append( ", replicaId => " + getReplicaId());
3735 sb.append(" }");
3736 return sb.toString();
3737 }
3738
3739 @Override
3740 public byte[] getStartKey() {
3741 if (this.metaEntry != null) {
3742 return this.metaEntry.getStartKey();
3743 } else if (this.hdfsEntry != null) {
3744 return this.hdfsEntry.hri.getStartKey();
3745 } else {
3746 LOG.error("Entry " + this + " has no meta or hdfs region start key.");
3747 return null;
3748 }
3749 }
3750
3751 @Override
3752 public byte[] getEndKey() {
3753 if (this.metaEntry != null) {
3754 return this.metaEntry.getEndKey();
3755 } else if (this.hdfsEntry != null) {
3756 return this.hdfsEntry.hri.getEndKey();
3757 } else {
3758 LOG.error("Entry " + this + " has no meta or hdfs region start key.");
3759 return null;
3760 }
3761 }
3762
3763 public TableName getTableName() {
3764 if (this.metaEntry != null) {
3765 return this.metaEntry.getTable();
3766 } else if (this.hdfsEntry != null) {
3767
3768
3769 Path tableDir = this.hdfsEntry.hdfsRegionDir.getParent();
3770 return FSUtils.getTableName(tableDir);
3771 } else {
3772
3773 for (OnlineEntry e : deployedEntries) {
3774 return e.hri.getTable();
3775 }
3776 return null;
3777 }
3778 }
3779
3780 public String getRegionNameAsString() {
3781 if (metaEntry != null) {
3782 return metaEntry.getRegionNameAsString();
3783 } else if (hdfsEntry != null) {
3784 if (hdfsEntry.hri != null) {
3785 return hdfsEntry.hri.getRegionNameAsString();
3786 }
3787 } else {
3788
3789 for (OnlineEntry e : deployedEntries) {
3790 return e.hri.getRegionNameAsString();
3791 }
3792 }
3793 return null;
3794 }
3795
3796 public byte[] getRegionName() {
3797 if (metaEntry != null) {
3798 return metaEntry.getRegionName();
3799 } else if (hdfsEntry != null) {
3800 return hdfsEntry.hri.getRegionName();
3801 } else {
3802
3803 for (OnlineEntry e : deployedEntries) {
3804 return e.hri.getRegionName();
3805 }
3806 return null;
3807 }
3808 }
3809
3810 public HRegionInfo getPrimaryHRIForDeployedReplica() {
3811 return primaryHRIForDeployedReplica;
3812 }
3813
3814 Path getHdfsRegionDir() {
3815 if (hdfsEntry == null) {
3816 return null;
3817 }
3818 return hdfsEntry.hdfsRegionDir;
3819 }
3820
3821 boolean containsOnlyHdfsEdits() {
3822 if (hdfsEntry == null) {
3823 return false;
3824 }
3825 return hdfsEntry.hdfsOnlyEdits;
3826 }
3827
3828 boolean isHdfsRegioninfoPresent() {
3829 if (hdfsEntry == null) {
3830 return false;
3831 }
3832 return hdfsEntry.hdfsRegioninfoFilePresent;
3833 }
3834
3835 long getModTime() {
3836 if (hdfsEntry == null) {
3837 return 0;
3838 }
3839 return hdfsEntry.hdfsRegionDirModTime;
3840 }
3841
3842 HRegionInfo getHdfsHRI() {
3843 if (hdfsEntry == null) {
3844 return null;
3845 }
3846 return hdfsEntry.hri;
3847 }
3848
3849 public void setSkipChecks(boolean skipChecks) {
3850 this.skipChecks = skipChecks;
3851 }
3852
3853 public boolean isSkipChecks() {
3854 return skipChecks;
3855 }
3856
3857 public void setMerged(boolean isMerged) {
3858 this.isMerged = isMerged;
3859 }
3860
3861 public boolean isMerged() {
3862 return this.isMerged;
3863 }
3864 }
3865
3866 final static Comparator<HbckInfo> cmp = new Comparator<HbckInfo>() {
3867 @Override
3868 public int compare(HbckInfo l, HbckInfo r) {
3869 if (l == r) {
3870
3871 return 0;
3872 }
3873
3874 int tableCompare = l.getTableName().compareTo(r.getTableName());
3875 if (tableCompare != 0) {
3876 return tableCompare;
3877 }
3878
3879 int startComparison = RegionSplitCalculator.BYTES_COMPARATOR.compare(
3880 l.getStartKey(), r.getStartKey());
3881 if (startComparison != 0) {
3882 return startComparison;
3883 }
3884
3885
3886 byte[] endKey = r.getEndKey();
3887 endKey = (endKey.length == 0) ? null : endKey;
3888 byte[] endKey2 = l.getEndKey();
3889 endKey2 = (endKey2.length == 0) ? null : endKey2;
3890 int endComparison = RegionSplitCalculator.BYTES_COMPARATOR.compare(
3891 endKey2, endKey);
3892
3893 if (endComparison != 0) {
3894 return endComparison;
3895 }
3896
3897
3898
3899 if (l.hdfsEntry == null && r.hdfsEntry == null) {
3900 return 0;
3901 }
3902 if (l.hdfsEntry == null && r.hdfsEntry != null) {
3903 return 1;
3904 }
3905
3906 if (r.hdfsEntry == null) {
3907 return -1;
3908 }
3909
3910 return (int) (l.hdfsEntry.hri.getRegionId()- r.hdfsEntry.hri.getRegionId());
3911 }
3912 };
3913
3914
3915
3916
3917 private void printTableSummary(SortedMap<TableName, TableInfo> tablesInfo) {
3918 StringBuilder sb = new StringBuilder();
3919 int numOfSkippedRegions;
3920 errors.print("Summary:");
3921 for (TableInfo tInfo : tablesInfo.values()) {
3922 numOfSkippedRegions = (skippedRegions.containsKey(tInfo.getName())) ?
3923 skippedRegions.get(tInfo.getName()).size() : 0;
3924
3925 if (errors.tableHasErrors(tInfo)) {
3926 errors.print("Table " + tInfo.getName() + " is inconsistent.");
3927 } else if (numOfSkippedRegions > 0){
3928 errors.print("Table " + tInfo.getName() + " is okay (with "
3929 + numOfSkippedRegions + " skipped regions).");
3930 }
3931 else {
3932 errors.print("Table " + tInfo.getName() + " is okay.");
3933 }
3934 errors.print(" Number of regions: " + tInfo.getNumRegions());
3935 if (numOfSkippedRegions > 0) {
3936 Set<String> skippedRegionStrings = skippedRegions.get(tInfo.getName());
3937 System.out.println(" Number of skipped regions: " + numOfSkippedRegions);
3938 System.out.println(" List of skipped regions:");
3939 for(String sr : skippedRegionStrings) {
3940 System.out.println(" " + sr);
3941 }
3942 }
3943 sb.setLength(0);
3944 sb.append(" Deployed on: ");
3945 for (ServerName server : tInfo.deployedOn) {
3946 sb.append(" " + server.toString());
3947 }
3948 errors.print(sb.toString());
3949 }
3950 }
3951
3952 static ErrorReporter getErrorReporter(
3953 final Configuration conf) throws ClassNotFoundException {
3954 Class<? extends ErrorReporter> reporter = conf.getClass("hbasefsck.errorreporter", PrintingErrorReporter.class, ErrorReporter.class);
3955 return ReflectionUtils.newInstance(reporter, conf);
3956 }
3957
3958 public interface ErrorReporter {
3959 enum ERROR_CODE {
3960 UNKNOWN, NO_META_REGION, NULL_META_REGION, NO_VERSION_FILE, NOT_IN_META_HDFS, NOT_IN_META,
3961 NOT_IN_META_OR_DEPLOYED, NOT_IN_HDFS_OR_DEPLOYED, NOT_IN_HDFS, SERVER_DOES_NOT_MATCH_META, NOT_DEPLOYED,
3962 MULTI_DEPLOYED, SHOULD_NOT_BE_DEPLOYED, MULTI_META_REGION, RS_CONNECT_FAILURE,
3963 FIRST_REGION_STARTKEY_NOT_EMPTY, LAST_REGION_ENDKEY_NOT_EMPTY, DUPE_STARTKEYS,
3964 HOLE_IN_REGION_CHAIN, OVERLAP_IN_REGION_CHAIN, REGION_CYCLE, DEGENERATE_REGION,
3965 ORPHAN_HDFS_REGION, LINGERING_SPLIT_PARENT, NO_TABLEINFO_FILE, LINGERING_REFERENCE_HFILE,
3966 WRONG_USAGE, EMPTY_META_CELL, EXPIRED_TABLE_LOCK, ORPHANED_ZK_TABLE_ENTRY, BOUNDARIES_ERROR,
3967 DUPE_ENDKEYS
3968 }
3969 void clear();
3970 void report(String message);
3971 void reportError(String message);
3972 void reportError(ERROR_CODE errorCode, String message);
3973 void reportError(ERROR_CODE errorCode, String message, TableInfo table);
3974 void reportError(ERROR_CODE errorCode, String message, TableInfo table, HbckInfo info);
3975 void reportError(
3976 ERROR_CODE errorCode,
3977 String message,
3978 TableInfo table,
3979 HbckInfo info1,
3980 HbckInfo info2
3981 );
3982 int summarize();
3983 void detail(String details);
3984 ArrayList<ERROR_CODE> getErrorList();
3985 void progress();
3986 void print(String message);
3987 void resetErrors();
3988 boolean tableHasErrors(TableInfo table);
3989 }
3990
3991 static class PrintingErrorReporter implements ErrorReporter {
3992 public int errorCount = 0;
3993 private int showProgress;
3994
3995 private static final int progressThreshold = 100;
3996
3997 Set<TableInfo> errorTables = new HashSet<TableInfo>();
3998
3999
4000 private ArrayList<ERROR_CODE> errorList = new ArrayList<ERROR_CODE>();
4001
4002 @Override
4003 public void clear() {
4004 errorTables.clear();
4005 errorList.clear();
4006 errorCount = 0;
4007 }
4008
4009 @Override
4010 public synchronized void reportError(ERROR_CODE errorCode, String message) {
4011 if (errorCode == ERROR_CODE.WRONG_USAGE) {
4012 System.err.println(message);
4013 return;
4014 }
4015
4016 errorList.add(errorCode);
4017 if (!getSUMMARY()) {
4018 System.out.println("ERROR: " + message);
4019 }
4020 errorCount++;
4021 showProgress = 0;
4022 }
4023
4024 @Override
4025 public synchronized void reportError(ERROR_CODE errorCode, String message, TableInfo table) {
4026 errorTables.add(table);
4027 reportError(errorCode, message);
4028 }
4029
4030 @Override
4031 public synchronized void reportError(ERROR_CODE errorCode, String message, TableInfo table,
4032 HbckInfo info) {
4033 errorTables.add(table);
4034 String reference = "(region " + info.getRegionNameAsString() + ")";
4035 reportError(errorCode, reference + " " + message);
4036 }
4037
4038 @Override
4039 public synchronized void reportError(ERROR_CODE errorCode, String message, TableInfo table,
4040 HbckInfo info1, HbckInfo info2) {
4041 errorTables.add(table);
4042 String reference = "(regions " + info1.getRegionNameAsString()
4043 + " and " + info2.getRegionNameAsString() + ")";
4044 reportError(errorCode, reference + " " + message);
4045 }
4046
4047 @Override
4048 public synchronized void reportError(String message) {
4049 reportError(ERROR_CODE.UNKNOWN, message);
4050 }
4051
4052
4053
4054
4055
4056
4057 @Override
4058 public synchronized void report(String message) {
4059 if (!getSUMMARY()) {
4060 System.out.println("ERROR: " + message);
4061 }
4062 showProgress = 0;
4063 }
4064
4065 @Override
4066 public synchronized int summarize() {
4067 System.out.println(Integer.toString(errorCount) +
4068 " inconsistencies detected.");
4069 if (errorCount == 0) {
4070 System.out.println("Status: OK");
4071 return 0;
4072 } else {
4073 System.out.println("Status: INCONSISTENT");
4074 return -1;
4075 }
4076 }
4077
4078 @Override
4079 public ArrayList<ERROR_CODE> getErrorList() {
4080 return errorList;
4081 }
4082
4083 @Override
4084 public synchronized void print(String message) {
4085 if (!getSUMMARY()) {
4086 System.out.println(message);
4087 }
4088 }
4089
4090 private synchronized static boolean getSUMMARY() {
4091 return SUMMARY;
4092 }
4093
4094 @Override
4095 public boolean tableHasErrors(TableInfo table) {
4096 return errorTables.contains(table);
4097 }
4098
4099 @Override
4100 public void resetErrors() {
4101 errorCount = 0;
4102 }
4103
4104 @Override
4105 public synchronized void detail(String message) {
4106 if (details) {
4107 System.out.println(message);
4108 }
4109 showProgress = 0;
4110 }
4111
4112 @Override
4113 public synchronized void progress() {
4114 if (showProgress++ == progressThreshold) {
4115 if (!getSUMMARY()) {
4116 System.out.print(".");
4117 }
4118 showProgress = 0;
4119 }
4120 }
4121 }
4122
4123
4124
4125
4126 static class WorkItemRegion implements Callable<Void> {
4127 private HBaseFsck hbck;
4128 private ServerName rsinfo;
4129 private ErrorReporter errors;
4130 private HConnection connection;
4131
4132 WorkItemRegion(HBaseFsck hbck, ServerName info,
4133 ErrorReporter errors, HConnection connection) {
4134 this.hbck = hbck;
4135 this.rsinfo = info;
4136 this.errors = errors;
4137 this.connection = connection;
4138 }
4139
4140 @Override
4141 public synchronized Void call() throws IOException {
4142 errors.progress();
4143 try {
4144 BlockingInterface server = connection.getAdmin(rsinfo);
4145
4146
4147 List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(server);
4148 regions = filterRegions(regions);
4149
4150 if (details) {
4151 errors.detail("RegionServer: " + rsinfo.getServerName() +
4152 " number of regions: " + regions.size());
4153 for (HRegionInfo rinfo: regions) {
4154 errors.detail(" " + rinfo.getRegionNameAsString() +
4155 " id: " + rinfo.getRegionId() +
4156 " encoded_name: " + rinfo.getEncodedName() +
4157 " start: " + Bytes.toStringBinary(rinfo.getStartKey()) +
4158 " end: " + Bytes.toStringBinary(rinfo.getEndKey()));
4159 }
4160 }
4161
4162
4163 for (HRegionInfo r:regions) {
4164 HbckInfo hbi = hbck.getOrCreateInfo(r.getEncodedName());
4165 hbi.addServer(r, rsinfo);
4166 }
4167 } catch (IOException e) {
4168 errors.reportError(ERROR_CODE.RS_CONNECT_FAILURE, "RegionServer: " + rsinfo.getServerName() +
4169 " Unable to fetch region information. " + e);
4170 throw e;
4171 }
4172 return null;
4173 }
4174
4175 private List<HRegionInfo> filterRegions(List<HRegionInfo> regions) {
4176 List<HRegionInfo> ret = Lists.newArrayList();
4177 for (HRegionInfo hri : regions) {
4178 if (hri.isMetaTable() || (!hbck.checkMetaOnly
4179 && hbck.isTableIncluded(hri.getTable()))) {
4180 ret.add(hri);
4181 }
4182 }
4183 return ret;
4184 }
4185 }
4186
4187
4188
4189
4190
4191 static class WorkItemHdfsDir implements Callable<Void> {
4192 private HBaseFsck hbck;
4193 private FileStatus tableDir;
4194 private ErrorReporter errors;
4195 private FileSystem fs;
4196
4197 WorkItemHdfsDir(HBaseFsck hbck, FileSystem fs, ErrorReporter errors,
4198 FileStatus status) {
4199 this.hbck = hbck;
4200 this.fs = fs;
4201 this.tableDir = status;
4202 this.errors = errors;
4203 }
4204
4205 @Override
4206 public synchronized Void call() throws IOException {
4207 try {
4208
4209
4210
4211
4212
4213
4214
4215
4216
4217
4218
4219
4220
4221
4222
4223
4224
4225
4226
4227
4228
4229
4230
4231
4232
4233
4234
4235
4236
4237
4238
4239
4240
4241
4242
4243
4244
4245
4246
4247
4248
4249
4250
4251
4252
4253
4254
4255
4256
4257
4258
4259
4260
4261
4262
4263
4264 static class WorkItemHdfsRegionInfo implements Callable<Void> {
4265 private HbckInfo hbi;
4266 private HBaseFsck hbck;
4267 private ErrorReporter errors;
4268
4269 WorkItemHdfsRegionInfo(HbckInfo hbi, HBaseFsck hbck, ErrorReporter errors) {
4270 this.hbi = hbi;
4271 this.hbck = hbck;
4272 this.errors = errors;
4273 }
4274
4275 @Override
4276 public synchronized Void call() throws IOException {
4277
4278 if (hbi.getHdfsHRI() == null) {
4279 try {
4280 errors.progress();
4281 hbck.loadHdfsRegioninfo(hbi);
4282 } catch (IOException ioe) {
4283 String msg = "Orphan region in HDFS: Unable to load .regioninfo from table "
4284 + hbi.getTableName() + " in hdfs dir "
4285 + hbi.getHdfsRegionDir()
4286 + "! It may be an invalid format or version file. Treating as "
4287 + "an orphaned regiondir.";
4288 errors.reportError(ERROR_CODE.ORPHAN_HDFS_REGION, msg);
4289 try {
4290 hbck.debugLsr(hbi.getHdfsRegionDir());
4291 } catch (IOException ioe2) {
4292 LOG.error("Unable to read directory " + hbi.getHdfsRegionDir(), ioe2);
4293 throw ioe2;
4294 }
4295 hbck.orphanHdfsDirs.add(hbi);
4296 throw ioe;
4297 }
4298 }
4299 return null;
4300 }
4301 };
4302
4303
4304
4305
4306
4307 public static void setDisplayFullReport() {
4308 details = true;
4309 }
4310
4311
4312
4313
4314 public static void setForceExclusive() {
4315 forceExclusive = true;
4316 }
4317
4318
4319
4320
4321 public boolean isExclusive() {
4322 return fixAny || forceExclusive;
4323 }
4324
4325
4326
4327
4328 public static void setDisableBalancer() {
4329 disableBalancer = true;
4330 }
4331
4332
4333
4334
4335
4336
4337 public boolean shouldDisableBalancer() {
4338 return fixAny || disableBalancer;
4339 }
4340
4341
4342
4343
4344
4345 synchronized static void setSummary() {
4346 SUMMARY = true;
4347 }
4348
4349
4350
4351
4352
4353 void setCheckMetaOnly() {
4354 checkMetaOnly = true;
4355 }
4356
4357
4358
4359
4360 void setRegionBoundariesCheck() {
4361 checkRegionBoundaries = true;
4362 }
4363
4364
4365
4366
4367
4368 public void setFixTableLocks(boolean shouldFix) {
4369 fixTableLocks = shouldFix;
4370 fixAny |= shouldFix;
4371 }
4372
4373
4374
4375
4376
4377 public void setFixTableZNodes(boolean shouldFix) {
4378 fixTableZNodes = shouldFix;
4379 fixAny |= shouldFix;
4380 }
4381
4382
4383
4384
4385
4386
4387
4388 void setShouldRerun() {
4389 rerun = true;
4390 }
4391
4392 boolean shouldRerun() {
4393 return rerun;
4394 }
4395
4396
4397
4398
4399
4400 public void setFixAssignments(boolean shouldFix) {
4401 fixAssignments = shouldFix;
4402 fixAny |= shouldFix;
4403 }
4404
4405 boolean shouldFixAssignments() {
4406 return fixAssignments;
4407 }
4408
4409 public void setFixMeta(boolean shouldFix) {
4410 fixMeta = shouldFix;
4411 fixAny |= shouldFix;
4412 }
4413
4414 boolean shouldFixMeta() {
4415 return fixMeta;
4416 }
4417
4418 public void setFixEmptyMetaCells(boolean shouldFix) {
4419 fixEmptyMetaCells = shouldFix;
4420 fixAny |= shouldFix;
4421 }
4422
4423 boolean shouldFixEmptyMetaCells() {
4424 return fixEmptyMetaCells;
4425 }
4426
4427 public void setCheckHdfs(boolean checking) {
4428 checkHdfs = checking;
4429 }
4430
4431 boolean shouldCheckHdfs() {
4432 return checkHdfs;
4433 }
4434
4435 public void setFixHdfsHoles(boolean shouldFix) {
4436 fixHdfsHoles = shouldFix;
4437 fixAny |= shouldFix;
4438 }
4439
4440 boolean shouldFixHdfsHoles() {
4441 return fixHdfsHoles;
4442 }
4443
4444 public void setFixTableOrphans(boolean shouldFix) {
4445 fixTableOrphans = shouldFix;
4446 fixAny |= shouldFix;
4447 }
4448
4449 boolean shouldFixTableOrphans() {
4450 return fixTableOrphans;
4451 }
4452
4453 public void setFixHdfsOverlaps(boolean shouldFix) {
4454 fixHdfsOverlaps = shouldFix;
4455 fixAny |= shouldFix;
4456 }
4457
4458 boolean shouldFixHdfsOverlaps() {
4459 return fixHdfsOverlaps;
4460 }
4461
4462 public void setFixHdfsOrphans(boolean shouldFix) {
4463 fixHdfsOrphans = shouldFix;
4464 fixAny |= shouldFix;
4465 }
4466
4467 boolean shouldFixHdfsOrphans() {
4468 return fixHdfsOrphans;
4469 }
4470
4471 public void setFixVersionFile(boolean shouldFix) {
4472 fixVersionFile = shouldFix;
4473 fixAny |= shouldFix;
4474 }
4475
4476 public boolean shouldFixVersionFile() {
4477 return fixVersionFile;
4478 }
4479
4480 public void setSidelineBigOverlaps(boolean sbo) {
4481 this.sidelineBigOverlaps = sbo;
4482 }
4483
4484 public boolean shouldSidelineBigOverlaps() {
4485 return sidelineBigOverlaps;
4486 }
4487
4488 public void setFixSplitParents(boolean shouldFix) {
4489 fixSplitParents = shouldFix;
4490 fixAny |= shouldFix;
4491 }
4492
4493 public void setRemoveParents(boolean shouldFix) {
4494 removeParents = shouldFix;
4495 fixAny |= shouldFix;
4496 }
4497
4498 boolean shouldFixSplitParents() {
4499 return fixSplitParents;
4500 }
4501
4502 boolean shouldRemoveParents() {
4503 return removeParents;
4504 }
4505
4506 public void setFixReferenceFiles(boolean shouldFix) {
4507 fixReferenceFiles = shouldFix;
4508 fixAny |= shouldFix;
4509 }
4510
4511 boolean shouldFixReferenceFiles() {
4512 return fixReferenceFiles;
4513 }
4514
4515 public boolean shouldIgnorePreCheckPermission() {
4516 return !fixAny || ignorePreCheckPermission;
4517 }
4518
4519 public void setIgnorePreCheckPermission(boolean ignorePreCheckPermission) {
4520 this.ignorePreCheckPermission = ignorePreCheckPermission;
4521 }
4522
4523
4524
4525
4526 public void setMaxMerge(int mm) {
4527 this.maxMerge = mm;
4528 }
4529
4530 public int getMaxMerge() {
4531 return maxMerge;
4532 }
4533
4534 public void setMaxOverlapsToSideline(int mo) {
4535 this.maxOverlapsToSideline = mo;
4536 }
4537
4538 public int getMaxOverlapsToSideline() {
4539 return maxOverlapsToSideline;
4540 }
4541
4542
4543
4544
4545
4546 boolean isTableIncluded(TableName table) {
4547 return (tablesIncluded.size() == 0) || tablesIncluded.contains(table);
4548 }
4549
4550 public void includeTable(TableName table) {
4551 tablesIncluded.add(table);
4552 }
4553
4554 Set<TableName> getIncludedTables() {
4555 return new HashSet<TableName>(tablesIncluded);
4556 }
4557
4558
4559
4560
4561
4562
4563 public void setTimeLag(long seconds) {
4564 timelag = seconds * 1000;
4565 }
4566
4567
4568
4569
4570
4571 public void setSidelineDir(String sidelineDir) {
4572 this.sidelineDir = new Path(sidelineDir);
4573 }
4574
4575 protected HFileCorruptionChecker createHFileCorruptionChecker(boolean sidelineCorruptHFiles) throws IOException {
4576 return new HFileCorruptionChecker(getConf(), executor, sidelineCorruptHFiles);
4577 }
4578
4579 public HFileCorruptionChecker getHFilecorruptionChecker() {
4580 return hfcc;
4581 }
4582
4583 public void setHFileCorruptionChecker(HFileCorruptionChecker hfcc) {
4584 this.hfcc = hfcc;
4585 }
4586
4587 public void setRetCode(int code) {
4588 this.retcode = code;
4589 }
4590
4591 public int getRetCode() {
4592 return retcode;
4593 }
4594
4595 protected HBaseFsck printUsageAndExit() {
4596 StringWriter sw = new StringWriter(2048);
4597 PrintWriter out = new PrintWriter(sw);
4598 out.println("Usage: fsck [opts] {only tables}");
4599 out.println(" where [opts] are:");
4600 out.println(" -help Display help options (this)");
4601 out.println(" -details Display full report of all regions.");
4602 out.println(" -timelag <timeInSeconds> Process only regions that " +
4603 " have not experienced any metadata updates in the last " +
4604 " <timeInSeconds> seconds.");
4605 out.println(" -sleepBeforeRerun <timeInSeconds> Sleep this many seconds" +
4606 " before checking if the fix worked if run with -fix");
4607 out.println(" -summary Print only summary of the tables and status.");
4608 out.println(" -metaonly Only check the state of the hbase:meta table.");
4609 out.println(" -sidelineDir <hdfs://> HDFS path to backup existing meta.");
4610 out.println(" -boundaries Verify that regions boundaries are the same between META and store files.");
4611 out.println(" -exclusive Abort if another hbck is exclusive or fixing.");
4612 out.println(" -disableBalancer Disable the load balancer.");
4613
4614 out.println("");
4615 out.println(" Metadata Repair options: (expert features, use with caution!)");
4616 out.println(" -fix Try to fix region assignments. This is for backwards compatiblity");
4617 out.println(" -fixAssignments Try to fix region assignments. Replaces the old -fix");
4618 out.println(" -fixMeta Try to fix meta problems. This assumes HDFS region info is good.");
4619 out.println(" -noHdfsChecking Don't load/check region info from HDFS."
4620 + " Assumes hbase:meta region info is good. Won't check/fix any HDFS issue, e.g. hole, orphan, or overlap");
4621 out.println(" -fixHdfsHoles Try to fix region holes in hdfs.");
4622 out.println(" -fixHdfsOrphans Try to fix region dirs with no .regioninfo file in hdfs");
4623 out.println(" -fixTableOrphans Try to fix table dirs with no .tableinfo file in hdfs (online mode only)");
4624 out.println(" -fixHdfsOverlaps Try to fix region overlaps in hdfs.");
4625 out.println(" -fixVersionFile Try to fix missing hbase.version file in hdfs.");
4626 out.println(" -maxMerge <n> When fixing region overlaps, allow at most <n> regions to merge. (n=" + DEFAULT_MAX_MERGE +" by default)");
4627 out.println(" -sidelineBigOverlaps When fixing region overlaps, allow to sideline big overlaps");
4628 out.println(" -maxOverlapsToSideline <n> When fixing region overlaps, allow at most <n> regions to sideline per group. (n=" + DEFAULT_OVERLAPS_TO_SIDELINE +" by default)");
4629 out.println(" -fixSplitParents Try to force offline split parents to be online.");
4630 out.println(" -removeParents Try to offline and sideline lingering parents and keep daughter regions.");
4631 out.println(" -ignorePreCheckPermission ignore filesystem permission pre-check");
4632 out.println(" -fixReferenceFiles Try to offline lingering reference store files");
4633 out.println(" -fixEmptyMetaCells Try to fix hbase:meta entries not referencing any region"
4634 + " (empty REGIONINFO_QUALIFIER rows)");
4635
4636 out.println("");
4637 out.println(" Datafile Repair options: (expert features, use with caution!)");
4638 out.println(" -checkCorruptHFiles Check all Hfiles by opening them to make sure they are valid");
4639 out.println(" -sidelineCorruptHFiles Quarantine corrupted HFiles. implies -checkCorruptHFiles");
4640
4641 out.println("");
4642 out.println(" Metadata Repair shortcuts");
4643 out.println(" -repair Shortcut for -fixAssignments -fixMeta -fixHdfsHoles " +
4644 "-fixHdfsOrphans -fixHdfsOverlaps -fixVersionFile -sidelineBigOverlaps " +
4645 "-fixReferenceFiles -fixTableLocks -fixOrphanedTableZnodes");
4646 out.println(" -repairHoles Shortcut for -fixAssignments -fixMeta -fixHdfsHoles");
4647
4648 out.println("");
4649 out.println(" Table lock options");
4650 out.println(" -fixTableLocks Deletes table locks held for a long time (hbase.table.lock.expire.ms, 10min by default)");
4651
4652 out.println("");
4653 out.println(" Table Znode options");
4654 out.println(" -fixOrphanedTableZnodes Set table state in ZNode to disabled if table does not exists");
4655
4656 out.flush();
4657 errors.reportError(ERROR_CODE.WRONG_USAGE, sw.toString());
4658
4659 setRetCode(-2);
4660 return this;
4661 }
4662
4663
4664
4665
4666
4667
4668
4669 public static void main(String[] args) throws Exception {
4670
4671 Configuration conf = HBaseConfiguration.create();
4672 Path hbasedir = FSUtils.getRootDir(conf);
4673 URI defaultFs = hbasedir.getFileSystem(conf).getUri();
4674 FSUtils.setFsDefault(conf, new Path(defaultFs));
4675 int ret = ToolRunner.run(new HBaseFsckTool(conf), args);
4676 System.exit(ret);
4677 }
4678
4679
4680
4681
4682 static class HBaseFsckTool extends Configured implements Tool {
4683 HBaseFsckTool(Configuration conf) { super(conf); }
4684 @Override
4685 public int run(String[] args) throws Exception {
4686 HBaseFsck hbck = new HBaseFsck(getConf());
4687 hbck.exec(hbck.executor, args);
4688 hbck.close();
4689 return hbck.getRetCode();
4690 }
4691 };
4692
4693
4694 public HBaseFsck exec(ExecutorService exec, String[] args) throws KeeperException, IOException,
4695 ServiceException, InterruptedException {
4696 long sleepBeforeRerun = DEFAULT_SLEEP_BEFORE_RERUN;
4697
4698 boolean checkCorruptHFiles = false;
4699 boolean sidelineCorruptHFiles = false;
4700
4701
4702 for (int i = 0; i < args.length; i++) {
4703 String cmd = args[i];
4704 if (cmd.equals("-help") || cmd.equals("-h")) {
4705 return printUsageAndExit();
4706 } else if (cmd.equals("-details")) {
4707 setDisplayFullReport();
4708 } else if (cmd.equals("-exclusive")) {
4709 setForceExclusive();
4710 } else if (cmd.equals("-disableBalancer")) {
4711 setDisableBalancer();
4712 } else if (cmd.equals("-timelag")) {
4713 if (i == args.length - 1) {
4714 errors.reportError(ERROR_CODE.WRONG_USAGE, "HBaseFsck: -timelag needs a value.");
4715 return printUsageAndExit();
4716 }
4717 try {
4718 long timelag = Long.parseLong(args[i+1]);
4719 setTimeLag(timelag);
4720 } catch (NumberFormatException e) {
4721 errors.reportError(ERROR_CODE.WRONG_USAGE, "-timelag needs a numeric value.");
4722 return printUsageAndExit();
4723 }
4724 i++;
4725 } else if (cmd.equals("-sleepBeforeRerun")) {
4726 if (i == args.length - 1) {
4727 errors.reportError(ERROR_CODE.WRONG_USAGE,
4728 "HBaseFsck: -sleepBeforeRerun needs a value.");
4729 return printUsageAndExit();
4730 }
4731 try {
4732 sleepBeforeRerun = Long.parseLong(args[i+1]);
4733 } catch (NumberFormatException e) {
4734 errors.reportError(ERROR_CODE.WRONG_USAGE, "-sleepBeforeRerun needs a numeric value.");
4735 return printUsageAndExit();
4736 }
4737 i++;
4738 } else if (cmd.equals("-sidelineDir")) {
4739 if (i == args.length - 1) {
4740 errors.reportError(ERROR_CODE.WRONG_USAGE, "HBaseFsck: -sidelineDir needs a value.");
4741 return printUsageAndExit();
4742 }
4743 i++;
4744 setSidelineDir(args[i]);
4745 } else if (cmd.equals("-fix")) {
4746 errors.reportError(ERROR_CODE.WRONG_USAGE,
4747 "This option is deprecated, please use -fixAssignments instead.");
4748 setFixAssignments(true);
4749 } else if (cmd.equals("-fixAssignments")) {
4750 setFixAssignments(true);
4751 } else if (cmd.equals("-fixMeta")) {
4752 setFixMeta(true);
4753 } else if (cmd.equals("-noHdfsChecking")) {
4754 setCheckHdfs(false);
4755 } else if (cmd.equals("-fixHdfsHoles")) {
4756 setFixHdfsHoles(true);
4757 } else if (cmd.equals("-fixHdfsOrphans")) {
4758 setFixHdfsOrphans(true);
4759 } else if (cmd.equals("-fixTableOrphans")) {
4760 setFixTableOrphans(true);
4761 } else if (cmd.equals("-fixHdfsOverlaps")) {
4762 setFixHdfsOverlaps(true);
4763 } else if (cmd.equals("-fixVersionFile")) {
4764 setFixVersionFile(true);
4765 } else if (cmd.equals("-sidelineBigOverlaps")) {
4766 setSidelineBigOverlaps(true);
4767 } else if (cmd.equals("-fixSplitParents")) {
4768 setFixSplitParents(true);
4769 } else if (cmd.equals("-removeParents")) {
4770 setRemoveParents(true);
4771 } else if (cmd.equals("-ignorePreCheckPermission")) {
4772 setIgnorePreCheckPermission(true);
4773 } else if (cmd.equals("-checkCorruptHFiles")) {
4774 checkCorruptHFiles = true;
4775 } else if (cmd.equals("-sidelineCorruptHFiles")) {
4776 sidelineCorruptHFiles = true;
4777 } else if (cmd.equals("-fixReferenceFiles")) {
4778 setFixReferenceFiles(true);
4779 } else if (cmd.equals("-fixEmptyMetaCells")) {
4780 setFixEmptyMetaCells(true);
4781 } else if (cmd.equals("-repair")) {
4782
4783
4784 setFixHdfsHoles(true);
4785 setFixHdfsOrphans(true);
4786 setFixMeta(true);
4787 setFixAssignments(true);
4788 setFixHdfsOverlaps(true);
4789 setFixVersionFile(true);
4790 setSidelineBigOverlaps(true);
4791 setFixSplitParents(false);
4792 setCheckHdfs(true);
4793 setFixReferenceFiles(true);
4794 setFixTableLocks(true);
4795 setFixTableZNodes(true);
4796 } else if (cmd.equals("-repairHoles")) {
4797
4798 setFixHdfsHoles(true);
4799 setFixHdfsOrphans(false);
4800 setFixMeta(true);
4801 setFixAssignments(true);
4802 setFixHdfsOverlaps(false);
4803 setSidelineBigOverlaps(false);
4804 setFixSplitParents(false);
4805 setCheckHdfs(true);
4806 } else if (cmd.equals("-maxOverlapsToSideline")) {
4807 if (i == args.length - 1) {
4808 errors.reportError(ERROR_CODE.WRONG_USAGE,
4809 "-maxOverlapsToSideline needs a numeric value argument.");
4810 return printUsageAndExit();
4811 }
4812 try {
4813 int maxOverlapsToSideline = Integer.parseInt(args[i+1]);
4814 setMaxOverlapsToSideline(maxOverlapsToSideline);
4815 } catch (NumberFormatException e) {
4816 errors.reportError(ERROR_CODE.WRONG_USAGE,
4817 "-maxOverlapsToSideline needs a numeric value argument.");
4818 return printUsageAndExit();
4819 }
4820 i++;
4821 } else if (cmd.equals("-maxMerge")) {
4822 if (i == args.length - 1) {
4823 errors.reportError(ERROR_CODE.WRONG_USAGE,
4824 "-maxMerge needs a numeric value argument.");
4825 return printUsageAndExit();
4826 }
4827 try {
4828 int maxMerge = Integer.parseInt(args[i+1]);
4829 setMaxMerge(maxMerge);
4830 } catch (NumberFormatException e) {
4831 errors.reportError(ERROR_CODE.WRONG_USAGE,
4832 "-maxMerge needs a numeric value argument.");
4833 return printUsageAndExit();
4834 }
4835 i++;
4836 } else if (cmd.equals("-summary")) {
4837 setSummary();
4838 } else if (cmd.equals("-metaonly")) {
4839 setCheckMetaOnly();
4840 } else if (cmd.equals("-boundaries")) {
4841 setRegionBoundariesCheck();
4842 } else if (cmd.equals("-fixTableLocks")) {
4843 setFixTableLocks(true);
4844 } else if (cmd.equals("-fixOrphanedTableZnodes")) {
4845 setFixTableZNodes(true);
4846 } else if (cmd.startsWith("-")) {
4847 errors.reportError(ERROR_CODE.WRONG_USAGE, "Unrecognized option:" + cmd);
4848 return printUsageAndExit();
4849 } else {
4850 includeTable(TableName.valueOf(cmd));
4851 errors.print("Allow checking/fixes for table: " + cmd);
4852 }
4853 }
4854
4855 errors.print("HBaseFsck command line options: " + StringUtils.join(args, " "));
4856
4857
4858 try {
4859 preCheckPermission();
4860 } catch (AccessDeniedException ace) {
4861 Runtime.getRuntime().exit(-1);
4862 } catch (IOException ioe) {
4863 Runtime.getRuntime().exit(-1);
4864 }
4865
4866
4867 connect();
4868
4869 try {
4870
4871 if (checkCorruptHFiles || sidelineCorruptHFiles) {
4872 LOG.info("Checking all hfiles for corruption");
4873 HFileCorruptionChecker hfcc = createHFileCorruptionChecker(sidelineCorruptHFiles);
4874 setHFileCorruptionChecker(hfcc);
4875 Collection<TableName> tables = getIncludedTables();
4876 Collection<Path> tableDirs = new ArrayList<Path>();
4877 Path rootdir = FSUtils.getRootDir(getConf());
4878 if (tables.size() > 0) {
4879 for (TableName t : tables) {
4880 tableDirs.add(FSUtils.getTableDir(rootdir, t));
4881 }
4882 } else {
4883 tableDirs = FSUtils.getTableDirs(FSUtils.getCurrentFileSystem(getConf()), rootdir);
4884 }
4885 hfcc.checkTables(tableDirs);
4886 hfcc.report(errors);
4887 }
4888
4889
4890 int code = onlineHbck();
4891 setRetCode(code);
4892
4893
4894
4895
4896 if (shouldRerun()) {
4897 try {
4898 LOG.info("Sleeping " + sleepBeforeRerun + "ms before re-checking after fix...");
4899 Thread.sleep(sleepBeforeRerun);
4900 } catch (InterruptedException ie) {
4901 LOG.warn("Interrupted while sleeping");
4902 return this;
4903 }
4904
4905 setFixAssignments(false);
4906 setFixMeta(false);
4907 setFixHdfsHoles(false);
4908 setFixHdfsOverlaps(false);
4909 setFixVersionFile(false);
4910 setFixTableOrphans(false);
4911 errors.resetErrors();
4912 code = onlineHbck();
4913 setRetCode(code);
4914 }
4915 } finally {
4916 IOUtils.closeQuietly(this);
4917 }
4918 return this;
4919 }
4920
4921
4922
4923
4924 void debugLsr(Path p) throws IOException {
4925 debugLsr(getConf(), p, errors);
4926 }
4927
4928
4929
4930
4931 public static void debugLsr(Configuration conf,
4932 Path p) throws IOException {
4933 debugLsr(conf, p, new PrintingErrorReporter());
4934 }
4935
4936
4937
4938
4939 public static void debugLsr(Configuration conf,
4940 Path p, ErrorReporter errors) throws IOException {
4941 if (!LOG.isDebugEnabled() || p == null) {
4942 return;
4943 }
4944 FileSystem fs = p.getFileSystem(conf);
4945
4946 if (!fs.exists(p)) {
4947
4948 return;
4949 }
4950 errors.print(p.toString());
4951
4952 if (fs.isFile(p)) {
4953 return;
4954 }
4955
4956 if (fs.getFileStatus(p).isDirectory()) {
4957 FileStatus[] fss= fs.listStatus(p);
4958 for (FileStatus status : fss) {
4959 debugLsr(conf, status.getPath(), errors);
4960 }
4961 }
4962 }
4963 }