HBASE-3446 ProcessServerShutdown fails if META moves, orphaning lots of regions
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1183158 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
69761be2c2
commit
8d8dc87d0d
|
@ -361,6 +361,8 @@ Release 0.92.0 - Unreleased
|
||||||
names, need to use a more consistent block naming scheme (jgray)
|
names, need to use a more consistent block naming scheme (jgray)
|
||||||
HBASE-4551 Fix pom and some test cases to compile and run against
|
HBASE-4551 Fix pom and some test cases to compile and run against
|
||||||
Hadoop 0.23 (todd)
|
Hadoop 0.23 (todd)
|
||||||
|
HBASE-3446 ProcessServerShutdown fails if META moves, orphaning lots of
|
||||||
|
regions
|
||||||
|
|
||||||
TESTS
|
TESTS
|
||||||
HBASE-4450 test for number of blocks read: to serve as baseline for expected
|
HBASE-4450 test for number of blocks read: to serve as baseline for expected
|
||||||
|
|
|
@ -185,10 +185,6 @@ public final class HConstants {
|
||||||
/** The file name used to store HTD in HDFS */
|
/** The file name used to store HTD in HDFS */
|
||||||
public static final String TABLEINFO_NAME = ".tableinfo";
|
public static final String TABLEINFO_NAME = ".tableinfo";
|
||||||
|
|
||||||
/** The metaupdated column qualifier */
|
|
||||||
public static final byte [] META_MIGRATION_QUALIFIER = Bytes.toBytes("metamigrated");
|
|
||||||
|
|
||||||
|
|
||||||
/** Default maximum file size */
|
/** Default maximum file size */
|
||||||
public static final long DEFAULT_MAX_FILE_SIZE = 256 * 1024 * 1024;
|
public static final long DEFAULT_MAX_FILE_SIZE = 256 * 1024 * 1024;
|
||||||
|
|
||||||
|
|
|
@ -1043,6 +1043,7 @@ public class KeyValue implements Writable, HeapSize {
|
||||||
* @return True if this KV is a {@link KeyValue.Type#Delete} type.
|
* @return True if this KV is a {@link KeyValue.Type#Delete} type.
|
||||||
*/
|
*/
|
||||||
public boolean isDeleteType() {
|
public boolean isDeleteType() {
|
||||||
|
// TODO: Fix this method name vis-a-vis isDelete!
|
||||||
return getType() == Type.Delete.getCode();
|
return getType() == Type.Delete.getCode();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||||
import org.apache.hadoop.hbase.security.User;
|
import org.apache.hadoop.hbase.security.User;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
|
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
|
||||||
|
import org.apache.hadoop.hbase.util.Threads;
|
||||||
|
|
||||||
import java.util.concurrent.CopyOnWriteArrayList;
|
import java.util.concurrent.CopyOnWriteArrayList;
|
||||||
import org.apache.hadoop.hbase.master.HMaster;
|
import org.apache.hadoop.hbase.master.HMaster;
|
||||||
|
@ -238,6 +239,7 @@ public class LocalHBaseCluster {
|
||||||
List<RegionServerThread> list = getRegionServers();
|
List<RegionServerThread> list = getRegionServers();
|
||||||
for (JVMClusterUtil.RegionServerThread rst: list) {
|
for (JVMClusterUtil.RegionServerThread rst: list) {
|
||||||
if (rst.isAlive()) liveServers.add(rst);
|
if (rst.isAlive()) liveServers.add(rst);
|
||||||
|
else LOG.info("Not alive " + rst.getName());
|
||||||
}
|
}
|
||||||
return liveServers;
|
return liveServers;
|
||||||
}
|
}
|
||||||
|
@ -386,12 +388,12 @@ public class LocalHBaseCluster {
|
||||||
*/
|
*/
|
||||||
public void join() {
|
public void join() {
|
||||||
if (this.regionThreads != null) {
|
if (this.regionThreads != null) {
|
||||||
for(Thread t: this.regionThreads) {
|
for(Thread t: this.regionThreads) {
|
||||||
if (t.isAlive()) {
|
if (t.isAlive()) {
|
||||||
try {
|
try {
|
||||||
t.join();
|
Threads.threadDumpingIsAlive(t);
|
||||||
} catch (InterruptedException e) {
|
} catch (InterruptedException e) {
|
||||||
// continue
|
LOG.debug("Interrupted", e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -400,9 +402,9 @@ public class LocalHBaseCluster {
|
||||||
for (Thread t : this.masterThreads) {
|
for (Thread t : this.masterThreads) {
|
||||||
if (t.isAlive()) {
|
if (t.isAlive()) {
|
||||||
try {
|
try {
|
||||||
t.join();
|
Threads.threadDumpingIsAlive(t);
|
||||||
} catch (InterruptedException e) {
|
} catch (InterruptedException e) {
|
||||||
// continue
|
LOG.debug("Interrupted", e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,6 +1,4 @@
|
||||||
/**
|
/**
|
||||||
* Copyright 2010 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
|
@ -33,7 +31,6 @@ import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hbase.Abortable;
|
import org.apache.hadoop.hbase.Abortable;
|
||||||
import org.apache.hadoop.hbase.HRegionInfo;
|
import org.apache.hadoop.hbase.HRegionInfo;
|
||||||
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
|
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
|
||||||
import org.apache.hadoop.hbase.NotServingRegionException;
|
|
||||||
import org.apache.hadoop.hbase.ServerName;
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
import org.apache.hadoop.hbase.client.HConnection;
|
import org.apache.hadoop.hbase.client.HConnection;
|
||||||
import org.apache.hadoop.hbase.client.HConnectionManager;
|
import org.apache.hadoop.hbase.client.HConnectionManager;
|
||||||
|
@ -60,30 +57,76 @@ import org.apache.hadoop.ipc.RemoteException;
|
||||||
* interrupt waits and close up shop.
|
* interrupt waits and close up shop.
|
||||||
*/
|
*/
|
||||||
public class CatalogTracker {
|
public class CatalogTracker {
|
||||||
|
// TODO: This class needs a rethink. The original intent was that it would be
|
||||||
|
// the one-stop-shop for root and meta locations and that it would get this
|
||||||
|
// info from reading and watching zk state. The class was to be used by
|
||||||
|
// servers when they needed to know of root and meta movement but also by
|
||||||
|
// client-side (inside in HTable) so rather than figure root and meta
|
||||||
|
// 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 .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.
|
||||||
|
//
|
||||||
private static final Log LOG = LogFactory.getLog(CatalogTracker.class);
|
private static final Log LOG = LogFactory.getLog(CatalogTracker.class);
|
||||||
private final Configuration conf;
|
|
||||||
private final HConnection connection;
|
private final HConnection connection;
|
||||||
private final ZooKeeperWatcher zookeeper;
|
private final ZooKeeperWatcher zookeeper;
|
||||||
private final RootRegionTracker rootRegionTracker;
|
private final RootRegionTracker rootRegionTracker;
|
||||||
private final MetaNodeTracker metaNodeTracker;
|
private final MetaNodeTracker metaNodeTracker;
|
||||||
private final AtomicBoolean metaAvailable = new AtomicBoolean(false);
|
private final AtomicBoolean metaAvailable = new AtomicBoolean(false);
|
||||||
/**
|
private boolean instantiatedzkw = false;
|
||||||
|
|
||||||
|
/*
|
||||||
* Do not clear this address once set. Its needed when we do
|
* Do not clear this address once set. Its needed when we do
|
||||||
* server shutdown processing -- we need to know who had .META. last. If you
|
* server shutdown processing -- we need to know who had .META. last. If you
|
||||||
* want to know if the address is good, rely on {@link #metaAvailable} value.
|
* want to know if the address is good, rely on {@link #metaAvailable} value.
|
||||||
*/
|
*/
|
||||||
private ServerName metaLocation;
|
private ServerName metaLocation;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Timeout waiting on root or meta to be set.
|
||||||
|
*/
|
||||||
private final int defaultTimeout;
|
private final int defaultTimeout;
|
||||||
|
|
||||||
private boolean stopped = false;
|
private boolean stopped = false;
|
||||||
|
|
||||||
public static final byte [] ROOT_REGION =
|
static final byte [] ROOT_REGION_NAME =
|
||||||
HRegionInfo.ROOT_REGIONINFO.getRegionName();
|
HRegionInfo.ROOT_REGIONINFO.getRegionName();
|
||||||
public static final byte [] META_REGION =
|
static final byte [] META_REGION_NAME =
|
||||||
HRegionInfo.FIRST_META_REGIONINFO.getRegionName();
|
HRegionInfo.FIRST_META_REGIONINFO.getRegionName();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructs a catalog tracker. Find current state of catalog tables and
|
* Constructs a catalog tracker. Find current state of catalog tables.
|
||||||
* begin active tracking by executing {@link #start()} post construction. Does
|
* Begin active tracking by executing {@link #start()} post construction. Does
|
||||||
* not timeout.
|
* not timeout.
|
||||||
*
|
*
|
||||||
* @param conf
|
* @param conf
|
||||||
|
@ -97,29 +140,36 @@ public class CatalogTracker {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructs the catalog tracker. Find current state of catalog tables and
|
* Constructs the catalog tracker. Find current state of catalog tables.
|
||||||
* begin active tracking by executing {@link #start()} post construction.
|
* Begin active tracking by executing {@link #start()} post construction.
|
||||||
* Does not timeout.
|
* Does not timeout.
|
||||||
* @param zk
|
* @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 connection server connection
|
* @param connection server connection
|
||||||
* @param abortable if fatal exception
|
* @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
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public CatalogTracker(final ZooKeeperWatcher zk, final Configuration conf,
|
public CatalogTracker(final ZooKeeperWatcher zk, final Configuration conf,
|
||||||
final Abortable abortable)
|
final Abortable abortable)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
this(zk, conf, abortable, 0);
|
this(zk, conf, abortable,
|
||||||
|
conf.getInt("hbase.catalogtracker.default.timeout", 1000));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructs the catalog tracker. Find current state of catalog tables and
|
* Constructs the catalog tracker. Find current state of catalog tables.
|
||||||
* begin active tracking by executing {@link #start()} post construction.
|
* Begin active tracking by executing {@link #start()} post construction.
|
||||||
* @param zk
|
* @param zk If zk is null, we'll create an instance (and shut it down
|
||||||
* @param connection server connection
|
* when {@link #stop()} is called) else we'll use what is passed.
|
||||||
* @param abortable if fatal exception
|
* @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.
|
||||||
* @param defaultTimeout Timeout to use. Pass zero for no timeout
|
* @param defaultTimeout Timeout to use. Pass zero for no timeout
|
||||||
* ({@link Object#wait(long)} when passed a <code>0</code> waits for ever).
|
* ({@link Object#wait(long)} when passed a <code>0</code> waits for ever).
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public CatalogTracker(final ZooKeeperWatcher zk, final Configuration conf,
|
public CatalogTracker(final ZooKeeperWatcher zk, final Configuration conf,
|
||||||
Abortable abortable, final int defaultTimeout)
|
Abortable abortable, final int defaultTimeout)
|
||||||
|
@ -130,14 +180,29 @@ public class CatalogTracker {
|
||||||
CatalogTracker(final ZooKeeperWatcher zk, final Configuration conf,
|
CatalogTracker(final ZooKeeperWatcher zk, final Configuration conf,
|
||||||
HConnection connection, Abortable abortable, final int defaultTimeout)
|
HConnection connection, Abortable abortable, final int defaultTimeout)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
this.conf = conf;
|
|
||||||
this.connection = connection;
|
this.connection = connection;
|
||||||
this.zookeeper = (zk == null) ? this.connection.getZooKeeperWatcher() : zk;
|
|
||||||
if (abortable == null) {
|
if (abortable == null) {
|
||||||
|
// A connection is abortable.
|
||||||
abortable = this.connection;
|
abortable = this.connection;
|
||||||
}
|
}
|
||||||
|
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.rootRegionTracker = new RootRegionTracker(zookeeper, abortable);
|
this.rootRegionTracker = new RootRegionTracker(zookeeper, abortable);
|
||||||
this.metaNodeTracker = new MetaNodeTracker(zookeeper, this, abortable);
|
final CatalogTracker ct = this;
|
||||||
|
// Override nodeDeleted so we get notified when meta node deleted
|
||||||
|
this.metaNodeTracker = new MetaNodeTracker(zookeeper, abortable) {
|
||||||
|
public void nodeDeleted(String path) {
|
||||||
|
if (!path.equals(node)) return;
|
||||||
|
ct.resetMetaLocation();
|
||||||
|
}
|
||||||
|
};
|
||||||
this.defaultTimeout = defaultTimeout;
|
this.defaultTimeout = defaultTimeout;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -149,9 +214,9 @@ public class CatalogTracker {
|
||||||
* @throws InterruptedException
|
* @throws InterruptedException
|
||||||
*/
|
*/
|
||||||
public void start() throws IOException, InterruptedException {
|
public void start() throws IOException, InterruptedException {
|
||||||
|
LOG.debug("Starting catalog tracker " + this);
|
||||||
this.rootRegionTracker.start();
|
this.rootRegionTracker.start();
|
||||||
this.metaNodeTracker.start();
|
this.metaNodeTracker.start();
|
||||||
LOG.debug("Starting catalog tracker " + this);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -173,6 +238,9 @@ public class CatalogTracker {
|
||||||
// IOException}, in reality, the implementation would never do that.
|
// IOException}, in reality, the implementation would never do that.
|
||||||
LOG.error("Attempt to close catalog tracker's connection failed.", e);
|
LOG.error("Attempt to close catalog tracker's connection failed.", e);
|
||||||
}
|
}
|
||||||
|
if (this.instantiatedzkw) {
|
||||||
|
this.zookeeper.close();
|
||||||
|
}
|
||||||
// Call this and it will interrupt any ongoing waits on meta.
|
// Call this and it will interrupt any ongoing waits on meta.
|
||||||
synchronized (this.metaAvailable) {
|
synchronized (this.metaAvailable) {
|
||||||
this.metaAvailable.notifyAll();
|
this.metaAvailable.notifyAll();
|
||||||
|
@ -183,7 +251,8 @@ public class CatalogTracker {
|
||||||
/**
|
/**
|
||||||
* Gets the current location for <code>-ROOT-</code> or null if location is
|
* Gets the current location for <code>-ROOT-</code> or null if location is
|
||||||
* not currently available.
|
* not currently available.
|
||||||
* @return server name
|
* @return {@link ServerName} for server hosting <code>-ROOT-</code> or null
|
||||||
|
* if none available
|
||||||
* @throws InterruptedException
|
* @throws InterruptedException
|
||||||
*/
|
*/
|
||||||
public ServerName getRootLocation() throws InterruptedException {
|
public ServerName getRootLocation() throws InterruptedException {
|
||||||
|
@ -191,8 +260,8 @@ public class CatalogTracker {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return Location of server hosting meta region formatted as per
|
* @return {@link ServerName} for server hosting <code>.META.</code> or null
|
||||||
* {@link ServerName}, or null if none available
|
* if none available
|
||||||
*/
|
*/
|
||||||
public ServerName getMetaLocation() {
|
public ServerName getMetaLocation() {
|
||||||
return this.metaLocation;
|
return this.metaLocation;
|
||||||
|
@ -213,7 +282,8 @@ public class CatalogTracker {
|
||||||
* for up to the specified timeout if not immediately available. Returns null
|
* for up to the specified timeout if not immediately available. Returns null
|
||||||
* if the timeout elapses before root is available.
|
* if the timeout elapses before root is available.
|
||||||
* @param timeout maximum time to wait for root availability, in milliseconds
|
* @param timeout maximum time to wait for root availability, in milliseconds
|
||||||
* @return Location of server hosting root region or null if none available
|
* @return {@link ServerName} for server hosting <code>-ROOT-</code> or null
|
||||||
|
* if none available
|
||||||
* @throws InterruptedException if interrupted while waiting
|
* @throws InterruptedException if interrupted while waiting
|
||||||
* @throws NotAllMetaRegionsOnlineException if root not available before
|
* @throws NotAllMetaRegionsOnlineException if root not available before
|
||||||
* timeout
|
* timeout
|
||||||
|
@ -230,13 +300,31 @@ public class CatalogTracker {
|
||||||
/**
|
/**
|
||||||
* Gets a connection to the server hosting root, as reported by ZooKeeper,
|
* Gets a connection to the server hosting root, as reported by ZooKeeper,
|
||||||
* waiting up to the specified timeout for availability.
|
* waiting up to the specified timeout for availability.
|
||||||
|
* @param timeout How long to wait on root location
|
||||||
|
* @see #waitForRoot(long) for additional information
|
||||||
|
* @return connection to server hosting root
|
||||||
|
* @throws InterruptedException
|
||||||
|
* @throws NotAllMetaRegionsOnlineException if timed out waiting
|
||||||
|
* @throws IOException
|
||||||
|
* @deprecated Use {@link #getRootServerConnection(long)}
|
||||||
|
*/
|
||||||
|
public HRegionInterface waitForRootServerConnection(long timeout)
|
||||||
|
throws InterruptedException, NotAllMetaRegionsOnlineException, IOException {
|
||||||
|
return getRootServerConnection(timeout);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Gets a connection to the server hosting root, as reported by ZooKeeper,
|
||||||
|
* waiting up to the specified timeout for availability.
|
||||||
|
* <p>WARNING: Does not retry. Use an {@link HTable} instead.
|
||||||
|
* @param timeout How long to wait on root location
|
||||||
* @see #waitForRoot(long) for additional information
|
* @see #waitForRoot(long) for additional information
|
||||||
* @return connection to server hosting root
|
* @return connection to server hosting root
|
||||||
* @throws InterruptedException
|
* @throws InterruptedException
|
||||||
* @throws NotAllMetaRegionsOnlineException if timed out waiting
|
* @throws NotAllMetaRegionsOnlineException if timed out waiting
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public HRegionInterface waitForRootServerConnection(long timeout)
|
HRegionInterface getRootServerConnection(long timeout)
|
||||||
throws InterruptedException, NotAllMetaRegionsOnlineException, IOException {
|
throws InterruptedException, NotAllMetaRegionsOnlineException, IOException {
|
||||||
return getCachedConnection(waitForRoot(timeout));
|
return getCachedConnection(waitForRoot(timeout));
|
||||||
}
|
}
|
||||||
|
@ -248,29 +336,17 @@ public class CatalogTracker {
|
||||||
* @return connection to server hosting root
|
* @return connection to server hosting root
|
||||||
* @throws NotAllMetaRegionsOnlineException if timed out waiting
|
* @throws NotAllMetaRegionsOnlineException if timed out waiting
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
|
* @deprecated Use {@link #getRootServerConnection(long)}
|
||||||
*/
|
*/
|
||||||
public HRegionInterface waitForRootServerConnectionDefault()
|
public HRegionInterface waitForRootServerConnectionDefault()
|
||||||
throws NotAllMetaRegionsOnlineException, IOException {
|
throws NotAllMetaRegionsOnlineException, IOException {
|
||||||
try {
|
try {
|
||||||
return getCachedConnection(waitForRoot(defaultTimeout));
|
return getRootServerConnection(this.defaultTimeout);
|
||||||
} catch (InterruptedException e) {
|
} catch (InterruptedException e) {
|
||||||
throw new NotAllMetaRegionsOnlineException("Interrupted");
|
throw new NotAllMetaRegionsOnlineException("Interrupted");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Gets a connection to the server hosting root, as reported by ZooKeeper,
|
|
||||||
* if available. Returns null if no location is immediately available.
|
|
||||||
* @return connection to server hosting root, null if not available
|
|
||||||
* @throws IOException
|
|
||||||
* @throws InterruptedException
|
|
||||||
*/
|
|
||||||
private HRegionInterface getRootServerConnection()
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
ServerName sn = this.rootRegionTracker.getRootRegionLocation();
|
|
||||||
return getCachedConnection(sn);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Gets a connection to the server currently hosting <code>.META.</code> or
|
* Gets a connection to the server currently hosting <code>.META.</code> or
|
||||||
* null if location is not currently available.
|
* null if location is not currently available.
|
||||||
|
@ -292,24 +368,28 @@ public class CatalogTracker {
|
||||||
synchronized (metaAvailable) {
|
synchronized (metaAvailable) {
|
||||||
if (metaAvailable.get()) {
|
if (metaAvailable.get()) {
|
||||||
HRegionInterface current = getCachedConnection(this.metaLocation);
|
HRegionInterface current = getCachedConnection(this.metaLocation);
|
||||||
if (verifyRegionLocation(current, this.metaLocation, META_REGION)) {
|
// If we are to refresh, verify we have a good connection by making
|
||||||
|
// an invocation on it.
|
||||||
|
if (verifyRegionLocation(current, this.metaLocation, META_REGION_NAME)) {
|
||||||
return current;
|
return current;
|
||||||
}
|
}
|
||||||
resetMetaLocation();
|
resetMetaLocation();
|
||||||
}
|
}
|
||||||
HRegionInterface rootConnection = getRootServerConnection();
|
// We got here because there is no meta available or because whats
|
||||||
if (rootConnection == null) {
|
// available is bad.
|
||||||
LOG.debug("-ROOT- server unavailable.");
|
|
||||||
return null;
|
// Now read the current .META. content from -ROOT-. Note: This goes via
|
||||||
}
|
// an HConnection. It has its own way of figuring root and meta locations
|
||||||
ServerName newLocation = MetaReader.readMetaLocation(rootConnection);
|
// which we have to wait on.
|
||||||
|
ServerName newLocation =
|
||||||
|
MetaReader.readRegionLocation(this, META_REGION_NAME);
|
||||||
if (newLocation == null) {
|
if (newLocation == null) {
|
||||||
LOG.debug(".META. server unavailable.");
|
LOG.debug(".META. server unavailable.");
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
HRegionInterface newConnection = getCachedConnection(newLocation);
|
HRegionInterface newConnection = getCachedConnection(newLocation);
|
||||||
if (verifyRegionLocation(newConnection, newLocation, META_REGION)) {
|
if (verifyRegionLocation(newConnection, newLocation, META_REGION_NAME)) {
|
||||||
setMetaLocation(newLocation);
|
setMetaLocation(newLocation);
|
||||||
return newConnection;
|
return newConnection;
|
||||||
} else {
|
} else {
|
||||||
|
@ -322,13 +402,19 @@ public class CatalogTracker {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Waits indefinitely for availability of <code>.META.</code>. Used during
|
* Waits indefinitely for availability of <code>.META.</code>. Used during
|
||||||
* cluster startup.
|
* cluster startup. Does not verify meta, just that something has been
|
||||||
|
* set up in zk.
|
||||||
|
* @see #waitForMeta(long)
|
||||||
* @throws InterruptedException if interrupted while waiting
|
* @throws InterruptedException if interrupted while waiting
|
||||||
*/
|
*/
|
||||||
public void waitForMeta() throws InterruptedException {
|
public void waitForMeta() throws InterruptedException {
|
||||||
synchronized (metaAvailable) {
|
while (!this.stopped) {
|
||||||
while (!stopped && !metaAvailable.get()) {
|
try {
|
||||||
metaAvailable.wait();
|
if (waitForMeta(100) != null) break;
|
||||||
|
} catch (NotAllMetaRegionsOnlineException e) {
|
||||||
|
LOG.info("Retrying", e);
|
||||||
|
} catch (IOException e) {
|
||||||
|
LOG.info("Retrying", e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -340,7 +426,8 @@ public class CatalogTracker {
|
||||||
* in that it will go ahead and verify the location gotten from ZooKeeper and
|
* in that it will go ahead and verify the location gotten from ZooKeeper and
|
||||||
* -ROOT- region by trying to use returned connection.
|
* -ROOT- region by trying to use returned connection.
|
||||||
* @param timeout maximum time to wait for meta availability, in milliseconds
|
* @param timeout maximum time to wait for meta availability, in milliseconds
|
||||||
* @return location of meta
|
* @return {@link ServerName} for server hosting <code>.META.</code> or null
|
||||||
|
* if none available
|
||||||
* @throws InterruptedException if interrupted while waiting
|
* @throws InterruptedException if interrupted while waiting
|
||||||
* @throws IOException unexpected exception connecting to meta server
|
* @throws IOException unexpected exception connecting to meta server
|
||||||
* @throws NotAllMetaRegionsOnlineException if meta not available before
|
* @throws NotAllMetaRegionsOnlineException if meta not available before
|
||||||
|
@ -359,8 +446,7 @@ public class CatalogTracker {
|
||||||
metaAvailable.wait(waitTime);
|
metaAvailable.wait(waitTime);
|
||||||
}
|
}
|
||||||
if (getMetaServerConnection() == null) {
|
if (getMetaServerConnection() == null) {
|
||||||
throw new NotAllMetaRegionsOnlineException(
|
throw new NotAllMetaRegionsOnlineException("Timed out (" + timeout + "ms)");
|
||||||
"Timed out (" + timeout + "ms)");
|
|
||||||
}
|
}
|
||||||
return metaLocation;
|
return metaLocation;
|
||||||
}
|
}
|
||||||
|
@ -374,6 +460,7 @@ public class CatalogTracker {
|
||||||
* @throws InterruptedException
|
* @throws InterruptedException
|
||||||
* @throws NotAllMetaRegionsOnlineException if timed out waiting
|
* @throws NotAllMetaRegionsOnlineException if timed out waiting
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
|
* @deprecated Does not retry; use an HTable instance instead.
|
||||||
*/
|
*/
|
||||||
public HRegionInterface waitForMetaServerConnection(long timeout)
|
public HRegionInterface waitForMetaServerConnection(long timeout)
|
||||||
throws InterruptedException, NotAllMetaRegionsOnlineException, IOException {
|
throws InterruptedException, NotAllMetaRegionsOnlineException, IOException {
|
||||||
|
@ -383,10 +470,12 @@ public class CatalogTracker {
|
||||||
/**
|
/**
|
||||||
* Gets a connection to the server hosting meta, as reported by ZooKeeper,
|
* Gets a connection to the server hosting meta, as reported by ZooKeeper,
|
||||||
* waiting up to the specified timeout for availability.
|
* waiting up to the specified timeout for availability.
|
||||||
|
* Used in tests.
|
||||||
* @see #waitForMeta(long) for additional information
|
* @see #waitForMeta(long) for additional information
|
||||||
* @return connection to server hosting meta
|
* @return connection to server hosting meta
|
||||||
* @throws NotAllMetaRegionsOnlineException if timed out or interrupted
|
* @throws NotAllMetaRegionsOnlineException if timed out or interrupted
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
|
* @deprecated Does not retry; use an HTable instance instead.
|
||||||
*/
|
*/
|
||||||
public HRegionInterface waitForMetaServerConnectionDefault()
|
public HRegionInterface waitForMetaServerConnectionDefault()
|
||||||
throws NotAllMetaRegionsOnlineException, IOException {
|
throws NotAllMetaRegionsOnlineException, IOException {
|
||||||
|
@ -397,12 +486,22 @@ public class CatalogTracker {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void resetMetaLocation() {
|
/**
|
||||||
LOG.debug("Current cached META location: " + metaLocation +
|
* Called when we figure current meta is off (called from zk callback).
|
||||||
" is not valid, resetting");
|
*/
|
||||||
this.metaAvailable.set(false);
|
public void resetMetaLocation() {
|
||||||
|
LOG.debug("Current cached META location, " + metaLocation +
|
||||||
|
", is not valid, resetting");
|
||||||
|
synchronized(this.metaAvailable) {
|
||||||
|
this.metaAvailable.set(false);
|
||||||
|
this.metaAvailable.notifyAll();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Caller must be synchronized on this.metaAvailable
|
||||||
|
* @param metaLocation
|
||||||
|
*/
|
||||||
private void setMetaLocation(final ServerName metaLocation) {
|
private void setMetaLocation(final ServerName metaLocation) {
|
||||||
LOG.debug("set new cached META location: " + metaLocation);
|
LOG.debug("set new cached META location: " + metaLocation);
|
||||||
metaAvailable.set(true);
|
metaAvailable.set(true);
|
||||||
|
@ -411,6 +510,13 @@ public class CatalogTracker {
|
||||||
this.metaAvailable.notifyAll();
|
this.metaAvailable.notifyAll();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param sn ServerName to get a connection against.
|
||||||
|
* @return The HRegionInterface 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
|
||||||
|
*/
|
||||||
private HRegionInterface getCachedConnection(ServerName sn)
|
private HRegionInterface getCachedConnection(ServerName sn)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
if (sn == null) {
|
if (sn == null) {
|
||||||
|
@ -446,17 +552,32 @@ public class CatalogTracker {
|
||||||
return protocol;
|
return protocol;
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean verifyRegionLocation(HRegionInterface metaServer,
|
/**
|
||||||
final ServerName address,
|
* Verify we can connect to <code>hostingServer</code> and that its carrying
|
||||||
byte [] regionName)
|
* <code>regionName</code>.
|
||||||
|
* @param hostingServer Interface to the server hosting <code>regionName</code>
|
||||||
|
* @param serverName 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 HRegionInterface
|
||||||
|
// 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(HRegionInterface hostingServer,
|
||||||
|
final ServerName address, final byte [] regionName)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
if (metaServer == null) {
|
if (hostingServer == null) {
|
||||||
LOG.info("Passed metaserver is null");
|
LOG.info("Passed hostingServer is null");
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
Throwable t = null;
|
Throwable t = null;
|
||||||
try {
|
try {
|
||||||
return metaServer.getRegionInfo(regionName) != null;
|
// Try and get regioninfo from the hosting server.
|
||||||
|
return hostingServer.getRegionInfo(regionName) != null;
|
||||||
} catch (ConnectException e) {
|
} catch (ConnectException e) {
|
||||||
t = e;
|
t = e;
|
||||||
} catch (RemoteException e) {
|
} catch (RemoteException e) {
|
||||||
|
@ -498,8 +619,7 @@ public class CatalogTracker {
|
||||||
}
|
}
|
||||||
return (connection == null)? false:
|
return (connection == null)? false:
|
||||||
verifyRegionLocation(connection,
|
verifyRegionLocation(connection,
|
||||||
this.rootRegionTracker.getRootRegionLocation(),
|
this.rootRegionTracker.getRootRegionLocation(), ROOT_REGION_NAME);
|
||||||
HRegionInfo.ROOT_REGIONINFO.getRegionName());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -523,6 +643,7 @@ public class CatalogTracker {
|
||||||
return connection != null;
|
return connection != null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Used by tests.
|
||||||
MetaNodeTracker getMetaNodeTracker() {
|
MetaNodeTracker getMetaNodeTracker() {
|
||||||
return this.metaNodeTracker;
|
return this.metaNodeTracker;
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,6 +1,4 @@
|
||||||
/**
|
/**
|
||||||
* Copyright 2010 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
|
@ -19,40 +17,121 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.catalog;
|
package org.apache.hadoop.hbase.catalog;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.net.ConnectException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.net.ConnectException;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.hbase.*;
|
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.Delete;
|
||||||
|
import org.apache.hadoop.hbase.client.HTable;
|
||||||
import org.apache.hadoop.hbase.client.Put;
|
import org.apache.hadoop.hbase.client.Put;
|
||||||
import org.apache.hadoop.hbase.ipc.HRegionInterface;
|
import org.apache.hadoop.hbase.client.Result;
|
||||||
import org.apache.hadoop.hbase.migration.HRegionInfo090x;
|
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.util.Writables;
|
import org.apache.hadoop.hbase.util.Writables;
|
||||||
import org.apache.hadoop.hbase.master.MasterServices;
|
|
||||||
import org.apache.hadoop.hbase.client.Result;
|
|
||||||
import org.apache.hadoop.hbase.catalog.MetaReader.Visitor;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Writes region and assignment information to <code>.META.</code>.
|
* Writes region and assignment information to <code>.META.</code>.
|
||||||
* <p>
|
|
||||||
* Uses the {@link CatalogTracker} to obtain locations and connections to
|
|
||||||
* catalogs.
|
|
||||||
*/
|
*/
|
||||||
public class MetaEditor {
|
public class MetaEditor {
|
||||||
|
// 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);
|
private static final Log LOG = LogFactory.getLog(MetaEditor.class);
|
||||||
|
|
||||||
private static Put makePutFromRegionInfo(HRegionInfo regionInfo) throws IOException {
|
private static Put makePutFromRegionInfo(HRegionInfo regionInfo)
|
||||||
|
throws IOException {
|
||||||
Put put = new Put(regionInfo.getRegionName());
|
Put put = new Put(regionInfo.getRegionName());
|
||||||
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
|
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
|
||||||
Writables.getBytes(regionInfo));
|
Writables.getBytes(regionInfo));
|
||||||
return put;
|
return put;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Put the passed <code>p</code> to the <code>.META.</code> table.
|
||||||
|
* @param ct CatalogTracker on whose back we will ride the edit.
|
||||||
|
* @param p Put to add to .META.
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
static void putToMetaTable(final CatalogTracker ct, final Put p)
|
||||||
|
throws IOException {
|
||||||
|
put(MetaReader.getMetaHTable(ct), p);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Put the passed <code>p</code> to the <code>.META.</code> table.
|
||||||
|
* @param ct CatalogTracker on whose back we will ride the edit.
|
||||||
|
* @param p Put to add to .META.
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
static void putToRootTable(final CatalogTracker ct, final Put p)
|
||||||
|
throws IOException {
|
||||||
|
put(MetaReader.getRootHTable(ct), p);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Put the passed <code>p</code> to a catalog table.
|
||||||
|
* @param ct CatalogTracker on whose back we will ride the edit.
|
||||||
|
* @param regionName Name of the catalog table to put too.
|
||||||
|
* @param p Put to add
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
static void putToCatalogTable(final CatalogTracker ct,
|
||||||
|
final byte [] regionName, final Put p)
|
||||||
|
throws IOException {
|
||||||
|
HTable t = MetaReader.getCatalogHTable(ct, regionName);
|
||||||
|
put(t, 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>.META.</code> table.
|
||||||
|
* @param ct CatalogTracker on whose back we will ride the edit.
|
||||||
|
* @param ps Put to add to .META.
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
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>.META.</code> table.
|
||||||
|
* @param ct CatalogTracker on whose back we will ride the edit.
|
||||||
|
* @param d Delete to add to .META.
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
static void deleteMetaTable(final CatalogTracker ct, final Delete d)
|
||||||
|
throws IOException {
|
||||||
|
HTable t = MetaReader.getMetaHTable(ct);
|
||||||
|
try {
|
||||||
|
t.delete(d);
|
||||||
|
} finally {
|
||||||
|
t.close();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Adds a META row for the specified new region.
|
* Adds a META row for the specified new region.
|
||||||
|
@ -62,8 +141,7 @@ public class MetaEditor {
|
||||||
public static void addRegionToMeta(CatalogTracker catalogTracker,
|
public static void addRegionToMeta(CatalogTracker catalogTracker,
|
||||||
HRegionInfo regionInfo)
|
HRegionInfo regionInfo)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
catalogTracker.waitForMetaServerConnectionDefault().put(
|
putToMetaTable(catalogTracker, makePutFromRegionInfo(regionInfo));
|
||||||
CatalogTracker.META_REGION, makePutFromRegionInfo(regionInfo));
|
|
||||||
LOG.info("Added region " + regionInfo.getRegionNameAsString() + " to META");
|
LOG.info("Added region " + regionInfo.getRegionNameAsString() + " to META");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -79,11 +157,9 @@ public class MetaEditor {
|
||||||
List<Put> puts = new ArrayList<Put>();
|
List<Put> puts = new ArrayList<Put>();
|
||||||
for (HRegionInfo regionInfo : regionInfos) {
|
for (HRegionInfo regionInfo : regionInfos) {
|
||||||
puts.add(makePutFromRegionInfo(regionInfo));
|
puts.add(makePutFromRegionInfo(regionInfo));
|
||||||
LOG.debug("Added region " + regionInfo.getRegionNameAsString() + " to META");
|
|
||||||
}
|
}
|
||||||
catalogTracker.waitForMetaServerConnectionDefault().put(
|
putsToMetaTable(catalogTracker, puts);
|
||||||
CatalogTracker.META_REGION, puts);
|
LOG.info("Added " + puts.size() + " regions in META");
|
||||||
LOG.info("Added " + puts.size() + " regions to META");
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -108,7 +184,7 @@ public class MetaEditor {
|
||||||
Writables.getBytes(a));
|
Writables.getBytes(a));
|
||||||
put.add(HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER,
|
put.add(HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER,
|
||||||
Writables.getBytes(b));
|
Writables.getBytes(b));
|
||||||
catalogTracker.waitForMetaServerConnectionDefault().put(CatalogTracker.META_REGION, put);
|
putToMetaTable(catalogTracker, put);
|
||||||
LOG.info("Offlined parent region " + parent.getRegionNameAsString() +
|
LOG.info("Offlined parent region " + parent.getRegionNameAsString() +
|
||||||
" in META");
|
" in META");
|
||||||
}
|
}
|
||||||
|
@ -116,14 +192,11 @@ public class MetaEditor {
|
||||||
public static void addDaughter(final CatalogTracker catalogTracker,
|
public static void addDaughter(final CatalogTracker catalogTracker,
|
||||||
final HRegionInfo regionInfo, final ServerName sn)
|
final HRegionInfo regionInfo, final ServerName sn)
|
||||||
throws NotAllMetaRegionsOnlineException, IOException {
|
throws NotAllMetaRegionsOnlineException, IOException {
|
||||||
HRegionInterface server = catalogTracker.waitForMetaServerConnectionDefault();
|
|
||||||
byte [] catalogRegionName = CatalogTracker.META_REGION;
|
|
||||||
Put put = new Put(regionInfo.getRegionName());
|
Put put = new Put(regionInfo.getRegionName());
|
||||||
addRegionInfo(put, regionInfo);
|
addRegionInfo(put, regionInfo);
|
||||||
if (sn != null) addLocation(put, sn);
|
if (sn != null) addLocation(put, sn);
|
||||||
server.put(catalogRegionName, put);
|
putToMetaTable(catalogTracker, put);
|
||||||
LOG.info("Added daughter " + regionInfo.getRegionNameAsString() +
|
LOG.info("Added daughter " + regionInfo.getRegionNameAsString() +
|
||||||
" in region " + Bytes.toString(catalogRegionName) +
|
|
||||||
(sn == null? ", serverName=null": ", serverName=" + sn.toString()));
|
(sn == null? ", serverName=null": ", serverName=" + sn.toString()));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -145,9 +218,7 @@ public class MetaEditor {
|
||||||
public static void updateMetaLocation(CatalogTracker catalogTracker,
|
public static void updateMetaLocation(CatalogTracker catalogTracker,
|
||||||
HRegionInfo regionInfo, ServerName sn)
|
HRegionInfo regionInfo, ServerName sn)
|
||||||
throws IOException, ConnectException {
|
throws IOException, ConnectException {
|
||||||
HRegionInterface server = catalogTracker.waitForRootServerConnectionDefault();
|
updateLocation(catalogTracker, regionInfo, sn);
|
||||||
if (server == null) throw new IOException("No server for -ROOT-");
|
|
||||||
updateLocation(server, CatalogTracker.ROOT_REGION, regionInfo, sn);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -165,8 +236,7 @@ public class MetaEditor {
|
||||||
public static void updateRegionLocation(CatalogTracker catalogTracker,
|
public static void updateRegionLocation(CatalogTracker catalogTracker,
|
||||||
HRegionInfo regionInfo, ServerName sn)
|
HRegionInfo regionInfo, ServerName sn)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
updateLocation(catalogTracker.waitForMetaServerConnectionDefault(),
|
updateLocation(catalogTracker, regionInfo, sn);
|
||||||
CatalogTracker.META_REGION, regionInfo, sn);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -175,22 +245,21 @@ public class MetaEditor {
|
||||||
* Connects to the specified server which should be hosting the specified
|
* Connects to the specified server which should be hosting the specified
|
||||||
* catalog region name to perform the edit.
|
* catalog region name to perform the edit.
|
||||||
*
|
*
|
||||||
* @param server connection to server hosting catalog region
|
* @param catalogTracker
|
||||||
* @param catalogRegionName name of catalog region being updated
|
|
||||||
* @param regionInfo region to update location of
|
* @param regionInfo region to update location of
|
||||||
* @param sn Server name
|
* @param sn Server name
|
||||||
* @throws IOException In particular could throw {@link java.net.ConnectException}
|
* @throws IOException In particular could throw {@link java.net.ConnectException}
|
||||||
* if the server is down on other end.
|
* if the server is down on other end.
|
||||||
*/
|
*/
|
||||||
private static void updateLocation(HRegionInterface server,
|
private static void updateLocation(final CatalogTracker catalogTracker,
|
||||||
byte [] catalogRegionName, HRegionInfo regionInfo, ServerName sn)
|
HRegionInfo regionInfo, ServerName sn)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
|
final byte [] regionName = regionInfo.getRegionName();
|
||||||
Put put = new Put(regionInfo.getRegionName());
|
Put put = new Put(regionInfo.getRegionName());
|
||||||
addLocation(put, sn);
|
addLocation(put, sn);
|
||||||
server.put(catalogRegionName, put);
|
putToCatalogTable(catalogTracker, regionName, put);
|
||||||
LOG.info("Updated row " + regionInfo.getRegionNameAsString() +
|
LOG.info("Updated row " + regionInfo.getRegionNameAsString() +
|
||||||
" in region " + Bytes.toStringBinary(catalogRegionName) + " with " +
|
" with server=" + sn);
|
||||||
"serverName=" + sn.toString());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -203,8 +272,7 @@ public class MetaEditor {
|
||||||
HRegionInfo regionInfo)
|
HRegionInfo regionInfo)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
Delete delete = new Delete(regionInfo.getRegionName());
|
Delete delete = new Delete(regionInfo.getRegionName());
|
||||||
catalogTracker.waitForMetaServerConnectionDefault().
|
deleteMetaTable(catalogTracker, delete);
|
||||||
delete(CatalogTracker.META_REGION, delete);
|
|
||||||
LOG.info("Deleted region " + regionInfo.getRegionNameAsString() + " from META");
|
LOG.info("Deleted region " + regionInfo.getRegionNameAsString() + " from META");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -223,142 +291,12 @@ public class MetaEditor {
|
||||||
throws NotAllMetaRegionsOnlineException, IOException {
|
throws NotAllMetaRegionsOnlineException, IOException {
|
||||||
Delete delete = new Delete(parent.getRegionName());
|
Delete delete = new Delete(parent.getRegionName());
|
||||||
delete.deleteColumns(HConstants.CATALOG_FAMILY, qualifier);
|
delete.deleteColumns(HConstants.CATALOG_FAMILY, qualifier);
|
||||||
catalogTracker.waitForMetaServerConnectionDefault().
|
deleteMetaTable(catalogTracker, delete);
|
||||||
delete(CatalogTracker.META_REGION, delete);
|
|
||||||
LOG.info("Deleted daughter reference " + daughter.getRegionNameAsString() +
|
LOG.info("Deleted daughter reference " + daughter.getRegionNameAsString() +
|
||||||
", qualifier=" + Bytes.toStringBinary(qualifier) + ", from parent " +
|
", qualifier=" + Bytes.toStringBinary(qualifier) + ", from parent " +
|
||||||
parent.getRegionNameAsString());
|
parent.getRegionNameAsString());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Update the metamigrated flag in -ROOT-.
|
|
||||||
* @param catalogTracker
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
public static void updateRootWithMetaMigrationStatus(
|
|
||||||
CatalogTracker catalogTracker) throws IOException {
|
|
||||||
updateRootWithMetaMigrationStatus(catalogTracker, true);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Update the metamigrated flag in -ROOT-.
|
|
||||||
* @param catalogTracker
|
|
||||||
* @param metaUpdated
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
public static void updateRootWithMetaMigrationStatus(
|
|
||||||
CatalogTracker catalogTracker, boolean metaUpdated)
|
|
||||||
throws IOException {
|
|
||||||
Put put = new Put(HRegionInfo.ROOT_REGIONINFO.getRegionName());
|
|
||||||
addMetaUpdateStatus(put, metaUpdated);
|
|
||||||
catalogTracker.waitForRootServerConnectionDefault().put(
|
|
||||||
CatalogTracker.ROOT_REGION, put);
|
|
||||||
LOG.info("Updated -ROOT- row with metaMigrated status = " + metaUpdated);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Update legacy META rows, removing HTD from HRI.
|
|
||||||
* @param masterServices
|
|
||||||
* @return
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
public static List<HTableDescriptor> updateMetaWithNewRegionInfo(
|
|
||||||
final MasterServices masterServices)
|
|
||||||
throws IOException {
|
|
||||||
final List<HTableDescriptor> htds = new ArrayList<HTableDescriptor>();
|
|
||||||
Visitor v = new Visitor() {
|
|
||||||
@Override
|
|
||||||
public boolean visit(Result r) throws IOException {
|
|
||||||
if (r == null || r.isEmpty()) return true;
|
|
||||||
HRegionInfo090x hrfm = getHRegionInfoForMigration(r);
|
|
||||||
if (hrfm == null) return true;
|
|
||||||
htds.add(hrfm.getTableDesc());
|
|
||||||
masterServices.getMasterFileSystem()
|
|
||||||
.createTableDescriptor(hrfm.getTableDesc());
|
|
||||||
updateHRI(masterServices.getCatalogTracker()
|
|
||||||
.waitForMetaServerConnectionDefault(),
|
|
||||||
hrfm, CatalogTracker.META_REGION);
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
};
|
|
||||||
MetaReader.fullScan(masterServices.getCatalogTracker(), v);
|
|
||||||
updateRootWithMetaMigrationStatus(masterServices.getCatalogTracker());
|
|
||||||
return htds;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Migrate root and meta to newer version. This updates the META and ROOT
|
|
||||||
* and removes the HTD from HRI.
|
|
||||||
* @param masterServices
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
public static void migrateRootAndMeta(final MasterServices masterServices)
|
|
||||||
throws IOException {
|
|
||||||
updateRootWithNewRegionInfo(masterServices);
|
|
||||||
updateMetaWithNewRegionInfo(masterServices);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Update the ROOT with new HRI. (HRI with no HTD)
|
|
||||||
* @param masterServices
|
|
||||||
* @return
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
public static List<HTableDescriptor> updateRootWithNewRegionInfo(
|
|
||||||
final MasterServices masterServices)
|
|
||||||
throws IOException {
|
|
||||||
final List<HTableDescriptor> htds = new ArrayList<HTableDescriptor>();
|
|
||||||
Visitor v = new Visitor() {
|
|
||||||
@Override
|
|
||||||
public boolean visit(Result r) throws IOException {
|
|
||||||
if (r == null || r.isEmpty()) return true;
|
|
||||||
HRegionInfo090x hrfm = getHRegionInfoForMigration(r);
|
|
||||||
if (hrfm == null) return true;
|
|
||||||
htds.add(hrfm.getTableDesc());
|
|
||||||
masterServices.getMasterFileSystem().createTableDescriptor(
|
|
||||||
hrfm.getTableDesc());
|
|
||||||
updateHRI(masterServices.getCatalogTracker()
|
|
||||||
.waitForRootServerConnectionDefault(),
|
|
||||||
hrfm, CatalogTracker.ROOT_REGION);
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
};
|
|
||||||
MetaReader.fullScan(
|
|
||||||
masterServices.getCatalogTracker().waitForRootServerConnectionDefault(),
|
|
||||||
v, HRegionInfo.ROOT_REGIONINFO.getRegionName(), null);
|
|
||||||
return htds;
|
|
||||||
}
|
|
||||||
|
|
||||||
private static void updateHRI(HRegionInterface hRegionInterface,
|
|
||||||
HRegionInfo090x hRegionInfo090x, byte[] regionName)
|
|
||||||
throws IOException {
|
|
||||||
HRegionInfo regionInfo = new HRegionInfo(hRegionInfo090x);
|
|
||||||
Put put = new Put(regionInfo.getRegionName());
|
|
||||||
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
|
|
||||||
Writables.getBytes(regionInfo));
|
|
||||||
hRegionInterface.put(regionName, put);
|
|
||||||
LOG.info("Updated region " + regionInfo + " to " + Bytes.toString(regionName));
|
|
||||||
}
|
|
||||||
|
|
||||||
public static HRegionInfo090x getHRegionInfoForMigration(
|
|
||||||
Result data) throws IOException {
|
|
||||||
HRegionInfo090x info = null;
|
|
||||||
byte [] bytes =
|
|
||||||
data.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
|
|
||||||
if (bytes == null) return null;
|
|
||||||
try {
|
|
||||||
info = Writables.getHRegionInfoForMigration(bytes);
|
|
||||||
} catch(IOException ioe) {
|
|
||||||
if (ioe.getMessage().equalsIgnoreCase("HTD not found in input buffer")) {
|
|
||||||
return null;
|
|
||||||
} else {
|
|
||||||
throw ioe;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
LOG.info("Current INFO from scan results = " + info);
|
|
||||||
return info;
|
|
||||||
}
|
|
||||||
|
|
||||||
public static HRegionInfo getHRegionInfo(
|
public static HRegionInfo getHRegionInfo(
|
||||||
Result data) throws IOException {
|
Result data) throws IOException {
|
||||||
byte [] bytes =
|
byte [] bytes =
|
||||||
|
@ -369,20 +307,6 @@ public class MetaEditor {
|
||||||
return info;
|
return info;
|
||||||
}
|
}
|
||||||
|
|
||||||
private static Put addMetaUpdateStatus(final Put p) {
|
|
||||||
p.add(HConstants.CATALOG_FAMILY, HConstants.META_MIGRATION_QUALIFIER,
|
|
||||||
Bytes.toBytes("true"));
|
|
||||||
return p;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
private static Put addMetaUpdateStatus(final Put p, final boolean metaUpdated) {
|
|
||||||
p.add(HConstants.CATALOG_FAMILY, HConstants.META_MIGRATION_QUALIFIER,
|
|
||||||
Bytes.toBytes(metaUpdated));
|
|
||||||
return p;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
private static Put addRegionInfo(final Put p, final HRegionInfo hri)
|
private static Put addRegionInfo(final Put p, final HRegionInfo hri)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
p.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
|
p.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
|
||||||
|
@ -397,4 +321,4 @@ public class MetaEditor {
|
||||||
Bytes.toBytes(sn.getStartcode()));
|
Bytes.toBytes(sn.getStartcode()));
|
||||||
return p;
|
return p;
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -0,0 +1,242 @@
|
||||||
|
/**
|
||||||
|
* 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.List;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
|
import org.apache.hadoop.hbase.HRegionInfo;
|
||||||
|
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
|
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.migration.HRegionInfo090x;
|
||||||
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
import org.apache.hadoop.hbase.util.Writables;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tools to help with migration of meta tables so they no longer host
|
||||||
|
* instances of HTableDescriptor.
|
||||||
|
* @deprecated Used migration from 0.90 to 0.92 so will be going away in next
|
||||||
|
* release
|
||||||
|
*/
|
||||||
|
public class MetaMigrationRemovingHTD {
|
||||||
|
private static final Log LOG = LogFactory.getLog(MetaMigrationRemovingHTD.class);
|
||||||
|
|
||||||
|
/** The metaupdated column qualifier */
|
||||||
|
public static final byte [] META_MIGRATION_QUALIFIER =
|
||||||
|
Bytes.toBytes("metamigrated");
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Update legacy META rows, removing HTD from HRI.
|
||||||
|
* @param masterServices
|
||||||
|
* @return
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public static List<HTableDescriptor> updateMetaWithNewRegionInfo(
|
||||||
|
final MasterServices masterServices)
|
||||||
|
throws IOException {
|
||||||
|
final List<HTableDescriptor> htds = new ArrayList<HTableDescriptor>();
|
||||||
|
Visitor v = new Visitor() {
|
||||||
|
@Override
|
||||||
|
public boolean visit(Result r) throws IOException {
|
||||||
|
if (r == null || r.isEmpty()) return true;
|
||||||
|
HRegionInfo090x hrfm = MetaMigrationRemovingHTD.getHRegionInfoForMigration(r);
|
||||||
|
if (hrfm == null) return true;
|
||||||
|
htds.add(hrfm.getTableDesc());
|
||||||
|
masterServices.getMasterFileSystem()
|
||||||
|
.createTableDescriptor(hrfm.getTableDesc());
|
||||||
|
updateHRI(masterServices.getCatalogTracker(), false, hrfm);
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
MetaReader.fullScan(masterServices.getCatalogTracker(), v);
|
||||||
|
MetaMigrationRemovingHTD.updateRootWithMetaMigrationStatus(masterServices.getCatalogTracker(), true);
|
||||||
|
return htds;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Update the ROOT with new HRI. (HRI with no HTD)
|
||||||
|
* @param masterServices
|
||||||
|
* @return
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public static List<HTableDescriptor> updateRootWithNewRegionInfo(
|
||||||
|
final MasterServices masterServices)
|
||||||
|
throws IOException {
|
||||||
|
final List<HTableDescriptor> htds = new ArrayList<HTableDescriptor>();
|
||||||
|
Visitor v = new Visitor() {
|
||||||
|
@Override
|
||||||
|
public boolean visit(Result r) throws IOException {
|
||||||
|
if (r == null || r.isEmpty()) return true;
|
||||||
|
HRegionInfo090x hrfm = MetaMigrationRemovingHTD.getHRegionInfoForMigration(r);
|
||||||
|
if (hrfm == null) return true;
|
||||||
|
htds.add(hrfm.getTableDesc());
|
||||||
|
masterServices.getMasterFileSystem().createTableDescriptor(
|
||||||
|
hrfm.getTableDesc());
|
||||||
|
updateHRI(masterServices.getCatalogTracker(), true, hrfm);
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
MetaReader.fullScan(masterServices.getCatalogTracker(), v, null, true);
|
||||||
|
return htds;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Migrate root and meta to newer version. This updates the META and ROOT
|
||||||
|
* and removes the HTD from HRI.
|
||||||
|
* @param masterServices
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public static void migrateRootAndMeta(final MasterServices masterServices)
|
||||||
|
throws IOException {
|
||||||
|
updateRootWithNewRegionInfo(masterServices);
|
||||||
|
updateMetaWithNewRegionInfo(masterServices);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Update the metamigrated flag in -ROOT-.
|
||||||
|
* @param catalogTracker
|
||||||
|
* @param metaUpdated
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public static void updateRootWithMetaMigrationStatus(
|
||||||
|
CatalogTracker catalogTracker, boolean metaUpdated)
|
||||||
|
throws IOException {
|
||||||
|
Put p = new Put(HRegionInfo.ROOT_REGIONINFO.getRegionName());
|
||||||
|
MetaMigrationRemovingHTD.addMetaUpdateStatus(p, metaUpdated);
|
||||||
|
MetaEditor.putToRootTable(catalogTracker, p);
|
||||||
|
LOG.info("Updated -ROOT- row with metaMigrated status = " + metaUpdated);
|
||||||
|
}
|
||||||
|
|
||||||
|
static void updateHRI(final CatalogTracker ct, final boolean rootTable,
|
||||||
|
final HRegionInfo090x hRegionInfo090x)
|
||||||
|
throws IOException {
|
||||||
|
HRegionInfo regionInfo = new HRegionInfo(hRegionInfo090x);
|
||||||
|
Put p = new Put(regionInfo.getRegionName());
|
||||||
|
p.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
|
||||||
|
Writables.getBytes(regionInfo));
|
||||||
|
if (rootTable) {
|
||||||
|
MetaEditor.putToRootTable(ct, p);
|
||||||
|
} else {
|
||||||
|
MetaEditor.putToMetaTable(ct, p);
|
||||||
|
}
|
||||||
|
LOG.info("Updated region " + regionInfo + " to " +
|
||||||
|
(rootTable? "-ROOT-": ".META."));
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @deprecated Going away in 0.94; used for migrating to 0.92 only.
|
||||||
|
*/
|
||||||
|
public static HRegionInfo090x getHRegionInfoForMigration(
|
||||||
|
Result data) throws IOException {
|
||||||
|
HRegionInfo090x info = null;
|
||||||
|
byte [] bytes =
|
||||||
|
data.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
|
||||||
|
if (bytes == null) return null;
|
||||||
|
try {
|
||||||
|
info = Writables.getHRegionInfoForMigration(bytes);
|
||||||
|
} catch(IOException ioe) {
|
||||||
|
if (ioe.getMessage().equalsIgnoreCase("HTD not found in input buffer")) {
|
||||||
|
return null;
|
||||||
|
} else {
|
||||||
|
throw ioe;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
LOG.info("Current INFO from scan results = " + info);
|
||||||
|
return info;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static List<HRegionInfo090x> fullScanMetaAndPrintHRIM(
|
||||||
|
CatalogTracker catalogTracker)
|
||||||
|
throws IOException {
|
||||||
|
final List<HRegionInfo090x> regions =
|
||||||
|
new ArrayList<HRegionInfo090x>();
|
||||||
|
Visitor v = new Visitor() {
|
||||||
|
@Override
|
||||||
|
public boolean visit(Result r) throws IOException {
|
||||||
|
if (r == null || r.isEmpty()) return true;
|
||||||
|
LOG.info("fullScanMetaAndPrint1.Current Meta Result: " + r);
|
||||||
|
HRegionInfo090x hrim = getHRegionInfoForMigration(r);
|
||||||
|
LOG.info("fullScanMetaAndPrint.HRIM Print= " + hrim);
|
||||||
|
regions.add(hrim);
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
MetaReader.fullScan(catalogTracker, v);
|
||||||
|
return regions;
|
||||||
|
}
|
||||||
|
|
||||||
|
static Put addMetaUpdateStatus(final Put p, final boolean metaUpdated) {
|
||||||
|
p.add(HConstants.CATALOG_FAMILY,
|
||||||
|
MetaMigrationRemovingHTD.META_MIGRATION_QUALIFIER,
|
||||||
|
Bytes.toBytes(metaUpdated));
|
||||||
|
return p;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return True if the meta table has been migrated.
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
// Public because used in tests
|
||||||
|
public static boolean isMetaHRIUpdated(final MasterServices services)
|
||||||
|
throws IOException {
|
||||||
|
boolean metaUpdated = false;
|
||||||
|
List<Result> results =
|
||||||
|
MetaReader.fullScanOfRoot(services.getCatalogTracker());
|
||||||
|
if (results == null || results.isEmpty()) {
|
||||||
|
LOG.info("metaUpdated = NULL.");
|
||||||
|
return metaUpdated;
|
||||||
|
}
|
||||||
|
// Presume only the one result.
|
||||||
|
Result r = results.get(0);
|
||||||
|
byte [] metaMigrated = r.getValue(HConstants.CATALOG_FAMILY,
|
||||||
|
MetaMigrationRemovingHTD.META_MIGRATION_QUALIFIER);
|
||||||
|
if (metaMigrated != null && metaMigrated.length > 0) {
|
||||||
|
metaUpdated = Bytes.toBoolean(metaMigrated);
|
||||||
|
}
|
||||||
|
LOG.info("Meta updated status = " + metaUpdated);
|
||||||
|
return metaUpdated;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return True if migrated.
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public static boolean updateMetaWithNewHRI(final MasterServices services)
|
||||||
|
throws IOException {
|
||||||
|
if (isMetaHRIUpdated(services)) {
|
||||||
|
LOG.info("ROOT/Meta already up-to date with new HRI.");
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
LOG.info("Meta has HRI with HTDs. Updating meta now.");
|
||||||
|
try {
|
||||||
|
migrateRootAndMeta(services);
|
||||||
|
LOG.info("ROOT and Meta updated with new HRI.");
|
||||||
|
return true;
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new RuntimeException("Update ROOT/Meta with new HRI failed." +
|
||||||
|
"Master startup aborted.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -1,6 +1,4 @@
|
||||||
/**
|
/**
|
||||||
* Copyright 2010 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
|
@ -26,16 +24,20 @@ import java.util.Map;
|
||||||
import java.util.NavigableMap;
|
import java.util.NavigableMap;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.TreeMap;
|
import java.util.TreeMap;
|
||||||
import java.util.TreeSet;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.hbase.*;
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
|
import org.apache.hadoop.hbase.HRegionInfo;
|
||||||
|
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.NotServingRegionException;
|
||||||
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
import org.apache.hadoop.hbase.client.Get;
|
import org.apache.hadoop.hbase.client.Get;
|
||||||
|
import org.apache.hadoop.hbase.client.HTable;
|
||||||
import org.apache.hadoop.hbase.client.Result;
|
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.Scan;
|
||||||
import org.apache.hadoop.hbase.ipc.HRegionInterface;
|
import org.apache.hadoop.hbase.ipc.HRegionInterface;
|
||||||
import org.apache.hadoop.hbase.migration.HRegionInfo090x;
|
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.util.Pair;
|
import org.apache.hadoop.hbase.util.Pair;
|
||||||
import org.apache.hadoop.hbase.util.Writables;
|
import org.apache.hadoop.hbase.util.Writables;
|
||||||
|
@ -43,14 +45,14 @@ import org.apache.hadoop.ipc.RemoteException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Reads region and assignment information from <code>.META.</code>.
|
* Reads region and assignment information from <code>.META.</code>.
|
||||||
* <p>
|
|
||||||
* Uses the {@link CatalogTracker} to obtain locations and connections to
|
|
||||||
* catalogs.
|
|
||||||
*/
|
*/
|
||||||
public class MetaReader {
|
public class 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(MetaReader.class);
|
private static final Log LOG = LogFactory.getLog(MetaReader.class);
|
||||||
|
|
||||||
public static final byte [] META_REGION_PREFIX;
|
static final byte [] META_REGION_PREFIX;
|
||||||
static {
|
static {
|
||||||
// Copy the prefix from FIRST_META_REGIONINFO into META_REGION_PREFIX.
|
// Copy the prefix from FIRST_META_REGIONINFO into META_REGION_PREFIX.
|
||||||
// FIRST_META_REGIONINFO == '.META.,,1'. META_REGION_PREFIX == '.META.,'
|
// FIRST_META_REGIONINFO == '.META.,,1'. META_REGION_PREFIX == '.META.,'
|
||||||
|
@ -60,48 +62,6 @@ public class MetaReader {
|
||||||
META_REGION_PREFIX, 0, len);
|
META_REGION_PREFIX, 0, len);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* @param ct
|
|
||||||
* @param tableName A user tablename or a .META. table name.
|
|
||||||
* @return Interface on to server hosting the <code>-ROOT-</code> or
|
|
||||||
* <code>.META.</code> regions.
|
|
||||||
* @throws NotAllMetaRegionsOnlineException
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
private static HRegionInterface getCatalogRegionInterface(final CatalogTracker ct,
|
|
||||||
final byte [] tableName)
|
|
||||||
throws NotAllMetaRegionsOnlineException, IOException {
|
|
||||||
return Bytes.equals(HConstants.META_TABLE_NAME, tableName)?
|
|
||||||
ct.waitForRootServerConnectionDefault():
|
|
||||||
ct.waitForMetaServerConnectionDefault();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @param tableName
|
|
||||||
* @return Returns region name to look in for regions for <code>tableName</code>;
|
|
||||||
* e.g. if we are looking for <code>.META.</code> regions, we need to look
|
|
||||||
* in the <code>-ROOT-</code> region, else if a user table, we need to look
|
|
||||||
* in the <code>.META.</code> region.
|
|
||||||
*/
|
|
||||||
private static byte [] getCatalogRegionNameForTable(final byte [] tableName) {
|
|
||||||
return Bytes.equals(HConstants.META_TABLE_NAME, tableName)?
|
|
||||||
HRegionInfo.ROOT_REGIONINFO.getRegionName():
|
|
||||||
HRegionInfo.FIRST_META_REGIONINFO.getRegionName();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @param regionName
|
|
||||||
* @return Returns region name to look in for <code>regionName</code>;
|
|
||||||
* e.g. if we are looking for <code>.META.,,1</code> region, we need to look
|
|
||||||
* in <code>-ROOT-</code> region, else if a user region, we need to look
|
|
||||||
* in the <code>.META.,,1</code> region.
|
|
||||||
*/
|
|
||||||
private static byte [] getCatalogRegionNameForRegion(final byte [] regionName) {
|
|
||||||
return isMetaRegion(regionName)?
|
|
||||||
HRegionInfo.ROOT_REGIONINFO.getRegionName():
|
|
||||||
HRegionInfo.FIRST_META_REGIONINFO.getRegionName();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @param regionName
|
* @param regionName
|
||||||
* @return True if <code>regionName</code> is from <code>.META.</code> table.
|
* @return True if <code>regionName</code> is from <code>.META.</code> table.
|
||||||
|
@ -117,34 +77,14 @@ public class MetaReader {
|
||||||
META_REGION_PREFIX, 0, META_REGION_PREFIX.length);
|
META_REGION_PREFIX, 0, META_REGION_PREFIX.length);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Performs a full scan of <code>.META.</code>.
|
|
||||||
* <p>
|
|
||||||
* Returns a map of every region to it's currently assigned server, according
|
|
||||||
* to META. If the region does not have an assignment it will have a null
|
|
||||||
* value in the map.
|
|
||||||
*
|
|
||||||
* @return map of regions to their currently assigned server where server is
|
|
||||||
* a String of <host> ':' <port>
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
public static Map<HRegionInfo, ServerName> fullScan(
|
|
||||||
CatalogTracker catalogTracker)
|
|
||||||
throws IOException {
|
|
||||||
return fullScan(catalogTracker, new TreeSet<String>());
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Performs a full scan of <code>.META.</code>, skipping regions from any
|
* Performs a full scan of <code>.META.</code>, skipping regions from any
|
||||||
* tables in the specified set of disabled tables.
|
* tables in the specified set of disabled tables.
|
||||||
* <p>
|
|
||||||
* Returns a map of every region to it's currently assigned server, according
|
|
||||||
* to META. If the region does not have an assignment it will have a null
|
|
||||||
* value in the map.
|
|
||||||
*
|
|
||||||
* @param catalogTracker
|
* @param catalogTracker
|
||||||
* @param disabledTables set of disabled tables that will not be returned
|
* @param disabledTables set of disabled tables that will not be returned
|
||||||
* @return map of regions to their currently assigned server
|
* @return Returns a map of every region to it's currently assigned server,
|
||||||
|
* according to META. If the region does not have an assignment it will have
|
||||||
|
* a null value in the map.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public static Map<HRegionInfo, ServerName> fullScan(
|
public static Map<HRegionInfo, ServerName> fullScan(
|
||||||
|
@ -156,16 +96,13 @@ public class MetaReader {
|
||||||
/**
|
/**
|
||||||
* Performs a full scan of <code>.META.</code>, skipping regions from any
|
* Performs a full scan of <code>.META.</code>, skipping regions from any
|
||||||
* tables in the specified set of disabled tables.
|
* tables in the specified set of disabled tables.
|
||||||
* <p>
|
|
||||||
* Returns a map of every region to it's currently assigned server, according
|
|
||||||
* to META. If the region does not have an assignment it will have a null
|
|
||||||
* value in the map.
|
|
||||||
*
|
|
||||||
* @param catalogTracker
|
* @param catalogTracker
|
||||||
* @param disabledTables set of disabled tables that will not be returned
|
* @param disabledTables set of disabled tables that will not be returned
|
||||||
* @param excludeOfflinedSplitParents If true, do not include offlined split
|
* @param excludeOfflinedSplitParents If true, do not include offlined split
|
||||||
* parents in the return.
|
* parents in the return.
|
||||||
* @return map of regions to their currently assigned server
|
* @return Returns a map of every region to it's currently assigned server,
|
||||||
|
* according to META. If the region does not have an assignment it will have
|
||||||
|
* a null value in the map.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public static Map<HRegionInfo, ServerName> fullScan(
|
public static Map<HRegionInfo, ServerName> fullScan(
|
||||||
|
@ -178,7 +115,7 @@ public class MetaReader {
|
||||||
@Override
|
@Override
|
||||||
public boolean visit(Result r) throws IOException {
|
public boolean visit(Result r) throws IOException {
|
||||||
if (r == null || r.isEmpty()) return true;
|
if (r == null || r.isEmpty()) return true;
|
||||||
Pair<HRegionInfo, ServerName> region = metaRowToRegionPair(r);
|
Pair<HRegionInfo, ServerName> region = parseCatalogResult(r);
|
||||||
if (region == null) return true;
|
if (region == null) return true;
|
||||||
HRegionInfo hri = region.getFirst();
|
HRegionInfo hri = region.getFirst();
|
||||||
if (disabledTables.contains(
|
if (disabledTables.contains(
|
||||||
|
@ -195,38 +132,32 @@ public class MetaReader {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Performs a full scan of <code>.META.</code>.
|
* Performs a full scan of <code>.META.</code>.
|
||||||
* <p>
|
* @return List of {@link Result}
|
||||||
* Returns a map of every region to it's currently assigned server, according
|
|
||||||
* to META. If the region does not have an assignment it will have a null
|
|
||||||
* value in the map.
|
|
||||||
*
|
|
||||||
* @return map of regions to their currently assigned server
|
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public static List<Result> fullScanOfResults(
|
public static List<Result> fullScan(CatalogTracker catalogTracker)
|
||||||
CatalogTracker catalogTracker)
|
|
||||||
throws IOException {
|
throws IOException {
|
||||||
final List<Result> regions = new ArrayList<Result>();
|
CollectAllVisitor v = new CollectAllVisitor();
|
||||||
Visitor v = new Visitor() {
|
fullScan(catalogTracker, v, null);
|
||||||
@Override
|
return v.getResults();
|
||||||
public boolean visit(Result r) throws IOException {
|
}
|
||||||
if (r == null || r.isEmpty()) return true;
|
|
||||||
regions.add(r);
|
/**
|
||||||
return true;
|
* Performs a full scan of a <code>-ROOT-</code> table.
|
||||||
}
|
* @return List of {@link Result}
|
||||||
};
|
* @throws IOException
|
||||||
fullScan(catalogTracker, v);
|
*/
|
||||||
return regions;
|
public static List<Result> fullScanOfRoot(CatalogTracker catalogTracker)
|
||||||
|
throws IOException {
|
||||||
|
CollectAllVisitor v = new CollectAllVisitor();
|
||||||
|
fullScan(catalogTracker, v, null, true);
|
||||||
|
return v.getResults();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Performs a full scan of <code>.META.</code>.
|
* Performs a full scan of <code>.META.</code>.
|
||||||
* <p>
|
|
||||||
* Returns a map of every region to it's currently assigned server, according
|
|
||||||
* to META. If the region does not have an assignment it will have a null
|
|
||||||
* value in the map.
|
|
||||||
* @param catalogTracker
|
* @param catalogTracker
|
||||||
* @param visitor
|
* @param visitor Visitor invoked against each row.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public static void fullScan(CatalogTracker catalogTracker,
|
public static void fullScan(CatalogTracker catalogTracker,
|
||||||
|
@ -237,24 +168,82 @@ public class MetaReader {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Performs a full scan of <code>.META.</code>.
|
* Performs a full scan of <code>.META.</code>.
|
||||||
* <p>
|
|
||||||
* Returns a map of every region to it's currently assigned server, according
|
|
||||||
* to META. If the region does not have an assignment it will have a null
|
|
||||||
* value in the map.
|
|
||||||
* @param catalogTracker
|
* @param catalogTracker
|
||||||
* @param visitor
|
* @param visitor Visitor invoked against each row.
|
||||||
* @param startrow Where to start the scan. Pass null if want to begin scan
|
* @param startrow Where to start the scan. Pass null if want to begin scan
|
||||||
* at first row.
|
* at first row (The visitor will stop the Scan when its done so no need to
|
||||||
|
* pass a stoprow).
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public static void fullScan(CatalogTracker catalogTracker,
|
public static void fullScan(CatalogTracker catalogTracker,
|
||||||
final Visitor visitor, final byte [] startrow)
|
final Visitor visitor, final byte [] startrow)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
HRegionInterface metaServer =
|
fullScan(catalogTracker, visitor, startrow, false);
|
||||||
catalogTracker.waitForMetaServerConnectionDefault();
|
}
|
||||||
fullScan(metaServer, visitor,
|
|
||||||
HRegionInfo.FIRST_META_REGIONINFO.getRegionName(), startrow);
|
/**
|
||||||
return;
|
* 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
|
||||||
|
*/
|
||||||
|
private static HTable getHTable(final CatalogTracker catalogTracker,
|
||||||
|
final byte [] tableName)
|
||||||
|
throws IOException {
|
||||||
|
// Passing the CatalogTracker's connection configuration ensures this
|
||||||
|
// HTable instance uses the CatalogTracker's connection.
|
||||||
|
return new HTable(catalogTracker.getConnection().getConfiguration(), tableName);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Callers should call close on the returned {@link HTable} instance.
|
||||||
|
* @param catalogTracker
|
||||||
|
* @param regionName
|
||||||
|
* @return
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
static HTable getCatalogHTable(final CatalogTracker catalogTracker,
|
||||||
|
final byte [] regionName)
|
||||||
|
throws IOException {
|
||||||
|
return isMetaRegion(regionName)?
|
||||||
|
getRootHTable(catalogTracker):
|
||||||
|
getMetaHTable(catalogTracker);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Callers should call close on the returned {@link HTable} instance.
|
||||||
|
* @param ct
|
||||||
|
* @return An {@link HTable} for <code>.META.</code>
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
static HTable getMetaHTable(final CatalogTracker ct)
|
||||||
|
throws IOException {
|
||||||
|
return getHTable(ct, HConstants.META_TABLE_NAME);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Callers should call close on the returned {@link HTable} instance.
|
||||||
|
* @param ct
|
||||||
|
* @return An {@link HTable} for <code>-ROOT-</code>
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
static HTable getRootHTable(final CatalogTracker ct)
|
||||||
|
throws IOException {
|
||||||
|
return getHTable(ct, HConstants.ROOT_TABLE_NAME);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param t Table to use (will be closed when done).
|
||||||
|
* @param g Get to run
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
private static Result get(final HTable t, final Get g) throws IOException {
|
||||||
|
try {
|
||||||
|
return t.get(g);
|
||||||
|
} finally {
|
||||||
|
t.close();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -262,28 +251,29 @@ public class MetaReader {
|
||||||
* @param metaServer connection to server hosting ROOT
|
* @param metaServer connection to server hosting ROOT
|
||||||
* @return location of META in ROOT where location, or null if not available
|
* @return location of META in ROOT where location, or null if not available
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
|
* @deprecated Does not retry; use {@link #readRegionLocation(CatalogTracker, byte[])
|
||||||
*/
|
*/
|
||||||
public static ServerName readMetaLocation(HRegionInterface metaServer)
|
public static ServerName readMetaLocation(HRegionInterface metaServer)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
return readLocation(metaServer, CatalogTracker.ROOT_REGION,
|
return readLocation(metaServer, CatalogTracker.ROOT_REGION_NAME,
|
||||||
CatalogTracker.META_REGION);
|
CatalogTracker.META_REGION_NAME);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Reads the location of the specified region from META.
|
* Reads the location of the specified region
|
||||||
* @param catalogTracker
|
* @param catalogTracker
|
||||||
* @param regionName region to read location of
|
* @param regionName region whose location we are after
|
||||||
* @return location of META in ROOT where location is, or null if not available
|
* @return location of region as a {@link ServerName} or null if not found
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public static ServerName readRegionLocation(CatalogTracker catalogTracker,
|
public static ServerName readRegionLocation(CatalogTracker catalogTracker,
|
||||||
byte [] regionName)
|
byte [] regionName)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
if (isMetaRegion(regionName)) throw new IllegalArgumentException("See readMetaLocation");
|
Pair<HRegionInfo, ServerName> pair = getRegion(catalogTracker, regionName);
|
||||||
return readLocation(catalogTracker.waitForMetaServerConnectionDefault(),
|
return (pair == null || pair.getSecond() == null)? null: pair.getSecond();
|
||||||
CatalogTracker.META_REGION, regionName);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// TODO: Remove when deprecated dependencies are removed.
|
||||||
private static ServerName readLocation(HRegionInterface metaServer,
|
private static ServerName readLocation(HRegionInterface metaServer,
|
||||||
byte [] catalogRegionName, byte [] regionName)
|
byte [] catalogRegionName, byte [] regionName)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
|
@ -324,15 +314,14 @@ public class MetaReader {
|
||||||
if (r == null || r.isEmpty()) {
|
if (r == null || r.isEmpty()) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
return getServerNameFromResult(r);
|
return getServerNameFromCatalogResult(r);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Gets the region info and assignment for the specified region from META.
|
* Gets the region info and assignment for the specified region.
|
||||||
* @param catalogTracker
|
* @param catalogTracker
|
||||||
* @param regionName
|
* @param regionName Region to lookup.
|
||||||
* @return location of META in ROOT where location is
|
* @return Location and HRegionInfo for <code>regionName</code>
|
||||||
* a String of <host> ':' <port>, or null if not available
|
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public static Pair<HRegionInfo, ServerName> getRegion(
|
public static Pair<HRegionInfo, ServerName> getRegion(
|
||||||
|
@ -340,41 +329,60 @@ public class MetaReader {
|
||||||
throws IOException {
|
throws IOException {
|
||||||
Get get = new Get(regionName);
|
Get get = new Get(regionName);
|
||||||
get.addFamily(HConstants.CATALOG_FAMILY);
|
get.addFamily(HConstants.CATALOG_FAMILY);
|
||||||
byte [] meta = getCatalogRegionNameForRegion(regionName);
|
Result r = get(getCatalogHTable(catalogTracker, regionName), get);
|
||||||
Result r = catalogTracker.waitForMetaServerConnectionDefault().get(meta, get);
|
return (r == null || r.isEmpty())? null: parseCatalogResult(r);
|
||||||
return (r == null || r.isEmpty())? null: metaRowToRegionPair(r);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @param data A .META. table row.
|
* Extract a {@link ServerName}
|
||||||
* @return A pair of the regioninfo and the ServerName
|
* For use on catalog table {@link Result}.
|
||||||
|
* @param r Result to pull from
|
||||||
|
* @return A ServerName instance or null if necessary fields not found or empty.
|
||||||
|
*/
|
||||||
|
public static ServerName getServerNameFromCatalogResult(final Result r) {
|
||||||
|
byte[] value = r.getValue(HConstants.CATALOG_FAMILY,
|
||||||
|
HConstants.SERVER_QUALIFIER);
|
||||||
|
if (value == null || value.length == 0) return null;
|
||||||
|
String hostAndPort = Bytes.toString(value);
|
||||||
|
value = r.getValue(HConstants.CATALOG_FAMILY,
|
||||||
|
HConstants.STARTCODE_QUALIFIER);
|
||||||
|
if (value == null || value.length == 0) return null;
|
||||||
|
return new ServerName(hostAndPort, Bytes.toLong(value));
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Extract a HRegionInfo and ServerName.
|
||||||
|
* For use on catalog table {@link Result}.
|
||||||
|
* @param r Result to pull from
|
||||||
|
* @return A pair of the {@link HRegionInfo} and the {@link ServerName}
|
||||||
* (or null for server address if no address set in .META.).
|
* (or null for server address if no address set in .META.).
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public static Pair<HRegionInfo, ServerName> metaRowToRegionPair(Result data)
|
public static Pair<HRegionInfo, ServerName> parseCatalogResult(final Result r)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
byte [] bytes = data.getValue(HConstants.CATALOG_FAMILY,
|
HRegionInfo info =
|
||||||
HConstants.REGIONINFO_QUALIFIER);
|
parseHRegionInfoFromCatalogResult(r, HConstants.REGIONINFO_QUALIFIER);
|
||||||
if (bytes == null) return null;
|
ServerName sn = getServerNameFromCatalogResult(r);
|
||||||
HRegionInfo info = Writables.getHRegionInfo(bytes);
|
|
||||||
ServerName sn = getServerNameFromResult(data);
|
|
||||||
// sn can be null in case where no server inof.
|
|
||||||
return new Pair<HRegionInfo, ServerName>(info, sn);
|
return new Pair<HRegionInfo, ServerName>(info, sn);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @param data Result to interrogate.
|
* Parse the content of the cell at {@link HConstants#CATALOG_FAMILY} and
|
||||||
* @return A ServerName instance or null if necessary fields not found or empty.
|
* <code>qualifier</code> as an HRegionInfo and return it, or null.
|
||||||
|
* For use on catalog table {@link Result}.
|
||||||
|
* @param r Result instance to pull from.
|
||||||
|
* @param qualifier Column family qualifier -- either
|
||||||
|
* {@link HConstants#SPLITA_QUALIFIER}, {@link HConstants#SPLITB_QUALIFIER} or
|
||||||
|
* {@link HConstants#REGIONINFO_QUALIFIER}.
|
||||||
|
* @return An HRegionInfo instance or null.
|
||||||
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
private static ServerName getServerNameFromResult(final Result data) {
|
public static HRegionInfo parseHRegionInfoFromCatalogResult(final Result r,
|
||||||
byte[] value = data.getValue(HConstants.CATALOG_FAMILY,
|
byte [] qualifier)
|
||||||
HConstants.SERVER_QUALIFIER);
|
throws IOException {
|
||||||
if (value == null || value.length == 0) return null;
|
byte [] bytes = r.getValue(HConstants.CATALOG_FAMILY, qualifier);
|
||||||
String hostAndPort = Bytes.toString(value);
|
if (bytes == null || bytes.length <= 0) return null;
|
||||||
value = data.getValue(HConstants.CATALOG_FAMILY,
|
return Writables.getHRegionInfoOrNull(bytes);
|
||||||
HConstants.STARTCODE_QUALIFIER);
|
|
||||||
if (value == null || value.length == 0) return null;
|
|
||||||
return new ServerName(hostAndPort, Bytes.toLong(value));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -393,21 +401,36 @@ public class MetaReader {
|
||||||
// Catalog tables always exist.
|
// Catalog tables always exist.
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
HRegionInterface metaServer =
|
final byte [] tableNameBytes = Bytes.toBytes(tableName);
|
||||||
catalogTracker.waitForMetaServerConnectionDefault();
|
// Make a version of ResultCollectingVisitor that only collects the first
|
||||||
Scan scan = getScanForTableName(Bytes.toBytes(tableName));
|
CollectingVisitor<HRegionInfo> visitor = new CollectingVisitor<HRegionInfo>() {
|
||||||
scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
|
private HRegionInfo current = null;
|
||||||
long scannerid = metaServer.openScanner(
|
|
||||||
HRegionInfo.FIRST_META_REGIONINFO.getRegionName(), scan);
|
@Override
|
||||||
try {
|
public boolean visit(Result r) throws IOException {
|
||||||
Result data = metaServer.next(scannerid);
|
this.current =
|
||||||
if (data != null && data.size() > 0) {
|
parseHRegionInfoFromCatalogResult(r, HConstants.REGIONINFO_QUALIFIER);
|
||||||
|
if (this.current == null) {
|
||||||
|
LOG.warn("No serialized HRegionInfo in " + r);
|
||||||
return true;
|
return true;
|
||||||
|
}
|
||||||
|
if (!isInsideTable(this.current, tableNameBytes)) return false;
|
||||||
|
if (this.current.isSplitParent()) return true;
|
||||||
|
// Else call super and add this Result to the collection.
|
||||||
|
super.visit(r);
|
||||||
|
// Stop collecting regions from table after we get one.
|
||||||
|
return false;
|
||||||
}
|
}
|
||||||
return false;
|
|
||||||
} finally {
|
@Override
|
||||||
metaServer.close(scannerid);
|
void add(Result r) {
|
||||||
}
|
// Add the current HRI.
|
||||||
|
this.results.add(this.current);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
fullScan(catalogTracker, visitor, getTableStartRowForMeta(tableNameBytes));
|
||||||
|
// If visitor has results >= 1 then table exists.
|
||||||
|
return visitor.getResults().size() >= 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -435,42 +458,46 @@ public class MetaReader {
|
||||||
public static List<HRegionInfo> getTableRegions(CatalogTracker catalogTracker,
|
public static List<HRegionInfo> getTableRegions(CatalogTracker catalogTracker,
|
||||||
byte [] tableName, final boolean excludeOfflinedSplitParents)
|
byte [] tableName, final boolean excludeOfflinedSplitParents)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
|
List<Pair<HRegionInfo, ServerName>> result = null;
|
||||||
// If root, do a bit of special handling.
|
|
||||||
List<HRegionInfo> list = new ArrayList<HRegionInfo>();
|
|
||||||
list.add(HRegionInfo.ROOT_REGIONINFO);
|
|
||||||
return list;
|
|
||||||
} else if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) {
|
|
||||||
// Same for .META. table
|
|
||||||
List<HRegionInfo> list = new ArrayList<HRegionInfo>();
|
|
||||||
list.add(HRegionInfo.FIRST_META_REGIONINFO);
|
|
||||||
return list;
|
|
||||||
}
|
|
||||||
|
|
||||||
// Its a user table.
|
|
||||||
HRegionInterface metaServer =
|
|
||||||
getCatalogRegionInterface(catalogTracker, tableName);
|
|
||||||
List<HRegionInfo> regions = new ArrayList<HRegionInfo>();
|
|
||||||
|
|
||||||
Scan scan = getScanForTableName(tableName);
|
|
||||||
scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
|
|
||||||
long scannerid =
|
|
||||||
metaServer.openScanner(getCatalogRegionNameForTable(tableName), scan);
|
|
||||||
try {
|
try {
|
||||||
Result data;
|
result = getTableRegionsAndLocations(catalogTracker, tableName,
|
||||||
while((data = metaServer.next(scannerid)) != null) {
|
excludeOfflinedSplitParents);
|
||||||
if (data != null && data.size() > 0) {
|
} catch (InterruptedException e) {
|
||||||
HRegionInfo info = Writables.getHRegionInfo(
|
throw new RuntimeException(e);
|
||||||
data.getValue(HConstants.CATALOG_FAMILY,
|
|
||||||
HConstants.REGIONINFO_QUALIFIER));
|
|
||||||
if (excludeOfflinedSplitParents && info.isSplitParent()) continue;
|
|
||||||
regions.add(info);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return regions;
|
|
||||||
} finally {
|
|
||||||
metaServer.close(scannerid);
|
|
||||||
}
|
}
|
||||||
|
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 byte [] tableName) {
|
||||||
|
return Bytes.equals(tableName, current.getTableName());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param tableName
|
||||||
|
* @return Place to start Scan in <code>.META.</code> when passed a
|
||||||
|
* <code>tableName</code>; returns <tableName&rt; <,&rt; <,&rt;
|
||||||
|
*/
|
||||||
|
static byte [] getTableStartRowForMeta(final byte [] tableName) {
|
||||||
|
byte [] startRow = new byte[tableName.length + 2];
|
||||||
|
System.arraycopy(tableName, 0, startRow, 0, tableName.length);
|
||||||
|
startRow[startRow.length - 2] = HRegionInfo.DELIMITER;
|
||||||
|
startRow[startRow.length - 1] = HRegionInfo.DELIMITER;
|
||||||
|
return startRow;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -504,8 +531,22 @@ public class MetaReader {
|
||||||
public static List<Pair<HRegionInfo, ServerName>>
|
public static List<Pair<HRegionInfo, ServerName>>
|
||||||
getTableRegionsAndLocations(CatalogTracker catalogTracker, String tableName)
|
getTableRegionsAndLocations(CatalogTracker catalogTracker, String tableName)
|
||||||
throws IOException, InterruptedException {
|
throws IOException, InterruptedException {
|
||||||
byte [] tableNameBytes = Bytes.toBytes(tableName);
|
return getTableRegionsAndLocations(catalogTracker, Bytes.toBytes(tableName),
|
||||||
if (Bytes.equals(tableNameBytes, HConstants.ROOT_TABLE_NAME)) {
|
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 byte [] tableName, final boolean excludeOfflinedSplitParents)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
|
||||||
// If root, do a bit of special handling.
|
// If root, do a bit of special handling.
|
||||||
ServerName serverName = catalogTracker.getRootLocation();
|
ServerName serverName = catalogTracker.getRootLocation();
|
||||||
List<Pair<HRegionInfo, ServerName>> list =
|
List<Pair<HRegionInfo, ServerName>> list =
|
||||||
|
@ -514,27 +555,36 @@ public class MetaReader {
|
||||||
serverName));
|
serverName));
|
||||||
return list;
|
return list;
|
||||||
}
|
}
|
||||||
HRegionInterface metaServer =
|
// Make a version of CollectingVisitor that collects HRegionInfo and ServerAddress
|
||||||
getCatalogRegionInterface(catalogTracker, tableNameBytes);
|
CollectingVisitor<Pair<HRegionInfo, ServerName>> visitor =
|
||||||
List<Pair<HRegionInfo, ServerName>> regions =
|
new CollectingVisitor<Pair<HRegionInfo, ServerName>>() {
|
||||||
new ArrayList<Pair<HRegionInfo, ServerName>>();
|
private Pair<HRegionInfo, ServerName> current = null;
|
||||||
Scan scan = getScanForTableName(tableNameBytes);
|
|
||||||
scan.addFamily(HConstants.CATALOG_FAMILY);
|
@Override
|
||||||
long scannerid =
|
public boolean visit(Result r) throws IOException {
|
||||||
metaServer.openScanner(getCatalogRegionNameForTable(tableNameBytes), scan);
|
HRegionInfo hri =
|
||||||
try {
|
parseHRegionInfoFromCatalogResult(r, HConstants.REGIONINFO_QUALIFIER);
|
||||||
Result data;
|
if (hri == null) {
|
||||||
while((data = metaServer.next(scannerid)) != null) {
|
LOG.warn("No serialized HRegionInfo in " + r);
|
||||||
if (data != null && data.size() > 0) {
|
return true;
|
||||||
Pair<HRegionInfo, ServerName> region = metaRowToRegionPair(data);
|
|
||||||
if (region == null) continue;
|
|
||||||
regions.add(region);
|
|
||||||
}
|
}
|
||||||
|
if (!isInsideTable(hri, tableName)) return false;
|
||||||
|
if (excludeOfflinedSplitParents && hri.isSplitParent()) return true;
|
||||||
|
ServerName sn = getServerNameFromCatalogResult(r);
|
||||||
|
// Populate this.current so available when we call #add
|
||||||
|
this.current = new Pair<HRegionInfo, ServerName>(hri, sn);
|
||||||
|
// Else call super and add this Result to the collection.
|
||||||
|
return super.visit(r);
|
||||||
}
|
}
|
||||||
return regions;
|
|
||||||
} finally {
|
@Override
|
||||||
metaServer.close(scannerid);
|
void add(Result r) {
|
||||||
}
|
this.results.add(this.current);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
fullScan(catalogTracker, visitor, getTableStartRowForMeta(tableName),
|
||||||
|
Bytes.equals(tableName, HConstants.META_TABLE_NAME));
|
||||||
|
return visitor.getResults();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -547,36 +597,31 @@ public class MetaReader {
|
||||||
public static NavigableMap<HRegionInfo, Result>
|
public static NavigableMap<HRegionInfo, Result>
|
||||||
getServerUserRegions(CatalogTracker catalogTracker, final ServerName serverName)
|
getServerUserRegions(CatalogTracker catalogTracker, final ServerName serverName)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
HRegionInterface metaServer =
|
final NavigableMap<HRegionInfo, Result> hris = new TreeMap<HRegionInfo, Result>();
|
||||||
catalogTracker.waitForMetaServerConnectionDefault();
|
// Fill the above hris map with entries from .META. that have the passed
|
||||||
NavigableMap<HRegionInfo, Result> hris = new TreeMap<HRegionInfo, Result>();
|
// servername.
|
||||||
Scan scan = new Scan();
|
CollectingVisitor<Result> v = new CollectingVisitor<Result>() {
|
||||||
scan.addFamily(HConstants.CATALOG_FAMILY);
|
@Override
|
||||||
long scannerid = metaServer.openScanner(
|
void add(Result r) {
|
||||||
HRegionInfo.FIRST_META_REGIONINFO.getRegionName(), scan);
|
if (r == null || r.isEmpty()) return;
|
||||||
try {
|
ServerName sn = getServerNameFromCatalogResult(r);
|
||||||
Result result;
|
if (sn != null && sn.equals(serverName)) this.results.add(r);
|
||||||
while((result = metaServer.next(scannerid)) != null) {
|
}
|
||||||
if (result != null && result.size() > 0) {
|
};
|
||||||
Pair<HRegionInfo, ServerName> pair = metaRowToRegionPair(result);
|
fullScan(catalogTracker, v);
|
||||||
if (pair == null) continue;
|
List<Result> results = v.getResults();
|
||||||
if (pair.getSecond() == null || !serverName.equals(pair.getSecond())) {
|
if (results != null && !results.isEmpty()) {
|
||||||
continue;
|
// Convert results to Map keyed by HRI
|
||||||
}
|
for (Result r: results) {
|
||||||
hris.put(pair.getFirst(), result);
|
Pair<HRegionInfo, ServerName> p = parseCatalogResult(r);
|
||||||
}
|
if (p != null && p.getFirst() != null) hris.put(p.getFirst(), r);
|
||||||
}
|
}
|
||||||
return hris;
|
|
||||||
} finally {
|
|
||||||
metaServer.close(scannerid);
|
|
||||||
}
|
}
|
||||||
|
return hris;
|
||||||
}
|
}
|
||||||
|
|
||||||
public static void fullScanMetaAndPrint(
|
public static void fullScanMetaAndPrint(final CatalogTracker catalogTracker)
|
||||||
CatalogTracker catalogTracker)
|
|
||||||
throws IOException {
|
throws IOException {
|
||||||
final List<HRegionInfo090x> regions =
|
|
||||||
new ArrayList<HRegionInfo090x>();
|
|
||||||
Visitor v = new Visitor() {
|
Visitor v = new Visitor() {
|
||||||
@Override
|
@Override
|
||||||
public boolean visit(Result r) throws IOException {
|
public boolean visit(Result r) throws IOException {
|
||||||
|
@ -590,27 +635,6 @@ public class MetaReader {
|
||||||
fullScan(catalogTracker, v);
|
fullScan(catalogTracker, v);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
public static List<HRegionInfo090x> fullScanMetaAndPrintHRIM(
|
|
||||||
CatalogTracker catalogTracker)
|
|
||||||
throws IOException {
|
|
||||||
final List<HRegionInfo090x> regions =
|
|
||||||
new ArrayList<HRegionInfo090x>();
|
|
||||||
Visitor v = new Visitor() {
|
|
||||||
@Override
|
|
||||||
public boolean visit(Result r) throws IOException {
|
|
||||||
if (r == null || r.isEmpty()) return true;
|
|
||||||
LOG.info("fullScanMetaAndPrint1.Current Meta Result: " + r);
|
|
||||||
HRegionInfo090x hrim = MetaEditor.getHRegionInfoForMigration(r);
|
|
||||||
LOG.info("fullScanMetaAndPrint.HRIM Print= " + hrim);
|
|
||||||
regions.add(hrim);
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
};
|
|
||||||
fullScan(catalogTracker, v);
|
|
||||||
return regions;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Fully scan a given region, on a given server starting with given row.
|
* Fully scan a given region, on a given server starting with given row.
|
||||||
* @param hRegionInterface region server
|
* @param hRegionInterface region server
|
||||||
|
@ -618,6 +642,8 @@ public class MetaReader {
|
||||||
* @param regionName name of region
|
* @param regionName name of region
|
||||||
* @param startrow start row
|
* @param startrow start row
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
|
* @deprecated Does not retry; use fullScan xxx instead.
|
||||||
|
x
|
||||||
*/
|
*/
|
||||||
public static void fullScan(HRegionInterface hRegionInterface,
|
public static void fullScan(HRegionInterface hRegionInterface,
|
||||||
Visitor visitor, final byte[] regionName,
|
Visitor visitor, final byte[] regionName,
|
||||||
|
@ -638,6 +664,38 @@ public class MetaReader {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* 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.
|
||||||
|
* @param scanRoot True if we are to scan <code>-ROOT-</code> rather than
|
||||||
|
* <code>.META.</code>, the default (pass false to scan .META.)
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
static void fullScan(CatalogTracker catalogTracker,
|
||||||
|
final Visitor visitor, final byte [] startrow, final boolean scanRoot)
|
||||||
|
throws IOException {
|
||||||
|
Scan scan = new Scan();
|
||||||
|
if (startrow != null) scan.setStartRow(startrow);
|
||||||
|
scan.addFamily(HConstants.CATALOG_FAMILY);
|
||||||
|
HTable metaTable = scanRoot?
|
||||||
|
getRootHTable(catalogTracker): getMetaHTable(catalogTracker);
|
||||||
|
ResultScanner scanner = metaTable.getScanner(scan);
|
||||||
|
try {
|
||||||
|
Result data;
|
||||||
|
while((data = scanner.next()) != null) {
|
||||||
|
if (data.isEmpty()) continue;
|
||||||
|
// Break if visit returns false.
|
||||||
|
if (!visitor.visit(data)) break;
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
scanner.close();
|
||||||
|
metaTable.close();
|
||||||
|
}
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Implementations 'visit' a catalog table row.
|
* Implementations 'visit' a catalog table row.
|
||||||
|
@ -651,4 +709,37 @@ public class MetaReader {
|
||||||
*/
|
*/
|
||||||
public boolean visit(final Result r) throws IOException;
|
public 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);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -54,7 +54,6 @@ import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
||||||
import org.apache.hadoop.hbase.catalog.CatalogTracker;
|
import org.apache.hadoop.hbase.catalog.CatalogTracker;
|
||||||
import org.apache.hadoop.hbase.catalog.MetaReader;
|
import org.apache.hadoop.hbase.catalog.MetaReader;
|
||||||
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
|
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
|
||||||
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
|
||||||
import org.apache.hadoop.hbase.ipc.HMasterInterface;
|
import org.apache.hadoop.hbase.ipc.HMasterInterface;
|
||||||
import org.apache.hadoop.hbase.ipc.HRegionInterface;
|
import org.apache.hadoop.hbase.ipc.HRegionInterface;
|
||||||
import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
|
import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
|
||||||
|
@ -138,7 +137,6 @@ public class HBaseAdmin implements Abortable, Closeable {
|
||||||
CatalogTracker ct = null;
|
CatalogTracker ct = null;
|
||||||
try {
|
try {
|
||||||
ct = new CatalogTracker(this.conf);
|
ct = new CatalogTracker(this.conf);
|
||||||
|
|
||||||
ct.start();
|
ct.start();
|
||||||
} catch (InterruptedException e) {
|
} catch (InterruptedException e) {
|
||||||
// Let it out as an IOE for now until we redo all so tolerate IEs
|
// Let it out as an IOE for now until we redo all so tolerate IEs
|
||||||
|
@ -1597,19 +1595,19 @@ public class HBaseAdmin implements Abortable, Closeable {
|
||||||
* get the regions of a given table.
|
* get the regions of a given table.
|
||||||
*
|
*
|
||||||
* @param tableName the name of the table
|
* @param tableName the name of the table
|
||||||
* @return Ordered list of {@link HRegionInfo}. *
|
* @return Ordered list of {@link HRegionInfo}.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public List<HRegionInfo> getTableRegions(final byte[] tableName)
|
public List<HRegionInfo> getTableRegions(final byte[] tableName)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
CatalogTracker ct = getCatalogTracker();
|
CatalogTracker ct = getCatalogTracker();
|
||||||
List<HRegionInfo> Regions;
|
List<HRegionInfo> Regions = null;
|
||||||
try {
|
try {
|
||||||
Regions = MetaReader.getTableRegions(ct, tableName, true);
|
Regions = MetaReader.getTableRegions(ct, tableName, true);
|
||||||
} finally {
|
} finally {
|
||||||
cleanupCatalogTracker(ct);
|
cleanupCatalogTracker(ct);
|
||||||
}
|
}
|
||||||
return Regions;
|
return Regions;
|
||||||
}
|
}
|
||||||
|
|
||||||
public void close() throws IOException {
|
public void close() throws IOException {
|
||||||
|
|
|
@ -130,8 +130,10 @@ import org.apache.zookeeper.KeeperException;
|
||||||
*/
|
*/
|
||||||
@SuppressWarnings("serial")
|
@SuppressWarnings("serial")
|
||||||
public class HConnectionManager {
|
public class HConnectionManager {
|
||||||
// A LRU Map of HConnectionKey -> HConnection (TableServer).
|
// An LRU Map of HConnectionKey -> HConnection (TableServer). All
|
||||||
private static final Map<HConnectionKey, HConnectionImplementation> HBASE_INSTANCES;
|
// access must be synchronized. This map is not private because tests
|
||||||
|
// need to be able to tinker with it.
|
||||||
|
static final Map<HConnectionKey, HConnectionImplementation> HBASE_INSTANCES;
|
||||||
|
|
||||||
public static final int MAX_CACHED_HBASE_INSTANCES;
|
public static final int MAX_CACHED_HBASE_INSTANCES;
|
||||||
|
|
||||||
|
@ -895,7 +897,7 @@ public class HConnectionManager {
|
||||||
deleteCachedLocation(tableName, row);
|
deleteCachedLocation(tableName, row);
|
||||||
}
|
}
|
||||||
|
|
||||||
// Query the root or meta region for the location of the meta region
|
// Query the root or meta region for the location of the meta region
|
||||||
regionInfoRow = server.getClosestRowBefore(
|
regionInfoRow = server.getClosestRowBefore(
|
||||||
metaLocation.getRegionInfo().getRegionName(), metaKey,
|
metaLocation.getRegionInfo().getRegionName(), metaKey,
|
||||||
HConstants.CATALOG_FAMILY);
|
HConstants.CATALOG_FAMILY);
|
||||||
|
@ -962,8 +964,8 @@ public class HConnectionManager {
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
LOG.debug("locateRegionInMeta parentTable=" +
|
LOG.debug("locateRegionInMeta parentTable=" +
|
||||||
Bytes.toString(parentTable) + ", metaLocation=" +
|
Bytes.toString(parentTable) + ", metaLocation=" +
|
||||||
((metaLocation == null)? "null": metaLocation) + ", attempt=" +
|
((metaLocation == null)? "null": "{" + metaLocation + "}") +
|
||||||
tries + " of " +
|
", attempt=" + tries + " of " +
|
||||||
this.numRetries + " failed; retrying after sleep of " +
|
this.numRetries + " failed; retrying after sleep of " +
|
||||||
getPauseTime(tries) + " because: " + e.getMessage());
|
getPauseTime(tries) + " because: " + e.getMessage());
|
||||||
}
|
}
|
||||||
|
@ -1201,7 +1203,7 @@ public class HConnectionManager {
|
||||||
} catch (RemoteException e) {
|
} catch (RemoteException e) {
|
||||||
LOG.warn("RemoteException connecting to RS", e);
|
LOG.warn("RemoteException connecting to RS", e);
|
||||||
// Throw what the RemoteException was carrying.
|
// Throw what the RemoteException was carrying.
|
||||||
throw RemoteExceptionHandler.decodeRemoteException(e);
|
throw e.unwrapRemoteException();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -1233,19 +1235,22 @@ public class HConnectionManager {
|
||||||
|
|
||||||
public <T> T getRegionServerWithRetries(ServerCallable<T> callable)
|
public <T> T getRegionServerWithRetries(ServerCallable<T> callable)
|
||||||
throws IOException, RuntimeException {
|
throws IOException, RuntimeException {
|
||||||
List<Throwable> exceptions = new ArrayList<Throwable>();
|
List<RetriesExhaustedException.ThrowableWithExtraContext> exceptions =
|
||||||
|
new ArrayList<RetriesExhaustedException.ThrowableWithExtraContext>();
|
||||||
for(int tries = 0; tries < numRetries; tries++) {
|
for(int tries = 0; tries < numRetries; tries++) {
|
||||||
try {
|
try {
|
||||||
callable.instantiateServer(tries != 0);
|
|
||||||
callable.beforeCall();
|
callable.beforeCall();
|
||||||
|
callable.connect(tries != 0);
|
||||||
return callable.call();
|
return callable.call();
|
||||||
} catch (Throwable t) {
|
} catch (Throwable t) {
|
||||||
callable.shouldRetry(t);
|
callable.shouldRetry(t);
|
||||||
t = translateException(t);
|
t = translateException(t);
|
||||||
exceptions.add(t);
|
RetriesExhaustedException.ThrowableWithExtraContext qt =
|
||||||
|
new RetriesExhaustedException.ThrowableWithExtraContext(t,
|
||||||
|
System.currentTimeMillis(), callable.toString());
|
||||||
|
exceptions.add(qt);
|
||||||
if (tries == numRetries - 1) {
|
if (tries == numRetries - 1) {
|
||||||
throw new RetriesExhaustedException(callable.getServerName(),
|
throw new RetriesExhaustedException(tries, exceptions);
|
||||||
callable.getRegionName(), callable.getRow(), tries, exceptions);
|
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
callable.afterCall();
|
callable.afterCall();
|
||||||
|
@ -1254,7 +1259,7 @@ public class HConnectionManager {
|
||||||
Thread.sleep(getPauseTime(tries));
|
Thread.sleep(getPauseTime(tries));
|
||||||
} catch (InterruptedException e) {
|
} catch (InterruptedException e) {
|
||||||
Thread.currentThread().interrupt();
|
Thread.currentThread().interrupt();
|
||||||
throw new IOException("Giving up trying to get region server: thread is interrupted.");
|
throw new IOException("Giving up after tries=" + tries, e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return null;
|
return null;
|
||||||
|
@ -1263,8 +1268,8 @@ public class HConnectionManager {
|
||||||
public <T> T getRegionServerWithoutRetries(ServerCallable<T> callable)
|
public <T> T getRegionServerWithoutRetries(ServerCallable<T> callable)
|
||||||
throws IOException, RuntimeException {
|
throws IOException, RuntimeException {
|
||||||
try {
|
try {
|
||||||
callable.instantiateServer(false);
|
|
||||||
callable.beforeCall();
|
callable.beforeCall();
|
||||||
|
callable.connect(false);
|
||||||
return callable.call();
|
return callable.call();
|
||||||
} catch (Throwable t) {
|
} catch (Throwable t) {
|
||||||
Throwable t2 = translateException(t);
|
Throwable t2 = translateException(t);
|
||||||
|
@ -1289,7 +1294,7 @@ public class HConnectionManager {
|
||||||
return server.multi(multi);
|
return server.multi(multi);
|
||||||
}
|
}
|
||||||
@Override
|
@Override
|
||||||
public void instantiateServer(boolean reload) throws IOException {
|
public void connect(boolean reload) throws IOException {
|
||||||
server =
|
server =
|
||||||
connection.getHRegionConnection(loc.getHostname(), loc.getPort());
|
connection.getHRegionConnection(loc.getHostname(), loc.getPort());
|
||||||
}
|
}
|
||||||
|
@ -1804,6 +1809,25 @@ public class HConnectionManager {
|
||||||
}
|
}
|
||||||
return hTableDescriptor;
|
return hTableDescriptor;
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Set the number of retries to use serverside when trying to communicate
|
||||||
|
* with another server over {@link HConnection}. Used updating catalog
|
||||||
|
* tables, etc. Call this method before we create any Connections.
|
||||||
|
* @param c The Configuration instance to set the retries into.
|
||||||
|
* @param log Used to log what we set in here.
|
||||||
|
*/
|
||||||
|
public static void setServerSideHConnectionRetries(final Configuration c,
|
||||||
|
final Log log) {
|
||||||
|
int hcRetries = c.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
|
||||||
|
HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
|
||||||
|
// Go big. Multiply by 10. If we can't get to meta after this many retries
|
||||||
|
// then something seriously wrong.
|
||||||
|
int serversideMultiplier =
|
||||||
|
c.getInt("hbase.client.serverside.retries.multiplier", 10);
|
||||||
|
int retries = hcRetries * serversideMultiplier;
|
||||||
|
c.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, retries);
|
||||||
|
log.debug("Set serverside HConnection retries=" + retries);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -53,7 +53,6 @@ import org.apache.hadoop.hbase.KeyValue;
|
||||||
import org.apache.hadoop.hbase.NotServingRegionException;
|
import org.apache.hadoop.hbase.NotServingRegionException;
|
||||||
import org.apache.hadoop.hbase.ServerName;
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
import org.apache.hadoop.hbase.UnknownScannerException;
|
import org.apache.hadoop.hbase.UnknownScannerException;
|
||||||
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
|
||||||
import org.apache.hadoop.hbase.client.HConnectionManager.HConnectable;
|
import org.apache.hadoop.hbase.client.HConnectionManager.HConnectable;
|
||||||
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
|
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
|
||||||
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
|
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
|
||||||
|
@ -194,7 +193,8 @@ public class HTable implements HTableInterface, Closeable {
|
||||||
}
|
}
|
||||||
this.connection = HConnectionManager.getConnection(conf);
|
this.connection = HConnectionManager.getConnection(conf);
|
||||||
this.scannerTimeout =
|
this.scannerTimeout =
|
||||||
(int) conf.getLong(HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY, HConstants.DEFAULT_HBASE_REGIONSERVER_LEASE_PERIOD);
|
(int) conf.getLong(HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY,
|
||||||
|
HConstants.DEFAULT_HBASE_REGIONSERVER_LEASE_PERIOD);
|
||||||
this.operationTimeout = HTableDescriptor.isMetaTable(tableName) ? HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT
|
this.operationTimeout = HTableDescriptor.isMetaTable(tableName) ? HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT
|
||||||
: conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
|
: conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
|
||||||
HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
|
HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
|
||||||
|
@ -418,8 +418,9 @@ public class HTable implements HTableInterface, Closeable {
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
MetaScanner.metaScan(configuration, visitor, this.tableName);
|
MetaScanner.metaScan(configuration, visitor, this.tableName);
|
||||||
return new Pair(startKeyList.toArray(new byte[startKeyList.size()][]),
|
return new Pair<byte [][], byte [][]>(
|
||||||
endKeyList.toArray(new byte[endKeyList.size()][]));
|
startKeyList.toArray(new byte[startKeyList.size()][]),
|
||||||
|
endKeyList.toArray(new byte[endKeyList.size()][]));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -872,7 +873,7 @@ public class HTable implements HTableInterface, Closeable {
|
||||||
@Override
|
@Override
|
||||||
public void flushCommits() throws IOException {
|
public void flushCommits() throws IOException {
|
||||||
try {
|
try {
|
||||||
connection.processBatchOfPuts(writeBuffer, tableName, pool);
|
this.connection.processBatchOfPuts(writeBuffer, tableName, pool);
|
||||||
} finally {
|
} finally {
|
||||||
if (clearBufferOnFail) {
|
if (clearBufferOnFail) {
|
||||||
writeBuffer.clear();
|
writeBuffer.clear();
|
||||||
|
|
|
@ -187,7 +187,7 @@ public class MetaScanner {
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
LOG.debug("Scanning " + Bytes.toString(metaTableName) +
|
LOG.debug("Scanning " + Bytes.toString(metaTableName) +
|
||||||
" starting at row=" + Bytes.toStringBinary(startRow) + " for max=" +
|
" starting at row=" + Bytes.toStringBinary(startRow) + " for max=" +
|
||||||
rowUpperLimit + " rows");
|
rowUpperLimit + " rows using " + connection.toString());
|
||||||
}
|
}
|
||||||
callable = new ScannerCallable(connection, metaTableName, scan, null);
|
callable = new ScannerCallable(connection, metaTableName, scan, null);
|
||||||
// Open scanner
|
// Open scanner
|
||||||
|
|
|
@ -20,12 +20,16 @@
|
||||||
|
|
||||||
package org.apache.hadoop.hbase.client;
|
package org.apache.hadoop.hbase.client;
|
||||||
|
|
||||||
import com.google.common.collect.Ordering;
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
|
import org.apache.hadoop.hbase.HRegionInfo;
|
||||||
import org.apache.hadoop.hbase.KeyValue;
|
import org.apache.hadoop.hbase.KeyValue;
|
||||||
import org.apache.hadoop.hbase.KeyValue.SplitKeyValue;
|
import org.apache.hadoop.hbase.KeyValue.SplitKeyValue;
|
||||||
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
||||||
import org.apache.hadoop.hbase.io.WritableWithSize;
|
import org.apache.hadoop.hbase.io.WritableWithSize;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
import org.apache.hadoop.hbase.util.Pair;
|
||||||
|
import org.apache.hadoop.hbase.util.Writables;
|
||||||
import org.apache.hadoop.io.Writable;
|
import org.apache.hadoop.io.Writable;
|
||||||
|
|
||||||
import java.io.DataInput;
|
import java.io.DataInput;
|
||||||
|
|
|
@ -15,9 +15,8 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.client;
|
package org.apache.hadoop.hbase.client;
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.Date;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -35,29 +34,54 @@ public class RetriesExhaustedException extends IOException {
|
||||||
super(msg, e);
|
super(msg, e);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Datastructure that allows adding more info around Throwable incident.
|
||||||
|
*/
|
||||||
|
public static class ThrowableWithExtraContext {
|
||||||
|
private final Throwable t;
|
||||||
|
private final long when;
|
||||||
|
private final String extras;
|
||||||
|
|
||||||
|
public ThrowableWithExtraContext(final Throwable t, final long when,
|
||||||
|
final String extras) {
|
||||||
|
this.t = t;
|
||||||
|
this.when = when;
|
||||||
|
this.extras = extras;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return new Date(this.when).toString() + ", " + extras + ", " + t.toString();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create a new RetriesExhaustedException from the list of prior failures.
|
* Create a new RetriesExhaustedException from the list of prior failures.
|
||||||
* @param serverName name of HRegionServer
|
* @param callableVitals Details from the {@link ServerCallable} we were using
|
||||||
* @param regionName name of region
|
* when we got this exception.
|
||||||
* @param row The row we were pursuing when we ran out of retries
|
|
||||||
* @param numTries The number of tries we made
|
* @param numTries The number of tries we made
|
||||||
* @param exceptions List of exceptions that failed before giving up
|
* @param exceptions List of exceptions that failed before giving up
|
||||||
*/
|
*/
|
||||||
public RetriesExhaustedException(String serverName, final byte [] regionName,
|
public RetriesExhaustedException(final String callableVitals, int numTries,
|
||||||
final byte [] row, int numTries, List<Throwable> exceptions) {
|
List<Throwable> exceptions) {
|
||||||
super(getMessage(serverName, regionName, row, numTries, exceptions));
|
super(getMessage(callableVitals, numTries, exceptions));
|
||||||
}
|
}
|
||||||
|
|
||||||
private static String getMessage(String serverName, final byte [] regionName,
|
/**
|
||||||
final byte [] row,
|
* Create a new RetriesExhaustedException from the list of prior failures.
|
||||||
int numTries, List<Throwable> exceptions) {
|
* @param numTries
|
||||||
StringBuilder buffer = new StringBuilder("Trying to contact region server ");
|
* @param exceptions List of exceptions that failed before giving up
|
||||||
buffer.append(serverName);
|
*/
|
||||||
buffer.append(" for region ");
|
public RetriesExhaustedException(final int numTries,
|
||||||
buffer.append(regionName == null? "": Bytes.toStringBinary(regionName));
|
final List<ThrowableWithExtraContext> exceptions) {
|
||||||
buffer.append(", row '");
|
super(getMessage(numTries, exceptions));
|
||||||
buffer.append(row == null? "": Bytes.toStringBinary(row));
|
}
|
||||||
buffer.append("', but failed after ");
|
|
||||||
|
private static String getMessage(String callableVitals, int numTries,
|
||||||
|
List<Throwable> exceptions) {
|
||||||
|
StringBuilder buffer = new StringBuilder("Failed contacting ");
|
||||||
|
buffer.append(callableVitals);
|
||||||
|
buffer.append(" after ");
|
||||||
buffer.append(numTries + 1);
|
buffer.append(numTries + 1);
|
||||||
buffer.append(" attempts.\nExceptions:\n");
|
buffer.append(" attempts.\nExceptions:\n");
|
||||||
for (Throwable t : exceptions) {
|
for (Throwable t : exceptions) {
|
||||||
|
@ -66,4 +90,16 @@ public class RetriesExhaustedException extends IOException {
|
||||||
}
|
}
|
||||||
return buffer.toString();
|
return buffer.toString();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static String getMessage(final int numTries,
|
||||||
|
final List<ThrowableWithExtraContext> exceptions) {
|
||||||
|
StringBuilder buffer = new StringBuilder("Failed after attempts=");
|
||||||
|
buffer.append(numTries + 1);
|
||||||
|
buffer.append(", exceptions:\n");
|
||||||
|
for (ThrowableWithExtraContext t : exceptions) {
|
||||||
|
buffer.append(t.toString());
|
||||||
|
buffer.append("\n");
|
||||||
|
}
|
||||||
|
return buffer.toString();
|
||||||
|
}
|
||||||
}
|
}
|
|
@ -67,9 +67,9 @@ public class ScannerCallable extends ServerCallable<Result[]> {
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public void instantiateServer(boolean reload) throws IOException {
|
public void connect(boolean reload) throws IOException {
|
||||||
if (!instantiated || reload) {
|
if (!instantiated || reload) {
|
||||||
super.instantiateServer(reload);
|
super.connect(reload);
|
||||||
checkIfRegionServerIsRemote();
|
checkIfRegionServerIsRemote();
|
||||||
instantiated = true;
|
instantiated = true;
|
||||||
}
|
}
|
||||||
|
|
|
@ -31,7 +31,14 @@ import org.apache.hadoop.hbase.ipc.HRegionInterface;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Abstract class that implements Callable, used by retryable actions.
|
* Abstract class that implements {@link Callable}. Implementation stipulates
|
||||||
|
* return type and method we actually invoke on remote Server. Usually
|
||||||
|
* used inside a try/catch that fields usual connection failures all wrapped
|
||||||
|
* up in a retry loop.
|
||||||
|
* <p>Call {@link #connect(boolean)} to connect to server hosting region
|
||||||
|
* that contains the passed row in the passed table before invoking
|
||||||
|
* {@link #call()}.
|
||||||
|
* @see HConnection#getRegionServerWithoutRetries(ServerCallable)
|
||||||
* @param <T> the class that the ServerCallable handles
|
* @param <T> the class that the ServerCallable handles
|
||||||
*/
|
*/
|
||||||
public abstract class ServerCallable<T> implements Callable<T> {
|
public abstract class ServerCallable<T> implements Callable<T> {
|
||||||
|
@ -44,9 +51,9 @@ public abstract class ServerCallable<T> implements Callable<T> {
|
||||||
protected long startTime, endTime;
|
protected long startTime, endTime;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @param connection connection callable is on
|
* @param connection Connection to use.
|
||||||
* @param tableName table name callable is on
|
* @param tableName Table name to which <code>row</code> belongs.
|
||||||
* @param row row we are querying
|
* @param row The row we want in <code>tableName</code>.
|
||||||
*/
|
*/
|
||||||
public ServerCallable(HConnection connection, byte [] tableName, byte [] row) {
|
public ServerCallable(HConnection connection, byte [] tableName, byte [] row) {
|
||||||
this(connection, tableName, row, HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
|
this(connection, tableName, row, HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
|
||||||
|
@ -58,34 +65,37 @@ public abstract class ServerCallable<T> implements Callable<T> {
|
||||||
this.row = row;
|
this.row = row;
|
||||||
this.callTimeout = callTimeout;
|
this.callTimeout = callTimeout;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*
|
* Connect to the server hosting region with row from tablename.
|
||||||
* @param reload set this to true if connection should re-find the region
|
* @param reload Set this to true if connection should re-find the region
|
||||||
* @throws IOException e
|
* @throws IOException e
|
||||||
*/
|
*/
|
||||||
public void instantiateServer(boolean reload) throws IOException {
|
public void connect(final boolean reload) throws IOException {
|
||||||
this.location = connection.getRegionLocation(tableName, row, reload);
|
this.location = connection.getRegionLocation(tableName, row, reload);
|
||||||
this.server = connection.getHRegionConnection(location.getHostname(),
|
this.server = connection.getHRegionConnection(location.getHostname(),
|
||||||
location.getPort());
|
location.getPort());
|
||||||
}
|
}
|
||||||
|
|
||||||
/** @return the server name */
|
/** @return the server name
|
||||||
|
* @deprecated Just use {@link #toString()} instead.
|
||||||
|
*/
|
||||||
public String getServerName() {
|
public String getServerName() {
|
||||||
if (location == null) {
|
if (location == null) return null;
|
||||||
return null;
|
|
||||||
}
|
|
||||||
return location.getHostnamePort();
|
return location.getHostnamePort();
|
||||||
}
|
}
|
||||||
|
|
||||||
/** @return the region name */
|
/** @return the region name
|
||||||
|
* @deprecated Just use {@link #toString()} instead.
|
||||||
|
*/
|
||||||
public byte[] getRegionName() {
|
public byte[] getRegionName() {
|
||||||
if (location == null) {
|
if (location == null) return null;
|
||||||
return null;
|
|
||||||
}
|
|
||||||
return location.getRegionInfo().getRegionName();
|
return location.getRegionInfo().getRegionName();
|
||||||
}
|
}
|
||||||
|
|
||||||
/** @return the row */
|
/** @return the row
|
||||||
|
* @deprecated Just use {@link #toString()} instead.
|
||||||
|
*/
|
||||||
public byte [] getRow() {
|
public byte [] getRow() {
|
||||||
return row;
|
return row;
|
||||||
}
|
}
|
||||||
|
|
|
@ -87,7 +87,7 @@ public interface HMasterInterface extends VersionedProtocol {
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public Pair<Integer, Integer> getAlterStatus(byte[] tableName)
|
public Pair<Integer, Integer> getAlterStatus(byte[] tableName)
|
||||||
throws IOException;
|
throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Adds a column to the specified table
|
* Adds a column to the specified table
|
||||||
|
|
|
@ -267,11 +267,12 @@ public class AssignmentManager extends ZooKeeperListener {
|
||||||
* @param tableName
|
* @param tableName
|
||||||
* @return Pair indicating the status of the alter command
|
* @return Pair indicating the status of the alter command
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
|
* @throws InterruptedException
|
||||||
*/
|
*/
|
||||||
public Pair<Integer, Integer> getReopenStatus(byte[] tableName)
|
public Pair<Integer, Integer> getReopenStatus(byte[] tableName)
|
||||||
throws IOException {
|
throws IOException, InterruptedException {
|
||||||
List <HRegionInfo> hris = MetaReader.getTableRegions(
|
List <HRegionInfo> hris =
|
||||||
this.master.getCatalogTracker(), tableName);
|
MetaReader.getTableRegions(this.master.getCatalogTracker(), tableName);
|
||||||
Integer pending = 0;
|
Integer pending = 0;
|
||||||
for(HRegionInfo hri : hris) {
|
for(HRegionInfo hri : hris) {
|
||||||
if(regionsToReopen.get(hri.getEncodedName()) != null) {
|
if(regionsToReopen.get(hri.getEncodedName()) != null) {
|
||||||
|
@ -730,7 +731,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
||||||
case RS_ZK_REGION_OPENING:
|
case RS_ZK_REGION_OPENING:
|
||||||
// Should see OPENING after we have asked it to OPEN or additional
|
// Should see OPENING after we have asked it to OPEN or additional
|
||||||
// times after already being in state of OPENING
|
// times after already being in state of OPENING
|
||||||
if(regionState == null ||
|
if (regionState == null ||
|
||||||
(!regionState.isPendingOpen() && !regionState.isOpening())) {
|
(!regionState.isPendingOpen() && !regionState.isOpening())) {
|
||||||
LOG.warn("Received OPENING for region " +
|
LOG.warn("Received OPENING for region " +
|
||||||
prettyPrintedRegionName +
|
prettyPrintedRegionName +
|
||||||
|
@ -1764,7 +1765,6 @@ public class AssignmentManager extends ZooKeeperListener {
|
||||||
// Presume that master has stale data. Presume remote side just split.
|
// Presume that master has stale data. Presume remote side just split.
|
||||||
// Presume that the split message when it comes in will fix up the master's
|
// Presume that the split message when it comes in will fix up the master's
|
||||||
// in memory cluster state.
|
// in memory cluster state.
|
||||||
return;
|
|
||||||
} catch (Throwable t) {
|
} catch (Throwable t) {
|
||||||
if (t instanceof RemoteException) {
|
if (t instanceof RemoteException) {
|
||||||
t = ((RemoteException)t).unwrapRemoteException();
|
t = ((RemoteException)t).unwrapRemoteException();
|
||||||
|
@ -2082,13 +2082,13 @@ public class AssignmentManager extends ZooKeeperListener {
|
||||||
Map<ServerName, List<Pair<HRegionInfo, Result>>> rebuildUserRegions()
|
Map<ServerName, List<Pair<HRegionInfo, Result>>> rebuildUserRegions()
|
||||||
throws IOException, KeeperException {
|
throws IOException, KeeperException {
|
||||||
// Region assignment from META
|
// Region assignment from META
|
||||||
List<Result> results = MetaReader.fullScanOfResults(this.catalogTracker);
|
List<Result> results = MetaReader.fullScan(this.catalogTracker);
|
||||||
// Map of offline servers and their regions to be returned
|
// Map of offline servers and their regions to be returned
|
||||||
Map<ServerName, List<Pair<HRegionInfo,Result>>> offlineServers =
|
Map<ServerName, List<Pair<HRegionInfo,Result>>> offlineServers =
|
||||||
new TreeMap<ServerName, List<Pair<HRegionInfo, Result>>>();
|
new TreeMap<ServerName, List<Pair<HRegionInfo, Result>>>();
|
||||||
// Iterate regions in META
|
// Iterate regions in META
|
||||||
for (Result result : results) {
|
for (Result result : results) {
|
||||||
Pair<HRegionInfo, ServerName> region = MetaReader.metaRowToRegionPair(result);
|
Pair<HRegionInfo, ServerName> region = MetaReader.parseCatalogResult(result);
|
||||||
if (region == null) continue;
|
if (region == null) continue;
|
||||||
HRegionInfo regionInfo = region.getFirst();
|
HRegionInfo regionInfo = region.getFirst();
|
||||||
ServerName regionLocation = region.getSecond();
|
ServerName regionLocation = region.getSecond();
|
||||||
|
|
|
@ -25,7 +25,6 @@ import java.lang.reflect.InvocationTargetException;
|
||||||
import java.net.InetAddress;
|
import java.net.InetAddress;
|
||||||
import java.net.InetSocketAddress;
|
import java.net.InetSocketAddress;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.atomic.AtomicReference;
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
|
@ -41,21 +40,18 @@ import org.apache.hadoop.hbase.HRegionInfo;
|
||||||
import org.apache.hadoop.hbase.HServerLoad;
|
import org.apache.hadoop.hbase.HServerLoad;
|
||||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
import org.apache.hadoop.hbase.MasterNotRunningException;
|
import org.apache.hadoop.hbase.MasterNotRunningException;
|
||||||
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
|
|
||||||
import org.apache.hadoop.hbase.Server;
|
import org.apache.hadoop.hbase.Server;
|
||||||
import org.apache.hadoop.hbase.ServerName;
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
import org.apache.hadoop.hbase.TableDescriptors;
|
import org.apache.hadoop.hbase.TableDescriptors;
|
||||||
import org.apache.hadoop.hbase.TableExistsException;
|
|
||||||
import org.apache.hadoop.hbase.TableNotDisabledException;
|
import org.apache.hadoop.hbase.TableNotDisabledException;
|
||||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||||
import org.apache.hadoop.hbase.UnknownRegionException;
|
import org.apache.hadoop.hbase.UnknownRegionException;
|
||||||
import org.apache.hadoop.hbase.catalog.CatalogTracker;
|
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.catalog.MetaReader;
|
||||||
import org.apache.hadoop.hbase.client.Get;
|
import org.apache.hadoop.hbase.client.HConnectionManager;
|
||||||
import org.apache.hadoop.hbase.client.MetaScanner;
|
import org.apache.hadoop.hbase.client.MetaScanner;
|
||||||
import org.apache.hadoop.hbase.client.Result;
|
|
||||||
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
|
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
|
||||||
|
import org.apache.hadoop.hbase.client.Result;
|
||||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
||||||
import org.apache.hadoop.hbase.executor.ExecutorService;
|
import org.apache.hadoop.hbase.executor.ExecutorService;
|
||||||
import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorType;
|
import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorType;
|
||||||
|
@ -64,7 +60,7 @@ import org.apache.hadoop.hbase.ipc.HBaseServer;
|
||||||
import org.apache.hadoop.hbase.ipc.HMasterInterface;
|
import org.apache.hadoop.hbase.ipc.HMasterInterface;
|
||||||
import org.apache.hadoop.hbase.ipc.HMasterRegionInterface;
|
import org.apache.hadoop.hbase.ipc.HMasterRegionInterface;
|
||||||
import org.apache.hadoop.hbase.ipc.RpcServer;
|
import org.apache.hadoop.hbase.ipc.RpcServer;
|
||||||
import org.apache.hadoop.hbase.master.RegionPlan;
|
import org.apache.hadoop.hbase.master.handler.CreateTableHandler;
|
||||||
import org.apache.hadoop.hbase.master.handler.DeleteTableHandler;
|
import org.apache.hadoop.hbase.master.handler.DeleteTableHandler;
|
||||||
import org.apache.hadoop.hbase.master.handler.DisableTableHandler;
|
import org.apache.hadoop.hbase.master.handler.DisableTableHandler;
|
||||||
import org.apache.hadoop.hbase.master.handler.EnableTableHandler;
|
import org.apache.hadoop.hbase.master.handler.EnableTableHandler;
|
||||||
|
@ -72,18 +68,14 @@ import org.apache.hadoop.hbase.master.handler.ModifyTableHandler;
|
||||||
import org.apache.hadoop.hbase.master.handler.TableAddFamilyHandler;
|
import org.apache.hadoop.hbase.master.handler.TableAddFamilyHandler;
|
||||||
import org.apache.hadoop.hbase.master.handler.TableDeleteFamilyHandler;
|
import org.apache.hadoop.hbase.master.handler.TableDeleteFamilyHandler;
|
||||||
import org.apache.hadoop.hbase.master.handler.TableModifyFamilyHandler;
|
import org.apache.hadoop.hbase.master.handler.TableModifyFamilyHandler;
|
||||||
import org.apache.hadoop.hbase.master.handler.CreateTableHandler;
|
|
||||||
import org.apache.hadoop.hbase.master.metrics.MasterMetrics;
|
import org.apache.hadoop.hbase.master.metrics.MasterMetrics;
|
||||||
import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
|
import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
|
||||||
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
||||||
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
|
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
|
||||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
|
||||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
|
||||||
import org.apache.hadoop.hbase.replication.regionserver.Replication;
|
import org.apache.hadoop.hbase.replication.regionserver.Replication;
|
||||||
import org.apache.hadoop.hbase.security.User;
|
import org.apache.hadoop.hbase.security.User;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.util.FSTableDescriptors;
|
import org.apache.hadoop.hbase.util.FSTableDescriptors;
|
||||||
import org.apache.hadoop.hbase.util.FSUtils;
|
|
||||||
import org.apache.hadoop.hbase.util.InfoServer;
|
import org.apache.hadoop.hbase.util.InfoServer;
|
||||||
import org.apache.hadoop.hbase.util.Pair;
|
import org.apache.hadoop.hbase.util.Pair;
|
||||||
import org.apache.hadoop.hbase.util.Sleeper;
|
import org.apache.hadoop.hbase.util.Sleeper;
|
||||||
|
@ -205,7 +197,8 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
|
||||||
public HMaster(final Configuration conf)
|
public HMaster(final Configuration conf)
|
||||||
throws IOException, KeeperException, InterruptedException {
|
throws IOException, KeeperException, InterruptedException {
|
||||||
this.conf = conf;
|
this.conf = conf;
|
||||||
|
// Set how many times to retry talking to another server over HConnection.
|
||||||
|
HConnectionManager.setServerSideHConnectionRetries(this.conf, LOG);
|
||||||
// Server to handle client requests.
|
// Server to handle client requests.
|
||||||
String hostname = DNS.getDefaultHost(
|
String hostname = DNS.getDefaultHost(
|
||||||
conf.get("hbase.master.dns.interface", "default"),
|
conf.get("hbase.master.dns.interface", "default"),
|
||||||
|
@ -470,7 +463,9 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
|
||||||
// Update meta with new HRI if required. i.e migrate all HRI with HTD to
|
// Update meta with new HRI if required. i.e migrate all HRI with HTD to
|
||||||
// HRI with out HTD in meta and update the status in ROOT. This must happen
|
// HRI with out HTD in meta and update the status in ROOT. This must happen
|
||||||
// before we assign all user regions or else the assignment will fail.
|
// before we assign all user regions or else the assignment will fail.
|
||||||
updateMetaWithNewHRI();
|
// TODO: Remove this when we do 0.94.
|
||||||
|
org.apache.hadoop.hbase.catalog.MetaMigrationRemovingHTD.
|
||||||
|
updateMetaWithNewHRI(this);
|
||||||
|
|
||||||
// Fixup assignment manager status
|
// Fixup assignment manager status
|
||||||
status.setStatus("Starting assignment manager");
|
status.setStatus("Starting assignment manager");
|
||||||
|
@ -500,45 +495,6 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean isMetaHRIUpdated()
|
|
||||||
throws IOException {
|
|
||||||
boolean metaUpdated = false;
|
|
||||||
Get get = new Get(HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
|
|
||||||
get.addColumn(HConstants.CATALOG_FAMILY,
|
|
||||||
HConstants.META_MIGRATION_QUALIFIER);
|
|
||||||
Result r =
|
|
||||||
catalogTracker.waitForRootServerConnectionDefault().get(
|
|
||||||
HRegionInfo.ROOT_REGIONINFO.getRegionName(), get);
|
|
||||||
if (r != null && r.getBytes() != null)
|
|
||||||
{
|
|
||||||
byte[] metaMigrated = r.getValue(HConstants.CATALOG_FAMILY,
|
|
||||||
HConstants.META_MIGRATION_QUALIFIER);
|
|
||||||
String migrated = Bytes.toString(metaMigrated);
|
|
||||||
metaUpdated = new Boolean(migrated).booleanValue();
|
|
||||||
} else {
|
|
||||||
LOG.info("metaUpdated = NULL.");
|
|
||||||
}
|
|
||||||
LOG.info("Meta updated status = " + metaUpdated);
|
|
||||||
return metaUpdated;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
boolean updateMetaWithNewHRI() throws IOException {
|
|
||||||
if (!isMetaHRIUpdated()) {
|
|
||||||
LOG.info("Meta has HRI with HTDs. Updating meta now.");
|
|
||||||
try {
|
|
||||||
MetaEditor.migrateRootAndMeta(this);
|
|
||||||
LOG.info("ROOT and Meta updated with new HRI.");
|
|
||||||
return true;
|
|
||||||
} catch (IOException e) {
|
|
||||||
throw new RuntimeException("Update ROOT/Meta with new HRI failed." +
|
|
||||||
"Master startup aborted.");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
LOG.info("ROOT/Meta already up-to date with new HRI.");
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Check <code>-ROOT-</code> and <code>.META.</code> are assigned. If not,
|
* Check <code>-ROOT-</code> and <code>.META.</code> are assigned. If not,
|
||||||
* assign them.
|
* assign them.
|
||||||
|
@ -1018,6 +974,7 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
|
||||||
Bytes.equals(tableName, HConstants.META_TABLE_NAME);
|
Bytes.equals(tableName, HConstants.META_TABLE_NAME);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public void deleteTable(final byte [] tableName) throws IOException {
|
public void deleteTable(final byte [] tableName) throws IOException {
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
cpHost.preDeleteTable(tableName);
|
cpHost.preDeleteTable(tableName);
|
||||||
|
@ -1029,16 +986,13 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Get the number of regions of the table that have been updated by the alter.
|
|
||||||
*
|
|
||||||
* @return Pair indicating the number of regions updated Pair.getFirst is the
|
|
||||||
* regions that are yet to be updated Pair.getSecond is the total number
|
|
||||||
* of regions of the table
|
|
||||||
*/
|
|
||||||
public Pair<Integer, Integer> getAlterStatus(byte[] tableName)
|
public Pair<Integer, Integer> getAlterStatus(byte[] tableName)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
return this.assignmentManager.getReopenStatus(tableName);
|
try {
|
||||||
|
return this.assignmentManager.getReopenStatus(tableName);
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
throw new IOException("Interrupted", e);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public void addColumn(byte [] tableName, HColumnDescriptor column)
|
public void addColumn(byte [] tableName, HColumnDescriptor column)
|
||||||
|
@ -1123,7 +1077,7 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
|
||||||
if (data == null || data.size() <= 0) {
|
if (data == null || data.size() <= 0) {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
Pair<HRegionInfo, ServerName> pair = MetaReader.metaRowToRegionPair(data);
|
Pair<HRegionInfo, ServerName> pair = MetaReader.parseCatalogResult(data);
|
||||||
if (pair == null) {
|
if (pair == null) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
|
@ -111,8 +111,8 @@ public class EnableTableHandler extends EventHandler {
|
||||||
boolean done = false;
|
boolean done = false;
|
||||||
// Get the regions of this table. We're done when all listed
|
// Get the regions of this table. We're done when all listed
|
||||||
// tables are onlined.
|
// tables are onlined.
|
||||||
List<HRegionInfo> regionsInMeta =
|
List<HRegionInfo> regionsInMeta;
|
||||||
MetaReader.getTableRegions(this.ct, tableName, true);
|
regionsInMeta = MetaReader.getTableRegions(this.ct, tableName, true);
|
||||||
int countOfRegionsInTable = regionsInMeta.size();
|
int countOfRegionsInTable = regionsInMeta.size();
|
||||||
List<HRegionInfo> regions = regionsToAssign(regionsInMeta);
|
List<HRegionInfo> regions = regionsToAssign(regionsInMeta);
|
||||||
int regionsCount = regions.size();
|
int regionsCount = regions.size();
|
||||||
|
|
|
@ -41,7 +41,6 @@ import org.apache.hadoop.hbase.master.DeadServer;
|
||||||
import org.apache.hadoop.hbase.master.MasterServices;
|
import org.apache.hadoop.hbase.master.MasterServices;
|
||||||
import org.apache.hadoop.hbase.master.ServerManager;
|
import org.apache.hadoop.hbase.master.ServerManager;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.util.Writables;
|
|
||||||
import org.apache.zookeeper.KeeperException;
|
import org.apache.zookeeper.KeeperException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -265,14 +264,16 @@ public class ServerShutdownHandler extends EventHandler {
|
||||||
LOG.debug("Removed " + rit.getRegion().getRegionNameAsString() +
|
LOG.debug("Removed " + rit.getRegion().getRegionNameAsString() +
|
||||||
" from list of regions to assign because in RIT" + " region state: "
|
" from list of regions to assign because in RIT" + " region state: "
|
||||||
+ rit.getState());
|
+ rit.getState());
|
||||||
hris.remove(rit.getRegion());
|
if (hris != null) hris.remove(rit.getRegion());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
LOG.info("Reassigning " + (hris == null? 0: hris.size()) +
|
assert regionsInTransition != null;
|
||||||
" region(s) that " + serverName +
|
LOG.info("Reassigning " + ((hris == null)? 0: hris.size()) +
|
||||||
" was carrying (skipping " + regionsInTransition.size() +
|
" region(s) that " + (serverName == null? "null": serverName) +
|
||||||
" regions(s) that are already in transition)");
|
" was carrying (skipping " +
|
||||||
|
regionsInTransition.size() +
|
||||||
|
" regions(s) that are already in transition)");
|
||||||
|
|
||||||
// Iterate regions that were on this server and assign them
|
// Iterate regions that were on this server and assign them
|
||||||
if (hris != null) {
|
if (hris != null) {
|
||||||
|
@ -342,7 +343,8 @@ public class ServerShutdownHandler extends EventHandler {
|
||||||
final AssignmentManager assignmentManager,
|
final AssignmentManager assignmentManager,
|
||||||
final CatalogTracker catalogTracker)
|
final CatalogTracker catalogTracker)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
HRegionInfo daughter = getHRegionInfo(result, qualifier);
|
HRegionInfo daughter =
|
||||||
|
MetaReader.parseHRegionInfoFromCatalogResult(result, qualifier);
|
||||||
if (daughter == null) return;
|
if (daughter == null) return;
|
||||||
if (isDaughterMissing(catalogTracker, daughter)) {
|
if (isDaughterMissing(catalogTracker, daughter)) {
|
||||||
LOG.info("Fixup; missing daughter " + daughter.getRegionNameAsString());
|
LOG.info("Fixup; missing daughter " + daughter.getRegionNameAsString());
|
||||||
|
@ -359,21 +361,6 @@ public class ServerShutdownHandler extends EventHandler {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Interpret the content of the cell at {@link HConstants#CATALOG_FAMILY} and
|
|
||||||
* <code>qualifier</code> as an HRegionInfo and return it, or null.
|
|
||||||
* @param r Result instance to pull from.
|
|
||||||
* @param qualifier Column family qualifier
|
|
||||||
* @return An HRegionInfo instance or null.
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
private static HRegionInfo getHRegionInfo(final Result r, byte [] qualifier)
|
|
||||||
throws IOException {
|
|
||||||
byte [] bytes = r.getValue(HConstants.CATALOG_FAMILY, qualifier);
|
|
||||||
if (bytes == null || bytes.length <= 0) return null;
|
|
||||||
return Writables.getHRegionInfoOrNull(bytes);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Look for presence of the daughter OR of a split of the daughter in .META.
|
* Look for presence of the daughter OR of a split of the daughter in .META.
|
||||||
* Daughter could have been split over on regionserver before a run of the
|
* Daughter could have been split over on regionserver before a run of the
|
||||||
|
@ -416,7 +403,8 @@ public class ServerShutdownHandler extends EventHandler {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean visit(Result r) throws IOException {
|
public boolean visit(Result r) throws IOException {
|
||||||
HRegionInfo hri = getHRegionInfo(r, HConstants.REGIONINFO_QUALIFIER);
|
HRegionInfo hri =
|
||||||
|
MetaReader.parseHRegionInfoFromCatalogResult(r, HConstants.REGIONINFO_QUALIFIER);
|
||||||
if (hri == null) {
|
if (hri == null) {
|
||||||
LOG.warn("No serialized HRegionInfo in " + r);
|
LOG.warn("No serialized HRegionInfo in " + r);
|
||||||
return true;
|
return true;
|
||||||
|
|
|
@ -3061,9 +3061,9 @@ public class HRegion implements HeapSize { // , Writable{
|
||||||
EnvironmentEdgeManager.currentTimeMillis(),
|
EnvironmentEdgeManager.currentTimeMillis(),
|
||||||
Writables.getBytes(r.getRegionInfo())));
|
Writables.getBytes(r.getRegionInfo())));
|
||||||
edits.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
|
edits.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
|
||||||
HConstants.META_MIGRATION_QUALIFIER,
|
org.apache.hadoop.hbase.catalog.MetaMigrationRemovingHTD.META_MIGRATION_QUALIFIER,
|
||||||
EnvironmentEdgeManager.currentTimeMillis(),
|
EnvironmentEdgeManager.currentTimeMillis(),
|
||||||
Bytes.toBytes("true")));
|
Bytes.toBytes(true)));
|
||||||
|
|
||||||
meta.put(HConstants.CATALOG_FAMILY, edits);
|
meta.put(HConstants.CATALOG_FAMILY, edits);
|
||||||
} finally {
|
} finally {
|
||||||
|
|
|
@ -83,6 +83,7 @@ import org.apache.hadoop.hbase.client.Action;
|
||||||
import org.apache.hadoop.hbase.client.Append;
|
import org.apache.hadoop.hbase.client.Append;
|
||||||
import org.apache.hadoop.hbase.client.Delete;
|
import org.apache.hadoop.hbase.client.Delete;
|
||||||
import org.apache.hadoop.hbase.client.Get;
|
import org.apache.hadoop.hbase.client.Get;
|
||||||
|
import org.apache.hadoop.hbase.client.HConnectionManager;
|
||||||
import org.apache.hadoop.hbase.client.Increment;
|
import org.apache.hadoop.hbase.client.Increment;
|
||||||
import org.apache.hadoop.hbase.client.MultiAction;
|
import org.apache.hadoop.hbase.client.MultiAction;
|
||||||
import org.apache.hadoop.hbase.client.MultiPut;
|
import org.apache.hadoop.hbase.client.MultiPut;
|
||||||
|
@ -334,6 +335,8 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
|
||||||
throws IOException, InterruptedException {
|
throws IOException, InterruptedException {
|
||||||
this.fsOk = true;
|
this.fsOk = true;
|
||||||
this.conf = conf;
|
this.conf = conf;
|
||||||
|
// Set how many times to retry talking to another server over HConnection.
|
||||||
|
HConnectionManager.setServerSideHConnectionRetries(this.conf, LOG);
|
||||||
this.isOnline = false;
|
this.isOnline = false;
|
||||||
checkCodecs(this.conf);
|
checkCodecs(this.conf);
|
||||||
|
|
||||||
|
|
|
@ -78,4 +78,4 @@ public interface RegionServerServices extends OnlineRegions {
|
||||||
*/
|
*/
|
||||||
public Map<byte[], Boolean> getRegionsInTransitionInRS();
|
public Map<byte[], Boolean> getRegionsInTransitionInRS();
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -274,17 +274,19 @@ public class ThriftServer {
|
||||||
public List<TRegionInfo> getTableRegions(ByteBuffer tableName)
|
public List<TRegionInfo> getTableRegions(ByteBuffer tableName)
|
||||||
throws IOError {
|
throws IOError {
|
||||||
try{
|
try{
|
||||||
List<HRegionInfo> HRegions = this.admin.getTableRegions(tableName.array());
|
List<HRegionInfo> hris = this.admin.getTableRegions(tableName.array());
|
||||||
List<TRegionInfo> regions = new ArrayList<TRegionInfo>();
|
List<TRegionInfo> regions = new ArrayList<TRegionInfo>();
|
||||||
|
|
||||||
for (HRegionInfo regionInfo : HRegions){
|
if (hris != null) {
|
||||||
TRegionInfo region = new TRegionInfo();
|
for (HRegionInfo regionInfo : hris){
|
||||||
region.startKey = ByteBuffer.wrap(regionInfo.getStartKey());
|
TRegionInfo region = new TRegionInfo();
|
||||||
region.endKey = ByteBuffer.wrap(regionInfo.getEndKey());
|
region.startKey = ByteBuffer.wrap(regionInfo.getStartKey());
|
||||||
region.id = regionInfo.getRegionId();
|
region.endKey = ByteBuffer.wrap(regionInfo.getEndKey());
|
||||||
region.name = ByteBuffer.wrap(regionInfo.getRegionName());
|
region.id = regionInfo.getRegionId();
|
||||||
region.version = regionInfo.getVersion();
|
region.name = ByteBuffer.wrap(regionInfo.getRegionName());
|
||||||
regions.add(region);
|
region.version = regionInfo.getVersion();
|
||||||
|
regions.add(region);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
return regions;
|
return regions;
|
||||||
} catch (IOException e){
|
} catch (IOException e){
|
||||||
|
|
|
@ -891,8 +891,7 @@ public class HBaseFsck {
|
||||||
|
|
||||||
// record the latest modification of this META record
|
// record the latest modification of this META record
|
||||||
long ts = Collections.max(result.list(), comp).getTimestamp();
|
long ts = Collections.max(result.list(), comp).getTimestamp();
|
||||||
Pair<HRegionInfo, ServerName> pair =
|
Pair<HRegionInfo, ServerName> pair = MetaReader.parseCatalogResult(result);
|
||||||
MetaReader.metaRowToRegionPair(result);
|
|
||||||
if (pair == null || pair.getFirst() == null) {
|
if (pair == null || pair.getFirst() == null) {
|
||||||
emptyRegionInfoQualifiers.add(result);
|
emptyRegionInfoQualifiers.add(result);
|
||||||
return true;
|
return true;
|
||||||
|
|
|
@ -19,52 +19,29 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.zookeeper;
|
package org.apache.hadoop.hbase.zookeeper;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.hbase.Abortable;
|
import org.apache.hadoop.hbase.Abortable;
|
||||||
import org.apache.hadoop.hbase.HRegionInfo;
|
import org.apache.hadoop.hbase.HRegionInfo;
|
||||||
import org.apache.hadoop.hbase.catalog.CatalogTracker;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Tracks the unassigned zookeeper node used by the META table.
|
* Tracks the unassigned zookeeper node used by the META table.
|
||||||
*
|
|
||||||
* A callback is made into the passed {@link CatalogTracker} when
|
|
||||||
* <code>.META.</code> completes a new assignment.
|
|
||||||
* <p>
|
* <p>
|
||||||
* If META is already assigned when instantiating this class, you will not
|
* If META is already assigned when instantiating this class, you will not
|
||||||
* receive any notification for that assignment. You will receive a
|
* receive any notification for that assignment. You will receive a
|
||||||
* notification after META has been successfully assigned to a new location.
|
* notification after META has been successfully assigned to a new location.
|
||||||
*/
|
*/
|
||||||
public class MetaNodeTracker extends ZooKeeperNodeTracker {
|
public class MetaNodeTracker extends ZooKeeperNodeTracker {
|
||||||
private static final Log LOG = LogFactory.getLog(MetaNodeTracker.class);
|
|
||||||
|
|
||||||
/** Catalog tracker to notify when META has a new assignment completed. */
|
|
||||||
private final CatalogTracker catalogTracker;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a meta node tracker.
|
* Creates a meta node tracker.
|
||||||
* @param watcher
|
* @param watcher
|
||||||
* @param abortable
|
* @param abortable
|
||||||
*/
|
*/
|
||||||
public MetaNodeTracker(final ZooKeeperWatcher watcher,
|
public MetaNodeTracker(final ZooKeeperWatcher watcher, final Abortable abortable) {
|
||||||
final CatalogTracker catalogTracker, final Abortable abortable) {
|
|
||||||
super(watcher, ZKUtil.joinZNode(watcher.assignmentZNode,
|
super(watcher, ZKUtil.joinZNode(watcher.assignmentZNode,
|
||||||
HRegionInfo.FIRST_META_REGIONINFO.getEncodedName()), abortable);
|
HRegionInfo.FIRST_META_REGIONINFO.getEncodedName()), abortable);
|
||||||
this.catalogTracker = catalogTracker;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void nodeDeleted(String path) {
|
public void nodeDeleted(String path) {
|
||||||
super.nodeDeleted(path);
|
super.nodeDeleted(path);
|
||||||
if (!path.equals(node)) return;
|
|
||||||
LOG.info("Detected completed assignment of META, notifying catalog tracker");
|
|
||||||
try {
|
|
||||||
this.catalogTracker.waitForMetaServerConnectionDefault();
|
|
||||||
} catch (IOException e) {
|
|
||||||
LOG.warn("Tried to reset META server location after seeing the " +
|
|
||||||
"completion of a new META assignment but got an IOE", e);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -263,8 +263,13 @@ module Hbase
|
||||||
|
|
||||||
#----------------------------------------------------------------------------------------------
|
#----------------------------------------------------------------------------------------------
|
||||||
# Truncates table (deletes all records by recreating the table)
|
# Truncates table (deletes all records by recreating the table)
|
||||||
def truncate(table_name)
|
def truncate(table_name, conf = nil)
|
||||||
h_table = org.apache.hadoop.hbase.client.HTable.new(table_name)
|
h_table = nil
|
||||||
|
unless conf
|
||||||
|
h_table = org.apache.hadoop.hbase.client.HTable.new(table_name)
|
||||||
|
else
|
||||||
|
h_table = org.apache.hadoop.hbase.client.HTable.new(conf, table_name)
|
||||||
|
end
|
||||||
table_description = h_table.getTableDescriptor()
|
table_description = h_table.getTableDescriptor()
|
||||||
yield 'Disabling table...' if block_given?
|
yield 'Disabling table...' if block_given?
|
||||||
disable(table_name)
|
disable(table_name)
|
||||||
|
|
|
@ -25,7 +25,6 @@ import static org.junit.Assert.fail;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
@ -81,15 +80,11 @@ public class TestRegionRebalancing {
|
||||||
this.table = new HTable(UTIL.getConfiguration(), this.desc.getName());
|
this.table = new HTable(UTIL.getConfiguration(), this.desc.getName());
|
||||||
CatalogTracker ct = new CatalogTracker(UTIL.getConfiguration());
|
CatalogTracker ct = new CatalogTracker(UTIL.getConfiguration());
|
||||||
ct.start();
|
ct.start();
|
||||||
Map<HRegionInfo, ServerName> regions = null;
|
|
||||||
try {
|
try {
|
||||||
regions = MetaReader.fullScan(ct);
|
MetaReader.fullScanMetaAndPrint(ct);
|
||||||
} finally {
|
} finally {
|
||||||
ct.stop();
|
ct.stop();
|
||||||
}
|
}
|
||||||
for (Map.Entry<HRegionInfo, ServerName> e: regions.entrySet()) {
|
|
||||||
LOG.info(e);
|
|
||||||
}
|
|
||||||
assertEquals("Test table should have right number of regions",
|
assertEquals("Test table should have right number of regions",
|
||||||
HBaseTestingUtility.KEYS.length + 1/*One extra to account for start/end keys*/,
|
HBaseTestingUtility.KEYS.length + 1/*One extra to account for start/end keys*/,
|
||||||
this.table.getStartKeys().length);
|
this.table.getStartKeys().length);
|
||||||
|
|
|
@ -35,16 +35,20 @@ import org.apache.hadoop.hbase.Abortable;
|
||||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
import org.apache.hadoop.hbase.HRegionInfo;
|
import org.apache.hadoop.hbase.HRegionInfo;
|
||||||
import org.apache.hadoop.hbase.HServerAddress;
|
import org.apache.hadoop.hbase.HRegionLocation;
|
||||||
import org.apache.hadoop.hbase.KeyValue;
|
import org.apache.hadoop.hbase.KeyValue;
|
||||||
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
|
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
|
||||||
import org.apache.hadoop.hbase.ServerName;
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
import org.apache.hadoop.hbase.client.Get;
|
import org.apache.hadoop.hbase.client.Get;
|
||||||
import org.apache.hadoop.hbase.client.HConnection;
|
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.client.Result;
|
||||||
|
import org.apache.hadoop.hbase.client.ServerCallable;
|
||||||
import org.apache.hadoop.hbase.ipc.HRegionInterface;
|
import org.apache.hadoop.hbase.ipc.HRegionInterface;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.util.Threads;
|
import org.apache.hadoop.hbase.util.Threads;
|
||||||
|
import org.apache.hadoop.hbase.util.Writables;
|
||||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||||
import org.apache.hadoop.util.Progressable;
|
import org.apache.hadoop.util.Progressable;
|
||||||
|
@ -160,27 +164,44 @@ public class TestCatalogTracker {
|
||||||
t.join();
|
t.join();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test public void testGetMetaServerConnectionFails()
|
@Test
|
||||||
|
public void testGetMetaServerConnectionFails()
|
||||||
throws IOException, InterruptedException, KeeperException {
|
throws IOException, InterruptedException, KeeperException {
|
||||||
HConnection connection = Mockito.mock(HConnection.class);
|
HConnection connection =
|
||||||
ConnectException connectException =
|
HConnectionTestingUtility.getMockedConnection(UTIL.getConfiguration());
|
||||||
new ConnectException("Connection refused");
|
|
||||||
final HRegionInterface implementation =
|
|
||||||
Mockito.mock(HRegionInterface.class);
|
|
||||||
Mockito.when(implementation.get((byte [])Mockito.any(), (Get)Mockito.any())).
|
|
||||||
thenThrow(connectException);
|
|
||||||
Mockito.when(connection.getHRegionConnection((HServerAddress)Matchers.anyObject(), Matchers.anyBoolean())).
|
|
||||||
thenReturn(implementation);
|
|
||||||
Assert.assertNotNull(connection.getHRegionConnection(new HServerAddress(), false));
|
|
||||||
final CatalogTracker ct = constructAndStartCatalogTracker(connection);
|
|
||||||
try {
|
try {
|
||||||
RootLocationEditor.setRootLocation(this.watcher,
|
// Mock an HRegionInterface.
|
||||||
new ServerName("example.com", 1234, System.currentTimeMillis()));
|
final HRegionInterface implementation = Mockito.mock(HRegionInterface.class);
|
||||||
Assert.assertFalse(ct.verifyMetaRegionLocation(100));
|
// If a 'get' is called on mocked interface, throw connection refused.
|
||||||
|
Mockito.when(implementation.get((byte[]) Mockito.any(), (Get) Mockito.any())).
|
||||||
|
thenThrow(new ConnectException("Connection refused"));
|
||||||
|
// Make it so our implementation is returned when we do a connection.
|
||||||
|
// Need to fake out the location lookup stuff first.
|
||||||
|
ServerName sn = new ServerName("example.com", 1234, System.currentTimeMillis());
|
||||||
|
final HRegionLocation anyLocation =
|
||||||
|
new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, sn.getHostname(),
|
||||||
|
sn.getPort());
|
||||||
|
Mockito.when(connection.getRegionLocation((byte[]) Mockito.any(),
|
||||||
|
(byte[]) Mockito.any(), Mockito.anyBoolean())).
|
||||||
|
thenReturn(anyLocation);
|
||||||
|
Mockito.when(connection.getHRegionConnection(Mockito.anyString(),
|
||||||
|
Mockito.anyInt(), Matchers.anyBoolean())).
|
||||||
|
thenReturn(implementation);
|
||||||
|
// Now start up the catalogtracker with our doctored Connection.
|
||||||
|
final CatalogTracker ct = constructAndStartCatalogTracker(connection);
|
||||||
|
try {
|
||||||
|
RootLocationEditor.setRootLocation(this.watcher, sn);
|
||||||
|
long timeout = UTIL.getConfiguration().
|
||||||
|
getLong("hbase.catalog.verification.timeout", 1000);
|
||||||
|
Assert.assertFalse(ct.verifyMetaRegionLocation(timeout));
|
||||||
|
} finally {
|
||||||
|
// Clean out root location or later tests will be confused... they
|
||||||
|
// presume start fresh in zk.
|
||||||
|
RootLocationEditor.deleteRootLocation(this.watcher);
|
||||||
|
}
|
||||||
} finally {
|
} finally {
|
||||||
// Clean out root location or later tests will be confused... they presume
|
// Clear out our doctored connection or could mess up subsequent tests.
|
||||||
// start fresh in zk.
|
HConnectionManager.deleteConnection(UTIL.getConfiguration(), true);
|
||||||
RootLocationEditor.deleteRootLocation(this.watcher);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -200,9 +221,9 @@ public class TestCatalogTracker {
|
||||||
Mockito.mock(HRegionInterface.class);
|
Mockito.mock(HRegionInterface.class);
|
||||||
Mockito.when(implementation.getRegionInfo((byte [])Mockito.any())).
|
Mockito.when(implementation.getRegionInfo((byte [])Mockito.any())).
|
||||||
thenThrow(connectException);
|
thenThrow(connectException);
|
||||||
Mockito.when(connection.getHRegionConnection((HServerAddress)Matchers.anyObject(), Matchers.anyBoolean())).
|
Mockito.when(connection.getHRegionConnection(Mockito.anyString(),
|
||||||
|
Mockito.anyInt(), Mockito.anyBoolean())).
|
||||||
thenReturn(implementation);
|
thenReturn(implementation);
|
||||||
Assert.assertNotNull(connection.getHRegionConnection(new HServerAddress(), false));
|
|
||||||
final CatalogTracker ct = constructAndStartCatalogTracker(connection);
|
final CatalogTracker ct = constructAndStartCatalogTracker(connection);
|
||||||
try {
|
try {
|
||||||
RootLocationEditor.setRootLocation(this.watcher,
|
RootLocationEditor.setRootLocation(this.watcher,
|
||||||
|
@ -225,8 +246,14 @@ public class TestCatalogTracker {
|
||||||
@Test (expected = NotAllMetaRegionsOnlineException.class)
|
@Test (expected = NotAllMetaRegionsOnlineException.class)
|
||||||
public void testTimeoutWaitForMeta()
|
public void testTimeoutWaitForMeta()
|
||||||
throws IOException, InterruptedException {
|
throws IOException, InterruptedException {
|
||||||
final CatalogTracker ct = constructAndStartCatalogTracker();
|
HConnection connection =
|
||||||
ct.waitForMeta(100);
|
HConnectionTestingUtility.getMockedConnection(UTIL.getConfiguration());
|
||||||
|
try {
|
||||||
|
final CatalogTracker ct = constructAndStartCatalogTracker(connection);
|
||||||
|
ct.waitForMeta(100);
|
||||||
|
} finally {
|
||||||
|
HConnectionManager.deleteConnection(UTIL.getConfiguration(), true);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -259,62 +286,84 @@ public class TestCatalogTracker {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test waiting on meta w/ no timeout specified.
|
* Test waiting on meta w/ no timeout specified.
|
||||||
* @throws IOException
|
* @throws Exception
|
||||||
* @throws InterruptedException
|
|
||||||
* @throws KeeperException
|
|
||||||
*/
|
*/
|
||||||
@Test public void testNoTimeoutWaitForMeta()
|
@Test public void testNoTimeoutWaitForMeta()
|
||||||
throws IOException, InterruptedException, KeeperException {
|
throws Exception {
|
||||||
// Mock an HConnection and a HRegionInterface implementation. Have the
|
// Mock an HConnection and a HRegionInterface implementation. Have the
|
||||||
// HConnection return the HRI. Have the HRI return a few mocked up responses
|
// HConnection return the HRI. Have the HRI return a few mocked up responses
|
||||||
// to make our test work.
|
// to make our test work.
|
||||||
HConnection connection = Mockito.mock(HConnection.class);
|
HConnection connection =
|
||||||
HRegionInterface mockHRI = Mockito.mock(HRegionInterface.class);
|
HConnectionTestingUtility.getMockedConnection(UTIL.getConfiguration());
|
||||||
// Make the HRI return an answer no matter how Get is called. Same for
|
try {
|
||||||
// getHRegionInfo. Thats enough for this test.
|
// Mock an HRegionInterface.
|
||||||
Mockito.when(connection.getHRegionConnection((String)Mockito.any(),
|
|
||||||
Matchers.anyInt())).thenReturn(mockHRI);
|
final HRegionInterface implementation = Mockito.mock(HRegionInterface.class);
|
||||||
|
// Make it so our implementation is returned when we do a connection.
|
||||||
|
// Need to fake out the location lookup stuff first.
|
||||||
|
ServerName sn = new ServerName("example.com", 1234, System.currentTimeMillis());
|
||||||
|
final HRegionLocation anyLocation =
|
||||||
|
new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, sn.getHostname(),
|
||||||
|
sn.getPort());
|
||||||
|
Mockito.when(connection.getRegionLocation((byte[]) Mockito.any(),
|
||||||
|
(byte[]) Mockito.any(), Mockito.anyBoolean())).
|
||||||
|
thenReturn(anyLocation);
|
||||||
|
// Have implementation returned which ever way getHRegionConnection is called.
|
||||||
|
Mockito.when(connection.getHRegionConnection(Mockito.anyString(),
|
||||||
|
Mockito.anyInt(), Matchers.anyBoolean())).
|
||||||
|
thenReturn(implementation);
|
||||||
|
Mockito.when(connection.getHRegionConnection(Mockito.anyString(),
|
||||||
|
Mockito.anyInt())).
|
||||||
|
thenReturn(implementation);
|
||||||
|
|
||||||
final CatalogTracker ct = constructAndStartCatalogTracker(connection);
|
final CatalogTracker ct = constructAndStartCatalogTracker(connection);
|
||||||
ServerName hsa = ct.getMetaLocation();
|
ServerName hsa = ct.getMetaLocation();
|
||||||
Assert.assertNull(hsa);
|
Assert.assertNull(hsa);
|
||||||
|
|
||||||
// Now test waiting on meta location getting set.
|
// Now test waiting on meta location getting set.
|
||||||
Thread t = new WaitOnMetaThread(ct) {
|
Thread t = new WaitOnMetaThread(ct) {
|
||||||
@Override
|
@Override
|
||||||
void doWaiting() throws InterruptedException {
|
void doWaiting() throws InterruptedException {
|
||||||
this.ct.waitForMeta();
|
this.ct.waitForMeta();
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
startWaitAliveThenWaitItLives(t, 1000);
|
startWaitAliveThenWaitItLives(t, 1000);
|
||||||
|
|
||||||
// Now the ct is up... set into the mocks some answers that make it look
|
// Now the ct is up... set into the mocks some answers that make it look
|
||||||
// like things have been getting assigned. Make it so we'll return a
|
// like things have been getting assigned. Make it so we'll return a
|
||||||
// location (no matter what the Get is). Same for getHRegionInfo -- always
|
// location (no matter what the Get is). Same for getHRegionInfo -- always
|
||||||
// just return the meta region.
|
// just return the meta region.
|
||||||
List<KeyValue> kvs = new ArrayList<KeyValue>();
|
List<KeyValue> kvs = new ArrayList<KeyValue>();
|
||||||
kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY,
|
kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY,
|
||||||
HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
|
HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
|
||||||
Bytes.toBytes(SN.getHostAndPort())));
|
Writables.getBytes(HRegionInfo.FIRST_META_REGIONINFO)));
|
||||||
kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY,
|
kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY,
|
||||||
HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER,
|
HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
|
||||||
Bytes.toBytes(SN.getStartcode())));
|
Bytes.toBytes(SN.getHostAndPort())));
|
||||||
final Result result = new Result(kvs);
|
kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY,
|
||||||
Mockito.when(mockHRI.get((byte [])Mockito.any(), (Get)Mockito.any())).
|
HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER,
|
||||||
thenReturn(result);
|
Bytes.toBytes(SN.getStartcode())));
|
||||||
Mockito.when(mockHRI.getRegionInfo((byte [])Mockito.any())).
|
final Result result = new Result(kvs);
|
||||||
thenReturn(HRegionInfo.FIRST_META_REGIONINFO);
|
Mockito.when(connection.getRegionServerWithRetries((ServerCallable<Result>)Mockito.any())).
|
||||||
// This should trigger wake up of meta wait (Its the removal of the meta
|
thenReturn(result);
|
||||||
// region unassigned node that triggers catalogtrackers that a meta has
|
Mockito.when(implementation.getRegionInfo((byte[]) Mockito.any())).
|
||||||
// been assigned.
|
thenReturn(HRegionInfo.FIRST_META_REGIONINFO);
|
||||||
String node = ct.getMetaNodeTracker().getNode();
|
// This should trigger wake up of meta wait (Its the removal of the meta
|
||||||
ZKUtil.createAndFailSilent(this.watcher, node);
|
// region unassigned node that triggers catalogtrackers that a meta has
|
||||||
MetaEditor.updateMetaLocation(ct, HRegionInfo.FIRST_META_REGIONINFO, SN);
|
// been assigned).
|
||||||
ZKUtil.deleteNode(this.watcher, node);
|
String node = ct.getMetaNodeTracker().getNode();
|
||||||
// Join the thread... should exit shortly.
|
ZKUtil.createAndFailSilent(this.watcher, node);
|
||||||
t.join();
|
MetaEditor.updateMetaLocation(ct, HRegionInfo.FIRST_META_REGIONINFO, SN);
|
||||||
// Now meta is available.
|
ZKUtil.deleteNode(this.watcher, node);
|
||||||
Assert.assertTrue(ct.getMetaLocation().equals(SN));
|
// Go get the new meta location. waitForMeta gets and verifies meta.
|
||||||
|
Assert.assertTrue(ct.waitForMeta(10000).equals(SN));
|
||||||
|
// Join the thread... should exit shortly.
|
||||||
|
t.join();
|
||||||
|
// Now meta is available.
|
||||||
|
Assert.assertTrue(ct.waitForMeta(10000).equals(SN));
|
||||||
|
} finally {
|
||||||
|
HConnectionManager.deleteConnection(UTIL.getConfiguration(), true);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void startWaitAliveThenWaitItLives(final Thread t, final int ms) {
|
private void startWaitAliveThenWaitItLives(final Thread t, final int ms) {
|
||||||
|
@ -361,4 +410,4 @@ public class TestCatalogTracker {
|
||||||
this.ct.waitForRoot();
|
this.ct.waitForRoot();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -19,7 +19,10 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.catalog;
|
package org.apache.hadoop.hbase.catalog;
|
||||||
|
|
||||||
import static org.junit.Assert.*;
|
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.io.IOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -34,8 +37,6 @@ import org.apache.hadoop.hbase.HConstants;
|
||||||
import org.apache.hadoop.hbase.HRegionInfo;
|
import org.apache.hadoop.hbase.HRegionInfo;
|
||||||
import org.apache.hadoop.hbase.ServerName;
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
import org.apache.hadoop.hbase.client.HBaseAdmin;
|
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.client.HTable;
|
import org.apache.hadoop.hbase.client.HTable;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.util.Pair;
|
import org.apache.hadoop.hbase.util.Pair;
|
||||||
|
@ -70,11 +71,15 @@ public class TestMetaReaderEditor {
|
||||||
};
|
};
|
||||||
|
|
||||||
@BeforeClass public static void beforeClass() throws Exception {
|
@BeforeClass public static void beforeClass() throws Exception {
|
||||||
UTIL.startMiniCluster();
|
UTIL.startMiniCluster(3);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Before public void setup() throws IOException, InterruptedException {
|
@Before public void setup() throws IOException, InterruptedException {
|
||||||
Configuration c = new Configuration(UTIL.getConfiguration());
|
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("hbase.client.retries.number", 10);
|
||||||
zkw = new ZooKeeperWatcher(c, "TestMetaReaderEditor", ABORTABLE);
|
zkw = new ZooKeeperWatcher(c, "TestMetaReaderEditor", ABORTABLE);
|
||||||
ct = new CatalogTracker(zkw, c, ABORTABLE);
|
ct = new CatalogTracker(zkw, c, ABORTABLE);
|
||||||
ct.start();
|
ct.start();
|
||||||
|
@ -84,6 +89,111 @@ public class TestMetaReaderEditor {
|
||||||
UTIL.shutdownMiniCluster();
|
UTIL.shutdownMiniCluster();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Does {@link MetaReader#getRegion(CatalogTracker, byte[])} and a write
|
||||||
|
* against .META. while its hosted server is restarted to prove our retrying
|
||||||
|
* works.
|
||||||
|
* @throws IOException
|
||||||
|
* @throws InterruptedException
|
||||||
|
*/
|
||||||
|
@Test (timeout = 180000) public void testRetrying()
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
final String name = "testRetrying";
|
||||||
|
LOG.info("Started " + name);
|
||||||
|
final byte [] nameBytes = Bytes.toBytes(name);
|
||||||
|
HTable t = UTIL.createTable(nameBytes, 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(this.ct, nameBytes, regionCount);
|
||||||
|
MetaTask reader = new MetaTask(this.ct, "reader") {
|
||||||
|
@Override
|
||||||
|
void metaTask() throws Throwable {
|
||||||
|
testGetRegion(this.ct, regions.get(0));
|
||||||
|
LOG.info("Read " + regions.get(0).getEncodedName());
|
||||||
|
}
|
||||||
|
};
|
||||||
|
MetaTask writer = new MetaTask(this.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();
|
||||||
|
// 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);
|
||||||
|
UTIL.getMiniHBaseCluster().abortRegionServer(index);
|
||||||
|
UTIL.getMiniHBaseCluster().waitOnRegionServer(index);
|
||||||
|
}
|
||||||
|
assertTrue(reader.toString(), reader.isProgressing());
|
||||||
|
assertTrue(writer.toString(), writer.isProgressing());
|
||||||
|
reader.stop = true;
|
||||||
|
writer.stop = true;
|
||||||
|
reader.join();
|
||||||
|
writer.join();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* 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()
|
@Test public void testGetRegionsCatalogTables()
|
||||||
throws IOException, InterruptedException {
|
throws IOException, InterruptedException {
|
||||||
List<HRegionInfo> regions =
|
List<HRegionInfo> regions =
|
||||||
|
@ -114,19 +224,9 @@ public class TestMetaReaderEditor {
|
||||||
@Test public void testGetRegion() throws IOException, InterruptedException {
|
@Test public void testGetRegion() throws IOException, InterruptedException {
|
||||||
final String name = "testGetRegion";
|
final String name = "testGetRegion";
|
||||||
LOG.info("Started " + name);
|
LOG.info("Started " + name);
|
||||||
final byte [] nameBytes = Bytes.toBytes(name);
|
|
||||||
HTable t = UTIL.createTable(nameBytes, HConstants.CATALOG_FAMILY);
|
|
||||||
int regionCount = UTIL.createMultiRegions(t, HConstants.CATALOG_FAMILY);
|
|
||||||
|
|
||||||
// Test it works getting a region from user table.
|
|
||||||
List<HRegionInfo> regions = MetaReader.getTableRegions(ct, nameBytes);
|
|
||||||
assertEquals(regionCount, regions.size());
|
|
||||||
Pair<HRegionInfo, ServerName> pair =
|
|
||||||
MetaReader.getRegion(ct, regions.get(0).getRegionName());
|
|
||||||
assertEquals(regions.get(0).getEncodedName(),
|
|
||||||
pair.getFirst().getEncodedName());
|
|
||||||
// Test get on non-existent region.
|
// Test get on non-existent region.
|
||||||
pair = MetaReader.getRegion(ct, Bytes.toBytes("nonexistent-region"));
|
Pair<HRegionInfo, ServerName> pair =
|
||||||
|
MetaReader.getRegion(ct, Bytes.toBytes("nonexistent-region"));
|
||||||
assertNull(pair);
|
assertNull(pair);
|
||||||
// Test it works getting a region from meta/root.
|
// Test it works getting a region from meta/root.
|
||||||
pair =
|
pair =
|
||||||
|
@ -137,7 +237,8 @@ public class TestMetaReaderEditor {
|
||||||
}
|
}
|
||||||
|
|
||||||
// Test for the optimization made in HBASE-3650
|
// Test for the optimization made in HBASE-3650
|
||||||
@Test public void testScanMetaForTable() throws IOException {
|
@Test public void testScanMetaForTable()
|
||||||
|
throws IOException, InterruptedException {
|
||||||
final String name = "testScanMetaForTable";
|
final String name = "testScanMetaForTable";
|
||||||
LOG.info("Started " + name);
|
LOG.info("Started " + name);
|
||||||
|
|
||||||
|
@ -165,4 +266,25 @@ public class TestMetaReaderEditor {
|
||||||
}
|
}
|
||||||
assertEquals(1, MetaReader.getTableRegions(ct, greaterName).size());
|
assertEquals(1, MetaReader.getTableRegions(ct, greaterName).size());
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
private static List<HRegionInfo> testGettingTableRegions(final CatalogTracker ct,
|
||||||
|
final byte [] nameBytes, final int regionCount)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
List<HRegionInfo> regions = MetaReader.getTableRegions(ct, nameBytes);
|
||||||
|
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());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
|
@ -0,0 +1,177 @@
|
||||||
|
/**
|
||||||
|
* 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.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.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.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.client.Scan;
|
||||||
|
import org.apache.hadoop.hbase.ipc.HRegionInterface;
|
||||||
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
import org.apache.hadoop.hbase.util.Writables;
|
||||||
|
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||||
|
import org.junit.After;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.mockito.Mockito;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test MetaReader/Editor but without spinning up a cluster.
|
||||||
|
* We mock regionserver back and forth (we do spin up a zk cluster).
|
||||||
|
*/
|
||||||
|
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 that MetaReader will ride over server throwing
|
||||||
|
* "Server not running" IOEs.
|
||||||
|
* @see https://issues.apache.org/jira/browse/HBASE-3446
|
||||||
|
* @throws IOException
|
||||||
|
* @throws InterruptedException
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testRideOverServerNotRunning() throws IOException, InterruptedException {
|
||||||
|
// 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 = new ServerName("example.com", 1234, System.currentTimeMillis());
|
||||||
|
|
||||||
|
HConnection connection = null;
|
||||||
|
CatalogTracker ct = null;
|
||||||
|
try {
|
||||||
|
// Mock an HRegionInterface. Our mock implementation will fail a few
|
||||||
|
// times when we go to open a scanner.
|
||||||
|
final HRegionInterface implementation = Mockito.mock(HRegionInterface.class);
|
||||||
|
// When openScanner 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 openscanner was indeed
|
||||||
|
// called the wanted number of times.
|
||||||
|
final long scannerid = 123L;
|
||||||
|
Mockito.when(implementation.openScanner((byte [])Mockito.any(),
|
||||||
|
(Scan)Mockito.any())).
|
||||||
|
thenThrow(new IOException("Server not running (1 of 3)")).
|
||||||
|
thenThrow(new IOException("Server not running (2 of 3)")).
|
||||||
|
thenThrow(new IOException("Server not running (3 of 3)")).
|
||||||
|
thenReturn(scannerid);
|
||||||
|
// Make it so a verifiable answer comes back when next is called. Return
|
||||||
|
// the verifiable answer and then a null so we stop scanning. Our
|
||||||
|
// verifiable answer is something that looks like a row in META with
|
||||||
|
// a server and startcode that is that of the above defined servername.
|
||||||
|
List<KeyValue> kvs = new ArrayList<KeyValue>();
|
||||||
|
final byte [] rowToVerify = Bytes.toBytes("rowToVerify");
|
||||||
|
kvs.add(new KeyValue(rowToVerify,
|
||||||
|
HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
|
||||||
|
Writables.getBytes(HRegionInfo.FIRST_META_REGIONINFO)));
|
||||||
|
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 Result [] result = new Result [] {new Result(kvs)};
|
||||||
|
Mockito.when(implementation.next(Mockito.anyLong(), Mockito.anyInt())).
|
||||||
|
thenReturn(result).
|
||||||
|
thenReturn(null);
|
||||||
|
|
||||||
|
// 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.getHostname(),
|
||||||
|
sn.getPort());
|
||||||
|
// Return the any location object when locateRegion is called in HTable
|
||||||
|
// constructor and when its called by ServerCallable (it uses getRegionLocation).
|
||||||
|
// 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
|
||||||
|
Mockito.doReturn(anyLocation).
|
||||||
|
when(connection).locateRegion((byte[]) Mockito.any(), (byte[]) Mockito.any());
|
||||||
|
Mockito.doReturn(anyLocation).
|
||||||
|
when(connection).getRegionLocation((byte[]) Mockito.any(),
|
||||||
|
(byte[]) Mockito.any(), Mockito.anyBoolean());
|
||||||
|
|
||||||
|
// Now shove our HRI implementation into the spied-upon connection.
|
||||||
|
Mockito.doReturn(implementation).
|
||||||
|
when(connection).getHRegionConnection(Mockito.anyString(), Mockito.anyInt());
|
||||||
|
|
||||||
|
// Now start up the catalogtracker with our doctored Connection.
|
||||||
|
ct = new CatalogTracker(zkw, null, connection, ABORTABLE, 0);
|
||||||
|
ct.start();
|
||||||
|
// Scan meta for user tables and verify we got back expected answer.
|
||||||
|
NavigableMap<HRegionInfo, Result> hris = MetaReader.getServerUserRegions(ct, sn);
|
||||||
|
assertTrue(hris.size() == 1);
|
||||||
|
assertTrue(hris.firstEntry().getKey().equals(HRegionInfo.FIRST_META_REGIONINFO));
|
||||||
|
assertTrue(Bytes.equals(rowToVerify, hris.firstEntry().getValue().getRow()));
|
||||||
|
// Finally verify that openscanner was called four times -- three times
|
||||||
|
// with exception and then on 4th attempt we succeed.
|
||||||
|
Mockito.verify(implementation, Mockito.times(4)).
|
||||||
|
openScanner((byte [])Mockito.any(), (Scan)Mockito.any());
|
||||||
|
} finally {
|
||||||
|
if (ct != null) ct.stop();
|
||||||
|
HConnectionManager.deleteConnection(UTIL.getConfiguration(), true);
|
||||||
|
zkw.close();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,85 @@
|
||||||
|
/**
|
||||||
|
* 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.client;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
||||||
|
import org.apache.hadoop.hbase.client.HConnectionManager.HConnectionImplementation;
|
||||||
|
import org.apache.hadoop.hbase.client.HConnectionManager.HConnectionKey;
|
||||||
|
import org.mockito.Mockito;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* {@link HConnection} testing utility.
|
||||||
|
*/
|
||||||
|
public class HConnectionTestingUtility {
|
||||||
|
/*
|
||||||
|
* Not part of {@link HBaseTestingUtility} because this class is not
|
||||||
|
* in same package as {@link HConnection}. Would have to reveal ugly
|
||||||
|
* {@link HConnectionManager} innards to HBaseTestingUtility to give it access.
|
||||||
|
*/
|
||||||
|
/**
|
||||||
|
* Get a Mocked {@link HConnection} that goes with the passed <code>conf</code>
|
||||||
|
* configuration instance. Minimally the mock will return
|
||||||
|
* <code>conf</conf> when {@link HConnection#getConfiguration()} is invoked.
|
||||||
|
* Be sure to shutdown the connection when done by calling
|
||||||
|
* {@link HConnectionManager#deleteConnection(Configuration, boolean)} else it
|
||||||
|
* will stick around; this is probably not what you want.
|
||||||
|
* @param conf configuration
|
||||||
|
* @return HConnection object for <code>conf</code>
|
||||||
|
* @throws ZooKeeperConnectionException
|
||||||
|
*/
|
||||||
|
public static HConnection getMockedConnection(final Configuration conf)
|
||||||
|
throws ZooKeeperConnectionException {
|
||||||
|
HConnectionKey connectionKey = new HConnectionKey(conf);
|
||||||
|
synchronized (HConnectionManager.HBASE_INSTANCES) {
|
||||||
|
HConnectionImplementation connection =
|
||||||
|
HConnectionManager.HBASE_INSTANCES.get(connectionKey);
|
||||||
|
if (connection == null) {
|
||||||
|
connection = Mockito.mock(HConnectionImplementation.class);
|
||||||
|
Mockito.when(connection.getConfiguration()).thenReturn(conf);
|
||||||
|
HConnectionManager.HBASE_INSTANCES.put(connectionKey, connection);
|
||||||
|
}
|
||||||
|
return connection;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get a Mockito spied-upon {@link HConnection} that goes with the passed
|
||||||
|
* <code>conf</code> configuration instance.
|
||||||
|
* Be sure to shutdown the connection when done by calling
|
||||||
|
* {@link HConnectionManager#deleteConnection(Configuration, boolean)} else it
|
||||||
|
* will stick around; this is probably not what you want.
|
||||||
|
* @param conf configuration
|
||||||
|
* @return HConnection object for <code>conf</code>
|
||||||
|
* @throws ZooKeeperConnectionException
|
||||||
|
* @see http://mockito.googlecode.com/svn/branches/1.6/javadoc/org/mockito/Mockito.html#spy(T)
|
||||||
|
*/
|
||||||
|
public static HConnection getSpiedConnection(final Configuration conf)
|
||||||
|
throws ZooKeeperConnectionException {
|
||||||
|
HConnectionKey connectionKey = new HConnectionKey(conf);
|
||||||
|
synchronized (HConnectionManager.HBASE_INSTANCES) {
|
||||||
|
HConnectionImplementation connection =
|
||||||
|
HConnectionManager.HBASE_INSTANCES.get(connectionKey);
|
||||||
|
if (connection == null) {
|
||||||
|
connection = Mockito.spy(new HConnectionImplementation(conf));
|
||||||
|
HConnectionManager.HBASE_INSTANCES.put(connectionKey, connection);
|
||||||
|
}
|
||||||
|
return connection;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -19,6 +19,10 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.client;
|
package org.apache.hadoop.hbase.client;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertNotNull;
|
||||||
|
import static org.junit.Assert.assertNull;
|
||||||
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
import java.lang.reflect.Field;
|
import java.lang.reflect.Field;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
|
@ -27,6 +31,8 @@ import java.util.Map;
|
||||||
import java.util.Random;
|
import java.util.Random;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||||
|
@ -38,12 +44,6 @@ import org.junit.AfterClass;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.BeforeClass;
|
import org.junit.BeforeClass;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
|
|
||||||
import static org.junit.Assert.assertNull;
|
|
||||||
import static org.junit.Assert.assertNotNull;
|
|
||||||
import static org.junit.Assert.assertTrue;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This class is for testing HCM features
|
* This class is for testing HCM features
|
||||||
|
@ -225,4 +225,4 @@ public class TestHCM {
|
||||||
Thread.sleep(50);
|
Thread.sleep(50);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -28,7 +28,7 @@ import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hbase.*;
|
import org.apache.hadoop.hbase.*;
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.catalog.MetaEditor;
|
import org.apache.hadoop.hbase.catalog.MetaMigrationRemovingHTD;
|
||||||
import org.apache.hadoop.hbase.catalog.MetaReader;
|
import org.apache.hadoop.hbase.catalog.MetaReader;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
|
@ -77,7 +77,8 @@ public class TestMetaMigration {
|
||||||
@Test
|
@Test
|
||||||
public void testMetaUpdatedFlagInROOT() throws Exception {
|
public void testMetaUpdatedFlagInROOT() throws Exception {
|
||||||
LOG.info("Starting testMetaUpdatedFlagInROOT");
|
LOG.info("Starting testMetaUpdatedFlagInROOT");
|
||||||
boolean metaUpdated = miniHBaseCluster.getMaster().isMetaHRIUpdated();
|
boolean metaUpdated =
|
||||||
|
MetaMigrationRemovingHTD.isMetaHRIUpdated(miniHBaseCluster.getMaster());
|
||||||
assertEquals(true, metaUpdated);
|
assertEquals(true, metaUpdated);
|
||||||
LOG.info("END testMetaUpdatedFlagInROOT");
|
LOG.info("END testMetaUpdatedFlagInROOT");
|
||||||
}
|
}
|
||||||
|
@ -97,16 +98,17 @@ public class TestMetaMigration {
|
||||||
Bytes.toBytes("region_b")});
|
Bytes.toBytes("region_b")});
|
||||||
CatalogTracker ct = miniHBaseCluster.getMaster().getCatalogTracker();
|
CatalogTracker ct = miniHBaseCluster.getMaster().getCatalogTracker();
|
||||||
// just for this test set it to false.
|
// just for this test set it to false.
|
||||||
MetaEditor.updateRootWithMetaMigrationStatus(ct, false);
|
MetaMigrationRemovingHTD.updateRootWithMetaMigrationStatus(ct, false);
|
||||||
MetaReader.fullScanMetaAndPrint(ct);
|
MetaReader.fullScanMetaAndPrint(ct);
|
||||||
LOG.info("MEta Print completed.testUpdatesOnMetaWithLegacyHRI");
|
LOG.info("Meta Print completed.testUpdatesOnMetaWithLegacyHRI");
|
||||||
|
|
||||||
List<HTableDescriptor> htds = MetaEditor.updateMetaWithNewRegionInfo(
|
List<HTableDescriptor> htds = MetaMigrationRemovingHTD.updateMetaWithNewRegionInfo(
|
||||||
TEST_UTIL.getHBaseCluster().getMaster());
|
TEST_UTIL.getHBaseCluster().getMaster());
|
||||||
MetaReader.fullScanMetaAndPrint(ct);
|
MetaReader.fullScanMetaAndPrint(ct);
|
||||||
assertEquals(3, htds.size());
|
assertEquals(3, htds.size());
|
||||||
// Assert that the flag in ROOT is updated to reflect the correct status
|
// Assert that the flag in ROOT is updated to reflect the correct status
|
||||||
boolean metaUpdated = miniHBaseCluster.getMaster().isMetaHRIUpdated();
|
boolean metaUpdated =
|
||||||
|
MetaMigrationRemovingHTD.isMetaHRIUpdated(miniHBaseCluster.getMaster());
|
||||||
assertEquals(true, metaUpdated);
|
assertEquals(true, metaUpdated);
|
||||||
LOG.info("END testMetaWithLegacyHRI");
|
LOG.info("END testMetaWithLegacyHRI");
|
||||||
|
|
||||||
|
@ -138,15 +140,16 @@ public class TestMetaMigration {
|
||||||
TEST_UTIL.createMultiRegionsWithLegacyHRI(conf, htd, FAMILY, 10);
|
TEST_UTIL.createMultiRegionsWithLegacyHRI(conf, htd, FAMILY, 10);
|
||||||
CatalogTracker ct = miniHBaseCluster.getMaster().getCatalogTracker();
|
CatalogTracker ct = miniHBaseCluster.getMaster().getCatalogTracker();
|
||||||
// just for this test set it to false.
|
// just for this test set it to false.
|
||||||
MetaEditor.updateRootWithMetaMigrationStatus(ct, false);
|
MetaMigrationRemovingHTD.updateRootWithMetaMigrationStatus(ct, false);
|
||||||
//MetaReader.fullScanMetaAndPrint(ct);
|
//MetaReader.fullScanMetaAndPrint(ct);
|
||||||
LOG.info("MEta Print completed.testUpdatesOnMetaWithLegacyHRI");
|
LOG.info("MEta Print completed.testUpdatesOnMetaWithLegacyHRI");
|
||||||
|
|
||||||
List<HTableDescriptor> htds = MetaEditor.updateMetaWithNewRegionInfo(
|
List<HTableDescriptor> htds = MetaMigrationRemovingHTD.updateMetaWithNewRegionInfo(
|
||||||
TEST_UTIL.getHBaseCluster().getMaster());
|
TEST_UTIL.getHBaseCluster().getMaster());
|
||||||
assertEquals(10, htds.size());
|
assertEquals(10, htds.size());
|
||||||
// Assert that the flag in ROOT is updated to reflect the correct status
|
// Assert that the flag in ROOT is updated to reflect the correct status
|
||||||
boolean metaUpdated = miniHBaseCluster.getMaster().isMetaHRIUpdated();
|
boolean metaUpdated =
|
||||||
|
MetaMigrationRemovingHTD.isMetaHRIUpdated(miniHBaseCluster.getMaster());
|
||||||
assertEquals(true, metaUpdated);
|
assertEquals(true, metaUpdated);
|
||||||
LOG.info("END testMetaWithLegacyHRI");
|
LOG.info("END testMetaWithLegacyHRI");
|
||||||
|
|
||||||
|
|
|
@ -47,6 +47,9 @@ import org.apache.hadoop.hbase.ServerName;
|
||||||
import org.apache.hadoop.hbase.TableDescriptors;
|
import org.apache.hadoop.hbase.TableDescriptors;
|
||||||
import org.apache.hadoop.hbase.TableExistsException;
|
import org.apache.hadoop.hbase.TableExistsException;
|
||||||
import org.apache.hadoop.hbase.catalog.CatalogTracker;
|
import org.apache.hadoop.hbase.catalog.CatalogTracker;
|
||||||
|
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.client.Result;
|
||||||
import org.apache.hadoop.hbase.executor.ExecutorService;
|
import org.apache.hadoop.hbase.executor.ExecutorService;
|
||||||
import org.apache.hadoop.hbase.io.Reference;
|
import org.apache.hadoop.hbase.io.Reference;
|
||||||
|
@ -61,20 +64,27 @@ import org.mockito.Mockito;
|
||||||
public class TestCatalogJanitor {
|
public class TestCatalogJanitor {
|
||||||
/**
|
/**
|
||||||
* Pseudo server for below tests.
|
* Pseudo server for below tests.
|
||||||
|
* Be sure to call stop on the way out else could leave some mess around.
|
||||||
*/
|
*/
|
||||||
class MockServer implements Server {
|
class MockServer implements Server {
|
||||||
|
private final HConnection connection;
|
||||||
private final Configuration c;
|
private final Configuration c;
|
||||||
private final CatalogTracker ct;
|
private final CatalogTracker ct;
|
||||||
|
|
||||||
MockServer(final HBaseTestingUtility htu)
|
MockServer(final HBaseTestingUtility htu)
|
||||||
throws NotAllMetaRegionsOnlineException, IOException {
|
throws NotAllMetaRegionsOnlineException, IOException, InterruptedException {
|
||||||
this.c = htu.getConfiguration();
|
this.c = htu.getConfiguration();
|
||||||
|
// Mock an HConnection and a HRegionInterface implementation. Have the
|
||||||
|
// HConnection return the HRI. Have the HRI return a few mocked up responses
|
||||||
|
// to make our test work.
|
||||||
|
this.connection = HConnectionTestingUtility.getMockedConnection(this.c);
|
||||||
// Set hbase.rootdir into test dir.
|
// Set hbase.rootdir into test dir.
|
||||||
FileSystem fs = FileSystem.get(this.c);
|
FileSystem fs = FileSystem.get(this.c);
|
||||||
Path rootdir = fs.makeQualified(new Path(this.c.get(HConstants.HBASE_DIR)));
|
Path rootdir = fs.makeQualified(new Path(this.c.get(HConstants.HBASE_DIR)));
|
||||||
this.c.set(HConstants.HBASE_DIR, rootdir.toString());
|
this.c.set(HConstants.HBASE_DIR, rootdir.toString());
|
||||||
this.ct = Mockito.mock(CatalogTracker.class);
|
this.ct = Mockito.mock(CatalogTracker.class);
|
||||||
HRegionInterface hri = Mockito.mock(HRegionInterface.class);
|
HRegionInterface hri = Mockito.mock(HRegionInterface.class);
|
||||||
|
Mockito.when(this.ct.getConnection()).thenReturn(this.connection);
|
||||||
Mockito.when(ct.waitForMetaServerConnectionDefault()).thenReturn(hri);
|
Mockito.when(ct.waitForMetaServerConnectionDefault()).thenReturn(hri);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -115,9 +125,13 @@ public class TestCatalogJanitor {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void stop(String why) {
|
public void stop(String why) {
|
||||||
//no-op
|
if (this.ct != null) {
|
||||||
|
this.ct.stop();
|
||||||
|
}
|
||||||
|
if (this.connection != null) {
|
||||||
|
HConnectionManager.deleteConnection(this.connection.getConfiguration(), true);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -267,42 +281,53 @@ public class TestCatalogJanitor {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testCleanParent() throws IOException {
|
public void testCleanParent() throws IOException, InterruptedException {
|
||||||
HBaseTestingUtility htu = new HBaseTestingUtility();
|
HBaseTestingUtility htu = new HBaseTestingUtility();
|
||||||
setRootDirAndCleanIt(htu, "testCleanParent");
|
setRootDirAndCleanIt(htu, "testCleanParent");
|
||||||
Server server = new MockServer(htu);
|
Server server = new MockServer(htu);
|
||||||
MasterServices services = new MockMasterServices(server);
|
try {
|
||||||
CatalogJanitor janitor = new CatalogJanitor(server, services);
|
MasterServices services = new MockMasterServices(server);
|
||||||
// Create regions.
|
CatalogJanitor janitor = new CatalogJanitor(server, services);
|
||||||
HTableDescriptor htd = createHTableDescriptor();
|
// Create regions.
|
||||||
HRegionInfo parent =
|
HTableDescriptor htd = new HTableDescriptor("table");
|
||||||
new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"),
|
htd.addFamily(new HColumnDescriptor("f"));
|
||||||
Bytes.toBytes("eee"));
|
HRegionInfo parent =
|
||||||
HRegionInfo splita =
|
new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"),
|
||||||
new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"),
|
Bytes.toBytes("eee"));
|
||||||
Bytes.toBytes("ccc"));
|
HRegionInfo splita =
|
||||||
HRegionInfo splitb =
|
new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"),
|
||||||
new HRegionInfo(htd.getName(), Bytes.toBytes("ccc"),
|
Bytes.toBytes("ccc"));
|
||||||
Bytes.toBytes("eee"));
|
HRegionInfo splitb =
|
||||||
// Test that when both daughter regions are in place, that we do not
|
new HRegionInfo(htd.getName(), Bytes.toBytes("ccc"),
|
||||||
// remove the parent.
|
Bytes.toBytes("eee"));
|
||||||
Result r = createResult(parent, splita, splitb);
|
// Test that when both daughter regions are in place, that we do not
|
||||||
// Add a reference under splitA directory so we don't clear out the parent.
|
// remove the parent.
|
||||||
Path rootdir = services.getMasterFileSystem().getRootDir();
|
List<KeyValue> kvs = new ArrayList<KeyValue>();
|
||||||
Path tabledir =
|
kvs.add(new KeyValue(parent.getRegionName(), HConstants.CATALOG_FAMILY,
|
||||||
HTableDescriptor.getTableDir(rootdir, htd.getName());
|
HConstants.SPLITA_QUALIFIER, Writables.getBytes(splita)));
|
||||||
Path storedir = Store.getStoreHomedir(tabledir, splita.getEncodedName(),
|
kvs.add(new KeyValue(parent.getRegionName(), HConstants.CATALOG_FAMILY,
|
||||||
htd.getColumnFamilies()[0].getName());
|
HConstants.SPLITB_QUALIFIER, Writables.getBytes(splitb)));
|
||||||
Reference ref = new Reference(Bytes.toBytes("ccc"), Reference.Range.top);
|
Result r = new Result(kvs);
|
||||||
long now = System.currentTimeMillis();
|
// Add a reference under splitA directory so we don't clear out the parent.
|
||||||
// Reference name has this format: StoreFile#REF_NAME_PARSER
|
Path rootdir = services.getMasterFileSystem().getRootDir();
|
||||||
Path p = new Path(storedir, Long.toString(now) + "." + parent.getEncodedName());
|
Path tabledir =
|
||||||
FileSystem fs = services.getMasterFileSystem().getFileSystem();
|
HTableDescriptor.getTableDir(rootdir, htd.getName());
|
||||||
ref.write(fs, p);
|
Path storedir = Store.getStoreHomedir(tabledir, splita.getEncodedName(),
|
||||||
assertFalse(janitor.cleanParent(parent, r));
|
htd.getColumnFamilies()[0].getName());
|
||||||
// Remove the reference file and try again.
|
Reference ref = new Reference(Bytes.toBytes("ccc"), Reference.Range.top);
|
||||||
assertTrue(fs.delete(p, true));
|
long now = System.currentTimeMillis();
|
||||||
assertTrue(janitor.cleanParent(parent, r));
|
// Reference name has this format: StoreFile#REF_NAME_PARSER
|
||||||
|
Path p = new Path(storedir, Long.toString(now) + "." + parent.getEncodedName());
|
||||||
|
FileSystem fs = services.getMasterFileSystem().getFileSystem();
|
||||||
|
Path path = ref.write(fs, p);
|
||||||
|
assertTrue(fs.exists(path));
|
||||||
|
assertFalse(janitor.cleanParent(parent, r));
|
||||||
|
// Remove the reference file and try again.
|
||||||
|
assertTrue(fs.delete(p, true));
|
||||||
|
assertTrue(janitor.cleanParent(parent, r));
|
||||||
|
} finally {
|
||||||
|
server.stop("shutdown");
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -459,4 +484,4 @@ public class TestCatalogJanitor {
|
||||||
htd.addFamily(new HColumnDescriptor("f"));
|
htd.addFamily(new HColumnDescriptor("f"));
|
||||||
return htd;
|
return htd;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -56,6 +56,7 @@ import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||||
import org.apache.hadoop.hbase.util.FSUtils;
|
import org.apache.hadoop.hbase.util.FSUtils;
|
||||||
|
import org.apache.hadoop.hbase.util.Threads;
|
||||||
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
|
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
|
||||||
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
|
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
|
||||||
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
|
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
|
||||||
|
@ -98,6 +99,9 @@ public class TestDistributedLogSplitting {
|
||||||
LOG.info("Waiting for active/ready master");
|
LOG.info("Waiting for active/ready master");
|
||||||
cluster.waitForActiveAndReadyMaster();
|
cluster.waitForActiveAndReadyMaster();
|
||||||
master = cluster.getMaster();
|
master = cluster.getMaster();
|
||||||
|
while (cluster.getLiveRegionServerThreads().size() < num_rs) {
|
||||||
|
Threads.sleep(1);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@After
|
@After
|
||||||
|
@ -111,7 +115,7 @@ public class TestDistributedLogSplitting {
|
||||||
final int NUM_REGIONS_TO_CREATE = 40;
|
final int NUM_REGIONS_TO_CREATE = 40;
|
||||||
final int NUM_ROWS_PER_REGION = 100;
|
final int NUM_ROWS_PER_REGION = 100;
|
||||||
|
|
||||||
startCluster(NUM_RS);
|
startCluster(NUM_RS); // NUM_RS=6.
|
||||||
|
|
||||||
ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf,
|
ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf,
|
||||||
"distributed log splitting test", null);
|
"distributed log splitting test", null);
|
||||||
|
|
|
@ -86,21 +86,20 @@ public class TestMaster {
|
||||||
tableRegions.get(0).getFirst().getEndKey());
|
tableRegions.get(0).getFirst().getEndKey());
|
||||||
|
|
||||||
// Now trigger a split and stop when the split is in progress
|
// Now trigger a split and stop when the split is in progress
|
||||||
|
CountDownLatch split = new CountDownLatch(1);
|
||||||
CountDownLatch aboutToOpen = new CountDownLatch(1);
|
|
||||||
CountDownLatch proceed = new CountDownLatch(1);
|
CountDownLatch proceed = new CountDownLatch(1);
|
||||||
RegionOpenListener list = new RegionOpenListener(aboutToOpen, proceed);
|
RegionSplitListener list = new RegionSplitListener(split, proceed);
|
||||||
cluster.getMaster().executorService.
|
cluster.getMaster().executorService.
|
||||||
registerListener(EventType.RS_ZK_REGION_OPENED, list);
|
registerListener(EventType.RS_ZK_REGION_SPLIT, list);
|
||||||
|
|
||||||
LOG.info("Splitting table");
|
LOG.info("Splitting table");
|
||||||
admin.split(TABLENAME);
|
admin.split(TABLENAME);
|
||||||
LOG.info("Waiting for split result to be about to open");
|
LOG.info("Waiting for split result to be about to open");
|
||||||
aboutToOpen.await(60, TimeUnit.SECONDS);
|
split.await(60, TimeUnit.SECONDS);
|
||||||
try {
|
try {
|
||||||
LOG.info("Making sure we can call getTableRegions while opening");
|
LOG.info("Making sure we can call getTableRegions while opening");
|
||||||
tableRegions = MetaReader.getTableRegionsAndLocations(
|
tableRegions = MetaReader.getTableRegionsAndLocations(m.getCatalogTracker(),
|
||||||
m.getCatalogTracker(), Bytes.toString(TABLENAME));
|
TABLENAME, false);
|
||||||
|
|
||||||
LOG.info("Regions: " + Joiner.on(',').join(tableRegions));
|
LOG.info("Regions: " + Joiner.on(',').join(tableRegions));
|
||||||
// We have three regions because one is split-in-progress
|
// We have three regions because one is split-in-progress
|
||||||
|
@ -118,22 +117,21 @@ public class TestMaster {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
static class RegionOpenListener implements EventHandlerListener {
|
static class RegionSplitListener implements EventHandlerListener {
|
||||||
CountDownLatch aboutToOpen, proceed;
|
CountDownLatch split, proceed;
|
||||||
|
|
||||||
public RegionOpenListener(CountDownLatch aboutToOpen, CountDownLatch proceed)
|
public RegionSplitListener(CountDownLatch split, CountDownLatch proceed) {
|
||||||
{
|
this.split = split;
|
||||||
this.aboutToOpen = aboutToOpen;
|
|
||||||
this.proceed = proceed;
|
this.proceed = proceed;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void afterProcess(EventHandler event) {
|
public void afterProcess(EventHandler event) {
|
||||||
if (event.getEventType() != EventType.RS_ZK_REGION_OPENED) {
|
if (event.getEventType() != EventType.RS_ZK_REGION_SPLIT) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
aboutToOpen.countDown();
|
split.countDown();
|
||||||
proceed.await(60, TimeUnit.SECONDS);
|
proceed.await(60, TimeUnit.SECONDS);
|
||||||
} catch (InterruptedException ie) {
|
} catch (InterruptedException ie) {
|
||||||
throw new RuntimeException(ie);
|
throw new RuntimeException(ie);
|
||||||
|
|
|
@ -23,7 +23,6 @@ import static org.junit.Assert.assertEquals;
|
||||||
import static org.junit.Assert.assertFalse;
|
import static org.junit.Assert.assertFalse;
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
@ -34,11 +33,18 @@ import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.hbase.*;
|
import org.apache.hadoop.hbase.Abortable;
|
||||||
import org.apache.hadoop.hbase.executor.RegionTransitionData;
|
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||||
|
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.MiniHBaseCluster;
|
||||||
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
|
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
|
||||||
|
import org.apache.hadoop.hbase.executor.RegionTransitionData;
|
||||||
import org.apache.hadoop.hbase.master.AssignmentManager.RegionState;
|
import org.apache.hadoop.hbase.master.AssignmentManager.RegionState;
|
||||||
import org.apache.hadoop.hbase.master.RegionPlan;
|
|
||||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
@ -49,8 +55,6 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
|
||||||
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
|
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
|
||||||
import org.apache.hadoop.hbase.zookeeper.ZKTable;
|
import org.apache.hadoop.hbase.zookeeper.ZKTable;
|
||||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||||
import org.apache.zookeeper.KeeperException;
|
|
||||||
import org.apache.zookeeper.KeeperException.NodeExistsException;
|
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
public class TestMasterFailover {
|
public class TestMasterFailover {
|
||||||
|
@ -131,7 +135,7 @@ public class TestMasterFailover {
|
||||||
// Stop the cluster
|
// Stop the cluster
|
||||||
TEST_UTIL.shutdownMiniCluster();
|
TEST_UTIL.shutdownMiniCluster();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testShouldCheckMasterFailOverWhenMETAIsInOpenedState()
|
public void testShouldCheckMasterFailOverWhenMETAIsInOpenedState()
|
||||||
throws Exception {
|
throws Exception {
|
||||||
|
@ -1006,4 +1010,4 @@ public class TestMasterFailover {
|
||||||
private void log(String string) {
|
private void log(String string) {
|
||||||
LOG.info("\n\n" + string + " \n\n");
|
LOG.info("\n\n" + string + " \n\n");
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -128,8 +128,10 @@ public class TestMergeTable {
|
||||||
assertTrue("originalTableRegions=" + originalTableRegions.size() +
|
assertTrue("originalTableRegions=" + originalTableRegions.size() +
|
||||||
", postMergeTableRegions=" + postMergeTableRegions.size(),
|
", postMergeTableRegions=" + postMergeTableRegions.size(),
|
||||||
postMergeTableRegions.size() < originalTableRegions.size());
|
postMergeTableRegions.size() < originalTableRegions.size());
|
||||||
|
LOG.info("Done with merge");
|
||||||
} finally {
|
} finally {
|
||||||
UTIL.shutdownMiniCluster();
|
UTIL.shutdownMiniCluster();
|
||||||
|
LOG.info("After cluster shutdown");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -152,7 +152,7 @@ module Hbase
|
||||||
assert_equal(['a:', 'b:'], table(@create_test_name).get_all_columns.sort)
|
assert_equal(['a:', 'b:'], table(@create_test_name).get_all_columns.sort)
|
||||||
end
|
end
|
||||||
|
|
||||||
define_test "create hould work with hash column args" do
|
define_test "create should work with hash column args" do
|
||||||
drop_test_table(@create_test_name)
|
drop_test_table(@create_test_name)
|
||||||
admin.create(@create_test_name, { NAME => 'a'}, { NAME => 'b'})
|
admin.create(@create_test_name, { NAME => 'a'}, { NAME => 'b'})
|
||||||
assert_equal(['a:', 'b:'], table(@create_test_name).get_all_columns.sort)
|
assert_equal(['a:', 'b:'], table(@create_test_name).get_all_columns.sort)
|
||||||
|
@ -160,14 +160,14 @@ module Hbase
|
||||||
|
|
||||||
#-------------------------------------------------------------------------------
|
#-------------------------------------------------------------------------------
|
||||||
|
|
||||||
# define_test "close should work without region server name" do
|
define_test "close should work without region server name" do
|
||||||
# if admin.exists?(@create_test_name)
|
if admin.exists?(@create_test_name)
|
||||||
# admin.disable(@create_test_name)
|
admin.disable(@create_test_name)
|
||||||
# admin.drop(@create_test_name)
|
admin.drop(@create_test_name)
|
||||||
# end
|
end
|
||||||
# admin.create(@create_test_name, 'foo')
|
admin.create(@create_test_name, 'foo')
|
||||||
# admin.close_region(@create_test_name + ',,0')
|
admin.close_region(@create_test_name + ',,0', nil)
|
||||||
# end
|
end
|
||||||
|
|
||||||
#-------------------------------------------------------------------------------
|
#-------------------------------------------------------------------------------
|
||||||
|
|
||||||
|
@ -187,13 +187,14 @@ module Hbase
|
||||||
table(@test_name).put(1, "x:a", 1)
|
table(@test_name).put(1, "x:a", 1)
|
||||||
table(@test_name).put(2, "x:a", 2)
|
table(@test_name).put(2, "x:a", 2)
|
||||||
assert_equal(2, table(@test_name).count)
|
assert_equal(2, table(@test_name).count)
|
||||||
admin.truncate(@test_name)
|
# This is hacky. Need to get the configuration into admin instance
|
||||||
|
admin.truncate(@test_name, $TEST_CLUSTER.getConfiguration)
|
||||||
assert_equal(0, table(@test_name).count)
|
assert_equal(0, table(@test_name).count)
|
||||||
end
|
end
|
||||||
|
|
||||||
define_test "truncate should yield log records" do
|
define_test "truncate should yield log records" do
|
||||||
logs = []
|
logs = []
|
||||||
admin.truncate(@test_name) do |log|
|
admin.truncate(@test_name, $TEST_CLUSTER.getConfiguration) do |log|
|
||||||
assert_kind_of(String, log)
|
assert_kind_of(String, log)
|
||||||
logs << log
|
logs << log
|
||||||
end
|
end
|
||||||
|
|
|
@ -25,7 +25,7 @@ require 'shell/formatter'
|
||||||
class ShellTest < Test::Unit::TestCase
|
class ShellTest < Test::Unit::TestCase
|
||||||
def setup
|
def setup
|
||||||
@formatter = ::Shell::Formatter::Console.new()
|
@formatter = ::Shell::Formatter::Console.new()
|
||||||
@hbase = ::Hbase::Hbase.new
|
@hbase = ::Hbase::Hbase.new($TEST_CLUSTER.getConfiguration)
|
||||||
@shell = Shell::Shell.new(@hbase, @formatter)
|
@shell = Shell::Shell.new(@hbase, @formatter)
|
||||||
end
|
end
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue