HBASE-5422 StartupBulkAssigner would cause a lot of timeout on RIT when assigning large numbers of regions (timeout = 3 mins)
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1292409 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
0f708f7370
commit
7b5ec59eda
|
@ -59,11 +59,12 @@ import org.apache.hadoop.hbase.catalog.CatalogTracker;
|
|||
import org.apache.hadoop.hbase.catalog.MetaReader;
|
||||
import org.apache.hadoop.hbase.catalog.RootLocationEditor;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
|
||||
import org.apache.hadoop.hbase.executor.EventHandler;
|
||||
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.AssignmentManager.RegionState;
|
||||
import org.apache.hadoop.hbase.master.AssignmentManager.RegionState.State;
|
||||
import org.apache.hadoop.hbase.master.handler.ClosedRegionHandler;
|
||||
import org.apache.hadoop.hbase.master.handler.DisableTableHandler;
|
||||
|
@ -268,6 +269,15 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a map of region plans.
|
||||
*/
|
||||
public void addPlans(Map<String, RegionPlan> plans) {
|
||||
synchronized (regionPlans) {
|
||||
regionPlans.putAll(plans);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the list of regions that will be reopened
|
||||
* because of an update in table schema
|
||||
|
@ -1341,6 +1351,15 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
states.add(forceRegionStateToOffline(region));
|
||||
}
|
||||
}
|
||||
// Add region plans, so we can updateTimers when one region is opened so
|
||||
// that unnecessary timeout on RIT is reduced.
|
||||
Map<String, RegionPlan> plans=new HashMap<String, RegionPlan>();
|
||||
for (HRegionInfo region : regions) {
|
||||
plans.put(region.getEncodedName(), new RegionPlan(region, null,
|
||||
destination));
|
||||
}
|
||||
this.addPlans(plans);
|
||||
|
||||
// Presumption is that only this thread will be updating the state at this
|
||||
// time; i.e. handlers on backend won't be trying to set it to OPEN, etc.
|
||||
AtomicInteger counter = new AtomicInteger(0);
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.hadoop.hbase.master;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
|
@ -56,12 +57,14 @@ public class BulkReOpen extends BulkAssigner {
|
|||
for (Map.Entry<ServerName, List<HRegionInfo>> e : rsToRegions
|
||||
.entrySet()) {
|
||||
final List<HRegionInfo> hris = e.getValue();
|
||||
// add a plan for each of the regions that needs to be reopened
|
||||
// add plans for the regions that need to be reopened
|
||||
Map<String, RegionPlan> plans = new HashMap<String, RegionPlan>();
|
||||
for (HRegionInfo hri : hris) {
|
||||
RegionPlan reOpenPlan = new RegionPlan(hri, null,
|
||||
assignmentManager.getRegionServerOfRegion(hri));
|
||||
assignmentManager.addPlan(hri.getEncodedName(), reOpenPlan);
|
||||
plans.put(hri.getEncodedName(), reOpenPlan);
|
||||
}
|
||||
assignmentManager.addPlans(plans);
|
||||
pool.execute(new Runnable() {
|
||||
public void run() {
|
||||
assignmentManager.unassign(hris);
|
||||
|
|
Loading…
Reference in New Issue