HBASE-3018 Bulk assignment on startup runs serially through the cluster servers assigning in bulk to one at a time
M src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java Removed reentrant lock that spanned moving region into regionsintransition and the getting of lock on state object. It was a bad idea. I found it actually deadlocked. It shouldn't be needed. The call to assign will fail if the state instance is not of the right 'state' type. Run a thread per server parcelling out region assignments so we assign across the cluster concurrently rather than in series as we were doing. git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@999237 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
34f17faa5d
commit
14d8fb2ab7
|
@ -530,6 +530,8 @@ Release 0.21.0 - Unreleased
|
|||
HBASE-3015 recovered.edits files not deleted if it only contain edits that
|
||||
have already been flushed; hurts perf for all future opens of
|
||||
the region
|
||||
HBASE-3018 Bulk assignment on startup runs serially through the cluster
|
||||
servers assigning in bulk to one at a time
|
||||
|
||||
IMPROVEMENTS
|
||||
HBASE-1760 Cleanup TODOs in HTable
|
||||
|
|
|
@ -118,8 +118,6 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
private final SortedMap<HRegionInfo,HServerInfo> regions =
|
||||
new TreeMap<HRegionInfo,HServerInfo>();
|
||||
|
||||
private final ReentrantLock assignLock = new ReentrantLock();
|
||||
|
||||
private final ExecutorService executorService;
|
||||
|
||||
/**
|
||||
|
@ -493,11 +491,6 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
// Grab the state of this region and synchronize on it
|
||||
String encodedName = region.getEncodedName();
|
||||
RegionState state;
|
||||
// This assignLock is used bridging the two synchronization blocks. Once
|
||||
// we've made it into the 'state' synchronization block, then we can let
|
||||
// go of this lock. There must be a better construct that this -- St.Ack 20100811
|
||||
this.assignLock.lock();
|
||||
try {
|
||||
synchronized (regionsInTransition) {
|
||||
state = regionsInTransition.get(encodedName);
|
||||
if (state == null) {
|
||||
|
@ -505,13 +498,12 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
regionsInTransition.put(encodedName, state);
|
||||
}
|
||||
}
|
||||
// This here gap between synchronizations looks like a hole but it should
|
||||
// be ok because the assign below would protect against being called with
|
||||
// a state instance that is not in the right 'state' -- St.Ack 20100920.
|
||||
synchronized (state) {
|
||||
this.assignLock.unlock();
|
||||
assign(state);
|
||||
}
|
||||
} finally {
|
||||
if (this.assignLock.isHeldByCurrentThread()) this.assignLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -675,32 +667,21 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
// Scan META for all user regions
|
||||
List<HRegionInfo> allRegions =
|
||||
MetaScanner.listAllRegions(master.getConfiguration());
|
||||
if (allRegions == null || allRegions.isEmpty()) {
|
||||
return;
|
||||
}
|
||||
if (allRegions == null || allRegions.isEmpty()) return;
|
||||
|
||||
// Get all available servers
|
||||
List<HServerInfo> servers = serverManager.getOnlineServersList();
|
||||
|
||||
LOG.info("Assigning " + allRegions.size() + " regions across " + servers.size() +
|
||||
" servers");
|
||||
LOG.info("Assigning " + allRegions.size() + " region(s) across " +
|
||||
servers.size() + " server(s)");
|
||||
|
||||
// Generate a cluster startup region placement plan
|
||||
Map<HServerInfo, List<HRegionInfo>> bulkPlan =
|
||||
LoadBalancer.bulkAssignment(allRegions, servers);
|
||||
|
||||
// For each server, create OFFLINE nodes and send OPEN RPCs
|
||||
// Now start a thread per server to run assignment.
|
||||
for (Map.Entry<HServerInfo,List<HRegionInfo>> entry: bulkPlan.entrySet()) {
|
||||
HServerInfo server = entry.getKey();
|
||||
List<HRegionInfo> regions = entry.getValue();
|
||||
LOG.debug("Assigning " + regions.size() + " regions to " + server);
|
||||
for (HRegionInfo region : regions) {
|
||||
LOG.debug("Assigning " + region.getRegionNameAsString() + " to " + server);
|
||||
String regionName = region.getEncodedName();
|
||||
RegionPlan plan = new RegionPlan(region, null,server);
|
||||
regionPlans.put(regionName, plan);
|
||||
assign(region);
|
||||
}
|
||||
Thread t = new BulkAssignServer(entry.getKey(), entry.getValue(), this.master);
|
||||
t.start();
|
||||
}
|
||||
|
||||
// Wait for no regions to be in transition
|
||||
|
@ -714,6 +695,46 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
LOG.info("All user regions have been assigned");
|
||||
}
|
||||
|
||||
/**
|
||||
* Class to run bulk assign to a single server.
|
||||
*/
|
||||
class BulkAssignServer extends Thread {
|
||||
private final List<HRegionInfo> regions;
|
||||
private final HServerInfo server;
|
||||
private final Stoppable stopper;
|
||||
|
||||
BulkAssignServer(final HServerInfo server,
|
||||
final List<HRegionInfo> regions, final Stoppable stopper) {
|
||||
super("serverassign-" + server.getServerName());
|
||||
setDaemon(true);
|
||||
this.server = server;
|
||||
this.regions = regions;
|
||||
this.stopper = stopper;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
// Insert a plan for each region with 'server' as the target regionserver.
|
||||
// Below, we run through regions one at a time. The call to assign will
|
||||
// move the region into the regionsInTransition which starts up a timer.
|
||||
// if the region is not out of the regionsInTransition by a certain time,
|
||||
// it will be reassigned. We don't want that to happen. So, do it this
|
||||
// way a region at a time for now. Presumably the regionserver will put
|
||||
// up a back pressure if opening a region takes time which is good since
|
||||
// this will block our adding new regions to regionsInTransition. Later
|
||||
// make it so we can send over a lump of regions in one rpc with the
|
||||
// regionserver on remote side tickling zk on a period to prevent our
|
||||
// regionsInTransition timing out. Currently its not possible given the
|
||||
// Executor architecture on the regionserver side. St.Ack 20100920.
|
||||
for (HRegionInfo region : regions) {
|
||||
LOG.debug("Assigning " + region.getRegionNameAsString() + " to " + this.server);
|
||||
regionPlans.put(region.getEncodedName(), new RegionPlan(region, null, server));
|
||||
assign(region);
|
||||
if (this.stopper.isStopped()) break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void rebuildUserRegions() throws IOException {
|
||||
Map<HRegionInfo,HServerAddress> allRegions =
|
||||
MetaReader.fullScan(catalogTracker);
|
||||
|
|
Loading…
Reference in New Issue