HBASE-5719 Enhance hbck to sideline overlapped mega regions (Jimmy Xiang)
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1325404 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
38eb4579e4
commit
36324e341d
|
@ -22,8 +22,10 @@ import java.util.ArrayList;
|
|||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Set;
|
||||
import java.util.SortedSet;
|
||||
|
@ -77,7 +79,6 @@ import org.apache.hadoop.hbase.util.hbck.TableIntegrityErrorHandler;
|
|||
import org.apache.hadoop.hbase.util.hbck.TableIntegrityErrorHandlerImpl;
|
||||
import org.apache.hadoop.hbase.zookeeper.RootRegionTracker;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKTable;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
|
@ -140,6 +141,7 @@ public class HBaseFsck {
|
|||
private static final int MAX_NUM_THREADS = 50; // #threads to contact regions
|
||||
private static final long THREADS_KEEP_ALIVE_SECONDS = 60;
|
||||
private static boolean rsSupportsOffline = true;
|
||||
private static final int DEFAULT_OVERLAPS_TO_SIDELINE = 2;
|
||||
private static final int DEFAULT_MAX_MERGE = 5;
|
||||
|
||||
/**********************
|
||||
|
@ -152,7 +154,6 @@ public class HBaseFsck {
|
|||
private HBaseAdmin admin;
|
||||
private HTable meta;
|
||||
private ThreadPoolExecutor executor; // threads to retrieve data from regionservers
|
||||
private int numThreads = MAX_NUM_THREADS;
|
||||
private long startMillis = System.currentTimeMillis();
|
||||
|
||||
/***********
|
||||
|
@ -170,6 +171,8 @@ public class HBaseFsck {
|
|||
// limit fixes to listed tables, if empty atttempt to fix all
|
||||
private List<byte[]> tablesToFix = new ArrayList<byte[]>();
|
||||
private int maxMerge = DEFAULT_MAX_MERGE; // maximum number of overlapping regions to merge
|
||||
private int maxOverlapsToSideline = DEFAULT_OVERLAPS_TO_SIDELINE; // maximum number of overlapping regions to sideline
|
||||
private boolean sidelineBigOverlaps = false; // sideline overlaps with >maxMerge regions
|
||||
|
||||
private boolean rerun = false; // if we tried to fix something, rerun hbck
|
||||
private static boolean summary = false; // if we want to print less output
|
||||
|
@ -215,7 +218,7 @@ public class HBaseFsck {
|
|||
ZooKeeperConnectionException, IOException {
|
||||
this.conf = conf;
|
||||
|
||||
int numThreads = conf.getInt("hbasefsck.numthreads", Integer.MAX_VALUE);
|
||||
int numThreads = conf.getInt("hbasefsck.numthreads", MAX_NUM_THREADS);
|
||||
executor = new ThreadPoolExecutor(1, numThreads,
|
||||
THREADS_KEEP_ALIVE_SECONDS, TimeUnit.SECONDS,
|
||||
new SynchronousQueue<Runnable>());
|
||||
|
@ -834,14 +837,14 @@ public class HBaseFsck {
|
|||
/**
|
||||
* Sideline a region dir (instead of deleting it)
|
||||
*/
|
||||
void sidelineRegionDir(FileSystem fs, HbckInfo hi)
|
||||
Path sidelineRegionDir(FileSystem fs, HbckInfo hi)
|
||||
throws IOException {
|
||||
String tableName = Bytes.toString(hi.getTableName());
|
||||
Path regionDir = hi.getHdfsRegionDir();
|
||||
|
||||
if (!fs.exists(regionDir)) {
|
||||
LOG.warn("No previous " + regionDir + " exists. Continuing.");
|
||||
return;
|
||||
return null;
|
||||
}
|
||||
|
||||
Path sidelineTableDir= new Path(getSidelineDir(), tableName);
|
||||
|
@ -874,7 +877,9 @@ public class HBaseFsck {
|
|||
// dst (foo/a) exists and is a dir, and the src (foo/b) is a dir,
|
||||
// it moves the src into the dst dir resulting in (foo/a/b). If
|
||||
// the dst does not exist, and the src a dir, src becomes dst. (foo/b)
|
||||
for (FileStatus hfile : fs.listStatus(src)) {
|
||||
FileStatus[] hfiles = fs.listStatus(src);
|
||||
if (hfiles != null && hfiles.length > 0) {
|
||||
for (FileStatus hfile : hfiles) {
|
||||
success = fs.rename(hfile.getPath(), dst);
|
||||
if (!success) {
|
||||
String msg = "Unable to rename file " + src + " to " + dst;
|
||||
|
@ -882,6 +887,7 @@ public class HBaseFsck {
|
|||
throw new IOException(msg);
|
||||
}
|
||||
}
|
||||
}
|
||||
LOG.debug("Sideline directory contents:");
|
||||
debugLsr(sidelineRegionDir);
|
||||
}
|
||||
|
@ -894,6 +900,7 @@ public class HBaseFsck {
|
|||
LOG.error(msg);
|
||||
throw new IOException(msg);
|
||||
}
|
||||
return sidelineRegionDir;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1552,6 +1559,9 @@ public class HBaseFsck {
|
|||
// backwards regions
|
||||
final List<HbckInfo> backwards = new ArrayList<HbckInfo>();
|
||||
|
||||
// sidelined big overlapped regions
|
||||
final Map<Path, HbckInfo> sidelinedRegions = new HashMap<Path, HbckInfo>();
|
||||
|
||||
// region split calculator
|
||||
final RegionSplitCalculator<HbckInfo> sc = new RegionSplitCalculator<HbckInfo>(cmp);
|
||||
|
||||
|
@ -1573,6 +1583,9 @@ public class HBaseFsck {
|
|||
private HTableDescriptor getHTD() {
|
||||
if (htds.size() == 1) {
|
||||
return (HTableDescriptor)htds.toArray()[0];
|
||||
} else {
|
||||
LOG.error("None/Multiple table descriptors found for table '"
|
||||
+ tableName + "' regions: " + htds);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
@ -1753,11 +1766,19 @@ public class HBaseFsck {
|
|||
|
||||
if (overlap.size() > maxMerge) {
|
||||
LOG.warn("Overlap group has " + overlap.size() + " overlapping " +
|
||||
"regions which is greater than " + maxMerge + ", the max " +
|
||||
"number of regions to merge.");
|
||||
"regions which is greater than " + maxMerge + ", the max number of regions to merge");
|
||||
if (sidelineBigOverlaps) {
|
||||
// we only sideline big overlapped groups that exceeds the max number of regions to merge
|
||||
sidelineBigOverlaps(overlap);
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
||||
mergeOverlaps(overlap);
|
||||
}
|
||||
|
||||
void mergeOverlaps(Collection<HbckInfo> overlap)
|
||||
throws IOException {
|
||||
LOG.info("== Merging regions into one region: "
|
||||
+ Joiner.on(",").join(overlap));
|
||||
// get the min / max range and close all concerned regions
|
||||
|
@ -1823,7 +1844,48 @@ public class HBaseFsck {
|
|||
fixes++;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
/**
|
||||
* Sideline some regions in a big overlap group so that it
|
||||
* will have fewer regions, and it is easier to merge them later on.
|
||||
*
|
||||
* @param bigOverlap the overlapped group with regions more than maxMerge
|
||||
* @throws IOException
|
||||
*/
|
||||
void sidelineBigOverlaps(
|
||||
Collection<HbckInfo> bigOverlap) throws IOException {
|
||||
int overlapsToSideline = bigOverlap.size() - maxMerge;
|
||||
if (overlapsToSideline > maxOverlapsToSideline) {
|
||||
overlapsToSideline = maxOverlapsToSideline;
|
||||
}
|
||||
List<HbckInfo> regionsToSideline =
|
||||
RegionSplitCalculator.findBigRanges(bigOverlap, overlapsToSideline);
|
||||
FileSystem fs = FileSystem.get(conf);
|
||||
for (HbckInfo regionToSideline: regionsToSideline) {
|
||||
try {
|
||||
LOG.info("Closing region: " + regionToSideline);
|
||||
closeRegion(regionToSideline);
|
||||
} catch (InterruptedException ie) {
|
||||
LOG.warn("Was unable to close region " + regionToSideline.getRegionNameAsString()
|
||||
+ ". Interrupted.");
|
||||
throw new IOException(ie);
|
||||
}
|
||||
|
||||
LOG.info("Offlining region: " + regionToSideline);
|
||||
offline(regionToSideline.getRegionName());
|
||||
|
||||
LOG.info("Before sideline big overlapped region: " + regionToSideline.toString());
|
||||
Path sidelineRegionDir = sidelineRegionDir(fs, regionToSideline);
|
||||
if (sidelineRegionDir != null) {
|
||||
sidelinedRegions.put(sidelineRegionDir, regionToSideline);
|
||||
LOG.info("After sidelined big overlapped region: "
|
||||
+ regionToSideline.getRegionNameAsString()
|
||||
+ " to " + sidelineRegionDir.toString());
|
||||
fixes++;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Check the region chain (from META) of this table. We are looking for
|
||||
|
@ -1919,6 +1981,12 @@ public class HBaseFsck {
|
|||
+ " overlap groups with " + overlapGroups.size()
|
||||
+ " overlapping regions");
|
||||
}
|
||||
if (!sidelinedRegions.isEmpty()) {
|
||||
LOG.warn("Sidelined big overlapped regions, please bulk load them!");
|
||||
System.out.println("---- Table '" + this.tableName
|
||||
+ "': sidelined big overlapped regions");
|
||||
dumpSidelinedRegions(sidelinedRegions);
|
||||
}
|
||||
return errors.getErrorList().size() == originalErrorsCount;
|
||||
}
|
||||
|
||||
|
@ -1954,6 +2022,13 @@ public class HBaseFsck {
|
|||
}
|
||||
}
|
||||
|
||||
public void dumpSidelinedRegions(Map<Path, HbckInfo> regions) {
|
||||
for (Path k : regions.keySet()) {
|
||||
System.out.println("To be bulk loaded sidelined region dir: "
|
||||
+ k.toString());
|
||||
}
|
||||
}
|
||||
|
||||
public Multimap<byte[], HbckInfo> getOverlapGroups(
|
||||
String table) {
|
||||
TableInfo ti = tablesInfo.get(table);
|
||||
|
@ -2761,6 +2836,14 @@ public class HBaseFsck {
|
|||
return fixVersionFile;
|
||||
}
|
||||
|
||||
public void setSidelineBigOverlaps(boolean sbo) {
|
||||
this.sidelineBigOverlaps = sbo;
|
||||
}
|
||||
|
||||
public boolean shouldSidelineBigOverlaps() {
|
||||
return sidelineBigOverlaps;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param mm maximum number of regions to merge into a single region.
|
||||
*/
|
||||
|
@ -2772,6 +2855,14 @@ public class HBaseFsck {
|
|||
return maxMerge;
|
||||
}
|
||||
|
||||
public void setMaxOverlapsToSideline(int mo) {
|
||||
this.maxOverlapsToSideline = mo;
|
||||
}
|
||||
|
||||
public int getMaxOverlapsToSideline() {
|
||||
return maxOverlapsToSideline;
|
||||
}
|
||||
|
||||
/**
|
||||
* Only fix tables specified by the list
|
||||
*/
|
||||
|
@ -2821,9 +2912,11 @@ public class HBaseFsck {
|
|||
System.err.println(" -fixHdfsOverlaps Try to fix region overlaps in hdfs.");
|
||||
System.err.println(" -fixVersionFile Try to fix missing hbase.version file in hdfs.");
|
||||
System.err.println(" -maxMerge <n> When fixing region overlaps, allow at most <n> regions to merge. (n=" + DEFAULT_MAX_MERGE +" by default)");
|
||||
System.err.println(" -sidelineBigOverlaps When fixing region overlaps, allow to sideline big overlaps");
|
||||
System.err.println(" -maxOverlapsToSideline <n> When fixing region overlaps, allow at most <n> regions to sideline per group. (n=" + DEFAULT_OVERLAPS_TO_SIDELINE +" by default)");
|
||||
System.err.println("");
|
||||
System.err.println(" -repair Shortcut for -fixAssignments -fixMeta -fixHdfsHoles " +
|
||||
"-fixHdfsOrphans -fixHdfsOverlaps -fixVersionFile");
|
||||
"-fixHdfsOrphans -fixHdfsOverlaps -fixVersionFile -sidelineBigOverlaps");
|
||||
System.err.println(" -repairHoles Shortcut for -fixAssignments -fixMeta -fixHdfsHoles -fixHdfsOrphans");
|
||||
|
||||
Runtime.getRuntime().exit(-2);
|
||||
|
@ -2888,6 +2981,8 @@ public class HBaseFsck {
|
|||
fsck.setFixHdfsOverlaps(true);
|
||||
} else if (cmd.equals("-fixVersionFile")) {
|
||||
fsck.setFixVersionFile(true);
|
||||
} else if (cmd.equals("-sidelineBigOverlaps")) {
|
||||
fsck.setSidelineBigOverlaps(true);
|
||||
} else if (cmd.equals("-repair")) {
|
||||
// this attempts to merge overlapping hdfs regions, needs testing
|
||||
// under load
|
||||
|
@ -2897,6 +2992,7 @@ public class HBaseFsck {
|
|||
fsck.setFixAssignments(true);
|
||||
fsck.setFixHdfsOverlaps(true);
|
||||
fsck.setFixVersionFile(true);
|
||||
fsck.setSidelineBigOverlaps(true);
|
||||
} else if (cmd.equals("-repairHoles")) {
|
||||
// this will make all missing hdfs regions available but may lose data
|
||||
fsck.setFixHdfsHoles(true);
|
||||
|
@ -2904,6 +3000,20 @@ public class HBaseFsck {
|
|||
fsck.setFixMeta(true);
|
||||
fsck.setFixAssignments(true);
|
||||
fsck.setFixHdfsOverlaps(false);
|
||||
fsck.setSidelineBigOverlaps(false);
|
||||
} else if (cmd.equals("-maxOverlapsToSideline")) {
|
||||
if (i == args.length - 1) {
|
||||
System.err.println("-maxOverlapsToSideline needs a numeric value argument.");
|
||||
printUsageAndExit();
|
||||
}
|
||||
try {
|
||||
int maxOverlapsToSideline = Integer.parseInt(args[i+1]);
|
||||
fsck.setMaxOverlapsToSideline(maxOverlapsToSideline);
|
||||
} catch (NumberFormatException e) {
|
||||
System.err.println("-maxOverlapsToSideline needs a numeric value argument.");
|
||||
printUsageAndExit();
|
||||
}
|
||||
i++;
|
||||
} else if (cmd.equals("-maxMerge")) {
|
||||
if (i == args.length - 1) {
|
||||
System.err.println("-maxMerge needs a numeric value argument.");
|
||||
|
|
|
@ -19,9 +19,12 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.util;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.TreeMap;
|
||||
import java.util.TreeSet;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
|
@ -96,7 +99,7 @@ public class RegionSplitCalculator<R extends KeyRange> {
|
|||
*
|
||||
* @return ENDKEY if end key is empty, else normal endkey.
|
||||
*/
|
||||
private byte[] specialEndKey(R range) {
|
||||
private static <R extends KeyRange> byte[] specialEndKey(R range) {
|
||||
byte[] end = range.getEndKey();
|
||||
if (end.length == 0) {
|
||||
return ENDKEY;
|
||||
|
@ -161,4 +164,75 @@ public class RegionSplitCalculator<R extends KeyRange> {
|
|||
return starts;
|
||||
}
|
||||
|
||||
/**
|
||||
* Find specified number of top ranges in a big overlap group.
|
||||
* It could return less if there are not that many top ranges.
|
||||
* Once these top ranges are excluded, the big overlap group will
|
||||
* be broken into ranges with no overlapping, or smaller overlapped
|
||||
* groups, and most likely some holes.
|
||||
*
|
||||
* @param bigOverlap a list of ranges that overlap with each other
|
||||
* @param count the max number of ranges to find
|
||||
* @return a list of ranges that overlap with most others
|
||||
*/
|
||||
public static <R extends KeyRange> List<R>
|
||||
findBigRanges(Collection<R> bigOverlap, int count) {
|
||||
List<R> bigRanges = new ArrayList<R>();
|
||||
|
||||
// The key is the count of overlaps,
|
||||
// The value is a list of ranges that have that many overlaps
|
||||
TreeMap<Integer, List<R>> overlapRangeMap = new TreeMap<Integer, List<R>>();
|
||||
for (R r: bigOverlap) {
|
||||
// Calculates the # of overlaps for each region
|
||||
// and populates rangeOverlapMap
|
||||
byte[] startKey = r.getStartKey();
|
||||
byte[] endKey = specialEndKey(r);
|
||||
|
||||
int overlappedRegions = 0;
|
||||
for (R rr: bigOverlap) {
|
||||
byte[] start = rr.getStartKey();
|
||||
byte[] end = specialEndKey(rr);
|
||||
|
||||
if (BYTES_COMPARATOR.compare(startKey, end) < 0
|
||||
&& BYTES_COMPARATOR.compare(endKey, start) > 0) {
|
||||
overlappedRegions++;
|
||||
}
|
||||
}
|
||||
|
||||
// One region always overlaps with itself,
|
||||
// so overlappedRegions should be more than 1
|
||||
// for actual overlaps.
|
||||
if (overlappedRegions > 1) {
|
||||
Integer key = Integer.valueOf(overlappedRegions);
|
||||
List<R> ranges = overlapRangeMap.get(key);
|
||||
if (ranges == null) {
|
||||
ranges = new ArrayList<R>();
|
||||
overlapRangeMap.put(key, ranges);
|
||||
}
|
||||
ranges.add(r);
|
||||
}
|
||||
}
|
||||
int toBeAdded = count;
|
||||
for (Integer key: overlapRangeMap.descendingKeySet()) {
|
||||
List<R> chunk = overlapRangeMap.get(key);
|
||||
int chunkSize = chunk.size();
|
||||
if (chunkSize <= toBeAdded) {
|
||||
bigRanges.addAll(chunk);
|
||||
toBeAdded -= chunkSize;
|
||||
if (toBeAdded > 0) continue;
|
||||
} else {
|
||||
// Try to use the middle chunk in case the overlapping is
|
||||
// chained, for example: [a, c), [b, e), [d, g), [f h)...
|
||||
// In such a case, sideline the middle chunk will break
|
||||
// the group efficiently.
|
||||
int start = (chunkSize - toBeAdded)/2;
|
||||
int end = start + toBeAdded;
|
||||
for (int i = start; i < end; i++) {
|
||||
bigRanges.add(chunk.get(i));
|
||||
}
|
||||
}
|
||||
break;
|
||||
}
|
||||
return bigRanges;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,9 +20,12 @@
|
|||
package org.apache.hadoop.hbase.util;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.SortedSet;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
|
@ -32,6 +35,7 @@ import org.junit.Test;
|
|||
|
||||
import com.google.common.collect.ComparisonChain;
|
||||
import com.google.common.collect.Multimap;
|
||||
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
@Category(SmallTests.class)
|
||||
|
@ -351,6 +355,42 @@ public class TestRegionSplitCalculator {
|
|||
+ "null:\t\n");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBigRanges() {
|
||||
SimpleRange ai = new SimpleRange(Bytes.toBytes("A"), Bytes.toBytes("I"));
|
||||
SimpleRange ae = new SimpleRange(Bytes.toBytes("A"), Bytes.toBytes("E"));
|
||||
SimpleRange ac = new SimpleRange(Bytes.toBytes("A"), Bytes.toBytes("C"));
|
||||
|
||||
Collection<SimpleRange> bigOverlap = new ArrayList<SimpleRange>();
|
||||
bigOverlap.add(new SimpleRange(Bytes.toBytes("A"), Bytes.toBytes("E")));
|
||||
bigOverlap.add(new SimpleRange(Bytes.toBytes("A"), Bytes.toBytes("C")));
|
||||
bigOverlap.add(new SimpleRange(Bytes.toBytes("A"), Bytes.toBytes("B")));
|
||||
bigOverlap.add(new SimpleRange(Bytes.toBytes("B"), Bytes.toBytes("C")));
|
||||
bigOverlap.add(new SimpleRange(Bytes.toBytes("E"), Bytes.toBytes("H")));
|
||||
bigOverlap.add(ai);
|
||||
bigOverlap.add(ae);
|
||||
bigOverlap.add(ac);
|
||||
|
||||
// Expect 1 range to be returned: ai
|
||||
List<SimpleRange> bigRanges = RegionSplitCalculator.findBigRanges(bigOverlap, 1);
|
||||
assertEquals(1, bigRanges.size());
|
||||
assertEquals(ai, bigRanges.get(0));
|
||||
|
||||
// Expect 3 ranges to be returned: ai, ae and ac
|
||||
bigRanges = RegionSplitCalculator.findBigRanges(bigOverlap, 3);
|
||||
assertEquals(3, bigRanges.size());
|
||||
assertEquals(ai, bigRanges.get(0));
|
||||
|
||||
SimpleRange r1 = bigRanges.get(1);
|
||||
SimpleRange r2 = bigRanges.get(2);
|
||||
assertEquals(Bytes.toString(r1.start), "A");
|
||||
assertEquals(Bytes.toString(r2.start), "A");
|
||||
String r1e = Bytes.toString(r1.end);
|
||||
String r2e = Bytes.toString(r2.end);
|
||||
assertTrue((r1e.equals("C") && r2e.equals("E"))
|
||||
|| (r1e.equals("E") && r2e.equals("C")));
|
||||
}
|
||||
|
||||
@org.junit.Rule
|
||||
public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
|
||||
new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
|
||||
|
|
Loading…
Reference in New Issue