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