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