HBASE-4495 CatalogTracker has an identity crisis; needs to be cut-back in scope (Mikhail Antonov)

This commit is contained in:
stack 2014-07-02 19:01:04 -07:00
parent 1d8958685a
commit ea085c6373
106 changed files with 796 additions and 4577 deletions

View File

@ -101,15 +101,13 @@ def getServerNameForRegion(admin, r)
if r.isMetaRegion()
# Hack
zkw = org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.new(admin.getConfiguration(), "region_mover", nil)
mtl = org.apache.hadoop.hbase.zookeeper.MetaTableLocator.new()
begin
tracker = org.apache.hadoop.hbase.zookeeper.MetaRegionTracker.new(zkw, RubyAbortable.new())
tracker.start()
while not tracker.isLocationAvailable()
while not mtl.isLocationAvailable(zkw)
sleep 0.1
end
# Make a fake servername by appending ','
metaServer = tracker.getMetaRegionLocation().toString() + ","
tracker.stop()
metaServer = mtl.getMetaRegionLocation(zkw).toString() + ","
return metaServer
ensure
zkw.close()

View File

@ -64,7 +64,7 @@ parallelMaven=1
# the others
#The ',' at the end is mandatory
flakyTests=
#org.apache.hadoop.hbase.mapreduce.TestTableInputFormatScan,org.apache.hadoop.hbase.catalog.TestMetaReaderEditorNoCluster,org.apache.hadoop.hbase.catalog.TestMetaReaderEditor,org.apache.hadoop.hbase.mapreduce.TestHFileOutputFormat,org.apache.hadoop.hbase.mapred.TestTableMapReduce,org.apache.hadoop.hbase.coprocessor.TestMasterCoprocessorExceptionWithAbort,org.apache.hadoop.hbase.coprocessor.TestMasterCoprocessorExceptionWithRemove,org.apache.hadoop.hbase.client.TestAdmin,org.apache.hadoop.hbase.master.TestMasterFailover,org.apache.hadoop.hbase.regionserver.wal.TestLogRolling,org.apache.hadoop.hbase.master.TestDistributedLogSplitting,org.apache.hadoop.hbase.master.TestMasterRestartAfterDisablingTable,org.apache.hadoop.hbase.TestGlobalMemStoreSize,
#org.apache.hadoop.hbase.mapreduce.TestTableInputFormatScan,org.apache.hadoop.hbase.catalog.TestMetaTableAccessorNoCluster,org.apache.hadoop.hbase.catalog.TestMetaTableAccessor,org.apache.hadoop.hbase.mapreduce.TestHFileOutputFormat,org.apache.hadoop.hbase.mapred.TestTableMapReduce,org.apache.hadoop.hbase.coprocessor.TestMasterCoprocessorExceptionWithAbort,org.apache.hadoop.hbase.coprocessor.TestMasterCoprocessorExceptionWithRemove,org.apache.hadoop.hbase.client.TestAdmin,org.apache.hadoop.hbase.master.TestMasterFailover,org.apache.hadoop.hbase.regionserver.wal.TestLogRolling,org.apache.hadoop.hbase.master.TestDistributedLogSplitting,org.apache.hadoop.hbase.master.TestMasterRestartAfterDisablingTable,org.apache.hadoop.hbase.TestGlobalMemStoreSize,
######################################### Internal parameters
#directory used for surefire & the source code.

View File

@ -1231,7 +1231,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
* @return A pair of the {@link HRegionInfo} and the {@link ServerName}
* (or null for server address if no address set in hbase:meta).
* @throws IOException
* @deprecated use MetaReader methods for interacting with meta layouts
* @deprecated use MetaTableAccessor methods for interacting with meta layouts
*/
@Deprecated
public static Pair<HRegionInfo, ServerName> getHRegionInfoAndServerName(final Result r) {
@ -1247,7 +1247,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
* table Result.
* @param data a Result object from the catalog table scan
* @return HRegionInfo or null
* @deprecated use MetaReader methods for interacting with meta layouts
* @deprecated use MetaTableAccessor methods for interacting with meta layouts
*/
@Deprecated
public static HRegionInfo getHRegionInfo(Result data) {
@ -1260,7 +1260,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
* @param data a Result object from the catalog table scan
* @return a pair of HRegionInfo or PairOfSameType(null, null) if the region is not a split
* parent
* @deprecated use MetaReader methods for interacting with meta layouts
* @deprecated use MetaTableAccessor methods for interacting with meta layouts
*/
@Deprecated
public static PairOfSameType<HRegionInfo> getDaughterRegions(Result data) throws IOException {
@ -1276,7 +1276,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
* @param data a Result object from the catalog table scan
* @return a pair of HRegionInfo or PairOfSameType(null, null) if the region is not a split
* parent
* @deprecated use MetaReader methods for interacting with meta layouts
* @deprecated use MetaTableAccessor methods for interacting with meta layouts
*/
@Deprecated
public static PairOfSameType<HRegionInfo> getMergeRegions(Result data) throws IOException {
@ -1294,7 +1294,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
* {@link HConstants#SPLITA_QUALIFIER}, {@link HConstants#SPLITB_QUALIFIER} or
* {@link HConstants#REGIONINFO_QUALIFIER}.
* @return An HRegionInfo instance or null.
* @deprecated use MetaReader methods for interacting with meta layouts
* @deprecated use MetaTableAccessor methods for interacting with meta layouts
*/
@Deprecated
public static HRegionInfo getHRegionInfo(final Result r, byte [] qualifier) {
@ -1305,7 +1305,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
}
/**
* @deprecated use MetaReader methods for interacting with meta layouts
* @deprecated use MetaTableAccessor methods for interacting with meta layouts
*/
@Deprecated
public static ServerName getServerName(final Result r) {
@ -1330,7 +1330,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
* E.g. the seqNum when the result of {@link #getServerName(Result)} was written.
* @param r Result to pull the seqNum from
* @return SeqNum, or HConstants.NO_SEQNUM if there's no value written.
* @deprecated use MetaReader methods for interacting with meta layouts
* @deprecated use MetaTableAccessor methods for interacting with meta layouts
*/
@Deprecated
public static long getSeqNumDuringOpen(final Result r) {

View File

@ -20,7 +20,8 @@ package org.apache.hadoop.hbase;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
/**
@ -40,9 +41,24 @@ public interface Server extends Abortable, Stoppable {
ZooKeeperWatcher getZooKeeper();
/**
* @return Master's instance of {@link CatalogTracker}
* Returns reference to wrapped short-circuit (i.e. local, bypassing RPC layer entirely)
* HConnection to this server, which may be used for miscellaneous needs.
*
* Important note: this method returns reference to connection which is managed
* by Server itself, so callers must NOT attempt to close connection obtained.
*
* See {@link org.apache.hadoop.hbase.client.ConnectionUtils#createShortCircuitHConnection}
* for details on short-circuit connections.
*/
CatalogTracker getCatalogTracker();
HConnection getShortCircuitConnection();
/**
* Returns instance of {@link org.apache.hadoop.hbase.zookeeper.MetaTableLocator}
* running inside this server. This MetaServerLocator is started and stopped by server, clients
* shouldn't manage it's lifecycle.
* @return instance of {@link MetaTableLocator} associated with this server.
*/
MetaTableLocator getMetaTableLocator();
/**
* @return The unique server name for this server.

View File

@ -1,457 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.catalog;
import com.google.common.base.Stopwatch;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
import org.apache.hadoop.hbase.ipc.RpcClient.FailedServerException;
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.ipc.RemoteException;
import java.io.EOFException;
import java.io.IOException;
import java.net.ConnectException;
import java.net.NoRouteToHostException;
import java.net.SocketException;
import java.net.SocketTimeoutException;
import java.net.UnknownHostException;
/**
* Tracks the availability of the catalog tables
* <code>hbase: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>hbase: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 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 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
// read an hbase table, the -ROOT- table, to figure out the hbase:meta region
// location which means we depend on an HConnection. HConnection will do
// retrying but also, it has its own mechanism for finding root and meta
// locations (and for 'verifying'; it tries the location and if it fails, does
// new lookup, etc.). So, at least for now, HConnection (or HTable) can't
// have a CT since CT needs a HConnection (Even then, do want HT to have a CT?
// For HT keep up a session with ZK? Rather, shouldn't we do like asynchbase
// where we'd open a connection to zk, read what we need then let the
// connection go?). The 'fix' is make it so both root and meta addresses
// are wholey up in zk -- not in zk (root) -- and in an hbase table (meta).
//
// But even then, this class does 'verification' of the location and it does
// this by making a call over an HConnection (which will do its own root
// and meta lookups). Isn't this verification 'useless' since when we
// return, whatever is dependent on the result of this call then needs to
// use HConnection; what we have verified may change in meantime (HConnection
// uses the CT primitives, the root and meta trackers finding root locations).
//
// When meta is moved to zk, this class may make more sense. In the
// meantime, it does not cohere. It should just watch meta and root and not
// NOT do verification -- let that be out in HConnection since its going to
// be done there ultimately anyways.
//
// This class has spread throughout the codebase. It needs to be reigned in.
// This class should be used server-side only, even if we move meta location
// up into zk. Currently its used over in the client package. Its used in
// MetaReader and MetaEditor classes usually just to get the Configuration
// its using (It does this indirectly by asking its HConnection for its
// Configuration and even then this is just used to get an HConnection out on
// the other end). I made https://issues.apache.org/jira/browse/HBASE-4495 for
// doing CT fixup. St.Ack 09/30/2011.
//
// TODO: Timeouts have never been as advertised in here and its worse now
// with retries; i.e. the HConnection retries and pause goes ahead whatever
// the passed timeout is. Fix.
private static final Log LOG = LogFactory.getLog(CatalogTracker.class);
private final HConnection connection;
private final ZooKeeperWatcher zookeeper;
private final MetaRegionTracker metaRegionTracker;
private boolean instantiatedzkw = false;
private Abortable abortable;
private boolean stopped = false;
static final byte [] META_REGION_NAME =
HRegionInfo.FIRST_META_REGIONINFO.getRegionName();
/**
* Constructs a catalog tracker. Find current state of catalog tables.
* Begin active tracking by executing {@link #start()} post construction. Does
* not timeout.
*
* @param conf
* the {@link Configuration} from which a {@link HConnection} will be
* obtained; if problem, this connections
* {@link HConnection#abort(String, Throwable)} will be called.
* @throws IOException
*/
public CatalogTracker(final Configuration conf) throws IOException {
this(null, conf, HConnectionManager.getConnection(conf), null);
}
/**
* Constructs the catalog tracker. Find current state of catalog tables.
* Begin active tracking by executing {@link #start()} post construction.
* Does not timeout.
* @param zk If zk is null, we'll create an instance (and shut it down
* when {@link #stop()} is called) else we'll use what is passed.
* @param conf
* @param abortable If fatal exception we'll call abort on this. May be null.
* If it is we'll use the Connection associated with the passed
* {@link Configuration} as our Abortable.
* @throws IOException
*/
public CatalogTracker(final ZooKeeperWatcher zk, final Configuration conf,
HConnection connection, Abortable abortable)
throws IOException {
this.connection = connection;
if (abortable == null) {
// A connection is abortable.
this.abortable = this.connection;
} else {
this.abortable = abortable;
}
Abortable throwableAborter = new Abortable() {
@Override
public void abort(String why, Throwable e) {
throw new RuntimeException(why, e);
}
@Override
public boolean isAborted() {
return true;
}
};
if (zk == null) {
// Create our own. Set flag so we tear it down on stop.
this.zookeeper =
new ZooKeeperWatcher(conf, "catalogtracker-on-" + connection.toString(),
abortable);
instantiatedzkw = true;
} else {
this.zookeeper = zk;
}
this.metaRegionTracker = new MetaRegionTracker(zookeeper, throwableAborter);
}
/**
* Starts the catalog tracker.
* Determines current availability of catalog tables and ensures all further
* transitions of either region are tracked.
* @throws IOException
* @throws InterruptedException
*/
public void start() throws IOException, InterruptedException {
LOG.debug("Starting catalog tracker " + this);
try {
this.metaRegionTracker.start();
} catch (RuntimeException e) {
Throwable t = e.getCause();
this.abortable.abort(e.getMessage(), t);
throw new IOException("Attempt to start meta tracker failed.", t);
}
}
/**
* Stop working.
* Interrupts any ongoing waits.
*/
public void stop() {
if (!this.stopped) {
LOG.debug("Stopping catalog tracker " + this);
this.stopped = true;
this.metaRegionTracker.stop();
try {
if (this.connection != null) {
this.connection.close();
}
} catch (IOException e) {
// Although the {@link Closeable} interface throws an {@link
// IOException}, in reality, the implementation would never do that.
LOG.error("Attempt to close catalog tracker's connection failed.", e);
}
if (this.instantiatedzkw) {
this.zookeeper.close();
}
}
}
/**
* Gets the current location for <code>hbase:meta</code> or null if location is
* not currently available.
* @return {@link ServerName} for server hosting <code>hbase:meta</code> or null
* if none available
* @throws InterruptedException
*/
public ServerName getMetaLocation() throws InterruptedException {
return this.metaRegionTracker.getMetaRegionLocation();
}
/**
* Checks whether meta regionserver znode has some non null data.
* @return true if data is not null, false otherwise.
*/
public boolean isMetaLocationAvailable() {
return this.metaRegionTracker.isLocationAvailable();
}
/**
* Gets the current location for <code>hbase: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>hbase:meta</code> or null
* if none available
* @throws InterruptedException if interrupted while waiting
* @throws NotAllMetaRegionsOnlineException if meta not available before
* timeout
*/
public ServerName waitForMeta(final long timeout)
throws InterruptedException, NotAllMetaRegionsOnlineException {
ServerName sn = metaRegionTracker.waitMetaRegionLocation(timeout);
if (sn == null) {
throw new NotAllMetaRegionsOnlineException("Timed out; " + timeout + "ms");
}
return sn;
}
/**
* 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 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 #getMetaServerConnection(long)
*/
public AdminService.BlockingInterface waitForMetaServerConnection(long timeout)
throws InterruptedException, NotAllMetaRegionsOnlineException, IOException {
return getMetaServerConnection(timeout);
}
/**
* 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 meta location
* @see #waitForMeta for additional information
* @return connection to server hosting meta
* @throws InterruptedException
* @throws NotAllMetaRegionsOnlineException if timed out waiting
* @throws IOException
*/
AdminService.BlockingInterface getMetaServerConnection(long timeout)
throws InterruptedException, NotAllMetaRegionsOnlineException, IOException {
return getCachedConnection(waitForMeta(timeout));
}
/**
* Waits indefinitely for availability of <code>hbase:meta</code>. Used during
* cluster startup. Does not verify meta, just that something has been
* set up in zk.
* @see #waitForMeta(long)
* @throws InterruptedException if interrupted while waiting
*/
public void waitForMeta() throws InterruptedException {
Stopwatch stopwatch = new Stopwatch().start();
while (!this.stopped) {
try {
if (waitForMeta(100) != null) break;
long sleepTime = stopwatch.elapsedMillis();
// +1 in case sleepTime=0
if ((sleepTime + 1) % 10000 == 0) {
LOG.warn("Have been waiting for meta to be assigned for " + sleepTime + "ms");
}
} catch (NotAllMetaRegionsOnlineException e) {
if (LOG.isTraceEnabled()) {
LOG.trace("hbase:meta still not available, sleeping and retrying." +
" Reason: " + e.getMessage());
}
}
}
}
/**
* @param sn ServerName to get a connection against.
* @return The AdminProtocol we got when we connected to <code>sn</code>
* May have come from cache, may not be good, may have been setup by this
* invocation, or may be null.
* @throws IOException
*/
@SuppressWarnings("deprecation")
private AdminService.BlockingInterface getCachedConnection(ServerName sn)
throws IOException {
if (sn == null) {
return null;
}
AdminService.BlockingInterface service = null;
try {
service = connection.getAdmin(sn);
} catch (RetriesExhaustedException e) {
if (e.getCause() != null && e.getCause() instanceof ConnectException) {
// Catch this; presume it means the cached connection has gone bad.
} else {
throw e;
}
} catch (SocketTimeoutException e) {
LOG.debug("Timed out connecting to " + sn);
} catch (NoRouteToHostException e) {
LOG.debug("Connecting to " + sn, e);
} catch (SocketException e) {
LOG.debug("Exception connecting to " + sn);
} catch (UnknownHostException e) {
LOG.debug("Unknown host exception connecting to " + sn);
} catch (FailedServerException e) {
if (LOG.isDebugEnabled()) {
LOG.debug("Server " + sn + " is in failed server list.");
}
} catch (IOException ioe) {
Throwable cause = ioe.getCause();
if (ioe instanceof ConnectException) {
// Catch. Connect refused.
} else if (cause != null && cause instanceof EOFException) {
// Catch. Other end disconnected us.
} else if (cause != null && cause.getMessage() != null &&
cause.getMessage().toLowerCase().contains("connection reset")) {
// Catch. Connection reset.
} else {
throw ioe;
}
}
return service;
}
/**
* 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 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
* the Interface.
* @throws IOException
*/
// TODO: We should be able to get the ServerName from the AdminProtocol
// rather than have to pass it in. Its made awkward by the fact that the
// HRI is likely a proxy against remote server so the getServerName needs
// to be fixed to go to a local method or to a cache before we can do this.
private boolean verifyRegionLocation(AdminService.BlockingInterface hostingServer,
final ServerName address, final byte [] regionName)
throws IOException {
if (hostingServer == null) {
LOG.info("Passed hostingServer is null");
return false;
}
Throwable t = null;
try {
// Try and get regioninfo from the hosting server.
return ProtobufUtil.getRegionInfo(hostingServer, regionName) != null;
} catch (ConnectException e) {
t = e;
} catch (RetriesExhaustedException e) {
t = e;
} catch (RemoteException e) {
IOException ioe = e.unwrapRemoteException();
t = ioe;
} catch (IOException e) {
Throwable cause = e.getCause();
if (cause != null && cause instanceof EOFException) {
t = cause;
} else if (cause != null && cause.getMessage() != null
&& cause.getMessage().contains("Connection reset")) {
t = cause;
} else {
t = e;
}
}
LOG.info("Failed verification of " + Bytes.toStringBinary(regionName) +
" at address=" + address + ", exception=" + t);
return false;
}
/**
* Verify <code>hbase:meta</code> is deployed and accessible.
* @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>hbase:meta</code> location is healthy.
* @throws IOException
* @throws InterruptedException
*/
public boolean verifyMetaRegionLocation(final long timeout)
throws InterruptedException, IOException {
AdminService.BlockingInterface service = null;
try {
service = waitForMetaServerConnection(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.
} catch (RegionServerStoppedException e) {
// Pass -- server name sends us to a server that is dying or already dead.
}
return (service == null)? false:
verifyRegionLocation(service,
this.metaRegionTracker.getMetaRegionLocation(), META_REGION_NAME);
}
public HConnection getConnection() {
return this.connection;
}
@Override
public String toString() {
return "CatalogTracker{" + "connection=" + connection + ", zookeeper=" + zookeeper +
", metaRegionTracker=" + metaRegionTracker + ", stopped=" + stopped + '}';
}
}

View File

@ -1,886 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.catalog;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map.Entry;
import java.util.NavigableMap;
import java.util.SortedMap;
import java.util.TreeMap;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.PairOfSameType;
import com.google.common.annotations.VisibleForTesting;
/**
* Reads region and assignment information from <code>hbase:meta</code>.
*/
@InterfaceAudience.Private
public class MetaReader {
/*
* HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the
* same table range (table, startKey, endKey). For every range, there will be at least one
* HRI defined which is called default replica.
*
* Meta layout (as of 0.98 + HBASE-10070) is like:
* For each table range, there is a single row, formatted like:
* <tableName>,<startKey>,<regionId>,<encodedRegionName>. This row corresponds to the regionName
* of the default region replica.
* Columns are:
* info:regioninfo => contains serialized HRI for the default region replica
* info:server => contains hostname:port (in string form) for the server hosting
* the default regionInfo replica
* info:server_<replicaId> => contains hostname:port (in string form) for the server hosting the
* regionInfo replica with replicaId
* info:serverstartcode => contains server start code (in binary long form) for the server
* hosting the default regionInfo replica
* info:serverstartcode_<replicaId> => contains server start code (in binary long form) for the
* server hosting the regionInfo replica with replicaId
* info:seqnumDuringOpen => contains seqNum (in binary long form) for the region at the time
* the server opened the region with default replicaId
* info:seqnumDuringOpen_<replicaId> => contains seqNum (in binary long form) for the region at
* the time the server opened the region with replicaId
* info:splitA => contains a serialized HRI for the first daughter region if the
* region is split
* info:splitB => contains a serialized HRI for the second daughter region if the
* region is split
* info:mergeA => contains a serialized HRI for the first parent region if the
* region is the result of a merge
* info:mergeB => contains a serialized HRI for the second parent region if the
* region is the result of a merge
*
* The actual layout of meta should be encapsulated inside MetaReader and MetaEditor methods,
* and should not leak out of those (through Result objects, etc)
*/
// TODO: Strip CatalogTracker from this class. Its all over and in the end
// its only used to get its Configuration so we can get associated
// Connection.
private static final Log LOG = LogFactory.getLog(MetaReader.class);
static final byte [] META_REGION_PREFIX;
static {
// Copy the prefix from FIRST_META_REGIONINFO into META_REGION_PREFIX.
// FIRST_META_REGIONINFO == 'hbase:meta,,1'. META_REGION_PREFIX == 'hbase:meta,'
int len = HRegionInfo.FIRST_META_REGIONINFO.getRegionName().length - 2;
META_REGION_PREFIX = new byte [len];
System.arraycopy(HRegionInfo.FIRST_META_REGIONINFO.getRegionName(), 0,
META_REGION_PREFIX, 0, len);
}
/** The delimiter for meta columns for replicaIds > 0 */
protected static final char META_REPLICA_ID_DELIMITER = '_';
/** A regex for parsing server columns from meta. See above javadoc for meta layout */
private static final Pattern SERVER_COLUMN_PATTERN
= Pattern.compile("^server(_[0-9a-fA-F]{4})?$");
/**
* Performs a full scan of <code>hbase:meta</code>.
* @return List of {@link Result}
* @throws IOException
*/
public static List<Result> fullScan(CatalogTracker catalogTracker)
throws IOException {
CollectAllVisitor v = new CollectAllVisitor();
fullScan(catalogTracker, v, null);
return v.getResults();
}
/**
* Performs a full scan of a <code>hbase:meta</code> table.
* @return List of {@link Result}
* @throws IOException
*/
public static List<Result> fullScanOfMeta(CatalogTracker catalogTracker)
throws IOException {
CollectAllVisitor v = new CollectAllVisitor();
fullScan(catalogTracker, v, null);
return v.getResults();
}
/**
* Performs a full scan of <code>hbase:meta</code>.
* @param catalogTracker
* @param visitor Visitor invoked against each row.
* @throws IOException
*/
public static void fullScan(CatalogTracker catalogTracker,
final Visitor visitor)
throws IOException {
fullScan(catalogTracker, visitor, null);
}
/**
* Callers should call close on the returned {@link HTable} instance.
* @param catalogTracker We'll use this catalogtracker's connection
* @param tableName Table to get an {@link HTable} against.
* @return An {@link HTable} for <code>tableName</code>
* @throws IOException
*/
@SuppressWarnings("deprecation")
private static HTable getHTable(final CatalogTracker catalogTracker,
final TableName tableName)
throws IOException {
// Passing the CatalogTracker's connection ensures this
// HTable instance uses the CatalogTracker's connection.
org.apache.hadoop.hbase.client.HConnection c = catalogTracker.getConnection();
if (c == null) throw new NullPointerException("No connection");
return new HTable(tableName, c);
}
/**
* Callers should call close on the returned {@link HTable} instance.
* @param catalogTracker
* @return An {@link HTable} for <code>hbase:meta</code>
* @throws IOException
*/
static HTable getCatalogHTable(final CatalogTracker catalogTracker)
throws IOException {
return getMetaHTable(catalogTracker);
}
/**
* Callers should call close on the returned {@link HTable} instance.
* @param ct
* @return An {@link HTable} for <code>hbase:meta</code>
* @throws IOException
*/
static HTable getMetaHTable(final CatalogTracker ct)
throws IOException {
return getHTable(ct, TableName.META_TABLE_NAME);
}
/**
* @param t Table to use (will be closed when done).
* @param g Get to run
* @throws IOException
*/
private static Result get(final HTable t, final Get g) throws IOException {
try {
return t.get(g);
} finally {
t.close();
}
}
/**
* Gets the region info and assignment for the specified region.
* @param catalogTracker
* @param regionName Region to lookup.
* @return Location and HRegionInfo for <code>regionName</code>
* @throws IOException
* @deprecated use {@link #getRegionLocation(CatalogTracker, byte[])} instead
*/
@Deprecated
public static Pair<HRegionInfo, ServerName> getRegion(
CatalogTracker catalogTracker, byte [] regionName)
throws IOException {
HRegionLocation location = getRegionLocation(catalogTracker, regionName);
return location == null
? null
: new Pair<HRegionInfo, ServerName>(location.getRegionInfo(), location.getServerName());
}
/**
* Returns the HRegionLocation from meta for the given region
* @param catalogTracker
* @param regionName
* @return HRegionLocation for the given region
* @throws IOException
*/
public static HRegionLocation getRegionLocation(CatalogTracker catalogTracker,
byte[] regionName) throws IOException {
byte[] row = regionName;
HRegionInfo parsedInfo = null;
try {
parsedInfo = parseRegionInfoFromRegionName(regionName);
row = getMetaKeyForRegion(parsedInfo);
} catch (Exception parseEx) {
// Ignore. This is used with tableName passed as regionName.
}
Get get = new Get(row);
get.addFamily(HConstants.CATALOG_FAMILY);
Result r = get(getCatalogHTable(catalogTracker), get);
RegionLocations locations = getRegionLocations(r);
return locations == null
? null
: locations.getRegionLocation(parsedInfo == null ? 0 : parsedInfo.getReplicaId());
}
/**
* Returns the HRegionLocation from meta for the given region
* @param catalogTracker
* @param regionInfo
* @return HRegionLocation for the given region
* @throws IOException
*/
public static HRegionLocation getRegionLocation(CatalogTracker catalogTracker,
HRegionInfo regionInfo) throws IOException {
byte[] row = getMetaKeyForRegion(regionInfo);
Get get = new Get(row);
get.addFamily(HConstants.CATALOG_FAMILY);
Result r = get(getCatalogHTable(catalogTracker), get);
return getRegionLocation(r, regionInfo, regionInfo.getReplicaId());
}
/** Returns the row key to use for this regionInfo */
public static byte[] getMetaKeyForRegion(HRegionInfo regionInfo) {
return RegionReplicaUtil.getRegionInfoForDefaultReplica(regionInfo).getRegionName();
}
/** Returns an HRI parsed from this regionName. Not all the fields of the HRI
* is stored in the name, so the returned object should only be used for the fields
* in the regionName.
*/
protected static HRegionInfo parseRegionInfoFromRegionName(byte[] regionName)
throws IOException {
byte[][] fields = HRegionInfo.parseRegionName(regionName);
long regionId = Long.parseLong(Bytes.toString(fields[2]));
int replicaId = fields.length > 3 ? Integer.parseInt(Bytes.toString(fields[3]), 16) : 0;
return new HRegionInfo(
TableName.valueOf(fields[0]), fields[1], fields[1], false, regionId, replicaId);
}
/**
* Gets the result in hbase:meta for the specified region.
* @param catalogTracker
* @param regionName
* @return result of the specified region
* @throws IOException
*/
public static Result getRegionResult(CatalogTracker catalogTracker,
byte[] regionName) throws IOException {
Get get = new Get(regionName);
get.addFamily(HConstants.CATALOG_FAMILY);
return get(getCatalogHTable(catalogTracker), get);
}
/**
* Get regions from the merge qualifier of the specified merged region
* @return null if it doesn't contain merge qualifier, else two merge regions
* @throws IOException
*/
public static Pair<HRegionInfo, HRegionInfo> getRegionsFromMergeQualifier(
CatalogTracker catalogTracker, byte[] regionName) throws IOException {
Result result = getRegionResult(catalogTracker, regionName);
HRegionInfo mergeA = getHRegionInfo(result, HConstants.MERGEA_QUALIFIER);
HRegionInfo mergeB = getHRegionInfo(result, HConstants.MERGEB_QUALIFIER);
if (mergeA == null && mergeB == null) {
return null;
}
return new Pair<HRegionInfo, HRegionInfo>(mergeA, mergeB);
}
/**
* Checks if the specified table exists. Looks at the hbase:meta table hosted on
* the specified server.
* @param catalogTracker
* @param tableName table to check
* @return true if the table exists in meta, false if not
* @throws IOException
*/
public static boolean tableExists(CatalogTracker catalogTracker,
final TableName tableName)
throws IOException {
if (tableName.equals(HTableDescriptor.META_TABLEDESC.getTableName())) {
// Catalog tables always exist.
return true;
}
// Make a version of ResultCollectingVisitor that only collects the first
CollectingVisitor<HRegionInfo> visitor = new CollectingVisitor<HRegionInfo>() {
private HRegionInfo current = null;
@Override
public boolean visit(Result r) throws IOException {
RegionLocations locations = getRegionLocations(r);
if (locations == null || locations.getRegionLocation().getRegionInfo() == null) {
LOG.warn("No serialized HRegionInfo in " + r);
return true;
}
this.current = locations.getRegionLocation().getRegionInfo();
if (this.current == null) {
LOG.warn("No serialized HRegionInfo in " + r);
return true;
}
if (!isInsideTable(this.current, tableName)) return false;
// Else call super and add this Result to the collection.
super.visit(r);
// Stop collecting regions from table after we get one.
return false;
}
@Override
void add(Result r) {
// Add the current HRI.
this.results.add(this.current);
}
};
fullScan(catalogTracker, visitor, getTableStartRowForMeta(tableName));
// If visitor has results >= 1 then table exists.
return visitor.getResults().size() >= 1;
}
/**
* Gets all of the regions of the specified table.
* @param catalogTracker
* @param tableName
* @return Ordered list of {@link HRegionInfo}.
* @throws IOException
*/
public static List<HRegionInfo> getTableRegions(CatalogTracker catalogTracker,
TableName tableName)
throws IOException {
return getTableRegions(catalogTracker, tableName, false);
}
/**
* Gets all of the regions of the specified table.
* @param catalogTracker
* @param tableName
* @param excludeOfflinedSplitParents If true, do not include offlined split
* parents in the return.
* @return Ordered list of {@link HRegionInfo}.
* @throws IOException
*/
public static List<HRegionInfo> getTableRegions(CatalogTracker catalogTracker,
TableName tableName, final boolean excludeOfflinedSplitParents)
throws IOException {
List<Pair<HRegionInfo, ServerName>> result = null;
try {
result = getTableRegionsAndLocations(catalogTracker, tableName,
excludeOfflinedSplitParents);
} catch (InterruptedException e) {
throw (InterruptedIOException)new InterruptedIOException().initCause(e);
}
return getListOfHRegionInfos(result);
}
static List<HRegionInfo> getListOfHRegionInfos(final List<Pair<HRegionInfo, ServerName>> pairs) {
if (pairs == null || pairs.isEmpty()) return null;
List<HRegionInfo> result = new ArrayList<HRegionInfo>(pairs.size());
for (Pair<HRegionInfo, ServerName> pair: pairs) {
result.add(pair.getFirst());
}
return result;
}
/**
* @param current
* @param tableName
* @return True if <code>current</code> tablename is equal to
* <code>tableName</code>
*/
static boolean isInsideTable(final HRegionInfo current, final TableName tableName) {
return tableName.equals(current.getTable());
}
/**
* @param tableName
* @return Place to start Scan in <code>hbase:meta</code> when passed a
* <code>tableName</code>; returns &lt;tableName&rt; &lt;,&rt; &lt;,&rt;
*/
static byte [] getTableStartRowForMeta(TableName tableName) {
byte [] startRow = new byte[tableName.getName().length + 2];
System.arraycopy(tableName.getName(), 0, startRow, 0, tableName.getName().length);
startRow[startRow.length - 2] = HConstants.DELIMITER;
startRow[startRow.length - 1] = HConstants.DELIMITER;
return startRow;
}
/**
* This method creates a Scan object that will only scan catalog rows that
* belong to the specified table. It doesn't specify any columns.
* This is a better alternative to just using a start row and scan until
* it hits a new table since that requires parsing the HRI to get the table
* name.
* @param tableName bytes of table's name
* @return configured Scan object
*/
public static Scan getScanForTableName(TableName tableName) {
String strName = tableName.getNameAsString();
// Start key is just the table name with delimiters
byte[] startKey = Bytes.toBytes(strName + ",,");
// Stop key appends the smallest possible char to the table name
byte[] stopKey = Bytes.toBytes(strName + " ,,");
Scan scan = new Scan(startKey);
scan.setStopRow(stopKey);
return scan;
}
/**
* @param catalogTracker
* @param tableName
* @return Return list of regioninfos and server.
* @throws IOException
* @throws InterruptedException
*/
public static List<Pair<HRegionInfo, ServerName>>
getTableRegionsAndLocations(CatalogTracker catalogTracker, TableName tableName)
throws IOException, InterruptedException {
return getTableRegionsAndLocations(catalogTracker, tableName,
true);
}
/**
* @param catalogTracker
* @param tableName
* @return Return list of regioninfos and server addresses.
* @throws IOException
* @throws InterruptedException
*/
public static List<Pair<HRegionInfo, ServerName>>
getTableRegionsAndLocations(final CatalogTracker catalogTracker,
final TableName tableName, final boolean excludeOfflinedSplitParents)
throws IOException, InterruptedException {
if (tableName.equals(TableName.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.FIRST_META_REGIONINFO,
serverName));
return list;
}
// Make a version of CollectingVisitor that collects HRegionInfo and ServerAddress
CollectingVisitor<Pair<HRegionInfo, ServerName>> visitor =
new CollectingVisitor<Pair<HRegionInfo, ServerName>>() {
private RegionLocations current = null;
@Override
public boolean visit(Result r) throws IOException {
current = getRegionLocations(r);
if (current == null || current.getRegionLocation().getRegionInfo() == null) {
LOG.warn("No serialized HRegionInfo in " + r);
return true;
}
HRegionInfo hri = current.getRegionLocation().getRegionInfo();
if (!isInsideTable(hri, tableName)) return false;
if (excludeOfflinedSplitParents && hri.isSplitParent()) return true;
// Else call super and add this Result to the collection.
return super.visit(r);
}
@Override
void add(Result r) {
if (current == null) {
return;
}
for (HRegionLocation loc : current.getRegionLocations()) {
if (loc != null) {
this.results.add(new Pair<HRegionInfo, ServerName>(
loc.getRegionInfo(), loc.getServerName()));
}
}
}
};
fullScan(catalogTracker, visitor, getTableStartRowForMeta(tableName));
return visitor.getResults();
}
/**
* @param catalogTracker
* @param serverName
* @return List of user regions installed on this server (does not include
* catalog regions).
* @throws IOException
*/
public static NavigableMap<HRegionInfo, Result>
getServerUserRegions(CatalogTracker catalogTracker, final ServerName serverName)
throws IOException {
final NavigableMap<HRegionInfo, Result> hris = new TreeMap<HRegionInfo, Result>();
// Fill the above hris map with entries from hbase:meta that have the passed
// servername.
CollectingVisitor<Result> v = new CollectingVisitor<Result>() {
@Override
void add(Result r) {
if (r == null || r.isEmpty()) return;
RegionLocations locations = getRegionLocations(r);
if (locations == null) return;
for (HRegionLocation loc : locations.getRegionLocations()) {
if (loc != null) {
if (loc.getServerName() != null && loc.getServerName().equals(serverName)) {
hris.put(loc.getRegionInfo(), r);
}
}
}
}
};
fullScan(catalogTracker, v);
return hris;
}
public static void fullScanMetaAndPrint(final CatalogTracker catalogTracker)
throws IOException {
Visitor v = new Visitor() {
@Override
public boolean visit(Result r) throws IOException {
if (r == null || r.isEmpty()) return true;
LOG.info("fullScanMetaAndPrint.Current Meta Row: " + r);
RegionLocations locations = getRegionLocations(r);
if (locations == null) return true;
for (HRegionLocation loc : locations.getRegionLocations()) {
if (loc != null) {
LOG.info("fullScanMetaAndPrint.HRI Print= " + loc.getRegionInfo());
}
}
return true;
}
};
fullScan(catalogTracker, v);
}
/**
* Performs a full scan of a catalog table.
* @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.
* <code>hbase:meta</code>, the default (pass false to scan hbase:meta)
* @throws IOException
*/
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) {
int caching = catalogTracker.getConnection().getConfiguration()
.getInt(HConstants.HBASE_META_SCANNER_CACHING, 100);
scan.setCaching(caching);
}
scan.addFamily(HConstants.CATALOG_FAMILY);
HTable metaTable = getMetaHTable(catalogTracker);
ResultScanner scanner = null;
try {
scanner = metaTable.getScanner(scan);
Result data;
while((data = scanner.next()) != null) {
if (data.isEmpty()) continue;
// Break if visit returns false.
if (!visitor.visit(data)) break;
}
} finally {
if (scanner != null) scanner.close();
metaTable.close();
}
return;
}
/**
* Returns the column family used for meta columns.
* @return HConstants.CATALOG_FAMILY.
*/
protected static byte[] getFamily() {
return HConstants.CATALOG_FAMILY;
}
/**
* Returns the column qualifier for serialized region info
* @return HConstants.REGIONINFO_QUALIFIER
*/
protected static byte[] getRegionInfoColumn() {
return HConstants.REGIONINFO_QUALIFIER;
}
/**
* Returns the column qualifier for server column for replicaId
* @param replicaId the replicaId of the region
* @return a byte[] for server column qualifier
*/
@VisibleForTesting
public static byte[] getServerColumn(int replicaId) {
return replicaId == 0
? HConstants.SERVER_QUALIFIER
: Bytes.toBytes(HConstants.SERVER_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
+ String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
}
/**
* Returns the column qualifier for server start code column for replicaId
* @param replicaId the replicaId of the region
* @return a byte[] for server start code column qualifier
*/
@VisibleForTesting
public static byte[] getStartCodeColumn(int replicaId) {
return replicaId == 0
? HConstants.STARTCODE_QUALIFIER
: Bytes.toBytes(HConstants.STARTCODE_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
+ String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
}
/**
* Returns the column qualifier for seqNum column for replicaId
* @param replicaId the replicaId of the region
* @return a byte[] for seqNum column qualifier
*/
@VisibleForTesting
public static byte[] getSeqNumColumn(int replicaId) {
return replicaId == 0
? HConstants.SEQNUM_QUALIFIER
: Bytes.toBytes(HConstants.SEQNUM_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
+ String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
}
/**
* Parses the replicaId from the server column qualifier. See top of the class javadoc
* for the actual meta layout
* @param serverColumn the column qualifier
* @return an int for the replicaId
*/
@VisibleForTesting
static int parseReplicaIdFromServerColumn(byte[] serverColumn) {
String serverStr = Bytes.toString(serverColumn);
Matcher matcher = SERVER_COLUMN_PATTERN.matcher(serverStr);
if (matcher.matches() && matcher.groupCount() > 0) {
String group = matcher.group(1);
if (group != null && group.length() > 0) {
return Integer.parseInt(group.substring(1), 16);
} else {
return 0;
}
}
return -1;
}
/**
* Returns a {@link ServerName} from catalog table {@link Result}.
* @param r Result to pull from
* @return A ServerName instance or null if necessary fields not found or empty.
*/
private static ServerName getServerName(final Result r, final int replicaId) {
byte[] serverColumn = getServerColumn(replicaId);
Cell cell = r.getColumnLatestCell(getFamily(), serverColumn);
if (cell == null || cell.getValueLength() == 0) return null;
String hostAndPort = Bytes.toString(
cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
byte[] startcodeColumn = getStartCodeColumn(replicaId);
cell = r.getColumnLatestCell(getFamily(), startcodeColumn);
if (cell == null || cell.getValueLength() == 0) return null;
return ServerName.valueOf(hostAndPort,
Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
}
/**
* The latest seqnum that the server writing to meta observed when opening the region.
* E.g. the seqNum when the result of {@link #getServerName(Result)} was written.
* @param r Result to pull the seqNum from
* @return SeqNum, or HConstants.NO_SEQNUM if there's no value written.
*/
private static long getSeqNumDuringOpen(final Result r, final int replicaId) {
Cell cell = r.getColumnLatestCell(getFamily(), getSeqNumColumn(replicaId));
if (cell == null || cell.getValueLength() == 0) return HConstants.NO_SEQNUM;
return Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
}
/**
* Returns an HRegionLocationList extracted from the result.
* @return an HRegionLocationList containing all locations for the region range or null if
* we can't deserialize the result.
*/
public static RegionLocations getRegionLocations(final Result r) {
if (r == null) return null;
HRegionInfo regionInfo = getHRegionInfo(r, getRegionInfoColumn());
if (regionInfo == null) return null;
List<HRegionLocation> locations = new ArrayList<HRegionLocation>(1);
NavigableMap<byte[],NavigableMap<byte[],byte[]>> familyMap = r.getNoVersionMap();
locations.add(getRegionLocation(r, regionInfo, 0));
NavigableMap<byte[], byte[]> infoMap = familyMap.get(getFamily());
if (infoMap == null) return new RegionLocations(locations);
// iterate until all serverName columns are seen
int replicaId = 0;
byte[] serverColumn = getServerColumn(replicaId);
SortedMap<byte[], byte[]> serverMap = infoMap.tailMap(serverColumn, false);
if (serverMap.isEmpty()) return new RegionLocations(locations);
for (Entry<byte[], byte[]> entry : serverMap.entrySet()) {
replicaId = parseReplicaIdFromServerColumn(entry.getKey());
if (replicaId < 0) {
break;
}
locations.add(getRegionLocation(r, regionInfo, replicaId));
}
return new RegionLocations(locations);
}
/**
* Returns the HRegionLocation parsed from the given meta row Result
* for the given regionInfo and replicaId. The regionInfo can be the default region info
* for the replica.
* @param r the meta row result
* @param regionInfo RegionInfo for default replica
* @param replicaId the replicaId for the HRegionLocation
* @return HRegionLocation parsed from the given meta row Result for the given replicaId
*/
private static HRegionLocation getRegionLocation(final Result r, final HRegionInfo regionInfo,
final int replicaId) {
ServerName serverName = getServerName(r, replicaId);
long seqNum = getSeqNumDuringOpen(r, replicaId);
HRegionInfo replicaInfo = RegionReplicaUtil.getRegionInfoForReplica(regionInfo, replicaId);
return new HRegionLocation(replicaInfo, serverName, seqNum);
}
/**
* Returns HRegionInfo object from the column
* HConstants.CATALOG_FAMILY:HConstants.REGIONINFO_QUALIFIER of the catalog
* table Result.
* @param data a Result object from the catalog table scan
* @return HRegionInfo or null
*/
public static HRegionInfo getHRegionInfo(Result data) {
return getHRegionInfo(data, HConstants.REGIONINFO_QUALIFIER);
}
/**
* Returns the HRegionInfo object from the column {@link HConstants#CATALOG_FAMILY} and
* <code>qualifier</code> of the catalog table result.
* @param r a Result object from the catalog table scan
* @param qualifier Column family qualifier
* @return An HRegionInfo instance or null.
*/
private static HRegionInfo getHRegionInfo(final Result r, byte [] qualifier) {
Cell cell = r.getColumnLatestCell(getFamily(), qualifier);
if (cell == null) return null;
return HRegionInfo.parseFromOrNull(cell.getValueArray(),
cell.getValueOffset(), cell.getValueLength());
}
/**
* Returns the daughter regions by reading the corresponding columns of the catalog table
* Result.
* @param data a Result object from the catalog table scan
* @return a pair of HRegionInfo or PairOfSameType(null, null) if the region is not a split
* parent
*/
public static PairOfSameType<HRegionInfo> getDaughterRegions(Result data) throws IOException {
HRegionInfo splitA = getHRegionInfo(data, HConstants.SPLITA_QUALIFIER);
HRegionInfo splitB = getHRegionInfo(data, HConstants.SPLITB_QUALIFIER);
return new PairOfSameType<HRegionInfo>(splitA, splitB);
}
/**
* Returns the merge regions by reading the corresponding columns of the catalog table
* Result.
* @param data a Result object from the catalog table scan
* @return a pair of HRegionInfo or PairOfSameType(null, null) if the region is not a split
* parent
*/
public static PairOfSameType<HRegionInfo> getMergeRegions(Result data) throws IOException {
HRegionInfo mergeA = getHRegionInfo(data, HConstants.MERGEA_QUALIFIER);
HRegionInfo mergeB = getHRegionInfo(data, HConstants.MERGEB_QUALIFIER);
return new PairOfSameType<HRegionInfo>(mergeA, mergeB);
}
/**
* Implementations 'visit' a catalog table row.
*/
public interface Visitor {
/**
* Visit the catalog table row.
* @param r A row from catalog table
* @return True if we are to proceed scanning the table, else false if
* we are to stop now.
*/
boolean visit(final Result r) throws IOException;
}
/**
* A {@link Visitor} that collects content out of passed {@link Result}.
*/
static abstract class CollectingVisitor<T> implements Visitor {
final List<T> results = new ArrayList<T>();
@Override
public boolean visit(Result r) throws IOException {
if (r == null || r.isEmpty()) return true;
add(r);
return true;
}
abstract void add(Result r);
/**
* @return Collected results; wait till visits complete to collect all
* possible results
*/
List<T> getResults() {
return this.results;
}
}
/**
* Collects all returned.
*/
static class CollectAllVisitor extends CollectingVisitor<Result> {
@Override
void add(Result r) {
this.results.add(r);
}
}
/**
* Count regions in <code>hbase:meta</code> for passed table.
* @param c
* @param tableName
* @return Count or regions in table <code>tableName</code>
* @throws IOException
*/
public static int getRegionCount(final Configuration c, final String tableName) throws IOException {
HTable t = new HTable(c, tableName);
try {
return t.getRegionLocations().size();
} finally {
t.close();
}
}
}

View File

@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.RegionTooBusyException;
import org.apache.hadoop.hbase.ServerName;
@ -60,7 +61,6 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotEnabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.client.AsyncProcess.AsyncRequestFuture;
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
@ -1199,7 +1199,7 @@ class ConnectionManager {
}
// convert the row result into the HRegionLocation we need!
RegionLocations locations = MetaReader.getRegionLocations(regionInfoRow);
RegionLocations locations = MetaTableAccessor.getRegionLocations(regionInfoRow);
if (locations == null || locations.getRegionLocation(replicaId) == null) {
throw new IOException("HRegionInfo was null in " +
tableName + ", row=" + regionInfoRow);

View File

@ -38,6 +38,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseIOException;
@ -47,6 +48,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.RegionException;
@ -59,8 +61,6 @@ import org.apache.hadoop.hbase.TableNotEnabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
@ -142,6 +142,7 @@ import org.apache.hadoop.hbase.util.Addressing;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.util.StringUtils;
import org.apache.zookeeper.KeeperException;
@ -163,6 +164,8 @@ import com.google.protobuf.ServiceException;
public class HBaseAdmin implements Admin {
private static final Log LOG = LogFactory.getLog(HBaseAdmin.class);
private static final String ZK_IDENTIFIER_PREFIX = "hbase-admin-on-";
// We use the implementation class rather then the interface because we
// need the package protected functions to get the connection to master
private ClusterConnection connection;
@ -232,30 +235,6 @@ public class HBaseAdmin implements Admin {
this.rpcCallerFactory = RpcRetryingCallerFactory.instantiate(this.conf);
}
/**
* @return A new CatalogTracker instance; call {@link #cleanupCatalogTracker(CatalogTracker)}
* to cleanup the returned catalog tracker.
* @throws org.apache.hadoop.hbase.ZooKeeperConnectionException
* @throws IOException
* @see #cleanupCatalogTracker(CatalogTracker)
*/
private synchronized CatalogTracker getCatalogTracker()
throws ZooKeeperConnectionException, IOException {
CatalogTracker ct = null;
try {
ct = new CatalogTracker(this.conf);
ct.start();
} catch (InterruptedException e) {
// Let it out as an IOE for now until we redo all so tolerate IEs
throw (InterruptedIOException)new InterruptedIOException("Interrupted").initCause(e);
}
return ct;
}
private void cleanupCatalogTracker(final CatalogTracker ct) {
ct.stop();
}
@Override
public void abort(String why, Throwable e) {
// Currently does nothing but throw the passed message and exception
@ -290,17 +269,8 @@ public class HBaseAdmin implements Admin {
* @return True if table exists already.
* @throws IOException
*/
@Override
public boolean tableExists(final TableName tableName)
throws IOException {
boolean b = false;
CatalogTracker ct = getCatalogTracker();
try {
b = MetaReader.tableExists(ct, tableName);
} finally {
cleanupCatalogTracker(ct);
}
return b;
public boolean tableExists(final TableName tableName) throws IOException {
return MetaTableAccessor.tableExists(connection, tableName);
}
public boolean tableExists(final byte[] tableName)
@ -534,7 +504,7 @@ public class HBaseAdmin implements Admin {
MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
@Override
public boolean processRow(Result rowResult) throws IOException {
RegionLocations list = MetaReader.getRegionLocations(rowResult);
RegionLocations list = MetaTableAccessor.getRegionLocations(rowResult);
if (list == null) {
LOG.warn("No serialized HRegionInfo in " + rowResult);
return true;
@ -680,7 +650,7 @@ public class HBaseAdmin implements Admin {
for (int tries = 0; tries < (this.numRetries * this.retryLongerMultiplier); tries++) {
try {
HRegionLocation firstMetaServer = getFirstMetaServerForTable(tableName);
Scan scan = MetaReader.getScanForTableName(tableName);
Scan scan = MetaTableAccessor.getScanForTableName(tableName);
scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
ScanRequest request = RequestConverter.buildScanRequest(
firstMetaServer.getRegionInfo().getRegionName(), scan, 1, true);
@ -1406,17 +1376,15 @@ public class HBaseAdmin implements Admin {
@Override
public void closeRegion(final byte [] regionname, final String serverName)
throws IOException {
CatalogTracker ct = getCatalogTracker();
try {
if (serverName != null) {
Pair<HRegionInfo, ServerName> pair = MetaReader.getRegion(ct, regionname);
Pair<HRegionInfo, ServerName> pair = MetaTableAccessor.getRegion(connection, regionname);
if (pair == null || pair.getFirst() == null) {
throw new UnknownRegionException(Bytes.toStringBinary(regionname));
} else {
closeRegion(ServerName.valueOf(serverName), pair.getFirst());
}
} else {
Pair<HRegionInfo, ServerName> pair = MetaReader.getRegion(ct, regionname);
Pair<HRegionInfo, ServerName> pair = MetaTableAccessor.getRegion(connection, regionname);
if (pair == null) {
throw new UnknownRegionException(Bytes.toStringBinary(regionname));
} else if (pair.getSecond() == null) {
@ -1425,9 +1393,6 @@ public class HBaseAdmin implements Admin {
closeRegion(pair.getSecond(), pair.getFirst());
}
}
} finally {
cleanupCatalogTracker(ct);
}
}
/**
@ -1525,10 +1490,8 @@ public class HBaseAdmin implements Admin {
@Override
public void flush(final byte[] tableNameOrRegionName)
throws IOException, InterruptedException {
CatalogTracker ct = getCatalogTracker();
try {
Pair<HRegionInfo, ServerName> regionServerPair
= getRegion(tableNameOrRegionName, ct);
= getRegion(tableNameOrRegionName);
if (regionServerPair != null) {
if (regionServerPair.getSecond() == null) {
throw new NoServerForRegionException(Bytes.toStringBinary(tableNameOrRegionName));
@ -1537,7 +1500,7 @@ public class HBaseAdmin implements Admin {
}
} else {
final TableName tableName = checkTableExists(
TableName.valueOf(tableNameOrRegionName), ct);
TableName.valueOf(tableNameOrRegionName));
if (isTableDisabled(tableName)) {
LOG.info("Table is disabled: " + tableName.getNameAsString());
return;
@ -1545,9 +1508,6 @@ public class HBaseAdmin implements Admin {
execProcedure("flush-table-proc", tableName.getNameAsString(),
new HashMap<String, String>());
}
} finally {
cleanupCatalogTracker(ct);
}
}
private void flush(final ServerName sn, final HRegionInfo hri)
@ -1692,10 +1652,10 @@ public class HBaseAdmin implements Admin {
private void compact(final byte[] tableNameOrRegionName,
final byte[] columnFamily,final boolean major)
throws IOException, InterruptedException {
CatalogTracker ct = getCatalogTracker();
ZooKeeperWatcher zookeeper = null;
try {
Pair<HRegionInfo, ServerName> regionServerPair
= getRegion(tableNameOrRegionName, ct);
= getRegion(tableNameOrRegionName);
if (regionServerPair != null) {
if (regionServerPair.getSecond() == null) {
throw new NoServerForRegionException(Bytes.toStringBinary(tableNameOrRegionName));
@ -1704,9 +1664,11 @@ public class HBaseAdmin implements Admin {
}
} else {
final TableName tableName =
checkTableExists(TableName.valueOf(tableNameOrRegionName), ct);
checkTableExists(TableName.valueOf(tableNameOrRegionName));
zookeeper = new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
new ThrowableAbortable());
List<Pair<HRegionInfo, ServerName>> pairs =
MetaReader.getTableRegionsAndLocations(ct,
MetaTableAccessor.getTableRegionsAndLocations(zookeeper, connection,
tableName);
for (Pair<HRegionInfo, ServerName> pair: pairs) {
if (pair.getFirst().isOffline()) continue;
@ -1723,7 +1685,9 @@ public class HBaseAdmin implements Admin {
}
}
} finally {
cleanupCatalogTracker(ct);
if (zookeeper != null) {
zookeeper.close();
}
}
}
@ -2034,10 +1998,10 @@ public class HBaseAdmin implements Admin {
@Override
public void split(final byte[] tableNameOrRegionName,
final byte [] splitPoint) throws IOException, InterruptedException {
CatalogTracker ct = getCatalogTracker();
ZooKeeperWatcher zookeeper = null;
try {
Pair<HRegionInfo, ServerName> regionServerPair
= getRegion(tableNameOrRegionName, ct);
= getRegion(tableNameOrRegionName);
if (regionServerPair != null) {
if (regionServerPair.getSecond() == null) {
throw new NoServerForRegionException(Bytes.toStringBinary(tableNameOrRegionName));
@ -2046,9 +2010,11 @@ public class HBaseAdmin implements Admin {
}
} else {
final TableName tableName =
checkTableExists(TableName.valueOf(tableNameOrRegionName), ct);
checkTableExists(TableName.valueOf(tableNameOrRegionName));
zookeeper = new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
new ThrowableAbortable());
List<Pair<HRegionInfo, ServerName>> pairs =
MetaReader.getTableRegionsAndLocations(ct,
MetaTableAccessor.getTableRegionsAndLocations(zookeeper, connection,
tableName);
for (Pair<HRegionInfo, ServerName> pair: pairs) {
// May not be a server for a particular row
@ -2063,7 +2029,9 @@ public class HBaseAdmin implements Admin {
}
}
} finally {
cleanupCatalogTracker(ct);
if (zookeeper != null) {
zookeeper.close();
}
}
}
@ -2116,19 +2084,19 @@ public class HBaseAdmin implements Admin {
/**
* @param tableNameOrRegionName Name of a table or name of a region.
* @param ct A {@link CatalogTracker} instance (caller of this method usually has one).
* @return a pair of HRegionInfo and ServerName if <code>tableNameOrRegionName</code> is
* a verified region name (we call {@link MetaReader#getRegion( CatalogTracker, byte[])}
* a verified region name (we call {@link
* MetaTableAccessor#getRegion(HConnection, byte[])}
* else null.
* Throw an exception if <code>tableNameOrRegionName</code> is null.
* @throws IOException
*/
Pair<HRegionInfo, ServerName> getRegion(final byte[] tableNameOrRegionName,
final CatalogTracker ct) throws IOException {
Pair<HRegionInfo, ServerName> getRegion(final byte[] tableNameOrRegionName) throws IOException {
if (tableNameOrRegionName == null) {
throw new IllegalArgumentException("Pass a table name or region name");
}
Pair<HRegionInfo, ServerName> pair = MetaReader.getRegion(ct, tableNameOrRegionName);
Pair<HRegionInfo, ServerName> pair =
MetaTableAccessor.getRegion(connection, tableNameOrRegionName);
if (pair == null) {
final AtomicReference<Pair<HRegionInfo, ServerName>> result =
new AtomicReference<Pair<HRegionInfo, ServerName>>(null);
@ -2170,33 +2138,24 @@ public class HBaseAdmin implements Admin {
HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes())) {
return HRegionInfo.FIRST_META_REGIONINFO.getRegionName();
}
CatalogTracker ct = getCatalogTracker();
byte[] tmp = regionNameOrEncodedRegionName;
try {
Pair<HRegionInfo, ServerName> regionServerPair
= getRegion(regionNameOrEncodedRegionName, ct);
Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionNameOrEncodedRegionName);
if (regionServerPair != null && regionServerPair.getFirst() != null) {
tmp = regionServerPair.getFirst().getRegionName();
}
} finally {
cleanupCatalogTracker(ct);
}
return tmp;
}
/**
* Check if table exists or not
* @param tableName Name of a table.
* @param ct A {@link CatalogTracker} instance (caller of this method usually has one).
* @return tableName instance
* @throws IOException if a remote or network exception occurs.
* @throws TableNotFoundException if table does not exist.
*/
//TODO rename this method
private TableName checkTableExists(
final TableName tableName, CatalogTracker ct)
private TableName checkTableExists(final TableName tableName)
throws IOException {
if (!MetaReader.tableExists(ct, tableName)) {
if (!MetaTableAccessor.tableExists(connection, tableName)) {
throw new TableNotFoundException(tableName);
}
return tableName;
@ -2492,12 +2451,14 @@ public class HBaseAdmin implements Admin {
@Override
public List<HRegionInfo> getTableRegions(final TableName tableName)
throws IOException {
CatalogTracker ct = getCatalogTracker();
ZooKeeperWatcher zookeeper =
new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
new ThrowableAbortable());
List<HRegionInfo> Regions = null;
try {
Regions = MetaReader.getTableRegions(ct, tableName, true);
Regions = MetaTableAccessor.getTableRegions(zookeeper, connection, tableName, true);
} finally {
cleanupCatalogTracker(ct);
zookeeper.close();
}
return Regions;
}
@ -2614,10 +2575,11 @@ public synchronized byte[][] rollHLogWriter(String serverName)
public CompactionState getCompactionState(final byte[] tableNameOrRegionName)
throws IOException, InterruptedException {
CompactionState state = CompactionState.NONE;
CatalogTracker ct = getCatalogTracker();
ZooKeeperWatcher zookeeper =
new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
new ThrowableAbortable());
try {
Pair<HRegionInfo, ServerName> regionServerPair
= getRegion(tableNameOrRegionName, ct);
Pair<HRegionInfo, ServerName> regionServerPair = getRegion(tableNameOrRegionName);
if (regionServerPair != null) {
if (regionServerPair.getSecond() == null) {
throw new NoServerForRegionException(Bytes.toStringBinary(tableNameOrRegionName));
@ -2631,9 +2593,9 @@ public synchronized byte[][] rollHLogWriter(String serverName)
}
} else {
final TableName tableName =
checkTableExists(TableName.valueOf(tableNameOrRegionName), ct);
checkTableExists(TableName.valueOf(tableNameOrRegionName));
List<Pair<HRegionInfo, ServerName>> pairs =
MetaReader.getTableRegionsAndLocations(ct, tableName);
MetaTableAccessor.getTableRegionsAndLocations(zookeeper, connection, tableName);
for (Pair<HRegionInfo, ServerName> pair: pairs) {
if (pair.getFirst().isOffline()) continue;
if (pair.getSecond() == null) continue;
@ -2682,7 +2644,7 @@ public synchronized byte[][] rollHLogWriter(String serverName)
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
} finally {
cleanupCatalogTracker(ct);
zookeeper.close();
}
return state;
}
@ -3546,4 +3508,20 @@ public synchronized byte[][] rollHLogWriter(String serverName)
public CoprocessorRpcChannel coprocessorService() {
return new MasterCoprocessorRpcChannel(connection);
}
/**
* Simple {@link Abortable}, throwing RuntimeException on abort.
*/
private static class ThrowableAbortable implements Abortable {
@Override
public void abort(String why, Throwable e) {
throw new RuntimeException(why, e);
}
@Override
public boolean isAborted() {
return true;
}
}
}

View File

@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
@ -49,7 +48,8 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
* connections are managed at a lower level.
*
* <p>HConnections are used by {@link HTable} mostly but also by
* {@link HBaseAdmin}, and {@link CatalogTracker}. HConnection instances can be shared. Sharing
* {@link HBaseAdmin}, and {@link org.apache.hadoop.hbase.zookeeper.MetaTableLocator}.
* HConnection instances can be shared. Sharing
* is usually what you want because rather than each HConnection instance
* having to do its own discovery of regions out on the cluster, instead, all
* clients get to share the one cache of locations. {@link HConnectionManager} does the

View File

@ -33,11 +33,11 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ExceptionUtil;
@ -53,7 +53,7 @@ import org.apache.hadoop.hbase.util.ExceptionUtil;
* see HBASE-5986, and {@link DefaultMetaScannerVisitor} for details. </p>
*/
@InterfaceAudience.Private
//TODO: merge this to MetaReader, get rid of it.
//TODO: merge this to MetaTableAccessor, get rid of it.
public class MetaScanner {
private static final Log LOG = LogFactory.getLog(MetaScanner.class);
/**
@ -227,7 +227,7 @@ public class MetaScanner {
* table Result.
* @param data a Result object from the catalog table scan
* @return HRegionInfo or null
* @deprecated Use {@link MetaReader#getRegionLocations(Result)}
* @deprecated Use {@link org.apache.hadoop.hbase.MetaTableAccessor#getRegionLocations(Result)}
*/
@Deprecated
public static HRegionInfo getHRegionInfo(Result data) {
@ -252,7 +252,7 @@ public class MetaScanner {
return true;
}
RegionLocations locations = MetaReader.getRegionLocations(result);
RegionLocations locations = MetaTableAccessor.getRegionLocations(result);
if (locations == null) return true;
for (HRegionLocation loc : locations.getRegionLocations()) {
if (loc != null) {
@ -285,7 +285,7 @@ public class MetaScanner {
MetaScannerVisitor visitor = new TableMetaScannerVisitor(tableName) {
@Override
public boolean processRowInternal(Result result) throws IOException {
RegionLocations locations = MetaReader.getRegionLocations(result);
RegionLocations locations = MetaTableAccessor.getRegionLocations(result);
if (locations == null) return true;
for (HRegionLocation loc : locations.getRegionLocations()) {
if (loc != null) {
@ -309,7 +309,7 @@ public class MetaScanner {
MetaScannerVisitor visitor = new TableMetaScannerVisitor(tableName) {
@Override
public boolean processRowInternal(Result result) throws IOException {
RegionLocations locations = MetaReader.getRegionLocations(result);
RegionLocations locations = MetaTableAccessor.getRegionLocations(result);
if (locations == null) return true;
regions.add(locations);
return true;

View File

@ -27,7 +27,7 @@ import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKTableStateClientSideReader;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@ -57,7 +57,7 @@ class ZooKeeperRegistry implements Registry {
if (LOG.isTraceEnabled()) {
LOG.trace("Looking up meta region location in ZK," + " connection=" + this);
}
ServerName servername = MetaRegionTracker.blockUntilAvailable(zkw, hci.rpcTimeout);
ServerName servername = new MetaTableLocator().blockUntilAvailable(zkw, hci.rpcTimeout);
if (LOG.isTraceEnabled()) {
LOG.trace("Looked up meta region location, connection=" + this +
"; serverName=" + ((servername == null) ? "null" : servername));

View File

@ -1,195 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.zookeeper;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HConstants;
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 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 MetaRegionTracker extends ZooKeeperNodeTracker {
/**
* Creates a meta region location tracker.
*
* <p>After construction, use {@link #start} to kick off tracking.
*
* @param watcher
* @param abortable
*/
public MetaRegionTracker(ZooKeeperWatcher watcher, Abortable abortable) {
super(watcher, watcher.metaServerZNode, abortable);
}
/**
* 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 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 getMetaRegionLocation() throws InterruptedException {
try {
return ServerName.parseFrom(super.getData(true));
} catch (DeserializationException e) {
LOG.warn("Failed parse", e);
return null;
}
}
/**
* 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 getMetaRegionLocation(final ZooKeeperWatcher zkw)
throws KeeperException {
try {
return ServerName.parseFrom(ZKUtil.getData(zkw, zkw.metaServerZNode));
} catch (DeserializationException e) {
throw ZKUtil.convert(e);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
return null;
}
}
/**
* 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 meta region formatted as per
* {@link ServerName}, or null if none available
* @throws InterruptedException if interrupted while waiting
*/
public ServerName waitMetaRegionLocation(long timeout)
throws InterruptedException {
if (false == checkIfBaseNodeAvailable()) {
String errorMsg = "Check the value configured in 'zookeeper.znode.parent'. "
+ "There could be a mismatch with the one configured in the master.";
LOG.error(errorMsg);
throw new IllegalArgumentException(errorMsg);
}
try {
return ServerName.parseFrom(super.blockUntilAvailable(timeout, true));
} catch (DeserializationException e) {
LOG.warn("Failed parse", e);
return null;
}
}
/**
* Sets the location of <code>hbase:meta</code> in ZooKeeper to the
* specified server address.
* @param zookeeper zookeeper reference
* @param location The server hosting <code>hbase:meta</code>
* @throws KeeperException unexpected zookeeper exception
*/
public static void setMetaLocation(ZooKeeperWatcher zookeeper,
final ServerName location)
throws KeeperException {
LOG.info("Setting hbase: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.metaServerZNode, data);
} catch(KeeperException.NodeExistsException nee) {
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 meta-region-server znode
*/
static byte [] toByteArray(final ServerName sn) {
// ZNode content is a pb message preceded by some pb magic.
HBaseProtos.ServerName pbsn =
HBaseProtos.ServerName.newBuilder()
.setHostName(sn.getHostname())
.setPort(sn.getPort())
.setStartCode(sn.getStartcode())
.build();
ZooKeeperProtos.MetaRegionServer pbrsr =
ZooKeeperProtos.MetaRegionServer.newBuilder()
.setServer(pbsn)
.setRpcVersion(HConstants.RPC_CURRENT_VERSION)
.build();
return ProtobufUtil.prependPBMagic(pbrsr.toByteArray());
}
/**
* Deletes the location of <code>hbase:meta</code> in ZooKeeper.
* @param zookeeper zookeeper reference
* @throws KeeperException unexpected zookeeper exception
*/
public static void deleteMetaLocation(ZooKeeperWatcher zookeeper)
throws KeeperException {
LOG.info("Unsetting hbase:meta region location in ZooKeeper");
try {
// Just delete the node. Don't need any watches.
ZKUtil.deleteNode(zookeeper, zookeeper.metaServerZNode);
} catch(KeeperException.NoNodeException nne) {
// Has already been deleted
}
}
/**
* Wait until the meta region is available.
* @param zkw
* @param timeout
* @return ServerName or null if we timed out.
* @throws InterruptedException
*/
public static ServerName blockUntilAvailable(final ZooKeeperWatcher zkw,
final long timeout)
throws InterruptedException {
byte [] data = ZKUtil.blockUntilAvailable(zkw, zkw.metaServerZNode, timeout);
if (data == null) return null;
try {
return ServerName.parseFrom(data);
} catch (DeserializationException e) {
LOG.warn("Failed parse", e);
return null;
}
}
}

View File

@ -48,7 +48,6 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionStoreSequenceIds;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ExceptionUtil;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.CreateAndFailSilent;
import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.DeleteNodeFailSilent;
@ -1590,7 +1589,8 @@ public class ZKUtil {
zkw.backupMasterAddressesZNode)) {
sb.append("\n ").append(child);
}
sb.append("\nRegion server holding hbase:meta: " + MetaRegionTracker.getMetaRegionLocation(zkw));
sb.append("\nRegion server holding hbase:meta: "
+ new MetaTableLocator().getMetaRegionLocation(zkw));
sb.append("\nRegion servers:");
for (String child : listChildrenNoWatch(zkw, zkw.rsZNode)) {
sb.append("\n ").append(child);

View File

@ -1,585 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.catalog;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService;
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads;
import com.google.protobuf.ServiceException;
/**
* Writes region and assignment information to <code>hbase:meta</code>.
*/
@InterfaceAudience.Private
public class MetaEditor extends MetaReader {
// TODO: Strip CatalogTracker from this class. Its all over and in the end
// its only used to get its Configuration so we can get associated
// Connection.
private static final Log LOG = LogFactory.getLog(MetaEditor.class);
/**
* Generates and returns a Put containing the region into for the catalog table
*/
public static Put makePutFromRegionInfo(HRegionInfo regionInfo)
throws IOException {
Put put = new Put(regionInfo.getRegionName());
addRegionInfo(put, regionInfo);
return put;
}
/**
* Generates and returns a Delete containing the region info for the catalog
* table
*/
public static Delete makeDeleteFromRegionInfo(HRegionInfo regionInfo) {
if (regionInfo == null) {
throw new IllegalArgumentException("Can't make a delete for null region");
}
Delete delete = new Delete(regionInfo.getRegionName());
return delete;
}
/**
* Adds split daughters to the Put
*/
public static Put addDaughtersToPut(Put put, HRegionInfo splitA, HRegionInfo splitB) {
if (splitA != null) {
put.addImmutable(
HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER, splitA.toByteArray());
}
if (splitB != null) {
put.addImmutable(
HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER, splitB.toByteArray());
}
return put;
}
/**
* Put the passed <code>p</code> to the <code>hbase:meta</code> table.
* @param ct CatalogTracker on whose back we will ride the edit.
* @param p Put to add to hbase:meta
* @throws IOException
*/
static void putToMetaTable(final CatalogTracker ct, final Put p)
throws IOException {
put(MetaReader.getMetaHTable(ct), p);
}
/**
* Put the passed <code>p</code> to a catalog table.
* @param ct CatalogTracker on whose back we will ride the edit.
* @param p Put to add
* @throws IOException
*/
static void putToCatalogTable(final CatalogTracker ct, final Put p)
throws IOException {
put(MetaReader.getCatalogHTable(ct), p);
}
/**
* @param t Table to use (will be closed when done).
* @param p
* @throws IOException
*/
private static void put(final HTable t, final Put p) throws IOException {
try {
t.put(p);
} finally {
t.close();
}
}
/**
* Put the passed <code>ps</code> to the <code>hbase:meta</code> table.
* @param ct CatalogTracker on whose back we will ride the edit.
* @param ps Put to add to hbase:meta
* @throws IOException
*/
public static void putsToMetaTable(final CatalogTracker ct, final List<Put> ps)
throws IOException {
HTable t = MetaReader.getMetaHTable(ct);
try {
t.put(ps);
} finally {
t.close();
}
}
/**
* Delete the passed <code>d</code> from the <code>hbase:meta</code> table.
* @param ct CatalogTracker on whose back we will ride the edit.
* @param d Delete to add to hbase:meta
* @throws IOException
*/
static void deleteFromMetaTable(final CatalogTracker ct, final Delete d)
throws IOException {
List<Delete> dels = new ArrayList<Delete>(1);
dels.add(d);
deleteFromMetaTable(ct, dels);
}
/**
* Delete the passed <code>deletes</code> from the <code>hbase:meta</code> table.
* @param ct CatalogTracker on whose back we will ride the edit.
* @param deletes Deletes to add to hbase:meta This list should support #remove.
* @throws IOException
*/
public static void deleteFromMetaTable(final CatalogTracker ct, final List<Delete> deletes)
throws IOException {
HTable t = MetaReader.getMetaHTable(ct);
try {
t.delete(deletes);
} finally {
t.close();
}
}
/**
* Deletes some replica columns corresponding to replicas for the passed rows
* @param metaRows
* @param replicaIndexToDeleteFrom the replica ID we would start deleting from
* @param numReplicasToRemove
* @param ct
* @throws IOException
*/
public static void removeRegionReplicasFromMeta(Set<byte[]> metaRows, int replicaIndexToDeleteFrom,
int numReplicasToRemove, CatalogTracker ct) throws IOException {
int absoluteIndex = replicaIndexToDeleteFrom + numReplicasToRemove;
for (byte[] row : metaRows) {
Delete deleteReplicaLocations = new Delete(row);
for (int i = replicaIndexToDeleteFrom; i < absoluteIndex; i++) {
deleteReplicaLocations.deleteColumns(HConstants.CATALOG_FAMILY,
MetaReader.getServerColumn(i));
deleteReplicaLocations.deleteColumns(HConstants.CATALOG_FAMILY,
MetaReader.getSeqNumColumn(i));
deleteReplicaLocations.deleteColumns(HConstants.CATALOG_FAMILY,
MetaReader.getStartCodeColumn(i));
}
deleteFromMetaTable(ct, deleteReplicaLocations);
}
}
/**
* Execute the passed <code>mutations</code> against <code>hbase:meta</code> table.
* @param ct CatalogTracker on whose back we will ride the edit.
* @param mutations Puts and Deletes to execute on hbase:meta
* @throws IOException
*/
public static void mutateMetaTable(final CatalogTracker ct, final List<Mutation> mutations)
throws IOException {
HTable t = MetaReader.getMetaHTable(ct);
try {
t.batch(mutations);
} catch (InterruptedException e) {
InterruptedIOException ie = new InterruptedIOException(e.getMessage());
ie.initCause(e);
throw ie;
} finally {
t.close();
}
}
/**
* Adds a hbase:meta row for the specified new region.
* @param regionInfo region information
* @throws IOException if problem connecting or updating meta
*/
public static void addRegionToMeta(CatalogTracker catalogTracker,
HRegionInfo regionInfo)
throws IOException {
putToMetaTable(catalogTracker, makePutFromRegionInfo(regionInfo));
LOG.info("Added " + regionInfo.getRegionNameAsString());
}
/**
* Adds a hbase:meta row for the specified new region to the given catalog table. The
* HTable is not flushed or closed.
* @param meta the HTable for META
* @param regionInfo region information
* @throws IOException if problem connecting or updating meta
*/
public static void addRegionToMeta(HTable meta, HRegionInfo regionInfo) throws IOException {
addRegionToMeta(meta, regionInfo, null, null);
}
/**
* Adds a (single) hbase:meta row for the specified new region and its daughters. Note that this does
* not add its daughter's as different rows, but adds information about the daughters
* in the same row as the parent. Use
* {@link #splitRegion(CatalogTracker, HRegionInfo, HRegionInfo, HRegionInfo, ServerName)}
* if you want to do that.
* @param meta the HTable for META
* @param regionInfo region information
* @param splitA first split daughter of the parent regionInfo
* @param splitB second split daughter of the parent regionInfo
* @throws IOException if problem connecting or updating meta
*/
public static void addRegionToMeta(HTable meta, HRegionInfo regionInfo,
HRegionInfo splitA, HRegionInfo splitB) throws IOException {
Put put = makePutFromRegionInfo(regionInfo);
addDaughtersToPut(put, splitA, splitB);
meta.put(put);
if (LOG.isDebugEnabled()) {
LOG.debug("Added " + regionInfo.getRegionNameAsString());
}
}
/**
* Adds a (single) hbase:meta row for the specified new region and its daughters. Note that this does
* not add its daughter's as different rows, but adds information about the daughters
* in the same row as the parent. Use
* {@link #splitRegion(CatalogTracker, HRegionInfo, HRegionInfo, HRegionInfo, ServerName)}
* if you want to do that.
* @param catalogTracker CatalogTracker on whose back we will ride the edit.
* @param regionInfo region information
* @param splitA first split daughter of the parent regionInfo
* @param splitB second split daughter of the parent regionInfo
* @throws IOException if problem connecting or updating meta
*/
public static void addRegionToMeta(CatalogTracker catalogTracker, HRegionInfo regionInfo,
HRegionInfo splitA, HRegionInfo splitB) throws IOException {
HTable meta = MetaReader.getMetaHTable(catalogTracker);
try {
addRegionToMeta(meta, regionInfo, splitA, splitB);
} finally {
meta.close();
}
}
/**
* Adds a hbase:meta row for each of the specified new regions.
* @param catalogTracker CatalogTracker
* @param regionInfos region information list
* @throws IOException if problem connecting or updating meta
*/
public static void addRegionsToMeta(CatalogTracker catalogTracker,
List<HRegionInfo> regionInfos)
throws IOException {
List<Put> puts = new ArrayList<Put>();
for (HRegionInfo regionInfo : regionInfos) {
if (RegionReplicaUtil.isDefaultReplica(regionInfo)) {
puts.add(makePutFromRegionInfo(regionInfo));
}
}
putsToMetaTable(catalogTracker, puts);
LOG.info("Added " + puts.size());
}
/**
* Adds a daughter region entry to meta.
* @param regionInfo the region to put
* @param sn the location of the region
* @param openSeqNum the latest sequence number obtained when the region was open
*/
public static void addDaughter(final CatalogTracker catalogTracker,
final HRegionInfo regionInfo, final ServerName sn, final long openSeqNum)
throws NotAllMetaRegionsOnlineException, IOException {
Put put = new Put(regionInfo.getRegionName());
addRegionInfo(put, regionInfo);
if (sn != null) {
addLocation(put, sn, openSeqNum, regionInfo.getReplicaId());
}
putToMetaTable(catalogTracker, put);
LOG.info("Added daughter " + regionInfo.getEncodedName() +
(sn == null? ", serverName=null": ", serverName=" + sn.toString()));
}
/**
* Merge the two regions into one in an atomic operation. Deletes the two
* merging regions in hbase:meta and adds the merged region with the information of
* two merging regions.
* @param catalogTracker the catalog tracker
* @param mergedRegion the merged region
* @param regionA
* @param regionB
* @param sn the location of the region
* @throws IOException
*/
public static void mergeRegions(final CatalogTracker catalogTracker,
HRegionInfo mergedRegion, HRegionInfo regionA, HRegionInfo regionB,
ServerName sn) throws IOException {
HTable meta = MetaReader.getMetaHTable(catalogTracker);
try {
HRegionInfo copyOfMerged = new HRegionInfo(mergedRegion);
// Put for parent
Put putOfMerged = makePutFromRegionInfo(copyOfMerged);
putOfMerged.addImmutable(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER,
regionA.toByteArray());
putOfMerged.addImmutable(HConstants.CATALOG_FAMILY, HConstants.MERGEB_QUALIFIER,
regionB.toByteArray());
// Deletes for merging regions
Delete deleteA = makeDeleteFromRegionInfo(regionA);
Delete deleteB = makeDeleteFromRegionInfo(regionB);
// The merged is a new region, openSeqNum = 1 is fine.
addLocation(putOfMerged, sn, 1, mergedRegion.getReplicaId());
byte[] tableRow = Bytes.toBytes(mergedRegion.getRegionNameAsString()
+ HConstants.DELIMITER);
multiMutate(meta, tableRow, putOfMerged, deleteA, deleteB);
} finally {
meta.close();
}
}
/**
* Splits the region into two in an atomic operation. Offlines the parent
* region with the information that it is split into two, and also adds
* the daughter regions. Does not add the location information to the daughter
* regions since they are not open yet.
* @param catalogTracker the catalog tracker
* @param parent the parent region which is split
* @param splitA Split daughter region A
* @param splitB Split daughter region A
* @param sn the location of the region
*/
public static void splitRegion(final CatalogTracker catalogTracker,
HRegionInfo parent, HRegionInfo splitA, HRegionInfo splitB,
ServerName sn) throws IOException {
HTable meta = MetaReader.getMetaHTable(catalogTracker);
try {
HRegionInfo copyOfParent = new HRegionInfo(parent);
copyOfParent.setOffline(true);
copyOfParent.setSplit(true);
//Put for parent
Put putParent = makePutFromRegionInfo(copyOfParent);
addDaughtersToPut(putParent, splitA, splitB);
//Puts for daughters
Put putA = makePutFromRegionInfo(splitA);
Put putB = makePutFromRegionInfo(splitB);
addLocation(putA, sn, 1, splitA.getReplicaId()); //new regions, openSeqNum = 1 is fine.
addLocation(putB, sn, 1, splitB.getReplicaId());
byte[] tableRow = Bytes.toBytes(parent.getRegionNameAsString() + HConstants.DELIMITER);
multiMutate(meta, tableRow, putParent, putA, putB);
} finally {
meta.close();
}
}
/**
* Performs an atomic multi-Mutate operation against the given table.
*/
private static void multiMutate(HTable table, byte[] row, Mutation... mutations) throws IOException {
CoprocessorRpcChannel channel = table.coprocessorService(row);
MutateRowsRequest.Builder mmrBuilder = MutateRowsRequest.newBuilder();
for (Mutation mutation : mutations) {
if (mutation instanceof Put) {
mmrBuilder.addMutationRequest(ProtobufUtil.toMutation(MutationType.PUT, mutation));
} else if (mutation instanceof Delete) {
mmrBuilder.addMutationRequest(ProtobufUtil.toMutation(MutationType.DELETE, mutation));
} else {
throw new DoNotRetryIOException("multi in MetaEditor doesn't support "
+ mutation.getClass().getName());
}
}
MultiRowMutationService.BlockingInterface service =
MultiRowMutationService.newBlockingStub(channel);
try {
service.mutateRows(null, mmrBuilder.build());
} catch (ServiceException ex) {
ProtobufUtil.toIOException(ex);
}
}
/**
* Updates the location of the specified region in hbase:meta to be the specified
* server hostname and startcode.
* <p>
* Uses passed catalog tracker to get a connection to the server hosting
* hbase:meta and makes edits to that region.
*
* @param catalogTracker catalog tracker
* @param regionInfo region to update location of
* @param sn Server name
* @throws IOException
*/
public static void updateRegionLocation(CatalogTracker catalogTracker,
HRegionInfo regionInfo, ServerName sn, long updateSeqNum)
throws IOException {
updateLocation(catalogTracker, regionInfo, sn, updateSeqNum);
}
/**
* Updates the location of the specified region to be the specified server.
* <p>
* Connects to the specified server which should be hosting the specified
* catalog region name to perform the edit.
*
* @param catalogTracker
* @param regionInfo region to update location of
* @param sn Server name
* @param openSeqNum the latest sequence number obtained when the region was open
* @throws IOException In particular could throw {@link java.net.ConnectException}
* if the server is down on other end.
*/
private static void updateLocation(final CatalogTracker catalogTracker,
HRegionInfo regionInfo, ServerName sn, long openSeqNum)
throws IOException {
// region replicas are kept in the primary region's row
Put put = new Put(getMetaKeyForRegion(regionInfo));
addLocation(put, sn, openSeqNum, regionInfo.getReplicaId());
putToCatalogTable(catalogTracker, put);
LOG.info("Updated row " + regionInfo.getRegionNameAsString() +
" with server=" + sn);
}
/**
* Deletes the specified region from META.
* @param catalogTracker
* @param regionInfo region to be deleted from META
* @throws IOException
*/
public static void deleteRegion(CatalogTracker catalogTracker,
HRegionInfo regionInfo)
throws IOException {
Delete delete = new Delete(regionInfo.getRegionName());
deleteFromMetaTable(catalogTracker, delete);
LOG.info("Deleted " + regionInfo.getRegionNameAsString());
}
/**
* Deletes the specified regions from META.
* @param catalogTracker
* @param regionsInfo list of regions to be deleted from META
* @throws IOException
*/
public static void deleteRegions(CatalogTracker catalogTracker,
List<HRegionInfo> regionsInfo) throws IOException {
List<Delete> deletes = new ArrayList<Delete>(regionsInfo.size());
for (HRegionInfo hri: regionsInfo) {
deletes.add(new Delete(hri.getRegionName()));
}
deleteFromMetaTable(catalogTracker, deletes);
LOG.info("Deleted " + regionsInfo);
}
/**
* Adds and Removes the specified regions from hbase:meta
* @param catalogTracker
* @param regionsToRemove list of regions to be deleted from META
* @param regionsToAdd list of regions to be added to META
* @throws IOException
*/
public static void mutateRegions(CatalogTracker catalogTracker,
final List<HRegionInfo> regionsToRemove, final List<HRegionInfo> regionsToAdd)
throws IOException {
List<Mutation> mutation = new ArrayList<Mutation>();
if (regionsToRemove != null) {
for (HRegionInfo hri: regionsToRemove) {
mutation.add(new Delete(hri.getRegionName()));
}
}
if (regionsToAdd != null) {
for (HRegionInfo hri: regionsToAdd) {
mutation.add(makePutFromRegionInfo(hri));
}
}
mutateMetaTable(catalogTracker, mutation);
if (regionsToRemove != null && regionsToRemove.size() > 0) {
LOG.debug("Deleted " + regionsToRemove);
}
if (regionsToAdd != null && regionsToAdd.size() > 0) {
LOG.debug("Added " + regionsToAdd);
}
}
/**
* Overwrites the specified regions from hbase:meta
* @param catalogTracker
* @param regionInfos list of regions to be added to META
* @throws IOException
*/
public static void overwriteRegions(CatalogTracker catalogTracker,
List<HRegionInfo> regionInfos) throws IOException {
deleteRegions(catalogTracker, regionInfos);
// Why sleep? This is the easiest way to ensure that the previous deletes does not
// eclipse the following puts, that might happen in the same ts from the server.
// See HBASE-9906, and HBASE-9879. Once either HBASE-9879, HBASE-8770 is fixed,
// or HBASE-9905 is fixed and meta uses seqIds, we do not need the sleep.
Threads.sleep(20);
addRegionsToMeta(catalogTracker, regionInfos);
LOG.info("Overwritten " + regionInfos);
}
/**
* Deletes merge qualifiers for the specified merged region.
* @param catalogTracker
* @param mergedRegion
* @throws IOException
*/
public static void deleteMergeQualifiers(CatalogTracker catalogTracker,
final HRegionInfo mergedRegion) throws IOException {
Delete delete = new Delete(mergedRegion.getRegionName());
delete.deleteColumns(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER);
delete.deleteColumns(HConstants.CATALOG_FAMILY, HConstants.MERGEB_QUALIFIER);
deleteFromMetaTable(catalogTracker, delete);
LOG.info("Deleted references in merged region "
+ mergedRegion.getRegionNameAsString() + ", qualifier="
+ Bytes.toStringBinary(HConstants.MERGEA_QUALIFIER) + " and qualifier="
+ Bytes.toStringBinary(HConstants.MERGEB_QUALIFIER));
}
private static Put addRegionInfo(final Put p, final HRegionInfo hri)
throws IOException {
p.addImmutable(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
hri.toByteArray());
return p;
}
public static Put addLocation(final Put p, final ServerName sn, long openSeqNum, int replicaId){
p.addImmutable(HConstants.CATALOG_FAMILY, MetaReader.getServerColumn(replicaId),
Bytes.toBytes(sn.getHostAndPort()));
p.addImmutable(HConstants.CATALOG_FAMILY, MetaReader.getStartCodeColumn(replicaId),
Bytes.toBytes(sn.getStartcode()));
p.addImmutable(HConstants.CATALOG_FAMILY, MetaReader.getSeqNumColumn(replicaId),
Bytes.toBytes(openSeqNum));
return p;
}
}

View File

@ -1,177 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.catalog;
import java.io.IOException;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.catalog.MetaReader.Visitor;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.util.Bytes;
/**
* A tool to migrate the data stored in hbase:meta table to pbuf serialization.
* Supports migrating from 0.92.x and 0.94.x to 0.96.x for the catalog table.
* @deprecated will be removed for the major release after 0.96.
*/
@Deprecated
public class MetaMigrationConvertingToPB {
private static final Log LOG = LogFactory.getLog(MetaMigrationConvertingToPB.class);
private static class ConvertToPBMetaVisitor implements Visitor {
private final MasterServices services;
private long numMigratedRows;
public ConvertToPBMetaVisitor(MasterServices services) {
this.services = services;
numMigratedRows = 0;
}
@Override
public boolean visit(Result r) throws IOException {
if (r == null || r.isEmpty()) return true;
// Check info:regioninfo, info:splitA, and info:splitB. Make sure all
// have migrated HRegionInfos.
byte [] hriBytes = getBytes(r, HConstants.REGIONINFO_QUALIFIER);
// Presumes that an edit updating all three cells either succeeds or
// doesn't -- that we don't have case of info:regioninfo migrated but not
// info:splitA.
if (isMigrated(hriBytes)) return true;
// OK. Need to migrate this row in meta.
//This will 'migrate' the HRI from 092.x and 0.94.x to 0.96+ by reading the
//writable serialization
HRegionInfo hri = parseFrom(hriBytes);
// Now make a put to write back to meta.
Put p = MetaEditor.makePutFromRegionInfo(hri);
// Now migrate info:splitA and info:splitB if they are not null
migrateSplitIfNecessary(r, p, HConstants.SPLITA_QUALIFIER);
migrateSplitIfNecessary(r, p, HConstants.SPLITB_QUALIFIER);
MetaEditor.putToCatalogTable(this.services.getCatalogTracker(), p);
if (LOG.isDebugEnabled()) {
LOG.debug("Migrated " + Bytes.toString(p.getRow()));
}
numMigratedRows++;
return true;
}
}
static void migrateSplitIfNecessary(final Result r, final Put p, final byte [] which)
throws IOException {
byte [] hriSplitBytes = getBytes(r, which);
if (!isMigrated(hriSplitBytes)) {
//This will 'migrate' the HRI from 092.x and 0.94.x to 0.96+ by reading the
//writable serialization
HRegionInfo hri = parseFrom(hriSplitBytes);
p.addImmutable(HConstants.CATALOG_FAMILY, which, hri.toByteArray());
}
}
static HRegionInfo parseFrom(byte[] hriBytes) throws IOException {
try {
return HRegionInfo.parseFrom(hriBytes);
} catch (DeserializationException ex) {
throw new IOException(ex);
}
}
/**
* @param r Result to dig in.
* @param qualifier Qualifier to look at in the passed <code>r</code>.
* @return Bytes for an HRegionInfo or null if no bytes or empty bytes found.
*/
static byte [] getBytes(final Result r, final byte [] qualifier) {
byte [] hriBytes = r.getValue(HConstants.CATALOG_FAMILY, qualifier);
if (hriBytes == null || hriBytes.length <= 0) return null;
return hriBytes;
}
static boolean isMigrated(final byte [] hriBytes) {
if (hriBytes == null || hriBytes.length <= 0) return true;
return ProtobufUtil.isPBMagicPrefix(hriBytes);
}
/**
* Converting writable serialization to PB, if it is needed.
* @param services MasterServices to get a handle on master
* @return num migrated rows
* @throws IOException or RuntimeException if something goes wrong
*/
public static long updateMetaIfNecessary(final MasterServices services)
throws IOException {
if (isMetaTableUpdated(services.getCatalogTracker())) {
LOG.info("META already up-to date with PB serialization");
return 0;
}
LOG.info("META has Writable serializations, migrating hbase:meta to PB serialization");
try {
long rows = updateMeta(services);
LOG.info("META updated with PB serialization. Total rows updated: " + rows);
return rows;
} catch (IOException e) {
LOG.warn("Update hbase:meta with PB serialization failed." + "Master startup aborted.");
throw e;
}
}
/**
* Update hbase:meta rows, converting writable serialization to PB
* @return num migrated rows
*/
static long updateMeta(final MasterServices masterServices) throws IOException {
LOG.info("Starting update of META");
ConvertToPBMetaVisitor v = new ConvertToPBMetaVisitor(masterServices);
MetaReader.fullScan(masterServices.getCatalogTracker(), v);
LOG.info("Finished update of META. Total rows updated:" + v.numMigratedRows);
return v.numMigratedRows;
}
/**
* @param catalogTracker the catalog tracker
* @return True if the meta table has been migrated.
* @throws IOException
*/
static boolean isMetaTableUpdated(final CatalogTracker catalogTracker) throws IOException {
List<Result> results = MetaReader.fullScanOfMeta(catalogTracker);
if (results == null || results.isEmpty()) {
LOG.info("hbase:meta doesn't have any entries to update.");
return true;
}
for (Result r : results) {
byte[] value = r.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
if (!isMigrated(value)) {
return false;
}
}
return true;
}
}

View File

@ -25,7 +25,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.mapreduce.MutationSerialization;
@ -212,7 +212,7 @@ public class TableMapReduceUtil {
MutationSerialization.class.getName(), ResultSerialization.class.getName());
if (partitioner == HRegionPartitioner.class) {
job.setPartitionerClass(HRegionPartitioner.class);
int regions = MetaReader.getRegionCount(HBaseConfiguration.create(job), table);
int regions = MetaTableAccessor.getRegionCount(HBaseConfiguration.create(job), table);
if (job.getNumReduceTasks() > regions) {
job.setNumReduceTasks(regions);
}
@ -278,7 +278,7 @@ public class TableMapReduceUtil {
*/
public static void limitNumReduceTasks(String table, JobConf job)
throws IOException {
int regions = MetaReader.getRegionCount(HBaseConfiguration.create(job), table);
int regions = MetaTableAccessor.getRegionCount(HBaseConfiguration.create(job), table);
if (job.getNumReduceTasks() > regions)
job.setNumReduceTasks(regions);
}
@ -293,7 +293,7 @@ public class TableMapReduceUtil {
*/
public static void limitNumMapTasks(String table, JobConf job)
throws IOException {
int regions = MetaReader.getRegionCount(HBaseConfiguration.create(job), table);
int regions = MetaTableAccessor.getRegionCount(HBaseConfiguration.create(job), table);
if (job.getNumMapTasks() > regions)
job.setNumMapTasks(regions);
}
@ -308,7 +308,7 @@ public class TableMapReduceUtil {
*/
public static void setNumReduceTasks(String table, JobConf job)
throws IOException {
job.setNumReduceTasks(MetaReader.getRegionCount(HBaseConfiguration.create(job), table));
job.setNumReduceTasks(MetaTableAccessor.getRegionCount(HBaseConfiguration.create(job), table));
}
/**
@ -321,7 +321,7 @@ public class TableMapReduceUtil {
*/
public static void setNumMapTasks(String table, JobConf job)
throws IOException {
job.setNumMapTasks(MetaReader.getRegionCount(HBaseConfiguration.create(job), table));
job.setNumMapTasks(MetaTableAccessor.getRegionCount(HBaseConfiguration.create(job), table));
}
/**

View File

@ -43,7 +43,7 @@ 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.catalog.MetaReader;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
@ -635,7 +635,7 @@ public class TableMapReduceUtil {
job.setOutputValueClass(Writable.class);
if (partitioner == HRegionPartitioner.class) {
job.setPartitionerClass(HRegionPartitioner.class);
int regions = MetaReader.getRegionCount(conf, table);
int regions = MetaTableAccessor.getRegionCount(conf, table);
if (job.getNumReduceTasks() > regions) {
job.setNumReduceTasks(regions);
}
@ -660,7 +660,7 @@ public class TableMapReduceUtil {
*/
public static void limitNumReduceTasks(String table, Job job)
throws IOException {
int regions = MetaReader.getRegionCount(job.getConfiguration(), table);
int regions = MetaTableAccessor.getRegionCount(job.getConfiguration(), table);
if (job.getNumReduceTasks() > regions)
job.setNumReduceTasks(regions);
}
@ -675,7 +675,7 @@ public class TableMapReduceUtil {
*/
public static void setNumReduceTasks(String table, Job job)
throws IOException {
job.setNumReduceTasks(MetaReader.getRegionCount(job.getConfiguration(), table));
job.setNumReduceTasks(MetaTableAccessor.getRegionCount(job.getConfiguration(), table));
}
/**

View File

@ -61,9 +61,8 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.TableStateManager;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
import org.apache.hadoop.hbase.coordination.OpenRegionCoordination;
@ -101,7 +100,7 @@ import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.PairOfSameType;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.Triple;
import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
@ -139,8 +138,6 @@ public class AssignmentManager extends ZooKeeperListener {
private boolean shouldAssignRegionsWithFavoredNodes;
private CatalogTracker catalogTracker;
private LoadBalancer balancer;
private final MetricsAssignmentManager metricsAssignmentManager;
@ -254,22 +251,23 @@ public class AssignmentManager extends ZooKeeperListener {
/**
* Constructs a new assignment manager.
*
* @param server
* @param serverManager
* @param catalogTracker
* @param service
* @param server instance of HMaster this AM running inside
* @param serverManager serverManager for associated HMaster
* @param balancer implementation of {@link LoadBalancer}
* @param service Executor service
* @param metricsMaster metrics manager
* @param tableLockManager TableLock manager
* @throws KeeperException
* @throws IOException
*/
public AssignmentManager(Server server, ServerManager serverManager,
CatalogTracker catalogTracker, final LoadBalancer balancer,
final LoadBalancer balancer,
final ExecutorService service, MetricsMaster metricsMaster,
final TableLockManager tableLockManager) throws KeeperException,
IOException, CoordinatedStateException {
super(server.getZooKeeper());
this.server = server;
this.serverManager = serverManager;
this.catalogTracker = catalogTracker;
this.executorService = service;
this.regionStateStore = new RegionStateStore(server);
this.regionsToReopen = Collections.synchronizedMap
@ -404,7 +402,8 @@ public class AssignmentManager extends ZooKeeperListener {
public Pair<Integer, Integer> getReopenStatus(TableName tableName)
throws IOException {
List <HRegionInfo> hris =
MetaReader.getTableRegions(this.server.getCatalogTracker(), tableName, true);
MetaTableAccessor.getTableRegions(this.watcher, this.server.getShortCircuitConnection(),
tableName, true);
Integer pending = 0;
for (HRegionInfo hri : hris) {
String name = hri.getEncodedName();
@ -759,7 +758,7 @@ public class AssignmentManager extends ZooKeeperListener {
if (regionInfo.isMetaRegion()) {
// If it's meta region, reset the meta location.
// So that master knows the right meta region server.
MetaRegionTracker.setMetaLocation(watcher, sn);
MetaTableLocator.setMetaLocation(watcher, sn);
} else {
// No matter the previous server is online or offline,
// we need to reset the last region server of the region.
@ -1129,7 +1128,8 @@ public class AssignmentManager extends ZooKeeperListener {
regionToFavoredNodes.put(region,
((FavoredNodeLoadBalancer)this.balancer).getFavoredNodes(region));
}
FavoredNodeAssignmentHelper.updateMetaWithFavoredNodesInfo(regionToFavoredNodes, catalogTracker);
FavoredNodeAssignmentHelper.updateMetaWithFavoredNodesInfo(regionToFavoredNodes,
this.server.getShortCircuitConnection());
}
/**
@ -1152,7 +1152,8 @@ public class AssignmentManager extends ZooKeeperListener {
} else {
try {
byte [] name = rt.getRegionName();
Pair<HRegionInfo, ServerName> p = MetaReader.getRegion(catalogTracker, name);
Pair<HRegionInfo, ServerName> p = MetaTableAccessor.getRegion(
this.server.getShortCircuitConnection(), name);
regionInfo = p.getFirst();
} catch (IOException e) {
LOG.info("Exception reading hbase:meta doing HBCK repair operation", e);
@ -1935,13 +1936,15 @@ public class AssignmentManager extends ZooKeeperListener {
final HRegionInfo region, final ServerName sn) {
try {
if (region.isMetaRegion()) {
ServerName server = catalogTracker.getMetaLocation();
ServerName server = this.server.getMetaTableLocator().
getMetaRegionLocation(this.server.getZooKeeper());
return regionStates.isServerDeadAndNotProcessed(server);
}
while (!server.isStopped()) {
try {
catalogTracker.waitForMeta();
Result r = MetaReader.getRegionResult(catalogTracker, region.getRegionName());
this.server.getMetaTableLocator().waitMetaRegionLocation(server.getZooKeeper());
Result r = MetaTableAccessor.getRegionResult(server.getShortCircuitConnection(),
region.getRegionName());
if (r == null || r.isEmpty()) return false;
ServerName server = HRegionInfo.getServerName(r);
return regionStates.isServerDeadAndNotProcessed(server);
@ -2554,7 +2557,7 @@ public class AssignmentManager extends ZooKeeperListener {
* @throws KeeperException
*/
public void assignMeta() throws KeeperException {
MetaRegionTracker.deleteMetaLocation(this.watcher);
this.server.getMetaTableLocator().deleteMetaLocation(this.watcher);
assign(HRegionInfo.FIRST_META_REGIONINFO, true);
}
@ -2754,7 +2757,7 @@ public class AssignmentManager extends ZooKeeperListener {
ZooKeeperProtos.Table.State.ENABLING);
// Region assignment from META
List<Result> results = MetaReader.fullScan(this.catalogTracker);
List<Result> results = MetaTableAccessor.fullScanOfMeta(server.getShortCircuitConnection());
// Get any new but slow to checkin region server that joined the cluster
Set<ServerName> onlineServers = serverManager.getOnlineServers().keySet();
// Set of offline servers to be returned
@ -2765,7 +2768,7 @@ public class AssignmentManager extends ZooKeeperListener {
LOG.debug("null result from meta - ignoring but this is strange.");
continue;
}
RegionLocations rl = MetaReader.getRegionLocations(result);
RegionLocations rl = MetaTableAccessor.getRegionLocations(result);
if (rl == null) continue;
HRegionLocation[] locations = rl.getRegionLocations();
if (locations == null) continue;
@ -2826,7 +2829,7 @@ public class AssignmentManager extends ZooKeeperListener {
LOG.info("The table " + tableName
+ " is in DISABLING state. Hence recovering by moving the table"
+ " to DISABLED state.");
new DisableTableHandler(this.server, tableName, catalogTracker,
new DisableTableHandler(this.server, tableName,
this, tableLockManager, true).prepare().process();
}
}
@ -2853,7 +2856,7 @@ public class AssignmentManager extends ZooKeeperListener {
// enableTable in sync way during master startup,
// no need to invoke coprocessor
EnableTableHandler eth = new EnableTableHandler(this.server, tableName,
catalogTracker, this, tableLockManager, true);
this, tableLockManager, true);
try {
eth.prepare();
} catch (TableNotFoundException e) {

View File

@ -40,8 +40,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.backup.HFileArchiver;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.MetaScanner;
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
import org.apache.hadoop.hbase.client.Result;
@ -199,7 +198,8 @@ public class CatalogJanitor extends Chore {
+ " from fs because merged region no longer holds references");
HFileArchiver.archiveRegion(this.services.getConfiguration(), fs, regionA);
HFileArchiver.archiveRegion(this.services.getConfiguration(), fs, regionB);
MetaEditor.deleteMergeQualifiers(server.getCatalogTracker(), mergedRegion);
MetaTableAccessor.deleteMergeQualifiers(server.getShortCircuitConnection(),
mergedRegion);
return true;
}
return false;
@ -331,7 +331,7 @@ public class CatalogJanitor extends Chore {
FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
if (LOG.isTraceEnabled()) LOG.trace("Archiving parent region: " + parent);
HFileArchiver.archiveRegion(this.services.getConfiguration(), fs, parent);
MetaEditor.deleteRegion(this.server.getCatalogTracker(), parent);
MetaTableAccessor.deleteRegion(this.server.getShortCircuitConnection(), parent);
result = true;
}
return result;
@ -403,8 +403,8 @@ public class CatalogJanitor extends Chore {
throws IOException {
// Get merge regions if it is a merged region and already has merge
// qualifier
Pair<HRegionInfo, HRegionInfo> mergeRegions = MetaReader
.getRegionsFromMergeQualifier(this.services.getCatalogTracker(),
Pair<HRegionInfo, HRegionInfo> mergeRegions = MetaTableAccessor
.getRegionsFromMergeQualifier(this.services.getShortCircuitConnection(),
region.getRegionName());
if (mergeRegions == null
|| (mergeRegions.getFirst() == null && mergeRegions.getSecond() == null)) {

View File

@ -64,7 +64,8 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.MetaMigrationConvertingToPB;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.MetaScanner;
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
@ -117,6 +118,7 @@ import org.apache.hadoop.hbase.util.VersionInfo;
import org.apache.hadoop.hbase.zookeeper.DrainingServerTracker;
import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.RegionServerTracker;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@ -409,7 +411,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
this.loadBalancerTracker = new LoadBalancerTracker(zooKeeper, this);
this.loadBalancerTracker.start();
this.assignmentManager = new AssignmentManager(this, serverManager,
this.catalogTracker, this.balancer, this.service, this.metricsMaster,
this.balancer, this.service, this.metricsMaster,
this.tableLockManager);
zooKeeper.registerListenerFirst(assignmentManager);
@ -482,8 +484,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
ZKClusterId.setClusterId(this.zooKeeper, fileSystemManager.getClusterId());
this.serverManager = createServerManager(this, this);
// Now we have the cluster ID, start catalog tracker
startCatalogTracker();
metaTableLocator = new MetaTableLocator();
shortCircuitConnection = createShortCircuitConnection();
// Invalidate all write locks held previously
this.tableLockManager.reapWriteLocks();
@ -523,7 +525,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
this.fileSystemManager.removeStaleRecoveringRegionsFromZK(previouslyFailedServers);
// log splitting for hbase:meta server
ServerName oldMetaServerLocation = this.catalogTracker.getMetaLocation();
ServerName oldMetaServerLocation = metaTableLocator.getMetaRegionLocation(this.getZooKeeper());
if (oldMetaServerLocation != null && previouslyFailedServers.contains(oldMetaServerLocation)) {
splitMetaLogBeforeAssignment(oldMetaServerLocation);
// Note: we can't remove oldMetaServerLocation from previousFailedServers list because it
@ -575,8 +577,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
// Update meta with new PB serialization if required. i.e migrate all HRI to PB serialization
// in meta. This must happen before we assign all user regions or else the assignment will
// fail.
org.apache.hadoop.hbase.catalog.MetaMigrationConvertingToPB
.updateMetaIfNecessary(this);
MetaMigrationConvertingToPB.updateMetaIfNecessary(this);
// Fix up assignment manager status
status.setStatus("Starting assignment manager");
@ -675,8 +676,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
regionStates.createRegionState(HRegionInfo.FIRST_META_REGIONINFO);
boolean rit = this.assignmentManager
.processRegionInTransitionAndBlockUntilAssigned(HRegionInfo.FIRST_META_REGIONINFO);
boolean metaRegionLocation = this.catalogTracker.verifyMetaRegionLocation(timeout);
ServerName currentMetaServer = this.catalogTracker.getMetaLocation();
boolean metaRegionLocation = metaTableLocator.verifyMetaRegionLocation(
this.getShortCircuitConnection(), this.getZooKeeper(), timeout);
ServerName currentMetaServer = metaTableLocator.getMetaRegionLocation(this.getZooKeeper());
if (!metaRegionLocation) {
// Meta location is not verified. It should be in transition, or offline.
// We will wait for it to be assigned in enableSSHandWaitForMeta below.
@ -724,7 +726,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
enableServerShutdownHandler(assigned != 0);
LOG.info("hbase:meta assigned=" + assigned + ", rit=" + rit +
", location=" + catalogTracker.getMetaLocation());
", location=" + metaTableLocator.getMetaRegionLocation(this.getZooKeeper()));
status.setStatus("META assigned.");
}
@ -764,7 +766,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
}
if (waitForMeta) {
this.catalogTracker.waitForMeta();
metaTableLocator.waitMetaRegionLocation(this.getZooKeeper());
// Above check waits for general meta availability but this does not
// guarantee that the transition has completed
this.assignmentManager.waitForAssignment(HRegionInfo.FIRST_META_REGIONINFO);
@ -1410,7 +1412,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
}
LOG.info(getClientIdAuditPrefix() + " enable " + tableName);
this.service.submit(new EnableTableHandler(this, tableName,
catalogTracker, assignmentManager, tableLockManager, false).prepare());
assignmentManager, tableLockManager, false).prepare());
if (cpHost != null) {
cpHost.postEnableTable(tableName);
}
@ -1424,7 +1426,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
}
LOG.info(getClientIdAuditPrefix() + " disable " + tableName);
this.service.submit(new DisableTableHandler(this, tableName,
catalogTracker, assignmentManager, tableLockManager, false).prepare());
assignmentManager, tableLockManager, false).prepare());
if (cpHost != null) {
cpHost.postDisableTable(tableName);
}
@ -1486,7 +1488,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
if (isCatalogTable(tableName)) {
throw new IOException("Can't modify catalog tables");
}
if (!MetaReader.tableExists(getCatalogTracker(), tableName)) {
if (!MetaTableAccessor.tableExists(getShortCircuitConnection(), tableName)) {
throw new TableNotFoundException(tableName);
}
if (!getAssignmentManager().getTableStateManager().

View File

@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.exceptions.MergeRegionException;
import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
@ -1086,7 +1086,7 @@ public class MasterRpcServices extends RSRpcServices
try {
master.checkInitialized();
Pair<HRegionInfo, ServerName> pair =
MetaReader.getRegion(master.getCatalogTracker(), regionName);
MetaTableAccessor.getRegion(master.getShortCircuitConnection(), regionName);
if (pair == null) throw new UnknownRegionException(Bytes.toStringBinary(regionName));
HRegionInfo hri = pair.getFirst();
if (master.cpHost != null) {
@ -1217,7 +1217,7 @@ public class MasterRpcServices extends RSRpcServices
+ " actual: " + type);
}
Pair<HRegionInfo, ServerName> pair =
MetaReader.getRegion(master.getCatalogTracker(), regionName);
MetaTableAccessor.getRegion(master.getShortCircuitConnection(), regionName);
if (pair == null) throw new UnknownRegionException(Bytes.toString(regionName));
HRegionInfo hri = pair.getFirst();
if (master.cpHost != null) {

View File

@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.tmpl.master.MasterStatusTmpl;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
/**
* The servlet responsible for rendering the index page of the
@ -87,12 +88,7 @@ public class MasterStatusServlet extends HttpServlet {
}
private ServerName getMetaLocationOrNull(HMaster master) {
try {
return (master.getCatalogTracker() == null) ? null : master.getCatalogTracker().getMetaLocation();
} catch (InterruptedException e) {
LOG.warn("Unable to get meta location", e);
return null;
}
return master.getMetaTableLocator().getMetaRegionLocation(master.getZooKeeper());
}
private Map<String, Integer> getFragmentationInfo(

View File

@ -73,7 +73,7 @@ public class RackManager {
/**
* Same as {@link #getRack(ServerName)} except that a list is passed
* @param servers
* @param servers list of servers we're requesting racks information for
* @return list of racks for the given list of servers
*/
public List<String> getRack(List<ServerName> servers) {

View File

@ -47,9 +47,9 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.master.balancer.FavoredNodeAssignmentHelper;
import org.apache.hadoop.hbase.master.balancer.FavoredNodesPlan;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@ -141,7 +141,7 @@ public class RegionPlacementMaintainer {
public SnapshotOfRegionAssignmentFromMeta getRegionAssignmentSnapshot()
throws IOException {
SnapshotOfRegionAssignmentFromMeta currentAssignmentShapshot =
new SnapshotOfRegionAssignmentFromMeta(new CatalogTracker(this.conf));
new SnapshotOfRegionAssignmentFromMeta(HConnectionManager.getConnection(conf));
currentAssignmentShapshot.initialize();
return currentAssignmentShapshot;
}

View File

@ -31,9 +31,7 @@ import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.Put;
@ -62,7 +60,6 @@ public class RegionStateStore {
private volatile boolean initialized;
private final boolean noPersistence;
private final CatalogTracker catalogTracker;
private final Server server;
/**
@ -76,7 +73,7 @@ public class RegionStateStore {
static ServerName getRegionServer(final Result r, int replicaId) {
Cell cell = r.getColumnLatestCell(HConstants.CATALOG_FAMILY, getServerNameColumn(replicaId));
if (cell == null || cell.getValueLength() == 0) {
RegionLocations locations = MetaReader.getRegionLocations(r);
RegionLocations locations = MetaTableAccessor.getRegionLocations(r);
if (locations != null) {
HRegionLocation location = locations.getRegionLocation(replicaId);
if (location != null) {
@ -138,7 +135,6 @@ public class RegionStateStore {
// No need to persist if using ZK but not migrating
noPersistence = ConfigUtil.useZKForAssignment(conf)
&& !conf.getBoolean("hbase.assignment.usezk.migrating", false);
catalogTracker = server.getCatalogTracker();
this.server = server;
initialized = false;
}
@ -152,7 +148,7 @@ public class RegionStateStore {
}
if (metaRegion == null) {
metaTable = new HTable(TableName.META_TABLE_NAME,
catalogTracker.getConnection());
server.getShortCircuitConnection());
}
}
initialized = true;
@ -189,7 +185,7 @@ public class RegionStateStore {
try {
int replicaId = hri.getReplicaId();
Put put = new Put(MetaReader.getMetaKeyForRegion(hri));
Put put = new Put(MetaTableAccessor.getMetaKeyForRegion(hri));
StringBuilder info = new StringBuilder("Updating row ");
info.append(hri.getRegionNameAsString()).append(" with state=").append(state);
if (serverName != null && !serverName.equals(oldServer)) {
@ -200,7 +196,7 @@ public class RegionStateStore {
if (openSeqNum >= 0) {
Preconditions.checkArgument(state == State.OPEN
&& serverName != null, "Open region should be on a server");
MetaEditor.addLocation(put, serverName, openSeqNum, replicaId);
MetaTableAccessor.addLocation(put, serverName, openSeqNum, replicaId);
info.append("&openSeqNum=").append(openSeqNum);
info.append("&server=").append(serverName);
}
@ -224,7 +220,7 @@ public class RegionStateStore {
if (metaRegion != null) {
LOG.info("Meta region shortcut failed", t);
metaTable = new HTable(TableName.META_TABLE_NAME,
catalogTracker.getConnection());
server.getShortCircuitConnection());
metaRegion = null;
}
}
@ -241,11 +237,11 @@ public class RegionStateStore {
void splitRegion(HRegionInfo p,
HRegionInfo a, HRegionInfo b, ServerName sn) throws IOException {
MetaEditor.splitRegion(catalogTracker, p, a, b, sn);
MetaTableAccessor.splitRegion(server.getShortCircuitConnection(), p, a, b, sn);
}
void mergeRegions(HRegionInfo p,
HRegionInfo a, HRegionInfo b, ServerName sn) throws IOException {
MetaEditor.mergeRegions(catalogTracker, p, a, b, sn);
MetaTableAccessor.mergeRegions(server.getShortCircuitConnection(), p, a, b, sn);
}
}

View File

@ -39,8 +39,8 @@ import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.master.RegionState.State;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
@ -930,7 +930,7 @@ public class RegionStates {
try {
Pair<HRegionInfo, ServerName> p =
MetaReader.getRegion(server.getCatalogTracker(), regionName);
MetaTableAccessor.getRegion(server.getShortCircuitConnection(), regionName);
HRegionInfo hri = p == null ? null : p.getFirst();
if (hri != null) {
createRegionState(hri);

View File

@ -37,9 +37,9 @@ import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.catalog.MetaReader.Visitor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.MetaTableAccessor.Visitor;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.master.balancer.FavoredNodeAssignmentHelper;
import org.apache.hadoop.hbase.master.balancer.FavoredNodesPlan;
@ -55,7 +55,7 @@ public class SnapshotOfRegionAssignmentFromMeta {
private static final Log LOG = LogFactory.getLog(SnapshotOfRegionAssignmentFromMeta.class
.getName());
private CatalogTracker tracker;
private final HConnection hConnection;
/** the table name to region map */
private final Map<TableName, List<HRegionInfo>> tableToRegionMap;
@ -72,13 +72,13 @@ public class SnapshotOfRegionAssignmentFromMeta {
private final Set<TableName> disabledTables;
private final boolean excludeOfflinedSplitParents;
public SnapshotOfRegionAssignmentFromMeta(CatalogTracker tracker) {
this(tracker, new HashSet<TableName>(), false);
public SnapshotOfRegionAssignmentFromMeta(HConnection hConnection) {
this(hConnection, new HashSet<TableName>(), false);
}
public SnapshotOfRegionAssignmentFromMeta(CatalogTracker tracker, Set<TableName> disabledTables,
public SnapshotOfRegionAssignmentFromMeta(HConnection hConnection, Set<TableName> disabledTables,
boolean excludeOfflinedSplitParents) {
this.tracker = tracker;
this.hConnection = hConnection;
tableToRegionMap = new HashMap<TableName, List<HRegionInfo>>();
regionToRegionServerMap = new HashMap<HRegionInfo, ServerName>();
regionServerToRegionMap = new HashMap<ServerName, List<HRegionInfo>>();
@ -95,13 +95,13 @@ public class SnapshotOfRegionAssignmentFromMeta {
public void initialize() throws IOException {
LOG.info("Start to scan the hbase:meta for the current region assignment " +
"snappshot");
// TODO: at some point this code could live in the MetaReader
// TODO: at some point this code could live in the MetaTableAccessor
Visitor v = new Visitor() {
@Override
public boolean visit(Result result) throws IOException {
try {
if (result == null || result.isEmpty()) return true;
RegionLocations rl = MetaReader.getRegionLocations(result);
RegionLocations rl = MetaTableAccessor.getRegionLocations(result);
if (rl == null) return true;
HRegionInfo hri = rl.getRegionLocation(0).getRegionInfo();
if (hri == null) return true;
@ -141,7 +141,7 @@ public class SnapshotOfRegionAssignmentFromMeta {
}
};
// Scan hbase:meta to pick up user regions
MetaReader.fullScan(tracker, v);
MetaTableAccessor.fullScan(hConnection, v);
//regionToRegionServerMap = regions;
LOG.info("Finished to scan the hbase:meta for the current region assignment" +
"snapshot");

View File

@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.NamespaceExistException;
import org.apache.hadoop.hbase.NamespaceNotFoundException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZKNamespaceManager;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
@ -80,7 +80,7 @@ public class TableNamespaceManager {
}
public void start() throws IOException {
if (!MetaReader.tableExists(masterServices.getCatalogTracker(),
if (!MetaTableAccessor.tableExists(masterServices.getShortCircuitConnection(),
TableName.NAMESPACE_TABLE_NAME)) {
LOG.info("Namespace table not found. Creating...");
createNamespaceTable(masterServices);

View File

@ -1069,7 +1069,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
* Subclasses should implement this to return true if the cluster has nodes that hosts
* multiple replicas for the same region, or, if there are multiple racks and the same
* rack hosts replicas of the same region
* @param c
* @param c Cluster information
* @return whether region replicas are currently co-located
*/
protected boolean areSomeRegionReplicasColocated(Cluster c) {

View File

@ -37,8 +37,8 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.master.RackManager;
@ -85,13 +85,13 @@ public class FavoredNodeAssignmentHelper {
/**
* Update meta table with favored nodes info
* @param regionToFavoredNodes
* @param catalogTracker
* @param regionToFavoredNodes map of HRegionInfo's to their favored nodes
* @param hConnection HConnection to be used
* @throws IOException
*/
public static void updateMetaWithFavoredNodesInfo(
Map<HRegionInfo, List<ServerName>> regionToFavoredNodes,
CatalogTracker catalogTracker) throws IOException {
HConnection hConnection) throws IOException {
List<Put> puts = new ArrayList<Put>();
for (Map.Entry<HRegionInfo, List<ServerName>> entry : regionToFavoredNodes.entrySet()) {
Put put = makePutFromRegionInfo(entry.getKey(), entry.getValue());
@ -99,7 +99,7 @@ public class FavoredNodeAssignmentHelper {
puts.add(put);
}
}
MetaEditor.putsToMetaTable(catalogTracker, puts);
MetaTableAccessor.putsToMetaTable(hConnection, puts);
LOG.info("Added " + puts.size() + " regions in META");
}
@ -141,7 +141,7 @@ public class FavoredNodeAssignmentHelper {
throws IOException {
Put put = null;
if (favoredNodeList != null) {
put = MetaEditor.makePutFromRegionInfo(regionInfo);
put = MetaTableAccessor.makePutFromRegionInfo(regionInfo);
byte[] favoredNodes = getFavoredNodes(favoredNodeList);
put.addImmutable(HConstants.CATALOG_FAMILY, FAVOREDNODES_QUALIFIER,
EnvironmentEdgeManager.currentTimeMillis(), favoredNodes);

View File

@ -74,7 +74,7 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer {
List<RegionPlan> plans = new ArrayList<RegionPlan>();
//perform a scan of the meta to get the latest updates (if any)
SnapshotOfRegionAssignmentFromMeta snaphotOfRegionAssignment =
new SnapshotOfRegionAssignmentFromMeta(super.services.getCatalogTracker());
new SnapshotOfRegionAssignmentFromMeta(super.services.getShortCircuitConnection());
try {
snaphotOfRegionAssignment.initialize();
} catch (IOException ie) {

View File

@ -36,10 +36,8 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.AssignmentManager;
@ -64,7 +62,6 @@ public class CreateTableHandler extends EventHandler {
protected final HTableDescriptor hTableDescriptor;
protected final Configuration conf;
private final AssignmentManager assignmentManager;
private final CatalogTracker catalogTracker;
private final TableLockManager tableLockManager;
private final HRegionInfo [] newRegions;
private final TableLock tableLock;
@ -78,7 +75,6 @@ public class CreateTableHandler extends EventHandler {
this.hTableDescriptor = hTableDescriptor;
this.conf = conf;
this.newRegions = newRegions;
this.catalogTracker = masterServices.getCatalogTracker();
this.assignmentManager = masterServices.getAssignmentManager();
this.tableLockManager = masterServices.getTableLockManager();
@ -92,7 +88,8 @@ public class CreateTableHandler extends EventHandler {
int timeout = conf.getInt("hbase.client.catalog.timeout", 10000);
// Need hbase:meta availability to create a table
try {
if (catalogTracker.waitForMeta(timeout) == null) {
if (server.getMetaTableLocator().waitMetaRegionLocation(
server.getZooKeeper(), timeout) == null) {
throw new NotAllMetaRegionsOnlineException();
}
} catch (InterruptedException e) {
@ -107,7 +104,7 @@ public class CreateTableHandler extends EventHandler {
boolean success = false;
try {
TableName tableName = this.hTableDescriptor.getTableName();
if (MetaReader.tableExists(catalogTracker, tableName)) {
if (MetaTableAccessor.tableExists(this.server.getShortCircuitConnection(), tableName)) {
throw new TableExistsException(tableName);
}
@ -242,7 +239,7 @@ public class CreateTableHandler extends EventHandler {
if (regionInfos != null && regionInfos.size() > 0) {
// 4. Add regions to META
addRegionsToMeta(this.catalogTracker, regionInfos);
addRegionsToMeta(regionInfos);
// 5. Add replicas if needed
regionInfos = addReplicas(hTableDescriptor, regionInfos);
@ -310,8 +307,8 @@ public class CreateTableHandler extends EventHandler {
/**
* Add the specified set of regions to the hbase:meta table.
*/
protected void addRegionsToMeta(final CatalogTracker ct, final List<HRegionInfo> regionInfos)
protected void addRegionsToMeta(final List<HRegionInfo> regionInfos)
throws IOException {
MetaEditor.addRegionsToMeta(this.catalogTracker, regionInfos);
MetaTableAccessor.addRegionsToMeta(this.server.getShortCircuitConnection(), regionInfos);
}
}

View File

@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.backup.HFileArchiver;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.master.AssignmentManager;
@ -133,7 +133,7 @@ public class DeleteTableHandler extends TableEventHandler {
throws IOException, CoordinatedStateException {
// 1. Remove regions from META
LOG.debug("Deleting regions from META");
MetaEditor.deleteRegions(this.server.getCatalogTracker(), regions);
MetaTableAccessor.deleteRegions(this.server.getShortCircuitConnection(), regions);
// -----------------------------------------------------------------------
// NOTE: At this point we still have data on disk, but nothing in hbase:meta

View File

@ -31,8 +31,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.TableNotEnabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.executor.EventType;
@ -56,17 +55,15 @@ public class DisableTableHandler extends EventHandler {
private final TableName tableName;
private final AssignmentManager assignmentManager;
private final TableLockManager tableLockManager;
private final CatalogTracker catalogTracker;
private final boolean skipTableStateCheck;
private TableLock tableLock;
public DisableTableHandler(Server server, TableName tableName,
CatalogTracker catalogTracker, AssignmentManager assignmentManager,
TableLockManager tableLockManager, boolean skipTableStateCheck) {
AssignmentManager assignmentManager, TableLockManager tableLockManager,
boolean skipTableStateCheck) {
super(server, EventType.C_M_DISABLE_TABLE);
this.tableName = tableName;
this.assignmentManager = assignmentManager;
this.catalogTracker = catalogTracker;
this.tableLockManager = tableLockManager;
this.skipTableStateCheck = skipTableStateCheck;
}
@ -84,7 +81,7 @@ public class DisableTableHandler extends EventHandler {
boolean success = false;
try {
// Check if table exists
if (!MetaReader.tableExists(catalogTracker, tableName)) {
if (!MetaTableAccessor.tableExists(this.server.getShortCircuitConnection(), tableName)) {
throw new TableNotFoundException(tableName);
}

View File

@ -34,8 +34,7 @@ import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.AssignmentManager;
@ -60,26 +59,24 @@ public class EnableTableHandler extends EventHandler {
private final TableName tableName;
private final AssignmentManager assignmentManager;
private final TableLockManager tableLockManager;
private final CatalogTracker catalogTracker;
private boolean skipTableStateCheck = false;
private TableLock tableLock;
private MasterServices services;
public EnableTableHandler(Server server, TableName tableName,
CatalogTracker catalogTracker, AssignmentManager assignmentManager,
TableLockManager tableLockManager, boolean skipTableStateCheck) {
AssignmentManager assignmentManager, TableLockManager tableLockManager,
boolean skipTableStateCheck) {
super(server, EventType.C_M_ENABLE_TABLE);
this.tableName = tableName;
this.catalogTracker = catalogTracker;
this.assignmentManager = assignmentManager;
this.tableLockManager = tableLockManager;
this.skipTableStateCheck = skipTableStateCheck;
}
public EnableTableHandler(MasterServices services, TableName tableName,
CatalogTracker catalogTracker, AssignmentManager assignmentManager,
AssignmentManager assignmentManager,
TableLockManager tableLockManager, boolean skipTableStateCheck) {
this((Server)services, tableName, catalogTracker, assignmentManager, tableLockManager,
this((Server)services, tableName, assignmentManager, tableLockManager,
skipTableStateCheck);
this.services = services;
}
@ -94,7 +91,7 @@ public class EnableTableHandler extends EventHandler {
boolean success = false;
try {
// Check if table exists
if (!MetaReader.tableExists(catalogTracker, tableName)) {
if (!MetaTableAccessor.tableExists(this.server.getShortCircuitConnection(), tableName)) {
// retainAssignment is true only during recovery. In normal case it is false
if (!this.skipTableStateCheck) {
throw new TableNotFoundException(tableName);
@ -192,8 +189,9 @@ public class EnableTableHandler extends EventHandler {
ServerManager serverManager = ((HMaster)this.server).getServerManager();
// Get the regions of this table. We're done when all listed
// tables are onlined.
List<Pair<HRegionInfo, ServerName>> tableRegionsAndLocations = MetaReader
.getTableRegionsAndLocations(this.catalogTracker, tableName, true);
List<Pair<HRegionInfo, ServerName>> tableRegionsAndLocations = MetaTableAccessor
.getTableRegionsAndLocations(this.server.getZooKeeper(),
this.server.getShortCircuitConnection(), tableName, true);
int countOfRegionsInTable = tableRegionsAndLocations.size();
Map<HRegionInfo, ServerName> regionsToAssign =
regionsToAssignWithServerName(tableRegionsAndLocations);

View File

@ -91,7 +91,7 @@ public class MetaServerShutdownHandler extends ServerShutdownHandler {
LOG.info("Server " + serverName + " was carrying META. Trying to assign.");
am.regionOffline(HRegionInfo.FIRST_META_REGIONINFO);
verifyAndAssignMetaWithRetries();
} else if (!this.services.getCatalogTracker().isMetaLocationAvailable()) {
} else if (!server.getMetaTableLocator().isLocationAvailable(this.server.getZooKeeper())) {
// the meta location as per master is null. This could happen in case when meta assignment
// in previous run failed, while meta znode has been updated to null. We should try to
// assign the meta again.
@ -154,14 +154,16 @@ public class MetaServerShutdownHandler extends ServerShutdownHandler {
throws InterruptedException, IOException, KeeperException {
long timeout = this.server.getConfiguration().
getLong("hbase.catalog.verification.timeout", 1000);
if (!this.server.getCatalogTracker().verifyMetaRegionLocation(timeout)) {
if (!server.getMetaTableLocator().verifyMetaRegionLocation(server.getShortCircuitConnection(),
this.server.getZooKeeper(), timeout)) {
this.services.getAssignmentManager().assignMeta();
} else if (serverName.equals(server.getCatalogTracker().getMetaLocation())) {
} else if (serverName.equals(server.getMetaTableLocator().getMetaRegionLocation(
this.server.getZooKeeper()))) {
throw new IOException("hbase:meta is onlined on the dead server "
+ serverName);
} else {
LOG.info("Skip assigning hbase:meta, because it is online on the "
+ server.getCatalogTracker().getMetaLocation());
+ server.getMetaTableLocator().getMetaRegionLocation(this.server.getZooKeeper()));
}
}

View File

@ -31,8 +31,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
@ -93,7 +92,7 @@ public class ModifyTableHandler extends TableEventHandler {
TableName table) throws IOException {
if (newReplicaCount >= oldReplicaCount) return;
Set<byte[]> tableRows = new HashSet<byte[]>();
Scan scan = MetaReader.getScanForTableName(table);
Scan scan = MetaTableAccessor.getScanForTableName(table);
scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
HTable htable = null;
try {
@ -102,8 +101,8 @@ public class ModifyTableHandler extends TableEventHandler {
for (Result result : resScanner) {
tableRows.add(result.getRow());
}
MetaEditor.removeRegionReplicasFromMeta(tableRows, newReplicaCount,
oldReplicaCount - newReplicaCount, masterServices.getCatalogTracker());
MetaTableAccessor.removeRegionReplicasFromMeta(tableRows, newReplicaCount,
oldReplicaCount - newReplicaCount, masterServices.getShortCircuitConnection());
} finally {
if (htable != null) {
htable.close();

View File

@ -32,8 +32,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.AssignmentManager;
@ -144,7 +143,7 @@ public class ServerShutdownHandler extends EventHandler {
// Wait on meta to come online; we need it to progress.
// TODO: Best way to hold strictly here? We should build this retry logic
// into the MetaReader operations themselves.
// into the MetaTableAccessor operations themselves.
// TODO: Is the reading of hbase:meta necessary when the Master has state of
// cluster in its head? It should be possible to do without reading hbase:meta
// in all but one case. On split, the RS updates the hbase:meta
@ -160,11 +159,11 @@ public class ServerShutdownHandler extends EventHandler {
Set<HRegionInfo> hris = null;
while (!this.server.isStopped()) {
try {
this.server.getCatalogTracker().waitForMeta();
server.getMetaTableLocator().waitMetaRegionLocation(server.getZooKeeper());
// Skip getting user regions if the server is stopped.
if (!this.server.isStopped()) {
if (ConfigUtil.useZKForAssignment(server.getConfiguration())) {
hris = MetaReader.getServerUserRegions(this.server.getCatalogTracker(),
hris = MetaTableAccessor.getServerUserRegions(this.server.getShortCircuitConnection(),
this.serverName).keySet();
} else {
// Not using ZK for assignment, regionStates has everything we want
@ -235,7 +234,7 @@ public class ServerShutdownHandler extends EventHandler {
Lock lock = am.acquireRegionLock(encodedName);
try {
RegionState rit = regionStates.getRegionTransitionState(hri);
if (processDeadRegion(hri, am, server.getCatalogTracker())) {
if (processDeadRegion(hri, am)) {
ServerName addressFromAM = regionStates.getRegionServerOfRegion(hri);
if (addressFromAM != null && !addressFromAM.equals(this.serverName)) {
// If this region is in transition on the dead server, it must be
@ -336,12 +335,11 @@ public class ServerShutdownHandler extends EventHandler {
* disabling or if the region has a partially completed split.
* @param hri
* @param assignmentManager
* @param catalogTracker
* @return Returns true if specified region should be assigned, false if not.
* @throws IOException
*/
public static boolean processDeadRegion(HRegionInfo hri,
AssignmentManager assignmentManager, CatalogTracker catalogTracker)
AssignmentManager assignmentManager)
throws IOException {
boolean tablePresent = assignmentManager.getTableStateManager().isTablePresent(hri.getTable());
if (!tablePresent) {

View File

@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.executor.EventType;
@ -125,8 +125,8 @@ public abstract class TableEventHandler extends EventHandler {
tableName);
List<HRegionInfo> hris =
MetaReader.getTableRegions(this.server.getCatalogTracker(),
tableName);
MetaTableAccessor.getTableRegions(this.server.getZooKeeper(),
this.server.getShortCircuitConnection(), tableName);
handleTableOperation(hris);
if (eventType.isOnlineSchemaChangeSupported() && this.masterServices.
getAssignmentManager().getTableStateManager().isTableState(

View File

@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.CoordinatedStateException;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
@ -124,7 +124,8 @@ public class TruncateTableHandler extends DeleteTableHandler {
}
// 4. Add regions to META
MetaEditor.addRegionsToMeta(masterServices.getCatalogTracker(), regionInfos);
MetaTableAccessor.addRegionsToMeta(masterServices.getShortCircuitConnection(),
regionInfos);
// 5. Trigger immediate assignment of the regions in round-robin fashion
ModifyRegionUtils.assignRegions(assignmentManager, regionInfos);

View File

@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.master.MasterServices;
@ -139,10 +138,10 @@ public class CloneSnapshotHandler extends CreateTableHandler implements Snapshot
}
@Override
protected void addRegionsToMeta(final CatalogTracker ct, final List<HRegionInfo> regionInfos)
protected void addRegionsToMeta(final List<HRegionInfo> regionInfos)
throws IOException {
super.addRegionsToMeta(ct, regionInfos);
metaChanges.updateMetaParentRegions(ct, regionInfos);
super.addRegionsToMeta(regionInfos);
metaChanges.updateMetaParentRegions(this.server.getShortCircuitConnection(), regionInfos);
}
@Override

View File

@ -32,8 +32,8 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
@ -150,8 +150,8 @@ public final class MasterSnapshotVerifier {
* @throws IOException if we can't reach hbase:meta or read the files from the FS
*/
private void verifyRegions(final SnapshotManifest manifest) throws IOException {
List<HRegionInfo> regions = MetaReader.getTableRegions(this.services.getCatalogTracker(),
tableName);
List<HRegionInfo> regions = MetaTableAccessor.getTableRegions(
this.services.getZooKeeper(), this.services.getShortCircuitConnection(), tableName);
// Remove the non-default regions
RegionReplicaUtil.removeNonDefaultRegions(regions);

View File

@ -32,8 +32,8 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.executor.EventType;
@ -109,7 +109,7 @@ public class RestoreSnapshotHandler extends TableEventHandler implements Snapsho
@Override
protected void handleTableOperation(List<HRegionInfo> hris) throws IOException {
MasterFileSystem fileSystemManager = masterServices.getMasterFileSystem();
CatalogTracker catalogTracker = masterServices.getCatalogTracker();
HConnection conn = masterServices.getShortCircuitConnection();
FileSystem fs = fileSystemManager.getFileSystem();
Path rootDir = fileSystemManager.getRootDir();
TableName tableName = hTableDescriptor.getTableName();
@ -149,7 +149,7 @@ public class RestoreSnapshotHandler extends TableEventHandler implements Snapsho
// that are not correct after the restore.
List<HRegionInfo> hrisToRemove = new LinkedList<HRegionInfo>();
if (metaChanges.hasRegionsToRemove()) hrisToRemove.addAll(metaChanges.getRegionsToRemove());
MetaEditor.deleteRegions(catalogTracker, hrisToRemove);
MetaTableAccessor.deleteRegions(conn, hrisToRemove);
// 4.2 Add the new set of regions to META
//
@ -159,11 +159,11 @@ public class RestoreSnapshotHandler extends TableEventHandler implements Snapsho
// in the snapshot folder.
hris.clear();
if (metaChanges.hasRegionsToAdd()) hris.addAll(metaChanges.getRegionsToAdd());
MetaEditor.addRegionsToMeta(catalogTracker, hris);
MetaTableAccessor.addRegionsToMeta(conn, hris);
if (metaChanges.hasRegionsToRestore()) {
MetaEditor.overwriteRegions(catalogTracker, metaChanges.getRegionsToRestore());
MetaTableAccessor.overwriteRegions(conn, metaChanges.getRegionsToRestore());
}
metaChanges.updateMetaParentRegions(catalogTracker, hris);
metaChanges.updateMetaParentRegions(this.server.getShortCircuitConnection(), hris);
// At this point the restore is complete. Next step is enabling the table.
LOG.info("Restore snapshot=" + ClientSnapshotDescriptionUtils.toString(snapshot) +

View File

@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.master.AssignmentManager;
@ -694,7 +694,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
SnapshotReferenceUtil.verifySnapshot(master.getConfiguration(), fs, manifest);
// Execute the restore/clone operation
if (MetaReader.tableExists(master.getCatalogTracker(), tableName)) {
if (MetaTableAccessor.tableExists(master.getShortCircuitConnection(), tableName)) {
if (master.getAssignmentManager().getTableStateManager().isTableState(
TableName.valueOf(fsSnapshot.getTable()), ZooKeeperProtos.Table.State.ENABLED)) {
throw new UnsupportedOperationException("Table '" +

View File

@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
@ -168,8 +168,8 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
monitor.rethrowException();
List<Pair<HRegionInfo, ServerName>> regionsAndLocations =
MetaReader.getTableRegionsAndLocations(this.server.getCatalogTracker(),
snapshotTable, false);
MetaTableAccessor.getTableRegionsAndLocations(this.server.getZooKeeper(),
this.server.getShortCircuitConnection(), snapshotTable, false);
// run the snapshot
snapshotRegions(regionsAndLocations);

View File

@ -40,7 +40,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
@ -431,8 +431,8 @@ public class NamespaceUpgrade implements Tool {
newRegionDir);
}
}
meta.put(MetaEditor.makePutFromRegionInfo(newRegionInfo));
meta.delete(MetaEditor.makeDeleteFromRegionInfo(oldRegionInfo));
meta.put(MetaTableAccessor.makePutFromRegionInfo(newRegionInfo));
meta.delete(MetaTableAccessor.makeDeleteFromRegionInfo(oldRegionInfo));
}
} finally {
meta.flushcache();

View File

@ -30,7 +30,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
@ -124,7 +124,8 @@ public class MasterFlushTableProcedureManager extends MasterProcedureManager {
List<Pair<HRegionInfo, ServerName>> regionsAndLocations = null;
try {
regionsAndLocations =
MetaReader.getTableRegionsAndLocations(this.master.getCatalogTracker(),
MetaTableAccessor.getTableRegionsAndLocations(this.master.getZooKeeper(),
this.master.getShortCircuitConnection(),
TableName.valueOf(desc.getInstance()), false);
} catch (InterruptedException e1) {
String msg = "Failed to get regions for '" + desc.getInstance() + "'";

View File

@ -72,8 +72,7 @@ import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.YouAreDeadException;
import org.apache.hadoop.hbase.ZNodeClearer;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.ConnectionUtils;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
@ -135,7 +134,7 @@ 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.MetaRegionTracker;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.RecoveringRegionWatcher;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@ -182,8 +181,20 @@ public class HRegionServer extends HasThread implements
protected HeapMemoryManager hMemManager;
// catalog tracker
protected CatalogTracker catalogTracker;
/*
* Short-circuit (ie. bypassing RPC layer) HConnection to this Server
* to be used internally for miscellaneous needs. Initialized at the server startup
* and closed when server shuts down. Clients must never close it explicitly.
*/
protected HConnection shortCircuitConnection;
/*
* Long-living meta table locator, which is created when the server is started and stopped
* when server shuts down. References to this locator shall be used to perform according
* operations in EventHandlers. Primary reason for this decision is to make it mockable
* for tests.
*/
protected MetaTableLocator metaTableLocator;
// Watch if a region is out of recovering state from ZooKeeper
@SuppressWarnings("unused")
@ -543,14 +554,13 @@ public class HRegionServer extends HasThread implements
}
/**
* Create CatalogTracker.
* Create wrapped short-circuit connection to this server.
* In its own method so can intercept and mock it over in tests.
* @throws IOException
*/
protected CatalogTracker createCatalogTracker() throws IOException {
HConnection conn = ConnectionUtils.createShortCircuitHConnection(
protected HConnection createShortCircuitConnection() throws IOException {
return ConnectionUtils.createShortCircuitHConnection(
HConnectionManager.getConnection(conf), serverName, rpcServices, rpcServices);
return new CatalogTracker(zooKeeper, conf, conn, this);
}
/**
@ -596,7 +606,7 @@ public class HRegionServer extends HasThread implements
* Bring up connection to zk ensemble and then wait until a master for this
* cluster and then after that, wait until cluster 'up' flag has been set.
* This is the order in which master does things.
* Finally put up a catalog tracker.
* Finally open long-living server short-circuit connection.
* @throws IOException
* @throws InterruptedException
*/
@ -625,8 +635,8 @@ public class HRegionServer extends HasThread implements
this.abort("Failed to retrieve Cluster ID",e);
}
// Now we have the cluster ID, start catalog tracker
startCatalogTracker();
shortCircuitConnection = createShortCircuitConnection();
metaTableLocator = new MetaTableLocator();
// watch for snapshots and other procedures
try {
@ -704,17 +714,6 @@ public class HRegionServer extends HasThread implements
}
}
/**
* Create and start the catalog tracker if not already done.
*/
protected synchronized void startCatalogTracker()
throws IOException, InterruptedException {
if (catalogTracker == null) {
catalogTracker = createCatalogTracker();
catalogTracker.start();
}
}
/**
* The HRegionServer sticks in this loop until closed.
*/
@ -859,9 +858,18 @@ public class HRegionServer extends HasThread implements
closeUserRegions(abortRequested);
LOG.info("stopping server " + this.serverName);
}
// Interrupt catalog tracker here in case any regions being opened out in
// handlers are stuck waiting on meta.
if (this.catalogTracker != null) this.catalogTracker.stop();
// so callers waiting for meta without timeout can stop
metaTableLocator.stop();
if (this.shortCircuitConnection != null && !shortCircuitConnection.isClosed()) {
try {
this.shortCircuitConnection.close();
} catch (IOException e) {
// Although the {@link Closeable} interface throws an {@link
// IOException}, in reality, the implementation would never do that.
LOG.error("Attempt to close server's short circuit HConnection failed.", e);
}
}
// Closing the compactSplit thread before closing meta regions
if (!this.killed && containsMetaTableRegions()) {
@ -1644,8 +1652,13 @@ public class HRegionServer extends HasThread implements
}
@Override
public CatalogTracker getCatalogTracker() {
return this.catalogTracker;
public HConnection getShortCircuitConnection() {
return this.shortCircuitConnection;
}
@Override
public MetaTableLocator getMetaTableLocator() {
return this.metaTableLocator;
}
@Override
@ -1672,7 +1685,7 @@ public class HRegionServer extends HasThread implements
}
@Override
public void postOpenDeployTasks(final HRegion r, final CatalogTracker ct)
public void postOpenDeployTasks(final HRegion r)
throws KeeperException, IOException {
rpcServices.checkOpen();
LOG.info("Post open deploy tasks for " + r.getRegionNameAsString());
@ -1694,9 +1707,9 @@ public class HRegionServer extends HasThread implements
// Update ZK, or META
if (r.getRegionInfo().isMetaRegion()) {
MetaRegionTracker.setMetaLocation(getZooKeeper(), serverName);
MetaTableLocator.setMetaLocation(getZooKeeper(), serverName);
} else if (useZKForAssignment) {
MetaEditor.updateRegionLocation(ct, r.getRegionInfo(),
MetaTableAccessor.updateRegionLocation(shortCircuitConnection, r.getRegionInfo(),
this.serverName, openSeqNum);
}
if (!useZKForAssignment && !reportRegionStateTransition(

View File

@ -52,7 +52,7 @@ import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.UnknownScannerException;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.ConnectionUtils;
import org.apache.hadoop.hbase.client.Delete;
@ -1206,8 +1206,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
}
// See HBASE-5094. Cross check with hbase:meta if still this RS is owning
// the region.
Pair<HRegionInfo, ServerName> p = MetaReader.getRegion(
regionServer.catalogTracker, region.getRegionName());
Pair<HRegionInfo, ServerName> p = MetaTableAccessor.getRegion(
regionServer.getShortCircuitConnection(), region.getRegionName());
if (regionServer.serverName.equals(p.getSecond())) {
Boolean closing = regionServer.regionsInTransitionInRS.get(region.getEncodedNameAsBytes());
// Map regionsInTransitionInRSOnly has an entry for a region only if the region

View File

@ -33,10 +33,9 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaMutationAnnotation;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
@ -325,11 +324,13 @@ public class RegionMergeTransaction {
// rollback
if (!testing && useCoordinationForAssignment) {
if (metaEntries.isEmpty()) {
MetaEditor.mergeRegions(server.getCatalogTracker(), mergedRegion.getRegionInfo(), region_a
.getRegionInfo(), region_b.getRegionInfo(), server.getServerName());
MetaTableAccessor.mergeRegions(server.getShortCircuitConnection(),
mergedRegion.getRegionInfo(), region_a.getRegionInfo(), region_b.getRegionInfo(),
server.getServerName());
} else {
mergeRegionsAndPutMetaEntries(server.getCatalogTracker(), mergedRegion.getRegionInfo(),
region_a.getRegionInfo(), region_b.getRegionInfo(), server.getServerName(), metaEntries);
mergeRegionsAndPutMetaEntries(server.getShortCircuitConnection(),
mergedRegion.getRegionInfo(), region_a.getRegionInfo(), region_b.getRegionInfo(),
server.getServerName(), metaEntries);
}
} else if (services != null && !useCoordinationForAssignment) {
if (!services.reportRegionStateTransition(TransitionCode.MERGE_PONR,
@ -343,11 +344,11 @@ public class RegionMergeTransaction {
return mergedRegion;
}
private void mergeRegionsAndPutMetaEntries(CatalogTracker catalogTracker,
HRegionInfo mergedRegion, HRegionInfo regionA, HRegionInfo regionB, ServerName serverName,
List<Mutation> metaEntries) throws IOException {
private void mergeRegionsAndPutMetaEntries(HConnection hConnection,
HRegionInfo mergedRegion, HRegionInfo regionA, HRegionInfo regionB,
ServerName serverName, List<Mutation> metaEntries) throws IOException {
prepareMutationsForMerge(mergedRegion, regionA, regionB, serverName, metaEntries);
MetaEditor.mutateMetaTable(catalogTracker, metaEntries);
MetaTableAccessor.mutateMetaTable(hConnection, metaEntries);
}
public void prepareMutationsForMerge(HRegionInfo mergedRegion, HRegionInfo regionA,
@ -355,13 +356,13 @@ public class RegionMergeTransaction {
HRegionInfo copyOfMerged = new HRegionInfo(mergedRegion);
// Put for parent
Put putOfMerged = MetaEditor.makePutFromRegionInfo(copyOfMerged);
Put putOfMerged = MetaTableAccessor.makePutFromRegionInfo(copyOfMerged);
putOfMerged.add(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER, regionA.toByteArray());
putOfMerged.add(HConstants.CATALOG_FAMILY, HConstants.MERGEB_QUALIFIER, regionB.toByteArray());
mutations.add(putOfMerged);
// Deletes for merging regions
Delete deleteA = MetaEditor.makeDeleteFromRegionInfo(regionA);
Delete deleteB = MetaEditor.makeDeleteFromRegionInfo(regionB);
Delete deleteA = MetaTableAccessor.makeDeleteFromRegionInfo(regionA);
Delete deleteB = MetaTableAccessor.makeDeleteFromRegionInfo(regionB);
mutations.add(deleteA);
mutations.add(deleteB);
// The merged is a new region, openSeqNum = 1 is fine.
@ -579,7 +580,7 @@ public class RegionMergeTransaction {
if (services != null) {
try {
if (useCoordinationForAssignment) {
services.postOpenDeployTasks(merged, server.getCatalogTracker());
services.postOpenDeployTasks(merged);
} else if (!services.reportRegionStateTransition(TransitionCode.MERGED,
mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo())) {
throw new IOException("Failed to report merged region to master: "
@ -753,8 +754,8 @@ public class RegionMergeTransaction {
if (services == null) return false;
// Get merge regions if it is a merged region and already has merge
// qualifier
Pair<HRegionInfo, HRegionInfo> mergeRegions = MetaReader
.getRegionsFromMergeQualifier(services.getCatalogTracker(), regionName);
Pair<HRegionInfo, HRegionInfo> mergeRegions = MetaTableAccessor
.getRegionsFromMergeQualifier(services.getShortCircuitConnection(), regionName);
if (mergeRegions != null &&
(mergeRegions.getFirst() != null || mergeRegions.getSecond() != null)) {
// It has merge qualifier

View File

@ -25,7 +25,6 @@ import java.util.concurrent.ConcurrentMap;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.ipc.RpcServerInterface;
import org.apache.hadoop.hbase.master.TableLockManager;
@ -73,11 +72,10 @@ public interface RegionServerServices
* regionserver
*
* @param r Region to open.
* @param ct Instance of {@link CatalogTracker}
* @throws KeeperException
* @throws IOException
*/
void postOpenDeployTasks(final HRegion r, final CatalogTracker ct)
void postOpenDeployTasks(final HRegion r)
throws KeeperException, IOException;
/**
@ -116,11 +114,6 @@ public interface RegionServerServices
*/
ExecutorService getExecutorService();
/**
* @return The RegionServer's CatalogTracker
*/
CatalogTracker getCatalogTracker();
/**
* @return set of recovering regions on the hosting region server
*/

View File

@ -39,8 +39,8 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
@ -277,11 +277,11 @@ public class SplitTransaction {
// and assign the parent region.
if (!testing && useZKForAssignment) {
if (metaEntries == null || metaEntries.isEmpty()) {
MetaEditor.splitRegion(server.getCatalogTracker(),
MetaTableAccessor.splitRegion(server.getShortCircuitConnection(),
parent.getRegionInfo(), daughterRegions.getFirst().getRegionInfo(),
daughterRegions.getSecond().getRegionInfo(), server.getServerName());
} else {
offlineParentInMetaAndputMetaEntries(server.getCatalogTracker(),
offlineParentInMetaAndputMetaEntries(server.getShortCircuitConnection(),
parent.getRegionInfo(), daughterRegions.getFirst().getRegionInfo(), daughterRegions
.getSecond().getRegionInfo(), server.getServerName(), metaEntries);
}
@ -418,7 +418,7 @@ public class SplitTransaction {
try {
if (useZKForAssignment) {
// add 2nd daughter first (see HBASE-4335)
services.postOpenDeployTasks(b, server.getCatalogTracker());
services.postOpenDeployTasks(b);
} else if (!services.reportRegionStateTransition(TransitionCode.SPLIT,
parent.getRegionInfo(), hri_a, hri_b)) {
throw new IOException("Failed to report split region to master: "
@ -427,7 +427,7 @@ public class SplitTransaction {
// Should add it to OnlineRegions
services.addToOnlineRegions(b);
if (useZKForAssignment) {
services.postOpenDeployTasks(a, server.getCatalogTracker());
services.postOpenDeployTasks(a);
}
services.addToOnlineRegions(a);
} catch (KeeperException ke) {
@ -482,7 +482,7 @@ public class SplitTransaction {
return regions;
}
private void offlineParentInMetaAndputMetaEntries(CatalogTracker catalogTracker,
private void offlineParentInMetaAndputMetaEntries(HConnection hConnection,
HRegionInfo parent, HRegionInfo splitA, HRegionInfo splitB,
ServerName serverName, List<Mutation> metaEntries) throws IOException {
List<Mutation> mutations = metaEntries;
@ -491,19 +491,19 @@ public class SplitTransaction {
copyOfParent.setSplit(true);
//Put for parent
Put putParent = MetaEditor.makePutFromRegionInfo(copyOfParent);
MetaEditor.addDaughtersToPut(putParent, splitA, splitB);
Put putParent = MetaTableAccessor.makePutFromRegionInfo(copyOfParent);
MetaTableAccessor.addDaughtersToPut(putParent, splitA, splitB);
mutations.add(putParent);
//Puts for daughters
Put putA = MetaEditor.makePutFromRegionInfo(splitA);
Put putB = MetaEditor.makePutFromRegionInfo(splitB);
Put putA = MetaTableAccessor.makePutFromRegionInfo(splitA);
Put putB = MetaTableAccessor.makePutFromRegionInfo(splitB);
addLocation(putA, serverName, 1); //these are new regions, openSeqNum = 1 is fine.
addLocation(putB, serverName, 1);
mutations.add(putA);
mutations.add(putB);
MetaEditor.mutateMetaTable(catalogTracker, mutations);
MetaTableAccessor.mutateMetaTable(hConnection, mutations);
}
public Put addLocation(final Put p, final ServerName sn, long openSeqNum) {

View File

@ -297,9 +297,7 @@ public class OpenRegionHandler extends EventHandler {
/**
* Thread to run region post open tasks. Call {@link #getException()} after
* the thread finishes to check for exceptions running
* {@link RegionServerServices#postOpenDeployTasks(
* HRegion, org.apache.hadoop.hbase.catalog.CatalogTracker)}
* .
* {@link RegionServerServices#postOpenDeployTasks(HRegion)
*/
static class PostOpenDeployTasksThread extends Thread {
private Throwable exception = null;
@ -320,8 +318,7 @@ public class OpenRegionHandler extends EventHandler {
public void run() {
try {
this.services.postOpenDeployTasks(this.region,
this.server.getCatalogTracker());
this.services.postOpenDeployTasks(this.region);
} catch (IOException e) {
server.abort("Exception running postOpenDeployTasks; region=" +
this.region.getRegionInfo().getEncodedName(), e);

View File

@ -28,9 +28,10 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
@ -149,7 +150,12 @@ public class ReplicationSyncUp extends Configured implements Tool {
}
@Override
public CatalogTracker getCatalogTracker() {
public HConnection getShortCircuitConnection() {
return null;
}
@Override
public MetaTableLocator getMetaTableLocator() {
return null;
}

View File

@ -48,7 +48,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability;
@ -1117,8 +1117,8 @@ public class AccessController extends BaseRegionObserver
@Override
public void postStartMaster(ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
if (!MetaReader.tableExists(ctx.getEnvironment().getMasterServices().getCatalogTracker(),
AccessControlLists.ACL_TABLE_NAME)) {
if (!MetaTableAccessor.tableExists(ctx.getEnvironment().getMasterServices()
.getShortCircuitConnection(), AccessControlLists.ACL_TABLE_NAME)) {
// initialize the ACL storage table
AccessControlLists.init(ctx.getEnvironment().getMasterServices());
} else {

View File

@ -55,7 +55,7 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Mutation;
@ -218,7 +218,7 @@ public class VisibilityController extends BaseRegionObserver implements MasterOb
public void postStartMaster(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
// Need to create the new system table for labels here
MasterServices master = ctx.getEnvironment().getMasterServices();
if (!MetaReader.tableExists(master.getCatalogTracker(), LABELS_TABLE_NAME)) {
if (!MetaTableAccessor.tableExists(master.getShortCircuitConnection(), LABELS_TABLE_NAME)) {
HTableDescriptor labelsTable = new HTableDescriptor(LABELS_TABLE_NAME);
HColumnDescriptor labelsColumn = new HColumnDescriptor(LABELS_TABLE_FAMILY);
labelsColumn.setBloomFilterType(BloomType.NONE);

View File

@ -19,7 +19,6 @@
package org.apache.hadoop.hbase.snapshot;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.ArrayList;
@ -45,8 +44,8 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.backup.HFileArchiver;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.io.Reference;
@ -58,7 +57,6 @@ import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionM
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
@ -253,7 +251,7 @@ public class RestoreSnapshotHelper {
/**
* Returns the list of new regions added during the on-disk restore.
* The caller is responsible to add the regions to META.
* e.g MetaEditor.addRegionsToMeta(...)
* e.g MetaTableAccessor.addRegionsToMeta(...)
* @return the list of regions to add to META
*/
public List<HRegionInfo> getRegionsToAdd() {
@ -286,7 +284,7 @@ public class RestoreSnapshotHelper {
/**
* Returns the list of regions removed during the on-disk restore.
* The caller is responsible to remove the regions from META.
* e.g. MetaEditor.deleteRegions(...)
* e.g. MetaTableAccessor.deleteRegions(...)
* @return the list of regions to remove from META
*/
public List<HRegionInfo> getRegionsToRemove() {
@ -315,7 +313,7 @@ public class RestoreSnapshotHelper {
regionsToRestore.add(hri);
}
public void updateMetaParentRegions(final CatalogTracker catalogTracker,
public void updateMetaParentRegions(HConnection hConnection,
final List<HRegionInfo> regionInfos) throws IOException {
if (regionInfos == null || parentsMap.isEmpty()) return;
@ -346,7 +344,7 @@ public class RestoreSnapshotHelper {
}
LOG.debug("Update splits parent " + regionInfo.getEncodedName() + " -> " + daughters);
MetaEditor.addRegionToMeta(catalogTracker, regionInfo,
MetaTableAccessor.addRegionToMeta(hConnection, regionInfo,
regionsByName.get(daughters.getFirst()),
regionsByName.get(daughters.getSecond()));
}

View File

@ -71,7 +71,7 @@ import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
@ -103,7 +103,7 @@ 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.util.hbck.TableLockChecker;
import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKTableStateClientSideReader;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.io.IOUtils;
@ -1137,7 +1137,7 @@ public class HBaseFsck extends Configured {
// add the row directly to meta.
HbckInfo hi = his.iterator().next();
HRegionInfo hri = hi.getHdfsHRI(); // hi.metaEntry;
Put p = MetaEditor.makePutFromRegionInfo(hri);
Put p = MetaTableAccessor.makePutFromRegionInfo(hri);
puts.add(p);
}
}
@ -1518,7 +1518,7 @@ public class HBaseFsck extends Configured {
ZooKeeperWatcher zkw = createZooKeeperWatcher();
ServerName sn = null;
try {
sn = MetaRegionTracker.getMetaRegionLocation(zkw);
sn = new MetaTableLocator().getMetaRegionLocation(zkw);
} finally {
zkw.close();
}
@ -1618,7 +1618,7 @@ public class HBaseFsck extends Configured {
HRegionInfo hri = new HRegionInfo(hi.metaEntry);
hri.setOffline(false);
hri.setSplit(false);
Put p = MetaEditor.makePutFromRegionInfo(hri);
Put p = MetaTableAccessor.makePutFromRegionInfo(hri);
mutations.add(p);
meta.mutateRow(mutations);

View File

@ -32,7 +32,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HTable;
@ -175,7 +175,7 @@ public class HBaseFsckRepair {
public static void fixMetaHoleOnline(Configuration conf,
HRegionInfo hri) throws IOException {
HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
MetaEditor.addRegionToMeta(meta, hri);
MetaTableAccessor.addRegionToMeta(meta, hri);
meta.close();
}

View File

@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnectable;
@ -331,7 +331,7 @@ class HMerge {
}
newRegion.getRegionInfo().setOffline(true);
MetaEditor.addRegionToMeta(table, newRegion.getRegionInfo());
MetaTableAccessor.addRegionToMeta(table, newRegion.getRegionInfo());
if(LOG.isDebugEnabled()) {
LOG.debug("updated columns in row: "

View File

@ -52,7 +52,7 @@ import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.NoServerForRegionException;
@ -388,7 +388,7 @@ public class RegionSplitter {
// NOTE: createTable is synchronous on the table, but not on the regions
int onlineRegions = 0;
while (onlineRegions < splitCount) {
onlineRegions = MetaReader.getRegionCount(conf, tableName);
onlineRegions = MetaTableAccessor.getRegionCount(conf, tableName);
LOG.debug(onlineRegions + " of " + splitCount + " regions online...");
if (onlineRegions < splitCount) {
Thread.sleep(10 * 1000); // sleep

View File

@ -30,6 +30,7 @@
import="org.apache.hadoop.hbase.ServerLoad"
import="org.apache.hadoop.hbase.RegionLoad"
import="org.apache.hadoop.hbase.master.HMaster"
import="org.apache.hadoop.hbase.zookeeper.MetaTableLocator"
import="org.apache.hadoop.hbase.util.Bytes"
import="org.apache.hadoop.hbase.util.FSUtils"
import="org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest"
@ -41,6 +42,7 @@
HMaster master = (HMaster)getServletContext().getAttribute(HMaster.MASTER);
Configuration conf = master.getConfiguration();
HBaseAdmin hbadmin = new HBaseAdmin(conf);
MetaTableLocator metaTableLocator = new MetaTableLocator();
String fqtn = request.getParameter("name");
HTable table = new HTable(conf, fqtn);
String tableHeader;
@ -51,7 +53,7 @@
} else {
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().getMetaLocation();
ServerName rl = metaTableLocator.getMetaRegionLocation(master.getZooKeeper());
boolean showFragmentation = conf.getBoolean("hbase.master.ui.fragmentation.enabled", false);
boolean readOnly = conf.getBoolean("hbase.master.ui.readonly", false);
Map<String, Integer> frags = null;
@ -203,7 +205,7 @@
<%
// NOTE: Presumes one meta region only.
HRegionInfo meta = HRegionInfo.FIRST_META_REGIONINFO;
ServerName metaLocation = master.getCatalogTracker().waitForMeta(1);
ServerName metaLocation = metaTableLocator.waitMetaRegionLocation(master.getZooKeeper(), 1);
for (int i = 0; i < 1; i++) {
String url = "//" + metaLocation.getHostname() + ":" + master.getRegionServerInfoPort(metaLocation) + "/";
%>

View File

@ -54,7 +54,6 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Waiter.Predicate;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get;
@ -2021,7 +2020,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
int j = (i + 1) % startKeys.length;
HRegionInfo hri = new HRegionInfo(table.getName(),
startKeys[i], startKeys[j]);
MetaEditor.addRegionToMeta(meta, hri);
MetaTableAccessor.addRegionToMeta(meta, hri);
newRegions.add(hri);
count++;
}
@ -2073,7 +2072,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
int j = (i + 1) % startKeys.length;
HRegionInfo hri = new HRegionInfo(htd.getTableName(), startKeys[i],
startKeys[j]);
MetaEditor.addRegionToMeta(meta, hri);
MetaTableAccessor.addRegionToMeta(meta, hri);
newRegions.add(hri);
}
@ -2087,7 +2086,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @throws IOException When reading the rows fails.
*/
public List<byte[]> getMetaTableRows() throws IOException {
// TODO: Redo using MetaReader class
// TODO: Redo using MetaTableAccessor class
HTable t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME);
List<byte[]> rows = new ArrayList<byte[]>();
ResultScanner s = t.getScanner(new Scan());
@ -2107,7 +2106,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @throws IOException When reading the rows fails.
*/
public List<byte[]> getMetaTableRows(TableName tableName) throws IOException {
// TODO: Redo using MetaReader.
// TODO: Redo using MetaTableAccessor.
HTable t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME);
List<byte[]> rows = new ArrayList<byte[]>();
ResultScanner s = t.getScanner(new Scan());

View File

@ -26,7 +26,7 @@ import java.util.concurrent.ConcurrentSkipListMap;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.ipc.RpcServerInterface;
@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.regionserver.ServerNonceManager;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
@ -92,7 +93,7 @@ class MockRegionServerServices implements RegionServerServices {
}
@Override
public void postOpenDeployTasks(HRegion r, CatalogTracker ct)
public void postOpenDeployTasks(HRegion r)
throws KeeperException, IOException {
addToOnlineRegions(r);
}
@ -127,7 +128,12 @@ class MockRegionServerServices implements RegionServerServices {
}
@Override
public CatalogTracker getCatalogTracker() {
public HConnection getShortCircuitConnection() {
return null;
}
@Override
public MetaTableLocator getMetaTableLocator() {
return null;
}

View File

@ -21,7 +21,6 @@ package org.apache.hadoop.hbase;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.executor.ExecutorService;
@ -94,7 +93,6 @@ public class TestDrainingServer {
final ServerName SERVERNAME_A = ServerName.valueOf("mockserver_a.org", 1000, 8000);
final ServerName SERVERNAME_B = ServerName.valueOf("mockserver_b.org", 1001, 8000);
LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(conf);
CatalogTracker catalogTracker = Mockito.mock(CatalogTracker.class);
final HRegionInfo REGIONINFO = new HRegionInfo(TableName.valueOf("table_test"),
HConstants.EMPTY_START_ROW, HConstants.EMPTY_START_ROW);
@ -138,7 +136,7 @@ public class TestDrainingServer {
Mockito.when(master.getServerManager()).thenReturn(serverManager);
am = new AssignmentManager(server, serverManager, catalogTracker,
am = new AssignmentManager(server, serverManager,
balancer, startupMasterExecutor("mockExecutorService"), null, null);
Mockito.when(master.getAssignmentManager()).thenReturn(am);
@ -165,7 +163,6 @@ public class TestDrainingServer {
public void testAssignmentManagerDoesntUseDrainedServerWithBulkAssign() throws Exception {
Configuration conf = TEST_UTIL.getConfiguration();
LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(conf);
CatalogTracker catalogTracker = Mockito.mock(CatalogTracker.class);
AssignmentManager am;
final HMaster master = Mockito.mock(HMaster.class);
final Server server = Mockito.mock(Server.class);
@ -243,7 +240,7 @@ public class TestDrainingServer {
drainedServers.add(SERVERNAME_C);
drainedServers.add(SERVERNAME_D);
am = new AssignmentManager(server, serverManager, catalogTracker,
am = new AssignmentManager(server, serverManager,
balancer, startupMasterExecutor("mockExecutorServiceBulk"), null, null);
Mockito.when(master.getAssignmentManager()).thenReturn(am);

View File

@ -29,8 +29,6 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.master.RegionStates;
@ -100,13 +98,9 @@ public class TestRegionRebalancing {
admin.createTable(this.desc, Arrays.copyOfRange(HBaseTestingUtility.KEYS,
1, HBaseTestingUtility.KEYS.length));
this.table = new HTable(UTIL.getConfiguration(), this.desc.getTableName());
CatalogTracker ct = new CatalogTracker(UTIL.getConfiguration());
ct.start();
try {
MetaReader.fullScanMetaAndPrint(ct);
} finally {
ct.stop();
}
MetaTableAccessor.fullScanMetaAndPrint(admin.getConnection());
assertEquals("Test table should have right number of regions",
HBaseTestingUtility.KEYS.length,
this.table.getStartKeys().length);

View File

@ -1,126 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.catalog;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Mocking utility for common hbase:meta functionality
*/
public class MetaMockingUtil {
/**
* Returns a Result object constructed from the given region information simulating
* a catalog table result.
* @param region the HRegionInfo object or null
* @return A mocked up Result that fakes a Get on a row in the <code>hbase:meta</code> table.
* @throws IOException
*/
public static Result getMetaTableRowResult(final HRegionInfo region)
throws IOException {
return getMetaTableRowResult(region, null, null, null);
}
/**
* Returns a Result object constructed from the given region information simulating
* a catalog table result.
* @param region the HRegionInfo object or null
* @param ServerName to use making startcode and server hostname:port in meta or null
* @return A mocked up Result that fakes a Get on a row in the <code>hbase:meta</code> table.
* @throws IOException
*/
public static Result getMetaTableRowResult(final HRegionInfo region, final ServerName sn)
throws IOException {
return getMetaTableRowResult(region, sn, null, null);
}
/**
* Returns a Result object constructed from the given region information simulating
* a catalog table result.
* @param region the HRegionInfo object or null
* @param ServerName to use making startcode and server hostname:port in meta or null
* @param splita daughter region or null
* @param splitb daughter region or null
* @return A mocked up Result that fakes a Get on a row in the <code>hbase:meta</code> table.
* @throws IOException
*/
public static Result getMetaTableRowResult(HRegionInfo region, final ServerName sn,
HRegionInfo splita, HRegionInfo splitb) throws IOException {
List<Cell> kvs = new ArrayList<Cell>();
if (region != null) {
kvs.add(new KeyValue(
region.getRegionName(),
HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
region.toByteArray()));
}
if (sn != null) {
kvs.add(new KeyValue(region.getRegionName(),
HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
Bytes.toBytes(sn.getHostAndPort())));
kvs.add(new KeyValue(region.getRegionName(),
HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER,
Bytes.toBytes(sn.getStartcode())));
}
if (splita != null) {
kvs.add(new KeyValue(
region.getRegionName(),
HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER,
splita.toByteArray()));
}
if (splitb != null) {
kvs.add(new KeyValue(
region.getRegionName(),
HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER,
splitb.toByteArray()));
}
//important: sort the kvs so that binary search work
Collections.sort(kvs, KeyValue.META_COMPARATOR);
return Result.create(kvs);
}
/**
* @param sn ServerName to use making startcode and server in meta
* @param hri Region to serialize into HRegionInfo
* @return A mocked up Result that fakes a Get on a row in the <code>hbase:meta</code> table.
* @throws IOException
*/
public static Result getMetaTableRowResultAsSplitRegion(final HRegionInfo hri, final ServerName sn)
throws IOException {
hri.setOffline(true);
hri.setSplit(true);
return getMetaTableRowResult(hri, sn);
}
}

View File

@ -1,395 +0,0 @@
/**
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.catalog;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.net.ConnectException;
import java.util.concurrent.atomic.AtomicInteger;
import junit.framework.Assert;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetResponse;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.util.Progressable;
import org.apache.zookeeper.KeeperException;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
/**
* Test {@link CatalogTracker}
*/
@Category(MediumTests.class)
public class TestCatalogTracker {
private static final Log LOG = LogFactory.getLog(TestCatalogTracker.class);
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static final ServerName SN =
ServerName.valueOf("example.org", 1234, System.currentTimeMillis());
private ZooKeeperWatcher watcher;
private Abortable abortable;
@BeforeClass public static void beforeClass() throws Exception {
// Set this down so tests run quicker
UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 3);
UTIL.startMiniZKCluster();
}
@AfterClass public static void afterClass() throws IOException {
UTIL.getZkCluster().shutdown();
}
@Before public void before() throws IOException {
this.abortable = new Abortable() {
@Override
public void abort(String why, Throwable e) {
LOG.info(why, e);
}
@Override
public boolean isAborted() {
return false;
}
};
this.watcher = new ZooKeeperWatcher(UTIL.getConfiguration(),
this.getClass().getSimpleName(), this.abortable, true);
}
@After public void after() {
try {
// Clean out meta location or later tests will be confused... they presume
// start fresh in zk.
MetaRegionTracker.deleteMetaLocation(this.watcher);
} catch (KeeperException e) {
LOG.warn("Unable to delete hbase:meta location", e);
}
// Clear out our doctored connection or could mess up subsequent tests.
HConnectionManager.deleteConnection(UTIL.getConfiguration());
this.watcher.close();
}
private CatalogTracker constructAndStartCatalogTracker(final HConnection c)
throws IOException, InterruptedException {
CatalogTracker ct = new CatalogTracker(this.watcher, UTIL.getConfiguration(),
c, this.abortable);
ct.start();
return ct;
}
/**
* Test that we get notification if hbase:meta moves.
* @throws IOException
* @throws InterruptedException
* @throws KeeperException
*/
@Test public void testThatIfMETAMovesWeAreNotified()
throws IOException, InterruptedException, KeeperException {
HConnection connection = Mockito.mock(HConnection.class);
constructAndStartCatalogTracker(connection);
MetaRegionTracker.setMetaLocation(this.watcher,
ServerName.valueOf("example.com", 1234, System.currentTimeMillis()));
}
/**
* Test interruptable while blocking wait on meta.
* @throws IOException
* @throws ServiceException
* @throws InterruptedException
*/
@Test public void testInterruptWaitOnMeta()
throws IOException, InterruptedException, ServiceException {
final ClientProtos.ClientService.BlockingInterface client =
Mockito.mock(ClientProtos.ClientService.BlockingInterface.class);
HConnection connection = mockConnection(null, client);
Mockito.when(client.get((RpcController)Mockito.any(), (GetRequest)Mockito.any())).
thenReturn(GetResponse.newBuilder().build());
final CatalogTracker ct = constructAndStartCatalogTracker(connection);
ServerName meta = ct.getMetaLocation();
Assert.assertNull(meta);
Thread t = new Thread() {
@Override
public void run() {
try {
ct.waitForMeta();
} catch (InterruptedException e) {
throw new RuntimeException("Interrupted", e);
}
}
};
t.start();
while (!t.isAlive())
Threads.sleep(1);
Threads.sleep(1);
assertTrue(t.isAlive());
ct.stop();
// Join the thread... should exit shortly.
t.join();
}
private void testVerifyMetaRegionLocationWithException(Exception ex)
throws IOException, InterruptedException, KeeperException, ServiceException {
// Mock an ClientProtocol.
final ClientProtos.ClientService.BlockingInterface implementation =
Mockito.mock(ClientProtos.ClientService.BlockingInterface.class);
HConnection connection = mockConnection(null, implementation);
// If a 'get' is called on mocked interface, throw connection refused.
Mockito.when(implementation.get((RpcController) Mockito.any(), (GetRequest) Mockito.any())).
thenThrow(new ServiceException(ex));
// Now start up the catalogtracker with our doctored Connection.
final CatalogTracker ct = constructAndStartCatalogTracker(connection);
MetaRegionTracker.setMetaLocation(this.watcher, SN);
long timeout = UTIL.getConfiguration().
getLong("hbase.catalog.verification.timeout", 1000);
Assert.assertFalse(ct.verifyMetaRegionLocation(timeout));
}
/**
* Test we survive a connection refused {@link ConnectException}
* @throws IOException
* @throws InterruptedException
* @throws KeeperException
* @throws ServiceException
*/
@Test
public void testGetMetaServerConnectionFails()
throws IOException, InterruptedException, KeeperException, ServiceException {
testVerifyMetaRegionLocationWithException(new ConnectException("Connection refused"));
}
/**
* Test that verifyMetaRegionLocation properly handles getting a
* ServerNotRunningException. See HBASE-4470.
* Note this doesn't check the exact exception thrown in the
* HBASE-4470 as there it is thrown from getHConnection() and
* here it is thrown from get() -- but those are both called
* from the same function anyway, and this way is less invasive than
* throwing from getHConnection would be.
*
* @throws IOException
* @throws InterruptedException
* @throws KeeperException
* @throws ServiceException
*/
@Test
public void testVerifyMetaRegionServerNotRunning()
throws IOException, InterruptedException, KeeperException, ServiceException {
testVerifyMetaRegionLocationWithException(new ServerNotRunningYetException("mock"));
}
/**
* Test get of meta region fails properly if nothing to connect to.
* @throws IOException
* @throws InterruptedException
* @throws KeeperException
* @throws ServiceException
*/
@Test
public void testVerifyMetaRegionLocationFails()
throws IOException, InterruptedException, KeeperException, ServiceException {
HConnection connection = Mockito.mock(HConnection.class);
ServiceException connectException =
new ServiceException(new ConnectException("Connection refused"));
final AdminProtos.AdminService.BlockingInterface implementation =
Mockito.mock(AdminProtos.AdminService.BlockingInterface.class);
Mockito.when(implementation.getRegionInfo((RpcController)Mockito.any(),
(GetRegionInfoRequest)Mockito.any())).thenThrow(connectException);
Mockito.when(connection.getAdmin(Mockito.any(ServerName.class), Mockito.anyBoolean())).
thenReturn(implementation);
final CatalogTracker ct = constructAndStartCatalogTracker(connection);
MetaRegionTracker.setMetaLocation(this.watcher,
ServerName.valueOf("example.com", 1234, System.currentTimeMillis()));
Assert.assertFalse(ct.verifyMetaRegionLocation(100));
}
@Test (expected = NotAllMetaRegionsOnlineException.class)
public void testTimeoutWaitForMeta()
throws IOException, InterruptedException {
HConnection connection = Mockito.mock(HConnection.class);
final CatalogTracker ct = constructAndStartCatalogTracker(connection);
ct.waitForMeta(100);
}
/**
* Test waiting on meat w/ no timeout specified.
* @throws IOException
* @throws InterruptedException
* @throws KeeperException
*/
@Test public void testNoTimeoutWaitForMeta()
throws IOException, InterruptedException, KeeperException {
HConnection connection = Mockito.mock(HConnection.class);
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);
startWaitAliveThenWaitItLives(t, 1);
// Set a meta location.
hsa = setMetaLocation();
// Join the thread... should exit shortly.
t.join();
// Now meta is available.
Assert.assertTrue(ct.getMetaLocation().equals(hsa));
}
private ServerName setMetaLocation() throws KeeperException {
MetaRegionTracker.setMetaLocation(this.watcher, SN);
return SN;
}
/**
* @param admin An {@link AdminProtos.AdminService.BlockingInterface} instance; you'll likely
* want to pass a mocked HRS; can be null.
* @param client A mocked ClientProtocol instance, can be null
* @return Mock up a connection that returns a {@link Configuration} when
* {@link HConnection#getConfiguration()} is called, a 'location' when
* {@link HConnection#getRegionLocation(byte[], byte[], boolean)} is called,
* and that returns the passed {@link AdminProtos.AdminService.BlockingInterface} instance when
* {@link HConnection#getAdmin(ServerName)} is called, returns the passed
* {@link ClientProtos.ClientService.BlockingInterface} instance when
* {@link HConnection#getClient(ServerName)} is called (Be sure to call
* {@link HConnectionManager#deleteConnection(org.apache.hadoop.conf.Configuration)}
* when done with this mocked Connection.
* @throws IOException
*/
private HConnection mockConnection(final AdminProtos.AdminService.BlockingInterface admin,
final ClientProtos.ClientService.BlockingInterface client)
throws IOException {
HConnection connection =
HConnectionTestingUtility.getMockedConnection(UTIL.getConfiguration());
Mockito.doNothing().when(connection).close();
// Make it so we return any old location when asked.
final HRegionLocation anyLocation =
new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, SN);
Mockito.when(connection.getRegionLocation((TableName) Mockito.any(),
(byte[]) Mockito.any(), Mockito.anyBoolean())).
thenReturn(anyLocation);
Mockito.when(connection.locateRegion((TableName) Mockito.any(),
(byte[]) Mockito.any())).
thenReturn(anyLocation);
if (admin != null) {
// If a call to getHRegionConnection, return this implementation.
Mockito.when(connection.getAdmin(Mockito.any(ServerName.class))).
thenReturn(admin);
}
if (client != null) {
// If a call to getClient, return this implementation.
Mockito.when(connection.getClient(Mockito.any(ServerName.class))).
thenReturn(client);
}
return connection;
}
/**
* @return A mocked up Result that fakes a Get on a row in the
* <code>hbase:meta</code> table.
* @throws IOException
*/
private Result getMetaTableRowResult() throws IOException {
return MetaMockingUtil.getMetaTableRowResult(HRegionInfo.FIRST_META_REGIONINFO, SN);
}
private void startWaitAliveThenWaitItLives(final Thread t, final int ms) {
t.start();
while(!t.isAlive()) {
// Wait
}
// Wait one second.
Threads.sleep(ms);
Assert.assertTrue("Assert " + t.getName() + " still waiting", t.isAlive());
}
class CountingProgressable implements Progressable {
final AtomicInteger counter = new AtomicInteger(0);
@Override
public void progress() {
this.counter.incrementAndGet();
}
}
/**
* Wait on META.
*/
class WaitOnMetaThread extends Thread {
final CatalogTracker ct;
WaitOnMetaThread(final CatalogTracker ct) {
super("WaitOnMeta");
this.ct = ct;
}
@Override
public void run() {
try {
doWaiting();
} catch (InterruptedException e) {
throw new RuntimeException("Failed wait", e);
}
LOG.info("Exiting " + getName());
}
void doWaiting() throws InterruptedException {
try {
while (this.ct.waitForMeta(100) == null);
} catch (NotAllMetaRegionsOnlineException e) {
// Ignore.
}
}
}
}

View File

@ -1,440 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.catalog;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import junit.framework.Assert;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.FsShell;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.migration.NamespaceUpgrade;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.util.ToolRunner;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* 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 hbase:meta info.
* @deprecated Remove after 0.96
*/
@Category(MediumTests.class)
@Deprecated
public class TestMetaMigrationConvertingToPB {
static final Log LOG = LogFactory.getLog(TestMetaMigrationConvertingToPB.class);
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private final static String TESTTABLE = "TestTable";
private final static int ROW_COUNT = 100;
private final static int REGION_COUNT = 9; //initial number of regions of the TestTable
private static final int META_VERSION_092 = 0;
/*
* This test uses a tgz file named "TestMetaMigrationConvertingToPB.tgz" under
* hbase-server/src/test/data which contains file data from a 0.92 cluster.
* The cluster has a table named "TestTable", which has 100 rows. 0.94 has same
* hbase:meta structure, so it should be the same.
*
* hbase(main):001:0> create 'TestTable', 'f1'
* hbase(main):002:0> for i in 1..100
* hbase(main):003:1> put 'TestTable', "row#{i}", "f1:c1", i
* hbase(main):004:1> end
*
* There are 9 regions in the table
*/
@BeforeClass
public static void setUpBeforeClass() throws Exception {
// Start up our mini cluster on top of an 0.92 root.dir that has data from
// a 0.92 hbase run -- it has a table with 100 rows in it -- and see if
// we can migrate from 0.92
TEST_UTIL.startMiniZKCluster();
TEST_UTIL.startMiniDFSCluster(1);
Path testdir = TEST_UTIL.getDataTestDir("TestMetaMigrationConvertToPB");
// Untar our test dir.
File untar = untar(new File(testdir.toString()));
// Now copy the untar up into hdfs so when we start hbase, we'll run from it.
Configuration conf = TEST_UTIL.getConfiguration();
FsShell shell = new FsShell(conf);
FileSystem fs = FileSystem.get(conf);
// find where hbase will root itself, so we can copy filesystem there
Path hbaseRootDir = TEST_UTIL.getDefaultRootDirPath();
if (!fs.isDirectory(hbaseRootDir.getParent())) {
// mkdir at first
fs.mkdirs(hbaseRootDir.getParent());
}
doFsCommand(shell,
new String [] {"-put", untar.toURI().toString(), hbaseRootDir.toString()});
//windows fix: tgz file has hbase:meta directory renamed as -META- since the original is an illegal
//name under windows. So we rename it back. See src/test/data//TestMetaMigrationConvertingToPB.README and
//https://issues.apache.org/jira/browse/HBASE-6821
doFsCommand(shell, new String [] {"-mv", new Path(hbaseRootDir, "-META-").toString(),
new Path(hbaseRootDir, ".META.").toString()});
// See whats in minihdfs.
doFsCommand(shell, new String [] {"-lsr", "/"});
//upgrade to namespace as well
Configuration toolConf = TEST_UTIL.getConfiguration();
conf.set(HConstants.HBASE_DIR, TEST_UTIL.getDefaultRootDirPath().toString());
ToolRunner.run(toolConf, new NamespaceUpgrade(), new String[]{"--upgrade"});
TEST_UTIL.startMiniHBaseCluster(1, 1);
// Assert we are running against the copied-up filesystem. The copied-up
// rootdir should have had a table named 'TestTable' in it. Assert it
// present.
HTable t = new HTable(TEST_UTIL.getConfiguration(), TESTTABLE);
ResultScanner scanner = t.getScanner(new Scan());
int count = 0;
while (scanner.next() != null) {
count++;
}
// Assert that we find all 100 rows that are in the data we loaded. If
// so then we must have migrated it from 0.90 to 0.92.
Assert.assertEquals(ROW_COUNT, count);
scanner.close();
t.close();
}
private static File untar(final File testdir) throws IOException {
// Find the src data under src/test/data
final String datafile = "TestMetaMigrationConvertToPB";
String srcTarFile =
System.getProperty("project.build.testSourceDirectory", "src/test") +
File.separator + "data" + File.separator + datafile + ".tgz";
File homedir = new File(testdir.toString());
File tgtUntarDir = new File(homedir, datafile);
if (tgtUntarDir.exists()) {
if (!FileUtil.fullyDelete(tgtUntarDir)) {
throw new IOException("Failed delete of " + tgtUntarDir.toString());
}
}
LOG.info("Untarring " + srcTarFile + " into " + homedir.toString());
FileUtil.unTar(new File(srcTarFile), homedir);
Assert.assertTrue(tgtUntarDir.exists());
return tgtUntarDir;
}
private static void doFsCommand(final FsShell shell, final String [] args)
throws Exception {
// Run the 'put' command.
int errcode = shell.run(args);
if (errcode != 0) throw new IOException("Failed put; errcode=" + errcode);
}
/**
* @throws java.lang.Exception
*/
@AfterClass
public static void tearDownAfterClass() throws Exception {
TEST_UTIL.shutdownMiniCluster();
}
@Test
public void testMetaUpdatedFlagInROOT() throws Exception {
HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
boolean metaUpdated = MetaMigrationConvertingToPB.
isMetaTableUpdated(master.getCatalogTracker());
assertEquals(true, metaUpdated);
verifyMetaRowsAreUpdated(master.getCatalogTracker());
}
@Test
public void testMetaMigration() throws Exception {
LOG.info("Starting testMetaMigration");
final byte [] FAMILY = Bytes.toBytes("family");
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testMetaMigration"));
HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
htd.addFamily(hcd);
Configuration conf = TEST_UTIL.getConfiguration();
byte[][] regionNames = new byte[][]{
HConstants.EMPTY_START_ROW,
Bytes.toBytes("region_a"),
Bytes.toBytes("region_b")};
createMultiRegionsWithWritableSerialization(conf,
htd.getTableName().getName(),
regionNames);
CatalogTracker ct =
TEST_UTIL.getMiniHBaseCluster().getMaster().getCatalogTracker();
// Erase the current version of root meta for this test.
undoVersionInRoot(ct);
MetaReader.fullScanMetaAndPrint(ct);
LOG.info("Meta Print completed.testMetaMigration");
long numMigratedRows = MetaMigrationConvertingToPB.updateMeta(
TEST_UTIL.getHBaseCluster().getMaster());
MetaReader.fullScanMetaAndPrint(ct);
// Should be one entry only and it should be for the table we just added.
assertEquals(regionNames.length, numMigratedRows);
// Assert that the flag in ROOT is updated to reflect the correct status
boolean metaUpdated =
MetaMigrationConvertingToPB.isMetaTableUpdated(
TEST_UTIL.getMiniHBaseCluster().getMaster().getCatalogTracker());
assertEquals(true, metaUpdated);
verifyMetaRowsAreUpdated(ct);
}
/**
* This test assumes a master crash/failure during the meta migration process
* and attempts to continue the meta migration process when a new master takes over.
* When a master dies during the meta migration we will have some rows of
* META.CatalogFamily updated with PB serialization and some
* still hanging with writable serialization. When the backup master/ or
* fresh start of master attempts the migration it will encounter some rows of META
* already updated with new HRI and some still legacy. This test will simulate this
* scenario and validates that the migration process can safely skip the updated
* rows and migrate any pending rows at startup.
* @throws Exception
*/
@Test
public void testMasterCrashDuringMetaMigration() throws Exception {
final byte[] FAMILY = Bytes.toBytes("family");
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf
("testMasterCrashDuringMetaMigration"));
HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
htd.addFamily(hcd);
Configuration conf = TEST_UTIL.getConfiguration();
// Create 10 New regions.
createMultiRegionsWithPBSerialization(conf, htd.getTableName().getName(), 10);
// Create 10 Legacy regions.
createMultiRegionsWithWritableSerialization(conf,
htd.getTableName().getName(), 10);
CatalogTracker ct =
TEST_UTIL.getMiniHBaseCluster().getMaster().getCatalogTracker();
// Erase the current version of root meta for this test.
undoVersionInRoot(ct);
MetaReader.fullScanMetaAndPrint(ct);
LOG.info("Meta Print completed.testUpdatesOnMetaWithLegacyHRI");
long numMigratedRows =
MetaMigrationConvertingToPB.updateMetaIfNecessary(
TEST_UTIL.getHBaseCluster().getMaster());
assertEquals(numMigratedRows, 10);
// Assert that the flag in ROOT is updated to reflect the correct status
boolean metaUpdated = MetaMigrationConvertingToPB.
isMetaTableUpdated(TEST_UTIL.getMiniHBaseCluster().getMaster().getCatalogTracker());
assertEquals(true, metaUpdated);
verifyMetaRowsAreUpdated(ct);
LOG.info("END testMasterCrashDuringMetaMigration");
}
/**
* Verify that every hbase:meta row is updated
*/
void verifyMetaRowsAreUpdated(CatalogTracker catalogTracker)
throws IOException {
List<Result> results = MetaReader.fullScan(catalogTracker);
assertTrue(results.size() >= REGION_COUNT);
for (Result result : results) {
byte[] hriBytes = result.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
assertTrue(hriBytes != null && hriBytes.length > 0);
assertTrue(MetaMigrationConvertingToPB.isMigrated(hriBytes));
byte[] splitA = result.getValue(HConstants.CATALOG_FAMILY,
HConstants.SPLITA_QUALIFIER);
if (splitA != null && splitA.length > 0) {
assertTrue(MetaMigrationConvertingToPB.isMigrated(splitA));
}
byte[] splitB = result.getValue(HConstants.CATALOG_FAMILY,
HConstants.SPLITB_QUALIFIER);
if (splitB != null && splitB.length > 0) {
assertTrue(MetaMigrationConvertingToPB.isMigrated(splitB));
}
}
}
/** Changes the version of hbase:meta to 0 to simulate 0.92 and 0.94 clusters*/
private void undoVersionInRoot(CatalogTracker ct) throws IOException {
Put p = new Put(HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
p.add(HConstants.CATALOG_FAMILY, HConstants.META_VERSION_QUALIFIER,
Bytes.toBytes(META_VERSION_092));
// TODO wire this MetaEditor.putToRootTable(ct, p);
LOG.info("Downgraded -ROOT- meta version=" + META_VERSION_092);
}
/**
* Inserts multiple regions into hbase:meta using Writable serialization instead of PB
*/
public int createMultiRegionsWithWritableSerialization(final Configuration c,
final byte[] tableName, int numRegions) throws IOException {
if (numRegions < 3) throw new IOException("Must create at least 3 regions");
byte [] startKey = Bytes.toBytes("aaaaa");
byte [] endKey = Bytes.toBytes("zzzzz");
byte [][] splitKeys = Bytes.split(startKey, endKey, numRegions - 3);
byte [][] regionStartKeys = new byte[splitKeys.length+1][];
for (int i=0;i<splitKeys.length;i++) {
regionStartKeys[i+1] = splitKeys[i];
}
regionStartKeys[0] = HConstants.EMPTY_BYTE_ARRAY;
return createMultiRegionsWithWritableSerialization(c, tableName, regionStartKeys);
}
public int createMultiRegionsWithWritableSerialization(final Configuration c,
final byte[] tableName, byte [][] startKeys)
throws IOException {
return createMultiRegionsWithWritableSerialization(c,
TableName.valueOf(tableName), startKeys);
}
/**
* Inserts multiple regions into hbase:meta using Writable serialization instead of PB
*/
public int createMultiRegionsWithWritableSerialization(final Configuration c,
final TableName tableName, byte [][] startKeys)
throws IOException {
Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
HTable meta = new HTable(c, TableName.META_TABLE_NAME);
List<HRegionInfo> newRegions
= new ArrayList<HRegionInfo>(startKeys.length);
int count = 0;
for (int i = 0; i < startKeys.length; i++) {
int j = (i + 1) % startKeys.length;
HRegionInfo hri = new HRegionInfo(tableName, startKeys[i], startKeys[j]);
Put put = new Put(hri.getRegionName());
put.setDurability(Durability.SKIP_WAL);
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
getBytes(hri)); //this is the old Writable serialization
//also add the region as it's daughters
put.add(HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER,
getBytes(hri)); //this is the old Writable serialization
put.add(HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER,
getBytes(hri)); //this is the old Writable serialization
meta.put(put);
LOG.info("createMultiRegionsWithWritableSerialization: PUT inserted " + hri.toString());
newRegions.add(hri);
count++;
}
meta.close();
return count;
}
@Deprecated
private byte[] getBytes(HRegionInfo hri) throws IOException {
DataOutputBuffer out = new DataOutputBuffer();
try {
hri.write(out);
return out.getData();
} finally {
if (out != null) {
out.close();
}
}
}
/**
* Inserts multiple regions into hbase:meta using PB serialization
*/
int createMultiRegionsWithPBSerialization(final Configuration c,
final byte[] tableName, int numRegions)
throws IOException {
if (numRegions < 3) throw new IOException("Must create at least 3 regions");
byte [] startKey = Bytes.toBytes("aaaaa");
byte [] endKey = Bytes.toBytes("zzzzz");
byte [][] splitKeys = Bytes.split(startKey, endKey, numRegions - 3);
byte [][] regionStartKeys = new byte[splitKeys.length+1][];
for (int i=0;i<splitKeys.length;i++) {
regionStartKeys[i+1] = splitKeys[i];
}
regionStartKeys[0] = HConstants.EMPTY_BYTE_ARRAY;
return createMultiRegionsWithPBSerialization(c, tableName, regionStartKeys);
}
/**
* Inserts multiple regions into hbase:meta using PB serialization
*/
int createMultiRegionsWithPBSerialization(final Configuration c, final byte[] tableName,
byte [][] startKeys) throws IOException {
return createMultiRegionsWithPBSerialization(c,
TableName.valueOf(tableName), startKeys);
}
int createMultiRegionsWithPBSerialization(final Configuration c,
final TableName tableName,
byte [][] startKeys) throws IOException {
Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
HTable meta = new HTable(c, TableName.META_TABLE_NAME);
List<HRegionInfo> newRegions
= new ArrayList<HRegionInfo>(startKeys.length);
int count = 0;
for (int i = 0; i < startKeys.length; i++) {
int j = (i + 1) % startKeys.length;
HRegionInfo hri = new HRegionInfo(tableName, startKeys[i], startKeys[j]);
Put put = MetaEditor.makePutFromRegionInfo(hri);
put.setDurability(Durability.SKIP_WAL);
meta.put(put);
LOG.info("createMultiRegionsWithPBSerialization: PUT inserted " + hri.toString());
newRegions.add(hri);
count++;
}
meta.close();
return count;
}
}

View File

@ -1,376 +0,0 @@
/**
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.catalog;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.List;
import java.util.Random;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test {@link MetaReader}, {@link MetaEditor}.
*/
@Category(MediumTests.class)
public class TestMetaReaderEditor {
private static final Log LOG = LogFactory.getLog(TestMetaReaderEditor.class);
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static CatalogTracker CT;
@BeforeClass public static void beforeClass() throws Exception {
UTIL.startMiniCluster(3);
Configuration c = new Configuration(UTIL.getConfiguration());
// Tests to 4 retries every 5 seconds. Make it try every 1 second so more
// responsive. 1 second is default as is ten retries.
c.setLong("hbase.client.pause", 1000);
c.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 10);
CT = new CatalogTracker(c);
CT.start();
}
@AfterClass public static void afterClass() throws Exception {
CT.stop();
UTIL.shutdownMiniCluster();
}
/**
* Does {@link MetaReader#getRegion(CatalogTracker, byte[])} and a write
* against hbase:meta while its hosted server is restarted to prove our retrying
* works.
* @throws IOException
* @throws InterruptedException
*/
@Test public void testRetrying()
throws IOException, InterruptedException {
final TableName name =
TableName.valueOf("testRetrying");
LOG.info("Started " + name);
HTable t = UTIL.createTable(name, HConstants.CATALOG_FAMILY);
int regionCount = UTIL.createMultiRegions(t, HConstants.CATALOG_FAMILY);
// Test it works getting a region from just made user table.
final List<HRegionInfo> regions =
testGettingTableRegions(CT, name, regionCount);
MetaTask reader = new MetaTask(CT, "reader") {
@Override
void metaTask() throws Throwable {
testGetRegion(this.ct, regions.get(0));
LOG.info("Read " + regions.get(0).getEncodedName());
}
};
MetaTask writer = new MetaTask(CT, "writer") {
@Override
void metaTask() throws Throwable {
MetaEditor.addRegionToMeta(this.ct, regions.get(0));
LOG.info("Wrote " + regions.get(0).getEncodedName());
}
};
reader.start();
writer.start();
// We're gonna check how it takes. If it takes too long, we will consider
// it as a fail. We can't put that in the @Test tag as we want to close
// the threads nicely
final long timeOut = 180000;
long startTime = System.currentTimeMillis();
try {
// Make sure reader and writer are working.
assertTrue(reader.isProgressing());
assertTrue(writer.isProgressing());
// Kill server hosting meta -- twice . See if our reader/writer ride over the
// meta moves. They'll need to retry.
for (int i = 0; i < 2; i++) {
LOG.info("Restart=" + i);
UTIL.ensureSomeRegionServersAvailable(2);
int index = -1;
do {
index = UTIL.getMiniHBaseCluster().getServerWithMeta();
} while (index == -1 &&
startTime + timeOut < System.currentTimeMillis());
if (index != -1){
UTIL.getMiniHBaseCluster().abortRegionServer(index);
UTIL.getMiniHBaseCluster().waitOnRegionServer(index);
}
}
assertTrue("reader: " + reader.toString(), reader.isProgressing());
assertTrue("writer: " + writer.toString(), writer.isProgressing());
} catch (IOException e) {
throw e;
} finally {
reader.stop = true;
writer.stop = true;
reader.join();
writer.join();
t.close();
}
long exeTime = System.currentTimeMillis() - startTime;
assertTrue("Timeout: test took " + exeTime / 1000 + " sec", exeTime < timeOut);
}
/**
* Thread that runs a MetaReader/MetaEditor task until asked stop.
*/
abstract static class MetaTask extends Thread {
boolean stop = false;
int count = 0;
Throwable t = null;
final CatalogTracker ct;
MetaTask(final CatalogTracker ct, final String name) {
super(name);
this.ct = ct;
}
@Override
public void run() {
try {
while(!this.stop) {
LOG.info("Before " + this.getName()+ ", count=" + this.count);
metaTask();
this.count += 1;
LOG.info("After " + this.getName() + ", count=" + this.count);
Thread.sleep(100);
}
} catch (Throwable t) {
LOG.info(this.getName() + " failed", t);
this.t = t;
}
}
boolean isProgressing() throws InterruptedException {
int currentCount = this.count;
while(currentCount == this.count) {
if (!isAlive()) return false;
if (this.t != null) return false;
Thread.sleep(10);
}
return true;
}
@Override
public String toString() {
return "count=" + this.count + ", t=" +
(this.t == null? "null": this.t.toString());
}
abstract void metaTask() throws Throwable;
}
@Test public void testGetRegionsCatalogTables()
throws IOException, InterruptedException {
List<HRegionInfo> regions =
MetaReader.getTableRegions(CT, TableName.META_TABLE_NAME);
assertTrue(regions.size() >= 1);
assertTrue(MetaReader.getTableRegionsAndLocations(CT,
TableName.META_TABLE_NAME).size() >= 1);
}
@Test public void testTableExists() throws IOException {
final TableName name =
TableName.valueOf("testTableExists");
assertFalse(MetaReader.tableExists(CT, name));
UTIL.createTable(name, HConstants.CATALOG_FAMILY);
assertTrue(MetaReader.tableExists(CT, name));
HBaseAdmin admin = UTIL.getHBaseAdmin();
admin.disableTable(name);
admin.deleteTable(name);
assertFalse(MetaReader.tableExists(CT, name));
assertTrue(MetaReader.tableExists(CT,
TableName.META_TABLE_NAME));
}
@Test public void testGetRegion() throws IOException, InterruptedException {
final String name = "testGetRegion";
LOG.info("Started " + name);
// Test get on non-existent region.
Pair<HRegionInfo, ServerName> pair =
MetaReader.getRegion(CT, Bytes.toBytes("nonexistent-region"));
assertNull(pair);
LOG.info("Finished " + name);
}
// Test for the optimization made in HBASE-3650
@Test public void testScanMetaForTable()
throws IOException, InterruptedException {
final TableName name =
TableName.valueOf("testScanMetaForTable");
LOG.info("Started " + name);
/** Create 2 tables
- testScanMetaForTable
- testScanMetaForTablf
**/
UTIL.createTable(name, HConstants.CATALOG_FAMILY);
// name that is +1 greater than the first one (e+1=f)
TableName greaterName =
TableName.valueOf("testScanMetaForTablf");
UTIL.createTable(greaterName, HConstants.CATALOG_FAMILY);
// Now make sure we only get the regions from 1 of the tables at a time
assertEquals(1, MetaReader.getTableRegions(CT, name).size());
assertEquals(1, MetaReader.getTableRegions(CT, greaterName).size());
}
private static List<HRegionInfo> testGettingTableRegions(final CatalogTracker ct,
final TableName name, final int regionCount)
throws IOException, InterruptedException {
List<HRegionInfo> regions = MetaReader.getTableRegions(ct, name);
assertEquals(regionCount, regions.size());
Pair<HRegionInfo, ServerName> pair =
MetaReader.getRegion(ct, regions.get(0).getRegionName());
assertEquals(regions.get(0).getEncodedName(),
pair.getFirst().getEncodedName());
return regions;
}
private static void testGetRegion(final CatalogTracker ct,
final HRegionInfo region)
throws IOException, InterruptedException {
Pair<HRegionInfo, ServerName> pair =
MetaReader.getRegion(ct, region.getRegionName());
assertEquals(region.getEncodedName(),
pair.getFirst().getEncodedName());
}
@Test
public void testParseReplicaIdFromServerColumn() {
String column1 = HConstants.SERVER_QUALIFIER_STR;
assertEquals(0, MetaReader.parseReplicaIdFromServerColumn(Bytes.toBytes(column1)));
String column2 = column1 + MetaReader.META_REPLICA_ID_DELIMITER;
assertEquals(-1, MetaReader.parseReplicaIdFromServerColumn(Bytes.toBytes(column2)));
String column3 = column2 + "00";
assertEquals(-1, MetaReader.parseReplicaIdFromServerColumn(Bytes.toBytes(column3)));
String column4 = column3 + "2A";
assertEquals(42, MetaReader.parseReplicaIdFromServerColumn(Bytes.toBytes(column4)));
String column5 = column4 + "2A";
assertEquals(-1, MetaReader.parseReplicaIdFromServerColumn(Bytes.toBytes(column5)));
String column6 = HConstants.STARTCODE_QUALIFIER_STR;
assertEquals(-1, MetaReader.parseReplicaIdFromServerColumn(Bytes.toBytes(column6)));
}
@Test
public void testMetaReaderGetColumnMethods() {
Assert.assertArrayEquals(HConstants.SERVER_QUALIFIER, MetaReader.getServerColumn(0));
Assert.assertArrayEquals(Bytes.toBytes(HConstants.SERVER_QUALIFIER_STR
+ MetaReader.META_REPLICA_ID_DELIMITER + "002A"), MetaReader.getServerColumn(42));
Assert.assertArrayEquals(HConstants.STARTCODE_QUALIFIER, MetaReader.getStartCodeColumn(0));
Assert.assertArrayEquals(Bytes.toBytes(HConstants.STARTCODE_QUALIFIER_STR
+ MetaReader.META_REPLICA_ID_DELIMITER + "002A"), MetaReader.getStartCodeColumn(42));
Assert.assertArrayEquals(HConstants.SEQNUM_QUALIFIER, MetaReader.getSeqNumColumn(0));
Assert.assertArrayEquals(Bytes.toBytes(HConstants.SEQNUM_QUALIFIER_STR
+ MetaReader.META_REPLICA_ID_DELIMITER + "002A"), MetaReader.getSeqNumColumn(42));
}
@Test
public void testMetaLocationsForRegionReplicas() throws IOException {
Random random = new Random();
ServerName serverName0 = ServerName.valueOf("foo", 60010, random.nextLong());
ServerName serverName1 = ServerName.valueOf("bar", 60010, random.nextLong());
ServerName serverName100 = ServerName.valueOf("baz", 60010, random.nextLong());
long regionId = System.currentTimeMillis();
HRegionInfo primary = new HRegionInfo(TableName.valueOf("table_foo"),
HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, regionId, 0);
HRegionInfo replica1 = new HRegionInfo(TableName.valueOf("table_foo"),
HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, regionId, 1);
HRegionInfo replica100 = new HRegionInfo(TableName.valueOf("table_foo"),
HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, regionId, 100);
long seqNum0 = random.nextLong();
long seqNum1 = random.nextLong();
long seqNum100 = random.nextLong();
HTable meta = MetaReader.getMetaHTable(CT);
try {
MetaEditor.updateRegionLocation(CT, primary, serverName0, seqNum0);
// assert that the server, startcode and seqNum columns are there for the primary region
assertMetaLocation(meta, primary.getRegionName(), serverName0, seqNum0, 0, true);
// add replica = 1
MetaEditor.updateRegionLocation(CT, replica1, serverName1, seqNum1);
// check whether the primary is still there
assertMetaLocation(meta, primary.getRegionName(), serverName0, seqNum0, 0, true);
// now check for replica 1
assertMetaLocation(meta, primary.getRegionName(), serverName1, seqNum1, 1, true);
// add replica = 1
MetaEditor.updateRegionLocation(CT, replica100, serverName100, seqNum100);
// check whether the primary is still there
assertMetaLocation(meta, primary.getRegionName(), serverName0, seqNum0, 0, true);
// check whether the replica 1 is still there
assertMetaLocation(meta, primary.getRegionName(), serverName1, seqNum1, 1, true);
// now check for replica 1
assertMetaLocation(meta, primary.getRegionName(), serverName100, seqNum100, 100, true);
} finally {
meta.close();
}
}
public static void assertMetaLocation(HTable meta, byte[] row, ServerName serverName,
long seqNum, int replicaId, boolean checkSeqNum) throws IOException {
Get get = new Get(row);
Result result = meta.get(get);
assertTrue(Bytes.equals(
result.getValue(HConstants.CATALOG_FAMILY, MetaReader.getServerColumn(replicaId)),
Bytes.toBytes(serverName.getHostAndPort())));
assertTrue(Bytes.equals(
result.getValue(HConstants.CATALOG_FAMILY, MetaReader.getStartCodeColumn(replicaId)),
Bytes.toBytes(serverName.getStartcode())));
if (checkSeqNum) {
assertTrue(Bytes.equals(
result.getValue(HConstants.CATALOG_FAMILY, MetaReader.getSeqNumColumn(replicaId)),
Bytes.toBytes(seqNum)));
}
}
}

View File

@ -1,228 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.catalog;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.NavigableMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellScannable;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
/**
* Test MetaReader/Editor but without spinning up a cluster.
* We mock regionserver back and forth (we do spin up a zk cluster).
*/
@Category(MediumTests.class)
public class TestMetaReaderEditorNoCluster {
private static final Log LOG = LogFactory.getLog(TestMetaReaderEditorNoCluster.class);
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static final Abortable ABORTABLE = new Abortable() {
boolean aborted = false;
@Override
public void abort(String why, Throwable e) {
LOG.info(why, e);
this.aborted = true;
throw new RuntimeException(e);
}
@Override
public boolean isAborted() {
return this.aborted;
}
};
@Before
public void before() throws Exception {
UTIL.startMiniZKCluster();
}
@After
public void after() throws IOException {
UTIL.shutdownMiniZKCluster();
}
@Test
public void testGetHRegionInfo() throws IOException {
assertNull(HRegionInfo.getHRegionInfo(new Result()));
List<Cell> kvs = new ArrayList<Cell>();
Result r = Result.create(kvs);
assertNull(HRegionInfo.getHRegionInfo(r));
byte [] f = HConstants.CATALOG_FAMILY;
// Make a key value that doesn't have the expected qualifier.
kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY, f,
HConstants.SERVER_QUALIFIER, f));
r = Result.create(kvs);
assertNull(HRegionInfo.getHRegionInfo(r));
// Make a key that does not have a regioninfo value.
kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY, f,
HConstants.REGIONINFO_QUALIFIER, f));
HRegionInfo hri = HRegionInfo.getHRegionInfo(Result.create(kvs));
assertTrue(hri == null);
// OK, give it what it expects
kvs.clear();
kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY, f,
HConstants.REGIONINFO_QUALIFIER,
HRegionInfo.FIRST_META_REGIONINFO.toByteArray()));
hri = HRegionInfo.getHRegionInfo(Result.create(kvs));
assertNotNull(hri);
assertTrue(hri.equals(HRegionInfo.FIRST_META_REGIONINFO));
}
/**
* Test that MetaReader will ride over server throwing
* "Server not running" IOEs.
* @see @link {https://issues.apache.org/jira/browse/HBASE-3446}
* @throws IOException
* @throws InterruptedException
*/
@Test
public void testRideOverServerNotRunning()
throws IOException, InterruptedException, ServiceException {
// Need a zk watcher.
ZooKeeperWatcher zkw = new ZooKeeperWatcher(UTIL.getConfiguration(),
this.getClass().getSimpleName(), ABORTABLE, true);
// This is a servername we use in a few places below.
ServerName sn = ServerName.valueOf("example.com", 1234, System.currentTimeMillis());
HConnection connection;
CatalogTracker ct = null;
try {
// Mock an ClientProtocol. Our mock implementation will fail a few
// times when we go to open a scanner.
final ClientProtos.ClientService.BlockingInterface implementation =
Mockito.mock(ClientProtos.ClientService.BlockingInterface.class);
// When scan called throw IOE 'Server not running' a few times
// before we return a scanner id. Whats WEIRD is that these
// exceptions do not show in the log because they are caught and only
// printed if we FAIL. We eventually succeed after retry so these don't
// show. We will know if they happened or not because we will ask
// mockito at the end of this test to verify that scan was indeed
// called the wanted number of times.
List<Cell> kvs = new ArrayList<Cell>();
final byte [] rowToVerify = Bytes.toBytes("rowToVerify");
kvs.add(new KeyValue(rowToVerify,
HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
HRegionInfo.FIRST_META_REGIONINFO.toByteArray()));
kvs.add(new KeyValue(rowToVerify,
HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
Bytes.toBytes(sn.getHostAndPort())));
kvs.add(new KeyValue(rowToVerify,
HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER,
Bytes.toBytes(sn.getStartcode())));
final List<CellScannable> cellScannables = new ArrayList<CellScannable>(1);
cellScannables.add(Result.create(kvs));
final ScanResponse.Builder builder = ScanResponse.newBuilder();
for (CellScannable result : cellScannables) {
builder.addCellsPerResult(((Result)result).size());
}
Mockito.when(implementation.scan((RpcController) Mockito.any(), (ScanRequest) Mockito.any()))
.thenThrow(new ServiceException("Server not running (1 of 3)"))
.thenThrow(new ServiceException("Server not running (2 of 3)"))
.thenThrow(new ServiceException("Server not running (3 of 3)"))
.thenReturn(ScanResponse.newBuilder().setScannerId(1234567890L).build())
.thenAnswer(new Answer<ScanResponse>() {
public ScanResponse answer(InvocationOnMock invocation) throws Throwable {
((PayloadCarryingRpcController) invocation.getArguments()[0]).setCellScanner(CellUtil
.createCellScanner(cellScannables));
return builder.build();
}
}).thenReturn(ScanResponse.newBuilder().setMoreResults(false).build());
// Associate a spied-upon HConnection with UTIL.getConfiguration. Need
// to shove this in here first so it gets picked up all over; e.g. by
// HTable.
connection = HConnectionTestingUtility.getSpiedConnection(UTIL.getConfiguration());
// Fix the location lookup so it 'works' though no network. First
// make an 'any location' object.
final HRegionLocation anyLocation =
new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, sn);
final RegionLocations rl = new RegionLocations(anyLocation);
// Return the RegionLocations object when locateRegion
// The ugly format below comes of 'Important gotcha on spying real objects!' from
// http://mockito.googlecode.com/svn/branches/1.6/javadoc/org/mockito/Mockito.html
ClusterConnection cConnection =
HConnectionTestingUtility.getSpiedClusterConnection(UTIL.getConfiguration());
Mockito.doReturn(rl).when
(cConnection).locateRegion((TableName)Mockito.any(), (byte[])Mockito.any(),
Mockito.anyBoolean(), Mockito.anyBoolean(), Mockito.anyInt());
// Now shove our HRI implementation into the spied-upon connection.
Mockito.doReturn(implementation).
when(connection).getClient(Mockito.any(ServerName.class));
// Now start up the catalogtracker with our doctored Connection.
ct = new CatalogTracker(zkw, null, connection, ABORTABLE);
ct.start();
// Scan meta for user tables and verify we got back expected answer.
NavigableMap<HRegionInfo, Result> hris = MetaReader.getServerUserRegions(ct, sn);
assertEquals(1, hris.size());
assertTrue(hris.firstEntry().getKey().equals(HRegionInfo.FIRST_META_REGIONINFO));
assertTrue(Bytes.equals(rowToVerify, hris.firstEntry().getValue().getRow()));
// Finally verify that scan was called four times -- three times
// with exception and then on 4th, 5th and 6th attempt we succeed
Mockito.verify(implementation, Mockito.times(6)).
scan((RpcController)Mockito.any(), (ScanRequest)Mockito.any());
} finally {
if (ct != null) ct.stop();
HConnectionManager.deleteConnection(UTIL.getConfiguration());
zkw.close();
}
}
}

View File

@ -54,7 +54,6 @@ import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotEnabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.master.AssignmentManager;
@ -1764,18 +1763,13 @@ public class TestAdmin {
final byte [] nameBytes = Bytes.toBytes(name);
HTable t = TEST_UTIL.createTable(nameBytes, HConstants.CATALOG_FAMILY);
TEST_UTIL.createMultiRegions(t, HConstants.CATALOG_FAMILY);
CatalogTracker ct = new CatalogTracker(TEST_UTIL.getConfiguration());
ct.start();
try {
HRegionLocation regionLocation = t.getRegionLocation("mmm");
HRegionInfo region = regionLocation.getRegionInfo();
byte[] regionName = region.getRegionName();
Pair<HRegionInfo, ServerName> pair = admin.getRegion(regionName, ct);
Pair<HRegionInfo, ServerName> pair = admin.getRegion(regionName);
assertTrue(Bytes.equals(regionName, pair.getFirst().getRegionName()));
pair = admin.getRegion(region.getEncodedNameAsBytes(), ct);
pair = admin.getRegion(region.getEncodedNameAsBytes());
assertTrue(Bytes.equals(regionName, pair.getFirst().getRegionName()));
} finally {
ct.stop();
}
}
}

View File

@ -24,7 +24,6 @@ import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.reset;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
import java.math.BigDecimal;
import java.util.List;
@ -34,14 +33,13 @@ import java.util.Random;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.StoppableImplementation;
import org.apache.hadoop.hbase.util.Threads;
@ -50,6 +48,7 @@ import org.junit.After;
import org.junit.Assert;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(MediumTests.class)
public class TestMetaScanner {
final Log LOG = LogFactory.getLog(getClass());
@ -127,8 +126,6 @@ public class TestMetaScanner {
TableName.valueOf("testConcurrentMetaScannerAndCatalogJanitor");
final byte[] FAMILY = Bytes.toBytes("family");
TEST_UTIL.createTable(TABLENAME, FAMILY);
final CatalogTracker catalogTracker = mock(CatalogTracker.class);
when(catalogTracker.getConnection()).thenReturn(TEST_UTIL.getHBaseAdmin().getConnection());
class RegionMetaSplitter extends StoppableImplementation implements Runnable {
Random random = new Random();
@ -169,7 +166,8 @@ public class TestMetaScanner {
Bytes.toBytes(midKey),
end);
MetaEditor.splitRegion(catalogTracker, parent, splita, splitb, ServerName.valueOf("fooserver", 1, 0));
MetaTableAccessor.splitRegion(TEST_UTIL.getHBaseAdmin().getConnection(),
parent, splita, splitb, ServerName.valueOf("fooserver", 1, 0));
Threads.sleep(random.nextInt(200));
} catch (Throwable e) {

View File

@ -27,7 +27,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
@ -165,7 +165,7 @@ public class TestScannerTimeout {
Scan scan = new Scan();
scan.setCaching(SCANNER_CACHING);
LOG.info("************ TEST3686A");
MetaReader.fullScanMetaAndPrint(TEST_UTIL.getHBaseCluster().getMaster().getCatalogTracker());
MetaTableAccessor.fullScanMetaAndPrint(TEST_UTIL.getHBaseAdmin().getConnection());
// Set a very high timeout, we want to test what happens when a RS
// fails but the region is recovered before the lease times out.
// Since the RS is already created, this conf is client-side only for

View File

@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.regionserver.HRegion;
@ -152,7 +152,7 @@ public class TestRegionServerObserver {
mergedRegion = rmt.stepsBeforePONR(rs, rs, false);
rmt.prepareMutationsForMerge(mergedRegion.getRegionInfo(), regionA.getRegionInfo(),
regionB.getRegionInfo(), rs.getServerName(), metaEntries);
MetaEditor.mutateMetaTable(rs.getCatalogTracker(), metaEntries);
MetaTableAccessor.mutateMetaTable(rs.getShortCircuitConnection(), metaEntries);
}
@Override

View File

@ -45,10 +45,9 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
@ -474,11 +473,12 @@ public class TestLoadIncrementalHFilesSplitRecovery {
dir = buildBulkFiles(tableName, 3);
// Mess it up by leaving a hole in the hbase:meta
CatalogTracker ct = new CatalogTracker(util.getConfiguration());
List<HRegionInfo> regionInfos = MetaReader.getTableRegions(ct, TableName.valueOf(tableName));
HConnection hConnection = HConnectionManager.getConnection(util.getConfiguration());
List<HRegionInfo> regionInfos = MetaTableAccessor.getTableRegions(
util.getZooKeeperWatcher(), hConnection, TableName.valueOf(tableName));
for (HRegionInfo regionInfo : regionInfos) {
if (Bytes.equals(regionInfo.getStartKey(), HConstants.EMPTY_BYTE_ARRAY)) {
MetaEditor.deleteRegion(ct, regionInfo);
MetaTableAccessor.deleteRegion(hConnection, regionInfo);
break;
}
}

View File

@ -36,8 +36,8 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.executor.ExecutorService;
@ -94,6 +94,7 @@ import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.regionserver.ServerNonceManager;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
@ -275,8 +276,12 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
}
@Override
public CatalogTracker getCatalogTracker() {
// TODO Auto-generated method stub
public HConnection getShortCircuitConnection() {
return null;
}
@Override
public MetaTableLocator getMetaTableLocator() {
return null;
}
@ -313,7 +318,7 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
}
@Override
public void postOpenDeployTasks(HRegion r, CatalogTracker ct)
public void postOpenDeployTasks(HRegion r)
throws KeeperException, IOException {
// TODO Auto-generated method stub
}

View File

@ -33,10 +33,11 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@ -302,7 +303,12 @@ public class TestActiveMasterManager {
}
@Override
public CatalogTracker getCatalogTracker() {
public HConnection getShortCircuitConnection() {
return null;
}
@Override
public MetaTableLocator getMetaTableLocator() {
return null;
}

View File

@ -48,8 +48,7 @@ import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaMockingUtil;
import org.apache.hadoop.hbase.MetaMockingUtil;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
import org.apache.hadoop.hbase.client.Result;
@ -80,6 +79,7 @@ import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@ -123,6 +123,7 @@ public class TestAssignmentManager {
private ServerManager serverManager;
private ZooKeeperWatcher watcher;
private CoordinatedStateManager cp;
private MetaTableLocator mtl;
private LoadBalancer balancer;
private HMaster master;
@ -148,7 +149,6 @@ public class TestAssignmentManager {
this.server = Mockito.mock(Server.class);
Mockito.when(server.getServerName()).thenReturn(ServerName.valueOf("master,1,1"));
Mockito.when(server.getConfiguration()).thenReturn(HTU.getConfiguration());
Mockito.when(server.getCatalogTracker()).thenReturn(null);
this.watcher =
new ZooKeeperWatcher(HTU.getConfiguration(), "mockedServer", this.server, true);
Mockito.when(server.getZooKeeper()).thenReturn(this.watcher);
@ -159,7 +159,18 @@ public class TestAssignmentManager {
cp.initialize(this.server);
cp.start();
mtl = Mockito.mock(MetaTableLocator.class);
Mockito.when(server.getCoordinatedStateManager()).thenReturn(cp);
Mockito.when(server.getMetaTableLocator()).thenReturn(mtl);
// Get a connection w/ mocked up common methods.
HConnection connection =
HConnectionTestingUtility.getMockedConnection(HTU.getConfiguration());
// Make it so we can get a catalogtracker from servermanager.. .needed
// down in guts of server shutdown handler.
Mockito.when(server.getShortCircuitConnection()).thenReturn(connection);
// Mock a ServerManager. Say server SERVERNAME_{A,B} are online. Also
// make it so if close or open, we return 'success'.
@ -385,12 +396,11 @@ public class TestAssignmentManager {
ExecutorService executor = startupMasterExecutor("testBalanceExecutor");
// We need a mocked catalog tracker.
CatalogTracker ct = Mockito.mock(CatalogTracker.class);
LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(server
.getConfiguration());
// Create an AM.
AssignmentManager am = new AssignmentManager(this.server,
this.serverManager, ct, balancer, executor, null, master.getTableLockManager());
this.serverManager, balancer, executor, null, master.getTableLockManager());
am.failoverCleanupDone.set(true);
try {
// Make sure our new AM gets callbacks; once registered, can't unregister.
@ -459,13 +469,11 @@ public class TestAssignmentManager {
// handling zk callbacks.
ExecutorService executor = startupMasterExecutor("testShutdownHandler");
// We need a mocked catalog tracker.
CatalogTracker ct = Mockito.mock(CatalogTracker.class);
// Create an AM.
AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(
this.server, this.serverManager);
try {
processServerShutdownHandler(ct, am, false);
processServerShutdownHandler(am, false);
} finally {
executor.shutdown();
am.shutdown();
@ -513,7 +521,6 @@ public class TestAssignmentManager {
// handling zk callbacks.
ExecutorService executor = startupMasterExecutor("testSSHWhenSplitRegionInProgress");
// We need a mocked catalog tracker.
CatalogTracker ct = Mockito.mock(CatalogTracker.class);
ZKAssign.deleteAllNodes(this.watcher);
// Create an AM.
@ -533,7 +540,7 @@ public class TestAssignmentManager {
ZKUtil.createAndWatch(this.watcher, node, data.toByteArray());
try {
processServerShutdownHandler(ct, am, regionSplitDone);
processServerShutdownHandler(am, regionSplitDone);
// check znode deleted or not.
// In both cases the znode should be deleted.
@ -561,14 +568,12 @@ public class TestAssignmentManager {
// Create and startup an executor. This is used by AssignmentManager
// handling zk callbacks.
ExecutorService executor = startupMasterExecutor("testSSHWhenDisableTableInProgress");
// We need a mocked catalog tracker.
CatalogTracker ct = Mockito.mock(CatalogTracker.class);
LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(server.getConfiguration());
ZKAssign.deleteAllNodes(this.watcher);
// Create an AM.
AssignmentManager am = new AssignmentManager(this.server,
this.serverManager, ct, balancer, executor, null, master.getTableLockManager());
this.serverManager, balancer, executor, null, master.getTableLockManager());
// adding region to regions and servers maps.
am.regionOnline(REGIONINFO, SERVERNAME_A);
// adding region in pending close.
@ -590,7 +595,7 @@ public class TestAssignmentManager {
ZKUtil.createAndWatch(this.watcher, node, data.toByteArray());
try {
processServerShutdownHandler(ct, am, false);
processServerShutdownHandler(am, false);
// check znode deleted or not.
// In both cases the znode should be deleted.
assertTrue("The znode should be deleted.", ZKUtil.checkExists(this.watcher, node) == -1);
@ -610,7 +615,7 @@ public class TestAssignmentManager {
}
}
private void processServerShutdownHandler(CatalogTracker ct, AssignmentManager am, boolean splitRegion)
private void processServerShutdownHandler(AssignmentManager am, boolean splitRegion)
throws IOException, ServiceException {
// Make sure our new AM gets callbacks; once registered, can't unregister.
// Thats ok because we make a new zk watcher for each test.
@ -655,8 +660,7 @@ public class TestAssignmentManager {
// Make it so we can get a catalogtracker from servermanager.. .needed
// down in guts of server shutdown handler.
Mockito.when(ct.getConnection()).thenReturn(connection);
Mockito.when(this.server.getCatalogTracker()).thenReturn(ct);
Mockito.when(this.server.getShortCircuitConnection()).thenReturn(connection);
// Now make a server shutdown handler instance and invoke process.
// Have it that SERVERNAME_A died.
@ -704,12 +708,11 @@ public class TestAssignmentManager {
// default null.
Mockito.when(this.serverManager.sendRegionClose(SERVERNAME_A, hri, -1)).thenReturn(true);
// Need a mocked catalog tracker.
CatalogTracker ct = Mockito.mock(CatalogTracker.class);
LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(server
.getConfiguration());
// Create an AM.
AssignmentManager am = new AssignmentManager(this.server,
this.serverManager, ct, balancer, null, null, master.getTableLockManager());
this.serverManager, balancer, null, null, master.getTableLockManager());
try {
// First make sure my mock up basically works. Unassign a region.
unassign(am, SERVERNAME_A, hri);
@ -885,7 +888,6 @@ public class TestAssignmentManager {
am.getRegionStates().logSplit(SERVERNAME_A); // Assume log splitting is done
am.getRegionStates().createRegionState(REGIONINFO);
am.gate.set(false);
CatalogTracker ct = Mockito.mock(CatalogTracker.class);
BaseCoordinatedStateManager cp = new ZkCoordinatedStateManager();
cp.initialize(server);
@ -899,7 +901,7 @@ public class TestAssignmentManager {
assertFalse(am.processRegionsInTransition(rt, REGIONINFO, orc, zkOrd));
am.getTableStateManager().setTableState(REGIONINFO.getTable(), Table.State.ENABLED);
processServerShutdownHandler(ct, am, false);
processServerShutdownHandler(am, false);
// Waiting for the assignment to get completed.
while (!am.gate.get()) {
Thread.sleep(10);
@ -934,6 +936,13 @@ public class TestAssignmentManager {
Server server = new HMaster(HTU.getConfiguration(), csm);
AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(server,
this.serverManager);
Whitebox.setInternalState(server, "metaTableLocator", Mockito.mock(MetaTableLocator.class));
// Make it so we can get a catalogtracker from servermanager.. .needed
// down in guts of server shutdown handler.
Whitebox.setInternalState(server, "shortCircuitConnection", am.getConnection());
AtomicBoolean gate = new AtomicBoolean(false);
if (balancer instanceof MockedLoadBalancer) {
((MockedLoadBalancer) balancer).setGateVariable(gate);
@ -982,12 +991,19 @@ public class TestAssignmentManager {
Whitebox.setInternalState(server, "serverManager", this.serverManager);
AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(server,
this.serverManager);
Whitebox.setInternalState(server, "metaTableLocator", Mockito.mock(MetaTableLocator.class));
// Make it so we can get a catalogtracker from servermanager.. .needed
// down in guts of server shutdown handler.
Whitebox.setInternalState(server, "shortCircuitConnection", am.getConnection());
try {
// set table in enabling state.
am.getTableStateManager().setTableState(REGIONINFO.getTable(),
Table.State.ENABLING);
new EnableTableHandler(server, REGIONINFO.getTable(),
am.getCatalogTracker(), am, new NullTableLockManager(), true).prepare()
am, new NullTableLockManager(), true).prepare()
.process();
assertEquals("Number of assignments should be 1.", 1, assignmentCount);
assertTrue("Table should be enabled.",
@ -1024,6 +1040,13 @@ public class TestAssignmentManager {
Whitebox.setInternalState(server, "serverManager", this.serverManager);
AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(server,
this.serverManager);
Whitebox.setInternalState(server, "metaTableLocator", Mockito.mock(MetaTableLocator.class));
// Make it so we can get a catalogtracker from servermanager.. .needed
// down in guts of server shutdown handler.
Whitebox.setInternalState(server, "shortCircuitConnection", am.getConnection());
try {
TableName tableName = TableName.valueOf("dummyTable");
// set table in enabling state.
@ -1043,8 +1066,6 @@ public class TestAssignmentManager {
@Test
public void testSSHTimesOutOpeningRegionTransition()
throws KeeperException, IOException, CoordinatedStateException, ServiceException {
// We need a mocked catalog tracker.
CatalogTracker ct = Mockito.mock(CatalogTracker.class);
// Create an AM.
AssignmentManagerWithExtrasForTesting am =
setUpMockedAssignmentManager(this.server, this.serverManager);
@ -1061,7 +1082,7 @@ public class TestAssignmentManager {
try {
am.assignInvoked = false;
processServerShutdownHandler(ct, am, false);
processServerShutdownHandler(am, false);
assertTrue(am.assignInvoked);
} finally {
am.getRegionStates().regionsInTransition.remove(REGIONINFO.getEncodedName());
@ -1155,8 +1176,6 @@ public class TestAssignmentManager {
private AssignmentManagerWithExtrasForTesting setUpMockedAssignmentManager(final Server server,
final ServerManager manager) throws IOException, KeeperException,
ServiceException, CoordinatedStateException {
// We need a mocked catalog tracker. Its used by our AM instance.
CatalogTracker ct = Mockito.mock(CatalogTracker.class);
// Make an RS Interface implementation. Make it so a scanner can go against
// it and a get to return the single region, REGIONINFO, this test is
// messing with. Needed when "new master" joins cluster. AM will try and
@ -1200,12 +1219,11 @@ public class TestAssignmentManager {
getMockedConnectionAndDecorate(HTU.getConfiguration(), null,
ri, SERVERNAME_B, REGIONINFO);
// Make it so we can get the connection from our mocked catalogtracker
Mockito.when(ct.getConnection()).thenReturn(connection);
// Create and startup an executor. Used by AM handling zk callbacks.
ExecutorService executor = startupMasterExecutor("mockedAMExecutor");
this.balancer = LoadBalancerFactory.getLoadBalancer(server.getConfiguration());
AssignmentManagerWithExtrasForTesting am = new AssignmentManagerWithExtrasForTesting(
server, manager, ct, this.balancer, executor, new NullTableLockManager());
server, connection, manager, this.balancer, executor, new NullTableLockManager());
return am;
}
@ -1215,20 +1233,19 @@ public class TestAssignmentManager {
class AssignmentManagerWithExtrasForTesting extends AssignmentManager {
// Keep a reference so can give it out below in {@link #getExecutorService}
private final ExecutorService es;
// Ditto for ct
private final CatalogTracker ct;
boolean processRITInvoked = false;
boolean assignInvoked = false;
AtomicBoolean gate = new AtomicBoolean(true);
private HConnection connection;
public AssignmentManagerWithExtrasForTesting(
final Server master, final ServerManager serverManager,
final CatalogTracker catalogTracker, final LoadBalancer balancer,
final Server master, HConnection connection, final ServerManager serverManager,
final LoadBalancer balancer,
final ExecutorService service, final TableLockManager tableLockManager)
throws KeeperException, IOException, CoordinatedStateException {
super(master, serverManager, catalogTracker, balancer, service, null, tableLockManager);
super(master, serverManager, balancer, service, null, tableLockManager);
this.es = service;
this.ct = catalogTracker;
this.connection = connection;
}
@Override
@ -1282,11 +1299,11 @@ public class TestAssignmentManager {
return this.es;
}
/**
* @return CatalogTracker used by this AM (Its a mock).
/*
* Convenient method to retrieve mocked up connection
*/
CatalogTracker getCatalogTracker() {
return this.ct;
HConnection getConnection() {
return this.connection;
}
}
@ -1333,12 +1350,11 @@ public class TestAssignmentManager {
// Region to use in test.
final HRegionInfo hri = HRegionInfo.FIRST_META_REGIONINFO;
// Need a mocked catalog tracker.
CatalogTracker ct = Mockito.mock(CatalogTracker.class);
LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(
server.getConfiguration());
// Create an AM.
AssignmentManager am = new AssignmentManager(this.server,
this.serverManager, ct, balancer, null, null, master.getTableLockManager());
this.serverManager, balancer, null, null, master.getTableLockManager());
RegionStates regionStates = am.getRegionStates();
try {
// First set the state of the region to merging
@ -1367,14 +1383,12 @@ public class TestAssignmentManager {
CoordinatedStateException {
// Region to use in test.
final HRegionInfo hri = HRegionInfo.FIRST_META_REGIONINFO;
// Need a mocked catalog tracker.
CatalogTracker ct = Mockito.mock(CatalogTracker.class);
LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(
server.getConfiguration());
final AtomicBoolean zkEventProcessed = new AtomicBoolean(false);
// Create an AM.
AssignmentManager am = new AssignmentManager(this.server,
this.serverManager, ct, balancer, null, null, master.getTableLockManager()) {
this.serverManager, balancer, null, null, master.getTableLockManager()) {
@Override
void handleRegion(final RegionTransition rt, OpenRegionCoordination coordination,
@ -1415,9 +1429,8 @@ public class TestAssignmentManager {
*/
@Test
public void testBalanceRegionOfDeletedTable() throws Exception {
CatalogTracker ct = Mockito.mock(CatalogTracker.class);
AssignmentManager am = new AssignmentManager(this.server, this.serverManager,
ct, balancer, null, null, master.getTableLockManager());
balancer, null, null, master.getTableLockManager());
RegionStates regionStates = am.getRegionStates();
HRegionInfo hri = REGIONINFO;
regionStates.createRegionState(hri);
@ -1443,12 +1456,11 @@ public class TestAssignmentManager {
this.server.getConfiguration().setInt("hbase.assignment.maximum.attempts", 100);
HRegionInfo hri = REGIONINFO;
CatalogTracker ct = Mockito.mock(CatalogTracker.class);
LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(
server.getConfiguration());
// Create an AM.
AssignmentManager am = new AssignmentManager(this.server,
this.serverManager, ct, balancer, null, null, master.getTableLockManager());
this.serverManager, balancer, null, null, master.getTableLockManager());
RegionStates regionStates = am.getRegionStates();
try {
am.regionPlans.put(REGIONINFO.getEncodedName(),

View File

@ -46,7 +46,7 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
@ -122,7 +122,7 @@ public class TestAssignmentManagerOnCluster {
HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
HRegionInfo hri = new HRegionInfo(
desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
MetaEditor.addRegionToMeta(meta, hri);
MetaTableAccessor.addRegionToMeta(meta, hri);
HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
master.assignRegion(hri);
@ -166,7 +166,7 @@ public class TestAssignmentManagerOnCluster {
HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
final HRegionInfo hri = new HRegionInfo(
desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
MetaEditor.addRegionToMeta(meta, hri);
MetaTableAccessor.addRegionToMeta(meta, hri);
master = TEST_UTIL.getHBaseCluster().getMaster();
Set<ServerName> onlineServers = master.serverManager.getOnlineServers().keySet();
@ -380,7 +380,7 @@ public class TestAssignmentManagerOnCluster {
HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
HRegionInfo hri = new HRegionInfo(
desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
MetaEditor.addRegionToMeta(meta, hri);
MetaTableAccessor.addRegionToMeta(meta, hri);
HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
master.assignRegion(hri);
@ -427,7 +427,7 @@ public class TestAssignmentManagerOnCluster {
HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
HRegionInfo hri = new HRegionInfo(
desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
MetaEditor.addRegionToMeta(meta, hri);
MetaTableAccessor.addRegionToMeta(meta, hri);
HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
master.assignRegion(hri);
@ -471,7 +471,7 @@ public class TestAssignmentManagerOnCluster {
HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
HRegionInfo hri = new HRegionInfo(
desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
MetaEditor.addRegionToMeta(meta, hri);
MetaTableAccessor.addRegionToMeta(meta, hri);
MyLoadBalancer.controledRegion = hri.getEncodedName();
@ -513,7 +513,7 @@ public class TestAssignmentManagerOnCluster {
HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
HRegionInfo hri = new HRegionInfo(
desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
MetaEditor.addRegionToMeta(meta, hri);
MetaTableAccessor.addRegionToMeta(meta, hri);
FileSystem fs = FileSystem.get(conf);
Path tableDir= FSUtils.getTableDir(FSUtils.getRootDir(conf), table);
@ -611,7 +611,7 @@ public class TestAssignmentManagerOnCluster {
HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
HRegionInfo hri = new HRegionInfo(
desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
MetaEditor.addRegionToMeta(meta, hri);
MetaTableAccessor.addRegionToMeta(meta, hri);
HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
master.assignRegion(hri);
@ -667,7 +667,7 @@ public class TestAssignmentManagerOnCluster {
HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
HRegionInfo hri = new HRegionInfo(
desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
MetaEditor.addRegionToMeta(meta, hri);
MetaTableAccessor.addRegionToMeta(meta, hri);
meta.close();
MyRegionObserver.postOpenEnabled.set(true);
@ -740,7 +740,7 @@ public class TestAssignmentManagerOnCluster {
HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
HRegionInfo hri = new HRegionInfo(
desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
MetaEditor.addRegionToMeta(meta, hri);
MetaTableAccessor.addRegionToMeta(meta, hri);
// Assign the region
master = (MyMaster)cluster.getMaster();
@ -814,7 +814,7 @@ public class TestAssignmentManagerOnCluster {
HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
HRegionInfo hri = new HRegionInfo(
desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
MetaEditor.addRegionToMeta(meta, hri);
MetaTableAccessor.addRegionToMeta(meta, hri);
// Assign the region
master = (MyMaster)cluster.getMaster();

View File

@ -50,8 +50,7 @@ import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaMockingUtil;
import org.apache.hadoop.hbase.MetaMockingUtil;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
@ -75,6 +74,7 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
import org.apache.hadoop.hbase.util.Triple;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@ -97,7 +97,6 @@ public class TestCatalogJanitor {
class MockServer implements Server {
private final HConnection connection;
private final Configuration c;
private final CatalogTracker ct;
MockServer(final HBaseTestingUtility htu)
throws NotAllMetaRegionsOnlineException, IOException, InterruptedException {
@ -137,16 +136,18 @@ public class TestCatalogJanitor {
FileSystem fs = FileSystem.get(this.c);
Path rootdir = FSUtils.getRootDir(this.c);
FSUtils.setRootDir(this.c, rootdir);
this.ct = Mockito.mock(CatalogTracker.class);
AdminProtos.AdminService.BlockingInterface hri =
Mockito.mock(AdminProtos.AdminService.BlockingInterface.class);
Mockito.when(this.ct.getConnection()).thenReturn(this.connection);
Mockito.when(ct.waitForMetaServerConnection(Mockito.anyLong())).thenReturn(hri);
}
@Override
public CatalogTracker getCatalogTracker() {
return this.ct;
public HConnection getShortCircuitConnection() {
return this.connection;
}
@Override
public MetaTableLocator getMetaTableLocator() {
return null;
}
@Override
@ -186,9 +187,6 @@ public class TestCatalogJanitor {
@Override
public void stop(String why) {
if (this.ct != null) {
this.ct.stop();
}
if (this.connection != null) {
HConnectionManager.deleteConnection(this.connection.getConfiguration());
}
@ -254,7 +252,12 @@ public class TestCatalogJanitor {
}
@Override
public CatalogTracker getCatalogTracker() {
public MetaTableLocator getMetaTableLocator() {
return null;
}
@Override
public HConnection getShortCircuitConnection() {
return null;
}

View File

@ -26,8 +26,9 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@ -42,7 +43,12 @@ public class TestClockSkewDetection {
final Configuration conf = HBaseConfiguration.create();
ServerManager sm = new ServerManager(new Server() {
@Override
public CatalogTracker getCatalogTracker() {
public HConnection getShortCircuitConnection() {
return null;
}
@Override
public MetaTableLocator getMetaTableLocator() {
return null;
}

View File

@ -21,7 +21,7 @@ package org.apache.hadoop.hbase.master;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
@ -74,7 +74,8 @@ public class TestMaster {
ht.close();
List<Pair<HRegionInfo, ServerName>> tableRegions =
MetaReader.getTableRegionsAndLocations(m.getCatalogTracker(), TABLENAME);
MetaTableAccessor.getTableRegionsAndLocations(m.getZooKeeper(),
m.getShortCircuitConnection(), TABLENAME);
LOG.info("Regions after load: " + Joiner.on(',').join(tableRegions));
assertEquals(1, tableRegions.size());
assertArrayEquals(HConstants.EMPTY_START_ROW,
@ -91,8 +92,8 @@ public class TestMaster {
Thread.sleep(100);
}
LOG.info("Making sure we can call getTableRegions while opening");
tableRegions = MetaReader.getTableRegionsAndLocations(m.getCatalogTracker(),
TABLENAME, false);
tableRegions = MetaTableAccessor.getTableRegionsAndLocations(m.getZooKeeper(),
m.getShortCircuitConnection(), TABLENAME, false);
LOG.info("Regions: " + Joiner.on(',').join(tableRegions));
// We have three regions because one is split-in-progress
@ -102,7 +103,7 @@ public class TestMaster {
m.getTableRegionForRow(TABLENAME, Bytes.toBytes("cde"));
LOG.info("Result is: " + pair);
Pair<HRegionInfo, ServerName> tableRegionFromName =
MetaReader.getRegion(m.getCatalogTracker(),
MetaTableAccessor.getRegion(m.getShortCircuitConnection(),
pair.getFirst().getRegionName());
assertEquals(tableRegionFromName.getFirst(), pair.getFirst());
}

View File

@ -49,7 +49,7 @@ import org.apache.hadoop.hbase.RegionTransition;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableStateManager;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.executor.EventType;
@ -67,7 +67,7 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZKTableStateManager;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@ -298,8 +298,10 @@ public class TestMasterFailover {
zkw, HRegionInfo.FIRST_META_REGIONINFO, hrs.getServerName());
ProtobufUtil.openRegion(hrs.getRSRpcServices(),
hrs.getServerName(), HRegionInfo.FIRST_META_REGIONINFO);
MetaTableLocator mtl = new MetaTableLocator();
while (true) {
ServerName sn = MetaRegionTracker.getMetaRegionLocation(zkw);
ServerName sn = mtl.getMetaRegionLocation(zkw);
if (sn != null && sn.equals(hrs.getServerName())) {
break;
}
@ -689,8 +691,10 @@ public class TestMasterFailover {
zkw, HRegionInfo.FIRST_META_REGIONINFO, hrs.getServerName());
ProtobufUtil.openRegion(hrs.getRSRpcServices(),
hrs.getServerName(), HRegionInfo.FIRST_META_REGIONINFO);
MetaTableLocator mtl = new MetaTableLocator();
while (true) {
ServerName sn = MetaRegionTracker.getMetaRegionLocation(zkw);
ServerName sn = mtl.getMetaRegionLocation(zkw);
if (sn != null && sn.equals(hrs.getServerName())) {
break;
}
@ -1234,7 +1238,7 @@ public class TestMasterFailover {
HRegionInfo hriOffline = new HRegionInfo(offlineTable.getTableName(), null, null);
createRegion(hriOffline, rootdir, conf, offlineTable);
MetaEditor.addRegionToMeta(master.getCatalogTracker(), hriOffline);
MetaTableAccessor.addRegionToMeta(master.getShortCircuitConnection(), hriOffline);
log("Regions in hbase:meta and namespace have been created");

View File

@ -44,8 +44,7 @@ import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaMockingUtil;
import org.apache.hadoop.hbase.MetaMockingUtil;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
import org.apache.hadoop.hbase.client.Result;
@ -54,7 +53,7 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@ -159,7 +158,7 @@ public class TestMasterNoCluster {
final MockRegionServer rs2 = new MockRegionServer(conf, sn2);
// 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());
MetaTableLocator.setMetaLocation(rs0.getZooKeeper(), rs0.getServerName());
final TableName tableName = TableName.valueOf("t");
Result [] results = new Result [] {
MetaMockingUtil.getMetaTableRowResult(
@ -206,15 +205,17 @@ public class TestMasterNoCluster {
}
@Override
protected CatalogTracker createCatalogTracker() throws IOException {
// Insert a mock for the connection used by the CatalogTracker. Any
// regionserver should do. Use TESTUTIL.getConfiguration rather than
public HConnection getShortCircuitConnection() {
// Insert a mock for the connection, use TESTUTIL.getConfiguration rather than
// the conf from the master; the conf will already have an HConnection
// associate so the below mocking of a connection will fail.
HConnection connection =
HConnectionTestingUtility.getMockedConnectionAndDecorate(TESTUTIL.getConfiguration(),
rs0, rs0, rs0.getServerName(), HRegionInfo.FIRST_META_REGIONINFO);
return new CatalogTracker(getZooKeeper(), getConfiguration(), connection, this);
try {
return HConnectionTestingUtility.getMockedConnectionAndDecorate(
TESTUTIL.getConfiguration(), rs0, rs0, rs0.getServerName(),
HRegionInfo.FIRST_META_REGIONINFO);
} catch (IOException e) {
return null;
}
}
@Override
@ -285,15 +286,17 @@ public class TestMasterNoCluster {
}
@Override
protected CatalogTracker createCatalogTracker() throws IOException {
// Insert a mock for the connection used by the CatalogTracker. Any
// regionserver should do. Use TESTUTIL.getConfiguration rather than
public HConnection getShortCircuitConnection() {
// Insert a mock for the connection, use TESTUTIL.getConfiguration rather than
// the conf from the master; the conf will already have an HConnection
// associate so the below mocking of a connection will fail.
HConnection connection =
HConnectionTestingUtility.getMockedConnectionAndDecorate(TESTUTIL.getConfiguration(),
rs0, rs0, rs0.getServerName(), HRegionInfo.FIRST_META_REGIONINFO);
return new CatalogTracker(getZooKeeper(), getConfiguration(), connection, this);
try {
return HConnectionTestingUtility.getMockedConnectionAndDecorate(
TESTUTIL.getConfiguration(), rs0, rs0, rs0.getServerName(),
HRegionInfo.FIRST_META_REGIONINFO);
} catch (IOException e) {
return null;
}
}
@Override

View File

@ -42,11 +42,11 @@ import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.catalog.MetaReader.Visitor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.MetaTableAccessor.Visitor;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result;
@ -62,10 +62,11 @@ public class TestMasterOperationsForRegionReplicas {
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private static HBaseAdmin admin;
private static int numSlaves = 2;
private static Configuration conf;
@BeforeClass
public static void setupBeforeClass() throws Exception {
Configuration conf = TEST_UTIL.getConfiguration();
conf = TEST_UTIL.getConfiguration();
conf.setBoolean("hbase.tests.use.shortcircuit.reads", false);
TEST_UTIL.startMiniCluster(numSlaves);
admin = new HBaseAdmin(conf);
@ -90,9 +91,9 @@ public class TestMasterOperationsForRegionReplicas {
desc.addFamily(new HColumnDescriptor("family"));
admin.createTable(desc, Bytes.toBytes("A"), Bytes.toBytes("Z"), numRegions);
CatalogTracker ct = new CatalogTracker(TEST_UTIL.getConfiguration());
validateNumberOfRowsInMeta(table, numRegions, ct);
List<HRegionInfo> hris = MetaReader.getTableRegions(ct, table);
validateNumberOfRowsInMeta(table, numRegions, admin.getConnection());
List<HRegionInfo> hris = MetaTableAccessor.getTableRegions(TEST_UTIL.getZooKeeperWatcher(),
admin.getConnection(), table);
assert(hris.size() == numRegions * numReplica);
} finally {
admin.disableTable(table);
@ -111,10 +112,10 @@ public class TestMasterOperationsForRegionReplicas {
desc.addFamily(new HColumnDescriptor("family"));
admin.createTable(desc, Bytes.toBytes("A"), Bytes.toBytes("Z"), numRegions);
TEST_UTIL.waitTableEnabled(table.getName());
CatalogTracker ct = new CatalogTracker(TEST_UTIL.getConfiguration());
validateNumberOfRowsInMeta(table, numRegions, ct);
validateNumberOfRowsInMeta(table, numRegions, admin.getConnection());
List<HRegionInfo> hris = MetaReader.getTableRegions(ct, table);
List<HRegionInfo> hris = MetaTableAccessor.getTableRegions(TEST_UTIL.getZooKeeperWatcher(),
admin.getConnection(), table);
assert(hris.size() == numRegions * numReplica);
// check that the master created expected number of RegionState objects
for (int i = 0; i < numRegions; i++) {
@ -126,10 +127,10 @@ public class TestMasterOperationsForRegionReplicas {
}
}
List<Result> metaRows = MetaReader.fullScan(ct);
List<Result> metaRows = MetaTableAccessor.fullScanOfMeta(admin.getConnection());
int numRows = 0;
for (Result result : metaRows) {
RegionLocations locations = MetaReader.getRegionLocations(result);
RegionLocations locations = MetaTableAccessor.getRegionLocations(result);
HRegionInfo hri = locations.getRegionLocation().getRegionInfo();
if (!hri.getTable().equals(table)) continue;
numRows += 1;
@ -142,7 +143,8 @@ public class TestMasterOperationsForRegionReplicas {
// The same verification of the meta as above but with the SnapshotOfRegionAssignmentFromMeta
// class
validateFromSnapshotFromMeta(TEST_UTIL, table, numRegions, numReplica, ct);
validateFromSnapshotFromMeta(TEST_UTIL, table, numRegions, numReplica,
admin.getConnection());
// Now kill the master, restart it and see if the assignments are kept
ServerName master = TEST_UTIL.getHBaseClusterInterface().getClusterStatus().getMaster();
@ -158,7 +160,8 @@ public class TestMasterOperationsForRegionReplicas {
assert (state != null);
}
}
validateFromSnapshotFromMeta(TEST_UTIL, table, numRegions, numReplica, ct);
validateFromSnapshotFromMeta(TEST_UTIL, table, numRegions, numReplica,
admin.getConnection());
// Now shut the whole cluster down, and verify the assignments are kept so that the
// availability constraints are met.
@ -166,16 +169,19 @@ public class TestMasterOperationsForRegionReplicas {
TEST_UTIL.shutdownMiniHBaseCluster();
TEST_UTIL.startMiniHBaseCluster(1, numSlaves);
TEST_UTIL.waitTableEnabled(table.getName());
ct = new CatalogTracker(TEST_UTIL.getConfiguration());
validateFromSnapshotFromMeta(TEST_UTIL, table, numRegions, numReplica, ct);
admin.close();
admin = new HBaseAdmin(conf);
validateFromSnapshotFromMeta(TEST_UTIL, table, numRegions, numReplica,
admin.getConnection());
// Now shut the whole cluster down, and verify regions are assigned even if there is only
// one server running
TEST_UTIL.shutdownMiniHBaseCluster();
TEST_UTIL.startMiniHBaseCluster(1, 1);
TEST_UTIL.waitTableEnabled(table.getName());
ct = new CatalogTracker(TEST_UTIL.getConfiguration());
validateSingleRegionServerAssignment(ct, numRegions, numReplica);
admin.close();
admin = new HBaseAdmin(conf);
validateSingleRegionServerAssignment(admin.getConnection(), numRegions, numReplica);
for (int i = 1; i < numSlaves; i++) { //restore the cluster
TEST_UTIL.getMiniHBaseCluster().startRegionServer();
}
@ -202,7 +208,8 @@ public class TestMasterOperationsForRegionReplicas {
.getAssignmentManager().getRegionStates().getRegionsOfTable(table);
assert(regions.size() == numRegions * numReplica);
//also make sure the meta table has the replica locations removed
hris = MetaReader.getTableRegions(ct, table);
hris = MetaTableAccessor.getTableRegions(TEST_UTIL.getZooKeeperWatcher(),
admin.getConnection(), table);
assert(hris.size() == numRegions * numReplica);
//just check that the number of default replica regions in the meta table are the same
//as the number of regions the table was created with, and the count of the
@ -236,21 +243,24 @@ public class TestMasterOperationsForRegionReplicas {
desc.addFamily(new HColumnDescriptor("family"));
admin.createTable(desc, Bytes.toBytes("A"), Bytes.toBytes("Z"), numRegions);
TEST_UTIL.waitTableEnabled(table.getName());
CatalogTracker ct = new CatalogTracker(TEST_UTIL.getConfiguration());
Set<byte[]> tableRows = new HashSet<byte[]>();
List<HRegionInfo> hris = MetaReader.getTableRegions(ct, table);
List<HRegionInfo> hris = MetaTableAccessor.getTableRegions(TEST_UTIL.getZooKeeperWatcher(),
admin.getConnection(), table);
for (HRegionInfo hri : hris) {
tableRows.add(hri.getRegionName());
}
admin.disableTable(table);
// now delete one replica info from all the rows
// this is to make the meta appear to be only partially updated
HTable metaTable = new HTable(TableName.META_TABLE_NAME, ct.getConnection());
HTable metaTable = new HTable(TableName.META_TABLE_NAME, admin.getConnection());
for (byte[] row : tableRows) {
Delete deleteOneReplicaLocation = new Delete(row);
deleteOneReplicaLocation.deleteColumns(HConstants.CATALOG_FAMILY, MetaReader.getServerColumn(1));
deleteOneReplicaLocation.deleteColumns(HConstants.CATALOG_FAMILY, MetaReader.getSeqNumColumn(1));
deleteOneReplicaLocation.deleteColumns(HConstants.CATALOG_FAMILY, MetaReader.getStartCodeColumn(1));
deleteOneReplicaLocation.deleteColumns(HConstants.CATALOG_FAMILY,
MetaTableAccessor.getServerColumn(1));
deleteOneReplicaLocation.deleteColumns(HConstants.CATALOG_FAMILY,
MetaTableAccessor.getSeqNumColumn(1));
deleteOneReplicaLocation.deleteColumns(HConstants.CATALOG_FAMILY,
MetaTableAccessor.getStartCodeColumn(1));
metaTable.delete(deleteOneReplicaLocation);
}
metaTable.close();
@ -275,8 +285,8 @@ public class TestMasterOperationsForRegionReplicas {
return strBuf.toString();
}
private void validateNumberOfRowsInMeta(final TableName table, int numRegions, CatalogTracker ct)
throws IOException {
private void validateNumberOfRowsInMeta(final TableName table, int numRegions,
HConnection hConnection) throws IOException {
assert(admin.tableExists(table));
final AtomicInteger count = new AtomicInteger();
Visitor visitor = new Visitor() {
@ -286,13 +296,14 @@ public class TestMasterOperationsForRegionReplicas {
return true;
}
};
MetaReader.fullScan(ct, visitor);
MetaTableAccessor.fullScan(hConnection, visitor);
assert(count.get() == numRegions);
}
private void validateFromSnapshotFromMeta(HBaseTestingUtility util, TableName table, int numRegions,
int numReplica, CatalogTracker ct) throws IOException {
SnapshotOfRegionAssignmentFromMeta snapshot = new SnapshotOfRegionAssignmentFromMeta(ct);
private void validateFromSnapshotFromMeta(HBaseTestingUtility util, TableName table,
int numRegions, int numReplica, HConnection hConnection) throws IOException {
SnapshotOfRegionAssignmentFromMeta snapshot = new SnapshotOfRegionAssignmentFromMeta(
hConnection);
snapshot.initialize();
Map<HRegionInfo, ServerName> regionToServerMap = snapshot.getRegionToRegionServerMap();
assert(regionToServerMap.size() == numRegions * numReplica + 1); //'1' for the namespace
@ -316,9 +327,10 @@ public class TestMasterOperationsForRegionReplicas {
}
}
private void validateSingleRegionServerAssignment(CatalogTracker ct, int numRegions,
private void validateSingleRegionServerAssignment(HConnection hConnection, int numRegions,
int numReplica) throws IOException {
SnapshotOfRegionAssignmentFromMeta snapshot = new SnapshotOfRegionAssignmentFromMeta(ct);
SnapshotOfRegionAssignmentFromMeta snapshot = new SnapshotOfRegionAssignmentFromMeta(
hConnection);
snapshot.initialize();
Map<HRegionInfo, ServerName> regionToServerMap = snapshot.getRegionToRegionServerMap();
assertEquals(regionToServerMap.size(), numRegions * numReplica + 1); //'1' for the namespace

View File

@ -29,7 +29,6 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.client.MetaScanner;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.util.Bytes;
@ -159,8 +158,8 @@ public class TestRestartCluster {
// We don't have to use SnapshotOfRegionAssignmentFromMeta.
// We use it here because AM used to use it to load all user region placements
CatalogTracker ct = new CatalogTracker(UTIL.getConfiguration());
SnapshotOfRegionAssignmentFromMeta snapshot = new SnapshotOfRegionAssignmentFromMeta(ct);
SnapshotOfRegionAssignmentFromMeta snapshot = new SnapshotOfRegionAssignmentFromMeta(
master.getShortCircuitConnection());
snapshot.initialize();
Map<HRegionInfo, ServerName> regionToRegionServerMap
= snapshot.getRegionToRegionServerMap();
@ -226,8 +225,7 @@ public class TestRestartCluster {
Threads.sleep(100);
}
ct = new CatalogTracker(UTIL.getConfiguration());
snapshot =new SnapshotOfRegionAssignmentFromMeta(ct);
snapshot =new SnapshotOfRegionAssignmentFromMeta(master.getShortCircuitConnection());
snapshot.initialize();
Map<HRegionInfo, ServerName> newRegionToRegionServerMap =
snapshot.getRegionToRegionServerMap();

View File

@ -35,9 +35,10 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.util.EnvironmentEdge;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.AfterClass;
import org.junit.BeforeClass;
@ -215,7 +216,12 @@ public class TestHFileCleaner {
}
@Override
public CatalogTracker getCatalogTracker() {
public HConnection getShortCircuitConnection() {
return null;
}
@Override
public MetaTableLocator getMetaTableLocator() {
return null;
}

View File

@ -32,12 +32,13 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@ -151,7 +152,12 @@ public class TestHFileLinkCleaner {
}
@Override
public CatalogTracker getCatalogTracker() {
public HConnection getShortCircuitConnection() {
return null;
}
@Override
public MetaTableLocator getMetaTableLocator() {
return null;
}

View File

@ -27,11 +27,12 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.replication.ReplicationFactory;
import org.apache.hadoop.hbase.replication.ReplicationQueues;
import org.apache.hadoop.hbase.replication.regionserver.Replication;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.AfterClass;
import org.junit.BeforeClass;
@ -157,7 +158,12 @@ public class TestLogsCleaner {
}
@Override
public CatalogTracker getCatalogTracker() {
public HConnection getShortCircuitConnection() {
return null;
}
@Override
public MetaTableLocator getMetaTableLocator() {
return null;
}

View File

@ -125,7 +125,7 @@ public class TestEndToEndSplitTransaction {
// 3. finish phase II
// note that this replicates some code from SplitTransaction
// 2nd daughter first
server.postOpenDeployTasks(regions.getSecond(), server.getCatalogTracker());
server.postOpenDeployTasks(regions.getSecond());
// Add to online regions
server.addToOnlineRegions(regions.getSecond());
// THIS is the crucial point:
@ -135,7 +135,7 @@ public class TestEndToEndSplitTransaction {
assertTrue(test(con, tableName, lastRow, server));
// first daughter second
server.postOpenDeployTasks(regions.getFirst(), server.getCatalogTracker());
server.postOpenDeployTasks(regions.getFirst());
// Add to online regions
server.addToOnlineRegions(regions.getFirst());
assertTrue(test(con, tableName, firstRow, server));
@ -293,7 +293,7 @@ public class TestEndToEndSplitTransaction {
}
/**
* Checks regions using MetaScanner, MetaReader and HTable methods
* Checks regions using MetaScanner, MetaTableAccessor and HTable methods
*/
static class RegionChecker extends Chore {
Configuration conf;

View File

@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Put;
@ -83,7 +83,7 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase {
i == 0? HConstants.EMPTY_BYTE_ARRAY: Bytes.toBytes((byte)i),
i == last? HConstants.EMPTY_BYTE_ARRAY: Bytes.toBytes((byte)i + interval));
Put put = MetaEditor.makePutFromRegionInfo(hri);
Put put = MetaTableAccessor.makePutFromRegionInfo(hri);
put.setDurability(Durability.SKIP_WAL);
mr.put(put);
}

View File

@ -32,8 +32,8 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.io.hfile.BlockCache;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
import org.apache.hadoop.hbase.io.hfile.CacheStats;
import org.apache.hadoop.hbase.io.hfile.Cacheable;
@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.io.hfile.CachedBlock;
import org.apache.hadoop.hbase.io.hfile.ResizableBlockCache;
import org.apache.hadoop.hbase.regionserver.HeapMemoryManager.TunerContext;
import org.apache.hadoop.hbase.regionserver.HeapMemoryManager.TunerResult;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@ -425,7 +426,12 @@ public class TestHeapMemoryManager {
}
@Override
public CatalogTracker getCatalogTracker() {
public HConnection getShortCircuitConnection() {
return null;
}
@Override
public MetaTableLocator getMetaTableLocator() {
return null;
}

View File

@ -40,7 +40,7 @@ import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
@ -174,14 +174,14 @@ public class TestRegionMergeTransactionOnCluster {
verifyRowCount(table, ROWSIZE);
table.close();
List<Pair<HRegionInfo, ServerName>> tableRegions = MetaReader
.getTableRegionsAndLocations(master.getCatalogTracker(),
tableName);
List<Pair<HRegionInfo, ServerName>> tableRegions = MetaTableAccessor
.getTableRegionsAndLocations(master.getZooKeeper(),
master.getShortCircuitConnection(), tableName);
HRegionInfo mergedRegionInfo = tableRegions.get(0).getFirst();
HTableDescriptor tableDescritor = master.getTableDescriptors().get(
tableName);
Result mergedRegionResult = MetaReader.getRegionResult(
master.getCatalogTracker(), mergedRegionInfo.getRegionName());
Result mergedRegionResult = MetaTableAccessor.getRegionResult(
master.getShortCircuitConnection(), mergedRegionInfo.getRegionName());
// contains merge reference in META
assertTrue(mergedRegionResult.getValue(HConstants.CATALOG_FAMILY,
@ -223,8 +223,8 @@ public class TestRegionMergeTransactionOnCluster {
assertFalse(fs.exists(regionAdir));
assertFalse(fs.exists(regionBdir));
mergedRegionResult = MetaReader.getRegionResult(
master.getCatalogTracker(), mergedRegionInfo.getRegionName());
mergedRegionResult = MetaTableAccessor.getRegionResult(
master.getShortCircuitConnection(), mergedRegionInfo.getRegionName());
assertFalse(mergedRegionResult.getValue(HConstants.CATALOG_FAMILY,
HConstants.MERGEA_QUALIFIER) != null);
assertFalse(mergedRegionResult.getValue(HConstants.CATALOG_FAMILY,
@ -299,9 +299,9 @@ public class TestRegionMergeTransactionOnCluster {
private PairOfSameType<HRegionInfo> requestMergeRegion(
HMaster master, TableName tablename,
int regionAnum, int regionBnum) throws Exception {
List<Pair<HRegionInfo, ServerName>> tableRegions = MetaReader
.getTableRegionsAndLocations(master.getCatalogTracker(),
tablename);
List<Pair<HRegionInfo, ServerName>> tableRegions = MetaTableAccessor
.getTableRegionsAndLocations(master.getZooKeeper(),
master.getShortCircuitConnection(), tablename);
HRegionInfo regionA = tableRegions.get(regionAnum).getFirst();
HRegionInfo regionB = tableRegions.get(regionBnum).getFirst();
TEST_UTIL.getHBaseAdmin().mergeRegions(
@ -316,8 +316,8 @@ public class TestRegionMergeTransactionOnCluster {
List<HRegionInfo> tableRegionsInMaster;
long timeout = System.currentTimeMillis() + waitTime;
while (System.currentTimeMillis() < timeout) {
tableRegionsInMeta = MetaReader.getTableRegionsAndLocations(
master.getCatalogTracker(), tablename);
tableRegionsInMeta = MetaTableAccessor.getTableRegionsAndLocations(
master.getZooKeeper(), master.getShortCircuitConnection(), tablename);
tableRegionsInMaster = master.getAssignmentManager().getRegionStates()
.getRegionsOfTable(tablename);
if (tableRegionsInMeta.size() == expectedRegionNum
@ -327,8 +327,8 @@ public class TestRegionMergeTransactionOnCluster {
Thread.sleep(250);
}
tableRegionsInMeta = MetaReader.getTableRegionsAndLocations(
master.getCatalogTracker(), tablename);
tableRegionsInMeta = MetaTableAccessor.getTableRegionsAndLocations(
master.getZooKeeper(), master.getShortCircuitConnection(), tablename);
LOG.info("Regions after merge:" + Joiner.on(',').join(tableRegionsInMeta));
assertEquals(expectedRegionNum, tableRegionsInMeta.size());
}
@ -354,15 +354,15 @@ public class TestRegionMergeTransactionOnCluster {
long timeout = System.currentTimeMillis() + waitTime;
List<Pair<HRegionInfo, ServerName>> tableRegions;
while (System.currentTimeMillis() < timeout) {
tableRegions = MetaReader.getTableRegionsAndLocations(
master.getCatalogTracker(), tablename);
tableRegions = MetaTableAccessor.getTableRegionsAndLocations(
master.getZooKeeper(), master.getShortCircuitConnection(), tablename);
if (tableRegions.size() == numRegions)
break;
Thread.sleep(250);
}
tableRegions = MetaReader.getTableRegionsAndLocations(
master.getCatalogTracker(), tablename);
tableRegions = MetaTableAccessor.getTableRegionsAndLocations(
master.getZooKeeper(), master.getShortCircuitConnection(), tablename);
LOG.info("Regions after load: " + Joiner.on(',').join(tableRegions));
assertEquals(numRegions, tableRegions.size());
return table;

View File

@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.catalog.TestMetaReaderEditor;
import org.apache.hadoop.hbase.TestMetaTableAccessor;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
@ -184,7 +184,7 @@ public class TestRegionReplicas {
HTable meta = null;
try {
meta = new HTable(HTU.getConfiguration(), TableName.META_TABLE_NAME);
TestMetaReaderEditor.assertMetaLocation(meta, hriPrimary.getRegionName()
TestMetaTableAccessor.assertMetaLocation(meta, hriPrimary.getRegionName()
, getRS().getServerName(), -1, 1, false);
} finally {
if (meta != null ) meta.close();

View File

@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
@ -43,7 +43,7 @@ import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
@ -111,8 +111,9 @@ public class TestRegionServerNoMaster {
zkw, HRegionInfo.FIRST_META_REGIONINFO, hrs.getServerName());
ProtobufUtil.openRegion(hrs.getRSRpcServices(),
hrs.getServerName(), HRegionInfo.FIRST_META_REGIONINFO);
MetaTableLocator mtl = new MetaTableLocator();
while (true) {
ServerName sn = MetaRegionTracker.getMetaRegionLocation(zkw);
ServerName sn = mtl.getMetaRegionLocation(zkw);
if (sn != null && sn.equals(hrs.getServerName())) {
break;
}
@ -301,7 +302,7 @@ public class TestRegionServerNoMaster {
try {
// we re-opened meta so some of its data is lost
ServerName sn = getRS().getServerName();
MetaEditor.updateRegionLocation(getRS().catalogTracker,
MetaTableAccessor.updateRegionLocation(getRS().getShortCircuitConnection(),
hri, sn, getRS().getRegion(regionName).getOpenSeqNum());
// fake region to be closing now, need to clear state afterwards
getRS().regionsInTransitionInRS.put(hri.getEncodedNameAsBytes(), Boolean.FALSE);

View File

@ -56,8 +56,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
@ -898,7 +897,7 @@ public class TestSplitTransactionOnCluster {
admin.setBalancerRunning(false, true);
// Turn off the meta scanner so it don't remove parent on us.
cluster.getMaster().setCatalogJanitorEnabled(false);
boolean tableExists = MetaReader.tableExists(regionServer.getCatalogTracker(),
boolean tableExists = MetaTableAccessor.tableExists(regionServer.getShortCircuitConnection(),
tableName);
assertEquals("The specified table should present.", true, tableExists);
final HRegion region = findSplittableRegion(regions);
@ -910,7 +909,7 @@ public class TestSplitTransactionOnCluster {
} catch (IOException e) {
}
tableExists = MetaReader.tableExists(regionServer.getCatalogTracker(),
tableExists = MetaTableAccessor.tableExists(regionServer.getShortCircuitConnection(),
tableName);
assertEquals("The specified table should present.", true, tableExists);
} finally {
@ -1420,13 +1419,15 @@ public class TestSplitTransactionOnCluster {
copyOfParent.setOffline(true);
copyOfParent.setSplit(true);
// Put for parent
Put putParent = MetaEditor.makePutFromRegionInfo(copyOfParent);
MetaEditor.addDaughtersToPut(putParent, daughterRegions.getFirst().getRegionInfo(),
Put putParent = MetaTableAccessor.makePutFromRegionInfo(copyOfParent);
MetaTableAccessor.addDaughtersToPut(putParent, daughterRegions.getFirst().getRegionInfo(),
daughterRegions.getSecond().getRegionInfo());
metaEntries.add(putParent);
// Puts for daughters
Put putA = MetaEditor.makePutFromRegionInfo(daughterRegions.getFirst().getRegionInfo());
Put putB = MetaEditor.makePutFromRegionInfo(daughterRegions.getSecond().getRegionInfo());
Put putA = MetaTableAccessor.makePutFromRegionInfo(
daughterRegions.getFirst().getRegionInfo());
Put putB = MetaTableAccessor.makePutFromRegionInfo(
daughterRegions.getSecond().getRegionInfo());
st.addLocation(putA, rs.getServerName(), 1);
st.addLocation(putB, rs.getServerName(), 1);
metaEntries.add(putA);

View File

@ -30,7 +30,8 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@ -148,7 +149,12 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
}
@Override
public CatalogTracker getCatalogTracker() {
public HConnection getShortCircuitConnection() {
return null;
}
@Override
public MetaTableLocator getMetaTableLocator() {
return null;
}

View File

@ -31,8 +31,9 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@ -224,7 +225,12 @@ public class TestReplicationTrackerZKImpl {
}
@Override
public CatalogTracker getCatalogTracker() {
public HConnection getShortCircuitConnection() {
return null;
}
@Override
public MetaTableLocator getMetaTableLocator() {
return null;
}

View File

@ -47,8 +47,8 @@ import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
@ -60,6 +60,7 @@ import org.apache.hadoop.hbase.replication.ReplicationQueues;
import org.apache.hadoop.hbase.replication.ReplicationSourceDummy;
import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@ -395,10 +396,14 @@ public class TestReplicationSourceManager {
public CoordinatedStateManager getCoordinatedStateManager() {
return null;
}
@Override
public HConnection getShortCircuitConnection() {
return null;
}
@Override
public CatalogTracker getCatalogTracker() {
return null; // To change body of implemented methods use File | Settings | File Templates.
public MetaTableLocator getMetaTableLocator() {
return null;
}
@Override

Some files were not shown because too many files have changed in this diff Show More