-
Notifications
You must be signed in to change notification settings - Fork 3.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
HBASE-24588 : Submit task for NormalizationPlan #1933
Conversation
🎊 +1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
if (plan.getType() == PlanType.SPLIT) { | ||
splitPlanCount++; | ||
} else if (plan.getType() == PlanType.MERGE) { | ||
mergePlanCount++; | ||
} | ||
} | ||
} | ||
for (Future<?> submittedPlan : submittedPlanList) { | ||
try { | ||
submittedPlan.get(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This changes behaviors for normalizer. Can we give some timeout for blocking so it wont block forever if something goes wrong with one of plans?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
True, there are either of these options we can follow:
- Timeout with Future.get()
- Perform all .get() operations asynchronously
With huge no of submitted plans, do you think it makes sense to even perform .get()
asynchronously by ThreadPoolExecutor and let it block until all plans are done and even update normalizationPlanFailureCount
count. What do you think?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't think the counter as a metric will help an operator very much. Regions can split or merge in the background, causing a planed action to fail. The failure should be logged, I think (maybe the normal split/merge pathways do this already? I haven't checked).
It would be nice to have a final log message for the normalizer run that says something like "successfully executed x of y normalization actions."
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
As Nick commented, need to think about the purpose of this change. I think the purpose is to know the result of plans, which comes with cost.
1). Timeout with Future.get().
When it times out, we do not know if plan succeeds or not. The only info it gives us is that the plan does not finish within a certain amount of time.
2). Perform all.get() asynchronously.
If get() blocks for whatever reason, there will be huge number of threads blocking for get(), system resource leak.
Maybe the best approach is per Nick's comments, log how many plans submitted. If some plans fail, we can go to procedure system for root cause.
try { | ||
submittedPlan.get(); | ||
} catch (Exception e) { | ||
normalizationPlanFailureCount++; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do not see count is being used. Want to add a logging message at the end to see how many plans fail.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah sure, we can put logger but my intention was to expose this as metric. Intentionally kept this as is for now, if you and other reviewers are fine with exposing this count as metric, we should be good to add it.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Going through the client's admin interface seems unnecessary for this as well. There's no way, for example, to log the PID of the action procedure that failed (unless it happens to be included in an exception message).
- Can the action be taken via the
MasterServices
interface? - Maybe it's fine to not wait on these actions, so long as the PIDs are successfully submitted. I wonder what kind of back-pressure the system has, though, if multiple normalizer runs result in 1000's of split/merge actions that can't be completed backing up over time. Right now, our protection against this is the lock against multiple concurrent normalizer runs.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Although going through client interface is not necessary, in some way it might simplify things:
- We won't have to re-write the code to convert encodedRegionName to RegionInfo, derive TableName etc and then use
MasterServices
interface, which requires additional info. - Since our goal is to
submit
async plans and notexecute
blocking operations, Admin interface already has nice non-blocking utility, which is again something we will have to implement on our own if we directly want to useMasterServices
(which does use ProcV2 but we want to finally log: x/y plans succeeded, and for that to happen, usingFuture.get()
and handling Exceptions sound better plan).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Maybe we don't actually want to log "plans succeeded". Maybe it's enough that we log "N plans submitted". I would be okay with that. Even better if the debug level could log the PIDs of the submitted plans, which, I believe, requires going through MasterServices
.
@@ -1294,7 +1294,7 @@ private void checkAndGetTableName(byte[] encodeRegionName, AtomicReference<Table | |||
return; | |||
} | |||
|
|||
MergeTableRegionsRequest request = null; | |||
MergeTableRegionsRequest request; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can this be made final
as well?
try { | ||
submittedPlan.get(); | ||
} catch (Exception e) { | ||
normalizationPlanFailureCount++; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Going through the client's admin interface seems unnecessary for this as well. There's no way, for example, to log the PID of the action procedure that failed (unless it happens to be included in an exception message).
- Can the action be taken via the
MasterServices
interface? - Maybe it's fine to not wait on these actions, so long as the PIDs are successfully submitted. I wonder what kind of back-pressure the system has, though, if multiple normalizer runs result in 1000's of split/merge actions that can't be completed backing up over time. Right now, our protection against this is the lock against multiple concurrent normalizer runs.
if (plan.getType() == PlanType.SPLIT) { | ||
splitPlanCount++; | ||
} else if (plan.getType() == PlanType.MERGE) { | ||
mergePlanCount++; | ||
} | ||
} | ||
} | ||
for (Future<?> submittedPlan : submittedPlanList) { | ||
try { | ||
submittedPlan.get(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't think the counter as a metric will help an operator very much. Regions can split or merge in the background, causing a planed action to fail. The failure should be logged, I think (maybe the normal split/merge pathways do this already? I haven't checked).
It would be nice to have a final log message for the normalizer run that says something like "successfully executed x of y normalization actions."
@@ -44,7 +45,34 @@ public static EmptyNormalizationPlan getInstance(){ | |||
* No-op for empty plan. | |||
*/ | |||
@Override | |||
public void execute(Admin admin) { | |||
public Future<Void> submit(Admin admin) { | |||
return new Future<Void>() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
No need for an anonymous class, just use CompletableFuture.completedFuture
.
🎊 +1 overall
This message was automatically generated. |
hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
Show resolved
Hide resolved
@@ -473,6 +473,10 @@ public void run() { | |||
// Cached clusterId on stand by masters to serve clusterID requests from clients. | |||
private final CachedClusterId cachedClusterId; | |||
|
|||
// Split/Merge Normalization plan executes asynchronously and the caller blocks on | |||
// waiting max 5 sec for single plan to complete with success/failure. | |||
private static final int NORMALIZATION_PLAN_WAIT_TIMEOUT = 5; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: i like to include a unit in these types of constants. i.e., NORMALIZATION_PLAN_WAIT_TIMEOUT_SEC
.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'm still not convinced we should do this, per my other comment :)
💔 -1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
I agree with logging "N plans submitted". That looks perfectly fine. Added commit for the same. So, now remains the only concern: Basically, I am talking about last execution timestamp check in addition to ReentrantLock which we already have. |
🎊 +1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
Not sure if we need to do this kind of check. For most of the clusters, I think 5 mins is too short. |
I was thinking about it for huge clusters and yes it is quite possible for all plan completion to take longer time but I don't have any concrete data as of now. If 5 min is short, should we not update default value of config: As far as plan computation and submission is concerned, we have ReentrantLock to prevent concurrent execution but it's about how much extra time we want to wait before we assume last round of all split/merge tasks should have been completed (failed/succeeded). |
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
I've also been thinking that 5 minutes is too short of a period, I believe originally it was 30 minutes. It's configurable, and an operator can make it as they like. Anyway, I think any changes to default settings are not backwards-compatible and need a separate ticket.
This goes against my previous suggestion of logging the number of actions/procedures submitted and finishing. To do this, the normalizer would have to block on completion of all pids in order to record the timestamp, or else record all pid's submitted in the previous run, and check all their statuses (do we have a procedure completion time stored anywhere) in order to make a decision... No, I think it's better to not try to get into that, especially if we think we'll bump the normalizer run interval to something much larger, like every 24 hours. |
Totally valid point, agree. Thank you @ndimiduk @huaxiangsun |
@@ -833,6 +833,9 @@ void unassign(byte[] regionName, boolean force) | |||
|
|||
/** | |||
* Invoke region normalizer. Can NOT run for various reasons. Check logs. | |||
* This is a non-blocking invocation to region normalizer. If return value is true, it means | |||
* the invocation was successful. We need to check logs for the details of which regions |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: "means the request was submitted successfully."
@@ -44,7 +44,8 @@ public static EmptyNormalizationPlan getInstance(){ | |||
* No-op for empty plan. | |||
*/ | |||
@Override | |||
public void execute(Admin admin) { | |||
public long submit(MasterServices masterServices) throws IOException { | |||
return -1; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@huaxiangsun does the pid -1
carry a special meaning? Just in case...
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Actually, this class isn't even used. I think you can delete it.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Just a couple nits left. Otherwise, yes, looks great, +1! Nice improvement here, thanks @virajjasani.
🎊 +1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
Looks like there were consistent unit test failures in TestNormalizerOnCluster. let's keep an eye on it. |
Signed-off-by: Nick Dimiduk <ndimiduk@apache.org>
Signed-off-by: Nick Dimiduk <ndimiduk@apache.org>
Yeah, they pass locally all the time, on the build, it was [failed-to-read] xml error. So yes, will keep an eye. |
Signed-off-by: Nick Dimiduk <ndimiduk@apache.org>
Signed-off-by: Nick Dimiduk <ndimiduk@apache.org>
…#1983) Signed-off-by: Nick Dimiduk <ndimiduk@apache.org>
No description provided.