HBASE-3028 No basescanner means no GC'ing of split, offlined parent regions
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1000715 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
2ecdd83e0d
commit
9ea9cba5b2
|
@ -537,6 +537,7 @@ Release 0.21.0 - Unreleased
|
|||
HBASE-3026 Fixup of "missing" daughters on split is too aggressive
|
||||
HBASE-3003 ClassSize constants dont use 'final'
|
||||
HBASE-3002 Fix zookeepers.sh to work properly with strange JVM options
|
||||
HBASE-3028 No basescanner means no GC'ing of split, offlined parent regions
|
||||
|
||||
IMPROVEMENTS
|
||||
HBASE-1760 Cleanup TODOs in HTable
|
||||
|
|
|
@ -529,6 +529,18 @@ public class HRegionInfo extends VersionedWritable implements WritableComparable
|
|||
this.offLine = offLine;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @return True if this is a split parent region.
|
||||
*/
|
||||
public boolean isSplitParent() {
|
||||
if (!isSplit()) return false;
|
||||
if (!isOffline()) {
|
||||
LOG.warn("Region is split but NOT offline: " + getRegionNameAsString());
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* @see java.lang.Object#toString()
|
||||
*/
|
||||
|
|
|
@ -182,12 +182,33 @@ public class MetaEditor {
|
|||
HRegionInfo regionInfo)
|
||||
throws IOException {
|
||||
Delete delete = new Delete(regionInfo.getRegionName());
|
||||
catalogTracker.waitForMetaServerConnectionDefault().delete(
|
||||
CatalogTracker.META_REGION, delete);
|
||||
|
||||
catalogTracker.waitForMetaServerConnectionDefault().
|
||||
delete(CatalogTracker.META_REGION, delete);
|
||||
LOG.info("Deleted region " + regionInfo.getRegionNameAsString() + " from META");
|
||||
}
|
||||
|
||||
/**
|
||||
* Deletes daughter reference in offlined split parent.
|
||||
* @param catalogTracker
|
||||
* @param parent Parent row we're to remove daughter reference from
|
||||
* @param qualifier SplitA or SplitB daughter to remove
|
||||
* @param daughter
|
||||
* @throws NotAllMetaRegionsOnlineException
|
||||
* @throws IOException
|
||||
*/
|
||||
public static void deleteDaughterReferenceInParent(CatalogTracker catalogTracker,
|
||||
final HRegionInfo parent, final byte [] qualifier,
|
||||
final HRegionInfo daughter)
|
||||
throws NotAllMetaRegionsOnlineException, IOException {
|
||||
Delete delete = new Delete(parent.getRegionName());
|
||||
delete.deleteColumns(HConstants.CATALOG_FAMILY, qualifier);
|
||||
catalogTracker.waitForMetaServerConnectionDefault().
|
||||
delete(CatalogTracker.META_REGION, delete);
|
||||
LOG.info("Deleted daughter " + daughter.getRegionNameAsString() +
|
||||
" reference " + Bytes.toString(qualifier) + " from " +
|
||||
parent.getRegionNameAsString() + " .META.");
|
||||
}
|
||||
|
||||
/**
|
||||
* Updates the region information for the specified region in META.
|
||||
* @param catalogTracker
|
||||
|
|
|
@ -125,11 +125,37 @@ public class MetaReader {
|
|||
* @throws IOException
|
||||
*/
|
||||
public static Map<HRegionInfo,HServerAddress> fullScan(CatalogTracker catalogTracker)
|
||||
throws IOException {
|
||||
final Map<HRegionInfo,HServerAddress> regions =
|
||||
new TreeMap<HRegionInfo,HServerAddress>();
|
||||
Visitor v = new Visitor() {
|
||||
@Override
|
||||
public boolean visit(Result r) throws IOException {
|
||||
if (r == null || r.isEmpty()) return true;
|
||||
Pair<HRegionInfo,HServerAddress> region = metaRowToRegionPair(r);
|
||||
regions.put(region.getFirst(), region.getSecond());
|
||||
return true;
|
||||
}
|
||||
};
|
||||
fullScan(catalogTracker, v);
|
||||
return regions;
|
||||
}
|
||||
|
||||
/**
|
||||
* Performs a full scan of <code>.META.</code>.
|
||||
* <p>
|
||||
* Returns a map of every region to it's currently assigned server, according
|
||||
* to META. If the region does not have an assignment it will have a null
|
||||
* value in the map.
|
||||
* @param catalogTracker
|
||||
* @param visitor
|
||||
* @throws IOException
|
||||
*/
|
||||
public static void fullScan(CatalogTracker catalogTracker,
|
||||
final Visitor visitor)
|
||||
throws IOException {
|
||||
HRegionInterface metaServer =
|
||||
catalogTracker.waitForMetaServerConnectionDefault();
|
||||
Map<HRegionInfo,HServerAddress> allRegions =
|
||||
new TreeMap<HRegionInfo,HServerAddress>();
|
||||
Scan scan = new Scan();
|
||||
scan.addFamily(HConstants.CATALOG_FAMILY);
|
||||
long scannerid = metaServer.openScanner(
|
||||
|
@ -137,16 +163,12 @@ public class MetaReader {
|
|||
try {
|
||||
Result data;
|
||||
while((data = metaServer.next(scannerid)) != null) {
|
||||
if (!data.isEmpty()) {
|
||||
Pair<HRegionInfo,HServerAddress> region =
|
||||
metaRowToRegionPair(data);
|
||||
allRegions.put(region.getFirst(), region.getSecond());
|
||||
}
|
||||
if (!data.isEmpty()) visitor.visit(data);
|
||||
}
|
||||
} finally {
|
||||
metaServer.close(scannerid);
|
||||
}
|
||||
return allRegions;
|
||||
return;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -423,4 +445,17 @@ public class MetaReader {
|
|||
metaServer.close(scannerid);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Implementations 'visit' a catalog table row.
|
||||
*/
|
||||
public interface Visitor {
|
||||
/**
|
||||
* Visit the catalog table row.
|
||||
* @param r A row from catalog table
|
||||
* @return True if we are to proceed scanning the table, else false if
|
||||
* we are to stop now.
|
||||
*/
|
||||
public boolean visit(final Result r) throws IOException;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,258 @@
|
|||
/**
|
||||
* Copyright 2008 The Apache Software Foundation
|
||||
*
|
||||
* 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.master;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.TreeMap;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.PathFilter;
|
||||
import org.apache.hadoop.hbase.Chore;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.catalog.MetaEditor;
|
||||
import org.apache.hadoop.hbase.catalog.MetaReader;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.Store;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.util.Writables;
|
||||
|
||||
/**
|
||||
* A janitor for the catalog tables. Scans the <code>.META.</code> catalog
|
||||
* table on a period looking for unused regions to garbage collect.
|
||||
*/
|
||||
class CatalogJanitor extends Chore {
|
||||
private static final Log LOG = LogFactory.getLog(CatalogJanitor.class.getName());
|
||||
private final Server server;
|
||||
private final MasterServices services;
|
||||
|
||||
CatalogJanitor(final Server server, final MasterServices services) {
|
||||
super(server.getServerName() + "-CatalogJanitor",
|
||||
server.getConfiguration().getInt("hbase.catalogjanitor.interval", 300000),
|
||||
server);
|
||||
this.server = server;
|
||||
this.services = services;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean initialChore() {
|
||||
try {
|
||||
scan();
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Failed initial scan of catalog table", e);
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void chore() {
|
||||
try {
|
||||
scan();
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Failed scan of catalog table", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Run janitorial scan of catalog <code>.META.</code> table looking for
|
||||
* garbage to collect.
|
||||
* @throws IOException
|
||||
*/
|
||||
void scan() throws IOException {
|
||||
// TODO: Only works with single .META. region currently. Fix.
|
||||
final AtomicInteger count = new AtomicInteger(0);
|
||||
// Keep Map of found split parents. There are candidates for cleanup.
|
||||
final Map<HRegionInfo, Result> splitParents =
|
||||
new TreeMap<HRegionInfo, Result>();
|
||||
// This visitor collects split parents and counts rows in the .META. table
|
||||
MetaReader.Visitor visitor = new MetaReader.Visitor() {
|
||||
@Override
|
||||
public boolean visit(Result r) throws IOException {
|
||||
if (r == null || r.isEmpty()) return true;
|
||||
count.incrementAndGet();
|
||||
HRegionInfo info = getHRegionInfo(r);
|
||||
if (info.isSplitParent()) splitParents.put(info, r);
|
||||
// Returning true means "keep scanning"
|
||||
return true;
|
||||
}
|
||||
};
|
||||
// Run full scan of .META. catalog table passing in our custom visitor
|
||||
MetaReader.fullScan(this.server.getCatalogTracker(), visitor);
|
||||
// Now work on our list of found parents. See if any we can clean up.
|
||||
int cleaned = 0;
|
||||
for (Map.Entry<HRegionInfo, Result> e : splitParents.entrySet()) {
|
||||
if (cleanParent(e.getKey(), e.getValue())) cleaned++;
|
||||
}
|
||||
LOG.info("Scanned " + count.get() + " catalog row(s) and gc'd " + cleaned +
|
||||
" unreferenced parent region(s)");
|
||||
}
|
||||
|
||||
/**
|
||||
* Get HRegionInfo from passed Map of row values.
|
||||
* @param result Map to do lookup in.
|
||||
* @return Null if not found (and logs fact that expected COL_REGIONINFO
|
||||
* was missing) else deserialized {@link HRegionInfo}
|
||||
* @throws IOException
|
||||
*/
|
||||
static HRegionInfo getHRegionInfo(final Result result)
|
||||
throws IOException {
|
||||
byte [] bytes =
|
||||
result.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
|
||||
if (bytes == null) {
|
||||
LOG.warn("REGIONINFO_QUALIFIER is empty in " + result);
|
||||
return null;
|
||||
}
|
||||
return Writables.getHRegionInfo(bytes);
|
||||
}
|
||||
|
||||
/**
|
||||
* If daughters no longer hold reference to the parents, delete the parent.
|
||||
* @param server HRegionInterface of meta server to talk to
|
||||
* @param parent HRegionInfo of split offlined parent
|
||||
* @param rowContent Content of <code>parent</code> row in
|
||||
* <code>metaRegionName</code>
|
||||
* @return True if we removed <code>parent</code> from meta table and from
|
||||
* the filesystem.
|
||||
* @throws IOException
|
||||
*/
|
||||
boolean cleanParent(final HRegionInfo parent,
|
||||
Result rowContent)
|
||||
throws IOException {
|
||||
boolean result = false;
|
||||
// Run checks on each daughter split.
|
||||
boolean hasReferencesA =
|
||||
checkDaughter(parent, rowContent, HConstants.SPLITA_QUALIFIER);
|
||||
boolean hasReferencesB =
|
||||
checkDaughter(parent, rowContent, HConstants.SPLITB_QUALIFIER);
|
||||
if (!hasReferencesA && !hasReferencesB) {
|
||||
LOG.info("Deleting region " + parent.getRegionNameAsString() +
|
||||
" because daughter splits no longer hold references");
|
||||
FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
|
||||
Path rootdir = this.services.getMasterFileSystem().getRootDir();
|
||||
HRegion.deleteRegion(fs, rootdir, parent);
|
||||
MetaEditor.deleteRegion(this.server.getCatalogTracker(), parent);
|
||||
result = true;
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* See if the passed daughter has references in the filesystem to the parent
|
||||
* and if not, remove the note of daughter region in the parent row: its
|
||||
* column info:splitA or info:splitB.
|
||||
* @param parent
|
||||
* @param rowContent
|
||||
* @param qualifier
|
||||
* @return True if this daughter still has references to the parent.
|
||||
* @throws IOException
|
||||
*/
|
||||
boolean checkDaughter(final HRegionInfo parent,
|
||||
final Result rowContent, final byte [] qualifier)
|
||||
throws IOException {
|
||||
HRegionInfo hri = getDaughterRegionInfo(rowContent, qualifier);
|
||||
return hasReferences(parent, rowContent, hri, qualifier);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get daughter HRegionInfo out of parent info:splitA/info:splitB columns.
|
||||
* @param result
|
||||
* @param which Whether "info:splitA" or "info:splitB" column
|
||||
* @return Deserialized content of the info:splitA or info:splitB as a
|
||||
* HRegionInfo
|
||||
* @throws IOException
|
||||
*/
|
||||
private HRegionInfo getDaughterRegionInfo(final Result result,
|
||||
final byte [] which)
|
||||
throws IOException {
|
||||
byte [] bytes = result.getValue(HConstants.CATALOG_FAMILY, which);
|
||||
return Writables.getHRegionInfoOrNull(bytes);
|
||||
}
|
||||
|
||||
/**
|
||||
* Remove mention of daughter from parent row.
|
||||
* parent row.
|
||||
* @param metaRegionName
|
||||
* @param srvr
|
||||
* @param parent
|
||||
* @param split
|
||||
* @param qualifier
|
||||
* @throws IOException
|
||||
*/
|
||||
private void removeDaughterFromParent(final HRegionInfo parent,
|
||||
final HRegionInfo split, final byte [] qualifier)
|
||||
throws IOException {
|
||||
MetaEditor.deleteDaughterReferenceInParent(this.server.getCatalogTracker(),
|
||||
parent, qualifier, split);
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if a daughter region -- either splitA or splitB -- still holds
|
||||
* references to parent. If not, removes reference to the split from
|
||||
* the parent meta region row so we don't check it any more.
|
||||
* @param parent Parent region name.
|
||||
* @param rowContent Keyed content of the parent row in meta region.
|
||||
* @param split Which column family.
|
||||
* @param qualifier Which of the daughters to look at, splitA or splitB.
|
||||
* @return True if still has references to parent.
|
||||
* @throws IOException
|
||||
*/
|
||||
boolean hasReferences(final HRegionInfo parent,
|
||||
final Result rowContent, final HRegionInfo split,
|
||||
final byte [] qualifier)
|
||||
throws IOException {
|
||||
boolean result = false;
|
||||
if (split == null) return result;
|
||||
FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
|
||||
Path rootdir = this.services.getMasterFileSystem().getRootDir();
|
||||
Path tabledir = new Path(rootdir, split.getTableDesc().getNameAsString());
|
||||
for (HColumnDescriptor family: split.getTableDesc().getFamilies()) {
|
||||
Path p = Store.getStoreHomedir(tabledir, split.getEncodedName(),
|
||||
family.getName());
|
||||
// Look for reference files. Call listStatus with anonymous instance of PathFilter.
|
||||
FileStatus [] ps = fs.listStatus(p,
|
||||
new PathFilter () {
|
||||
public boolean accept(Path path) {
|
||||
return StoreFile.isReference(path);
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
if (ps != null && ps.length > 0) {
|
||||
result = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (!result) {
|
||||
removeDaughterFromParent(parent, split, qualifier);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
}
|
|
@ -159,9 +159,11 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
|
|||
ExecutorService executorService;
|
||||
|
||||
private LoadBalancer balancer = new LoadBalancer();
|
||||
private Chore balancerChore;
|
||||
private Thread balancerChore;
|
||||
private volatile boolean balance = true;
|
||||
|
||||
private Thread catalogJanitorChore;
|
||||
|
||||
/**
|
||||
* Initializes the HMaster. The steps are as follows:
|
||||
*
|
||||
|
@ -313,32 +315,29 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
|
|||
}
|
||||
}
|
||||
|
||||
// Start balancer and meta catalog janitor after meta and regions have
|
||||
// been assigned.
|
||||
this.balancerChore = getAndStartBalancerChore(this);
|
||||
this.catalogJanitorChore =
|
||||
Threads.setDaemonThreadRunning(new CatalogJanitor(this, this));
|
||||
// Check if we should stop every second.
|
||||
Sleeper sleeper = new Sleeper(1000, this);
|
||||
while (!this.stopped) sleeper.sleep();
|
||||
} catch (Throwable t) {
|
||||
abort("Unhandled exception. Starting shutdown.", t);
|
||||
}
|
||||
// Stop balancer and meta catalog janitor
|
||||
if (this.balancerChore != null) this.balancerChore.interrupt();
|
||||
if (this.catalogJanitorChore != null) this.catalogJanitorChore.interrupt();
|
||||
|
||||
// Wait for all the remaining region servers to report in IFF we were
|
||||
// running a cluster shutdown AND we were NOT aborting.
|
||||
if (!this.abort && this.serverManager.isClusterShutdown()) {
|
||||
this.serverManager.letRegionServersShutdown();
|
||||
}
|
||||
|
||||
// Clean up and close up shop
|
||||
if (this.infoServer != null) {
|
||||
LOG.info("Stopping infoServer");
|
||||
try {
|
||||
this.infoServer.stop();
|
||||
} catch (Exception ex) {
|
||||
ex.printStackTrace();
|
||||
}
|
||||
}
|
||||
this.rpcServer.stop();
|
||||
if (this.balancerChore != null) this.balancerChore.interrupt();
|
||||
stopServiceThreads();
|
||||
// Stop services started up in the constructor.
|
||||
this.activeMasterManager.stop();
|
||||
this.executorService.shutdown();
|
||||
HConnectionManager.deleteConnection(this.conf, true);
|
||||
this.zooKeeper.close();
|
||||
LOG.info("HMaster main thread exiting");
|
||||
|
@ -459,7 +458,6 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
|
|||
this.infoServer.setAttribute(MASTER, this);
|
||||
this.infoServer.start();
|
||||
}
|
||||
this.balancerChore = getAndStartBalancerChore(this);
|
||||
|
||||
// Start the server last so everything else is running before we start
|
||||
// receiving requests.
|
||||
|
@ -480,10 +478,27 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
|
|||
}
|
||||
}
|
||||
|
||||
private static Chore getAndStartBalancerChore(final HMaster master) {
|
||||
private void stopServiceThreads() {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Stopping service threads");
|
||||
}
|
||||
this.rpcServer.stop();
|
||||
// Clean up and close up shop
|
||||
if (this.infoServer != null) {
|
||||
LOG.info("Stopping infoServer");
|
||||
try {
|
||||
this.infoServer.stop();
|
||||
} catch (Exception ex) {
|
||||
ex.printStackTrace();
|
||||
}
|
||||
}
|
||||
this.executorService.shutdown();
|
||||
}
|
||||
|
||||
private static Thread getAndStartBalancerChore(final HMaster master) {
|
||||
String name = master.getServerName() + "-balancerChore";
|
||||
int period = master.getConfiguration().
|
||||
getInt("hbase.master.balancer.period", 3000000);
|
||||
getInt("hbase.balancer.period", 3000000);
|
||||
// Start up the load balancer chore
|
||||
Chore chore = new Chore(name, period, master) {
|
||||
@Override
|
||||
|
@ -491,8 +506,7 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
|
|||
master.balance();
|
||||
}
|
||||
};
|
||||
Threads.setDaemonThreadRunning(chore, name);
|
||||
return chore;
|
||||
return Threads.setDaemonThreadRunning(chore);
|
||||
}
|
||||
|
||||
public MapWritable regionServerStartup(final HServerInfo serverInfo)
|
||||
|
|
|
@ -128,13 +128,12 @@ public class ServerManager {
|
|||
this.metrics = new MasterMetrics(master.getServerName());
|
||||
this.serverMonitorThread = new ServerMonitor(monitorInterval, master);
|
||||
String n = Thread.currentThread().getName();
|
||||
Threads.setDaemonThreadRunning(this.serverMonitorThread,
|
||||
n + ".serverMonitor");
|
||||
this.logCleaner = new LogCleaner(c.getInt("hbase.master.cleaner.interval", 60 * 1000),
|
||||
master, c, this.services.getMasterFileSystem().getFileSystem(),
|
||||
this.services.getMasterFileSystem().getOldLogDir());
|
||||
Threads.setDaemonThreadRunning(logCleaner,
|
||||
n + ".oldLogCleaner");
|
||||
Threads.setDaemonThreadRunning(this.serverMonitorThread, n + ".serverMonitor");
|
||||
this.logCleaner =
|
||||
new LogCleaner(c.getInt("hbase.master.cleaner.interval", 60 * 1000),
|
||||
master, c, this.services.getMasterFileSystem().getFileSystem(),
|
||||
this.services.getMasterFileSystem().getOldLogDir());
|
||||
Threads.setDaemonThreadRunning(logCleaner, n + ".oldLogCleaner");
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -2473,22 +2473,6 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Delete a region's meta information from the passed
|
||||
* <code>meta</code> region. Deletes the row.
|
||||
* @param srvr META server to be updated
|
||||
* @param metaRegionName Meta region name
|
||||
* @param regionName HRegion to remove from <code>meta</code>
|
||||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
public static void removeRegionFromMETA(final HRegionInterface srvr,
|
||||
final byte [] metaRegionName, final byte [] regionName)
|
||||
throws IOException {
|
||||
Delete delete = new Delete(regionName);
|
||||
srvr.delete(metaRegionName, delete);
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility method used by HMaster marking regions offlined.
|
||||
* @param srvr META server to be updated
|
||||
|
|
|
@ -1578,7 +1578,7 @@ public class HLog implements Syncable {
|
|||
//Either way, the caller should decide what to do. E.g. ignore if this is the last
|
||||
//log in sequence.
|
||||
//TODO is this scenario still possible if the log has been recovered (i.e. closed)
|
||||
LOG.warn("Could not open " + path + " for reading. File is empty" + e);
|
||||
LOG.warn("Could not open " + path + " for reading. File is empty: " + e);
|
||||
return;
|
||||
} else {
|
||||
throw e;
|
||||
|
|
|
@ -619,7 +619,7 @@ public class FSUtils {
|
|||
if (!(fs instanceof DistributedFileSystem)) {
|
||||
return;
|
||||
}
|
||||
LOG.info("Recovering file" + p);
|
||||
LOG.info("Recovering file " + p);
|
||||
long startWaiting = System.currentTimeMillis();
|
||||
|
||||
// Trying recovery
|
||||
|
|
|
@ -32,6 +32,15 @@ import java.lang.Thread.UncaughtExceptionHandler;
|
|||
public class Threads {
|
||||
protected static final Log LOG = LogFactory.getLog(Threads.class);
|
||||
|
||||
/**
|
||||
* Utility method that sets name, daemon status and starts passed thread.
|
||||
* @param t thread to run
|
||||
* @return Returns the passed Thread <code>t</code>.
|
||||
*/
|
||||
public static Thread setDaemonThreadRunning(final Thread t) {
|
||||
return setDaemonThreadRunning(t, t.getName());
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility method that sets name, daemon status and starts passed thread.
|
||||
* @param t thread to frob
|
||||
|
|
|
@ -0,0 +1,228 @@
|
|||
/**
|
||||
* Copyright 2010 The Apache Software Foundation
|
||||
*
|
||||
* 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.master;
|
||||
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
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.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.catalog.CatalogTracker;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.executor.ExecutorService;
|
||||
import org.apache.hadoop.hbase.io.Reference;
|
||||
import org.apache.hadoop.hbase.ipc.HRegionInterface;
|
||||
import org.apache.hadoop.hbase.regionserver.Store;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Writables;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
public class TestCatalogJanitor {
|
||||
|
||||
/**
|
||||
* Pseudo server for below tests.
|
||||
*/
|
||||
class MockServer implements Server {
|
||||
private final Configuration c;
|
||||
private final CatalogTracker ct;
|
||||
|
||||
MockServer(final HBaseTestingUtility htu)
|
||||
throws NotAllMetaRegionsOnlineException, IOException {
|
||||
this.c = htu.getConfiguration();
|
||||
// Set hbase.rootdir into test dir.
|
||||
FileSystem fs = FileSystem.get(this.c);
|
||||
Path rootdir =
|
||||
fs.makeQualified(HBaseTestingUtility.getTestDir(HConstants.HBASE_DIR));
|
||||
this.c.set(HConstants.HBASE_DIR, rootdir.toString());
|
||||
this.ct = Mockito.mock(CatalogTracker.class);
|
||||
HRegionInterface hri = Mockito.mock(HRegionInterface.class);
|
||||
Mockito.when(ct.waitForMetaServerConnectionDefault()).thenReturn(hri);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CatalogTracker getCatalogTracker() {
|
||||
return this.ct;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Configuration getConfiguration() {
|
||||
return this.c;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getServerName() {
|
||||
// TODO Auto-generated method stub
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ZooKeeperWatcher getZooKeeper() {
|
||||
// TODO Auto-generated method stub
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort(String why, Throwable e) {
|
||||
// TODO Auto-generated method stub
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isStopped() {
|
||||
// TODO Auto-generated method stub
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stop(String why) {
|
||||
// TODO Auto-generated method stub
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Mock MasterServices for tests below.
|
||||
*/
|
||||
class MockMasterServices implements MasterServices {
|
||||
private final MasterFileSystem mfs;
|
||||
|
||||
MockMasterServices(final Server server) throws IOException {
|
||||
this.mfs = new MasterFileSystem(server);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void checkTableModifiable(byte[] tableName) throws IOException {
|
||||
// TODO Auto-generated method stub
|
||||
}
|
||||
|
||||
@Override
|
||||
public AssignmentManager getAssignmentManager() {
|
||||
// TODO Auto-generated method stub
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ExecutorService getExecutorService() {
|
||||
// TODO Auto-generated method stub
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public MasterFileSystem getMasterFileSystem() {
|
||||
return this.mfs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ServerManager getServerManager() {
|
||||
// TODO Auto-generated method stub
|
||||
return null;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetHRegionInfo() throws IOException {
|
||||
assertNull(CatalogJanitor.getHRegionInfo(new Result()));
|
||||
List<KeyValue> kvs = new ArrayList<KeyValue>();
|
||||
Result r = new Result(kvs);
|
||||
assertNull(CatalogJanitor.getHRegionInfo(r));
|
||||
byte [] f = HConstants.CATALOG_FAMILY;
|
||||
// Make a key value that doesn't have the expected qualifier.
|
||||
kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY, f,
|
||||
HConstants.SERVER_QUALIFIER, f));
|
||||
r = new Result(kvs);
|
||||
assertNull(CatalogJanitor.getHRegionInfo(r));
|
||||
// Make a key that does not have a regioninfo value.
|
||||
kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY, f,
|
||||
HConstants.REGIONINFO_QUALIFIER, f));
|
||||
boolean exception = false;
|
||||
try {
|
||||
CatalogJanitor.getHRegionInfo(new Result(kvs));
|
||||
} catch (Exception ioe) {
|
||||
exception = true;
|
||||
}
|
||||
assertTrue(exception);
|
||||
// OK, give it what it expects
|
||||
kvs.clear();
|
||||
kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY, f,
|
||||
HConstants.REGIONINFO_QUALIFIER,
|
||||
Writables.getBytes(HRegionInfo.FIRST_META_REGIONINFO)));
|
||||
HRegionInfo hri = CatalogJanitor.getHRegionInfo(new Result(kvs));
|
||||
assertNotNull(hri);
|
||||
assertTrue(hri.equals(HRegionInfo.FIRST_META_REGIONINFO));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCleanParent() throws IOException {
|
||||
HBaseTestingUtility htu = new HBaseTestingUtility();
|
||||
Server server = new MockServer(htu);
|
||||
MasterServices services = new MockMasterServices(server);
|
||||
CatalogJanitor janitor = new CatalogJanitor(server, services);
|
||||
// Create regions.
|
||||
HTableDescriptor htd = new HTableDescriptor("table");
|
||||
htd.addFamily(new HColumnDescriptor("family"));
|
||||
HRegionInfo parent =
|
||||
new HRegionInfo(htd, Bytes.toBytes("aaa"), Bytes.toBytes("eee"));
|
||||
HRegionInfo splita =
|
||||
new HRegionInfo(htd, Bytes.toBytes("aaa"), Bytes.toBytes("ccc"));
|
||||
HRegionInfo splitb =
|
||||
new HRegionInfo(htd, Bytes.toBytes("ccc"), Bytes.toBytes("eee"));
|
||||
// Test that when both daughter regions are in place, that we do not
|
||||
// remove the parent.
|
||||
List<KeyValue> kvs = new ArrayList<KeyValue>();
|
||||
kvs.add(new KeyValue(parent.getRegionName(), HConstants.CATALOG_FAMILY,
|
||||
HConstants.SPLITA_QUALIFIER, Writables.getBytes(splita)));
|
||||
kvs.add(new KeyValue(parent.getRegionName(), HConstants.CATALOG_FAMILY,
|
||||
HConstants.SPLITB_QUALIFIER, Writables.getBytes(splitb)));
|
||||
Result r = new Result(kvs);
|
||||
// Add a reference under splitA directory so we don't clear out the parent.
|
||||
Path rootdir = services.getMasterFileSystem().getRootDir();
|
||||
Path tabledir =
|
||||
HTableDescriptor.getTableDir(rootdir, htd.getName());
|
||||
Path storedir = Store.getStoreHomedir(tabledir, splita.getEncodedName(),
|
||||
htd.getColumnFamilies()[0].getName());
|
||||
Reference ref = new Reference(Bytes.toBytes("ccc"), Reference.Range.top);
|
||||
long now = System.currentTimeMillis();
|
||||
// Reference name has this format: StoreFile#REF_NAME_PARSER
|
||||
Path p = new Path(storedir, Long.toString(now) + "." + parent.getEncodedName());
|
||||
FileSystem fs = services.getMasterFileSystem().getFileSystem();
|
||||
ref.write(fs, p);
|
||||
assertFalse(janitor.cleanParent(parent, r));
|
||||
// Remove the reference file and try again.
|
||||
assertTrue(fs.delete(p, true));
|
||||
assertTrue(janitor.cleanParent(parent, r));
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue