HBASE-22403 Balance in RSGroup should consider throttling and a failure affects the whole
Signed-off-by: Guanghao Zhang <zghao@apache.org>
This commit is contained in:
parent
be432b7c45
commit
a4738e5184
|
@ -409,7 +409,6 @@ public class RSGroupAdminServer implements RSGroupAdmin {
|
|||
@Override
|
||||
public boolean balanceRSGroup(String groupName) throws IOException {
|
||||
ServerManager serverManager = master.getServerManager();
|
||||
AssignmentManager assignmentManager = master.getAssignmentManager();
|
||||
LoadBalancer balancer = master.getLoadBalancer();
|
||||
|
||||
synchronized (balancer) {
|
||||
|
@ -447,16 +446,11 @@ public class RSGroupAdminServer implements RSGroupAdmin {
|
|||
plans.addAll(partialPlans);
|
||||
}
|
||||
}
|
||||
long startTime = System.currentTimeMillis();
|
||||
boolean balancerRan = !plans.isEmpty();
|
||||
if (balancerRan) {
|
||||
LOG.info("RSGroup balance {} starting with plan count: {}", groupName, plans.size());
|
||||
for (RegionPlan plan: plans) {
|
||||
LOG.info("balance {}", plan);
|
||||
assignmentManager.moveAsync(plan);
|
||||
}
|
||||
LOG.info("RSGroup balance {} completed after {} seconds", groupName,
|
||||
(System.currentTimeMillis() - startTime));
|
||||
master.executeRegionPlansWithThrottling(plans);
|
||||
LOG.info("RSGroup balance " + groupName + " completed");
|
||||
}
|
||||
return balancerRan;
|
||||
}
|
||||
|
|
|
@ -1672,7 +1672,6 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
return false;
|
||||
}
|
||||
|
||||
int maxRegionsInTransition = getMaxRegionsInTransition();
|
||||
synchronized (this.balancer) {
|
||||
// If balance not true, don't run balancer.
|
||||
if (!this.loadBalancerTracker.isBalancerOn()) return false;
|
||||
|
@ -1731,45 +1730,11 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
}
|
||||
}
|
||||
|
||||
long balanceStartTime = System.currentTimeMillis();
|
||||
long cutoffTime = balanceStartTime + this.maxBlancingTime;
|
||||
int rpCount = 0; // number of RegionPlans balanced so far
|
||||
if (plans != null && !plans.isEmpty()) {
|
||||
int balanceInterval = this.maxBlancingTime / plans.size();
|
||||
LOG.info("Balancer plans size is " + plans.size() + ", the balance interval is "
|
||||
+ balanceInterval + " ms, and the max number regions in transition is "
|
||||
+ maxRegionsInTransition);
|
||||
|
||||
for (RegionPlan plan: plans) {
|
||||
LOG.info("balance " + plan);
|
||||
//TODO: bulk assign
|
||||
try {
|
||||
this.assignmentManager.moveAsync(plan);
|
||||
} catch (HBaseIOException hioe) {
|
||||
//should ignore failed plans here, avoiding the whole balance plans be aborted
|
||||
//later calls of balance() can fetch up the failed and skipped plans
|
||||
LOG.warn("Failed balance plan: {}, just skip it", plan, hioe);
|
||||
}
|
||||
//rpCount records balance plans processed, does not care if a plan succeeds
|
||||
rpCount++;
|
||||
|
||||
balanceThrottling(balanceStartTime + rpCount * balanceInterval, maxRegionsInTransition,
|
||||
cutoffTime);
|
||||
|
||||
// if performing next balance exceeds cutoff time, exit the loop
|
||||
if (rpCount < plans.size() && System.currentTimeMillis() > cutoffTime) {
|
||||
// TODO: After balance, there should not be a cutoff time (keeping it as
|
||||
// a security net for now)
|
||||
LOG.debug("No more balancing till next balance run; maxBalanceTime="
|
||||
+ this.maxBlancingTime);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
List<RegionPlan> sucRPs = executeRegionPlansWithThrottling(plans);
|
||||
|
||||
if (this.cpHost != null) {
|
||||
try {
|
||||
this.cpHost.postBalance(rpCount < plans.size() ? plans.subList(0, rpCount) : plans);
|
||||
this.cpHost.postBalance(sucRPs);
|
||||
} catch (IOException ioe) {
|
||||
// balancing already succeeded so don't change the result
|
||||
LOG.error("Error invoking master coprocessor postBalance()", ioe);
|
||||
|
@ -1781,6 +1746,47 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
return true;
|
||||
}
|
||||
|
||||
public List<RegionPlan> executeRegionPlansWithThrottling(List<RegionPlan> plans) {
|
||||
List<RegionPlan> sucRPs = new ArrayList<>();
|
||||
int maxRegionsInTransition = getMaxRegionsInTransition();
|
||||
long balanceStartTime = System.currentTimeMillis();
|
||||
long cutoffTime = balanceStartTime + this.maxBlancingTime;
|
||||
int rpCount = 0; // number of RegionPlans balanced so far
|
||||
if (plans != null && !plans.isEmpty()) {
|
||||
int balanceInterval = this.maxBlancingTime / plans.size();
|
||||
LOG.info("Balancer plans size is " + plans.size() + ", the balance interval is "
|
||||
+ balanceInterval + " ms, and the max number regions in transition is "
|
||||
+ maxRegionsInTransition);
|
||||
|
||||
for (RegionPlan plan: plans) {
|
||||
LOG.info("balance " + plan);
|
||||
//TODO: bulk assign
|
||||
try {
|
||||
this.assignmentManager.moveAsync(plan);
|
||||
} catch (HBaseIOException hioe) {
|
||||
//should ignore failed plans here, avoiding the whole balance plans be aborted
|
||||
//later calls of balance() can fetch up the failed and skipped plans
|
||||
LOG.warn("Failed balance plan: {}, just skip it", plan, hioe);
|
||||
}
|
||||
//rpCount records balance plans processed, does not care if a plan succeeds
|
||||
rpCount++;
|
||||
|
||||
balanceThrottling(balanceStartTime + rpCount * balanceInterval, maxRegionsInTransition,
|
||||
cutoffTime);
|
||||
|
||||
// if performing next balance exceeds cutoff time, exit the loop
|
||||
if (rpCount < plans.size() && System.currentTimeMillis() > cutoffTime) {
|
||||
// TODO: After balance, there should not be a cutoff time (keeping it as
|
||||
// a security net for now)
|
||||
LOG.debug("No more balancing till next balance run; maxBalanceTime="
|
||||
+ this.maxBlancingTime);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
return sucRPs;
|
||||
}
|
||||
|
||||
@Override
|
||||
@VisibleForTesting
|
||||
public RegionNormalizer getRegionNormalizer() {
|
||||
|
|
|
@ -529,4 +529,12 @@ public interface MasterServices extends Server {
|
|||
* @return the {@link ZKPermissionWatcher}
|
||||
*/
|
||||
ZKPermissionWatcher getZKPermissionWatcher();
|
||||
|
||||
/**
|
||||
* Execute region plans with throttling
|
||||
* @param plans to execute
|
||||
* @return succeeded plans
|
||||
*/
|
||||
List<RegionPlan> executeRegionPlansWithThrottling(List<RegionPlan> plans);
|
||||
|
||||
}
|
||||
|
|
|
@ -481,6 +481,11 @@ public class MockNoopMasterServices implements MasterServices {
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<RegionPlan> executeRegionPlansWithThrottling(List<RegionPlan> plans) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AsyncClusterConnection getAsyncClusterConnection() {
|
||||
return null;
|
||||
|
|
Loading…
Reference in New Issue