SE-4377 [hbck] Offline rebuild .META. from fs data only

(Jonathan Hsieh)


git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1196527 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Zhihong Yu 2011-11-02 09:51:19 +00:00
parent 3fab67a8b5
commit 061c22b302
11 changed files with 989 additions and 64 deletions

View File

@ -789,6 +789,8 @@ Release 0.92.0 - Unreleased
HBASE-4057 Implement HBase version of "show processlist" (Riley Patterson)
HBASE-4219 Per Column Family Metrics
HBASE-4219 Addendum for failure of TestHFileBlock
HBASE-4377 [hbck] Offline rebuild .META. from fs data only
(Jonathan Hsieh)
Release 0.90.5 - Unreleased

View File

@ -501,6 +501,9 @@ implements WritableComparable<HRegionInfo> {
* @return byte array of table name
*/
public byte[] getTableName() {
if (tableName == null || tableName.length == 0) {
tableName = getTableName(getRegionName());
}
return tableName;
}
@ -790,4 +793,4 @@ implements WritableComparable<HRegionInfo> {
return isRootRegion()? KeyValue.ROOT_COMPARATOR: isMetaRegion()?
KeyValue.META_COMPARATOR: KeyValue.COMPARATOR;
}
}
}

View File

@ -372,7 +372,10 @@ public class MasterFileSystem {
}
}
private static void setInfoFamilyCachingForRoot(final boolean b) {
/**
* Enable in-memory caching for -ROOT-
*/
public static void setInfoFamilyCachingForRoot(final boolean b) {
for (HColumnDescriptor hcd:
HTableDescriptor.ROOT_TABLEDESC.getColumnFamilies()) {
if (Bytes.equals(hcd.getName(), HConstants.CATALOG_FAMILY)) {
@ -382,7 +385,10 @@ public class MasterFileSystem {
}
}
private static void setInfoFamilyCachingForMeta(final boolean b) {
/**
* Enable in memory caching for .META.
*/
public static void setInfoFamilyCachingForMeta(final boolean b) {
for (HColumnDescriptor hcd:
HTableDescriptor.META_TABLEDESC.getColumnFamilies()) {
if (Bytes.equals(hcd.getName(), HConstants.CATALOG_FAMILY)) {

View File

@ -26,6 +26,7 @@ import java.util.Collections;
import java.util.Comparator;
import java.util.HashSet;
import java.util.List;
import java.util.Map.Entry;
import java.util.Set;
import java.util.SortedSet;
import java.util.TreeMap;
@ -38,6 +39,7 @@ import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -60,13 +62,17 @@ import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HConnectionManager.HConnectable;
import org.apache.hadoop.hbase.client.MetaScanner;
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.zookeeper.RootRegionTracker;
import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
import org.apache.hadoop.hbase.zookeeper.RootRegionTracker;
import org.apache.hadoop.hbase.zookeeper.ZKTable;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.io.MultipleIOException;
import org.apache.zookeeper.KeeperException;
import com.google.common.base.Joiner;
@ -142,7 +148,7 @@ public class HBaseFsck {
* @throws KeeperException
* @throws InterruptedException
*/
int doWork() throws IOException, KeeperException, InterruptedException {
public int doWork() throws IOException, KeeperException, InterruptedException {
// print hbase server version
errors.print("Version: " + status.getHBaseVersion());
@ -247,6 +253,252 @@ public class HBaseFsck {
return errors;
}
/**
* Populate a specific hbi from regioninfo on file system.
*/
private void loadMetaEntry(HbckInfo hbi) throws IOException {
Path regionDir = hbi.foundRegionDir.getPath();
Path regioninfo = new Path(regionDir, HRegion.REGIONINFO_FILE);
FileSystem fs = FileSystem.get(conf);
FSDataInputStream in = fs.open(regioninfo);
byte[] tableName = Bytes.toBytes(hbi.hdfsTableName);
HRegionInfo hri = new HRegionInfo(tableName);
hri.readFields(in);
in.close();
LOG.debug("HRegionInfo read: " + hri.toString());
hbi.metaEntry = new MetaEntry(hri, null,
hbi.foundRegionDir.getModificationTime());
}
public static class RegionInfoLoadException extends IOException {
private static final long serialVersionUID = 1L;
final IOException ioe;
public RegionInfoLoadException(String s, IOException ioe) {
super(s);
this.ioe = ioe;
}
}
/**
* Populate hbi's from regionInfos loaded from file system.
*/
private void loadTableInfo() throws IOException {
List<IOException> ioes = new ArrayList<IOException>();
// generate region split structure
for (HbckInfo hbi : regionInfo.values()) {
// only load entries that haven't been loaded yet.
if (hbi.metaEntry == null) {
try {
loadMetaEntry(hbi);
} catch (IOException ioe) {
String msg = "Unable to load region info for table " + hbi.hdfsTableName
+ "! It may be an invalid format or version file. You may want to "
+ "remove " + hbi.foundRegionDir.getPath()
+ " region from hdfs and retry.";
errors.report(msg);
LOG.error(msg, ioe);
ioes.add(new RegionInfoLoadException(msg, ioe));
continue;
}
}
// get table name from hdfs, populate various HBaseFsck tables.
String tableName = hbi.hdfsTableName;
TInfo modTInfo = tablesInfo.get(tableName);
if (modTInfo == null) {
modTInfo = new TInfo(tableName);
}
modTInfo.addRegionInfo(hbi);
tablesInfo.put(tableName, modTInfo);
}
if (ioes.size() != 0) {
throw MultipleIOException.createIOException(ioes);
}
}
/**
* This borrows code from MasterFileSystem.bootstrap()
*
* @return an open .META. HRegion
*/
private HRegion createNewRootAndMeta() throws IOException {
Path rootdir = new Path(conf.get(HConstants.HBASE_DIR));
Configuration c = conf;
HRegionInfo rootHRI = new HRegionInfo(HRegionInfo.ROOT_REGIONINFO);
MasterFileSystem.setInfoFamilyCachingForRoot(false);
HRegionInfo metaHRI = new HRegionInfo(HRegionInfo.FIRST_META_REGIONINFO);
MasterFileSystem.setInfoFamilyCachingForMeta(false);
HRegion root = HRegion.createHRegion(rootHRI, rootdir, c,
HTableDescriptor.ROOT_TABLEDESC);
HRegion meta = HRegion.createHRegion(metaHRI, rootdir, c,
HTableDescriptor.META_TABLEDESC);
MasterFileSystem.setInfoFamilyCachingForRoot(true);
MasterFileSystem.setInfoFamilyCachingForMeta(true);
// Add first region from the META table to the ROOT region.
HRegion.addRegionToMETA(root, meta);
root.close();
root.getLog().closeAndDelete();
return meta;
}
/**
* Generate set of puts to add to new meta. This expects the tables to be
* clean with no overlaps or holes. If there are any problems it returns null.
*
* @return An array list of puts to do in bulk, null if tables have problems
*/
private ArrayList<Put> generatePuts() throws IOException {
ArrayList<Put> puts = new ArrayList<Put>();
boolean hasProblems = false;
for (Entry<String, TInfo> e : tablesInfo.entrySet()) {
String name = e.getKey();
// skip "-ROOT-" and ".META."
if (Bytes.compareTo(Bytes.toBytes(name), HConstants.ROOT_TABLE_NAME) == 0
|| Bytes.compareTo(Bytes.toBytes(name), HConstants.META_TABLE_NAME) == 0) {
continue;
}
TInfo ti = e.getValue();
for (Entry<byte[], Collection<HbckInfo>> spl : ti.sc.getStarts().asMap()
.entrySet()) {
Collection<HbckInfo> his = spl.getValue();
int sz = his.size();
if (sz != 1) {
// problem
LOG.error("Split starting at " + Bytes.toStringBinary(spl.getKey())
+ " had " + sz + " regions instead of exactly 1." );
hasProblems = true;
continue;
}
// add the row directly to meta.
HbckInfo hi = his.iterator().next();
HRegionInfo hri = hi.metaEntry;
Put p = new Put(hri.getRegionName());
p.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(hri));
puts.add(p);
}
}
return hasProblems ? null : puts;
}
/**
* Suggest fixes for each table
*/
private void suggestFixes(TreeMap<String, TInfo> tablesInfo) {
for (TInfo tInfo : tablesInfo.values()) {
tInfo.checkRegionChain();
}
}
/**
* Rebuilds meta from information in hdfs/fs. Depends on configuration
* settings passed into hbck constructor to point to a particular fs/dir.
*
* @return true if successful, false if attempt failed.
*/
public boolean rebuildMeta() throws IOException, InterruptedException {
// TODO check to make sure hbase is offline. (or at least the table
// currently being worked on is off line)
// Determine what's on HDFS
LOG.info("Loading HBase regioninfo from HDFS...");
checkHdfs(); // populating regioninfo table.
loadTableInfo(); // update tableInfos based on region info in fs.
LOG.info("Checking HBase region split map from HDFS data...");
int errs = errors.getErrorList().size();
for (TInfo tInfo : tablesInfo.values()) {
if (!tInfo.checkRegionChain()) {
// should dump info as well.
errors.report("Found inconsistency in table " + tInfo.getName());
}
}
// make sure ok.
if (errors.getErrorList().size() != errs) {
suggestFixes(tablesInfo);
// Not ok, bail out.
return false;
}
// we can rebuild, move old root and meta out of the way and start
LOG.info("HDFS regioninfo's seems good. Sidelining old .META.");
sidelineOldRootAndMeta();
LOG.info("Creating new .META.");
HRegion meta = createNewRootAndMeta();
// populate meta
List<Put> puts = generatePuts();
if (puts == null) {
LOG.fatal("Problem encountered when creating new .META. entries. " +
"You may need to restore the previously sidlined -ROOT- and .META.");
return false;
}
meta.put(puts.toArray(new Put[0]));
meta.close();
meta.getLog().closeAndDelete();
LOG.info("Success! .META. table rebuilt.");
return true;
}
void sidelineTable(FileSystem fs, byte[] table, Path hbaseDir,
Path backupHbaseDir) throws IOException {
String tableName = Bytes.toString(table);
Path tableDir = new Path(hbaseDir, tableName);
if (fs.exists(tableDir)) {
Path backupTableDir= new Path(backupHbaseDir, tableName);
boolean success = fs.rename(tableDir, backupTableDir);
if (!success) {
throw new IOException("Failed to move " + tableName + " from "
+ tableDir.getName() + " to " + backupTableDir.getName());
}
} else {
LOG.info("No previous " + tableName + " exists. Continuing.");
}
}
/**
* @return Path to backup of original directory
* @throws IOException
*/
Path sidelineOldRootAndMeta() throws IOException {
// put current -ROOT- and .META. aside.
Path hbaseDir = new Path(conf.get(HConstants.HBASE_DIR));
FileSystem fs = hbaseDir.getFileSystem(conf);
long now = System.currentTimeMillis();
Path backupDir = new Path(hbaseDir.getParent(), hbaseDir.getName() + "-"
+ now);
fs.mkdirs(backupDir);
sidelineTable(fs, HConstants.ROOT_TABLE_NAME, hbaseDir, backupDir);
try {
sidelineTable(fs, HConstants.META_TABLE_NAME, hbaseDir, backupDir);
} catch (IOException e) {
LOG.error("Attempt to sideline meta failed, attempt to revert...", e);
try {
// move it back.
sidelineTable(fs, HConstants.ROOT_TABLE_NAME, backupDir, hbaseDir);
LOG.warn("... revert succeed. -ROOT- and .META. still in "
+ "original state.");
} catch (IOException ioe) {
LOG.fatal("... failed to sideline root and meta and failed to restore "
+ "prevoius state. Currently in inconsistent state. To restore "
+ "try to rename -ROOT- in " + backupDir.getName() + " to "
+ hbaseDir.getName() + ".", ioe);
}
throw e; // throw original exception
}
return backupDir;
}
/**
* Load the list of disabled tables in ZK into local set.
* @throws ZooKeeperConnectionException
@ -284,7 +536,7 @@ public class HBaseFsck {
* Scan HDFS for all regions, recording their information into
* regionInfo
*/
void checkHdfs() throws IOException, InterruptedException {
public void checkHdfs() throws IOException, InterruptedException {
Path rootDir = new Path(conf.get(HConstants.HBASE_DIR));
FileSystem fs = rootDir.getFileSystem(conf);
@ -647,8 +899,8 @@ public class HBaseFsck {
// TODO offline fix region hole.
errors.reportError(ERROR_CODE.FIRST_REGION_STARTKEY_NOT_EMPTY,
"First region should start with an empty key. When HBase is "
+ "online, create a new regio to plug the hole using hbck -fix",
"First region should start with an empty key. You need to "
+ " create a new region and regioninfo in HDFS to plug the hole.",
this, rng);
}
}
@ -714,23 +966,27 @@ public class HBaseFsck {
if (holeStopKey != null) {
// hole
errors.reportError(ERROR_CODE.HOLE_IN_REGION_CHAIN,
"There is a hole in the region chain between "
"There is a hole in the region chain between "
+ Bytes.toStringBinary(key) + " and "
+ Bytes.toStringBinary(holeStopKey)
+ ". When HBase is online, create a new regioninfo and region "
+ "dir to plug the hole.");
+ ". You need to create a new regioninfo and region "
+ "dir in hdfs to plug the hole.");
}
}
}
prevKey = key;
}
if (details) {
// do full region split map dump
System.out.println("---- Table '" + this.tableName
+ "': region split map");
dump(splits, regions);
System.out.println("---- Table '" + this.tableName
+ "': overlap groups");
dumpOverlapProblems(overlapGroups);
System.out.println("There are " + overlapGroups.keySet().size()
+ " problem groups with " + overlapGroups.size()
+ " problem regions");
+ " overlap groups with " + overlapGroups.size()
+ " overlapping regions");
}
return errors.getErrorList().size() == originalErrorsCount;
}
@ -744,10 +1000,10 @@ public class HBaseFsck {
void dump(SortedSet<byte[]> splits, Multimap<byte[], HbckInfo> regions) {
// we display this way because the last end key should be displayed as well.
for (byte[] k : splits) {
System.out.print(Bytes.toString(k) + ":\t");
System.out.print(Bytes.toStringBinary(k) + ":\t");
for (HbckInfo r : regions.get(k)) {
System.out.print("[ "+ r.toString() + ", "
+ Bytes.toString(r.getEndKey())+ "]\t");
System.out.print("[ "+ r.toString() + ", "
+ Bytes.toStringBinary(r.getEndKey())+ "]\t");
}
System.out.println();
}
@ -947,6 +1203,7 @@ public class HBaseFsck {
* Stores the entries scanned from META
*/
static class MetaEntry extends HRegionInfo {
private static final Log LOG = LogFactory.getLog(HRegionInfo.class);
ServerName regionServer; // server hosting this region
long modTime; // timestamp of most recent modification metadata
@ -965,6 +1222,7 @@ public class HBaseFsck {
MetaEntry metaEntry = null;
FileStatus foundRegionDir = null;
List<ServerName> deployedOn = Lists.newArrayList();
String hdfsTableName = null; // This is set in the workitem loader.
HbckInfo(MetaEntry metaEntry) {
this.metaEntry = metaEntry;
@ -1052,7 +1310,7 @@ public class HBaseFsck {
}
}
interface ErrorReporter {
public interface ErrorReporter {
public static enum ERROR_CODE {
UNKNOWN, NO_META_REGION, NULL_ROOT_REGION, NO_VERSION_FILE, NOT_IN_META_HDFS, NOT_IN_META,
NOT_IN_META_OR_DEPLOYED, NOT_IN_HDFS_OR_DEPLOYED, NOT_IN_HDFS, SERVER_DOES_NOT_MATCH_META, NOT_DEPLOYED,
@ -1291,6 +1549,7 @@ public class HBaseFsck {
if (!encodedName.toLowerCase().matches("[0-9a-f]+")) continue;
HbckInfo hbi = hbck.getOrCreateInfo(encodedName);
hbi.hdfsTableName = tableName;
synchronized (hbi) {
if (hbi.foundRegionDir != null) {
errors.print("Directory " + encodedName + " duplicate??" +
@ -1323,10 +1582,10 @@ public class HBaseFsck {
}
/**
* Display the full report from fsck.
* This displays all live and dead region servers, and all known regions.
* Display the full report from fsck. This displays all live and dead region
* servers, and all known regions.
*/
void displayFullReport() {
public void displayFullReport() {
details = true;
}
@ -1364,7 +1623,7 @@ public class HBaseFsck {
* Fix inconsistencies found by fsck. This should try to fix errors (if any)
* found by fsck utility.
*/
void setFixErrors(boolean shouldFix) {
public void setFixErrors(boolean shouldFix) {
fix = shouldFix;
}
@ -1377,7 +1636,7 @@ public class HBaseFsck {
* META during the last few seconds specified by hbase.admin.fsck.timelag
* @param seconds - the time in seconds
*/
void setTimeLag(long seconds) {
public void setTimeLag(long seconds) {
timelag = seconds * 1000; // convert to milliseconds
}

View File

@ -0,0 +1,101 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.util.hbck;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.util.HBaseFsck;
import org.apache.hadoop.io.MultipleIOException;
/**
* This code is used to rebuild meta off line from file system data. If there
* are any problem detected, it will fail suggesting actions for the user to do
* to "fix" problems. If it succeeds, it will backup the previous .META. and
* -ROOT- dirs and write new tables in place.
*
* This is an advanced feature, so is only exposed for use if explicitly
* mentioned.
*
* hbase org.apache.hadoop.hbase.util.hbck.OfflineMetaRepair ...
*/
public class OfflineMetaRepair {
private static final Log LOG = LogFactory.getLog(HBaseFsck.class.getName());
HBaseFsck fsck;
protected static void printUsageAndExit() {
System.err.println("Usage: OfflineMetaRepair [opts] ");
System.err.println(" where [opts] are:");
System.err
.println(" -details Display full report of all regions.");
System.err.println(" -base <hdfs://> Base Hbase Data directory");
Runtime.getRuntime().exit(-2);
}
/**
* Main program
*
* @param args
* @throws Exception
*/
public static void main(String[] args) throws Exception {
// create a fsck object
Configuration conf = HBaseConfiguration.create();
conf.set("fs.defaultFS", conf.get(HConstants.HBASE_DIR));
HBaseFsck fsck = new HBaseFsck(conf);
// Process command-line args.
for (int i = 0; i < args.length; i++) {
String cmd = args[i];
if (cmd.equals("-details")) {
fsck.displayFullReport();
} else if (cmd.equals("-base")) {
// update hbase root dir to user-specified base
i++;
String path = args[i];
conf.set(HConstants.HBASE_DIR, path);
conf.set("fs.defaultFS", conf.get(HConstants.HBASE_DIR));
} else {
String str = "Unknown command line option : " + cmd;
LOG.info(str);
System.out.println(str);
printUsageAndExit();
}
}
// Fsck doesn't shutdown and and doesn't provide a way to shutdown its
// threads cleanly, so we do a System.exit.
boolean success = false;
try {
success = fsck.rebuildMeta();
} catch (MultipleIOException mioes) {
for (IOException ioe : mioes.getExceptions()) {
LOG.error("Bailed out due to:", ioe);
}
} catch (Exception e) {
LOG.error("Bailed out due to: ", e);
} finally {
System.exit(success ? 0 : 1);
}
}
}

View File

@ -19,10 +19,12 @@
*/
package org.apache.hadoop.hbase.util;
import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.assertErrors;
import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.assertNoErrors;
import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.doFsck;
import static org.junit.Assert.assertEquals;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
@ -38,7 +40,6 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HBaseAdmin;
@ -64,7 +65,9 @@ public class TestHBaseFsck {
// for the instance, reset every test run
private HTable tbl;
private final static byte[][] splits= new byte[][] { Bytes.toBytes("A"),
Bytes.toBytes("B"), Bytes.toBytes("C") };
@BeforeClass
public static void setUpBeforeClass() throws Exception {
TEST_UTIL.startMiniCluster(3);
@ -75,34 +78,14 @@ public class TestHBaseFsck {
TEST_UTIL.shutdownMiniCluster();
}
private HBaseFsck doFsck(boolean fix) throws Exception {
HBaseFsck fsck = new HBaseFsck(conf);
fsck.connect();
fsck.displayFullReport(); // i.e. -details
fsck.setTimeLag(0);
fsck.setFixErrors(fix);
fsck.doWork();
return fsck;
}
private void assertNoErrors(HBaseFsck fsck) throws Exception {
List<ERROR_CODE> errs = fsck.getErrors().getErrorList();
assertEquals(0, errs.size());
}
private void assertErrors(HBaseFsck fsck, ERROR_CODE[] expectedErrors) {
List<ERROR_CODE> errs = fsck.getErrors().getErrorList();
assertEquals(Arrays.asList(expectedErrors), errs);
}
@Test
public void testHBaseFsck() throws Exception {
assertNoErrors(doFsck(false));
assertNoErrors(doFsck(conf, false));
String table = "tableBadMetaAssign";
TEST_UTIL.createTable(Bytes.toBytes(table), FAM);
// We created 1 table, should be fine
assertNoErrors(doFsck(false));
assertNoErrors(doFsck(conf, false));
// Now let's mess it up and change the assignment in .META. to
// point to a different region server
@ -134,11 +117,11 @@ public class TestHBaseFsck {
}
// Try to fix the data
assertErrors(doFsck(true), new ERROR_CODE[]{
assertErrors(doFsck(conf, true), new ERROR_CODE[]{
ERROR_CODE.SERVER_DOES_NOT_MATCH_META});
// Should be fixed now
assertNoErrors(doFsck(false));
assertNoErrors(doFsck(conf, false));
// comment needed - what is the purpose of this line
new HTable(conf, Bytes.toBytes(table)).getScanner(new Scan());
@ -206,16 +189,16 @@ public class TestHBaseFsck {
* @throws InterruptedException
* @throws KeeperException
*/
void setupTable(String tablename) throws Exception {
byte[][] startKeys = new byte[][] { Bytes.toBytes("A"), Bytes.toBytes("B"),
Bytes.toBytes("C") };
HTable setupTable(String tablename) throws Exception {
HTableDescriptor desc = new HTableDescriptor(tablename);
HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toString(FAM));
desc.addFamily(hcd); // If a table has no CF's it doesn't get checked
TEST_UTIL.getHBaseAdmin().createTable(desc, startKeys);
TEST_UTIL.getHBaseAdmin().createTable(desc, splits);
tbl = new HTable(TEST_UTIL.getConfiguration(), tablename);
return tbl;
}
/**
* delete table in preparation for next test
*
@ -236,16 +219,16 @@ public class TestHBaseFsck {
*/
@Test
public void testHBaseFsckClean() throws Exception {
assertNoErrors(doFsck(false));
assertNoErrors(doFsck(conf, false));
String table = "tableClean";
try {
HBaseFsck hbck = doFsck(false);
HBaseFsck hbck = doFsck(conf, false);
assertNoErrors(hbck);
setupTable(table);
// We created 1 table, should be fine
hbck = doFsck( false);
hbck = doFsck(conf, false);
assertNoErrors(hbck);
assertEquals(0, hbck.getOverlapGroups(table).size());
} finally {
@ -261,7 +244,7 @@ public class TestHBaseFsck {
String table = "tableDupeStartKey";
try {
setupTable(table);
assertNoErrors(doFsck(false));
assertNoErrors(doFsck(conf, false));
// Now let's mess it up, by adding a region with a duplicate startkey
HRegionInfo hriDupe = createRegion(conf, tbl.getTableDescriptor(),
@ -270,7 +253,7 @@ public class TestHBaseFsck {
TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager()
.waitForAssignment(hriDupe);
HBaseFsck hbck = doFsck(false);
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.DUPE_STARTKEYS,
ERROR_CODE.DUPE_STARTKEYS});
assertEquals(2, hbck.getOverlapGroups(table).size());
@ -287,7 +270,7 @@ public class TestHBaseFsck {
String table = "tableDegenerateRegions";
try {
setupTable(table);
assertNoErrors(doFsck(false));
assertNoErrors(doFsck(conf,false));
// Now let's mess it up, by adding a region with a duplicate startkey
HRegionInfo hriDupe = createRegion(conf, tbl.getTableDescriptor(),
@ -296,7 +279,7 @@ public class TestHBaseFsck {
TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager()
.waitForAssignment(hriDupe);
HBaseFsck hbck = doFsck(false);
HBaseFsck hbck = doFsck(conf,false);
assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.DEGENERATE_REGION,
ERROR_CODE.DUPE_STARTKEYS, ERROR_CODE.DUPE_STARTKEYS});
assertEquals(2, hbck.getOverlapGroups(table).size());
@ -321,7 +304,7 @@ public class TestHBaseFsck {
TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager()
.waitForAssignment(hriOverlap);
HBaseFsck hbck = doFsck(false);
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck, new ERROR_CODE[] {
ERROR_CODE.OVERLAP_IN_REGION_CHAIN,
ERROR_CODE.OVERLAP_IN_REGION_CHAIN });
@ -351,7 +334,7 @@ public class TestHBaseFsck {
deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("C"), Bytes.toBytes(""));
TEST_UTIL.getHBaseAdmin().enableTable(table);
HBaseFsck hbck = doFsck(false);
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.HOLE_IN_REGION_CHAIN });
// holes are separate from overlap groups
assertEquals(0, hbck.getOverlapGroups(table).size());
@ -359,5 +342,4 @@ public class TestHBaseFsck {
deleteTable(table);
}
}
}

View File

@ -0,0 +1,49 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.util.hbck;
import static org.junit.Assert.assertEquals;
import java.util.Arrays;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.util.HBaseFsck;
import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
public class HbckTestingUtil {
public static HBaseFsck doFsck(Configuration conf, boolean fix) throws Exception {
HBaseFsck fsck = new HBaseFsck(conf);
fsck.connect();
fsck.displayFullReport(); // i.e. -details
fsck.setTimeLag(0);
fsck.setFixErrors(fix);
fsck.doWork();
return fsck;
}
public static void assertNoErrors(HBaseFsck fsck) throws Exception {
List<ERROR_CODE> errs = fsck.getErrors().getErrorList();
assertEquals(0, errs.size());
}
public static void assertErrors(HBaseFsck fsck, ERROR_CODE[] expectedErrors) {
List<ERROR_CODE> errs = fsck.getErrors().getErrorList();
assertEquals(Arrays.asList(expectedErrors), errs);
}
}

View File

@ -0,0 +1,277 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.util.hbck;
import static org.junit.Assert.assertEquals;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.zookeeper.KeeperException;
import org.junit.After;
import org.junit.Before;
/**
* This testing base class creates a minicluster and testing table table
* and shuts down the cluster afterwards. It also provides methods wipes out
* meta and to inject errors into meta and the file system.
*
* Tests should generally break stuff, then attempt to rebuild the meta table
* offline, then restart hbase, and finally perform checks.
*
* NOTE: This is a slow set of tests which takes ~30s each needs to run on a
* relatively beefy machine. It seems necessary to have each test in a new jvm
* since minicluster startup and tear downs seem to leak file handles and
* eventually cause out of file handle exceptions.
*/
public class OfflineMetaRebuildTestCore {
protected final static Log LOG = LogFactory
.getLog(OfflineMetaRebuildTestCore.class);
protected HBaseTestingUtility TEST_UTIL;
protected Configuration conf;
private final static byte[] FAM = Bytes.toBytes("fam");
// for the instance, reset every test run
protected HTable htbl;
protected final static byte[][] splits = new byte[][] { Bytes.toBytes("A"),
Bytes.toBytes("B"), Bytes.toBytes("C") };
private final static String TABLE_BASE = "tableMetaRebuild";
private static int tableIdx = 0;
protected String table = "tableMetaRebuild";
@Before
public void setUpBefore() throws Exception {
TEST_UTIL = new HBaseTestingUtility();
TEST_UTIL.getConfiguration().setInt("dfs.datanode.max.xceivers", 9192);
TEST_UTIL.startMiniCluster(3);
conf = TEST_UTIL.getConfiguration();
assertEquals(0, TEST_UTIL.getHBaseAdmin().listTables().length);
// setup the table
table = TABLE_BASE + "-" + tableIdx;
tableIdx++;
htbl = setupTable(table);
populateTable(htbl);
assertEquals(4, scanMeta());
LOG.info("Table " + table + " has " + tableRowCount(conf, table)
+ " entries.");
assertEquals(16, tableRowCount(conf, table));
TEST_UTIL.getHBaseAdmin().disableTable(table);
assertEquals(1, TEST_UTIL.getHBaseAdmin().listTables().length);
}
@After
public void tearDownAfter() throws Exception {
TEST_UTIL.shutdownMiniCluster();
HConnectionManager.deleteConnection(conf, true);
}
/**
* Setup a clean table before we start mucking with it.
*
* @throws IOException
* @throws InterruptedException
* @throws KeeperException
*/
private HTable setupTable(String tablename) throws Exception {
HTableDescriptor desc = new HTableDescriptor(tablename);
HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toString(FAM));
desc.addFamily(hcd); // If a table has no CF's it doesn't get checked
TEST_UTIL.getHBaseAdmin().createTable(desc, splits);
return new HTable(TEST_UTIL.getConfiguration(), tablename);
}
private void dumpMeta(HTableDescriptor htd) throws IOException {
List<byte[]> metaRows = TEST_UTIL.getMetaTableRows(htd.getName());
for (byte[] row : metaRows) {
LOG.info(Bytes.toString(row));
}
}
private void populateTable(HTable tbl) throws IOException {
byte[] values = { 'A', 'B', 'C', 'D' };
for (int i = 0; i < values.length; i++) {
for (int j = 0; j < values.length; j++) {
Put put = new Put(new byte[] { values[i], values[j] });
put.add(Bytes.toBytes("fam"), new byte[] {}, new byte[] { values[i],
values[j] });
tbl.put(put);
}
}
tbl.flushCommits();
}
/**
* delete table in preparation for next test
*
* @param tablename
* @throws IOException
*/
void deleteTable(HBaseAdmin admin, String tablename) throws IOException {
try {
byte[] tbytes = Bytes.toBytes(tablename);
admin.disableTable(tbytes);
admin.deleteTable(tbytes);
} catch (Exception e) {
// Do nothing.
}
}
protected void deleteRegion(Configuration conf, final HTable tbl,
byte[] startKey, byte[] endKey) throws IOException {
LOG.info("Before delete:");
HTableDescriptor htd = tbl.getTableDescriptor();
dumpMeta(htd);
Map<HRegionInfo, HServerAddress> hris = tbl.getRegionsInfo();
for (Entry<HRegionInfo, HServerAddress> e : hris.entrySet()) {
HRegionInfo hri = e.getKey();
HServerAddress hsa = e.getValue();
if (Bytes.compareTo(hri.getStartKey(), startKey) == 0
&& Bytes.compareTo(hri.getEndKey(), endKey) == 0) {
LOG.info("RegionName: " + hri.getRegionNameAsString());
byte[] deleteRow = hri.getRegionName();
TEST_UTIL.getHBaseAdmin().unassign(deleteRow, true);
LOG.info("deleting hdfs data: " + hri.toString() + hsa.toString());
Path rootDir = new Path(conf.get(HConstants.HBASE_DIR));
FileSystem fs = rootDir.getFileSystem(conf);
Path p = new Path(rootDir + "/" + htd.getNameAsString(),
hri.getEncodedName());
fs.delete(p, true);
HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
Delete delete = new Delete(deleteRow);
meta.delete(delete);
}
LOG.info(hri.toString() + hsa.toString());
}
TEST_UTIL.getMetaTableRows(htd.getName());
LOG.info("After delete:");
dumpMeta(htd);
}
protected HRegionInfo createRegion(Configuration conf, final HTable htbl,
byte[] startKey, byte[] endKey) throws IOException {
HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
HTableDescriptor htd = htbl.getTableDescriptor();
HRegionInfo hri = new HRegionInfo(htbl.getTableName(), startKey, endKey);
LOG.info("manually adding regioninfo and hdfs data: " + hri.toString());
Path rootDir = new Path(conf.get(HConstants.HBASE_DIR));
FileSystem fs = rootDir.getFileSystem(conf);
Path p = new Path(rootDir + "/" + htd.getNameAsString(),
hri.getEncodedName());
fs.mkdirs(p);
Path riPath = new Path(p, HRegion.REGIONINFO_FILE);
FSDataOutputStream out = fs.create(riPath);
hri.write(out);
out.close();
// add to meta.
Put put = new Put(hri.getRegionName());
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(hri));
meta.put(put);
meta.flushCommits();
return hri;
}
protected void wipeOutMeta() throws IOException {
// Mess it up by blowing up meta.
HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
Scan s = new Scan();
HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
ResultScanner scanner = meta.getScanner(s);
List<Delete> dels = new ArrayList<Delete>();
for (Result r : scanner) {
Delete d = new Delete(r.getRow());
dels.add(d);
admin.unassign(r.getRow(), true);
}
meta.delete(dels);
meta.flushCommits();
}
/**
* Returns the number of rows in a given table. HBase must be up and the table
* should be present (will wait for timeout for a while otherwise)
*
* @return # of rows in the specified table
*/
protected int tableRowCount(Configuration conf, String table)
throws IOException {
HTable t = new HTable(conf, table);
Scan st = new Scan();
ResultScanner rst = t.getScanner(st);
int count = 0;
for (@SuppressWarnings("unused")
Result rt : rst) {
count++;
}
return count;
}
/**
* Dumps .META. table info
*
* @return # of entries in meta.
*/
protected int scanMeta() throws IOException {
int count = 0;
HTable meta = new HTable(conf, HTableDescriptor.META_TABLEDESC.getName());
ResultScanner scanner = meta.getScanner(new Scan());
LOG.info("Table: " + Bytes.toString(meta.getTableName()));
for (Result res : scanner) {
LOG.info(Bytes.toString(res.getRow()));
count++;
}
return count;
}
}

View File

@ -0,0 +1,77 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.util.hbck;
import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.assertErrors;
import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.doFsck;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.util.Arrays;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.util.HBaseFsck;
import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
import org.junit.Test;
/**
* This builds a table, removes info from meta, and then rebuilds meta.
*/
public class TestOfflineMetaRebuildBase extends OfflineMetaRebuildTestCore {
@Test(timeout = 120000)
public void testMetaRebuild() throws Exception {
wipeOutMeta();
// is meta really messed up?
assertEquals(0, scanMeta());
assertErrors(doFsck(conf, false),
new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
ERROR_CODE.NOT_IN_META_OR_DEPLOYED, });
// Note, would like to check # of tables, but this takes a while to time
// out.
// shutdown the minicluster
TEST_UTIL.shutdownMiniHBaseCluster();
TEST_UTIL.shutdownMiniZKCluster();
HConnectionManager.deleteConnection(conf, false);
// rebuild meta table from scratch
HBaseFsck fsck = new HBaseFsck(conf);
assertTrue(fsck.rebuildMeta());
// bring up the minicluster
TEST_UTIL.startMiniZKCluster(); // tables seem enabled by default
TEST_UTIL.restartHBaseCluster(3);
// everything is good again.
assertEquals(4, scanMeta());
HTableDescriptor[] htbls = TEST_UTIL.getHBaseAdmin().listTables();
LOG.info("Tables present after restart: " + Arrays.toString(htbls));
assertEquals(1, htbls.length);
assertErrors(doFsck(conf, false), new ERROR_CODE[] {});
LOG.info("Table " + table + " has " + tableRowCount(conf, table)
+ " entries.");
assertEquals(16, tableRowCount(conf, table));
}
}

View File

@ -0,0 +1,79 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.util.hbck;
import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.assertErrors;
import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.doFsck;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import java.util.Arrays;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.util.HBaseFsck;
import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
import org.junit.Test;
/**
* This builds a table, removes info from meta, and then fails when attempting
* to rebuild meta.
*/
public class TestOfflineMetaRebuildHole extends OfflineMetaRebuildTestCore {
@Test(timeout = 120000)
public void testMetaRebuildHoleFail() throws Exception {
// Fully remove a meta entry and hdfs region
byte[] startKey = splits[1];
byte[] endKey = splits[2];
deleteRegion(conf, htbl, startKey, endKey);
wipeOutMeta();
// is meta really messed up?
assertEquals(0, scanMeta());
assertErrors(doFsck(conf, false), new ERROR_CODE[] {
ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
ERROR_CODE.NOT_IN_META_OR_DEPLOYED, });
// Note, would like to check # of tables, but this takes a while to time
// out.
// shutdown the minicluster
TEST_UTIL.shutdownMiniHBaseCluster();
TEST_UTIL.shutdownMiniZKCluster();
// attempt to rebuild meta table from scratch
HBaseFsck fsck = new HBaseFsck(conf);
assertFalse(fsck.rebuildMeta());
// bring up the minicluster
TEST_UTIL.startMiniZKCluster(); // tables seem enabled by default
TEST_UTIL.restartHBaseCluster(3);
// Meta still messed up.
assertEquals(0, scanMeta());
HTableDescriptor[] htbls = TEST_UTIL.getHBaseAdmin().listTables();
LOG.info("Tables present after restart: " + Arrays.toString(htbls));
// After HBASE-451 HBaseAdmin.listTables() gets table descriptors from FS,
// so the table is still present and this should be 1.
assertEquals(1, htbls.length);
assertErrors(doFsck(conf, false), new ERROR_CODE[] {
ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
ERROR_CODE.NOT_IN_META_OR_DEPLOYED, });
}
}

View File

@ -0,0 +1,90 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.util.hbck;
import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.assertErrors;
import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.doFsck;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import java.util.Arrays;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.util.HBaseFsck;
import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
import org.apache.hadoop.hbase.util.HBaseFsck.HbckInfo;
import org.junit.Test;
import com.google.common.collect.Multimap;
/**
* This builds a table, builds an overlap, and then fails when attempting to
* rebuild meta.
*/
public class TestOfflineMetaRebuildOverlap extends OfflineMetaRebuildTestCore {
@Test(timeout = 120000)
public void testMetaRebuildOverlapFail() throws Exception {
// Add a new .regioninfo meta entry in hdfs
byte[] startKey = splits[0];
byte[] endKey = splits[2];
createRegion(conf, htbl, startKey, endKey);
wipeOutMeta();
// is meta really messed up?
assertEquals(0, scanMeta());
assertErrors(doFsck(conf, false),
new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
ERROR_CODE.NOT_IN_META_OR_DEPLOYED, });
// Note, would like to check # of tables, but this takes a while to time
// out.
// shutdown the minicluster
TEST_UTIL.shutdownMiniHBaseCluster();
TEST_UTIL.shutdownMiniZKCluster();
// attempt to rebuild meta table from scratch
HBaseFsck fsck = new HBaseFsck(conf);
assertFalse(fsck.rebuildMeta());
Multimap<byte[], HbckInfo> problems = fsck.getOverlapGroups(table);
assertEquals(1, problems.keySet().size());
assertEquals(3, problems.size());
// bring up the minicluster
TEST_UTIL.startMiniZKCluster(); // tables seem enabled by default
TEST_UTIL.restartHBaseCluster(3);
// Meta still messed up.
assertEquals(0, scanMeta());
HTableDescriptor[] htbls = TEST_UTIL.getHBaseAdmin().listTables();
LOG.info("Tables present after restart: " + Arrays.toString(htbls));
// After HBASE-451 HBaseAdmin.listTables() gets table descriptors from FS,
// so the table is still present and this should be 1.
assertEquals(1, htbls.length);
assertErrors(doFsck(conf, false),
new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
ERROR_CODE.NOT_IN_META_OR_DEPLOYED, });
}
}