HBASE-18946 Stochastic load balancer assigns replica regions to the same RS
Added new bulk assign createRoundRobinAssignProcedure to complement the existing createAssignProcedure. The former asks the balancer for target servers to set into the created AssignProcedures. The latter sets no target server into AssignProcedure. When no target server is specified, we make effort at assign-time at trying to deploy the region to its old location if there was one. The new round robin assign procedure creator does not do this. Use the new round robin method on table create or reenabling offline regions. Use the old assign in ServerCrashProcedure or in EnableTable so there is a chance we retain locality. Bulk preassigning passing all to-be-assigned to the balancer in one go is good for ensuring good distribution especially when read replicas in the mix. The old assign was single-assign scoped so region replicas could end up on the same server. M hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java Cleanup around forceNewPlan. Was confusing. Added a Comparator to sort AssignProcedures so meta and system tables come ahead of user-space tables. M hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java Remove the forceNewPlan argument on createAssignProcedure. Didn't make sense given we were creating a new AssignProcedure; the arg had no effect. (createRoundRobinAssignProcedures) Recast to feed all regions to the balancer in bulk and to sort the return so meta and system tables take precedence. Miscellaneous fixes including keeping the Master around until all RegionServers are down, documentation on how assignment retention works, etc.
This commit is contained in:
parent
11e82de8a1
commit
010012cbcb
|
@ -880,6 +880,9 @@ public class ProcedureExecutor<TEnvironment> {
|
|||
public void submitProcedures(final Procedure[] procs) {
|
||||
Preconditions.checkArgument(lastProcId.get() >= 0);
|
||||
Preconditions.checkArgument(isRunning(), "executor not running");
|
||||
if (procs == null || procs.length <= 0) {
|
||||
return;
|
||||
}
|
||||
|
||||
// Prepare procedure
|
||||
for (int i = 0; i < procs.length; ++i) {
|
||||
|
|
|
@ -198,8 +198,8 @@ public class ZKNamespaceManager extends ZKListener {
|
|||
String path = n.getNode();
|
||||
String namespace = ZKUtil.getNodeName(path);
|
||||
byte[] nodeData = n.getData();
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Updating namespace cache from node "+namespace+" with data: "+
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Updating namespace cache from node " + namespace + " with data: " +
|
||||
Bytes.toStringBinary(nodeData));
|
||||
}
|
||||
NamespaceDescriptor ns =
|
||||
|
|
|
@ -178,6 +178,7 @@ public class ExecutorService {
|
|||
// name the threads for this threadpool
|
||||
ThreadFactoryBuilder tfb = new ThreadFactoryBuilder();
|
||||
tfb.setNameFormat(this.name + "-%d");
|
||||
// TODO: Should these executors be daemon threads?
|
||||
this.threadPoolExecutor.setThreadFactory(tfb.build());
|
||||
}
|
||||
|
||||
|
|
|
@ -761,10 +761,7 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
|
||||
/*
|
||||
* We are active master now... go initialize components we need to run.
|
||||
* Note, there may be dross in zk from previous runs; it'll get addressed
|
||||
* below after we determine if cluster startup or failover.
|
||||
*/
|
||||
|
||||
status.setStatus("Initializing Master file system");
|
||||
|
||||
this.masterActiveTime = System.currentTimeMillis();
|
||||
|
@ -1173,12 +1170,6 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
super.stopServiceThreads();
|
||||
stopChores();
|
||||
|
||||
// Wait for all the remaining region servers to report in IFF we were
|
||||
// running a cluster shutdown AND we were NOT aborting.
|
||||
if (!isAborted() && this.serverManager != null &&
|
||||
this.serverManager.isClusterShutdown()) {
|
||||
this.serverManager.letRegionServersShutdown();
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Stopping service threads");
|
||||
}
|
||||
|
@ -3516,10 +3507,10 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
@Override
|
||||
public boolean recoverMeta() throws IOException {
|
||||
ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch(2, 0);
|
||||
LOG.info("Running RecoverMetaProcedure to ensure proper hbase:meta deploy.");
|
||||
long procId = procedureExecutor.submitProcedure(new RecoverMetaProcedure(null, true, latch));
|
||||
LOG.info("Waiting on RecoverMetaProcedure submitted with procId=" + procId);
|
||||
latch.await();
|
||||
LOG.info("Default replica of hbase:meta, location=" +
|
||||
LOG.info("hbase:meta (default replica) deployed at=" +
|
||||
getMetaTableLocator().getMetaRegionLocation(getZooKeeper()));
|
||||
return assignmentManager.isMetaInitialized();
|
||||
}
|
||||
|
@ -3531,4 +3522,4 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
public SpaceQuotaSnapshotNotifier getSpaceQuotaSnapshotNotifier() {
|
||||
return this.spaceQuotaSnapshotNotifier;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -538,7 +538,7 @@ public class MasterRpcServices extends RSRpcServices
|
|||
master.cpHost.preAssign(regionInfo);
|
||||
}
|
||||
LOG.info(master.getClientIdAuditPrefix() + " assign " + regionInfo.getRegionNameAsString());
|
||||
master.getAssignmentManager().assign(regionInfo, true);
|
||||
master.getAssignmentManager().assign(regionInfo);
|
||||
if (master.cpHost != null) {
|
||||
master.cpHost.postAssign(regionInfo);
|
||||
}
|
||||
|
|
|
@ -321,8 +321,7 @@ public class ServerManager {
|
|||
* @param sl the server load on the server
|
||||
* @return true if the server is recorded, otherwise, false
|
||||
*/
|
||||
boolean checkAndRecordNewServer(
|
||||
final ServerName serverName, final ServerLoad sl) {
|
||||
boolean checkAndRecordNewServer(final ServerName serverName, final ServerLoad sl) {
|
||||
ServerName existingServer = null;
|
||||
synchronized (this.onlineServers) {
|
||||
existingServer = findServerWithSameHostnamePortWithLock(serverName);
|
||||
|
@ -343,7 +342,8 @@ public class ServerManager {
|
|||
|
||||
// Note that we assume that same ts means same server, and don't expire in that case.
|
||||
// TODO: ts can theoretically collide due to clock shifts, so this is a bit hacky.
|
||||
if (existingServer != null && (existingServer.getStartcode() < serverName.getStartcode())) {
|
||||
if (existingServer != null &&
|
||||
(existingServer.getStartcode() < serverName.getStartcode())) {
|
||||
LOG.info("Triggering server recovery; existingServer " +
|
||||
existingServer + " looks stale, new server:" + serverName);
|
||||
expireServer(existingServer);
|
||||
|
@ -659,7 +659,9 @@ public class ServerManager {
|
|||
}
|
||||
|
||||
if (!master.getAssignmentManager().isFailoverCleanupDone()) {
|
||||
LOG.debug("AssignmentManager failover cleanup not done.");
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("AssignmentManager failover cleanup not done.");
|
||||
}
|
||||
}
|
||||
|
||||
for (Map.Entry<ServerName, Boolean> entry : requeuedDeadServers.entrySet()) {
|
||||
|
@ -857,7 +859,7 @@ public class ServerManager {
|
|||
for (ServerListener listener: this.listeners) {
|
||||
listener.waiting();
|
||||
}
|
||||
while (!this.master.isStopped() && count < maxToStart &&
|
||||
while (!this.master.isStopped() && !isClusterShutdown() && count < maxToStart &&
|
||||
((lastCountChange + interval) > now || timeout > slept || count < minToStart)) {
|
||||
// Log some info at every interval time or if there is a change
|
||||
if (oldCount != count || lastLogTime + interval < now) {
|
||||
|
@ -882,6 +884,10 @@ public class ServerManager {
|
|||
lastCountChange = now;
|
||||
}
|
||||
}
|
||||
// Did we exit the loop because cluster is going down?
|
||||
if (isClusterShutdown()) {
|
||||
this.master.stop("Cluster shutdown");
|
||||
}
|
||||
LOG.info("Finished wait on RegionServer count=" + count + "; waited=" + slept + "ms," +
|
||||
" expected min=" + minToStart + " server(s), max=" + getStrForMax(maxToStart) + " server(s),"+
|
||||
" master is "+ (this.master.isStopped() ? "stopped.": "running"));
|
||||
|
@ -955,7 +961,6 @@ public class ServerManager {
|
|||
String statusStr = "Cluster shutdown requested of master=" + this.master.getServerName();
|
||||
LOG.info(statusStr);
|
||||
this.clusterShutdown.set(true);
|
||||
this.master.stop(statusStr);
|
||||
}
|
||||
|
||||
public boolean isClusterShutdown() {
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
/**
|
||||
/*
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
|
@ -20,6 +20,7 @@
|
|||
package org.apache.hadoop.hbase.master.assignment;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Comparator;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
@ -73,6 +74,9 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto
|
|||
public class AssignProcedure extends RegionTransitionProcedure {
|
||||
private static final Log LOG = LogFactory.getLog(AssignProcedure.class);
|
||||
|
||||
/**
|
||||
* Set to true when we need recalibrate -- choose a new target -- because original assign failed.
|
||||
*/
|
||||
private boolean forceNewPlan = false;
|
||||
|
||||
/**
|
||||
|
@ -84,24 +88,24 @@ public class AssignProcedure extends RegionTransitionProcedure {
|
|||
*/
|
||||
protected volatile ServerName targetServer;
|
||||
|
||||
/**
|
||||
* Comparator that will sort AssignProcedures so meta assigns come first, then system table
|
||||
* assigns and finally user space assigns.
|
||||
*/
|
||||
public static final CompareAssignProcedure COMPARATOR = new CompareAssignProcedure();
|
||||
|
||||
public AssignProcedure() {
|
||||
// Required by the Procedure framework to create the procedure on replay
|
||||
super();
|
||||
}
|
||||
|
||||
public AssignProcedure(final RegionInfo regionInfo) {
|
||||
this(regionInfo, false);
|
||||
}
|
||||
|
||||
public AssignProcedure(final RegionInfo regionInfo, final boolean forceNewPlan) {
|
||||
super(regionInfo);
|
||||
this.forceNewPlan = forceNewPlan;
|
||||
this.targetServer = null;
|
||||
}
|
||||
|
||||
public AssignProcedure(final RegionInfo regionInfo, final ServerName destinationServer) {
|
||||
super(regionInfo);
|
||||
this.forceNewPlan = false;
|
||||
this.targetServer = destinationServer;
|
||||
}
|
||||
|
||||
|
@ -361,4 +365,32 @@ public class AssignProcedure extends RegionTransitionProcedure {
|
|||
protected ProcedureMetrics getProcedureMetrics(MasterProcedureEnv env) {
|
||||
return env.getAssignmentManager().getAssignmentManagerMetrics().getAssignProcMetrics();
|
||||
}
|
||||
|
||||
/**
|
||||
* Sort AssignProcedures such that meta and system assigns come first before user-space assigns.
|
||||
* Have to do it this way w/ distinct Comparator because Procedure is already Comparable on
|
||||
* 'Env'(?).
|
||||
*/
|
||||
public static class CompareAssignProcedure implements Comparator<AssignProcedure> {
|
||||
@Override
|
||||
public int compare(AssignProcedure left, AssignProcedure right) {
|
||||
if (left.getRegionInfo().isMetaRegion()) {
|
||||
if (right.getRegionInfo().isMetaRegion()) {
|
||||
return RegionInfo.COMPARATOR.compare(left.getRegionInfo(), right.getRegionInfo());
|
||||
}
|
||||
return -1;
|
||||
} else if (left.getRegionInfo().isMetaRegion()) {
|
||||
return +1;
|
||||
}
|
||||
if (left.getRegionInfo().getTable().isSystemTable()) {
|
||||
if (right.getRegionInfo().getTable().isSystemTable()) {
|
||||
return RegionInfo.COMPARATOR.compare(left.getRegionInfo(), right.getRegionInfo());
|
||||
}
|
||||
return -1;
|
||||
} else if (left.getRegionInfo().getTable().isSystemTable()) {
|
||||
return +1;
|
||||
}
|
||||
return RegionInfo.COMPARATOR.compare(left.getRegionInfo(), right.getRegionInfo());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
/**
|
||||
/*
|
||||
* 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
|
||||
|
@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.master.assignment;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
|
@ -464,7 +465,7 @@ public class AssignmentManager implements ServerListener {
|
|||
proc = createAssignProcedure(metaRegionInfo, serverName);
|
||||
} else {
|
||||
LOG.debug("Assigning " + metaRegionInfo.getRegionNameAsString());
|
||||
proc = createAssignProcedure(metaRegionInfo, false);
|
||||
proc = createAssignProcedure(metaRegionInfo);
|
||||
}
|
||||
ProcedureSyncWait.submitAndWaitProcedure(master.getMasterProcedureExecutor(), proc);
|
||||
}
|
||||
|
@ -526,11 +527,7 @@ public class AssignmentManager implements ServerListener {
|
|||
}
|
||||
|
||||
public void assign(final RegionInfo regionInfo) throws IOException {
|
||||
assign(regionInfo, true);
|
||||
}
|
||||
|
||||
public void assign(final RegionInfo regionInfo, final boolean forceNewPlan) throws IOException {
|
||||
AssignProcedure proc = createAssignProcedure(regionInfo, forceNewPlan);
|
||||
AssignProcedure proc = createAssignProcedure(regionInfo);
|
||||
ProcedureSyncWait.submitAndWaitProcedure(master.getMasterProcedureExecutor(), proc);
|
||||
}
|
||||
|
||||
|
@ -605,23 +602,84 @@ public class AssignmentManager implements ServerListener {
|
|||
// RegionTransition procedures helpers
|
||||
// ============================================================================================
|
||||
|
||||
public AssignProcedure[] createAssignProcedures(final Collection<RegionInfo> regionInfo) {
|
||||
return createAssignProcedures(regionInfo, false);
|
||||
/**
|
||||
* Create round-robin assigns. Use on table creation to distribute out regions across cluster.
|
||||
* @return AssignProcedures made out of the passed in <code>hris</code> and a call
|
||||
* to the balancer to populate the assigns with targets chosen using round-robin (default
|
||||
* balancer scheme). If at assign-time, the target chosen is no longer up, thats fine,
|
||||
* the AssignProcedure will ask the balancer for a new target, and so on.
|
||||
*/
|
||||
public AssignProcedure[] createRoundRobinAssignProcedures(final List<RegionInfo> hris) {
|
||||
if (hris.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
try {
|
||||
// Ask the balancer to assign our regions. Pass the regions en masse. The balancer can do
|
||||
// a better job if it has all the assignments in the one lump.
|
||||
Map<ServerName, List<RegionInfo>> assignments = getBalancer().roundRobinAssignment(hris,
|
||||
this.master.getServerManager().createDestinationServersList(null));
|
||||
// Return mid-method!
|
||||
return createAssignProcedures(assignments, hris.size());
|
||||
} catch (HBaseIOException hioe) {
|
||||
LOG.warn("Failed roundRobinAssignment", hioe);
|
||||
}
|
||||
// If an error above, fall-through to this simpler assign. Last resort.
|
||||
return createAssignProcedures(hris);
|
||||
}
|
||||
|
||||
public AssignProcedure[] createAssignProcedures(final Collection<RegionInfo> regionInfo,
|
||||
final boolean forceNewPlan) {
|
||||
if (regionInfo.isEmpty()) return null;
|
||||
final AssignProcedure[] procs = new AssignProcedure[regionInfo.size()];
|
||||
int index = 0;
|
||||
for (RegionInfo hri: regionInfo) {
|
||||
procs[index++] = createAssignProcedure(hri, forceNewPlan);
|
||||
/**
|
||||
* Create an array of AssignProcedures w/o specifying a target server.
|
||||
* If no target server, at assign time, we will try to use the former location of the region
|
||||
* if one exists. This is how we 'retain' the old location across a server restart.
|
||||
* Used by {@link ServerCrashProcedure} assigning regions on a server that has crashed (SCP is
|
||||
* also used across a cluster-restart just-in-case to ensure we do cleanup of any old WALs or
|
||||
* server processes).
|
||||
*/
|
||||
public AssignProcedure[] createAssignProcedures(final List<RegionInfo> hris) {
|
||||
if (hris.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
return procs;
|
||||
int index = 0;
|
||||
AssignProcedure [] procedures = new AssignProcedure[hris.size()];
|
||||
for (RegionInfo hri : hris) {
|
||||
// Sort the procedures so meta and system regions are first in the returned array.
|
||||
procedures[index++] = createAssignProcedure(hri);
|
||||
}
|
||||
if (procedures.length > 1) {
|
||||
// Sort the procedures so meta and system regions are first in the returned array.
|
||||
Arrays.sort(procedures, AssignProcedure.COMPARATOR);
|
||||
}
|
||||
return procedures;
|
||||
}
|
||||
|
||||
// Make this static for the method below where we use it typing the AssignProcedure array we
|
||||
// return as result.
|
||||
private static final AssignProcedure [] ASSIGN_PROCEDURE_ARRAY_TYPE = new AssignProcedure[] {};
|
||||
|
||||
/**
|
||||
* @param assignments Map of assignments from which we produce an array of AssignProcedures.
|
||||
* @param size Count of assignments to make (the caller may know the total count)
|
||||
* @return Assignments made from the passed in <code>assignments</code>
|
||||
*/
|
||||
private AssignProcedure[] createAssignProcedures(Map<ServerName, List<RegionInfo>> assignments,
|
||||
int size) {
|
||||
List<AssignProcedure> procedures = new ArrayList<>(size > 0? size: 8/*Arbitrary*/);
|
||||
for (Map.Entry<ServerName, List<RegionInfo>> e: assignments.entrySet()) {
|
||||
for (RegionInfo ri: e.getValue()) {
|
||||
AssignProcedure ap = createAssignProcedure(ri, e.getKey());
|
||||
ap.setOwner(getProcedureEnvironment().getRequestUser().getShortName());
|
||||
procedures.add(ap);
|
||||
}
|
||||
}
|
||||
if (procedures.size() > 1) {
|
||||
// Sort the procedures so meta and system regions are first in the returned array.
|
||||
procedures.sort(AssignProcedure.COMPARATOR);
|
||||
}
|
||||
return procedures.toArray(ASSIGN_PROCEDURE_ARRAY_TYPE);
|
||||
}
|
||||
|
||||
// Needed for the following method so it can type the created Array we return
|
||||
private static final UnassignProcedure [] UNASSIGNED_PROCEDURE_FOR_TYPE_INFO =
|
||||
private static final UnassignProcedure [] UNASSIGN_PROCEDURE_ARRAY_TYPE =
|
||||
new UnassignProcedure[0];
|
||||
|
||||
UnassignProcedure[] createUnassignProcedures(final Collection<RegionStateNode> nodes) {
|
||||
|
@ -634,7 +692,7 @@ public class AssignmentManager implements ServerListener {
|
|||
assert node.getRegionLocation() != null: node.toString();
|
||||
procs.add(createUnassignProcedure(node.getRegionInfo(), node.getRegionLocation(), false));
|
||||
}
|
||||
return procs.toArray(UNASSIGNED_PROCEDURE_FOR_TYPE_INFO);
|
||||
return procs.toArray(UNASSIGN_PROCEDURE_ARRAY_TYPE);
|
||||
}
|
||||
|
||||
public MoveRegionProcedure[] createReopenProcedures(final Collection<RegionInfo> regionInfo) {
|
||||
|
@ -648,14 +706,6 @@ public class AssignmentManager implements ServerListener {
|
|||
return procs;
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by things like EnableTableProcedure to get a list of AssignProcedure
|
||||
* to assign the regions of the table.
|
||||
*/
|
||||
public AssignProcedure[] createAssignProcedures(final TableName tableName) {
|
||||
return createAssignProcedures(regionStates.getRegionsOfTable(tableName));
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by things like DisableTableProcedure to get a list of UnassignProcedure
|
||||
* to unassign the regions of the table.
|
||||
|
@ -664,17 +714,8 @@ public class AssignmentManager implements ServerListener {
|
|||
return createUnassignProcedures(regionStates.getTableRegionStateNodes(tableName));
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by things like ModifyColumnFamilyProcedure to get a list of MoveRegionProcedure
|
||||
* to reopen the regions of the table.
|
||||
*/
|
||||
public MoveRegionProcedure[] createReopenProcedures(final TableName tableName) {
|
||||
return createReopenProcedures(regionStates.getRegionsOfTable(tableName));
|
||||
}
|
||||
|
||||
public AssignProcedure createAssignProcedure(final RegionInfo regionInfo,
|
||||
final boolean forceNewPlan) {
|
||||
AssignProcedure proc = new AssignProcedure(regionInfo, forceNewPlan);
|
||||
public AssignProcedure createAssignProcedure(final RegionInfo regionInfo) {
|
||||
AssignProcedure proc = new AssignProcedure(regionInfo);
|
||||
proc.setOwner(getProcedureEnvironment().getRequestUser().getShortName());
|
||||
return proc;
|
||||
}
|
||||
|
@ -686,7 +727,7 @@ public class AssignmentManager implements ServerListener {
|
|||
return proc;
|
||||
}
|
||||
|
||||
public UnassignProcedure createUnassignProcedure(final RegionInfo regionInfo,
|
||||
UnassignProcedure createUnassignProcedure(final RegionInfo regionInfo,
|
||||
final ServerName destinationServer, final boolean force) {
|
||||
// If destinationServer is null, figure it.
|
||||
ServerName sn = destinationServer != null? destinationServer:
|
||||
|
@ -954,7 +995,8 @@ public class AssignmentManager implements ServerListener {
|
|||
final RegionStateNode regionNode = regionStates.getOrCreateRegionStateNode(hri);
|
||||
LOG.info("META REPORTED: " + regionNode);
|
||||
if (!reportTransition(regionNode, serverNode, TransitionCode.OPENED, 0)) {
|
||||
LOG.warn("META REPORTED but no procedure found (complete?)");
|
||||
LOG.warn("META REPORTED but no procedure found (complete?); set location=" +
|
||||
serverNode.getServerName());
|
||||
regionNode.setRegionLocation(serverNode.getServerName());
|
||||
} else if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("META REPORTED: " + regionNode);
|
||||
|
@ -1157,19 +1199,18 @@ public class AssignmentManager implements ServerListener {
|
|||
public void joinCluster() throws IOException {
|
||||
final long startTime = System.currentTimeMillis();
|
||||
|
||||
LOG.info("Joining the cluster...");
|
||||
LOG.info("Joining cluster...Loading hbase:meta content.");
|
||||
|
||||
// Scan hbase:meta to build list of existing regions, servers, and assignment
|
||||
loadMeta();
|
||||
|
||||
for (int i = 0; master.getServerManager().countOfRegionServers() < 1; ++i) {
|
||||
LOG.info("waiting for RS to join");
|
||||
LOG.info("Waiting for RegionServers to join; current count=" +
|
||||
master.getServerManager().countOfRegionServers());
|
||||
Threads.sleep(250);
|
||||
}
|
||||
LOG.info("RS joined. Num RS = " + master.getServerManager().countOfRegionServers());
|
||||
LOG.info("Number of RegionServers=" + master.getServerManager().countOfRegionServers());
|
||||
|
||||
// This method will assign all user regions if a clean server startup or
|
||||
// it will reconstruct master state and cleanup any leftovers from previous master process.
|
||||
boolean failover = processofflineServersWithOnlineRegions();
|
||||
|
||||
// Start the RIT chore
|
||||
|
@ -1220,54 +1261,74 @@ public class AssignmentManager implements ServerListener {
|
|||
wakeMetaLoadedEvent();
|
||||
}
|
||||
|
||||
// TODO: the assumption here is that if RSs are crashing while we are executing this
|
||||
// they will be handled by the SSH that are put in the ServerManager "queue".
|
||||
// we can integrate this a bit better.
|
||||
/**
|
||||
* Look at what is in meta and the list of servers that have checked in and make reconciliation.
|
||||
* We cannot tell definitively the difference between a clean shutdown and a cluster that has
|
||||
* been crashed down. At this stage of a Master startup, they look the same: they have the
|
||||
* same state in hbase:meta. We could do detective work probing ZK and the FS for old WALs to
|
||||
* split but SCP does this already so just let it do its job.
|
||||
* <p>>The profiles of clean shutdown and cluster crash-down are the same because on clean
|
||||
* shutdown currently, we do not update hbase:meta with region close state (In AMv2, region
|
||||
* state is kept in hbse:meta). Usually the master runs all region transitions as of AMv2 but on
|
||||
* cluster controlled shutdown, the RegionServers close all their regions only reporting the
|
||||
* final change to the Master. Currently this report is ignored. Later we could take it and
|
||||
* update as many regions as we can before hbase:meta goes down or have the master run the
|
||||
* close of all regions out on the cluster but we may never be able to achieve the proper state on
|
||||
* all regions (at least not w/o lots of painful manipulations and waiting) so clean shutdown
|
||||
* might not be possible especially on big clusters.... And clean shutdown will take time. Given
|
||||
* this current state of affairs, we just run ServerCrashProcedure in both cases. It will always
|
||||
* do the right thing.
|
||||
* @return True if for sure this is a failover where a Master is starting up into an already
|
||||
* running cluster.
|
||||
*/
|
||||
// The assumption here is that if RSs are crashing while we are executing this
|
||||
// they will be handled by the SSH that are put in the ServerManager deadservers "queue".
|
||||
private boolean processofflineServersWithOnlineRegions() {
|
||||
boolean failover = !master.getServerManager().getDeadServers().isEmpty();
|
||||
|
||||
final Set<ServerName> offlineServersWithOnlineRegions = new HashSet<ServerName>();
|
||||
final ArrayList<RegionInfo> regionsToAssign = new ArrayList<RegionInfo>();
|
||||
long st, et;
|
||||
|
||||
st = System.currentTimeMillis();
|
||||
boolean deadServers = !master.getServerManager().getDeadServers().isEmpty();
|
||||
final Set<ServerName> offlineServersWithOnlineRegions = new HashSet<>();
|
||||
int size = regionStates.getRegionStateNodes().size();
|
||||
final List<RegionInfo> offlineRegionsToAssign = new ArrayList<>(size);
|
||||
long startTime = System.currentTimeMillis();
|
||||
// If deadservers then its a failover, else, we are not sure yet.
|
||||
boolean failover = deadServers;
|
||||
for (RegionStateNode regionNode: regionStates.getRegionStateNodes()) {
|
||||
// Region State can be OPEN even if we did controlled cluster shutdown; Master does not close
|
||||
// the regions in this case. The RegionServer does the close so hbase:meta is state in
|
||||
// hbase:meta is not updated -- Master does all updates -- and is left with OPEN as region
|
||||
// state in meta. How to tell difference between ordered shutdown and crashed-down cluster
|
||||
// then? We can't. Not currently. Perhaps if we updated hbase:meta with CLOSED on ordered
|
||||
// shutdown. This would slow shutdown though and not all edits would make it in anyways.
|
||||
// TODO: Examine.
|
||||
// Because we can't be sure it an ordered shutdown, we run ServerCrashProcedure always.
|
||||
// ServerCrashProcedure will try to retain old deploy when it goes to assign.
|
||||
if (regionNode.getState() == State.OPEN) {
|
||||
final ServerName serverName = regionNode.getRegionLocation();
|
||||
if (!master.getServerManager().isServerOnline(serverName)) {
|
||||
offlineServersWithOnlineRegions.add(serverName);
|
||||
} else {
|
||||
// Server is online. This a failover. Master is starting into already-running cluster.
|
||||
failover = true;
|
||||
}
|
||||
} else if (regionNode.getState() == State.OFFLINE) {
|
||||
if (isTableEnabled(regionNode.getTable())) {
|
||||
regionsToAssign.add(regionNode.getRegionInfo());
|
||||
offlineRegionsToAssign.add(regionNode.getRegionInfo());
|
||||
}
|
||||
}
|
||||
}
|
||||
et = System.currentTimeMillis();
|
||||
LOG.info("[STEP-1] " + StringUtils.humanTimeDiff(et - st));
|
||||
|
||||
// kill servers with online regions
|
||||
st = System.currentTimeMillis();
|
||||
// Kill servers with online regions just-in-case. Runs ServerCrashProcedure.
|
||||
for (ServerName serverName: offlineServersWithOnlineRegions) {
|
||||
if (!master.getServerManager().isServerOnline(serverName)) {
|
||||
LOG.info("KILL RS hosting regions but not online " + serverName +
|
||||
" (master=" + master.getServerName() + ")");
|
||||
LOG.info("KILL RegionServer=" + serverName + " hosting regions but not online.");
|
||||
killRegionServer(serverName);
|
||||
}
|
||||
}
|
||||
et = System.currentTimeMillis();
|
||||
LOG.info("[STEP-2] " + StringUtils.humanTimeDiff(et - st));
|
||||
|
||||
setFailoverCleanupDone(true);
|
||||
|
||||
// assign offline regions
|
||||
st = System.currentTimeMillis();
|
||||
for (RegionInfo regionInfo: getOrderedRegions(regionsToAssign)) {
|
||||
master.getMasterProcedureExecutor().submitProcedure(
|
||||
createAssignProcedure(regionInfo, false));
|
||||
// Assign offline regions. Uses round-robin.
|
||||
if (offlineRegionsToAssign.size() > 0) {
|
||||
master.getMasterProcedureExecutor().submitProcedures(master.getAssignmentManager().
|
||||
createRoundRobinAssignProcedures(offlineRegionsToAssign));
|
||||
}
|
||||
et = System.currentTimeMillis();
|
||||
LOG.info("[STEP-3] " + StringUtils.humanTimeDiff(et - st));
|
||||
|
||||
return failover;
|
||||
}
|
||||
|
@ -1369,27 +1430,6 @@ public class AssignmentManager implements ServerListener {
|
|||
return new Pair<Integer, Integer>(ritCount, states.size());
|
||||
}
|
||||
|
||||
/**
|
||||
* Used when assign regions, this method will put system regions in
|
||||
* front of user regions
|
||||
* @param regions
|
||||
* @return A list of regions with system regions at front
|
||||
*/
|
||||
public List<RegionInfo> getOrderedRegions(
|
||||
final List<RegionInfo> regions) {
|
||||
if (regions == null) return Collections.emptyList();
|
||||
|
||||
List<RegionInfo> systemList = new ArrayList<>();
|
||||
List<RegionInfo> userList = new ArrayList<>();
|
||||
for (RegionInfo hri : regions) {
|
||||
if (hri.getTable().isSystemTable()) systemList.add(hri);
|
||||
else userList.add(hri);
|
||||
}
|
||||
// Append userList to systemList
|
||||
systemList.addAll(userList);
|
||||
return systemList;
|
||||
}
|
||||
|
||||
// ============================================================================================
|
||||
// TODO: Region State In Transition
|
||||
// ============================================================================================
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
/**
|
||||
/*
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
|
@ -71,7 +71,7 @@ public class MoveRegionProcedure extends AbstractStateMachineRegionProcedure<Mov
|
|||
break;
|
||||
case MOVE_REGION_ASSIGN:
|
||||
AssignProcedure assignProcedure = plan.getDestination() == null ?
|
||||
new AssignProcedure(plan.getRegionInfo(), true) :
|
||||
new AssignProcedure(plan.getRegionInfo()):
|
||||
new AssignProcedure(plan.getRegionInfo(), plan.getDestination());
|
||||
addChildProcedure(assignProcedure);
|
||||
return Flow.NO_MORE_STATE;
|
||||
|
|
|
@ -35,11 +35,13 @@ import org.apache.hadoop.hbase.procedure2.Procedure;
|
|||
import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
|
||||
import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
|
||||
import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionTransitionState;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
|
||||
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* Base class for the Assign and Unassign Procedure.
|
||||
*
|
||||
|
@ -102,7 +104,8 @@ public abstract class RegionTransitionProcedure
|
|||
this.regionInfo = regionInfo;
|
||||
}
|
||||
|
||||
protected RegionInfo getRegionInfo() {
|
||||
@VisibleForTesting
|
||||
public RegionInfo getRegionInfo() {
|
||||
return regionInfo;
|
||||
}
|
||||
|
||||
|
|
|
@ -150,7 +150,7 @@ public class CloneSnapshotProcedure
|
|||
break;
|
||||
case CLONE_SNAPSHOT_ASSIGN_REGIONS:
|
||||
CreateTableProcedure.setEnablingState(env, getTableName());
|
||||
addChildProcedure(env.getAssignmentManager().createAssignProcedures(newRegions));
|
||||
addChildProcedure(env.getAssignmentManager().createRoundRobinAssignProcedures(newRegions));
|
||||
setNextState(CloneSnapshotState.CLONE_SNAPSHOT_UPDATE_DESC_CACHE);
|
||||
break;
|
||||
case CLONE_SNAPSHOT_UPDATE_DESC_CACHE:
|
||||
|
|
|
@ -106,7 +106,7 @@ public class CreateTableProcedure
|
|||
break;
|
||||
case CREATE_TABLE_ASSIGN_REGIONS:
|
||||
setEnablingState(env, getTableName());
|
||||
addChildProcedure(env.getAssignmentManager().createAssignProcedures(newRegions));
|
||||
addChildProcedure(env.getAssignmentManager().createRoundRobinAssignProcedures(newRegions));
|
||||
setNextState(CreateTableState.CREATE_TABLE_UPDATE_DESC_CACHE);
|
||||
break;
|
||||
case CREATE_TABLE_UPDATE_DESC_CACHE:
|
||||
|
|
|
@ -175,7 +175,8 @@ public class EnableTableProcedure
|
|||
currentMaxReplica + 1, regionReplicaCount);
|
||||
}
|
||||
}
|
||||
// Assign all the table regions. (including region replicas if added)
|
||||
// Assign all the table regions. (including region replicas if added).
|
||||
// createAssignProcedure will try to retain old assignments if possible.
|
||||
addChildProcedure(env.getAssignmentManager().createAssignProcedures(regionsOfTable));
|
||||
setNextState(EnableTableState.ENABLE_TABLE_SET_ENABLED_TABLE_STATE);
|
||||
break;
|
||||
|
|
|
@ -129,7 +129,7 @@ public class RecoverMetaProcedure
|
|||
if (failedMetaServer != null) {
|
||||
LOG.info(this + "; Assigning meta with new plan. previous meta server=" +
|
||||
failedMetaServer);
|
||||
metaAssignProcedure = master.getAssignmentManager().createAssignProcedure(hri, true);
|
||||
metaAssignProcedure = master.getAssignmentManager().createAssignProcedure(hri);
|
||||
} else {
|
||||
// get server carrying meta from zk
|
||||
ServerName metaServer =
|
||||
|
|
|
@ -163,10 +163,8 @@ implements ServerProcedureInterface {
|
|||
}
|
||||
handleRIT(env, regionsOnCrashedServer);
|
||||
AssignmentManager am = env.getAssignmentManager();
|
||||
// forceNewPlan is set to false. Balancer is expected to find most suitable target
|
||||
// server if retention is not possible.
|
||||
addChildProcedure(am.
|
||||
createAssignProcedures(am.getOrderedRegions(regionsOnCrashedServer), false));
|
||||
// createAssignProcedure will try to use the old location for the region deploy.
|
||||
addChildProcedure(am.createAssignProcedures(regionsOnCrashedServer));
|
||||
}
|
||||
setNextState(ServerCrashState.SERVER_CRASH_FINISH);
|
||||
break;
|
||||
|
@ -185,6 +183,7 @@ implements ServerProcedureInterface {
|
|||
return Flow.HAS_MORE_STATE;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @param env
|
||||
* @throws IOException
|
||||
|
|
|
@ -129,7 +129,7 @@ public class TruncateTableProcedure
|
|||
break;
|
||||
case TRUNCATE_TABLE_ASSIGN_REGIONS:
|
||||
CreateTableProcedure.setEnablingState(env, getTableName());
|
||||
addChildProcedure(env.getAssignmentManager().createAssignProcedures(regions));
|
||||
addChildProcedure(env.getAssignmentManager().createRoundRobinAssignProcedures(regions));
|
||||
setNextState(TruncateTableState.TRUNCATE_TABLE_POST_OPERATION);
|
||||
tableDescriptor = null;
|
||||
regions = null;
|
||||
|
|
|
@ -2275,7 +2275,10 @@ public class HRegionServer extends HasThread implements
|
|||
ReportRegionStateTransitionRequest request = builder.build();
|
||||
int tries = 0;
|
||||
long pauseTime = INIT_PAUSE_TIME_MS;
|
||||
while (keepLooping()) {
|
||||
// Keep looping till we get an error. We want to send reports even though server is going down.
|
||||
// Only go down if clusterConnection is null. It is set to null almost as last thing as the
|
||||
// HRegionServer does down.
|
||||
while (this.clusterConnection != null && !this.clusterConnection.isClosed()) {
|
||||
RegionServerStatusService.BlockingInterface rss = rssStub;
|
||||
try {
|
||||
if (rss == null) {
|
||||
|
@ -2286,8 +2289,7 @@ public class HRegionServer extends HasThread implements
|
|||
rss.reportRegionStateTransition(null, request);
|
||||
if (response.hasErrorMessage()) {
|
||||
LOG.info("TRANSITION FAILED " + request + ": " + response.getErrorMessage());
|
||||
// NOTE: Return mid-method!!!
|
||||
return false;
|
||||
break;
|
||||
}
|
||||
// Log if we had to retry else don't log unless TRACE. We want to
|
||||
// know if were successful after an attempt showed in logs as failed.
|
||||
|
@ -2319,7 +2321,6 @@ public class HRegionServer extends HasThread implements
|
|||
}
|
||||
}
|
||||
}
|
||||
LOG.info("TRANSITION NOT REPORTED " + request);
|
||||
return false;
|
||||
}
|
||||
|
||||
|
|
|
@ -2408,9 +2408,12 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
} catch (IOException ie) {
|
||||
throw new ServiceException(ie);
|
||||
} finally {
|
||||
if (regionServer.metricsRegionServer != null) {
|
||||
regionServer.metricsRegionServer.updateGet(
|
||||
region.getTableDescriptor().getTableName(), EnvironmentEdgeManager.currentTime() - before);
|
||||
MetricsRegionServer mrs = regionServer.metricsRegionServer;
|
||||
if (mrs != null) {
|
||||
TableDescriptor td = region != null? region.getTableDescriptor(): null;
|
||||
if (td != null) {
|
||||
mrs.updateGet(td.getTableName(), EnvironmentEdgeManager.currentTime() - before);
|
||||
}
|
||||
}
|
||||
if (quota != null) {
|
||||
quota.close();
|
||||
|
|
|
@ -89,6 +89,7 @@ import org.junit.After;
|
|||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TestName;
|
||||
|
@ -273,13 +274,15 @@ public abstract class AbstractTestDLS {
|
|||
|
||||
// abort RS
|
||||
LOG.info("Aborting region server: " + hrs.getServerName());
|
||||
int countBefore = cluster.getLiveRegionServerThreads().size();
|
||||
hrs.abort("testing");
|
||||
|
||||
// wait for abort completes
|
||||
TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate<Exception>() {
|
||||
@Override
|
||||
public boolean evaluate() throws Exception {
|
||||
return (cluster.getLiveRegionServerThreads().size() <= (NUM_RS - 1));
|
||||
int count = cluster.getLiveRegionServerThreads().size();
|
||||
return count <= (NUM_RS - 1);
|
||||
}
|
||||
});
|
||||
|
||||
|
|
|
@ -26,33 +26,31 @@ import java.util.List;
|
|||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.CategoryBasedTimeout;
|
||||
import org.apache.hadoop.hbase.ClusterStatus;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.MiniHBaseCluster;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.master.RegionState.State;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.testclassification.FlakeyTests;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
|
||||
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
import org.junit.rules.TestRule;
|
||||
|
||||
@Category({FlakeyTests.class, LargeTests.class})
|
||||
public class TestMasterFailover {
|
||||
private static final Log LOG = LogFactory.getLog(TestMasterFailover.class);
|
||||
|
||||
// TODO: Next test to add is with testing permutations of the RIT or the RS
|
||||
// killed are hosting ROOT and hbase:meta regions.
|
||||
|
||||
private void log(String string) {
|
||||
LOG.info("\n\n" + string + " \n\n");
|
||||
}
|
||||
@Rule public TestName name = new TestName();
|
||||
@Rule public final TestRule timeout = CategoryBasedTimeout.builder().
|
||||
withTimeout(this.getClass()).
|
||||
withLookingForStuckThread(true).
|
||||
build();
|
||||
|
||||
/**
|
||||
* Simple test of master failover.
|
||||
|
@ -61,112 +59,115 @@ public class TestMasterFailover {
|
|||
* master. Ensures the final master becomes active and we can still contact
|
||||
* the cluster.
|
||||
*/
|
||||
@Test (timeout=240000)
|
||||
@Test
|
||||
public void testSimpleMasterFailover() throws Exception {
|
||||
|
||||
final int NUM_MASTERS = 3;
|
||||
final int NUM_RS = 3;
|
||||
|
||||
// Start the cluster
|
||||
HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
try {
|
||||
TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
|
||||
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
|
||||
|
||||
TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
|
||||
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
|
||||
// get all the master threads
|
||||
List<MasterThread> masterThreads = cluster.getMasterThreads();
|
||||
|
||||
// get all the master threads
|
||||
List<MasterThread> masterThreads = cluster.getMasterThreads();
|
||||
|
||||
// wait for each to come online
|
||||
for (MasterThread mt : masterThreads) {
|
||||
assertTrue(mt.isAlive());
|
||||
}
|
||||
|
||||
// verify only one is the active master and we have right number
|
||||
int numActive = 0;
|
||||
int activeIndex = -1;
|
||||
ServerName activeName = null;
|
||||
HMaster active = null;
|
||||
for (int i = 0; i < masterThreads.size(); i++) {
|
||||
if (masterThreads.get(i).getMaster().isActiveMaster()) {
|
||||
numActive++;
|
||||
activeIndex = i;
|
||||
active = masterThreads.get(activeIndex).getMaster();
|
||||
activeName = active.getServerName();
|
||||
// wait for each to come online
|
||||
for (MasterThread mt : masterThreads) {
|
||||
assertTrue(mt.isAlive());
|
||||
}
|
||||
}
|
||||
assertEquals(1, numActive);
|
||||
assertEquals(NUM_MASTERS, masterThreads.size());
|
||||
LOG.info("Active master " + activeName);
|
||||
|
||||
// Check that ClusterStatus reports the correct active and backup masters
|
||||
assertNotNull(active);
|
||||
ClusterStatus status = active.getClusterStatus();
|
||||
assertTrue(status.getMaster().equals(activeName));
|
||||
assertEquals(2, status.getBackupMastersSize());
|
||||
assertEquals(2, status.getBackupMasters().size());
|
||||
|
||||
// attempt to stop one of the inactive masters
|
||||
int backupIndex = (activeIndex == 0 ? 1 : activeIndex - 1);
|
||||
HMaster master = cluster.getMaster(backupIndex);
|
||||
LOG.debug("\n\nStopping a backup master: " + master.getServerName() + "\n");
|
||||
cluster.stopMaster(backupIndex, false);
|
||||
cluster.waitOnMaster(backupIndex);
|
||||
|
||||
// Verify still one active master and it's the same
|
||||
for (int i = 0; i < masterThreads.size(); i++) {
|
||||
if (masterThreads.get(i).getMaster().isActiveMaster()) {
|
||||
assertTrue(activeName.equals(masterThreads.get(i).getMaster().getServerName()));
|
||||
activeIndex = i;
|
||||
active = masterThreads.get(activeIndex).getMaster();
|
||||
// verify only one is the active master and we have right number
|
||||
int numActive = 0;
|
||||
int activeIndex = -1;
|
||||
ServerName activeName = null;
|
||||
HMaster active = null;
|
||||
for (int i = 0; i < masterThreads.size(); i++) {
|
||||
if (masterThreads.get(i).getMaster().isActiveMaster()) {
|
||||
numActive++;
|
||||
activeIndex = i;
|
||||
active = masterThreads.get(activeIndex).getMaster();
|
||||
activeName = active.getServerName();
|
||||
}
|
||||
}
|
||||
assertEquals(1, numActive);
|
||||
assertEquals(NUM_MASTERS, masterThreads.size());
|
||||
LOG.info("Active master " + activeName);
|
||||
|
||||
// Check that ClusterStatus reports the correct active and backup masters
|
||||
assertNotNull(active);
|
||||
ClusterStatus status = active.getClusterStatus();
|
||||
assertTrue(status.getMaster().equals(activeName));
|
||||
assertEquals(2, status.getBackupMastersSize());
|
||||
assertEquals(2, status.getBackupMasters().size());
|
||||
|
||||
// attempt to stop one of the inactive masters
|
||||
int backupIndex = (activeIndex == 0 ? 1 : activeIndex - 1);
|
||||
HMaster master = cluster.getMaster(backupIndex);
|
||||
LOG.debug("\n\nStopping a backup master: " + master.getServerName() + "\n");
|
||||
cluster.stopMaster(backupIndex, false);
|
||||
cluster.waitOnMaster(backupIndex);
|
||||
|
||||
// Verify still one active master and it's the same
|
||||
for (int i = 0; i < masterThreads.size(); i++) {
|
||||
if (masterThreads.get(i).getMaster().isActiveMaster()) {
|
||||
assertTrue(activeName.equals(masterThreads.get(i).getMaster().getServerName()));
|
||||
activeIndex = i;
|
||||
active = masterThreads.get(activeIndex).getMaster();
|
||||
}
|
||||
}
|
||||
assertEquals(1, numActive);
|
||||
assertEquals(2, masterThreads.size());
|
||||
int rsCount = masterThreads.get(activeIndex).getMaster().getClusterStatus().getServersSize();
|
||||
LOG.info("Active master " + active.getServerName() + " managing " + rsCount +
|
||||
" regions servers");
|
||||
assertEquals(3, rsCount);
|
||||
|
||||
// Check that ClusterStatus reports the correct active and backup masters
|
||||
assertNotNull(active);
|
||||
status = active.getClusterStatus();
|
||||
assertTrue(status.getMaster().equals(activeName));
|
||||
assertEquals(1, status.getBackupMastersSize());
|
||||
assertEquals(1, status.getBackupMasters().size());
|
||||
|
||||
// kill the active master
|
||||
LOG.debug("\n\nStopping the active master " + active.getServerName() + "\n");
|
||||
cluster.stopMaster(activeIndex, false);
|
||||
cluster.waitOnMaster(activeIndex);
|
||||
|
||||
// wait for an active master to show up and be ready
|
||||
assertTrue(cluster.waitForActiveAndReadyMaster());
|
||||
|
||||
LOG.debug("\n\nVerifying backup master is now active\n");
|
||||
// should only have one master now
|
||||
assertEquals(1, masterThreads.size());
|
||||
|
||||
// and he should be active
|
||||
active = masterThreads.get(0).getMaster();
|
||||
assertNotNull(active);
|
||||
status = active.getClusterStatus();
|
||||
ServerName mastername = status.getMaster();
|
||||
assertTrue(mastername.equals(active.getServerName()));
|
||||
assertTrue(active.isActiveMaster());
|
||||
assertEquals(0, status.getBackupMastersSize());
|
||||
assertEquals(0, status.getBackupMasters().size());
|
||||
int rss = status.getServersSize();
|
||||
LOG.info("Active master " + mastername.getServerName() + " managing " +
|
||||
rss + " region servers");
|
||||
assertEquals(3, rss);
|
||||
} finally {
|
||||
// Stop the cluster
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
assertEquals(1, numActive);
|
||||
assertEquals(2, masterThreads.size());
|
||||
int rsCount = masterThreads.get(activeIndex).getMaster().getClusterStatus().getServersSize();
|
||||
LOG.info("Active master " + active.getServerName() + " managing " + rsCount + " regions servers");
|
||||
assertEquals(3, rsCount);
|
||||
|
||||
// Check that ClusterStatus reports the correct active and backup masters
|
||||
assertNotNull(active);
|
||||
status = active.getClusterStatus();
|
||||
assertTrue(status.getMaster().equals(activeName));
|
||||
assertEquals(1, status.getBackupMastersSize());
|
||||
assertEquals(1, status.getBackupMasters().size());
|
||||
|
||||
// kill the active master
|
||||
LOG.debug("\n\nStopping the active master " + active.getServerName() + "\n");
|
||||
cluster.stopMaster(activeIndex, false);
|
||||
cluster.waitOnMaster(activeIndex);
|
||||
|
||||
// wait for an active master to show up and be ready
|
||||
assertTrue(cluster.waitForActiveAndReadyMaster());
|
||||
|
||||
LOG.debug("\n\nVerifying backup master is now active\n");
|
||||
// should only have one master now
|
||||
assertEquals(1, masterThreads.size());
|
||||
|
||||
// and he should be active
|
||||
active = masterThreads.get(0).getMaster();
|
||||
assertNotNull(active);
|
||||
status = active.getClusterStatus();
|
||||
ServerName mastername = status.getMaster();
|
||||
assertTrue(mastername.equals(active.getServerName()));
|
||||
assertTrue(active.isActiveMaster());
|
||||
assertEquals(0, status.getBackupMastersSize());
|
||||
assertEquals(0, status.getBackupMasters().size());
|
||||
int rss = status.getServersSize();
|
||||
LOG.info("Active master " + mastername.getServerName() + " managing " +
|
||||
rss + " region servers");
|
||||
assertEquals(3, rss);
|
||||
|
||||
// Stop the cluster
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
/**
|
||||
* Test meta in transition when master failover
|
||||
* Test meta in transition when master failover.
|
||||
* This test used to manipulate region state up in zk. That is not allowed any more in hbase2
|
||||
* so I removed that messing. That makes this test anemic.
|
||||
*/
|
||||
@Test(timeout = 180000)
|
||||
@Test
|
||||
public void testMetaInTransitionWhenMasterFailover() throws Exception {
|
||||
final int NUM_MASTERS = 1;
|
||||
final int NUM_RS = 1;
|
||||
|
@ -174,110 +175,43 @@ public class TestMasterFailover {
|
|||
// Start the cluster
|
||||
HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
|
||||
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
|
||||
log("Cluster started");
|
||||
try {
|
||||
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
|
||||
LOG.info("Cluster started");
|
||||
|
||||
log("Moving meta off the master");
|
||||
HMaster activeMaster = cluster.getMaster();
|
||||
HRegionServer rs = cluster.getRegionServer(0);
|
||||
ServerName metaServerName = cluster.getLiveRegionServerThreads()
|
||||
.get(0).getRegionServer().getServerName();
|
||||
activeMaster.move(HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes(),
|
||||
Bytes.toBytes(metaServerName.getServerName()));
|
||||
TEST_UTIL.waitUntilNoRegionsInTransition(60000);
|
||||
assertEquals("Meta should be assigned on expected regionserver",
|
||||
metaServerName, activeMaster.getMetaTableLocator()
|
||||
.getMetaRegionLocation(activeMaster.getZooKeeper()));
|
||||
HMaster activeMaster = cluster.getMaster();
|
||||
ServerName metaServerName = cluster.getServerHoldingMeta();
|
||||
HRegionServer hrs = cluster.getRegionServer(metaServerName);
|
||||
|
||||
// Now kill master, meta should remain on rs, where we placed it before.
|
||||
log("Aborting master");
|
||||
activeMaster.abort("test-kill");
|
||||
cluster.waitForMasterToStop(activeMaster.getServerName(), 30000);
|
||||
log("Master has aborted");
|
||||
// Now kill master, meta should remain on rs, where we placed it before.
|
||||
LOG.info("Aborting master");
|
||||
activeMaster.abort("test-kill");
|
||||
cluster.waitForMasterToStop(activeMaster.getServerName(), 30000);
|
||||
LOG.info("Master has aborted");
|
||||
|
||||
// meta should remain where it was
|
||||
RegionState metaState =
|
||||
MetaTableLocator.getMetaRegionState(rs.getZooKeeper());
|
||||
assertEquals("hbase:meta should be online on RS",
|
||||
metaState.getServerName(), rs.getServerName());
|
||||
assertEquals("hbase:meta should be online on RS",
|
||||
metaState.getState(), State.OPEN);
|
||||
// meta should remain where it was
|
||||
RegionState metaState = MetaTableLocator.getMetaRegionState(hrs.getZooKeeper());
|
||||
assertEquals("hbase:meta should be online on RS",
|
||||
metaState.getServerName(), metaServerName);
|
||||
assertEquals("hbase:meta should be online on RS", metaState.getState(), State.OPEN);
|
||||
|
||||
// Start up a new master
|
||||
log("Starting up a new master");
|
||||
activeMaster = cluster.startMaster().getMaster();
|
||||
log("Waiting for master to be ready");
|
||||
cluster.waitForActiveAndReadyMaster();
|
||||
log("Master is ready");
|
||||
// Start up a new master
|
||||
LOG.info("Starting up a new master");
|
||||
activeMaster = cluster.startMaster().getMaster();
|
||||
LOG.info("Waiting for master to be ready");
|
||||
cluster.waitForActiveAndReadyMaster();
|
||||
LOG.info("Master is ready");
|
||||
|
||||
// ensure meta is still deployed on RS
|
||||
metaState =
|
||||
MetaTableLocator.getMetaRegionState(activeMaster.getZooKeeper());
|
||||
assertEquals("hbase:meta should be online on RS",
|
||||
metaState.getServerName(), rs.getServerName());
|
||||
assertEquals("hbase:meta should be online on RS",
|
||||
metaState.getState(), State.OPEN);
|
||||
// ensure meta is still deployed on RS
|
||||
metaState = MetaTableLocator.getMetaRegionState(activeMaster.getZooKeeper());
|
||||
assertEquals("hbase:meta should be online on RS",
|
||||
metaState.getServerName(), metaServerName);
|
||||
assertEquals("hbase:meta should be online on RS", metaState.getState(), State.OPEN);
|
||||
|
||||
// Update meta state as OPENING, then kill master
|
||||
// that simulates, that RS successfully deployed, but
|
||||
// RPC was lost right before failure.
|
||||
// region server should expire (how it can be verified?)
|
||||
MetaTableLocator.setMetaLocation(activeMaster.getZooKeeper(),
|
||||
rs.getServerName(), State.OPENING);
|
||||
HRegion meta = rs.getRegion(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
|
||||
rs.removeRegion(meta, null);
|
||||
meta.close();
|
||||
|
||||
log("Aborting master");
|
||||
activeMaster.abort("test-kill");
|
||||
cluster.waitForMasterToStop(activeMaster.getServerName(), 30000);
|
||||
log("Master has aborted");
|
||||
|
||||
// Start up a new master
|
||||
log("Starting up a new master");
|
||||
activeMaster = cluster.startMaster().getMaster();
|
||||
log("Waiting for master to be ready");
|
||||
cluster.waitForActiveAndReadyMaster();
|
||||
log("Master is ready");
|
||||
|
||||
TEST_UTIL.waitUntilNoRegionsInTransition(60000);
|
||||
log("Meta was assigned");
|
||||
|
||||
metaState =
|
||||
MetaTableLocator.getMetaRegionState(activeMaster.getZooKeeper());
|
||||
assertEquals("hbase:meta should be online on RS",
|
||||
metaState.getServerName(), rs.getServerName());
|
||||
assertEquals("hbase:meta should be online on RS",
|
||||
metaState.getState(), State.OPEN);
|
||||
|
||||
// Update meta state as CLOSING, then kill master
|
||||
// that simulates, that RS successfully deployed, but
|
||||
// RPC was lost right before failure.
|
||||
// region server should expire (how it can be verified?)
|
||||
MetaTableLocator.setMetaLocation(activeMaster.getZooKeeper(),
|
||||
rs.getServerName(), State.CLOSING);
|
||||
|
||||
log("Aborting master");
|
||||
activeMaster.abort("test-kill");
|
||||
cluster.waitForMasterToStop(activeMaster.getServerName(), 30000);
|
||||
log("Master has aborted");
|
||||
|
||||
rs.getRSRpcServices().closeRegion(null, ProtobufUtil.buildCloseRegionRequest(
|
||||
rs.getServerName(), HRegionInfo.FIRST_META_REGIONINFO.getRegionName()));
|
||||
|
||||
// Start up a new master
|
||||
log("Starting up a new master");
|
||||
activeMaster = cluster.startMaster().getMaster();
|
||||
assertNotNull(activeMaster);
|
||||
log("Waiting for master to be ready");
|
||||
cluster.waitForActiveAndReadyMaster();
|
||||
log("Master is ready");
|
||||
|
||||
TEST_UTIL.waitUntilNoRegionsInTransition(60000);
|
||||
log("Meta was assigned");
|
||||
|
||||
// Done, shutdown the cluster
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
// Done, shutdown the cluster
|
||||
} finally {
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
/**
|
||||
/*
|
||||
* 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
|
||||
|
@ -216,7 +216,7 @@ public class TestAssignmentManager {
|
|||
final TableName tableName = TableName.valueOf("testAssignAndCrashBeforeResponse");
|
||||
final RegionInfo hri = createRegionInfo(tableName, 1);
|
||||
rsDispatcher.setMockRsExecutor(new HangThenRSCrashExecutor());
|
||||
AssignProcedure proc = am.createAssignProcedure(hri, false);
|
||||
AssignProcedure proc = am.createAssignProcedure(hri);
|
||||
waitOnFuture(submitProcedure(proc));
|
||||
}
|
||||
|
||||
|
@ -226,7 +226,7 @@ public class TestAssignmentManager {
|
|||
final RegionInfo hri = createRegionInfo(tableName, 1);
|
||||
rsDispatcher.setMockRsExecutor(new HangOnCloseThenRSCrashExecutor());
|
||||
for (int i = 0; i < HangOnCloseThenRSCrashExecutor.TYPES_OF_FAILURE; i++) {
|
||||
AssignProcedure assign = am.createAssignProcedure(hri, false);
|
||||
AssignProcedure assign = am.createAssignProcedure(hri);
|
||||
waitOnFuture(submitProcedure(assign));
|
||||
UnassignProcedure unassign = am.createUnassignProcedure(hri,
|
||||
am.getRegionStates().getRegionServerOfRegion(hri), false);
|
||||
|
@ -243,7 +243,7 @@ public class TestAssignmentManager {
|
|||
// Loop a bunch of times so we hit various combos of exceptions.
|
||||
for (int i = 0; i < 10; i++) {
|
||||
LOG.info("" + i);
|
||||
AssignProcedure proc = am.createAssignProcedure(hri, false);
|
||||
AssignProcedure proc = am.createAssignProcedure(hri);
|
||||
waitOnFuture(submitProcedure(proc));
|
||||
}
|
||||
}
|
||||
|
@ -257,7 +257,7 @@ public class TestAssignmentManager {
|
|||
collectAssignmentManagerMetrics();
|
||||
|
||||
rsDispatcher.setMockRsExecutor(new SocketTimeoutRsExecutor(20, 3));
|
||||
waitOnFuture(submitProcedure(am.createAssignProcedure(hri, false)));
|
||||
waitOnFuture(submitProcedure(am.createAssignProcedure(hri)));
|
||||
|
||||
rsDispatcher.setMockRsExecutor(new SocketTimeoutRsExecutor(20, 1));
|
||||
// exception.expect(ServerCrashException.class);
|
||||
|
@ -285,7 +285,7 @@ public class TestAssignmentManager {
|
|||
// Test Assign operation failure
|
||||
rsDispatcher.setMockRsExecutor(executor);
|
||||
try {
|
||||
waitOnFuture(submitProcedure(am.createAssignProcedure(hri, false)));
|
||||
waitOnFuture(submitProcedure(am.createAssignProcedure(hri)));
|
||||
fail("unexpected assign completion");
|
||||
} catch (RetriesExhaustedException e) {
|
||||
// expected exception
|
||||
|
@ -294,7 +294,7 @@ public class TestAssignmentManager {
|
|||
|
||||
// Assign the region (without problems)
|
||||
rsDispatcher.setMockRsExecutor(new GoodRsExecutor());
|
||||
waitOnFuture(submitProcedure(am.createAssignProcedure(hri, false)));
|
||||
waitOnFuture(submitProcedure(am.createAssignProcedure(hri)));
|
||||
|
||||
// TODO: Currently unassign just keeps trying until it sees a server crash.
|
||||
// There is no count on unassign.
|
||||
|
@ -345,7 +345,7 @@ public class TestAssignmentManager {
|
|||
// Test Assign operation failure
|
||||
rsDispatcher.setMockRsExecutor(executor);
|
||||
try {
|
||||
waitOnFuture(submitProcedure(am.createAssignProcedure(hri, false)));
|
||||
waitOnFuture(submitProcedure(am.createAssignProcedure(hri)));
|
||||
fail("unexpected assign completion");
|
||||
} catch (RetriesExhaustedException e) {
|
||||
// expected exception
|
||||
|
@ -388,7 +388,7 @@ public class TestAssignmentManager {
|
|||
|
||||
rsDispatcher.setMockRsExecutor(new GoodRsExecutor());
|
||||
|
||||
final Future<byte[]> futureA = submitProcedure(am.createAssignProcedure(hri, false));
|
||||
final Future<byte[]> futureA = submitProcedure(am.createAssignProcedure(hri));
|
||||
|
||||
// wait first assign
|
||||
waitOnFuture(futureA);
|
||||
|
@ -396,7 +396,7 @@ public class TestAssignmentManager {
|
|||
// Second should be a noop. We should recognize region is already OPEN internally
|
||||
// and skip out doing nothing.
|
||||
// wait second assign
|
||||
final Future<byte[]> futureB = submitProcedure(am.createAssignProcedure(hri, false));
|
||||
final Future<byte[]> futureB = submitProcedure(am.createAssignProcedure(hri));
|
||||
waitOnFuture(futureB);
|
||||
am.getRegionStates().isRegionInState(hri, State.OPEN);
|
||||
// TODO: What else can we do to ensure just a noop.
|
||||
|
@ -419,7 +419,7 @@ public class TestAssignmentManager {
|
|||
rsDispatcher.setMockRsExecutor(new GoodRsExecutor());
|
||||
|
||||
// assign the region first
|
||||
waitOnFuture(submitProcedure(am.createAssignProcedure(hri, false)));
|
||||
waitOnFuture(submitProcedure(am.createAssignProcedure(hri)));
|
||||
|
||||
final Future<byte[]> futureA = submitProcedure(am.createUnassignProcedure(hri, null, false));
|
||||
|
||||
|
@ -516,7 +516,7 @@ public class TestAssignmentManager {
|
|||
|
||||
private AssignProcedure createAndSubmitAssign(TableName tableName, int regionId) {
|
||||
RegionInfo hri = createRegionInfo(tableName, regionId);
|
||||
AssignProcedure proc = am.createAssignProcedure(hri, false);
|
||||
AssignProcedure proc = am.createAssignProcedure(hri);
|
||||
master.getMasterProcedureExecutor().submitProcedure(proc);
|
||||
return proc;
|
||||
}
|
||||
|
|
|
@ -178,6 +178,11 @@ public class TestRegionsOnMasterOptions {
|
|||
while (!cluster.getMaster().isInitialized()) {
|
||||
Threads.sleep(10);
|
||||
}
|
||||
while (cluster.getMaster().getAssignmentManager().
|
||||
computeRegionInTransitionStat().getTotalRITs() > 0) {
|
||||
Threads.sleep(100);
|
||||
LOG.info("Waiting on RIT to go to zero before calling balancer...");
|
||||
}
|
||||
LOG.info("Cluster is up; running balancer");
|
||||
cluster.getMaster().balance();
|
||||
regions = cluster.getMaster().getRegions();
|
||||
|
|
|
@ -0,0 +1,90 @@
|
|||
/*
|
||||
* Copyright The Apache Software Foundation
|
||||
*
|
||||
* 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.master.snapshot;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.CategoryBasedTimeout;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
|
||||
import org.apache.hadoop.hbase.master.assignment.AssignProcedure;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
import org.junit.rules.TestRule;
|
||||
|
||||
import static junit.framework.TestCase.assertTrue;
|
||||
|
||||
|
||||
@Category({RegionServerTests.class, SmallTests.class})
|
||||
public class TestAssignProcedure {
|
||||
@Rule public TestName name = new TestName();
|
||||
@Rule public final TestRule timeout = CategoryBasedTimeout.builder().
|
||||
withTimeout(this.getClass()).
|
||||
withLookingForStuckThread(true).
|
||||
build();
|
||||
|
||||
@Test
|
||||
public void testSimpleComparator() {
|
||||
List<AssignProcedure> procedures = new ArrayList<AssignProcedure>();
|
||||
RegionInfo user1 = RegionInfoBuilder.newBuilder(TableName.valueOf("user_space1")).build();
|
||||
procedures.add(new AssignProcedure(user1));
|
||||
RegionInfo user2 = RegionInfoBuilder.newBuilder(TableName.valueOf("user_space2")).build();
|
||||
procedures.add(new AssignProcedure(RegionInfoBuilder.FIRST_META_REGIONINFO));
|
||||
procedures.add(new AssignProcedure(user2));
|
||||
RegionInfo system = RegionInfoBuilder.newBuilder(TableName.NAMESPACE_TABLE_NAME).build();
|
||||
procedures.add(new AssignProcedure(system));
|
||||
procedures.sort(AssignProcedure.COMPARATOR);
|
||||
assertTrue(procedures.get(0).isMeta());
|
||||
assertTrue(procedures.get(1).getRegionInfo().getTable().equals(TableName.NAMESPACE_TABLE_NAME));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testComparatorWithMetas() {
|
||||
List<AssignProcedure> procedures = new ArrayList<AssignProcedure>();
|
||||
RegionInfo user1 = RegionInfoBuilder.newBuilder(TableName.valueOf("user_space1")).build();
|
||||
procedures.add(new AssignProcedure(user1));
|
||||
RegionInfo meta2 = RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).
|
||||
setStartKey(Bytes.toBytes("002")).build();
|
||||
procedures.add(new AssignProcedure(meta2));
|
||||
RegionInfo meta1 = RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).
|
||||
setStartKey(Bytes.toBytes("001")).build();
|
||||
procedures.add(new AssignProcedure(meta1));
|
||||
procedures.add(new AssignProcedure(RegionInfoBuilder.FIRST_META_REGIONINFO));
|
||||
RegionInfo meta0 = RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).
|
||||
setStartKey(Bytes.toBytes("000")).build();
|
||||
procedures.add(new AssignProcedure(meta0));
|
||||
RegionInfo user2 = RegionInfoBuilder.newBuilder(TableName.valueOf("user_space2")).build();
|
||||
procedures.add(new AssignProcedure(user2));
|
||||
RegionInfo system = RegionInfoBuilder.newBuilder(TableName.NAMESPACE_TABLE_NAME).build();
|
||||
procedures.add(new AssignProcedure(system));
|
||||
procedures.sort(AssignProcedure.COMPARATOR);
|
||||
assertTrue(procedures.get(0).getRegionInfo().equals(RegionInfoBuilder.FIRST_META_REGIONINFO));
|
||||
assertTrue(procedures.get(1).getRegionInfo().equals(meta0));
|
||||
assertTrue(procedures.get(2).getRegionInfo().equals(meta1));
|
||||
assertTrue(procedures.get(3).getRegionInfo().equals(meta2));
|
||||
assertTrue(procedures.get(4).getRegionInfo().getTable().equals(TableName.NAMESPACE_TABLE_NAME));
|
||||
}
|
||||
}
|
|
@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
|||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
@ -60,7 +61,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto
|
|||
* from list of online regions. See HBASE-9593.
|
||||
*/
|
||||
@Category({RegionServerTests.class, MediumTests.class})
|
||||
public class TestRSKilledWhenInitializing {
|
||||
@Ignore("See HBASE-19515") public class TestRSKilledWhenInitializing {
|
||||
private static final Log LOG = LogFactory.getLog(TestRSKilledWhenInitializing.class);
|
||||
@Rule public TestName testName = new TestName();
|
||||
@Rule public final TestRule timeout = CategoryBasedTimeout.builder().
|
||||
|
@ -141,6 +142,10 @@ public class TestRSKilledWhenInitializing {
|
|||
LOG.info("Move " + hri.getEncodedName() + " to " + killedRS.get());
|
||||
master.getMaster().move(hri.getEncodedNameAsBytes(),
|
||||
Bytes.toBytes(killedRS.get().toString()));
|
||||
|
||||
// TODO: This test could do more to verify fix. It could create a table
|
||||
// and do round-robin assign. It should fail if zombie RS. HBASE-19515.
|
||||
|
||||
// Wait until the RS no longer shows as registered in Master.
|
||||
while (onlineServersList.size() > (NUM_RS + 1)) {
|
||||
Thread.sleep(100);
|
||||
|
|
|
@ -153,7 +153,7 @@ public class TestRegionMergeTransactionOnCluster {
|
|||
RegionStates regionStates = am.getRegionStates();
|
||||
|
||||
// We should not be able to assign it again
|
||||
am.assign(hri, true);
|
||||
am.assign(hri);
|
||||
assertFalse("Merged region can't be assigned",
|
||||
regionStates.isRegionInTransition(hri));
|
||||
|
||||
|
|
|
@ -0,0 +1,164 @@
|
|||
/*
|
||||
* 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.regionserver;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.CategoryBasedTimeout;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.RegionSplitter;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
import org.junit.rules.TestRule;
|
||||
|
||||
@Category({RegionServerTests.class, MediumTests.class})
|
||||
public class TestRegionReplicasWithRestartScenarios {
|
||||
private static final Log LOG = LogFactory.getLog(TestRegionReplicasWithRestartScenarios.class);
|
||||
@Rule public TestName name = new TestName();
|
||||
@Rule public final TestRule timeout = CategoryBasedTimeout.builder().
|
||||
withTimeout(this.getClass()).
|
||||
withLookingForStuckThread(true).
|
||||
build();
|
||||
|
||||
private static final int NB_SERVERS = 3;
|
||||
private Table table;
|
||||
|
||||
private static final HBaseTestingUtility HTU = new HBaseTestingUtility();
|
||||
private static final byte[] f = HConstants.CATALOG_FAMILY;
|
||||
|
||||
@BeforeClass
|
||||
public static void beforeClass() throws Exception {
|
||||
// Reduce the hdfs block size and prefetch to trigger the file-link reopen
|
||||
// when the file is moved to archive (e.g. compaction)
|
||||
HTU.getConfiguration().setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 8192);
|
||||
HTU.getConfiguration().setInt(DFSConfigKeys.DFS_CLIENT_READ_PREFETCH_SIZE_KEY, 1);
|
||||
HTU.getConfiguration().setInt(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 128 * 1024 * 1024);
|
||||
HTU.getConfiguration().setInt(">hbase.master.wait.on.regionservers.mintostart", 3);
|
||||
HTU.startMiniCluster(NB_SERVERS);
|
||||
}
|
||||
|
||||
@Before
|
||||
public void before() throws IOException {
|
||||
TableName tableName = TableName.valueOf(this.name.getMethodName());
|
||||
// Create table then get the single region for our new table.
|
||||
this.table = createTableDirectlyFromHTD(tableName);
|
||||
}
|
||||
|
||||
@After
|
||||
public void after() throws IOException {
|
||||
this.table.close();
|
||||
}
|
||||
|
||||
private static Table createTableDirectlyFromHTD(final TableName tableName) throws IOException {
|
||||
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
|
||||
builder.setRegionReplication(3);
|
||||
return HTU.createTable(builder.build(), new byte[][] { f }, getSplits(20),
|
||||
new Configuration(HTU.getConfiguration()));
|
||||
}
|
||||
|
||||
private static byte[][] getSplits(int numRegions) {
|
||||
RegionSplitter.UniformSplit split = new RegionSplitter.UniformSplit();
|
||||
split.setFirstRow(Bytes.toBytes(0L));
|
||||
split.setLastRow(Bytes.toBytes(Long.MAX_VALUE));
|
||||
return split.split(numRegions);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void afterClass() throws Exception {
|
||||
HRegionServer.TEST_SKIP_REPORTING_TRANSITION = false;
|
||||
HTU.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
private HRegionServer getRS() {
|
||||
return HTU.getMiniHBaseCluster().getRegionServer(0);
|
||||
}
|
||||
|
||||
private HRegionServer getSecondaryRS() {
|
||||
return HTU.getMiniHBaseCluster().getRegionServer(1);
|
||||
}
|
||||
|
||||
private HRegionServer getTertiaryRS() {
|
||||
return HTU.getMiniHBaseCluster().getRegionServer(2);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRegionReplicasCreated() throws Exception {
|
||||
Collection<HRegion> onlineRegions = getRS().getOnlineRegionsLocalContext();
|
||||
boolean res = checkDuplicates(onlineRegions);
|
||||
assertFalse(res);
|
||||
Collection<HRegion> onlineRegions2 = getSecondaryRS().getOnlineRegionsLocalContext();
|
||||
res = checkDuplicates(onlineRegions2);
|
||||
assertFalse(res);
|
||||
Collection<HRegion> onlineRegions3 = getTertiaryRS().getOnlineRegionsLocalContext();
|
||||
checkDuplicates(onlineRegions3);
|
||||
assertFalse(res);
|
||||
int totalRegions = onlineRegions.size() + onlineRegions2.size() + onlineRegions3.size();
|
||||
assertEquals(62, totalRegions);
|
||||
}
|
||||
|
||||
private boolean checkDuplicates(Collection<HRegion> onlineRegions3) throws Exception {
|
||||
ArrayList<Region> copyOfRegion = new ArrayList<Region>(onlineRegions3);
|
||||
for (Region region : copyOfRegion) {
|
||||
RegionInfo regionInfo = region.getRegionInfo();
|
||||
RegionInfo regionInfoForReplica =
|
||||
RegionReplicaUtil.getRegionInfoForDefaultReplica(regionInfo);
|
||||
int i = 0;
|
||||
for (Region actualRegion : onlineRegions3) {
|
||||
if (regionInfoForReplica.equals(
|
||||
RegionReplicaUtil.getRegionInfoForDefaultReplica(actualRegion.getRegionInfo()))) {
|
||||
i++;
|
||||
if (i > 1) {
|
||||
LOG.info("Duplicate found " + actualRegion.getRegionInfo() + " " +
|
||||
region.getRegionInfo());
|
||||
assertTrue(Bytes.equals(region.getRegionInfo().getStartKey(),
|
||||
actualRegion.getRegionInfo().getStartKey()));
|
||||
assertTrue(Bytes.equals(region.getRegionInfo().getEndKey(),
|
||||
actualRegion.getRegionInfo().getEndKey()));
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
}
|
|
@ -716,7 +716,7 @@ public class TestSplitTransactionOnCluster {
|
|||
assertTrue(regionStates.isRegionInState(daughters.get(1).getRegionInfo(), State.OPEN));
|
||||
|
||||
// We should not be able to assign it again
|
||||
am.assign(hri, true);
|
||||
am.assign(hri);
|
||||
assertFalse("Split region can't be assigned",
|
||||
regionStates.isRegionInTransition(hri));
|
||||
assertTrue(regionStates.isRegionInState(hri, State.SPLIT));
|
||||
|
|
Loading…
Reference in New Issue