HBASE-3171 Drop ROOT and instead store META location(s) directly in ZooKeeper

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1454607 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2013-03-08 22:37:19 +00:00
parent 7f2768955c
commit 3a23adf5ab
58 changed files with 432 additions and 1602 deletions

View File

@ -99,7 +99,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
*<p>
* **NOTE**
*
* ROOT, the first META region, and regions created by an older
* The first META region, and regions created by an older
* version of HBase (0.20 or prior) will continue to use the
* old region name format.
*/
@ -142,7 +142,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
regionName.length - MD5_HEX_LENGTH - 1,
MD5_HEX_LENGTH);
} else {
// old format region name. ROOT and first META region also
// old format region name. First META region also
// use this format.EncodedName is the JenkinsHash value.
int hashVal = Math.abs(JenkinsHash.getInstance().hash(regionName,
regionName.length, 0));
@ -154,14 +154,11 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
/**
* Use logging.
* @param encodedRegionName The encoded regionname.
* @return <code>-ROOT-</code> if passed <code>70236052</code> or
* <code>.META.</code> if passed </code>1028785192</code> else returns
* @return <code>.META.</code> if passed </code>1028785192</code> else returns
* <code>encodedRegionName</code>
*/
public static String prettyPrint(final String encodedRegionName) {
if (encodedRegionName.equals("70236052")) {
return encodedRegionName + "/-ROOT-";
} else if (encodedRegionName.equals("1028785192")) {
if (encodedRegionName.equals("1028785192")) {
return encodedRegionName + "/.META.";
}
return encodedRegionName;
@ -206,14 +203,14 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
/**
* Private constructor used constructing HRegionInfo for the catalog root and
* Private constructor used constructing HRegionInfo for the
* first meta regions
*/
private HRegionInfo(long regionId, byte[] tableName) {
super();
this.regionId = regionId;
this.tableName = tableName.clone();
// Note: Root & First Meta regions names are still in old format
// Note: First Meta regions names are still in old format
this.regionName = createRegionName(tableName, null,
regionId, false);
this.regionNameStr = Bytes.toStringBinary(this.regionName);
@ -568,16 +565,11 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY));
}
/** @return true if this is the root region */
public boolean isRootRegion() {
return Bytes.equals(tableName, HRegionInfo.ROOT_REGIONINFO.getTableName());
}
/** @return true if this region is from a table that is a meta table,
* either <code>.META.</code> or <code>-ROOT-</code>
/**
* @return true if this region is from .META.
*/
public boolean isMetaTable() {
return isRootRegion() || isMetaRegion();
return isMetaRegion();
}
/** @return true if this region is a meta region */
@ -804,7 +796,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
* @return Comparator to use comparing {@link KeyValue}s.
*/
public KVComparator getComparator() {
return isRootRegion()? KeyValue.ROOT_COMPARATOR: isMetaRegion()?
return isMetaRegion()?
KeyValue.META_COMPARATOR: KeyValue.COMPARATOR;
}
@ -848,9 +840,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
public static HRegionInfo convert(final RegionInfo proto) {
if (proto == null) return null;
byte [] tableName = proto.getTableName().toByteArray();
if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
return ROOT_REGIONINFO;
} else if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) {
if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) {
return FIRST_META_REGIONINFO;
}
long regionId = proto.getRegionId();

View File

@ -33,8 +33,7 @@ import org.apache.hadoop.hbase.exceptions.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.exceptions.ServerNotRunningYetException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.MetaNodeTracker;
import org.apache.hadoop.hbase.zookeeper.RootRegionTracker;
import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.ipc.RemoteException;
@ -45,28 +44,26 @@ import java.net.NoRouteToHostException;
import java.net.SocketException;
import java.net.SocketTimeoutException;
import java.net.UnknownHostException;
import java.util.concurrent.atomic.AtomicBoolean;
/**
* Tracks the availability of the catalog tables <code>-ROOT-</code> and
* Tracks the availability of the catalog tables
* <code>.META.</code>.
*
* This class is "read-only" in that the locations of the catalog tables cannot
* be explicitly set. Instead, ZooKeeper is used to learn of the availability
* and location of <code>-ROOT-</code>. <code>-ROOT-</code> is used to learn of
* the location of <code>.META.</code> If not available in <code>-ROOT-</code>,
* ZooKeeper is used to monitor for a new location of <code>.META.</code>.
* and location of <code>.META.</code>.
*
* <p>Call {@link #start()} to start up operation. Call {@link #stop()}} to
* interrupt waits and close up shop.
*/
@InterfaceAudience.Private
public class CatalogTracker {
// TODO JDC 11/30 We don't even have ROOT anymore, revisit
// TODO: This class needs a rethink. The original intent was that it would be
// the one-stop-shop for root and meta locations and that it would get this
// the one-stop-shop for meta locations and that it would get this
// info from reading and watching zk state. The class was to be used by
// servers when they needed to know of root and meta movement but also by
// client-side (inside in HTable) so rather than figure root and meta
// servers when they needed to know of meta movement but also by
// client-side (inside in HTable) so rather than figure meta
// locations on fault, the client would instead get notifications out of zk.
//
// But this original intent is frustrated by the fact that this class has to
@ -109,9 +106,7 @@ public class CatalogTracker {
private static final Log LOG = LogFactory.getLog(CatalogTracker.class);
private final HConnection connection;
private final ZooKeeperWatcher zookeeper;
private final RootRegionTracker rootRegionTracker;
private final MetaNodeTracker metaNodeTracker;
private final AtomicBoolean metaAvailable = new AtomicBoolean(false);
private final MetaRegionTracker metaRegionTracker;
private boolean instantiatedzkw = false;
private Abortable abortable;
@ -124,8 +119,6 @@ public class CatalogTracker {
private boolean stopped = false;
static final byte [] ROOT_REGION_NAME =
HRegionInfo.ROOT_REGIONINFO.getRegionName();
static final byte [] META_REGION_NAME =
HRegionInfo.FIRST_META_REGIONINFO.getRegionName();
@ -192,15 +185,7 @@ public class CatalogTracker {
} else {
this.zookeeper = zk;
}
this.rootRegionTracker = new RootRegionTracker(zookeeper, throwableAborter);
final CatalogTracker ct = this;
// Override nodeDeleted so we get notified when meta node deleted
this.metaNodeTracker = new MetaNodeTracker(zookeeper, throwableAborter) {
public void nodeDeleted(String path) {
if (!path.equals(node)) return;
ct.resetMetaLocation();
}
};
this.metaRegionTracker = new MetaRegionTracker(zookeeper, throwableAborter);
}
/**
@ -213,12 +198,11 @@ public class CatalogTracker {
public void start() throws IOException, InterruptedException {
LOG.debug("Starting catalog tracker " + this);
try {
this.rootRegionTracker.start();
this.metaNodeTracker.start();
this.metaRegionTracker.start();
} catch (RuntimeException e) {
Throwable t = e.getCause();
this.abortable.abort(e.getMessage(), t);
throw new IOException("Attempt to start root/meta tracker failed.", t);
throw new IOException("Attempt to start meta tracker failed.", t);
}
}
@ -230,8 +214,7 @@ public class CatalogTracker {
if (!this.stopped) {
LOG.debug("Stopping catalog tracker " + this);
this.stopped = true;
this.rootRegionTracker.stop();
this.metaNodeTracker.stop();
this.metaRegionTracker.stop();
try {
if (this.connection != null) {
this.connection.close();
@ -244,61 +227,34 @@ public class CatalogTracker {
if (this.instantiatedzkw) {
this.zookeeper.close();
}
// Call this and it will interrupt any ongoing waits on meta.
synchronized (this.metaAvailable) {
this.metaAvailable.notifyAll();
}
}
}
/**
* Gets the current location for <code>-ROOT-</code> or null if location is
* Gets the current location for <code>.META.</code> or null if location is
* not currently available.
* @return {@link ServerName} for server hosting <code>-ROOT-</code> or null
* @return {@link ServerName} for server hosting <code>.META.</code> or null
* if none available
* @throws InterruptedException
*/
public ServerName getRootLocation() throws InterruptedException {
return this.rootRegionTracker.getRootRegionLocation();
public ServerName getMetaLocation() throws InterruptedException {
return this.metaRegionTracker.getMetaRegionLocation();
}
/**
* @return {@link ServerName} for server hosting <code>.META.</code> or null
* if none available
*/
public ServerName getMetaLocation() {
return this.metaLocation;
}
/**
* Method used by master on startup trying to figure state of cluster.
* Returns the current meta location unless its null. In this latter case,
* it has not yet been set so go check whats up in <code>-ROOT-</code> and
* return that.
* @return {@link ServerName} for server hosting <code>.META.</code> or if null,
* we'll read the location that is up in <code>-ROOT-</code> table (which
* could be null or just plain stale).
* @throws IOException
*/
public ServerName getMetaLocationOrReadLocationFromRoot() throws IOException {
ServerName sn = getMetaLocation();
return sn != null? sn: MetaReader.getMetaRegionLocation(this);
}
/**
* Gets the current location for <code>-ROOT-</code> if available and waits
* Gets the current location for <code>.META.</code> if available and waits
* for up to the specified timeout if not immediately available. Returns null
* if the timeout elapses before root is available.
* @param timeout maximum time to wait for root availability, in milliseconds
* @return {@link ServerName} for server hosting <code>-ROOT-</code> or null
* @return {@link ServerName} for server hosting <code>.META.</code> or null
* if none available
* @throws InterruptedException if interrupted while waiting
* @throws NotAllMetaRegionsOnlineException if root not available before
* @throws NotAllMetaRegionsOnlineException if meta not available before
* timeout
*/
public ServerName waitForRoot(final long timeout)
public ServerName waitForMeta(final long timeout)
throws InterruptedException, NotAllMetaRegionsOnlineException {
ServerName sn = rootRegionTracker.waitRootRegionLocation(timeout);
ServerName sn = metaRegionTracker.waitMetaRegionLocation(timeout);
if (sn == null) {
throw new NotAllMetaRegionsOnlineException("Timed out; " + timeout + "ms");
}
@ -306,86 +262,35 @@ public class CatalogTracker {
}
/**
* Gets a connection to the server hosting root, as reported by ZooKeeper,
* Gets a connection to the server hosting meta, as reported by ZooKeeper,
* waiting up to the specified timeout for availability.
* @param timeout How long to wait on root location
* @see #waitForRoot(long) for additional information
* @return connection to server hosting root
* @param timeout How long to wait on meta location
* @see #waitForMeta for additional information
* @return connection to server hosting meta
* @throws InterruptedException
* @throws NotAllMetaRegionsOnlineException if timed out waiting
* @throws IOException
* @deprecated Use #getRootServerConnection(long)
* @deprecated Use #getMetaServerConnection(long)
*/
public AdminProtocol waitForRootServerConnection(long timeout)
public AdminProtocol waitForMetaServerConnection(long timeout)
throws InterruptedException, NotAllMetaRegionsOnlineException, IOException {
return getRootServerConnection(timeout);
return getMetaServerConnection(timeout);
}
/**
* Gets a connection to the server hosting root, as reported by ZooKeeper,
* Gets a connection to the server hosting meta, as reported by ZooKeeper,
* waiting up to the specified timeout for availability.
* <p>WARNING: Does not retry. Use an {@link HTable} instead.
* @param timeout How long to wait on root location
* @see #waitForRoot(long) for additional information
* @return connection to server hosting root
* @param timeout How long to wait on meta location
* @see #waitForMeta for additional information
* @return connection to server hosting meta
* @throws InterruptedException
* @throws NotAllMetaRegionsOnlineException if timed out waiting
* @throws IOException
*/
AdminProtocol getRootServerConnection(long timeout)
AdminProtocol getMetaServerConnection(long timeout)
throws InterruptedException, NotAllMetaRegionsOnlineException, IOException {
return getCachedConnection(waitForRoot(timeout));
}
/**
* Gets a connection to the server currently hosting <code>.META.</code> or
* null if location is not currently available.
* <p>
* If a location is known, a connection to the cached location is returned.
* If refresh is true, the cached connection is verified first before
* returning. If the connection is not valid, it is reset and rechecked.
* <p>
* If no location for meta is currently known, method checks ROOT for a new
* location, verifies META is currently there, and returns a cached connection
* to the server hosting META.
*
* @return connection to server hosting meta, null if location not available
* @throws IOException
* @throws InterruptedException
*/
private AdminProtocol getMetaServerConnection()
throws IOException, InterruptedException {
synchronized (metaAvailable) {
if (metaAvailable.get()) {
AdminProtocol current = getCachedConnection(this.metaLocation);
// If we are to refresh, verify we have a good connection by making
// an invocation on it.
if (verifyRegionLocation(current, this.metaLocation, META_REGION_NAME)) {
return current;
}
resetMetaLocation();
}
// We got here because there is no meta available or because whats
// available is bad.
// Now read the current .META. content from -ROOT-. Note: This goes via
// an HConnection. It has its own way of figuring root and meta locations
// which we have to wait on.
ServerName newLocation = MetaReader.getMetaRegionLocation(this);
if (newLocation == null) return null;
AdminProtocol newConnection = getCachedConnection(newLocation);
if (verifyRegionLocation(newConnection, newLocation, META_REGION_NAME)) {
setMetaLocation(newLocation);
return newConnection;
} else {
if (LOG.isTraceEnabled()) {
LOG.trace("New .META. server: " + newLocation + " isn't valid." +
" Cached .META. server: " + this.metaLocation);
}
}
return null;
}
return getCachedConnection(waitForMeta(timeout));
}
/**
@ -410,79 +315,6 @@ public class CatalogTracker {
}
}
/**
* Gets the current location for <code>.META.</code> if available and waits
* for up to the specified timeout if not immediately available. Throws an
* exception if timed out waiting. This method differs from {@link #waitForMeta()}
* in that it will go ahead and verify the location gotten from ZooKeeper and
* -ROOT- region by trying to use returned connection.
* @param timeout maximum time to wait for meta availability, in milliseconds
* @return {@link ServerName} for server hosting <code>.META.</code> or null
* if none available
* @throws InterruptedException if interrupted while waiting
* @throws IOException unexpected exception connecting to meta server
* @throws NotAllMetaRegionsOnlineException if meta not available before
* timeout
*/
public ServerName waitForMeta(long timeout)
throws InterruptedException, IOException, NotAllMetaRegionsOnlineException {
long stop = timeout == 0 ? Long.MAX_VALUE : System.currentTimeMillis() + timeout;
long waitTime = Math.min(50, timeout);
synchronized (metaAvailable) {
while(!stopped && System.currentTimeMillis() < stop) {
if (getMetaServerConnection() != null) {
return metaLocation;
}
// perhaps -ROOT- region isn't available, let us wait a bit and retry.
metaAvailable.wait(waitTime);
}
if (getMetaServerConnection() == null) {
throw new NotAllMetaRegionsOnlineException("Timed out (" + timeout + "ms)");
}
return metaLocation;
}
}
/**
* Gets a connection to the server hosting meta, as reported by ZooKeeper,
* waiting up to the specified timeout for availability.
* @see #waitForMeta(long) for additional information
* @return connection to server hosting meta
* @throws InterruptedException
* @throws NotAllMetaRegionsOnlineException if timed out waiting
* @throws IOException
* @deprecated Does not retry; use an HTable instance instead.
*/
public AdminProtocol waitForMetaServerConnection(long timeout)
throws InterruptedException, NotAllMetaRegionsOnlineException, IOException {
return getCachedConnection(waitForMeta(timeout));
}
/**
* Called when we figure current meta is off (called from zk callback).
*/
public void resetMetaLocation() {
LOG.debug("Current cached META location, " + metaLocation +
", is not valid, resetting");
synchronized(this.metaAvailable) {
this.metaAvailable.set(false);
this.metaAvailable.notifyAll();
}
}
/**
* @param metaLocation
*/
void setMetaLocation(final ServerName metaLocation) {
LOG.debug("Set new cached META location: " + metaLocation);
synchronized (this.metaAvailable) {
this.metaLocation = metaLocation;
this.metaAvailable.set(true);
// no synchronization because these are private and already under lock
this.metaAvailable.notifyAll();
}
}
/**
* @param sn ServerName to get a connection against.
* @return The AdminProtocol we got when we connected to <code>sn</code>
@ -533,7 +365,7 @@ public class CatalogTracker {
* Verify we can connect to <code>hostingServer</code> and that its carrying
* <code>regionName</code>.
* @param hostingServer Interface to the server hosting <code>regionName</code>
* @param serverName The servername that goes with the <code>metaServer</code>
* @param address The servername that goes with the <code>metaServer</code>
* Interface. Used logging.
* @param regionName The regionname we are interested in.
* @return True if we were able to verify the region located at other side of
@ -578,37 +410,12 @@ public class CatalogTracker {
return false;
}
/**
* Verify <code>-ROOT-</code> is deployed and accessible.
* @param timeout How long to wait on zk for root address (passed through to
* the internal call to {@link #waitForRootServerConnection(long)}.
* @return True if the <code>-ROOT-</code> location is healthy.
* @throws IOException
* @throws InterruptedException
*/
public boolean verifyRootRegionLocation(final long timeout)
throws InterruptedException, IOException {
AdminProtocol connection = null;
try {
connection = waitForRootServerConnection(timeout);
} catch (NotAllMetaRegionsOnlineException e) {
// Pass
} catch (ServerNotRunningYetException e) {
// Pass -- remote server is not up so can't be carrying root
} catch (UnknownHostException e) {
// Pass -- server name doesn't resolve so it can't be assigned anything.
}
return (connection == null)? false:
verifyRegionLocation(connection,
this.rootRegionTracker.getRootRegionLocation(), ROOT_REGION_NAME);
}
/**
* Verify <code>.META.</code> is deployed and accessible.
* @param timeout How long to wait on zk for <code>.META.</code> address
* (passed through to the internal call to {@link #waitForMetaServerConnection(long)}.
* @param timeout How long to wait on zk for meta address (passed through to
* the internal call to {@link #waitForMetaServerConnection(long)}.
* @return True if the <code>.META.</code> location is healthy.
* @throws IOException Some unexpected IOE.
* @throws IOException
* @throws InterruptedException
*/
public boolean verifyMetaRegionLocation(final long timeout)
@ -619,19 +426,13 @@ public class CatalogTracker {
} catch (NotAllMetaRegionsOnlineException e) {
// Pass
} catch (ServerNotRunningYetException e) {
// Pass -- remote server is not up so can't be carrying .META.
// Pass -- remote server is not up so can't be carrying root
} catch (UnknownHostException e) {
// Pass -- server name doesn't resolve so it can't be assigned anything.
} catch (RetriesExhaustedException e) {
// Pass -- failed after bunch of retries.
LOG.debug("Failed verify meta region location after retries", e);
}
return connection != null;
}
// Used by tests.
MetaNodeTracker getMetaNodeTracker() {
return this.metaNodeTracker;
return (connection == null)? false:
verifyRegionLocation(connection,
this.metaRegionTracker.getMetaRegionLocation(), META_REGION_NAME);
}
public HConnection getConnection() {

View File

@ -61,21 +61,6 @@ public class MetaReader {
META_REGION_PREFIX, 0, len);
}
/**
* @param row
* @return True if <code>row</code> is row of <code>-ROOT-</code> table.
*/
private static boolean isRootTableRow(final byte [] row) {
if (row.length < META_REGION_PREFIX.length + 2 /* ',', + '1' */) {
// Can't be meta table region.
return false;
}
// Compare the prefix of row. If it matches META_REGION_PREFIX prefix,
// then this is row from -ROOT_ table.
return Bytes.equals(row, 0, META_REGION_PREFIX.length,
META_REGION_PREFIX, 0, META_REGION_PREFIX.length);
}
/**
* Performs a full scan of <code>.META.</code>, skipping regions from any
* tables in the specified set of disabled tables.
@ -143,14 +128,14 @@ public class MetaReader {
}
/**
* Performs a full scan of a <code>-ROOT-</code> table.
* Performs a full scan of a <code>.META.</code> table.
* @return List of {@link Result}
* @throws IOException
*/
public static List<Result> fullScanOfRoot(CatalogTracker catalogTracker)
public static List<Result> fullScanOfMeta(CatalogTracker catalogTracker)
throws IOException {
CollectAllVisitor v = new CollectAllVisitor();
fullScan(catalogTracker, v, null, true);
fullScan(catalogTracker, v, null);
return v.getResults();
}
@ -166,21 +151,6 @@ public class MetaReader {
fullScan(catalogTracker, visitor, null);
}
/**
* Performs a full scan of <code>.META.</code>.
* @param catalogTracker
* @param visitor Visitor invoked against each row.
* @param startrow Where to start the scan. Pass null if want to begin scan
* at first row (The visitor will stop the Scan when its done so no need to
* pass a stoprow).
* @throws IOException
*/
public static void fullScan(CatalogTracker catalogTracker,
final Visitor visitor, final byte [] startrow)
throws IOException {
fullScan(catalogTracker, visitor, startrow, false);
}
/**
* Callers should call close on the returned {@link HTable} instance.
* @param catalogTracker We'll use this catalogtracker's connection
@ -201,16 +171,12 @@ public class MetaReader {
/**
* Callers should call close on the returned {@link HTable} instance.
* @param catalogTracker
* @param row Row we are putting
* @return
* @throws IOException
*/
static HTable getCatalogHTable(final CatalogTracker catalogTracker,
final byte [] row)
static HTable getCatalogHTable(final CatalogTracker catalogTracker)
throws IOException {
return isRootTableRow(row)?
getRootHTable(catalogTracker):
getMetaHTable(catalogTracker);
return getMetaHTable(catalogTracker);
}
/**
@ -224,17 +190,6 @@ public class MetaReader {
return getHTable(ct, HConstants.META_TABLE_NAME);
}
/**
* Callers should call close on the returned {@link HTable} instance.
* @param ct
* @return An {@link HTable} for <code>-ROOT-</code>
* @throws IOException
*/
static HTable getRootHTable(final CatalogTracker ct)
throws IOException {
return getHTable(ct, HConstants.ROOT_TABLE_NAME);
}
/**
* @param t Table to use (will be closed when done).
* @param g Get to run
@ -248,19 +203,6 @@ public class MetaReader {
}
}
/**
* Gets the location of <code>.META.</code> region by reading content of
* <code>-ROOT-</code>.
* @param ct
* @return location of <code>.META.</code> region as a {@link ServerName} or
* null if not found
* @throws IOException
*/
static ServerName getMetaRegionLocation(final CatalogTracker ct)
throws IOException {
return MetaReader.readRegionLocation(ct, CatalogTracker.META_REGION_NAME);
}
/**
* Reads the location of the specified region
* @param catalogTracker
@ -287,7 +229,7 @@ public class MetaReader {
throws IOException {
Get get = new Get(regionName);
get.addFamily(HConstants.CATALOG_FAMILY);
Result r = get(getCatalogHTable(catalogTracker, regionName), get);
Result r = get(getCatalogHTable(catalogTracker), get);
return (r == null || r.isEmpty())? null: HRegionInfo.getHRegionInfoAndServerName(r);
}
@ -302,8 +244,7 @@ public class MetaReader {
public static boolean tableExists(CatalogTracker catalogTracker,
String tableName)
throws IOException {
if (tableName.equals(HTableDescriptor.ROOT_TABLEDESC.getNameAsString()) ||
tableName.equals(HTableDescriptor.META_TABLEDESC.getNameAsString())) {
if (tableName.equals(HTableDescriptor.META_TABLEDESC.getNameAsString())) {
// Catalog tables always exist.
return true;
}
@ -451,12 +392,12 @@ public class MetaReader {
getTableRegionsAndLocations(final CatalogTracker catalogTracker,
final byte [] tableName, final boolean excludeOfflinedSplitParents)
throws IOException, InterruptedException {
if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
// If root, do a bit of special handling.
ServerName serverName = catalogTracker.getRootLocation();
if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) {
// If meta, do a bit of special handling.
ServerName serverName = catalogTracker.getMetaLocation();
List<Pair<HRegionInfo, ServerName>> list =
new ArrayList<Pair<HRegionInfo, ServerName>>();
list.add(new Pair<HRegionInfo, ServerName>(HRegionInfo.ROOT_REGIONINFO,
list.add(new Pair<HRegionInfo, ServerName>(HRegionInfo.FIRST_META_REGIONINFO,
serverName));
return list;
}
@ -487,8 +428,7 @@ public class MetaReader {
this.results.add(this.current);
}
};
fullScan(catalogTracker, visitor, getTableStartRowForMeta(tableName),
Bytes.equals(tableName, HConstants.META_TABLE_NAME));
fullScan(catalogTracker, visitor, getTableStartRowForMeta(tableName));
return visitor.getResults();
}
@ -546,23 +486,21 @@ public class MetaReader {
* @param visitor Visitor invoked against each row.
* @param startrow Where to start the scan. Pass null if want to begin scan
* at first row.
* @param scanRoot True if we are to scan <code>-ROOT-</code> rather than
* <code>.META.</code>, the default (pass false to scan .META.)
* @throws IOException
*/
static void fullScan(CatalogTracker catalogTracker,
final Visitor visitor, final byte [] startrow, final boolean scanRoot)
public static void fullScan(CatalogTracker catalogTracker,
final Visitor visitor, final byte [] startrow)
throws IOException {
Scan scan = new Scan();
if (startrow != null) scan.setStartRow(startrow);
if (startrow == null && !scanRoot) {
if (startrow == null) {
int caching = catalogTracker.getConnection().getConfiguration()
.getInt(HConstants.HBASE_META_SCANNER_CACHING, 100);
scan.setCaching(caching);
}
scan.addFamily(HConstants.CATALOG_FAMILY);
HTable metaTable = scanRoot?
getRootHTable(catalogTracker): getMetaHTable(catalogTracker);
HTable metaTable = getMetaHTable(catalogTracker);
ResultScanner scanner = metaTable.getScanner(scan);
try {
Result data;

View File

@ -60,12 +60,8 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.SoftValueSortedMap;
import org.apache.hadoop.hbase.util.Triple;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.RootRegionTracker;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKTableReadOnly;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.hbase.zookeeper.*;
import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.zookeeper.KeeperException;
@ -107,7 +103,7 @@ import java.util.concurrent.atomic.AtomicInteger;
* instance does not change its object identity). Sharing {@link HConnection}
* instances is usually what you want; all clients of the {@link HConnection}
* instances share the HConnections' cache of Region locations rather than each
* having to discover for itself the location of meta, root, etc. It makes
* having to discover for itself the location of meta, etc. It makes
* sense for the likes of the pool of HTables class {@link HTablePool}, for
* instance (If concerned that a single {@link HConnection} is insufficient
* for sharing amongst clients in say an heavily-multithreaded environment,
@ -873,10 +869,6 @@ public class HConnectionManager {
*/
private boolean testTableOnlineState(byte [] tableName, boolean enabled)
throws IOException {
if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
// The root region is always enabled
return enabled;
}
String tableNameStr = Bytes.toString(tableName);
ZooKeeperKeepAliveConnection zkw = getKeepAliveZooKeeperWatcher();
try {
@ -946,27 +938,24 @@ public class HConnectionManager {
"table name cannot be null or zero length");
}
if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) {
ZooKeeperKeepAliveConnection zkw = getKeepAliveZooKeeperWatcher();
try {
LOG.debug("Looking up root region location in ZK," +
LOG.debug("Looking up meta region location in ZK," +
" connection=" + this);
ServerName servername =
RootRegionTracker.blockUntilAvailable(zkw, this.rpcTimeout);
MetaRegionTracker.blockUntilAvailable(zkw, this.rpcTimeout);
LOG.debug("Looked up root region location, connection=" + this +
LOG.debug("Looked up meta region location, connection=" + this +
"; serverName=" + ((servername == null) ? "null" : servername));
if (servername == null) return null;
return new HRegionLocation(HRegionInfo.ROOT_REGIONINFO, servername, 0);
return new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, servername, 0);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
return null;
} finally {
zkw.close();
}
} else if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) {
return locateRegionInMeta(HConstants.ROOT_TABLE_NAME, tableName, row,
useCache, metaRegionLock, retry);
} else {
// Region not in the cache - have to go to the meta RS
return locateRegionInMeta(HConstants.META_TABLE_NAME, tableName, row,
@ -1025,7 +1014,7 @@ public class HConnectionManager {
}
/*
* Search one of the meta tables (-ROOT- or .META.) for the HRegionLocation
* Search the .META. table for the HRegionLocation
* info that contains the table and row we're seeking.
*/
private HRegionLocation locateRegionInMeta(final byte [] parentTable,
@ -1055,7 +1044,7 @@ public class HConnectionManager {
HRegionLocation metaLocation = null;
try {
// locate the root or meta region
// locate the meta region
metaLocation = locateRegion(parentTable, metaKey, true, false);
// If null still, go around again.
if (metaLocation == null) continue;
@ -1087,7 +1076,7 @@ public class HConnectionManager {
forceDeleteCachedLocation(tableName, row);
}
// Query the root or meta region for the location of the meta region
// Query the meta region for the location of the meta region
regionInfoRow = ProtobufUtil.getRowOrBefore(server,
metaLocation.getRegionInfo().getRegionName(), metaKey,
HConstants.CATALOG_FAMILY);
@ -2436,9 +2425,6 @@ public class HConnectionManager {
public HTableDescriptor getHTableDescriptor(final byte[] tableName)
throws IOException {
if (tableName == null || tableName.length == 0) return null;
if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
return new UnmodifyableHTableDescriptor(HTableDescriptor.ROOT_TABLEDESC);
}
if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) {
return HTableDescriptor.META_TABLEDESC;
}

View File

@ -19,46 +19,46 @@ package org.apache.hadoop.hbase.zookeeper;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.zookeeper.KeeperException;
/**
* Tracks the root region server location node in zookeeper.
* Root region location is set by <code>RegionServerServices</code>.
* This class has a watcher on the root location and notices changes.
* Tracks the meta region server location node in zookeeper.
* Meta region location is set by <code>RegionServerServices</code>.
* This class has a watcher on the meta location and notices changes.
*/
@InterfaceAudience.Private
public class RootRegionTracker extends ZooKeeperNodeTracker {
public class MetaRegionTracker extends ZooKeeperNodeTracker {
/**
* Creates a root region location tracker.
* Creates a meta region location tracker.
*
* <p>After construction, use {@link #start} to kick off tracking.
*
* @param watcher
* @param abortable
*/
public RootRegionTracker(ZooKeeperWatcher watcher, Abortable abortable) {
super(watcher, watcher.rootServerZNode, abortable);
public MetaRegionTracker(ZooKeeperWatcher watcher, Abortable abortable) {
super(watcher, watcher.metaServerZNode, abortable);
}
/**
* Checks if the root region location is available.
* @return true if root region location is available, false if not
* Checks if the meta region location is available.
* @return true if meta region location is available, false if not
*/
public boolean isLocationAvailable() {
return super.getData(true) != null;
}
/**
* Gets the root region location, if available. Does not block. Sets a watcher.
* Gets the meta region location, if available. Does not block. Sets a watcher.
* @return server name or null if we failed to get the data.
* @throws InterruptedException
*/
public ServerName getRootRegionLocation() throws InterruptedException {
public ServerName getMetaRegionLocation() throws InterruptedException {
try {
return ServerName.parseFrom(super.getData(true));
} catch (DeserializationException e) {
@ -68,32 +68,32 @@ public class RootRegionTracker extends ZooKeeperNodeTracker {
}
/**
* Gets the root region location, if available. Does not block. Does not set
* a watcher (In this regard it differs from {@link #getRootRegionLocation()}.
* Gets the meta region location, if available. Does not block. Does not set
* a watcher (In this regard it differs from {@link #getMetaRegionLocation}.
* @param zkw
* @return server name or null if we failed to get the data.
* @throws KeeperException
*/
public static ServerName getRootRegionLocation(final ZooKeeperWatcher zkw)
public static ServerName getMetaRegionLocation(final ZooKeeperWatcher zkw)
throws KeeperException {
try {
return ServerName.parseFrom(ZKUtil.getData(zkw, zkw.rootServerZNode));
return ServerName.parseFrom(ZKUtil.getData(zkw, zkw.metaServerZNode));
} catch (DeserializationException e) {
throw ZKUtil.convert(e);
}
}
/**
* Gets the root region location, if available, and waits for up to the
* Gets the meta region location, if available, and waits for up to the
* specified timeout if not immediately available.
* Given the zookeeper notification could be delayed, we will try to
* get the latest data.
* @param timeout maximum time to wait, in millis
* @return server name for server hosting root region formatted as per
* @return server name for server hosting meta region formatted as per
* {@link ServerName}, or null if none available
* @throws InterruptedException if interrupted while waiting
*/
public ServerName waitRootRegionLocation(long timeout)
public ServerName waitMetaRegionLocation(long timeout)
throws InterruptedException {
if (false == checkIfBaseNodeAvailable()) {
String errorMsg = "Check the value configured in 'zookeeper.znode.parent'. "
@ -110,31 +110,31 @@ public class RootRegionTracker extends ZooKeeperNodeTracker {
}
/**
* Sets the location of <code>-ROOT-</code> in ZooKeeper to the
* Sets the location of <code>.META.</code> in ZooKeeper to the
* specified server address.
* @param zookeeper zookeeper reference
* @param location The server hosting <code>-ROOT-</code>
* @param location The server hosting <code>.META.</code>
* @throws KeeperException unexpected zookeeper exception
*/
public static void setRootLocation(ZooKeeperWatcher zookeeper,
public static void setMetaLocation(ZooKeeperWatcher zookeeper,
final ServerName location)
throws KeeperException {
LOG.info("Setting ROOT region location in ZooKeeper as " + location);
// Make the RootRegionServer pb and then get its bytes and save this as
LOG.info("Setting META region location in ZooKeeper as " + location);
// Make the MetaRegionServer pb and then get its bytes and save this as
// the znode content.
byte [] data = toByteArray(location);
try {
ZKUtil.createAndWatch(zookeeper, zookeeper.rootServerZNode, data);
ZKUtil.createAndWatch(zookeeper, zookeeper.metaServerZNode, data);
} catch(KeeperException.NodeExistsException nee) {
LOG.debug("ROOT region location already existed, updated location");
ZKUtil.setData(zookeeper, zookeeper.rootServerZNode, data);
LOG.debug("META region location already existed, updated location");
ZKUtil.setData(zookeeper, zookeeper.metaServerZNode, data);
}
}
/**
* Build up the znode content.
* @param sn What to put into the znode.
* @return The content of the root-region-server znode
* @return The content of the meta-region-server znode
*/
static byte [] toByteArray(final ServerName sn) {
// ZNode content is a pb message preceeded by some pb magic.
@ -147,23 +147,23 @@ public class RootRegionTracker extends ZooKeeperNodeTracker {
}
/**
* Deletes the location of <code>-ROOT-</code> in ZooKeeper.
* Deletes the location of <code>.META.</code> in ZooKeeper.
* @param zookeeper zookeeper reference
* @throws KeeperException unexpected zookeeper exception
*/
public static void deleteRootLocation(ZooKeeperWatcher zookeeper)
public static void deleteMetaLocation(ZooKeeperWatcher zookeeper)
throws KeeperException {
LOG.info("Unsetting ROOT region location in ZooKeeper");
LOG.info("Unsetting META region location in ZooKeeper");
try {
// Just delete the node. Don't need any watches.
ZKUtil.deleteNode(zookeeper, zookeeper.rootServerZNode);
ZKUtil.deleteNode(zookeeper, zookeeper.metaServerZNode);
} catch(KeeperException.NoNodeException nne) {
// Has already been deleted
}
}
/**
* Wait until the root region is available.
* Wait until the meta region is available.
* @param zkw
* @param timeout
* @return ServerName or null if we timed out.
@ -172,7 +172,7 @@ public class RootRegionTracker extends ZooKeeperNodeTracker {
public static ServerName blockUntilAvailable(final ZooKeeperWatcher zkw,
final long timeout)
throws InterruptedException {
byte [] data = ZKUtil.blockUntilAvailable(zkw, zkw.rootServerZNode, timeout);
byte [] data = ZKUtil.blockUntilAvailable(zkw, zkw.metaServerZNode, timeout);
if (data == null) return null;
try {
return ServerName.parseFrom(data);

View File

@ -949,7 +949,7 @@ public class ZKUtil {
// Certain znodes are accessed directly by the client,
// so they must be readable by non-authenticated clients
if ((node.equals(zkw.baseZNode) == true) ||
(node.equals(zkw.rootServerZNode) == true) ||
(node.equals(zkw.metaServerZNode) == true) ||
(node.equals(zkw.getMasterAddressZNode()) == true) ||
(node.equals(zkw.clusterIdZNode) == true) ||
(node.equals(zkw.rsZNode) == true) ||
@ -1579,7 +1579,7 @@ public class ZKUtil {
zkw.backupMasterAddressesZNode)) {
sb.append("\n ").append(child);
}
sb.append("\nRegion server holding ROOT: " + RootRegionTracker.getRootRegionLocation(zkw));
sb.append("\nRegion server holding .META.: " + MetaRegionTracker.getMetaRegionLocation(zkw));
sb.append("\nRegion servers:");
for (String child : listChildrenNoWatch(zkw, zkw.rsZNode)) {
sb.append("\n ").append(child);
@ -1685,7 +1685,7 @@ public class ZKUtil {
(data == null? "null": data.length == 0? "empty": (
znode.startsWith(zkw.assignmentZNode)?
ZKAssign.toString(data): // We should not be doing this reaching into another class
znode.startsWith(zkw.rootServerZNode)?
znode.startsWith(zkw.metaServerZNode)?
getServerNameOrEmptyString(data):
znode.startsWith(zkw.backupMasterAddressesZNode)?
getServerNameOrEmptyString(data):

View File

@ -81,8 +81,8 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
// base znode for this cluster
public String baseZNode;
// znode containing location of server hosting root region
public String rootServerZNode;
// znode containing location of server hosting meta region
public String metaServerZNode;
// znode containing ephemeral nodes of the regionservers
public String rsZNode;
// znode containing ephemeral nodes of the draining regionservers
@ -196,8 +196,8 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
private void setNodeNames(Configuration conf) {
baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
rootServerZNode = ZKUtil.joinZNode(baseZNode,
conf.get("zookeeper.znode.rootserver", "root-region-server"));
metaServerZNode = ZKUtil.joinZNode(baseZNode,
conf.get("zookeeper.znode.metaserver", "meta-region-server"));
rsZNode = ZKUtil.joinZNode(baseZNode,
conf.get("zookeeper.znode.rs", "rs"));
drainingZNode = ZKUtil.joinZNode(baseZNode,

View File

@ -61,8 +61,7 @@ if (toRemove > 0) {
Map.Entry<String, RegionState> e = it.next();
if (HRegionInfo.FIRST_META_REGIONINFO.getEncodedName().equals(
e.getKey()) ||
HRegionInfo.ROOT_REGIONINFO.getEncodedName().equals(
e.getKey()) || regionIDForOldestRIT.equals(e.getKey())) {
regionIDForOldestRIT.equals(e.getKey())) {
// don't remove the meta & the oldest rit regions, they're too interesting!
continue;
}

View File

@ -20,7 +20,6 @@ limitations under the License.
HMaster master;
HBaseAdmin admin;
Map<String, Integer> frags = null;
ServerName rootLocation = null;
ServerName metaLocation = null;
List<ServerName> servers = null;
Set<ServerName> deadServers = null;
@ -159,7 +158,7 @@ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
</%if>
</div>
<div class="tab-pane" id="tab_catalogTables">
<%if (rootLocation != null) %>
<%if (metaLocation != null) %>
<& catalogTables &>
</%if>
</div>
@ -222,7 +221,7 @@ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
<tr>
<td>Fragmentation</td>
<td><% frags.get("-TOTAL-") != null ? frags.get("-TOTAL-").intValue() + "%" : "n/a" %></td>
<td>Overall fragmentation of all tables, including .META. and -ROOT-.</td>
<td>Overall fragmentation of all tables, including .META.</td>
</tr>
</%if>
<tr>
@ -266,13 +265,6 @@ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
<th title="Fragmentation - Will be 0% after a major compaction and fluctuate during normal usage.">Frag.</th>
</%if>
<th>Description</th>
</tr>
<tr>
<td><a href="table.jsp?name=<% Bytes.toString(HConstants.ROOT_TABLE_NAME) %>"><% Bytes.toString(HConstants.ROOT_TABLE_NAME) %></a></td>
<%if (frags != null)%>
<td align="center"><% frags.get("-ROOT-") != null ? frags.get("-ROOT-").intValue() + "%" : "n/a" %></td>
</%if>
<td>The -ROOT- table holds references to all .META. regions.</td>
</tr>
<%if (metaLocation != null) %>
<tr>

View File

@ -64,10 +64,9 @@
the region named
<em>domains,apache.org,5464829424211263407</em> is party to the table
<em>domains</em>, has an id of <em>5464829424211263407</em> and the first key
in the region is <em>apache.org</em>. The <em>-ROOT-</em>
and <em>.META.</em> 'tables' are internal sytem tables (or 'catalog' tables in db-speak).
The -ROOT- keeps a list of all regions in the .META. table. The .META. table
keeps a list of all regions in the system. The empty key is used to denote
in the region is <em>apache.org</em>. The <em>.META.</em> 'table' is an internal
system table (or 'catalog' tables in db-speak).
The .META. table keeps a list of all regions in the system. The empty key is used to denote
table start and table end. A region with an empty start key is the first region in a table.
If region has both an empty start and an empty end key, its the only region in the table. See
<a href="http://hbase.org">HBase Home</a> for further explication.<p>

View File

@ -89,17 +89,6 @@ public class MetaEditor {
put(MetaReader.getMetaHTable(ct), p);
}
/**
* Put the passed <code>p</code> to the <code>.META.</code> table.
* @param ct CatalogTracker on whose back we will ride the edit.
* @param p Put to add to .META.
* @throws IOException
*/
static void putToRootTable(final CatalogTracker ct, final Put p)
throws IOException {
put(MetaReader.getRootHTable(ct), p);
}
/**
* Put the passed <code>p</code> to a catalog table.
* @param ct CatalogTracker on whose back we will ride the edit.
@ -108,7 +97,7 @@ public class MetaEditor {
*/
static void putToCatalogTable(final CatalogTracker ct, final Put p)
throws IOException {
HTable t = MetaReader.getCatalogHTable(ct, p.getRow());
HTable t = MetaReader.getCatalogHTable(ct);
put(t, p);
}

View File

@ -164,7 +164,7 @@ public class MetaMigrationConvertingToPB {
throws IOException {
LOG.info("Starting update of ROOT");
ConvertToPBMetaVisitor v = new ConvertToPBMetaVisitor(masterServices);
MetaReader.fullScan(masterServices.getCatalogTracker(), v, null, true);
MetaReader.fullScan(masterServices.getCatalogTracker(), v, null);
LOG.info("Finished update of ROOT. Total rows updated:" + v.numMigratedRows);
return v.numMigratedRows;
}
@ -177,7 +177,7 @@ public class MetaMigrationConvertingToPB {
LOG.info("Starting update of META");
ConvertToPBMetaVisitor v = new ConvertToPBMetaVisitor(masterServices);
MetaReader.fullScan(masterServices.getCatalogTracker(), v);
updateRootWithMetaMigrationStatus(masterServices.getCatalogTracker());
//updateRootWithMetaMigrationStatus(masterServices.getCatalogTracker());
LOG.info("Finished update of META. Total rows updated:" + v.numMigratedRows);
return v.numMigratedRows;
}
@ -192,7 +192,8 @@ public class MetaMigrationConvertingToPB {
Put p = new Put(HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
p.add(HConstants.CATALOG_FAMILY, HConstants.META_VERSION_QUALIFIER,
Bytes.toBytes(HConstants.META_VERSION));
MetaEditor.putToRootTable(catalogTracker, p);
// TODO so wrong
//MetaEditor.putToRootTable(catalogTracker, p);
LOG.info("Updated -ROOT- meta version=" + HConstants.META_VERSION);
}
@ -202,7 +203,7 @@ public class MetaMigrationConvertingToPB {
* @throws IOException
*/
static boolean isMetaHRIUpdated(final CatalogTracker catalogTracker) throws IOException {
List<Result> results = MetaReader.fullScanOfRoot(catalogTracker);
List<Result> results = MetaReader.fullScanOfMeta(catalogTracker);
if (results == null || results.isEmpty()) {
LOG.info(".META. is not migrated");
return false;

View File

@ -73,7 +73,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.KeyLocker;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.RootRegionTracker;
import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZKTable;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@ -501,7 +501,7 @@ public class AssignmentManager extends ZooKeeperListener {
}
/**
* This call is invoked only (1) master assign root and meta;
* This call is invoked only (1) master assign meta;
* (2) during failover mode startup, zk assignment node processing.
* The locker is set in the caller.
*
@ -2143,31 +2143,18 @@ public class AssignmentManager extends ZooKeeperListener {
}
}
/**
* Assigns the ROOT region.
* <p>
* Assumes that ROOT is currently closed and is not being actively served by
* any RegionServer.
* <p>
* Forcibly unsets the current root region location in ZooKeeper and assigns
* ROOT to a random RegionServer.
* @throws KeeperException
*/
public void assignRoot() throws KeeperException {
RootRegionTracker.deleteRootLocation(this.watcher);
assign(HRegionInfo.ROOT_REGIONINFO, true);
}
/**
* Assigns the META region.
* <p>
* Assumes that META is currently closed and is not being actively served by
* any RegionServer.
* <p>
* Forcibly assigns META to a random RegionServer.
* Forcibly unsets the current meta region location in ZooKeeper and assigns
* META to a random RegionServer.
* @throws KeeperException
*/
public void assignMeta() {
// Force assignment to a random server
public void assignMeta() throws KeeperException {
MetaRegionTracker.deleteMetaLocation(this.watcher);
assign(HRegionInfo.FIRST_META_REGIONINFO, true);
}
@ -2747,10 +2734,6 @@ public class AssignmentManager extends ZooKeeperListener {
threadPoolExecutorService.submit(new UnAssignCallable(this, regionInfo));
}
public boolean isCarryingRoot(ServerName serverName) {
return isCarryingRegion(serverName, HRegionInfo.ROOT_REGIONINFO);
}
public boolean isCarryingMeta(ServerName serverName) {
return isCarryingRegion(serverName, HRegionInfo.FIRST_META_REGIONINFO);
}

View File

@ -296,7 +296,7 @@ Server {
private volatile boolean isActiveMaster = false;
// flag set after we complete initialization once active (used for testing)
private volatile boolean initialized = false;
// flag set after we complete assignRootAndMeta.
// flag set after we complete assignMeta.
private volatile boolean serverShutdownHandlerEnabled = false;
// Instance of the hbase executor service.
@ -652,7 +652,7 @@ Server {
* <li>Set cluster as UP in ZooKeeper</li>
* <li>Wait for RegionServers to check-in</li>
* <li>Split logs and perform data recovery, if necessary</li>
* <li>Ensure assignment of root and meta regions<li>
* <li>Ensure assignment of meta regions<li>
* <li>Handle either fresh cluster start or master failover</li>
* </ol>
*
@ -734,8 +734,8 @@ Server {
status.setStatus("Splitting logs after master startup");
splitLogAfterStartup(this.fileSystemManager);
// Make sure root and meta assigned before proceeding.
if (!assignRootAndMeta(status)) return;
// Make sure meta assigned before proceeding.
if (!assignMeta(status)) return;
enableServerShutdownHandler();
// Update meta with new PB serialization if required. i.e migrate all HRI
@ -826,76 +826,43 @@ Server {
}
/**
* Check <code>-ROOT-</code> and <code>.META.</code> are assigned. If not,
* Check <code>.META.</code> are assigned. If not,
* assign them.
* @throws InterruptedException
* @throws IOException
* @throws KeeperException
* @return True if root and meta are healthy, assigned
* @return True if meta is healthy, assigned
*/
boolean assignRootAndMeta(MonitoredTask status)
boolean assignMeta(MonitoredTask status)
throws InterruptedException, IOException, KeeperException {
int assigned = 0;
long timeout = this.conf.getLong("hbase.catalog.verification.timeout", 1000);
// Work on ROOT region. Is it in zk in transition?
status.setStatus("Assigning ROOT region");
assignmentManager.getRegionStates().createRegionState(
HRegionInfo.ROOT_REGIONINFO);
boolean rit = this.assignmentManager.
processRegionInTransitionAndBlockUntilAssigned(HRegionInfo.ROOT_REGIONINFO);
ServerName currentRootServer = null;
boolean rootRegionLocation = catalogTracker.verifyRootRegionLocation(timeout);
if (!rit && !rootRegionLocation) {
currentRootServer = this.catalogTracker.getRootLocation();
splitLogAndExpireIfOnline(currentRootServer);
this.assignmentManager.assignRoot();
// Make sure a -ROOT- location is set.
if (!isRootLocation()) return false;
// This guarantees that the transition assigning -ROOT- has completed
this.assignmentManager.waitForAssignment(HRegionInfo.ROOT_REGIONINFO);
assigned++;
} else if (rit && !rootRegionLocation) {
// Make sure a -ROOT- location is set.
if (!isRootLocation()) return false;
// This guarantees that the transition assigning -ROOT- has completed
this.assignmentManager.waitForAssignment(HRegionInfo.ROOT_REGIONINFO);
assigned++;
} else if (rootRegionLocation) {
// Region already assigned. We didn't assign it. Add to in-memory state.
this.assignmentManager.regionOnline(HRegionInfo.ROOT_REGIONINFO,
this.catalogTracker.getRootLocation());
}
// Enable the ROOT table if on process fail over the RS containing ROOT
// was active.
enableCatalogTables(Bytes.toString(HConstants.ROOT_TABLE_NAME));
// Check for stopped, just in case
if (this.stopped) return false;
LOG.info("-ROOT- assigned=" + assigned + ", rit=" + rit +
", location=" + catalogTracker.getRootLocation());
// Work on meta region
// Work on .META. region. Is it in zk in transition?
status.setStatus("Assigning META region");
assignmentManager.getRegionStates().createRegionState(
HRegionInfo.FIRST_META_REGIONINFO);
rit = this.assignmentManager.
boolean rit = this.assignmentManager.
processRegionInTransitionAndBlockUntilAssigned(HRegionInfo.FIRST_META_REGIONINFO);
boolean metaRegionLocation = this.catalogTracker.verifyMetaRegionLocation(timeout);
ServerName currentMetaServer = null;
boolean metaRegionLocation = catalogTracker.verifyMetaRegionLocation(timeout);
if (!rit && !metaRegionLocation) {
ServerName currentMetaServer =
this.catalogTracker.getMetaLocationOrReadLocationFromRoot();
if (currentMetaServer != null
&& !currentMetaServer.equals(currentRootServer)) {
currentMetaServer = this.catalogTracker.getMetaLocation();
splitLogAndExpireIfOnline(currentMetaServer);
}
assignmentManager.assignMeta();
this.assignmentManager.assignMeta();
enableSSHandWaitForMeta();
// Make sure a .META. location is set.
if (!isMetaLocation()) return false;
// This guarantees that the transition assigning .META. has completed
this.assignmentManager.waitForAssignment(HRegionInfo.FIRST_META_REGIONINFO);
assigned++;
} else if (rit && !metaRegionLocation) {
// Wait until META region added to region server onlineRegions. See HBASE-5875.
enableSSHandWaitForMeta();
// Make sure a .META. location is set.
if (!isMetaLocation()) return false;
// This guarantees that the transition assigning .META. has completed
this.assignmentManager.waitForAssignment(HRegionInfo.FIRST_META_REGIONINFO);
assigned++;
} else {
} else if (metaRegionLocation) {
// Region already assigned. We didn't assign it. Add to in-memory state.
this.assignmentManager.regionOnline(HRegionInfo.FIRST_META_REGIONINFO,
this.catalogTracker.getMetaLocation());
@ -903,7 +870,7 @@ Server {
enableCatalogTables(Bytes.toString(HConstants.META_TABLE_NAME));
LOG.info(".META. assigned=" + assigned + ", rit=" + rit +
", location=" + catalogTracker.getMetaLocation());
status.setStatus("META and ROOT assigned.");
status.setStatus("META assigned.");
return true;
}
@ -916,17 +883,17 @@ Server {
}
/**
* @return True if there a root available
* @return True if there a meta available
* @throws InterruptedException
*/
private boolean isRootLocation() throws InterruptedException {
private boolean isMetaLocation() throws InterruptedException {
// Cycle up here in master rather than down in catalogtracker so we can
// check the master stopped flag every so often.
while (!this.stopped) {
try {
if (this.catalogTracker.waitForRoot(100) != null) break;
if (this.catalogTracker.waitForMeta(100) != null) break;
} catch (NotAllMetaRegionsOnlineException e) {
// Ignore. I know -ROOT- is not online yet.
// Ignore. I know .META. is not online yet.
}
}
// We got here because we came of above loop.
@ -1537,8 +1504,7 @@ Server {
}
private static boolean isCatalogTable(final byte [] tableName) {
return Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME) ||
Bytes.equals(tableName, HConstants.META_TABLE_NAME);
return Bytes.equals(tableName, HConstants.META_TABLE_NAME);
}
@Override
@ -1928,7 +1894,7 @@ Server {
* 1. Create a new ZK session. (since our current one is expired)
* 2. Try to become a primary master again
* 3. Initialize all ZK based system trackers.
* 4. Assign root and meta. (they are already assigned, but we need to update our
* 4. Assign meta. (they are already assigned, but we need to update our
* internal memory state to reflect it)
* 5. Process any RIT if any during the process of our recovery.
*
@ -2146,8 +2112,8 @@ Server {
/**
* ServerShutdownHandlerEnabled is set false before completing
* assignRootAndMeta to prevent processing of ServerShutdownHandler.
* @return true if assignRootAndMeta has completed;
* assignMeta to prevent processing of ServerShutdownHandler.
* @return true if assignMeta has completed;
*/
public boolean isServerShutdownHandlerEnabled() {
return this.serverShutdownHandlerEnabled;

View File

@ -136,8 +136,8 @@ public class MasterFileSystem {
/**
* Create initial layout in filesystem.
* <ol>
* <li>Check if the root region exists and is readable, if not create it.
* Create hbase.version and the -ROOT- directory if not one.
* <li>Check if the meta region exists and is readable, if not create it.
* Create hbase.version and the .META. directory if not one.
* </li>
* <li>Create a log archive directory for RS to put archived logs</li>
* </ol>
@ -438,14 +438,12 @@ public class MasterFileSystem {
}
clusterId = FSUtils.getClusterId(fs, rd);
// Make sure the root region directory exists!
if (!FSUtils.rootRegionExists(fs, rd)) {
// Make sure the meta region directory exists!
if (!FSUtils.metaRegionExists(fs, rd)) {
bootstrap(rd, c);
}
// Create tableinfo-s for ROOT and META if not already there. This also updates the
//descriptors if they are older versions.
FSTableDescriptors.createTableDescriptor(fs, rd, HTableDescriptor.ROOT_TABLEDESC, false);
// Create tableinfo-s for META if not already there.
FSTableDescriptors.createTableDescriptor(fs, rd, HTableDescriptor.META_TABLEDESC, false);
return rd;
@ -479,25 +477,17 @@ public class MasterFileSystem {
private static void bootstrap(final Path rd, final Configuration c)
throws IOException {
LOG.info("BOOTSTRAP: creating ROOT and first META regions");
LOG.info("BOOTSTRAP: creating first META region");
try {
// Bootstrapping, make sure blockcache is off. Else, one will be
// created here in bootstap and it'll need to be cleaned up. Better to
// not make it in first place. Turn off block caching for bootstrap.
// Enable after.
HRegionInfo rootHRI = new HRegionInfo(HRegionInfo.ROOT_REGIONINFO);
setInfoFamilyCachingForRoot(false);
HRegionInfo metaHRI = new HRegionInfo(HRegionInfo.FIRST_META_REGIONINFO);
setInfoFamilyCachingForMeta(false);
HRegion root = HRegion.createHRegion(rootHRI, rd, c,
HTableDescriptor.ROOT_TABLEDESC);
HRegion meta = HRegion.createHRegion(metaHRI, rd, c,
HTableDescriptor.META_TABLEDESC);
setInfoFamilyCachingForRoot(true);
setInfoFamilyCachingForMeta(true);
// Add first region from the META table to the ROOT region.
HRegion.addRegionToMETA(root, meta);
HRegion.closeHRegion(root);
HRegion.closeHRegion(meta);
} catch (IOException e) {
e = RemoteExceptionHandler.checkIOException(e);
@ -506,19 +496,6 @@ public class MasterFileSystem {
}
}
/**
* 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)) {
hcd.setBlockCacheEnabled(b);
hcd.setInMemory(b);
}
}
}
/**
* Enable in memory caching for .META.
*/

View File

@ -60,8 +60,8 @@ public class MasterStatusServlet extends HttpServlet {
Map<String, Integer> frags = getFragmentationInfo(master, conf);
ServerName rootLocation = getRootLocationOrNull(master);
ServerName metaLocation = master.getCatalogTracker().getMetaLocation();
ServerName metaLocation = getMetaLocationOrNull(master);
//ServerName metaLocation = master.getCatalogTracker().getMetaLocation();
List<ServerName> servers = master.getServerManager().getOnlineServersList();
Set<ServerName> deadServers = master.getServerManager().getDeadServers().copyServerNames();
@ -71,7 +71,6 @@ public class MasterStatusServlet extends HttpServlet {
tmpl = new MasterStatusTmpl()
.setFrags(frags)
.setShowAppendWarning(shouldShowAppendWarning(conf))
.setRootLocation(rootLocation)
.setMetaLocation(metaLocation)
.setServers(servers)
.setDeadServers(deadServers)
@ -88,11 +87,11 @@ public class MasterStatusServlet extends HttpServlet {
master, admin);
}
private ServerName getRootLocationOrNull(HMaster master) {
private ServerName getMetaLocationOrNull(HMaster master) {
try {
return master.getCatalogTracker().getRootLocation();
return master.getCatalogTracker().getMetaLocation();
} catch (InterruptedException e) {
LOG.warn("Unable to get root location", e);
LOG.warn("Unable to get meta location", e);
return null;
}
}

View File

@ -468,7 +468,7 @@ public class RegionStates {
} else {
for (Map.Entry<ServerName, Set<HRegionInfo>> e: serverHoldings.entrySet()) {
for (HRegionInfo hri: e.getValue()) {
if (hri.isMetaRegion() || hri.isRootRegion()) continue;
if (hri.isMetaRegion()) continue;
String tablename = hri.getTableNameAsString();
Map<ServerName, List<HRegionInfo>> svrToRegions = result.get(tablename);
if (svrToRegions == null) {

View File

@ -164,7 +164,7 @@ public class ServerManager {
* For all the region servers in this set, HLog split is already completed.
* <p>
* ServerShutdownHandler processes a dead server submitted to the handler after
* the handler is enabled. It may not be able to complete the processing because root/meta
* the handler is enabled. It may not be able to complete the processing because meta
* is not yet online or master is currently in startup mode. In this case, the dead
* server will be parked in this set temporarily.
*/
@ -484,28 +484,26 @@ public class ServerManager {
return;
}
boolean carryingRoot = services.getAssignmentManager().isCarryingRoot(serverName);
boolean carryingMeta = services.getAssignmentManager().isCarryingMeta(serverName);
if (carryingRoot || carryingMeta) {
if (carryingMeta) {
this.services.getExecutorService().submit(new MetaServerShutdownHandler(this.master,
this.services, this.deadservers, serverName, carryingRoot, carryingMeta));
this.services, this.deadservers, serverName, carryingMeta));
} else {
this.services.getExecutorService().submit(new ServerShutdownHandler(this.master,
this.services, this.deadservers, serverName, true));
}
LOG.debug("Added=" + serverName +
" to dead servers, submitted shutdown handler to be executed, root=" +
carryingRoot + ", meta=" + carryingMeta);
" to dead servers, submitted shutdown handler to be executed meta=" + carryingMeta);
}
public synchronized void processDeadServer(final ServerName serverName) {
// When assignment manager is cleaning up the zookeeper nodes and rebuilding the
// in-memory region states, region servers could be down. Root/meta table can and
// in-memory region states, region servers could be down. Meta table can and
// should be re-assigned, log splitting can be done too. However, it is better to
// wait till the cleanup is done before re-assigning user regions.
//
// We should not wait in the server shutdown handler thread since it can clog
// the handler threads and root/meta table could not be re-assigned in case
// the handler threads and meta table could not be re-assigned in case
// the corresponding server is down. So we queue them up here instead.
if (!services.getAssignmentManager().isFailoverCleanupDone()) {
requeuedDeadServers.add(serverName);
@ -519,7 +517,7 @@ public class ServerManager {
/**
* Process the servers which died during master's initialization. It will be
* called after HMaster#assignRootAndMeta and AssignmentManager#joinCluster.
* called after HMaster#assignMeta and AssignmentManager#joinCluster.
* */
synchronized void processQueuedDeadServers() {
if (!services.isServerShutdownHandlerEnabled()) {

View File

@ -376,7 +376,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
// If this is a special region we always try not to move it.
// so clear out r. try again
if (r.isMetaRegion() || r.isRootRegion() ) {
if (r.isMetaRegion()) {
r = null;
}
}

View File

@ -44,9 +44,8 @@ public class ClosedRegionHandler extends EventHandler implements TotesHRegionInf
private final ClosedPriority priority;
private enum ClosedPriority {
ROOT (1),
META (2),
USER (3);
META (1),
USER (2);
private final int value;
ClosedPriority(int value) {
@ -62,9 +61,7 @@ public class ClosedRegionHandler extends EventHandler implements TotesHRegionInf
super(server, EventType.RS_ZK_REGION_CLOSED);
this.assignmentManager = assignmentManager;
this.regionInfo = regionInfo;
if(regionInfo.isRootRegion()) {
priority = ClosedPriority.ROOT;
} else if(regionInfo.isMetaRegion()) {
if(regionInfo.isMetaRegion()) {
priority = ClosedPriority.META;
} else {
priority = ClosedPriority.USER;

View File

@ -37,16 +37,14 @@ import org.apache.zookeeper.KeeperException;
*/
@InterfaceAudience.Private
public class MetaServerShutdownHandler extends ServerShutdownHandler {
private final boolean carryingRoot;
private final boolean carryingMeta;
private static final Log LOG = LogFactory.getLog(MetaServerShutdownHandler.class);
public MetaServerShutdownHandler(final Server server,
final MasterServices services,
final DeadServer deadServers, final ServerName serverName,
final boolean carryingRoot, final boolean carryingMeta) {
final boolean carryingMeta) {
super(server, services, deadServers, serverName,
EventType.M_META_SERVER_SHUTDOWN, true);
this.carryingRoot = carryingRoot;
this.carryingMeta = carryingMeta;
}
@ -65,22 +63,7 @@ public class MetaServerShutdownHandler extends ServerShutdownHandler {
}
// Assign root and meta if we were carrying them.
if (isCarryingRoot()) { // -ROOT-
// Check again: region may be assigned to other where because of RIT
// timeout
if (this.services.getAssignmentManager().isCarryingRoot(serverName)) {
LOG.info("Server " + serverName
+ " was carrying ROOT. Trying to assign.");
this.services.getAssignmentManager().regionOffline(
HRegionInfo.ROOT_REGIONINFO);
verifyAndAssignRootWithRetries();
} else {
LOG.info("ROOT has been assigned to otherwhere, skip assigning.");
}
}
// Carrying meta?
if (isCarryingMeta()) {
if (isCarryingMeta()) { // .META.
// Check again: region may be assigned to other where because of RIT
// timeout
if (this.services.getAssignmentManager().isCarryingMeta(serverName)) {
@ -88,37 +71,37 @@ public class MetaServerShutdownHandler extends ServerShutdownHandler {
+ " was carrying META. Trying to assign.");
this.services.getAssignmentManager().regionOffline(
HRegionInfo.FIRST_META_REGIONINFO);
this.services.getAssignmentManager().assignMeta();
verifyAndAssignMetaWithRetries();
} else {
LOG.info("META has been assigned to otherwhere, skip assigning.");
}
}
super.process();
}
/**
* Before assign the ROOT region, ensure it haven't
* Before assign the META region, ensure it haven't
* been assigned by other place
* <p>
* Under some scenarios, the ROOT region can be opened twice, so it seemed online
* Under some scenarios, the META region can be opened twice, so it seemed online
* in two regionserver at the same time.
* If the ROOT region has been assigned, so the operation can be canceled.
* If the META region has been assigned, so the operation can be canceled.
* @throws InterruptedException
* @throws IOException
* @throws KeeperException
*/
private void verifyAndAssignRoot()
private void verifyAndAssignMeta()
throws InterruptedException, IOException, KeeperException {
long timeout = this.server.getConfiguration().
getLong("hbase.catalog.verification.timeout", 1000);
if (!this.server.getCatalogTracker().verifyRootRegionLocation(timeout)) {
this.services.getAssignmentManager().assignRoot();
} else if (serverName.equals(server.getCatalogTracker().getRootLocation())) {
if (!this.server.getCatalogTracker().verifyMetaRegionLocation(timeout)) {
this.services.getAssignmentManager().assignMeta();
} else if (serverName.equals(server.getCatalogTracker().getMetaLocation())) {
throw new IOException("-ROOT- is onlined on the dead server "
+ serverName);
} else {
LOG.info("Skip assigning -ROOT-, because it is online on the "
+ server.getCatalogTracker().getRootLocation());
+ server.getCatalogTracker().getMetaLocation());
}
}
@ -126,7 +109,7 @@ public class MetaServerShutdownHandler extends ServerShutdownHandler {
* Failed many times, shutdown processing
* @throws IOException
*/
private void verifyAndAssignRootWithRetries() throws IOException {
private void verifyAndAssignMetaWithRetries() throws IOException {
int iTimes = this.server.getConfiguration().getInt(
"hbase.catalog.verification.retries", 10);
@ -136,14 +119,14 @@ public class MetaServerShutdownHandler extends ServerShutdownHandler {
int iFlag = 0;
while (true) {
try {
verifyAndAssignRoot();
verifyAndAssignMeta();
break;
} catch (KeeperException e) {
this.server.abort("In server shutdown processing, assigning root", e);
this.server.abort("In server shutdown processing, assigning meta", e);
throw new IOException("Aborting", e);
} catch (Exception e) {
if (iFlag >= iTimes) {
this.server.abort("verifyAndAssignRoot failed after" + iTimes
this.server.abort("verifyAndAssignMeta failed after" + iTimes
+ " times retries, aborting", e);
throw new IOException("Aborting", e);
}
@ -159,10 +142,6 @@ public class MetaServerShutdownHandler extends ServerShutdownHandler {
}
}
boolean isCarryingRoot() {
return this.carryingRoot;
}
boolean isCarryingMeta() {
return this.carryingMeta;
}

View File

@ -45,9 +45,8 @@ public class OpenedRegionHandler extends EventHandler implements TotesHRegionInf
private final int expectedVersion;
private enum OpenedPriority {
ROOT (1),
META (2),
USER (3);
META (1),
USER (2);
private final int value;
OpenedPriority(int value) {
@ -66,9 +65,7 @@ public class OpenedRegionHandler extends EventHandler implements TotesHRegionInf
this.regionInfo = regionInfo;
this.sn = sn;
this.expectedVersion = expectedVersion;
if(regionInfo.isRootRegion()) {
priority = OpenedPriority.ROOT;
} else if(regionInfo.isMetaRegion()) {
if(regionInfo.isMetaRegion()) {
priority = OpenedPriority.META;
} else {
priority = OpenedPriority.USER;

View File

@ -87,13 +87,6 @@ public class ServerShutdownHandler extends EventHandler {
}
}
/**
* @return True if the server we are processing was carrying <code>-ROOT-</code>
*/
boolean isCarryingRoot() {
return false;
}
/**
* @return True if the server we are processing was carrying <code>.META.</code>
*/
@ -130,8 +123,8 @@ public class ServerShutdownHandler extends EventHandler {
serverName + ", will retry", ioe);
}
// We don't want worker thread in the MetaServerShutdownHandler
// executor pool to block by waiting availability of -ROOT-
// and .META. server. Otherwise, it could run into the following issue:
// executor pool to block by waiting availability of .META.
// Otherwise, it could run into the following issue:
// 1. The current MetaServerShutdownHandler instance For RS1 waits for the .META.
// to come online.
// 2. The newly assigned .META. region server RS2 was shutdown right after
@ -150,7 +143,7 @@ public class ServerShutdownHandler extends EventHandler {
// If AssignmentManager hasn't finished rebuilding user regions,
// we are not ready to assign dead regions either. So we re-queue up
// the dead server for further processing too.
if (isCarryingRoot() || isCarryingMeta() // -ROOT- or .META.
if (isCarryingMeta() // .META.
|| !services.getAssignmentManager().isFailoverCleanupDone()) {
this.services.getServerManager().processDeadServer(serverName);
return;

View File

@ -4078,13 +4078,14 @@ public class HRegion implements HeapSize { // , Writable{
/**
* Inserts a new region's meta information into the passed
* <code>meta</code> region. Used by the HMaster bootstrap code adding
* new table to ROOT table.
* new table to META table.
*
* @param meta META HRegion to be updated
* @param r HRegion to add to <code>meta</code>
*
* @throws IOException
*/
// TODO remove since only test and merge use this
public static void addRegionToMETA(HRegion meta, HRegion r)
throws IOException {
meta.checkResources();
@ -5113,13 +5114,9 @@ public class HRegion implements HeapSize { // , Writable{
final boolean majorCompact)
throws IOException {
HRegion region = null;
String rootStr = Bytes.toString(HConstants.ROOT_TABLE_NAME);
String metaStr = Bytes.toString(HConstants.META_TABLE_NAME);
// Currently expects tables have one region only.
if (p.getName().startsWith(rootStr)) {
region = HRegion.newHRegion(p, log, fs, c, HRegionInfo.ROOT_REGIONINFO,
HTableDescriptor.ROOT_TABLEDESC, null);
} else if (p.getName().startsWith(metaStr)) {
if (p.getName().startsWith(metaStr)) {
region = HRegion.newHRegion(p, log, fs, c,
HRegionInfo.FIRST_META_REGIONINFO, HTableDescriptor.META_TABLEDESC, null);
} else {
@ -5186,10 +5183,10 @@ public class HRegion implements HeapSize { // , Writable{
* is based on the size of the store.
*/
public byte[] checkSplit() {
// Can't split ROOT/META
// Can't split META
if (this.getRegionInfo().isMetaTable()) {
if (shouldForceSplit()) {
LOG.warn("Cannot split root/meta regions in HBase 0.20 and above");
LOG.warn("Cannot split meta region in HBase 0.20 and above");
}
return null;
}

View File

@ -46,7 +46,6 @@ import java.util.TreeSet;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import javax.management.ObjectName;
@ -179,10 +178,8 @@ import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler;
import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler;
import org.apache.hadoop.hbase.regionserver.handler.CloseRootHandler;
import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler;
import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
import org.apache.hadoop.hbase.regionserver.handler.OpenRootHandler;
import org.apache.hadoop.hbase.regionserver.snapshot.RegionServerSnapshotManager;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
@ -202,11 +199,11 @@ import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.VersionInfo;
import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.RootRegionTracker;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.metrics.util.MBeanUtil;
import org.apache.hadoop.net.DNS;
@ -312,7 +309,6 @@ public class HRegionServer implements ClientProtocol,
protected final Configuration conf;
protected final AtomicBoolean haveRootRegion = new AtomicBoolean(false);
private boolean useHBaseChecksum; // verify hbase checksums?
private Path rootDir;
@ -567,7 +563,7 @@ public class HRegionServer implements ClientProtocol,
/**
* Utility used ensuring higher quality of service for priority rpcs; e.g.
* rpcs to .META. and -ROOT-, etc.
* rpcs to .META., etc.
*/
class QosFunction implements Function<RpcRequestBody,Integer> {
private final Map<String, Integer> annotatedQos;
@ -968,7 +964,7 @@ public class HRegionServer implements ClientProtocol,
LOG.info("stopping server " + this.serverNameFromMasterPOV);
}
// Interrupt catalog tracker here in case any regions being opened out in
// handlers are stuck waiting on meta or root.
// handlers are stuck waiting on meta.
if (this.catalogTracker != null) this.catalogTracker.stop();
// stop the snapshot handler, forcefully killing all running tasks
@ -1027,8 +1023,7 @@ public class HRegionServer implements ClientProtocol,
}
private boolean containsMetaTableRegions() {
return onlineRegions.containsKey(HRegionInfo.ROOT_REGIONINFO.getEncodedName())
|| onlineRegions.containsKey(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
return onlineRegions.containsKey(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
}
private boolean areAllUserRegionsOffline() {
@ -1534,14 +1529,10 @@ public class HRegionServer implements ClientProtocol,
this.service = new ExecutorService(getServerName().toString());
this.service.startExecutorService(ExecutorType.RS_OPEN_REGION,
conf.getInt("hbase.regionserver.executor.openregion.threads", 3));
this.service.startExecutorService(ExecutorType.RS_OPEN_ROOT,
conf.getInt("hbase.regionserver.executor.openroot.threads", 1));
this.service.startExecutorService(ExecutorType.RS_OPEN_META,
conf.getInt("hbase.regionserver.executor.openmeta.threads", 1));
this.service.startExecutorService(ExecutorType.RS_CLOSE_REGION,
conf.getInt("hbase.regionserver.executor.closeregion.threads", 3));
this.service.startExecutorService(ExecutorType.RS_CLOSE_ROOT,
conf.getInt("hbase.regionserver.executor.closeroot.threads", 1));
this.service.startExecutorService(ExecutorType.RS_CLOSE_META,
conf.getInt("hbase.regionserver.executor.closemeta.threads", 1));
if (conf.getBoolean(StoreScanner.STORESCANNER_PARALLEL_SEEK_ENABLE, false)) {
@ -1708,13 +1699,10 @@ public class HRegionServer implements ClientProtocol,
LOG.error("No sequence number found when opening " + r.getRegionNameAsString());
openSeqNum = 0;
}
// Update ZK, ROOT or META
if (r.getRegionInfo().isRootRegion()) {
RootRegionTracker.setRootLocation(getZooKeeper(),
// Update ZK, or META
if (r.getRegionInfo().isMetaRegion()) {
MetaRegionTracker.setMetaLocation(getZooKeeper(),
this.serverNameFromMasterPOV);
} else if (r.getRegionInfo().isMetaRegion()) {
MetaEditor.updateMetaLocation(ct, r.getRegionInfo(),
this.serverNameFromMasterPOV, openSeqNum);
} else {
MetaEditor.updateRegionLocation(ct, r.getRegionInfo(),
this.serverNameFromMasterPOV, openSeqNum);
@ -1974,28 +1962,24 @@ public class HRegionServer implements ClientProtocol,
}
/**
* Close root and meta regions if we carry them
* Close meta region if we carry it
* @param abort Whether we're running an abort.
*/
void closeMetaTableRegions(final boolean abort) {
HRegion meta = null;
HRegion root = null;
this.lock.writeLock().lock();
try {
for (Map.Entry<String, HRegion> e: onlineRegions.entrySet()) {
HRegionInfo hri = e.getValue().getRegionInfo();
if (hri.isRootRegion()) {
root = e.getValue();
} else if (hri.isMetaRegion()) {
if (hri.isMetaRegion()) {
meta = e.getValue();
}
if (meta != null && root != null) break;
if (meta != null) break;
}
} finally {
this.lock.writeLock().unlock();
}
if (meta != null) closeRegionIgnoreErrors(meta.getRegionInfo(), abort);
if (root != null) closeRegionIgnoreErrors(root.getRegionInfo(), abort);
}
/**
@ -2490,9 +2474,7 @@ public class HRegionServer implements ClientProtocol,
CloseRegionHandler crh;
final HRegionInfo hri = actualRegion.getRegionInfo();
if (hri.isRootRegion()) {
crh = new CloseRootHandler(this, this, hri, abort, zk, versionOfClosingNode);
} else if (hri.isMetaRegion()) {
if (hri.isMetaRegion()) {
crh = new CloseMetaHandler(this, this, hri, abort, zk, versionOfClosingNode);
} else {
crh = new CloseRegionHandler(this, this, hri, abort, zk, versionOfClosingNode, sn);
@ -3487,10 +3469,7 @@ public class HRegionServer implements ClientProtocol,
if (previous == null) {
// If there is no action in progress, we can submit a specific handler.
// Need to pass the expected version in the constructor.
if (region.isRootRegion()) {
this.service.submit(new OpenRootHandler(this, this, region, htd,
versionOfOfflineNode));
} else if (region.isMetaRegion()) {
if (region.isMetaRegion()) {
this.service.submit(new OpenMetaHandler(this, this, region, htd,
versionOfOfflineNode));
} else {

View File

@ -218,7 +218,7 @@ public class OpenRegionHandler extends EventHandler {
}
/**
* Update ZK, ROOT or META. This can take a while if for example the
* Update ZK or META. This can take a while if for example the
* .META. is not available -- if server hosting .META. crashed and we are
* waiting on it to come back -- so run in a thread and keep updating znode
* state meantime so master doesn't timeout our region-in-transition.

View File

@ -205,9 +205,9 @@ public class AccessController extends BaseRegionObserver
HRegionInfo hri = e.getRegion().getRegionInfo();
byte[] tableName = hri.getTableName();
// 1. All users need read access to .META. and -ROOT- tables.
// 1. All users need read access to .META. table.
// this is a very common operation, so deal with it quickly.
if (hri.isRootRegion() || hri.isMetaRegion()) {
if (hri.isMetaRegion()) {
if (permRequest == Permission.Action.READ) {
return AuthResult.allow(request, "All users allowed", user,
permRequest, tableName, families);
@ -225,7 +225,7 @@ public class AccessController extends BaseRegionObserver
// e.g. When a table is removed an entry is removed from .META. and _acl_
// and the user need to be allowed to write on both tables.
if (permRequest == Permission.Action.WRITE &&
(hri.isRootRegion() || hri.isMetaRegion() ||
(hri.isMetaRegion() ||
Bytes.equals(tableName, AccessControlLists.ACL_GLOBAL_NAME)) &&
(authManager.authorize(user, Permission.Action.CREATE) ||
authManager.authorize(user, Permission.Action.ADMIN)))

View File

@ -453,7 +453,7 @@ public abstract class FSUtils {
throws IOException, DeserializationException {
String version = getVersion(fs, rootdir);
if (version == null) {
if (!rootRegionExists(fs, rootdir)) {
if (!metaRegionExists(fs, rootdir)) {
// rootDir is empty (no version file and no root region)
// just create new version file (HBASE-1195)
setVersion(fs, rootdir, wait, retries);
@ -774,10 +774,10 @@ public abstract class FSUtils {
* @return true if exists
* @throws IOException e
*/
public static boolean rootRegionExists(FileSystem fs, Path rootdir)
public static boolean metaRegionExists(FileSystem fs, Path rootdir)
throws IOException {
Path rootRegionDir =
HRegion.getRegionDir(rootdir, HRegionInfo.ROOT_REGIONINFO);
HRegion.getRegionDir(rootdir, HRegionInfo.FIRST_META_REGIONINFO);
return fs.exists(rootRegionDir);
}

View File

@ -94,7 +94,7 @@ import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
import org.apache.hadoop.hbase.util.hbck.HFileCorruptionChecker;
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.MetaRegionTracker;
import org.apache.hadoop.hbase.zookeeper.ZKTableReadOnly;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.security.AccessControlException;
@ -197,7 +197,7 @@ public class HBaseFsck extends Configured implements Tool {
private boolean fixReferenceFiles = false; // fix lingering reference store file
// limit checking/fixes to listed tables, if empty attempt to check/fix all
// -ROOT- and .META. are always checked
// .META. are always checked
private Set<String> tablesIncluded = new HashSet<String>();
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
@ -234,7 +234,7 @@ public class HBaseFsck extends Configured implements Tool {
* If tablesIncluded is empty, this map contains all tables.
* Otherwise, it contains only meta tables and tables in tablesIncluded,
* unless checkMetaOnly is specified, in which case, it contains only
* the meta tables (.META. and -ROOT-).
* the meta table
*/
private SortedMap<String, TableInfo> tablesInfo = new ConcurrentSkipListMap<String,TableInfo>();
@ -258,7 +258,7 @@ public class HBaseFsck extends Configured implements Tool {
errors = getErrorReporter(conf);
int numThreads = conf.getInt("hbasefsck.numthreads", MAX_NUM_THREADS);
executor = new ScheduledThreadPoolExecutor(numThreads);
executor = new ScheduledThreadPoolExecutor(numThreads, Threads.newDaemonThreadFactory("hbasefsck"));
}
/**
@ -692,8 +692,7 @@ public class HBaseFsck extends Configured implements Tool {
String tableName = td.getNameAsString();
errors.detail(" Table: " + tableName + "\t" +
(td.isReadOnly() ? "ro" : "rw") + "\t" +
(td.isRootRegion() ? "ROOT" :
(td.isMetaRegion() ? "META" : " ")) + "\t" +
(td.isMetaRegion() ? "META" : " ") + "\t" +
" families: " + td.getFamilies().size());
}
}
@ -918,23 +917,14 @@ public class HBaseFsck extends Configured implements Tool {
*
* @return an open .META. HRegion
*/
private HRegion createNewRootAndMeta() throws IOException {
private HRegion createNewMeta() throws IOException {
Path rootdir = FSUtils.getRootDir(getConf());
Configuration c = getConf();
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);
HRegion.closeHRegion(root);
return meta;
}
@ -950,9 +940,8 @@ public class HBaseFsck extends Configured implements Tool {
for (Entry<String, TableInfo> 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) {
// skip ".META."
if (Bytes.compareTo(Bytes.toBytes(name), HConstants.META_TABLE_NAME) == 0) {
continue;
}
@ -1032,24 +1021,24 @@ public class HBaseFsck extends Configured implements Tool {
}
}
// we can rebuild, move old root and meta out of the way and start
// we can rebuild, move old meta out of the way and start
LOG.info("HDFS regioninfo's seems good. Sidelining old .META.");
Path backupDir = sidelineOldRootAndMeta();
Path backupDir = sidelineOldMeta();
LOG.info("Creating new .META.");
HRegion meta = createNewRootAndMeta();
HRegion meta = createNewMeta();
// populate meta
List<Put> puts = generatePuts(tablesInfo);
if (puts == null) {
LOG.fatal("Problem encountered when creating new .META. entries. " +
"You may need to restore the previously sidelined -ROOT- and .META.");
"You may need to restore the previously sidelined .META.");
return false;
}
meta.put(puts.toArray(new Put[0]));
HRegion.closeHRegion(meta);
LOG.info("Success! .META. table rebuilt.");
LOG.info("Old -ROOT- and .META. are moved into " + backupDir);
LOG.info("Old .META. is moved into " + backupDir);
return true;
}
@ -1189,29 +1178,19 @@ public class HBaseFsck extends Configured implements Tool {
/**
* @return Path to backup of original directory
*/
Path sidelineOldRootAndMeta() throws IOException {
// put current -ROOT- and .META. aside.
Path sidelineOldMeta() throws IOException {
// put current .META. aside.
Path hbaseDir = FSUtils.getRootDir(getConf());
FileSystem fs = hbaseDir.getFileSystem(getConf());
Path backupDir = getSidelineDir();
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);
}
LOG.fatal("... failed to sideline meta. Currently in inconsistent state. To restore "
+ "try to rename .META. in " + backupDir.getName() + " to "
+ hbaseDir.getName() + ".", e);
throw e; // throw original exception
}
return backupDir;
@ -1268,7 +1247,6 @@ public class HBaseFsck extends Configured implements Tool {
foundVersionFile = true;
} else {
if ((!checkMetaOnly && isTableIncluded(dirName)) ||
dirName.equals("-ROOT-") ||
dirName.equals(".META.")) {
tableDirs.add(file);
}
@ -1312,31 +1290,29 @@ public class HBaseFsck extends Configured implements Tool {
}
/**
* Record the location of the ROOT region as found in ZooKeeper,
* as if it were in a META table. This is so that we can check
* deployment of ROOT.
* Record the location of the META region as found in ZooKeeper.
*/
private boolean recordRootRegion() throws IOException {
HRegionLocation rootLocation = connection.locateRegion(
HConstants.ROOT_TABLE_NAME, HConstants.EMPTY_START_ROW);
private boolean recordMetaRegion() throws IOException {
HRegionLocation metaLocation = connection.locateRegion(
HConstants.META_TABLE_NAME, HConstants.EMPTY_START_ROW);
// Check if Root region is valid and existing
if (rootLocation == null || rootLocation.getRegionInfo() == null ||
rootLocation.getHostname() == null) {
errors.reportError(ERROR_CODE.NULL_ROOT_REGION,
"Root Region or some of its attributes are null.");
// Check if Meta region is valid and existing
if (metaLocation == null || metaLocation.getRegionInfo() == null ||
metaLocation.getHostname() == null) {
errors.reportError(ERROR_CODE.NULL_META_REGION,
"META region or some of its attributes are null.");
return false;
}
ServerName sn;
try {
sn = getRootRegionServerName();
sn = getMetaRegionServerName();
} catch (KeeperException e) {
throw new IOException(e);
}
MetaEntry m =
new MetaEntry(rootLocation.getRegionInfo(), sn, System.currentTimeMillis());
new MetaEntry(metaLocation.getRegionInfo(), sn, System.currentTimeMillis());
HbckInfo hbInfo = new HbckInfo(m);
regionInfoMap.put(rootLocation.getRegionInfo().getEncodedName(), hbInfo);
regionInfoMap.put(metaLocation.getRegionInfo().getEncodedName(), hbInfo);
return true;
}
@ -1356,12 +1332,12 @@ public class HBaseFsck extends Configured implements Tool {
});
}
private ServerName getRootRegionServerName()
private ServerName getMetaRegionServerName()
throws IOException, KeeperException {
ZooKeeperWatcher zkw = createZooKeeperWatcher();
ServerName sn = null;
try {
sn = RootRegionTracker.getRootRegionLocation(zkw);
sn = MetaRegionTracker.getMetaRegionLocation(zkw);
} finally {
zkw.close();
}
@ -2524,14 +2500,14 @@ public class HBaseFsck extends Configured implements Tool {
}
/**
* Scan .META. and -ROOT-, adding all regions found to the regionInfo map.
* Scan .META., adding all regions found to the regionInfo map.
* @throws IOException if an error is encountered
*/
boolean loadMetaEntries() throws IOException {
// get a list of all regions from the master. This involves
// scanning the META table
if (!recordRootRegion()) {
if (!recordMetaRegion()) {
// Will remove later if we can fix it
errors.reportError("Fatal error: unable to get root region location. Exiting...");
return false;
@ -2563,7 +2539,7 @@ public class HBaseFsck extends Configured implements Tool {
}
HRegionInfo hri = pair.getFirst();
if (!(isTableIncluded(hri.getTableNameAsString())
|| hri.isMetaRegion() || hri.isRootRegion())) {
|| hri.isMetaRegion())) {
return true;
}
PairOfSameType<HRegionInfo> daughters = HRegionInfo.getDaughterRegions(result);
@ -2586,11 +2562,6 @@ public class HBaseFsck extends Configured implements Tool {
}
}
};
// Scan -ROOT- to pick up META regions
MetaScanner.metaScan(getConf(), visitor, null, null,
Integer.MAX_VALUE, HConstants.ROOT_TABLE_NAME);
if (!checkMetaOnly) {
// Scan .META. to pick up user regions
MetaScanner.metaScan(getConf(), visitor);
@ -2874,7 +2845,7 @@ public class HBaseFsck extends Configured implements Tool {
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,
UNKNOWN, NO_META_REGION, NULL_META_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,
MULTI_DEPLOYED, SHOULD_NOT_BE_DEPLOYED, MULTI_META_REGION, RS_CONNECT_FAILURE,
FIRST_REGION_STARTKEY_NOT_EMPTY, LAST_REGION_ENDKEY_NOT_EMPTY, DUPE_STARTKEYS,
@ -3419,8 +3390,8 @@ public class HBaseFsck extends Configured implements Tool {
out.println(" -sleepBeforeRerun <timeInSeconds> Sleep this many seconds" +
" before checking if the fix worked if run with -fix");
out.println(" -summary Print only summary of the tables and status.");
out.println(" -metaonly Only check the state of ROOT and META tables.");
out.println(" -sidelineDir <hdfs://> HDFS path to backup existing meta and root.");
out.println(" -metaonly Only check the state of the .META. table.");
out.println(" -sidelineDir <hdfs://> HDFS path to backup existing meta.");
out.println("");
out.println(" Metadata Repair options: (expert features, use with caution!)");

View File

@ -121,7 +121,7 @@ class HMerge {
throw new IllegalStateException(
"Can not compact META table if instance is on-line");
}
new OfflineMerger(conf, fs).process();
// TODO reenable new OfflineMerger(conf, fs).process();
} else {
if(!masterIsRunning) {
throw new IllegalStateException(
@ -333,90 +333,4 @@ class HMerge {
}
}
}
/** Instantiated to compact the meta region */
private static class OfflineMerger extends Merger {
private final List<HRegionInfo> metaRegions = new ArrayList<HRegionInfo>();
private final HRegion root;
OfflineMerger(Configuration conf, FileSystem fs)
throws IOException {
super(conf, fs, HConstants.META_TABLE_NAME);
Path rootDir = FSUtils.getRootDir(conf);
// Scan root region to find all the meta regions
root = HRegion.openHRegion(conf, fs, rootDir, HRegionInfo.ROOT_REGIONINFO,
HTableDescriptor.ROOT_TABLEDESC, hlog);
Scan scan = new Scan();
scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
InternalScanner rootScanner = root.getScanner(scan);
try {
List<KeyValue> results = new ArrayList<KeyValue>();
boolean hasMore;
do {
hasMore = rootScanner.next(results);
for(KeyValue kv: results) {
HRegionInfo info = HRegionInfo.parseFromOrNull(kv.getValue());
if (info != null) {
metaRegions.add(info);
}
}
} while (hasMore);
} finally {
rootScanner.close();
try {
root.close();
} catch(IOException e) {
LOG.error(e);
}
}
}
@Override
protected HRegionInfo[] next() {
HRegionInfo[] results = null;
if (metaRegions.size() > 0) {
results = metaRegions.toArray(new HRegionInfo[metaRegions.size()]);
metaRegions.clear();
}
return results;
}
@Override
protected void updateMeta(final byte [] oldRegion1,
final byte [] oldRegion2, HRegion newRegion)
throws IOException {
byte[][] regionsToDelete = {oldRegion1, oldRegion2};
for(int r = 0; r < regionsToDelete.length; r++) {
Delete delete = new Delete(regionsToDelete[r]);
delete.deleteColumns(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
delete.deleteColumns(HConstants.CATALOG_FAMILY,
HConstants.SERVER_QUALIFIER);
delete.deleteColumns(HConstants.CATALOG_FAMILY,
HConstants.STARTCODE_QUALIFIER);
delete.deleteColumns(HConstants.CATALOG_FAMILY,
HConstants.SPLITA_QUALIFIER);
delete.deleteColumns(HConstants.CATALOG_FAMILY,
HConstants.SPLITB_QUALIFIER);
root.delete(delete, true);
if(LOG.isDebugEnabled()) {
LOG.debug("updated columns in row: " + Bytes.toStringBinary(regionsToDelete[r]));
}
}
HRegionInfo newInfo = newRegion.getRegionInfo();
newInfo.setOffline(true);
Put put = MetaEditor.makePutFromRegionInfo(newInfo);
root.put(put);
if(LOG.isDebugEnabled()) {
LOG.debug("updated columns in row: " + Bytes.toStringBinary(newRegion.getRegionName()));
}
}
}
}

View File

@ -109,23 +109,10 @@ public class Merge extends Configured implements Tool {
this.utils = new MetaUtils(getConf());
this.rootdir = FSUtils.getRootDir(getConf());
try {
if (isMetaTable) {
mergeTwoMetaRegions();
} else {
mergeTwoRegions();
}
return 0;
} catch (Exception e) {
LOG.fatal("Merge failed", e);
utils.scanMetaRegion(HRegionInfo.FIRST_META_REGIONINFO,
new MetaUtils.ScannerListener() {
public boolean processRow(HRegionInfo info) {
System.err.println(info.toString());
return true;
}
}
);
return -1;
} finally {
@ -140,156 +127,57 @@ public class Merge extends Configured implements Tool {
return this.mergeInfo;
}
/*
* Merge two meta regions. This is unlikely to be needed soon as we have only
* seend the meta table split once and that was with 64MB regions. With 256MB
* regions, it will be some time before someone has enough data in HBase to
* split the meta region and even less likely that a merge of two meta
* regions will be needed, but it is included for completeness.
*/
private void mergeTwoMetaRegions() throws IOException {
HRegion rootRegion = utils.getRootRegion();
Get get = new Get(region1);
get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
Result result1 = rootRegion.get(get);
Preconditions.checkState(!result1.isEmpty(), "First region cells can not be null");
HRegionInfo info1 = HRegionInfo.getHRegionInfo(result1);
get = new Get(region2);
get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
Result result2 = rootRegion.get(get);
Preconditions.checkState(!result2.isEmpty(), "Second region cells can not be null");
HRegionInfo info2 = HRegionInfo.getHRegionInfo(result2);
HRegion merged = merge(HTableDescriptor.META_TABLEDESC, info1, rootRegion, info2, rootRegion);
LOG.info("Adding " + merged.getRegionInfo() + " to " +
rootRegion.getRegionInfo());
HRegion.addRegionToMETA(rootRegion, merged);
merged.close();
}
private static class MetaScannerListener
implements MetaUtils.ScannerListener {
private final byte [] region1;
private final byte [] region2;
private HRegionInfo meta1 = null;
private HRegionInfo meta2 = null;
MetaScannerListener(final byte [] region1, final byte [] region2) {
this.region1 = region1;
this.region2 = region2;
}
public boolean processRow(HRegionInfo info) {
if (meta1 == null && HRegion.rowIsInRange(info, region1)) {
meta1 = info;
}
if (region2 != null && meta2 == null &&
HRegion.rowIsInRange(info, region2)) {
meta2 = info;
}
return meta1 == null || (region2 != null && meta2 == null);
}
HRegionInfo getMeta1() {
return meta1;
}
HRegionInfo getMeta2() {
return meta2;
}
}
/*
* Merges two regions from a user table.
*/
private void mergeTwoRegions() throws IOException {
LOG.info("Merging regions " + Bytes.toStringBinary(this.region1) + " and " +
Bytes.toStringBinary(this.region2) + " in table " + Bytes.toString(this.tableName));
// Scan the root region for all the meta regions that contain the regions
// we're merging.
MetaScannerListener listener = new MetaScannerListener(region1, region2);
this.utils.scanRootRegion(listener);
HRegionInfo meta1 = listener.getMeta1();
if (meta1 == null) {
throw new IOException("Could not find meta region for " + Bytes.toStringBinary(region1));
}
HRegionInfo meta2 = listener.getMeta2();
if (meta2 == null) {
throw new IOException("Could not find meta region for " + Bytes.toStringBinary(region2));
}
LOG.info("Found meta for region1 " + Bytes.toStringBinary(meta1.getRegionName()) +
", meta for region2 " + Bytes.toStringBinary(meta2.getRegionName()));
HRegion metaRegion1 = this.utils.getMetaRegion(meta1);
HRegion meta = this.utils.getMetaRegion();
Get get = new Get(region1);
get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
Result result1 = metaRegion1.get(get);
Result result1 = meta.get(get);
Preconditions.checkState(!result1.isEmpty(),
"First region cells can not be null");
HRegionInfo info1 = HRegionInfo.getHRegionInfo(result1);
if (info1 == null) {
throw new NullPointerException("info1 is null using key " +
Bytes.toStringBinary(region1) + " in " + meta1);
}
HRegion metaRegion2;
if (Bytes.equals(meta1.getRegionName(), meta2.getRegionName())) {
metaRegion2 = metaRegion1;
} else {
metaRegion2 = utils.getMetaRegion(meta2);
Bytes.toStringBinary(region1) + " in " + meta);
}
get = new Get(region2);
get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
Result result2 = metaRegion2.get(get);
Result result2 = meta.get(get);
Preconditions.checkState(!result2.isEmpty(),
"Second region cells can not be null");
HRegionInfo info2 = HRegionInfo.getHRegionInfo(result2);
if (info2 == null) {
throw new NullPointerException("info2 is null using key " + meta2);
throw new NullPointerException("info2 is null using key " + meta);
}
HTableDescriptor htd = FSTableDescriptors.getTableDescriptor(FileSystem.get(getConf()),
this.rootdir, this.tableName);
HRegion merged = merge(htd, info1, metaRegion1, info2, metaRegion2);
HRegion merged = merge(htd, meta, info1, info2);
// Now find the meta region which will contain the newly merged region
listener = new MetaScannerListener(merged.getRegionName(), null);
utils.scanRootRegion(listener);
HRegionInfo mergedInfo = listener.getMeta1();
if (mergedInfo == null) {
throw new IOException("Could not find meta region for " +
Bytes.toStringBinary(merged.getRegionName()));
}
HRegion mergeMeta;
if (Bytes.equals(mergedInfo.getRegionName(), meta1.getRegionName())) {
mergeMeta = metaRegion1;
} else if (Bytes.equals(mergedInfo.getRegionName(), meta2.getRegionName())) {
mergeMeta = metaRegion2;
} else {
mergeMeta = utils.getMetaRegion(mergedInfo);
}
LOG.info("Adding " + merged.getRegionInfo() + " to " +
mergeMeta.getRegionInfo());
meta.getRegionInfo());
HRegion.addRegionToMETA(mergeMeta, merged);
HRegion.addRegionToMETA(meta, merged);
merged.close();
}
/*
* Actually merge two regions and update their info in the meta region(s)
* If the meta is split, meta1 may be different from meta2. (and we may have
* to scan the meta if the resulting merged region does not go in either)
* Returns HRegion object for newly merged region
*/
private HRegion merge(final HTableDescriptor htd, HRegionInfo info1,
HRegion meta1, HRegionInfo info2, HRegion meta2)
private HRegion merge(final HTableDescriptor htd, HRegion meta,
HRegionInfo info1, HRegionInfo info2)
throws IOException {
if (info1 == null) {
throw new IOException("Could not find " + Bytes.toStringBinary(region1) + " in " +
Bytes.toStringBinary(meta1.getRegionName()));
Bytes.toStringBinary(meta.getRegionName()));
}
if (info2 == null) {
throw new IOException("Cound not find " + Bytes.toStringBinary(region2) + " in " +
Bytes.toStringBinary(meta2.getRegionName()));
Bytes.toStringBinary(meta.getRegionName()));
}
HRegion merged = null;
HLog log = utils.getLog();
@ -312,8 +200,8 @@ public class Merge extends Configured implements Tool {
// Remove the old regions from meta.
// HRegion.merge has already deleted their files
removeRegionFromMeta(meta1, info1);
removeRegionFromMeta(meta2, info2);
removeRegionFromMeta(meta, info1);
removeRegionFromMeta(meta, info2);
this.mergeInfo = merged.getRegionInfo();
return merged;

View File

@ -31,7 +31,6 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
@ -62,7 +61,7 @@ public class MetaUtils {
private final Configuration conf;
private FileSystem fs;
private HLog log;
private HRegion rootRegion;
private HRegion metaRegion;
private Map<byte [], HRegion> metaRegions = Collections.synchronizedSortedMap(
new TreeMap<byte [], HRegion>(Bytes.BYTES_COMPARATOR));
@ -80,7 +79,7 @@ public class MetaUtils {
public MetaUtils(Configuration conf) throws IOException {
this.conf = conf;
conf.setInt("hbase.client.retries.number", 1);
this.rootRegion = null;
this.metaRegion = null;
initialize();
}
@ -107,31 +106,14 @@ public class MetaUtils {
}
/**
* @return HRegion for root region
* @return HRegion for meta region
* @throws IOException e
*/
public HRegion getRootRegion() throws IOException {
if (this.rootRegion == null) {
openRootRegion();
public HRegion getMetaRegion() throws IOException {
if (this.metaRegion == null) {
openMetaRegion();
}
return this.rootRegion;
}
/**
* Open or return cached opened meta region
*
* @param metaInfo HRegionInfo for meta region
* @return meta HRegion
* @throws IOException e
*/
public HRegion getMetaRegion(HRegionInfo metaInfo) throws IOException {
HRegion meta = metaRegions.get(metaInfo.getRegionName());
if (meta == null) {
meta = openMetaRegion(metaInfo);
LOG.info("OPENING META " + meta.toString());
this.metaRegions.put(metaInfo.getRegionName(), meta);
}
return meta;
return this.metaRegion;
}
/**
@ -140,13 +122,13 @@ public class MetaUtils {
* MetaUtils edit session.
*/
public void shutdown() {
if (this.rootRegion != null) {
if (this.metaRegion != null) {
try {
this.rootRegion.close();
this.metaRegion.close();
} catch (IOException e) {
LOG.error("closing root region", e);
LOG.error("closing meta region", e);
} finally {
this.rootRegion = null;
this.metaRegion = null;
}
}
try {
@ -171,229 +153,14 @@ public class MetaUtils {
}
}
/**
* Used by scanRootRegion and scanMetaRegion to call back the caller so it
* can process the data for a row.
*/
public interface ScannerListener {
/**
* Callback so client of scanner can process row contents
*
* @param info HRegionInfo for row
* @return false to terminate the scan
* @throws IOException e
*/
public boolean processRow(HRegionInfo info) throws IOException;
private synchronized HRegion openMetaRegion() throws IOException {
if (this.metaRegion != null) {
return this.metaRegion;
}
/**
* Scans the root region. For every meta region found, calls the listener with
* the HRegionInfo of the meta region.
*
* @param listener method to be called for each meta region found
* @throws IOException e
*/
public void scanRootRegion(ScannerListener listener) throws IOException {
// Open root region so we can scan it
if (this.rootRegion == null) {
openRootRegion();
}
scanMetaRegion(this.rootRegion, listener);
}
/**
* Scan the passed in metaregion <code>m</code> invoking the passed
* <code>listener</code> per row found.
* @param r region
* @param listener scanner listener
* @throws IOException e
*/
public void scanMetaRegion(final HRegion r, final ScannerListener listener)
throws IOException {
Scan scan = new Scan();
scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
InternalScanner s = r.getScanner(scan);
try {
List<KeyValue> results = new ArrayList<KeyValue>();
boolean hasNext = true;
do {
hasNext = s.next(results);
HRegionInfo info = null;
for (KeyValue kv: results) {
info = HRegionInfo.parseFromOrNull(kv.getValue());
if (info == null) {
LOG.warn("Region info is null for row " +
Bytes.toStringBinary(kv.getRow()) + " in table " +
r.getTableDesc().getNameAsString());
}
continue;
}
if (!listener.processRow(info)) {
break;
}
results.clear();
} while (hasNext);
} finally {
s.close();
}
}
/**
* Scans a meta region. For every region found, calls the listener with
* the HRegionInfo of the region.
* TODO: Use Visitor rather than Listener pattern. Allow multiple Visitors.
* Use this everywhere we scan meta regions: e.g. in metascanners, in close
* handling, etc. Have it pass in the whole row, not just HRegionInfo.
* <p>Use for reading meta only. Does not close region when done.
* Use {@link #getMetaRegion(HRegionInfo)} instead if writing. Adds
* meta region to list that will get a close on {@link #shutdown()}.
*
* @param metaRegionInfo HRegionInfo for meta region
* @param listener method to be called for each meta region found
* @throws IOException e
*/
public void scanMetaRegion(HRegionInfo metaRegionInfo,
ScannerListener listener)
throws IOException {
// Open meta region so we can scan it
HRegion metaRegion = openMetaRegion(metaRegionInfo);
scanMetaRegion(metaRegion, listener);
}
private synchronized HRegion openRootRegion() throws IOException {
if (this.rootRegion != null) {
return this.rootRegion;
}
this.rootRegion = HRegion.openHRegion(HRegionInfo.ROOT_REGIONINFO,
HTableDescriptor.ROOT_TABLEDESC, getLog(),
this.metaRegion = HRegion.openHRegion(HRegionInfo.FIRST_META_REGIONINFO,
HTableDescriptor.META_TABLEDESC, getLog(),
this.conf);
this.rootRegion.compactStores();
return this.rootRegion;
}
private HRegion openMetaRegion(HRegionInfo metaInfo) throws IOException {
HRegion meta = HRegion.openHRegion(metaInfo, HTableDescriptor.META_TABLEDESC,
getLog(), this.conf);
meta.compactStores();
return meta;
}
/**
* Set a single region on/offline.
* This is a tool to repair tables that have offlined tables in their midst.
* Can happen on occasion. Use at your own risk. Call from a bit of java
* or jython script. This method is 'expensive' in that it creates a
* {@link HTable} instance per invocation to go against <code>.META.</code>
* @param c A configuration that has its <code>hbase.master</code>
* properly set.
* @param row Row in the catalog .META. table whose HRegionInfo's offline
* status we want to change.
* @param onlineOffline Pass <code>true</code> to OFFLINE the region.
* @throws IOException e
*/
public static void changeOnlineStatus (final Configuration c,
final byte [] row, final boolean onlineOffline)
throws IOException {
HTable t = new HTable(c, HConstants.META_TABLE_NAME);
Get get = new Get(row);
get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
Result res = t.get(get);
KeyValue [] kvs = res.raw();
if(kvs.length <= 0) {
throw new IOException("no information for row " + Bytes.toString(row));
}
HRegionInfo info = HRegionInfo.getHRegionInfo(res);
if (info == null) {
throw new IOException("no information for row " + Bytes.toString(row));
}
info.setOffline(onlineOffline);
MetaEditor.addRegionToMeta(t, info);
Delete delete = new Delete(row);
delete.deleteColumns(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
delete.deleteColumns(HConstants.CATALOG_FAMILY,
HConstants.STARTCODE_QUALIFIER);
t.delete(delete);
}
/**
* Update COL_REGIONINFO in meta region r with HRegionInfo hri
*
* @param r region
* @param hri region info
* @throws IOException e
*/
public void updateMETARegionInfo(HRegion r, final HRegionInfo hri)
throws IOException {
if (LOG.isDebugEnabled()) {
Get get = new Get(hri.getRegionName());
get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
Result res = r.get(get);
KeyValue [] kvs = res.raw();
if(kvs.length <= 0) {
return;
}
HRegionInfo h = HRegionInfo.getHRegionInfo(res);
if (h == null) {
return;
}
LOG.debug("Old " + Bytes.toString(HConstants.CATALOG_FAMILY) + ":" +
Bytes.toString(HConstants.REGIONINFO_QUALIFIER) + " for " +
hri.toString() + " in " + r.toString() + " is: " + h.toString());
}
Put put = MetaEditor.makePutFromRegionInfo(hri);
r.put(put);
if (LOG.isDebugEnabled()) {
Get get = new Get(hri.getRegionName());
get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
Result res = r.get(get);
KeyValue [] kvs = res.raw();
if(kvs.length <= 0) {
return;
}
HRegionInfo h = HRegionInfo.getHRegionInfo(res);
if (h == null) {
return;
}
LOG.debug("New " + Bytes.toString(HConstants.CATALOG_FAMILY) + ":" +
Bytes.toString(HConstants.REGIONINFO_QUALIFIER) + " for " +
hri.toString() + " in " + r.toString() + " is: " + h.toString());
}
}
/**
* @return List of {@link HRegionInfo} rows found in the ROOT or META
* catalog table.
* @param tableName Name of table to go looking for.
* @throws IOException e
* @see #getMetaRegion(HRegionInfo)
*/
public List<HRegionInfo> getMETARows(final byte [] tableName)
throws IOException {
final List<HRegionInfo> result = new ArrayList<HRegionInfo>();
// If passed table name is META, then return the root region.
if (Bytes.equals(HConstants.META_TABLE_NAME, tableName)) {
result.add(openRootRegion().getRegionInfo());
return result;
}
// Return all meta regions that contain the passed tablename.
scanRootRegion(new ScannerListener() {
private final Log SL_LOG = LogFactory.getLog(this.getClass());
public boolean processRow(HRegionInfo info) throws IOException {
SL_LOG.debug("Testing " + info);
if (Bytes.equals(info.getTableName(),
HConstants.META_TABLE_NAME)) {
result.add(info);
return false;
}
return true;
}});
return result;
this.metaRegion.compactStores();
return this.metaRegion;
}
}

View File

@ -41,7 +41,7 @@
String tableName = request.getParameter("name");
HTable table = new HTable(conf, tableName);
String tableHeader = "<h2>Table Regions</h2><table class=\"table table-striped\"><tr><th>Name</th><th>Region Server</th><th>Start Key</th><th>End Key</th><th>Requests</th></tr>";
ServerName rl = master.getCatalogTracker().getRootLocation();
ServerName rl = master.getCatalogTracker().getMetaLocation();
boolean showFragmentation = conf.getBoolean("hbase.master.ui.fragmentation.enabled", false);
boolean readOnly = conf.getBoolean("hbase.master.ui.readonly", false);
Map<String, Integer> frags = null;
@ -191,22 +191,7 @@
</div>
<div class="row">
<%
if(tableName.equals(Bytes.toString(HConstants.ROOT_TABLE_NAME))) {
%>
<%= tableHeader %>
<%
String url = "http://" + rl.getHostname() + ":" + infoPort + "/";
%>
<tr>
<td><%= tableName %></td>
<td><a href="<%= url %>"><%= rl.getHostname() %>:<%= rl.getPort() %></a></td>
<td>-</td>
<td></td>
<td>-</td>
</tr>
</table>
<%
} else if(tableName.equals(Bytes.toString(HConstants.META_TABLE_NAME))) {
if(tableName.equals(Bytes.toString(HConstants.META_TABLE_NAME))) {
%>
<%= tableHeader %>
<%

View File

@ -313,7 +313,6 @@ module Hbase
def describe(table_name)
tables = @admin.listTables.to_a
tables << org.apache.hadoop.hbase.HTableDescriptor::META_TABLEDESC
tables << org.apache.hadoop.hbase.HTableDescriptor::ROOT_TABLEDESC
tables.each do |t|
# Found the table

View File

@ -459,7 +459,7 @@ EOF
# Checks if current table is one of the 'meta' tables
def is_meta_table?
tn = @table.table_name
org.apache.hadoop.hbase.util.Bytes.equals(tn, org.apache.hadoop.hbase.HConstants::META_TABLE_NAME) || org.apache.hadoop.hbase.util.Bytes.equals(tn, org.apache.hadoop.hbase.HConstants::ROOT_TABLE_NAME)
org.apache.hadoop.hbase.util.Bytes.equals(tn, org.apache.hadoop.hbase.HConstants::META_TABLE_NAME)
end
# Returns family and (when has it) qualifier for a column name

View File

@ -60,7 +60,6 @@ public abstract class HBaseTestCase extends TestCase {
private boolean localfs = false;
protected static Path testDir = null;
protected FileSystem fs = null;
protected HRegion root = null;
protected HRegion meta = null;
protected static final char FIRST_CHAR = 'a';
protected static final char LAST_CHAR = 'z';
@ -656,17 +655,13 @@ public abstract class HBaseTestCase extends TestCase {
* method. It does cleanup.
* @throws IOException
*/
protected void createRootAndMetaRegions() throws IOException {
root = HRegion.createHRegion(HRegionInfo.ROOT_REGIONINFO, testDir,
conf, HTableDescriptor.ROOT_TABLEDESC);
protected void createMetaRegion() throws IOException {
meta = HRegion.createHRegion(HRegionInfo.FIRST_META_REGIONINFO, testDir,
conf, HTableDescriptor.META_TABLEDESC);
HRegion.addRegionToMETA(root, meta);
}
protected void closeRootAndMeta() throws IOException {
HRegion.closeHRegion(meta);
HRegion.closeHRegion(root);
}
public static void assertByteEquals(byte[] expected,

View File

@ -290,7 +290,7 @@ public class TestDrainingServer {
private static boolean isAllRegionsOnline() {
return TEST_UTIL.getMiniHBaseCluster().countServedRegions() ==
(COUNT_OF_REGIONS + 2 /*catalog regions*/);
(COUNT_OF_REGIONS + 1 /*catalog regions*/);
}
}

View File

@ -188,7 +188,7 @@ public class TestRegionRebalancing {
if (!(avg > 2.0 && serverLoad <= avgLoadPlusSlop
&& serverLoad >= avgLoadMinusSlop)) {
for (HRegionInfo hri : ProtobufUtil.getOnlineRegions(server)) {
if (hri.isMetaRegion() || hri.isRootRegion()) serverLoad--;
if (hri.isMetaRegion()) serverLoad--;
// LOG.debug(hri.getRegionNameAsString());
}
if (!(serverLoad <= avgLoadPlusSlop && serverLoad >= avgLoadMinusSlop)) {
@ -234,7 +234,7 @@ public class TestRegionRebalancing {
* Wait until all the regions are assigned.
*/
private void waitForAllRegionsAssigned() throws IOException {
int totalRegions = HBaseTestingUtility.KEYS.length+2;
int totalRegions = HBaseTestingUtility.KEYS.length+1;
while (getRegionCount() < totalRegions) {
// while (!cluster.getMaster().allRegionsAssigned()) {
LOG.debug("Waiting for there to be "+ totalRegions +" regions, but there are " + getRegionCount() + " right now.");

View File

@ -54,7 +54,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetResponse;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.RootRegionTracker;
import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.util.Progressable;
@ -132,22 +132,22 @@ public class TestCatalogTracker {
HConnection connection = Mockito.mock(HConnection.class);
constructAndStartCatalogTracker(connection);
try {
RootRegionTracker.setRootLocation(this.watcher,
MetaRegionTracker.setMetaLocation(this.watcher,
new ServerName("example.com", 1234, System.currentTimeMillis()));
} finally {
// Clean out root location or later tests will be confused... they presume
// Clean out meta location or later tests will be confused... they presume
// start fresh in zk.
RootRegionTracker.deleteRootLocation(this.watcher);
MetaRegionTracker.deleteMetaLocation(this.watcher);
}
}
/**
* Test interruptable while blocking wait on root and meta.
* Test interruptable while blocking wait on meta.
* @throws IOException
* @throws ServiceException
* @throws InterruptedException
*/
@Test public void testInterruptWaitOnMetaAndRoot()
@Test public void testInterruptWaitOnMeta()
throws IOException, InterruptedException, ServiceException {
final ClientProtocol client = Mockito.mock(ClientProtocol.class);
HConnection connection = mockConnection(null, client);
@ -155,8 +155,6 @@ public class TestCatalogTracker {
Mockito.when(client.get((RpcController)Mockito.any(), (GetRequest)Mockito.any())).
thenReturn(GetResponse.newBuilder().build());
final CatalogTracker ct = constructAndStartCatalogTracker(connection);
ServerName hsa = ct.getRootLocation();
Assert.assertNull(hsa);
ServerName meta = ct.getMetaLocation();
Assert.assertNull(meta);
Thread t = new Thread() {
@ -182,86 +180,6 @@ public class TestCatalogTracker {
}
}
/**
* Test for HBASE-4288. Throw an IOE when trying to verify meta region and
* prove it doesn't cause master shutdown.
* @see <a href="https://issues.apache.org/jira/browse/HBASE-4288">HBASE-4288</a>
* @throws IOException
* @throws InterruptedException
* @throws KeeperException
*/
@Test
public void testServerNotRunningIOException()
throws IOException, InterruptedException, KeeperException, ServiceException {
// Mock an Admin and a Client.
final AdminProtocol admin = Mockito.mock(AdminProtocol.class);
final ClientProtocol client = Mockito.mock(ClientProtocol.class);
HConnection connection = mockConnection(admin, client);
try {
// If a 'getRegionInfo' is called on mocked AdminProtocol, throw IOE
// the first time. 'Succeed' the second time we are called.
GetRegionInfoResponse.Builder builder = GetRegionInfoResponse.newBuilder();
builder.setRegionInfo(HRegionInfo.convert(new HRegionInfo(Bytes.toBytes("test"))));
Mockito.when(admin.getRegionInfo((RpcController)Mockito.any(),
(GetRegionInfoRequest)Mockito.any())).thenThrow(
new ServiceException(new IOException("Server not running, aborting"))).
thenReturn(builder.build());
// After we encounter the above 'Server not running', we should catch the
// IOE and go into retrying for the meta mode. We'll do gets on -ROOT- to
// get new meta location. Return something so this 'get' succeeds
// (here we mock up getRegionServerWithRetries, the wrapper around
// the actual get).
// TODO: Refactor. This method has been moved out of HConnection.
// It works for now but has been deprecated.
Mockito.when(connection.getRegionServerWithRetries((ServerCallable<Result>)Mockito.any())).
thenReturn(getMetaTableRowResult());
Mockito.when(client.get((RpcController)Mockito.any(), (GetRequest)Mockito.any())).
thenReturn(GetResponse.newBuilder().build());
// Now start up the catalogtracker with our doctored Connection.
final CatalogTracker ct = constructAndStartCatalogTracker(connection);
try {
// Set a location for root and meta.
RootRegionTracker.setRootLocation(this.watcher, SN);
ct.setMetaLocation(SN);
// Call the method that HBASE-4288 calls. It will try and verify the
// meta location and will fail on first attempt then go into a long wait.
// So, do this in a thread and then reset meta location to break it out
// of its wait after a bit of time.
final AtomicBoolean metaSet = new AtomicBoolean(false);
final CountDownLatch latch = new CountDownLatch(1);
Thread t = new Thread() {
@Override
public void run() {
try {
latch.countDown();
metaSet.set(ct.waitForMeta(100000) != null);
} catch (Exception e) {
throw new RuntimeException(e);
}
}
};
t.start();
latch.await();
Threads.sleep(1);
// Now reset the meta as though it were redeployed.
ct.setMetaLocation(SN);
t.join();
Assert.assertTrue(metaSet.get());
} finally {
// Clean out root and meta locations or later tests will be confused...
// they presume start fresh in zk.
ct.resetMetaLocation();
RootRegionTracker.deleteRootLocation(this.watcher);
}
} finally {
// Clear out our doctored connection or could mess up subsequent tests.
HConnectionManager.deleteConnection(UTIL.getConfiguration());
}
}
private void testVerifyMetaRegionLocationWithException(Exception ex)
throws IOException, InterruptedException, KeeperException, ServiceException {
// Mock an ClientProtocol.
@ -274,14 +192,14 @@ public class TestCatalogTracker {
// Now start up the catalogtracker with our doctored Connection.
final CatalogTracker ct = constructAndStartCatalogTracker(connection);
try {
RootRegionTracker.setRootLocation(this.watcher, SN);
MetaRegionTracker.setMetaLocation(this.watcher, SN);
long timeout = UTIL.getConfiguration().
getLong("hbase.catalog.verification.timeout", 1000);
Assert.assertFalse(ct.verifyMetaRegionLocation(timeout));
} finally {
// Clean out root location or later tests will be confused... they
// Clean out meta location or later tests will be confused... they
// presume start fresh in zk.
RootRegionTracker.deleteRootLocation(this.watcher);
MetaRegionTracker.deleteMetaLocation(this.watcher);
}
} finally {
// Clear out our doctored connection or could mess up subsequent tests.
@ -323,14 +241,14 @@ public class TestCatalogTracker {
}
/**
* Test get of root region fails properly if nothing to connect to.
* Test get of meta region fails properly if nothing to connect to.
* @throws IOException
* @throws InterruptedException
* @throws KeeperException
* @throws ServiceException
*/
@Test
public void testVerifyRootRegionLocationFails()
public void testVerifyMetaRegionLocationFails()
throws IOException, InterruptedException, KeeperException, ServiceException {
HConnection connection = Mockito.mock(HConnection.class);
ServiceException connectException =
@ -343,127 +261,51 @@ public class TestCatalogTracker {
thenReturn(implementation);
final CatalogTracker ct = constructAndStartCatalogTracker(connection);
try {
RootRegionTracker.setRootLocation(this.watcher,
MetaRegionTracker.setMetaLocation(this.watcher,
new ServerName("example.com", 1234, System.currentTimeMillis()));
Assert.assertFalse(ct.verifyRootRegionLocation(100));
Assert.assertFalse(ct.verifyMetaRegionLocation(100));
} finally {
// Clean out root location or later tests will be confused... they presume
// Clean out meta location or later tests will be confused... they presume
// start fresh in zk.
RootRegionTracker.deleteRootLocation(this.watcher);
MetaRegionTracker.deleteMetaLocation(this.watcher);
}
}
@Test (expected = NotAllMetaRegionsOnlineException.class)
public void testTimeoutWaitForRoot()
public void testTimeoutWaitForMeta()
throws IOException, InterruptedException {
HConnection connection = Mockito.mock(HConnection.class);
final CatalogTracker ct = constructAndStartCatalogTracker(connection);
ct.waitForRoot(100);
}
@Test (expected = RetriesExhaustedException.class)
public void testTimeoutWaitForMeta()
throws IOException, InterruptedException {
HConnection connection =
HConnectionTestingUtility.getMockedConnection(UTIL.getConfiguration());
try {
final CatalogTracker ct = constructAndStartCatalogTracker(connection);
ct.waitForMeta(100);
} finally {
HConnectionManager.deleteConnection(UTIL.getConfiguration());
}
}
/**
* Test waiting on root w/ no timeout specified.
* Test waiting on meat w/ no timeout specified.
* @throws IOException
* @throws InterruptedException
* @throws KeeperException
*/
@Test public void testNoTimeoutWaitForRoot()
@Test public void testNoTimeoutWaitForMeta()
throws IOException, InterruptedException, KeeperException {
HConnection connection = Mockito.mock(HConnection.class);
final CatalogTracker ct = constructAndStartCatalogTracker(connection);
ServerName hsa = ct.getRootLocation();
Assert.assertNull(hsa);
// Now test waiting on root location getting set.
Thread t = new WaitOnMetaThread(ct);
startWaitAliveThenWaitItLives(t, 1);
// Set a root location.
hsa = setRootLocation();
// Join the thread... should exit shortly.
t.join();
// Now root is available.
Assert.assertTrue(ct.getRootLocation().equals(hsa));
}
private ServerName setRootLocation() throws KeeperException {
RootRegionTracker.setRootLocation(this.watcher, SN);
return SN;
}
/**
* Test waiting on meta w/ no timeout specified.
* @throws Exception
*/
@Ignore // Can't make it work reliably on all platforms; mockito gets confused
// Throwing: org.mockito.exceptions.misusing.WrongTypeOfReturnValue:
// Result cannot be returned by locateRegion()
// If you plug locateRegion, it then throws for incCounter, and if you plug
// that ... and so one.
@Test public void testNoTimeoutWaitForMeta()
throws Exception {
// Mock an HConnection and a AdminProtocol implementation. Have the
// HConnection return the HRI. Have the HRI return a few mocked up responses
// to make our test work.
// Mock an AdminProtocol.
final AdminProtocol implementation = Mockito.mock(AdminProtocol.class);
HConnection connection = mockConnection(implementation, null);
try {
// Now the ct is up... set into the mocks some answers that make it look
// like things have been getting assigned. Make it so we'll return a
// location (no matter what the Get is). Same for getHRegionInfo -- always
// just return the meta region.
final Result result = getMetaTableRowResult();
// TODO: Refactor. This method has been moved out of HConnection.
// It works for now but has been deprecated.
Mockito.when(connection.getRegionServerWithRetries((ServerCallable<Result>)Mockito.any())).
thenReturn(result);
GetRegionInfoResponse.Builder builder = GetRegionInfoResponse.newBuilder();
builder.setRegionInfo(HRegionInfo.convert(HRegionInfo.FIRST_META_REGIONINFO));
Mockito.when(implementation.getRegionInfo((RpcController)Mockito.any(),
(GetRegionInfoRequest)Mockito.any())).thenReturn(builder.build());
final CatalogTracker ct = constructAndStartCatalogTracker(connection);
ServerName hsa = ct.getMetaLocation();
Assert.assertNull(hsa);
// Now test waiting on meta location getting set.
Thread t = new WaitOnMetaThread(ct) {
@Override
void doWaiting() throws InterruptedException {
this.ct.waitForMeta();
}
};
startWaitAliveThenWaitItLives(t, 1000);
// This should trigger wake up of meta wait (Its the removal of the meta
// region unassigned node that triggers catalogtrackers that a meta has
// been assigned).
String node = ct.getMetaNodeTracker().getNode();
ZKUtil.createAndFailSilent(this.watcher, node);
MetaEditor.updateMetaLocation(ct, HRegionInfo.FIRST_META_REGIONINFO, SN, 0);
ZKUtil.deleteNode(this.watcher, node);
// Go get the new meta location. waitForMeta gets and verifies meta.
Assert.assertTrue(ct.waitForMeta(10000).equals(SN));
Thread t = new WaitOnMetaThread(ct);
startWaitAliveThenWaitItLives(t, 1);
// Set a meta location.
hsa = setMetaLocation();
// Join the thread... should exit shortly.
t.join();
// Now meta is available.
Assert.assertTrue(ct.waitForMeta(10000).equals(SN));
} finally {
HConnectionManager.deleteConnection(UTIL.getConfiguration());
Assert.assertTrue(ct.getMetaLocation().equals(hsa));
}
private ServerName setMetaLocation() throws KeeperException {
MetaRegionTracker.setMetaLocation(this.watcher, SN);
return SN;
}
/**
@ -537,7 +379,6 @@ public class TestCatalogTracker {
/**
* Wait on META.
* Default is wait on -ROOT-.
*/
class WaitOnMetaThread extends Thread {
final CatalogTracker ct;
@ -559,7 +400,7 @@ public class TestCatalogTracker {
void doWaiting() throws InterruptedException {
try {
while (this.ct.waitForRoot(100) == null);
while (this.ct.waitForMeta(100) == null);
} catch (NotAllMetaRegionsOnlineException e) {
// Ignore.
}

View File

@ -56,6 +56,7 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* TODO reenable the tests once a migration path is figured without ROOT
* Test migration that changes HRI serialization into PB. Tests by bringing up a cluster from actual
* data from a 0.92 cluster, as well as manually downgrading and then upgrading the META info.
* @deprecated Remove after 0.96
@ -168,7 +169,7 @@ public class TestMetaMigrationConvertingToPB {
TEST_UTIL.shutdownMiniCluster();
}
@Test
//@Test
public void testMetaUpdatedFlagInROOT() throws Exception {
HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
boolean metaUpdated = MetaMigrationConvertingToPB.
@ -177,7 +178,7 @@ public class TestMetaMigrationConvertingToPB {
verifyMetaRowsAreUpdated(master.getCatalogTracker());
}
@Test
//@Test
public void testMetaMigration() throws Exception {
LOG.info("Starting testMetaMigration");
final byte [] FAMILY = Bytes.toBytes("family");
@ -224,7 +225,7 @@ public class TestMetaMigrationConvertingToPB {
* rows and migrate any pending rows at startup.
* @throws Exception
*/
@Test
//@Test
public void testMasterCrashDuringMetaMigration() throws Exception {
final byte[] FAMILY = Bytes.toBytes("family");
HTableDescriptor htd = new HTableDescriptor("testMasterCrashDuringMetaMigration");
@ -293,7 +294,7 @@ public class TestMetaMigrationConvertingToPB {
p.add(HConstants.CATALOG_FAMILY, HConstants.META_VERSION_QUALIFIER,
Bytes.toBytes(META_VERSION_092));
MetaEditor.putToRootTable(ct, p);
// TODO wire this MetaEditor.putToRootTable(ct, p);
LOG.info("Downgraded -ROOT- meta version=" + META_VERSION_092);
}

View File

@ -42,7 +42,7 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test {@link MetaReader}, {@link MetaEditor}, and {@link RootLocationEditor}.
* Test {@link MetaReader}, {@link MetaEditor}.
*/
@Category(MediumTests.class)
public class TestMetaReaderEditor {
@ -218,8 +218,6 @@ public class TestMetaReaderEditor {
assertTrue(regions.size() >= 1);
assertTrue(MetaReader.getTableRegionsAndLocations(CT,
Bytes.toString(HConstants.META_TABLE_NAME)).size() >= 1);
assertTrue(MetaReader.getTableRegionsAndLocations(CT,
Bytes.toString(HConstants.ROOT_TABLE_NAME)).size() == 1);
}
@Test public void testTableExists() throws IOException {
@ -234,8 +232,6 @@ public class TestMetaReaderEditor {
assertFalse(MetaReader.tableExists(CT, name));
assertTrue(MetaReader.tableExists(CT,
Bytes.toString(HConstants.META_TABLE_NAME)));
assertTrue(MetaReader.tableExists(CT,
Bytes.toString(HConstants.ROOT_TABLE_NAME)));
}
@Test public void testGetRegion() throws IOException, InterruptedException {
@ -245,11 +241,6 @@ public class TestMetaReaderEditor {
Pair<HRegionInfo, ServerName> pair =
MetaReader.getRegion(CT, Bytes.toBytes("nonexistent-region"));
assertNull(pair);
// Test it works getting a region from meta/root.
pair =
MetaReader.getRegion(CT, HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
assertEquals(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName(),
pair.getFirst().getEncodedName());
LOG.info("Finished " + name);
}

View File

@ -1621,18 +1621,4 @@ public class TestAdmin {
ct.stop();
}
}
@Test
public void testRootTableSplit() throws Exception {
ServerName serverName = TEST_UTIL.getMiniHBaseCluster().getServerHoldingRoot();
Scan s = new Scan();
HTable rootTable = new HTable(TEST_UTIL.getConfiguration(), HConstants.ROOT_TABLE_NAME);
ResultScanner scanner = rootTable.getScanner(s);
Result metaEntry = scanner.next();
this.admin.split(HConstants.ROOT_TABLE_NAME, metaEntry.getRow());
Thread.sleep(1000);
List<HRegionInfo> onlineRegions = this.admin.getOnlineRegions(serverName);
assertTrue(onlineRegions != null && onlineRegions.contains(HRegionInfo.ROOT_REGIONINFO));
}
}

View File

@ -402,7 +402,7 @@ class MockRegionServer implements AdminProtocol, ClientProtocol, RegionServerSer
public GetRegionInfoResponse getRegionInfo(RpcController controller,
GetRegionInfoRequest request) throws ServiceException {
GetRegionInfoResponse.Builder builder = GetRegionInfoResponse.newBuilder();
builder.setRegionInfo(HRegionInfo.convert(HRegionInfo.ROOT_REGIONINFO));
builder.setRegionInfo(HRegionInfo.convert(HRegionInfo.FIRST_META_REGIONINFO));
return builder.build();
}

View File

@ -260,7 +260,7 @@ public class TestDistributedLogSplitting {
}
start = EnvironmentEdgeManager.currentTimeMillis();
while (getAllOnlineRegions(cluster).size() < (NUM_REGIONS_TO_CREATE + 2)) {
while (getAllOnlineRegions(cluster).size() < (NUM_REGIONS_TO_CREATE + 1)) {
if (EnvironmentEdgeManager.currentTimeMillis() - start > 60000) {
assertTrue("Timedout", false);
}
@ -357,18 +357,18 @@ public class TestDistributedLogSplitting {
blockUntilNoRIT(zkw, master);
NavigableSet<String> regions = getAllOnlineRegions(cluster);
LOG.debug("Verifying only catalog regions are assigned\n");
if (regions.size() != 2) {
if (regions.size() != 1) {
for (String oregion : regions)
LOG.debug("Region still online: " + oregion);
}
assertEquals(2, regions.size());
assertEquals(1, regions.size());
LOG.debug("Enabling table\n");
TEST_UTIL.getHBaseAdmin().enableTable(table);
LOG.debug("Waiting for no more RIT\n");
blockUntilNoRIT(zkw, master);
LOG.debug("Verifying there are " + numRegions + " assigned on cluster\n");
regions = getAllOnlineRegions(cluster);
assertEquals(numRegions + 2, regions.size());
assertEquals(numRegions + 1, regions.size());
return ht;
}

View File

@ -215,7 +215,7 @@ public class TestMasterFailover {
log("Regions in META have been created");
// at this point we only expect 2 regions to be assigned out (catalogs)
assertEquals(2, cluster.countServedRegions());
assertEquals(1, cluster.countServedRegions());
// Let's just assign everything to first RS
HRegionServer hrs = cluster.getRegionServer(0);
@ -522,7 +522,7 @@ public class TestMasterFailover {
log("Regions in META have been created");
// at this point we only expect 2 regions to be assigned out (catalogs)
assertEquals(2, cluster.countServedRegions());
assertEquals(1, cluster.countServedRegions());
// The first RS will stay online
List<RegionServerThread> regionservers =

View File

@ -88,7 +88,7 @@ public class TestMasterMetrics {
public void testDefaultMasterMetrics() throws Exception {
MetricsMasterSource masterSource = master.getMetrics().getMetricsSource();
metricsHelper.assertGauge( "numRegionServers", 1, masterSource);
metricsHelper.assertGauge( "averageLoad", 2, masterSource);
metricsHelper.assertGauge( "averageLoad", 1, masterSource);
metricsHelper.assertGauge( "numDeadRegionServers", 0, masterSource);
metricsHelper.assertGauge("masterStartTime", master.getMasterStartTime(), masterSource);

View File

@ -44,7 +44,7 @@ import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.RootRegionTracker;
import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.hbase.MediumTests;
@ -122,7 +122,7 @@ public class TestMasterNoCluster {
throws IOException, KeeperException, InterruptedException {
HMaster master = new HMaster(TESTUTIL.getConfiguration());
master.start();
// Immediately have it stop. We used hang in assigning root.
// Immediately have it stop. We used hang in assigning meta.
master.stopMaster();
master.join();
}
@ -134,7 +134,7 @@ public class TestMasterNoCluster {
* @throws KeeperException
* @throws InterruptedException
*/
@Test
@Test (timeout=30000)
public void testFailover()
throws IOException, KeeperException, InterruptedException, ServiceException {
final long now = System.currentTimeMillis();
@ -149,14 +149,9 @@ public class TestMasterNoCluster {
final MockRegionServer rs0 = new MockRegionServer(conf, sn0);
final MockRegionServer rs1 = new MockRegionServer(conf, sn1);
final MockRegionServer rs2 = new MockRegionServer(conf, sn2);
// Put some data into the servers. Make it look like sn0 has the root
// w/ an entry that points to sn1 as the host of .META. Put data into sn2
// so it looks like it has a few regions for a table named 't'.
RootRegionTracker.setRootLocation(rs0.getZooKeeper(), rs0.getServerName());
byte [] rootregion = Bytes.toBytes("-ROOT-,,0");
rs0.setGetResult(rootregion, HRegionInfo.FIRST_META_REGIONINFO.getRegionName(),
MetaMockingUtil.getMetaTableRowResult(HRegionInfo.FIRST_META_REGIONINFO,
rs1.getServerName()));
// Put some data into the servers. Make it look like sn0 has the metaH
// Put data into sn2 so it looks like it has a few regions for a table named 't'.
MetaRegionTracker.setMetaLocation(rs0.getZooKeeper(), rs0.getServerName());
final byte [] tableName = Bytes.toBytes("t");
Result [] results = new Result [] {
MetaMockingUtil.getMetaTableRowResult(
@ -244,7 +239,7 @@ public class TestMasterNoCluster {
* @throws DeserializationException
* @throws ServiceException
*/
@Test
@Test (timeout=30000)
public void testCatalogDeploys()
throws IOException, KeeperException, InterruptedException, DeserializationException, ServiceException {
final Configuration conf = TESTUTIL.getConfiguration();
@ -293,7 +288,7 @@ public class TestMasterNoCluster {
// of a connection will fail.
HConnection connection =
HConnectionTestingUtility.getMockedConnectionAndDecorate(TESTUTIL.getConfiguration(),
rs0, rs0, rs0.getServerName(), HRegionInfo.ROOT_REGIONINFO);
rs0, rs0, rs0.getServerName(), HRegionInfo.FIRST_META_REGIONINFO);
return new CatalogTracker(zk, conf, connection, abortable);
}
};
@ -321,38 +316,21 @@ public class TestMasterNoCluster {
// Assert hostname is as expected.
assertEquals(rs0.getServerName().getHostname(), rshostname);
// Now master knows there is at least one regionserver checked in and so
// it'll wait a while to see if more and when none, will assign root and
// meta to this single server. Will do an rpc open but we've
// it'll wait a while to see if more and when none, will assign meta
// to this single server. Will do an rpc open but we've
// mocked it above in our master override to return 'success'. As part of
// region open, master will have set an unassigned znode for the region up
// into zk for the regionserver to transition. Lets do that now to
// complete fake of a successful open.
Mocking.fakeRegionServerRegionOpenInZK(master, rs0.getZooKeeper(),
rs0.getServerName(), HRegionInfo.ROOT_REGIONINFO);
rs0.getServerName(), HRegionInfo.FIRST_META_REGIONINFO);
LOG.info("fakeRegionServerRegionOpenInZK has started");
// Need to set root location as r1. Usually the regionserver does this
// when its figured it just opened the root region by setting the root
// Need to set meta location as r0. Usually the regionserver does this
// when its figured it just opened the meta region by setting the meta
// location up into zk. Since we're mocking regionserver, need to do this
// ourselves.
RootRegionTracker.setRootLocation(rs0.getZooKeeper(), rs0.getServerName());
// Do same transitions for .META. (presuming master has by now assigned
// .META. to rs1).
Mocking.fakeRegionServerRegionOpenInZK(master, rs0.getZooKeeper(),
rs0.getServerName(), HRegionInfo.FIRST_META_REGIONINFO);
// Now trigger our mock regionserver to start returning a row when we
// go to get .META. entry in -ROOT-. We do it by setting into
// our MockRegionServer some data to be returned when there is a get on
// -ROOT- table (up to this its been returning null making master think
// nothing assigned, not even .META.). The region for -ROOT- table we
// hardcode below. Its always the same, at least in tests. We need to do
// this because CatalogTracker runs inside in Master initialization to
// confirm .META. has a server.
byte [] rootregion = Bytes.toBytes("-ROOT-,,0");
rs0.setGetResult(rootregion, HRegionInfo.FIRST_META_REGIONINFO.getRegionName(),
MetaMockingUtil.getMetaTableRowResult(HRegionInfo.FIRST_META_REGIONINFO,
rs0.getServerName()));
MetaRegionTracker.setMetaLocation(rs0.getZooKeeper(), rs0.getServerName());
// Master should now come up.
while (!master.isInitialized()) {Threads.sleep(10);}
assertTrue(master.isInitialized());

View File

@ -74,7 +74,7 @@ public class TestMasterRestartAfterDisablingTable {
HTable ht = TEST_UTIL.createTable(table, family);
int numRegions = TEST_UTIL.createMultiRegions(conf, ht, family,
NUM_REGIONS_TO_CREATE);
numRegions += 2; // catalogs
numRegions += 1; // catalogs
log("Waiting for no more RIT\n");
blockUntilNoRIT(zkw, master);
log("Disabling table\n");
@ -83,7 +83,7 @@ public class TestMasterRestartAfterDisablingTable {
NavigableSet<String> regions = getAllOnlineRegions(cluster);
assertEquals(
"The number of regions for the table tableRestart should be 0 and only"
+ "the catalog tables should be present.", 2, regions.size());
+ "the catalog tables should be present.", 1, regions.size());
List<MasterThread> masterThreads = cluster.getMasterThreads();
MasterThread activeMaster = null;
@ -111,7 +111,7 @@ public class TestMasterRestartAfterDisablingTable {
regions = getAllOnlineRegions(cluster);
assertEquals(
"The assigned regions were not onlined after master switch except for the catalog tables.",
6, regions.size());
5, regions.size());
assertTrue("The table should be in enabled state", cluster.getMaster()
.getAssignmentManager().getZKTable().isEnabledTable("tableRestart"));
ht.close();

View File

@ -109,19 +109,10 @@ public class TestMasterStatusServlet {
}
@Test
public void testStatusTemplateRootAvailable() throws IOException {
new MasterStatusTmpl()
.setRootLocation(new ServerName("rootserver:123,12345"))
.render(new StringWriter(),
master, admin);
}
@Test
public void testStatusTemplateRootAndMetaAvailable() throws IOException {
public void testStatusTemplateMetaAvailable() throws IOException {
setupMockTables();
new MasterStatusTmpl()
.setRootLocation(new ServerName("rootserver:123,12345"))
.setMetaLocation(new ServerName("metaserver:123,12345"))
.render(new StringWriter(),
master, admin);
@ -141,7 +132,6 @@ public class TestMasterStatusServlet {
);
new MasterStatusTmpl()
.setRootLocation(new ServerName("rootserver:123,12345"))
.setMetaLocation(new ServerName("metaserver:123,12345"))
.setServers(servers)
.setDeadServers(deadServers)

View File

@ -79,7 +79,7 @@ public class TestRollingRestart {
HTable ht = TEST_UTIL.createTable(table, family);
int numRegions = TEST_UTIL.createMultiRegions(conf, ht, family,
NUM_REGIONS_TO_CREATE);
numRegions += 2; // catalogs
numRegions += 1; // catalogs
log("Waiting for no more RIT\n");
blockUntilNoRIT(zkw, master);
log("Disabling table\n");
@ -88,10 +88,10 @@ public class TestRollingRestart {
blockUntilNoRIT(zkw, master);
NavigableSet<String> regions = getAllOnlineRegions(cluster);
log("Verifying only catalog regions are assigned\n");
if (regions.size() != 2) {
if (regions.size() != 1) {
for (String oregion : regions) log("Region still online: " + oregion);
}
assertEquals(2, regions.size());
assertEquals(1, regions.size());
log("Enabling table\n");
TEST_UTIL.getHBaseAdmin().enableTable(table);
log("Waiting for no more RIT\n");
@ -195,15 +195,11 @@ public class TestRollingRestart {
i++;
}
}
log("Stopping server hosting ROOT");
rootServer.getRegionServer().stop("Stopping ROOT server");
log("Stopping server hosting META #1");
metaServer.getRegionServer().stop("Stopping META server");
cluster.hbaseCluster.waitOnRegionServer(rootServer);
log("Root server down");
cluster.hbaseCluster.waitOnRegionServer(metaServer);
log("Meta server down #1");
expectedNumRS -= 2;
expectedNumRS--;
log("Waiting for meta server #1 RS shutdown to be handled by master");
waitForRSShutdownToStartAndFinish(activeMaster,
metaServer.getRegionServer().getServerName());
@ -290,7 +286,7 @@ public class TestRollingRestart {
log("RS: " + rst.getRegionServer().getServerName());
}
}
assertEquals(1, cluster.getRegionServerThreads().size());
assertEquals(2, cluster.getRegionServerThreads().size());
// TODO: Bring random 3 of 4 RS down at the same time

View File

@ -143,7 +143,6 @@ public class TestHRegionInfo {
@Test
public void testMetaTables() {
assertTrue(HRegionInfo.ROOT_REGIONINFO.isMetaTable());
assertTrue(HRegionInfo.FIRST_META_REGIONINFO.isMetaTable());
}

View File

@ -167,18 +167,15 @@ public class TestRSKilledWhenMasterInitializing {
// Second kill meta server
int metaServerNum = cluster.getServerWithMeta();
int rootServerNum = cluster.getServerWith(HRegionInfo.ROOT_REGIONINFO
.getRegionName());
HRegionServer metaRS = cluster.getRegionServer(metaServerNum);
LOG.debug("Killing metaRS and carryingRoot = "
+ (metaServerNum == rootServerNum));
LOG.debug("Killing metaRS");
metaRS.kill();
metaRS.join();
/*
* Sleep double time of TestingMaster.sleep.duration, so we can ensure that
* master has already assigned ROOTandMETA or is blocking on assigning
* ROOTandMETA
* master has already assigned META or is blocking on assigning
* META
*/
Thread.sleep(10000 * 2);
@ -187,34 +184,7 @@ public class TestRSKilledWhenMasterInitializing {
// Third check whether data is correct in meta region
assertTrue(hbaseAdmin.isTableAvailable(TABLENAME));
/*
* NO.2 -ROOT- region correctness . If the .META. server killed in the NO.1
* is also carrying -ROOT- region, it is not needed
*/
if (rootServerNum != metaServerNum) {
// First abort master
abortMaster(cluster);
master = startMasterAndWaitUntilLogSplit(cluster);
// Second kill meta server
HRegionServer rootRS = cluster.getRegionServer(rootServerNum);
LOG.debug("Killing rootRS");
rootRS.kill();
rootRS.join();
/*
* Sleep double time of TestingMaster.sleep.duration, so we can ensure
* that master has already assigned ROOTandMETA or is blocking on
* assigning ROOTandMETA
*/
Thread.sleep(10000 * 2);
waitUntilMasterIsInitialized(master);
// Third check whether data is correct in meta region
assertTrue(hbaseAdmin.isTableAvailable(TABLENAME));
}
/* NO.3 data region correctness */
/* NO.2 data region correctness */
ServerManager serverManager = cluster.getMaster().getServerManager();
while (serverManager.areDeadServersInProgress()) {
Thread.sleep(100);

View File

@ -109,17 +109,13 @@ public class TestStoreFileBlockCacheSummary {
bc.getBlockCacheColumnFamilySummaries(TEST_UTIL.getConfiguration());
LOG.info("blockCacheSummary: " + bcs);
assertEquals("blockCache summary has entries", 3, bcs.size());
assertEquals("blockCache summary has entries", 2, bcs.size());
BlockCacheColumnFamilySummary e = bcs.get(0);
assertEquals("table", "-ROOT-", e.getTable());
assertEquals("cf", "info", e.getColumnFamily());
e = bcs.get(1);
assertEquals("table", TEST_TABLE, e.getTable());
assertEquals("cf", TEST_CF, e.getColumnFamily());
e = bcs.get(2);
e = bcs.get(1);
assertEquals("table", TEST_TABLE2, e.getTable());
assertEquals("cf", TEST_CF, e.getColumnFamily());

View File

@ -42,7 +42,6 @@ import org.junit.experimental.categories.Category;
@Category(MediumTests.class)
public class TestStatusResource {
private static final byte[] ROOT_REGION_NAME = Bytes.toBytes("-ROOT-,,0");
private static final byte[] META_REGION_NAME = Bytes.toBytes(".META.,,1");
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@ -59,20 +58,17 @@ public class TestStatusResource {
assertNotNull(model.getLiveNodes());
assertNotNull(model.getDeadNodes());
assertFalse(model.getLiveNodes().isEmpty());
boolean foundRoot = false, foundMeta = false;
boolean foundMeta = false;
for (StorageClusterStatusModel.Node node: model.getLiveNodes()) {
assertNotNull(node.getName());
assertTrue(node.getStartCode() > 0L);
assertTrue(node.getRequests() >= 0);
for (StorageClusterStatusModel.Node.Region region: node.getRegions()) {
if (Bytes.equals(region.getName(), ROOT_REGION_NAME)) {
foundRoot = true;
} else if (Bytes.equals(region.getName(), META_REGION_NAME)) {
if (Bytes.equals(region.getName(), META_REGION_NAME)) {
foundMeta = true;
}
}
}
assertTrue(foundRoot);
assertTrue(foundMeta);
}

View File

@ -34,8 +34,7 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.*;
import org.apache.commons.io.IOUtils;
import org.apache.commons.logging.Log;
@ -105,6 +104,7 @@ public class TestHBaseFsck {
private final static byte[] FAM = Bytes.toBytes(FAM_STR);
private final static int REGION_ONLINE_TIMEOUT = 800;
private static RegionStates regionStates;
private static ExecutorService executorService;
// for the instance, reset every test run
private HTable tbl;
@ -118,8 +118,13 @@ public class TestHBaseFsck {
@BeforeClass
public static void setUpBeforeClass() throws Exception {
TEST_UTIL.getConfiguration().setBoolean(HConstants.DISTRIBUTED_LOG_SPLITTING_KEY, false);
TEST_UTIL.getConfiguration().setInt("hbase.regionserver.handler.count", 2);
TEST_UTIL.getConfiguration().setInt("hbase.regionserver.metahandler.count", 2);
TEST_UTIL.startMiniCluster(3);
executorService = new ThreadPoolExecutor(1, Integer.MAX_VALUE, 60, TimeUnit.SECONDS,
new SynchronousQueue<Runnable>(), Threads.newDaemonThreadFactory("testhbck"));
AssignmentManager assignmentManager =
TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager();
regionStates = assignmentManager.getRegionStates();
@ -141,7 +146,7 @@ public class TestHBaseFsck {
// Now let's mess it up and change the assignment in .META. to
// point to a different region server
HTable meta = new HTable(conf, HTableDescriptor.META_TABLEDESC.getName());
HTable meta = new HTable(conf, HTableDescriptor.META_TABLEDESC.getName(), executorService);
ResultScanner scanner = meta.getScanner(new Scan());
HRegionInfo hri = null;
@ -181,7 +186,7 @@ public class TestHBaseFsck {
assertNoErrors(doFsck(conf, false));
// comment needed - what is the purpose of this line
HTable t = new HTable(conf, Bytes.toBytes(table));
HTable t = new HTable(conf, Bytes.toBytes(table), executorService);
ResultScanner s = t.getScanner(new Scan());
s.close();
t.close();
@ -196,7 +201,7 @@ public class TestHBaseFsck {
private HRegionInfo createRegion(Configuration conf, final HTableDescriptor
htd, byte[] startKey, byte[] endKey)
throws IOException {
HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
HTable meta = new HTable(conf, HConstants.META_TABLE_NAME, executorService);
HRegionInfo hri = new HRegionInfo(htd.getName(), startKey, endKey);
MetaEditor.addRegionToMeta(meta, hri);
meta.close();
@ -288,7 +293,7 @@ public class TestHBaseFsck {
}
if (metaRow) {
HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
HTable meta = new HTable(conf, HConstants.META_TABLE_NAME, executorService);
Delete delete = new Delete(deleteRow);
meta.delete(delete);
}
@ -313,7 +318,7 @@ public class TestHBaseFsck {
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);
tbl = new HTable(TEST_UTIL.getConfiguration(), tablename);
tbl = new HTable(TEST_UTIL.getConfiguration(), Bytes.toBytes(tablename), executorService);
List<Put> puts = new ArrayList<Put>();
for (byte[] row : ROWKEYS) {
@ -350,9 +355,16 @@ public class TestHBaseFsck {
admin.getConnection().clearRegionCache();
byte[] tbytes = Bytes.toBytes(tablename);
admin.disableTableAsync(tbytes);
long totalWait = 0;
long maxWait = 30*1000;
long sleepTime = 250;
while (!admin.isTableDisabled(tbytes)) {
try {
Thread.sleep(250);
Thread.sleep(sleepTime);
totalWait += sleepTime;
if (totalWait >= maxWait) {
fail("Waited too long for table to be disabled + " + tablename);
}
} catch (InterruptedException e) {
e.printStackTrace();
fail("Interrupted when trying to disable table " + tablename);
@ -723,7 +735,7 @@ public class TestHBaseFsck {
assertNotNull(regionName);
assertNotNull(serverName);
HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
HTable meta = new HTable(conf, HConstants.META_TABLE_NAME, executorService);
Put put = new Put(regionName);
put.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
Bytes.toBytes(serverName.getHostAndPort()));
@ -1211,7 +1223,7 @@ public class TestHBaseFsck {
Bytes.toBytes("C"), true, true, false);
// Create a new meta entry to fake it as a split parent.
meta = new HTable(conf, HTableDescriptor.META_TABLEDESC.getName());
meta = new HTable(conf, HTableDescriptor.META_TABLEDESC.getName(), executorService);
HRegionInfo hri = location.getRegionInfo();
HRegionInfo a = new HRegionInfo(tbl.getTableName(),
@ -1384,7 +1396,7 @@ public class TestHBaseFsck {
* This creates and fixes a bad table with a missing region which is the 1st region -- hole in
* meta and data missing in the fs.
*/
@Test
@Test(timeout=120000)
public void testMissingFirstRegion() throws Exception {
String table = "testMissingFirstRegion";
try {
@ -1412,7 +1424,7 @@ public class TestHBaseFsck {
* This creates and fixes a bad table with missing last region -- hole in meta and data missing in
* the fs.
*/
@Test
@Test(timeout=120000)
public void testMissingLastRegion() throws Exception {
String table = "testMissingLastRegion";
try {

View File

@ -137,8 +137,8 @@ public class TestMergeTool extends HBaseTestCase {
super.setUp();
try {
// Create root and meta regions
createRootAndMetaRegions();
// Create meta region
createMetaRegion();
FSTableDescriptors.createTableDescriptor(this.fs, this.testDir, this.desc);
/*
* Create the regions we will merge

View File

@ -75,7 +75,6 @@ module Hbase
define_test "list should not return meta tables" do
assert(!admin.list.member?('.META.'))
assert(!admin.list.member?('-ROOT-'))
end
#-------------------------------------------------------------------------------

View File

@ -58,10 +58,6 @@ module Hbase
assert(table('.META.').is_meta_table?)
end
define_test "is_meta_table? method should return true for the root table" do
assert(table('-ROOT-').is_meta_table?)
end
define_test "is_meta_table? method should return false for a normal table" do
assert(!@test_table.is_meta_table?)
end