- 
                Notifications
    You must be signed in to change notification settings 
- Fork 9.1k
HDFS-17766. [ARR] Avoid initializing unused threadPool in RouterAsyncRpcClient. #7561
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
| Hi, @KeeProMise @Hexiaoqiao . Could you please help review this simple changes when have free time? Thanks a lot! | 
| 💔 -1 overall 
 
 
 This message was automatically generated. | 
fe5b01f    to
    c6db077      
    Compare
  
    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 @hfutatzhanghb Leave some comments.
| } else { | ||
| this.executorService = new ThreadPoolExecutor(numThreads, numThreads, | ||
| 0L, TimeUnit.MILLISECONDS, workQueue, threadFactory); | ||
| } | 
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.
From my perspective, I think we can add an initExecutorService() method in RouterRpcClient, and then have RouterAsyncRpcClient inherit this method. In RouterRpcClient, we can initialize the executorService normally, but in RouterAsyncRpcClient, this method can be left as an empty implementation.
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.
// in RouterRpcClient
public void initExecutorService(Conf conf) {
     int numThreads = conf.getInt(
        RBFConfigKeys.DFS_ROUTER_CLIENT_THREADS_SIZE,
        RBFConfigKeys.DFS_ROUTER_CLIENT_THREADS_SIZE_DEFAULT);
    ThreadFactory threadFactory = new ThreadFactoryBuilder()
        .setNameFormat("RPC Router Client-%d")
        .build();
    BlockingQueue<Runnable> workQueue;
    if (conf.getBoolean(
        RBFConfigKeys.DFS_ROUTER_CLIENT_REJECT_OVERLOAD,
        RBFConfigKeys.DFS_ROUTER_CLIENT_REJECT_OVERLOAD_DEFAULT)) {
      workQueue = new ArrayBlockingQueue<>(numThreads);
    } else {
      workQueue = new LinkedBlockingQueue<>();
    }
    this.executorService = new ThreadPoolExecutor(numThreads, numThreads,
        0L, TimeUnit.MILLISECONDS, workQueue, threadFactory);
}
// in RouteAsyncRpcClient
public void initExecutorService(Conf conf) {
}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.
@KeeProMise Thanks very much for this nice suggestion. It looks better and i will optimize this PR base on your advise.
| 💔 -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. | 
| 💔 -1 overall 
 
 This message was automatically generated. | 
| 💔 -1 overall 
 
 
 This message was automatically generated. | 
| 🎊 +1 overall 
 
 This message was automatically generated. | 
| @VisibleForTesting | ||
| public boolean isAsync() { | ||
| return getRpcServer() != null ? getRpcServer().isAsync() : | ||
| getConfig() != null ? getConfig().getBoolean(DFS_ROUTER_ASYNC_RPC_ENABLE_KEY, | ||
| DFS_ROUTER_ASYNC_RPC_ENABLE_DEFAULT) : DFS_ROUTER_ASYNC_RPC_ENABLE_DEFAULT; | ||
| } | 
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 @hfutatzhanghb Can this method be deleted?
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.
@KeeProMise Nice advise, have deleted it.
| 🎊 +1 overall 
 
 This message was automatically generated. | 
| 🎊 +1 overall 
 
 This message was automatically generated. | 
| Hi @hfutatzhanghb thanks for your contribution, merged! | 
| @KeeProMise Thanks very much for reviewing and merging! | 
…RpcClient. (apache#7561). Contributed by hfutatzhanghb. Reviewed-by: Jian Zhang <[email protected]>
Description of PR
Avoid initializing unused threadPool in RouterAsyncRpcClient.
Actually, we use async handler to send rpc to NN rather than the original threadPool.