diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/BulkReOpen.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/BulkReOpen.java index 218d5e4..19741dc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/BulkReOpen.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/BulkReOpen.java @@ -35,17 +35,13 @@ import org.apache.hadoop.hbase.ServerName; * Performs bulk reopen of the list of regions provided to it. */ @InterfaceAudience.Private -public class BulkReOpen extends BulkAssigner { - private final Map> rsToRegions; - private final AssignmentManager assignmentManager; +public class BulkReOpen extends GeneralBulkAssigner { private static final Log LOG = LogFactory.getLog(BulkReOpen.class); public BulkReOpen(final Server server, final Map> serverToRegions, - final AssignmentManager am) { - super(server); - this.assignmentManager = am; - this.rsToRegions = serverToRegions; + final AssignmentManager am, final boolean waitTillAllReOpened) { + super(server, serverToRegions, am, waitTillAllReOpened); } /** @@ -55,8 +51,8 @@ public class BulkReOpen extends BulkAssigner { @Override protected void populatePool(ExecutorService pool) { LOG.debug("Creating threads for each region server "); - for (Map.Entry> e : rsToRegions - .entrySet()) { + this.pool = pool; + for (Map.Entry> e : bulkPlan.entrySet()) { final List hris = e.getValue(); // add plans for the regions that need to be reopened Map plans = new HashMap(); @@ -78,15 +74,6 @@ public class BulkReOpen extends BulkAssigner { } } - /** - * Reopen the regions asynchronously, so always returns true immediately. - * @return true - */ - @Override - protected boolean waitUntilDone(long timeout) { - return true; - } - /** * Configuration knobs "hbase.bulk.reopen.threadpool.size" number of regions * that can be reopened concurrently. The maximum number of threads the master diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/GeneralBulkAssigner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/GeneralBulkAssigner.java index cfbcb6d..1c0e155 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/GeneralBulkAssigner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/GeneralBulkAssigner.java @@ -45,9 +45,9 @@ import org.apache.hadoop.hbase.master.RegionState.State; public class GeneralBulkAssigner extends BulkAssigner { private static final Log LOG = LogFactory.getLog(GeneralBulkAssigner.class); - private Map> failedPlans + protected Map> failedPlans = new ConcurrentHashMap>(); - private ExecutorService pool; + protected ExecutorService pool; final Map> bulkPlan; final AssignmentManager assignmentManager; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TableEventHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TableEventHandler.java index 9ea1f08..6caaf2a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TableEventHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TableEventHandler.java @@ -196,7 +196,7 @@ public abstract class TableEventHandler extends EventHandler { + serverToRegions.size() + " region servers."); this.masterServices.getAssignmentManager().setRegionsToReopen(reRegions); BulkReOpen bulkReopen = new BulkReOpen(this.server, serverToRegions, - this.masterServices.getAssignmentManager()); + this.masterServices.getAssignmentManager(), true); while (true) { try { if (bulkReopen.bulkReOpen()) {