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:
parent
3fab67a8b5
commit
061c22b302
|
@ -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
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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)) {
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
|
||||
}
|
|
@ -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, });
|
||||
}
|
||||
}
|
|
@ -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, });
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue