HBASE-3984 CT.verifyRegionLocation isn't doing a very good check,
can delay cluster recovery git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1141290 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
71607dc16f
commit
70d9045694
15
CHANGES.txt
15
CHANGES.txt
|
@ -136,12 +136,12 @@ Release 0.91.0 - Unreleased
|
|||
(Akash Ashok via Ted Yu)
|
||||
HBASE-4037 Add timeout annotations to preempt surefire killing
|
||||
all tests
|
||||
HBASE-4024 Major compaction may not be triggered, even though region
|
||||
server log says it is triggered (Ted Yu)
|
||||
HBASE-4016 HRegion.incrementColumnValue() doesn't have a consistent
|
||||
behavior when the field that we are incrementing is less
|
||||
than 8 bytes long (Li Pi)
|
||||
HBASE-4012 Further optimize byte comparison methods (Ted Yu)
|
||||
HBASE-4024 Major compaction may not be triggered, even though region
|
||||
server log says it is triggered (Ted Yu)
|
||||
HBASE-4016 HRegion.incrementColumnValue() doesn't have a consistent
|
||||
behavior when the field that we are incrementing is less
|
||||
than 8 bytes long (Li Pi)
|
||||
HBASE-4012 Further optimize byte comparison methods (Ted Yu)
|
||||
|
||||
IMPROVEMENTS
|
||||
HBASE-3290 Max Compaction Size (Nicolas Spiegelberg via Stack)
|
||||
|
@ -377,7 +377,8 @@ Release 0.90.4 - Unreleased
|
|||
(Lars George via Ted Yu)
|
||||
HBASE-4020 "testWritesWhileGetting" unit test needs to be fixed.
|
||||
(Vandana Ayyalasomayajula via Ted Yu)
|
||||
|
||||
HBASE-3984 CT.verifyRegionLocation isn't doing a very good check,
|
||||
can delay cluster recovery
|
||||
|
||||
IMPROVEMENT
|
||||
HBASE-3882 hbase-config.sh needs to be updated so it can auto-detects the
|
||||
|
|
|
@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.client.HConnection;
|
|||
import org.apache.hadoop.hbase.client.HConnectionManager;
|
||||
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
|
||||
import org.apache.hadoop.hbase.ipc.HRegionInterface;
|
||||
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.zookeeper.MetaNodeTracker;
|
||||
import org.apache.hadoop.hbase.zookeeper.RootRegionTracker;
|
||||
|
@ -490,7 +491,7 @@ public class CatalogTracker {
|
|||
connection = waitForRootServerConnection(timeout);
|
||||
} catch (NotAllMetaRegionsOnlineException e) {
|
||||
// Pass
|
||||
} catch (org.apache.hadoop.hbase.ipc.ServerNotRunningException e) {
|
||||
} catch (ServerNotRunningYetException e) {
|
||||
// Pass -- remote server is not up so can't be carrying root
|
||||
} catch (IOException e) {
|
||||
// Unexpected exception
|
||||
|
|
|
@ -1061,7 +1061,7 @@ public abstract class HBaseServer implements RpcServer {
|
|||
CurCall.set(call);
|
||||
try {
|
||||
if (!started)
|
||||
throw new ServerNotRunningException("Server is not running yet");
|
||||
throw new ServerNotRunningYetException("Server is not running yet");
|
||||
value = call(call.connection.protocol, call.param, call.timestamp); // make the call
|
||||
} catch (Throwable e) {
|
||||
LOG.debug(getName()+", call "+call+": error: " + e, e);
|
||||
|
@ -1238,7 +1238,7 @@ public abstract class HBaseServer implements RpcServer {
|
|||
|
||||
/**
|
||||
* Starts the service threads but does not allow requests to be responded yet.
|
||||
* Client will get {@link ServerNotRunningException} instead.
|
||||
* Client will get {@link ServerNotRunningYetException} instead.
|
||||
*/
|
||||
@Override
|
||||
public synchronized void startThreads() {
|
||||
|
|
|
@ -288,7 +288,7 @@ public interface HRegionInterface extends VersionedProtocol, Stoppable, Abortabl
|
|||
* @return All regions online on this region server
|
||||
* @throws IOException e
|
||||
*/
|
||||
public List<HRegionInfo> getOnlineRegions();
|
||||
public List<HRegionInfo> getOnlineRegions() throws IOException;
|
||||
|
||||
/**
|
||||
* Method used when a master is taking the place of another failed one.
|
||||
|
|
|
@ -22,8 +22,8 @@ package org.apache.hadoop.hbase.ipc;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
public class ServerNotRunningException extends IOException {
|
||||
public ServerNotRunningException(String s) {
|
||||
public class ServerNotRunningYetException extends IOException {
|
||||
public ServerNotRunningYetException(String s) {
|
||||
super(s);
|
||||
}
|
||||
}
|
|
@ -56,11 +56,13 @@ import org.apache.hadoop.hbase.client.Result;
|
|||
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
|
||||
import org.apache.hadoop.hbase.executor.ExecutorService;
|
||||
import org.apache.hadoop.hbase.executor.RegionTransitionData;
|
||||
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
|
||||
import org.apache.hadoop.hbase.master.LoadBalancer.RegionPlan;
|
||||
import org.apache.hadoop.hbase.master.handler.ClosedRegionHandler;
|
||||
import org.apache.hadoop.hbase.master.handler.OpenedRegionHandler;
|
||||
import org.apache.hadoop.hbase.master.handler.ServerShutdownHandler;
|
||||
import org.apache.hadoop.hbase.master.handler.SplitRegionHandler;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
|
@ -1030,6 +1032,9 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
*/
|
||||
void assign(final ServerName destination,
|
||||
final List<HRegionInfo> regions) {
|
||||
if (regions.size() == 0) {
|
||||
return;
|
||||
}
|
||||
LOG.debug("Bulk assigning " + regions.size() + " region(s) to " +
|
||||
destination.toString());
|
||||
|
||||
|
@ -1063,10 +1068,8 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
}
|
||||
// Move on to open regions.
|
||||
try {
|
||||
// Send OPEN RPC. This can fail if the server on other end is is not up.
|
||||
// If we fail, fail the startup by aborting the server. There is one
|
||||
// exception we will tolerate: ServerNotRunningException. This is thrown
|
||||
// between report of regionserver being up and
|
||||
// Send OPEN RPC. If it fails on a IOE or RemoteException, the
|
||||
// TimeoutMonitor will pick up the pieces.
|
||||
long maxWaitTime = System.currentTimeMillis() +
|
||||
this.master.getConfiguration().
|
||||
getLong("hbase.regionserver.rpc.startup.waittime", 60000);
|
||||
|
@ -1074,18 +1077,29 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
try {
|
||||
this.serverManager.sendRegionOpen(destination, regions);
|
||||
break;
|
||||
} catch (org.apache.hadoop.hbase.ipc.ServerNotRunningException e) {
|
||||
// This is the one exception to retry. For all else we should just fail
|
||||
// the startup.
|
||||
long now = System.currentTimeMillis();
|
||||
if (now > maxWaitTime) throw e;
|
||||
LOG.debug("Server is not yet up; waiting up to " +
|
||||
(maxWaitTime - now) + "ms", e);
|
||||
Thread.sleep(1000);
|
||||
} catch (RemoteException e) {
|
||||
IOException decodedException = e.unwrapRemoteException();
|
||||
if (decodedException instanceof RegionServerStoppedException) {
|
||||
LOG.warn("The region server was shut down, ", decodedException);
|
||||
// No need to retry, the region server is a goner.
|
||||
return;
|
||||
} else if (decodedException instanceof ServerNotRunningYetException) {
|
||||
// This is the one exception to retry. For all else we should just fail
|
||||
// the startup.
|
||||
long now = System.currentTimeMillis();
|
||||
if (now > maxWaitTime) throw e;
|
||||
LOG.debug("Server is not yet up; waiting up to " +
|
||||
(maxWaitTime - now) + "ms", e);
|
||||
Thread.sleep(1000);
|
||||
}
|
||||
|
||||
throw decodedException;
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
// Can be a socket timeout, EOF, NoRouteToHost, etc
|
||||
LOG.info("Unable to communicate with the region server in order" +
|
||||
" to assign regions", e);
|
||||
} catch (InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
|
|
|
@ -106,7 +106,7 @@ import org.apache.hadoop.hbase.ipc.HMasterRegionInterface;
|
|||
import org.apache.hadoop.hbase.ipc.HRegionInterface;
|
||||
import org.apache.hadoop.hbase.ipc.Invocation;
|
||||
import org.apache.hadoop.hbase.ipc.RpcServer;
|
||||
import org.apache.hadoop.hbase.ipc.ServerNotRunningException;
|
||||
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
|
||||
import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
|
||||
import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler;
|
||||
import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler;
|
||||
|
@ -1520,7 +1520,7 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
|
|||
} catch (IOException e) {
|
||||
e = e instanceof RemoteException ?
|
||||
((RemoteException)e).unwrapRemoteException() : e;
|
||||
if (e instanceof ServerNotRunningException) {
|
||||
if (e instanceof ServerNotRunningYetException) {
|
||||
LOG.info("Master isn't available yet, retrying");
|
||||
} else {
|
||||
LOG.warn("Unable to connect to master. Retrying. Error was:", e);
|
||||
|
@ -1624,7 +1624,8 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
|
|||
@Override
|
||||
@QosPriority(priority=HIGH_QOS)
|
||||
public HRegionInfo getRegionInfo(final byte[] regionName)
|
||||
throws NotServingRegionException {
|
||||
throws NotServingRegionException, IOException {
|
||||
checkOpen();
|
||||
requestCount.incrementAndGet();
|
||||
return getRegion(regionName).getRegionInfo();
|
||||
}
|
||||
|
@ -2140,9 +2141,9 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
|
|||
|
||||
public int delete(final byte[] regionName, final List<Delete> deletes)
|
||||
throws IOException {
|
||||
checkOpen();
|
||||
// Count of Deletes processed.
|
||||
int i = 0;
|
||||
checkOpen();
|
||||
HRegion region = null;
|
||||
try {
|
||||
region = getRegion(regionName);
|
||||
|
@ -2262,6 +2263,7 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
|
|||
@Override
|
||||
public void bulkLoadHFile(String hfilePath, byte[] regionName,
|
||||
byte[] familyName) throws IOException {
|
||||
checkOpen();
|
||||
HRegion region = getRegion(regionName);
|
||||
region.bulkLoadHFile(hfilePath, familyName);
|
||||
}
|
||||
|
@ -2296,12 +2298,12 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
|
|||
@QosPriority(priority=HIGH_QOS)
|
||||
public void openRegion(HRegionInfo region)
|
||||
throws IOException {
|
||||
checkOpen();
|
||||
if (this.regionsInTransitionInRS.contains(region.getEncodedNameAsBytes())) {
|
||||
throw new RegionAlreadyInTransitionException("open", region.getEncodedName());
|
||||
}
|
||||
LOG.info("Received request to open region: " +
|
||||
region.getRegionNameAsString());
|
||||
if (this.stopped) throw new RegionServerStoppedException();
|
||||
HTableDescriptor htd = this.tableDescriptors.get(region.getTableName());
|
||||
if (region.isRootRegion()) {
|
||||
this.service.submit(new OpenRootHandler(this, this, region, htd));
|
||||
|
@ -2316,6 +2318,7 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
|
|||
@QosPriority(priority=HIGH_QOS)
|
||||
public void openRegions(List<HRegionInfo> regions)
|
||||
throws IOException {
|
||||
checkOpen();
|
||||
LOG.info("Received request to open " + regions.size() + " region(s)");
|
||||
for (HRegionInfo region: regions) openRegion(region);
|
||||
}
|
||||
|
@ -2331,6 +2334,7 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
|
|||
@QosPriority(priority=HIGH_QOS)
|
||||
public boolean closeRegion(HRegionInfo region, final boolean zk)
|
||||
throws IOException {
|
||||
checkOpen();
|
||||
LOG.info("Received close region: " + region.getRegionNameAsString());
|
||||
boolean hasit = this.onlineRegions.containsKey(region.getEncodedName());
|
||||
if (!hasit) {
|
||||
|
@ -2378,6 +2382,7 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
|
|||
@QosPriority(priority=HIGH_QOS)
|
||||
public void flushRegion(HRegionInfo regionInfo)
|
||||
throws NotServingRegionException, IOException {
|
||||
checkOpen();
|
||||
LOG.info("Flushing " + regionInfo.getRegionNameAsString());
|
||||
HRegion region = getRegion(regionInfo.getRegionName());
|
||||
region.flushcache();
|
||||
|
@ -2393,6 +2398,7 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
|
|||
@Override
|
||||
public void splitRegion(HRegionInfo regionInfo, byte[] splitPoint)
|
||||
throws NotServingRegionException, IOException {
|
||||
checkOpen();
|
||||
HRegion region = getRegion(regionInfo.getRegionName());
|
||||
region.flushcache();
|
||||
region.forceSplit(splitPoint);
|
||||
|
@ -2403,6 +2409,7 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
|
|||
@QosPriority(priority=HIGH_QOS)
|
||||
public void compactRegion(HRegionInfo regionInfo, boolean major)
|
||||
throws NotServingRegionException, IOException {
|
||||
checkOpen();
|
||||
HRegion region = getRegion(regionInfo.getRegionName());
|
||||
if (major) {
|
||||
region.triggerMajorCompaction();
|
||||
|
@ -2444,7 +2451,8 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
|
|||
|
||||
@Override
|
||||
@QosPriority(priority=HIGH_QOS)
|
||||
public List<HRegionInfo> getOnlineRegions() {
|
||||
public List<HRegionInfo> getOnlineRegions() throws IOException {
|
||||
checkOpen();
|
||||
List<HRegionInfo> list = new ArrayList<HRegionInfo>(onlineRegions.size());
|
||||
for (Map.Entry<String,HRegion> e: this.onlineRegions.entrySet()) {
|
||||
list.add(e.getValue().getRegionInfo());
|
||||
|
@ -2613,11 +2621,11 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
|
|||
*/
|
||||
protected void checkOpen() throws IOException {
|
||||
if (this.stopped || this.abortRequested) {
|
||||
throw new IOException("Server not running"
|
||||
throw new RegionServerStoppedException("Server not running"
|
||||
+ (this.abortRequested ? ", aborting" : ""));
|
||||
}
|
||||
if (!fsOk) {
|
||||
throw new IOException("File system not available");
|
||||
throw new RegionServerStoppedException("File system not available");
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -2736,6 +2744,7 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
|
|||
@Override
|
||||
@QosPriority(priority=HIGH_QOS)
|
||||
public HServerInfo getHServerInfo() throws IOException {
|
||||
checkOpen();
|
||||
return new HServerInfo(new HServerAddress(this.isa),
|
||||
this.startcode, this.webuiport);
|
||||
}
|
||||
|
@ -2743,6 +2752,7 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
|
|||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public <R> MultiResponse multi(MultiAction<R> multi) throws IOException {
|
||||
checkOpen();
|
||||
MultiResponse response = new MultiResponse();
|
||||
for (Map.Entry<byte[], List<Action<R>>> e : multi.actions.entrySet()) {
|
||||
byte[] regionName = e.getKey();
|
||||
|
@ -2843,6 +2853,7 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
|
|||
*/
|
||||
@Override
|
||||
public MultiPutResponse multiPut(MultiPut puts) throws IOException {
|
||||
checkOpen();
|
||||
MultiPutResponse resp = new MultiPutResponse();
|
||||
|
||||
// do each region as it's own.
|
||||
|
@ -2981,6 +2992,7 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
|
|||
@Override
|
||||
public void replicateLogEntries(final HLog.Entry[] entries)
|
||||
throws IOException {
|
||||
checkOpen();
|
||||
if (this.replicationHandler == null) return;
|
||||
this.replicationHandler.replicateLogEntries(entries);
|
||||
}
|
||||
|
|
|
@ -19,14 +19,15 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Thrown by the region server when it is shutting down state.
|
||||
*
|
||||
* Should NEVER be thrown to HBase clients, they will abort the call chain
|
||||
* and not retry even though regions will transition to new servers.
|
||||
* Thrown by the region server when it is in shutting down state.
|
||||
*/
|
||||
@SuppressWarnings("serial")
|
||||
public class RegionServerStoppedException extends DoNotRetryIOException {
|
||||
public class RegionServerStoppedException extends IOException {
|
||||
|
||||
public RegionServerStoppedException(String s) {
|
||||
super(s);
|
||||
}
|
||||
}
|
|
@ -20,6 +20,8 @@
|
|||
package org.apache.hadoop.hbase;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.ArrayList;
|
||||
import org.apache.commons.logging.Log;
|
||||
|
@ -100,7 +102,7 @@ public class TestGlobalMemStoreSize {
|
|||
}
|
||||
|
||||
/** figure out how many regions are currently being served. */
|
||||
private int getRegionCount() {
|
||||
private int getRegionCount() throws IOException {
|
||||
int total = 0;
|
||||
for (HRegionServer server : getOnlineRegionServers()) {
|
||||
total += server.getOnlineRegions().size();
|
||||
|
@ -122,7 +124,7 @@ public class TestGlobalMemStoreSize {
|
|||
/**
|
||||
* Wait until all the regions are assigned.
|
||||
*/
|
||||
private void waitForAllRegionsAssigned() {
|
||||
private void waitForAllRegionsAssigned() throws IOException {
|
||||
while (getRegionCount() < totalRegionNum) {
|
||||
LOG.debug("Waiting for there to be "+totalRegionNum+" regions, but there are " + getRegionCount() + " right now.");
|
||||
try {
|
||||
|
|
|
@ -152,7 +152,7 @@ public class TestRegionRebalancing extends HBaseClusterTestCase {
|
|||
}
|
||||
|
||||
/** figure out how many regions are currently being served. */
|
||||
private int getRegionCount() {
|
||||
private int getRegionCount() throws IOException {
|
||||
int total = 0;
|
||||
for (HRegionServer server : getOnlineRegionServers()) {
|
||||
total += server.getOnlineRegions().size();
|
||||
|
@ -165,7 +165,7 @@ public class TestRegionRebalancing extends HBaseClusterTestCase {
|
|||
* number of online servers, then test if each server is +/- 1 of average
|
||||
* rounded up.
|
||||
*/
|
||||
private void assertRegionsAreBalanced() {
|
||||
private void assertRegionsAreBalanced() throws IOException {
|
||||
// TODO: Fix this test. Old balancer used to run with 'slop'. New
|
||||
// balancer does not.
|
||||
boolean success = false;
|
||||
|
@ -229,7 +229,7 @@ public class TestRegionRebalancing extends HBaseClusterTestCase {
|
|||
/**
|
||||
* Wait until all the regions are assigned.
|
||||
*/
|
||||
private void waitForAllRegionsAssigned() {
|
||||
private void waitForAllRegionsAssigned() throws IOException {
|
||||
while (getRegionCount() < 22) {
|
||||
// while (!cluster.getMaster().allRegionsAssigned()) {
|
||||
LOG.debug("Waiting for there to be 22 regions, but there are " + getRegionCount() + " right now.");
|
||||
|
|
|
@ -413,7 +413,8 @@ public class TestDistributedLogSplitting {
|
|||
}
|
||||
}
|
||||
|
||||
private NavigableSet<String> getAllOnlineRegions(MiniHBaseCluster cluster) {
|
||||
private NavigableSet<String> getAllOnlineRegions(MiniHBaseCluster cluster)
|
||||
throws IOException {
|
||||
NavigableSet<String> online = new TreeSet<String>();
|
||||
for (RegionServerThread rst : cluster.getLiveRegionServerThreads()) {
|
||||
for (HRegionInfo region : rst.getRegionServer().getOnlineRegions()) {
|
||||
|
|
|
@ -53,7 +53,8 @@ public class TestHMasterRPCException {
|
|||
inf.isMasterRunning();
|
||||
fail();
|
||||
} catch (RemoteException ex) {
|
||||
assertTrue(ex.getMessage().startsWith("org.apache.hadoop.hbase.ipc.ServerNotRunningException: Server is not running yet"));
|
||||
assertTrue(ex.getMessage().startsWith(
|
||||
"org.apache.hadoop.hbase.ipc.ServerNotRunningYetException: Server is not running yet"));
|
||||
} catch (Throwable t) {
|
||||
fail("Unexpected throwable: " + t);
|
||||
}
|
||||
|
|
|
@ -20,6 +20,8 @@
|
|||
package org.apache.hadoop.hbase.master;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.NavigableSet;
|
||||
import java.util.Set;
|
||||
|
@ -323,16 +325,18 @@ public class TestRollingRestart {
|
|||
LOG.debug("\n\nTRR: " + msg + "\n");
|
||||
}
|
||||
|
||||
private RegionServerThread getServerHostingMeta(MiniHBaseCluster cluster) {
|
||||
private RegionServerThread getServerHostingMeta(MiniHBaseCluster cluster)
|
||||
throws IOException {
|
||||
return getServerHosting(cluster, HRegionInfo.FIRST_META_REGIONINFO);
|
||||
}
|
||||
|
||||
private RegionServerThread getServerHostingRoot(MiniHBaseCluster cluster) {
|
||||
private RegionServerThread getServerHostingRoot(MiniHBaseCluster cluster)
|
||||
throws IOException {
|
||||
return getServerHosting(cluster, HRegionInfo.ROOT_REGIONINFO);
|
||||
}
|
||||
|
||||
private RegionServerThread getServerHosting(MiniHBaseCluster cluster,
|
||||
HRegionInfo region) {
|
||||
HRegionInfo region) throws IOException {
|
||||
for (RegionServerThread rst : cluster.getRegionServerThreads()) {
|
||||
if (rst.getRegionServer().getOnlineRegions().contains(region)) {
|
||||
return rst;
|
||||
|
@ -342,7 +346,7 @@ public class TestRollingRestart {
|
|||
}
|
||||
|
||||
private void assertRegionsAssigned(MiniHBaseCluster cluster,
|
||||
Set<String> expectedRegions) {
|
||||
Set<String> expectedRegions) throws IOException {
|
||||
int numFound = 0;
|
||||
for (RegionServerThread rst : cluster.getLiveRegionServerThreads()) {
|
||||
numFound += rst.getRegionServer().getNumberOfOnlineRegions();
|
||||
|
@ -371,7 +375,8 @@ public class TestRollingRestart {
|
|||
}
|
||||
}
|
||||
|
||||
private NavigableSet<String> getAllOnlineRegions(MiniHBaseCluster cluster) {
|
||||
private NavigableSet<String> getAllOnlineRegions(MiniHBaseCluster cluster)
|
||||
throws IOException {
|
||||
NavigableSet<String> online = new TreeSet<String>();
|
||||
for (RegionServerThread rst : cluster.getLiveRegionServerThreads()) {
|
||||
for (HRegionInfo region : rst.getRegionServer().getOnlineRegions()) {
|
||||
|
@ -382,7 +387,7 @@ public class TestRollingRestart {
|
|||
}
|
||||
|
||||
private NavigableSet<String> getDoubleAssignedRegions(
|
||||
MiniHBaseCluster cluster) {
|
||||
MiniHBaseCluster cluster) throws IOException {
|
||||
NavigableSet<String> online = new TreeSet<String>();
|
||||
NavigableSet<String> doubled = new TreeSet<String>();
|
||||
for (RegionServerThread rst : cluster.getLiveRegionServerThreads()) {
|
||||
|
|
|
@ -64,13 +64,18 @@ public class TestSplitTransactionOnCluster {
|
|||
LogFactory.getLog(TestSplitTransactionOnCluster.class);
|
||||
private HBaseAdmin admin = null;
|
||||
private MiniHBaseCluster cluster = null;
|
||||
private static final int NB_SERVERS = 2;
|
||||
|
||||
private static final HBaseTestingUtility TESTING_UTIL =
|
||||
new HBaseTestingUtility();
|
||||
|
||||
@BeforeClass public static void before() throws Exception {
|
||||
TESTING_UTIL.getConfiguration().setInt("hbase.balancer.period", 60000);
|
||||
TESTING_UTIL.startMiniCluster(2);
|
||||
// Needed because some tests have splits happening on RS that are killed
|
||||
// We don't want to wait 3min for the master to figure it out
|
||||
TESTING_UTIL.getConfiguration().setInt(
|
||||
"hbase.master.assignment.timeoutmonitor.timeout", 4000);
|
||||
TESTING_UTIL.startMiniCluster(NB_SERVERS);
|
||||
}
|
||||
|
||||
@AfterClass public static void after() throws Exception {
|
||||
|
@ -78,7 +83,7 @@ public class TestSplitTransactionOnCluster {
|
|||
}
|
||||
|
||||
@Before public void setup() throws IOException {
|
||||
TESTING_UTIL.ensureSomeRegionServersAvailable(2);
|
||||
TESTING_UTIL.ensureSomeRegionServersAvailable(NB_SERVERS);
|
||||
this.admin = new HBaseAdmin(TESTING_UTIL.getConfiguration());
|
||||
this.cluster = TESTING_UTIL.getMiniHBaseCluster();
|
||||
}
|
||||
|
@ -144,10 +149,8 @@ public class TestSplitTransactionOnCluster {
|
|||
rtd.getEventType().equals(EventType.RS_ZK_REGION_SPLITTING));
|
||||
// Now crash the server
|
||||
cluster.abortRegionServer(tableRegionIndex);
|
||||
while(server.getOnlineRegions().size() > 0) {
|
||||
LOG.info("Waiting on server to go down");
|
||||
Thread.sleep(100);
|
||||
}
|
||||
waitUntilRegionServerDead();
|
||||
|
||||
// Wait till regions are back on line again.
|
||||
while(cluster.getRegions(tableName).size() < daughters.size()) {
|
||||
LOG.info("Waiting for repair to happen");
|
||||
|
@ -263,10 +266,7 @@ public class TestSplitTransactionOnCluster {
|
|||
removeDaughterFromMeta(daughters.get(0).getRegionName());
|
||||
// Now crash the server
|
||||
cluster.abortRegionServer(tableRegionIndex);
|
||||
while(server.getOnlineRegions().size() > 0) {
|
||||
LOG.info("Waiting on server to go down");
|
||||
Thread.sleep(100);
|
||||
}
|
||||
waitUntilRegionServerDead();
|
||||
// Wait till regions are back on line again.
|
||||
while(cluster.getRegions(tableName).size() < daughters.size()) {
|
||||
LOG.info("Waiting for repair to happen");
|
||||
|
@ -339,10 +339,7 @@ public class TestSplitTransactionOnCluster {
|
|||
daughters = cluster.getRegions(tableName);
|
||||
// Now crash the server
|
||||
cluster.abortRegionServer(tableRegionIndex);
|
||||
while(server.getOnlineRegions().size() > 0) {
|
||||
LOG.info("Waiting on server to go down");
|
||||
Thread.sleep(100);
|
||||
}
|
||||
waitUntilRegionServerDead();
|
||||
// Wait till regions are back on line again.
|
||||
while(cluster.getRegions(tableName).size() < daughters.size()) {
|
||||
LOG.info("Waiting for repair to happen");
|
||||
|
@ -445,10 +442,20 @@ public class TestSplitTransactionOnCluster {
|
|||
return null;
|
||||
}
|
||||
|
||||
private void printOutRegions(final HRegionServer hrs, final String prefix) {
|
||||
private void printOutRegions(final HRegionServer hrs, final String prefix)
|
||||
throws IOException {
|
||||
List<HRegionInfo> regions = hrs.getOnlineRegions();
|
||||
for (HRegionInfo region: regions) {
|
||||
LOG.info(prefix + region.getRegionNameAsString());
|
||||
}
|
||||
}
|
||||
|
||||
private void waitUntilRegionServerDead() throws InterruptedException {
|
||||
// Wait until the master processes the RS shutdown
|
||||
while (cluster.getMaster().getClusterStatus().
|
||||
getServers().size() == NB_SERVERS) {
|
||||
LOG.info("Waiting on server to go down");
|
||||
Thread.sleep(100);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue