-
Notifications
You must be signed in to change notification settings - Fork 9k
HDFS-17595. [ARR] ErasureCoding supports asynchronous rpc. #6983
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
Conversation
f2814d8
to
058c751
Compare
// Map<FederationNamespaceInfo, ErasureCodingPolicyInfo[]> ret = | ||
rpcClient.invokeConcurrent( | ||
nss, method, true, false, ErasureCodingPolicyInfo[].class); | ||
asyncApply((ApplyFunction<Map<FederationNamespaceInfo, ErasureCodingPolicyInfo[]>, ErasureCodingPolicyInfo[]>) ret -> { | ||
return merge(ret, ErasureCodingPolicyInfo.class); | ||
}); | ||
|
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.
Thank you for your contribution. You can delete the comment there and pay attention to the length of the line.
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.
Sir, thanks for reviewing. have fixed.
// Map<FederationNamespaceInfo, ECTopologyVerifierResult> ret | ||
rpcClient.invokeConcurrent(nss, method, true, false, | ||
ECTopologyVerifierResult.class); | ||
asyncApply((ApplyFunction<Map<FederationNamespaceInfo, ECTopologyVerifierResult>, ECTopologyVerifierResult>) ret -> { | ||
for (Map.Entry<FederationNamespaceInfo, ECTopologyVerifierResult> entry : ret | ||
.entrySet()) { | ||
if (!entry.getValue().isSupported()) { | ||
return entry.getValue(); | ||
} | ||
} | ||
// If no negative result, return the result from the first namespace. | ||
return ret.get(nss.iterator().next()); | ||
}); |
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.
Same as above.
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.
Fixed.
Hi @hfutatzhanghb, Thanks for your contribution, you can add a UT for testing AsyncErasureCoding. |
OK, will add UTs these days. |
4e0b405
to
a591902
Compare
Hi, @hfutatzhanghb HDFS-17545 has already been merged into HDFS-17531, please rebase your development branch using HDFS-17531. |
6d6f9c7
to
60bc890
Compare
@hfutatzhanghb You can add a UT, like #6994, TestRouterAsyncSnapshot. |
import static org.apache.hadoop.hdfs.server.federation.router.async.AsyncUtil.asyncApply; | ||
import static org.apache.hadoop.hdfs.server.federation.router.async.AsyncUtil.asyncReturn; | ||
|
||
public class AsyncErasureCoding extends ErasureCoding{ |
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.
"ErasureCoding{" -> "ErasureCoding {"
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.
Thanks, have fixed.
💔 -1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
@Hexiaoqiao @KeeProMise Sir, have added UT. PTAL when you have free time, thanks~ |
💔 -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. |
@hfutatzhanghb LGTM, @Hexiaoqiao if you have time, please take a look at the PR. |
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.
Thanks @KeeProMise and @hfutatzhanghb for your works. Leave nit comments inline. Another un-related thing, is it better to organize async feature to one single package under router
? Thanks again.
return asyncReturn(ErasureCodingPolicy.class); | ||
} | ||
|
||
|
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.
extra blank line.
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.
Thanks sir. fixed.
import static org.apache.hadoop.hdfs.server.federation.router.async.AsyncUtil.asyncApply; | ||
import static org.apache.hadoop.hdfs.server.federation.router.async.AsyncUtil.asyncReturn; | ||
|
||
public class AsyncErasureCoding extends ErasureCoding { |
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.
Some other interface such as removeErasureCodingPolicy
do not need to override with async?
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.
Yes, such interface do not need to refactor. They can work well in both sync and async mode.
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.
Hi, @Hexiaoqiao because we used option 2, removeErasureCodingPolicy does not need to thenApply the request result, so the caller can directly obtain the CompletableFuture object from the thread variable.here's an explanation:
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.
Got it. Thanks.
🎊 +1 overall
This message was automatically generated. |
@hfutatzhanghb Thanks for your contribution! |
…). Contributed by hfutatzhanghb. Reviewed-by: Jian Zhang <keepromise@apache.org> Signed-off-by: He Xiaoqiao <hexiaoqiao@apache.org>
…). Contributed by hfutatzhanghb. Reviewed-by: Jian Zhang <keepromise@apache.org> Signed-off-by: He Xiaoqiao <hexiaoqiao@apache.org>
…). Contributed by hfutatzhanghb. Reviewed-by: Jian Zhang <keepromise@apache.org> Signed-off-by: He Xiaoqiao <hexiaoqiao@apache.org>
…). Contributed by hfutatzhanghb. Reviewed-by: Jian Zhang <keepromise@apache.org> Signed-off-by: He Xiaoqiao <hexiaoqiao@apache.org>
…). Contributed by hfutatzhanghb. Reviewed-by: Jian Zhang <keepromise@apache.org> Signed-off-by: He Xiaoqiao <hexiaoqiao@apache.org>
…ontributed by hfutatzhanghb. Reviewed-by: Jian Zhang <keepromise@apache.org> Signed-off-by: He Xiaoqiao <hexiaoqiao@apache.org>
…). Contributed by hfutatzhanghb. Reviewed-by: Jian Zhang <keepromise@apache.org> Signed-off-by: He Xiaoqiao <hexiaoqiao@apache.org>
…). Contributed by hfutatzhanghb. Reviewed-by: Jian Zhang <keepromise@apache.org> Signed-off-by: He Xiaoqiao <hexiaoqiao@apache.org>
…). Contributed by hfutatzhanghb. Reviewed-by: Jian Zhang <keepromise@apache.org> Signed-off-by: He Xiaoqiao <hexiaoqiao@apache.org>
…). Contributed by hfutatzhanghb. Reviewed-by: Jian Zhang <keepromise@apache.org> Signed-off-by: He Xiaoqiao <hexiaoqiao@apache.org>
…). Contributed by hfutatzhanghb. Reviewed-by: Jian Zhang <keepromise@apache.org> Signed-off-by: He Xiaoqiao <hexiaoqiao@apache.org>
Description of PR
The main new addition is AsyncErasureCoding, which extends ErasureCoding so that supports asynchronous rpc.