HBASE-4495 CatalogTracker has an identity crisis; needs to be cut-back in scope (Mikhail Antonov)
This commit is contained in:
parent
1d8958685a
commit
ea085c6373
|
@ -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()
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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 + '}';
|
||||
}
|
||||
}
|
|
@ -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 <tableName&rt; <,&rt; <,&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();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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)) {
|
||||
|
|
|
@ -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().
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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()));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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) +
|
||||
|
|
|
@ -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 '" +
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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() + "'";
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
*/
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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()));
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
|
|
|
@ -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: "
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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) + "/";
|
||||
%>
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
|
@ -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.
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
||||
}
|
|
@ -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)));
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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(),
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
|
|
@ -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");
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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
Loading…
Reference in New Issue