Skip to content

Commit 812f525

Browse files
hfutatzhanghbKeeProMise
authored andcommitted
HDFS-17651.[ARR] Async handler executor isolation (apache#7244). Contributed by hfutatzhanghb.
Reviewed-by: Jian Zhang <keepromise@apache.org> Signed-off-by: He Xiaoqiao <hexiaoqiao@apache.org>
1 parent 310837d commit 812f525

File tree

13 files changed

+190
-93
lines changed

13 files changed

+190
-93
lines changed

hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/AsyncRpcProtocolPBUtil.java

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -55,7 +55,7 @@
5555
public final class AsyncRpcProtocolPBUtil {
5656
public static final Logger LOG = LoggerFactory.getLogger(AsyncRpcProtocolPBUtil.class);
5757
/** The executor used for handling responses asynchronously. */
58-
private static Executor worker;
58+
private static Executor asyncResponderExecutor;
5959

6060
private AsyncRpcProtocolPBUtil() {}
6161

@@ -97,7 +97,7 @@ public static <T, R> R asyncIpcClient(
9797
} catch (Exception ex) {
9898
throw warpCompletionException(ex);
9999
}
100-
}, worker));
100+
}, asyncResponderExecutor));
101101
return asyncReturn(clazz);
102102
}
103103

@@ -144,10 +144,10 @@ public static <T> void asyncRouterServer(ServerReq<T> req, ServerRes<T> res) {
144144
* Sets the executor used for handling responses asynchronously within
145145
* the utility class.
146146
*
147-
* @param worker The executor to be used for handling responses asynchronously.
147+
* @param asyncResponderExecutor The executor to be used for handling responses asynchronously.
148148
*/
149-
public static void setWorker(Executor worker) {
150-
AsyncRpcProtocolPBUtil.worker = worker;
149+
public static void setAsyncResponderExecutor(Executor asyncResponderExecutor) {
150+
AsyncRpcProtocolPBUtil.asyncResponderExecutor = asyncResponderExecutor;
151151
}
152152

153153
@FunctionalInterface

hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java

Lines changed: 16 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -72,15 +72,22 @@ public class RBFConfigKeys extends CommonConfigurationKeysPublic {
7272
public static final String DFS_ROUTER_RPC_ENABLE =
7373
FEDERATION_ROUTER_PREFIX + "rpc.enable";
7474
public static final boolean DFS_ROUTER_RPC_ENABLE_DEFAULT = true;
75-
public static final String DFS_ROUTER_RPC_ENABLE_ASYNC =
76-
FEDERATION_ROUTER_PREFIX + "rpc.async.enable";
77-
public static final boolean DFS_ROUTER_RPC_ENABLE_ASYNC_DEFAULT = false;
78-
public static final String DFS_ROUTER_RPC_ASYNC_HANDLER_COUNT =
79-
FEDERATION_ROUTER_PREFIX + "rpc.async.handler.count";
80-
public static final int DFS_ROUTER_RPC_ASYNC_HANDLER_COUNT_DEFAULT = 2;
81-
public static final String DFS_ROUTER_RPC_ASYNC_RESPONDER_COUNT =
82-
FEDERATION_ROUTER_PREFIX + "rpc.async.responder.count";
83-
public static final int DFS_ROUTER_RPC_ASYNC_RESPONDER_COUNT_DEFAULT = 10;
75+
// HDFS Router Asynchronous RPC
76+
public static final String DFS_ROUTER_ASYNC_RPC_ENABLE_KEY =
77+
FEDERATION_ROUTER_PREFIX + "async.rpc.enable";
78+
public static final boolean DFS_ROUTER_ASYNC_RPC_ENABLE_DEFAULT = false;
79+
public static final String FEDERATION_ROUTER_ASYNC_RPC_PREFIX =
80+
FEDERATION_ROUTER_PREFIX + "async.rpc.";
81+
// Example: ns1:count1,ns2:count2,ns3:count3
82+
public static final String DFS_ROUTER_ASYNC_RPC_NS_HANDLER_COUNT_KEY =
83+
FEDERATION_ROUTER_ASYNC_RPC_PREFIX + "ns.handler.count";
84+
public static final String DFS_ROUTER_ASYNC_RPC_NS_HANDLER_COUNT_DEFAULT = "";
85+
public static final String DFS_ROUTER_ASYNC_RPC_HANDLER_COUNT_KEY =
86+
FEDERATION_ROUTER_ASYNC_RPC_PREFIX + "handler.count";
87+
public static final int DFS_ROUTER_ASYNC_RPC_HANDLER_COUNT_DEFAULT = 10;
88+
public static final String DFS_ROUTER_ASYNC_RPC_RESPONDER_COUNT_KEY =
89+
FEDERATION_ROUTER_ASYNC_RPC_PREFIX + "responder.count";
90+
public static final int DFS_ROUTER_ASYNCRPC_RESPONDER_COUNT_DEFAULT = 10;
8491

8592
public static final String DFS_ROUTER_METRICS_ENABLE =
8693
FEDERATION_ROUTER_PREFIX + "metrics.enable";

hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java

Lines changed: 104 additions & 35 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,16 @@
1818
package org.apache.hadoop.hdfs.server.federation.router;
1919

2020
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION;
21+
import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNCRPC_RESPONDER_COUNT_DEFAULT;
22+
import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNC_RPC_ENABLE_DEFAULT;
23+
import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNC_RPC_ENABLE_KEY;
24+
import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNC_RPC_HANDLER_COUNT_DEFAULT;
25+
import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNC_RPC_HANDLER_COUNT_KEY;
26+
import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNC_RPC_NS_HANDLER_COUNT_DEFAULT;
27+
import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNC_RPC_NS_HANDLER_COUNT_KEY;
28+
import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNC_RPC_RESPONDER_COUNT_KEY;
29+
import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_FEDERATION_RENAME_OPTION;
30+
import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_FEDERATION_RENAME_OPTION_DEFAULT;
2131
import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_HANDLER_COUNT_DEFAULT;
2232
import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_HANDLER_COUNT_KEY;
2333
import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_HANDLER_QUEUE_SIZE_DEFAULT;
@@ -26,16 +36,8 @@
2636
import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_READER_COUNT_KEY;
2737
import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_READER_QUEUE_SIZE_DEFAULT;
2838
import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_READER_QUEUE_SIZE_KEY;
29-
import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_RPC_ASYNC_HANDLER_COUNT;
30-
import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_RPC_ASYNC_HANDLER_COUNT_DEFAULT;
31-
import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_RPC_ASYNC_RESPONDER_COUNT;
32-
import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_RPC_ASYNC_RESPONDER_COUNT_DEFAULT;
33-
import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_RPC_ENABLE_ASYNC;
34-
import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_RPC_ENABLE_ASYNC_DEFAULT;
3539
import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DN_REPORT_CACHE_EXPIRE;
3640
import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DN_REPORT_CACHE_EXPIRE_MS_DEFAULT;
37-
import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_FEDERATION_RENAME_OPTION;
38-
import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_FEDERATION_RENAME_OPTION_DEFAULT;
3941
import static org.apache.hadoop.hdfs.server.federation.router.RouterFederationRename.RouterRenameOption;
4042
import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncApply;
4143
import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncCatch;
@@ -56,22 +58,24 @@
5658
import java.util.ArrayList;
5759
import java.util.Collection;
5860
import java.util.EnumSet;
61+
import java.util.HashSet;
5962
import java.util.Iterator;
6063
import java.util.LinkedHashMap;
6164
import java.util.LinkedHashSet;
6265
import java.util.List;
6366
import java.util.Map;
6467
import java.util.Map.Entry;
6568
import java.util.Set;
69+
import java.util.concurrent.ConcurrentHashMap;
6670
import java.util.concurrent.ExecutionException;
67-
import java.util.concurrent.Executor;
6871
import java.util.concurrent.ExecutorService;
6972
import java.util.concurrent.Executors;
7073
import java.util.concurrent.ThreadFactory;
7174
import java.util.concurrent.TimeUnit;
7275
import java.util.concurrent.atomic.AtomicInteger;
7376
import java.util.stream.Collectors;
7477

78+
import org.apache.commons.lang3.StringUtils;
7579
import org.apache.hadoop.fs.Path;
7680
import org.apache.hadoop.hdfs.HAUtil;
7781
import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
@@ -209,6 +213,7 @@
209213
import org.apache.hadoop.tools.protocolPB.GetUserMappingsProtocolPB;
210214
import org.apache.hadoop.tools.protocolPB.GetUserMappingsProtocolServerSideTranslatorPB;
211215
import org.apache.hadoop.util.ReflectionUtils;
216+
import org.checkerframework.checker.nullness.qual.NonNull;
212217
import org.slf4j.Logger;
213218
import org.slf4j.LoggerFactory;
214219

@@ -228,8 +233,9 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol,
228233

229234
private static final Logger LOG =
230235
LoggerFactory.getLogger(RouterRpcServer.class);
231-
private ExecutorService asyncRouterHandler;
232-
private ExecutorService asyncRouterResponder;
236+
237+
/** Name service keyword to identify fan-out calls. */
238+
public static final String CONCURRENT_NS = "concurrent";
233239

234240
/** Configuration for the RPC server. */
235241
private Configuration conf;
@@ -287,6 +293,10 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol,
287293
/** Schedule the router federation rename jobs. */
288294
private BalanceProcedureScheduler fedRenameScheduler;
289295
private boolean enableAsync;
296+
private Map<String, Integer> nsAsyncHandlerCount = new ConcurrentHashMap<>();
297+
private Map<String, ExecutorService> asyncRouterHandlerExecutors = new ConcurrentHashMap<>();
298+
private ExecutorService routerAsyncResponderExecutor;
299+
private ExecutorService routerDefaultAsyncHandlerExecutor;
290300

291301
/**
292302
* Construct a router RPC server.
@@ -318,11 +328,11 @@ public RouterRpcServer(Configuration conf, Router router,
318328
int handlerQueueSize = this.conf.getInt(DFS_ROUTER_HANDLER_QUEUE_SIZE_KEY,
319329
DFS_ROUTER_HANDLER_QUEUE_SIZE_DEFAULT);
320330

321-
this.enableAsync = conf.getBoolean(DFS_ROUTER_RPC_ENABLE_ASYNC,
322-
DFS_ROUTER_RPC_ENABLE_ASYNC_DEFAULT);
323-
LOG.info("Router enable async {}", this.enableAsync);
331+
this.enableAsync = conf.getBoolean(DFS_ROUTER_ASYNC_RPC_ENABLE_KEY,
332+
DFS_ROUTER_ASYNC_RPC_ENABLE_DEFAULT);
333+
LOG.info("Router enable async rpc: {}", this.enableAsync);
324334
if (this.enableAsync) {
325-
initAsyncThreadPool();
335+
initAsyncThreadPools(conf);
326336
}
327337
// Override Hadoop Common IPC setting
328338
int readerQueueSize = this.conf.getInt(DFS_ROUTER_READER_QUEUE_SIZE_KEY,
@@ -446,8 +456,7 @@ public RouterRpcServer(Configuration conf, Router router,
446456
// Create the client
447457
if (this.enableAsync) {
448458
this.rpcClient = new RouterAsyncRpcClient(this.conf, this.router,
449-
this.namenodeResolver, this.rpcMonitor,
450-
routerStateIdContext, asyncRouterHandler);
459+
this.namenodeResolver, this.rpcMonitor, routerStateIdContext);
451460
this.clientProto = new RouterAsyncClientProtocol(conf, this);
452461
this.nnProto = new RouterAsyncNamenodeProtocol(this);
453462
this.routerProto = new RouterAsyncUserProtocol(this);
@@ -491,23 +500,77 @@ public RouterRpcServer(Configuration conf, Router router,
491500

492501
/**
493502
* Init router async handlers and router async responders.
503+
* @param configuration the configuration.
494504
*/
495-
public void initAsyncThreadPool() {
496-
int asyncHandlerCount = conf.getInt(DFS_ROUTER_RPC_ASYNC_HANDLER_COUNT,
497-
DFS_ROUTER_RPC_ASYNC_HANDLER_COUNT_DEFAULT);
498-
int asyncResponderCount = conf.getInt(DFS_ROUTER_RPC_ASYNC_RESPONDER_COUNT,
499-
DFS_ROUTER_RPC_ASYNC_RESPONDER_COUNT_DEFAULT);
500-
if (asyncRouterHandler == null) {
501-
LOG.info("init router async handler count: {}", asyncHandlerCount);
502-
asyncRouterHandler = Executors.newFixedThreadPool(
503-
asyncHandlerCount, new AsyncThreadFactory("router async handler "));
505+
public void initAsyncThreadPools(Configuration configuration) {
506+
LOG.info("Begin initialize asynchronous handler and responder thread pool.");
507+
initNsAsyncHandlerCount();
508+
Set<String> allConfiguredNS = FederationUtil.getAllConfiguredNS(configuration);
509+
Set<String> unassignedNS = new HashSet<>();
510+
allConfiguredNS.add(CONCURRENT_NS);
511+
512+
for (String nsId : allConfiguredNS) {
513+
int dedicatedHandlers = nsAsyncHandlerCount.getOrDefault(nsId, 0);
514+
LOG.info("Dedicated handlers {} for ns {} ", dedicatedHandlers, nsId);
515+
if (dedicatedHandlers > 0) {
516+
initAsyncHandlerThreadPools4Ns(nsId, dedicatedHandlers);
517+
LOG.info("Assigned {} async handlers to nsId {} ", dedicatedHandlers, nsId);
518+
} else {
519+
unassignedNS.add(nsId);
520+
}
521+
}
522+
523+
int asyncHandlerCountDefault = configuration.getInt(DFS_ROUTER_ASYNC_RPC_HANDLER_COUNT_KEY,
524+
DFS_ROUTER_ASYNC_RPC_HANDLER_COUNT_DEFAULT);
525+
526+
if (!unassignedNS.isEmpty()) {
527+
LOG.warn("Async handler unassigned ns: {}", unassignedNS);
528+
LOG.info("Use default async handler count {} for unassigned ns.", asyncHandlerCountDefault);
529+
for (String nsId : unassignedNS) {
530+
initAsyncHandlerThreadPools4Ns(nsId, asyncHandlerCountDefault);
531+
}
504532
}
505-
if (asyncRouterResponder == null) {
506-
LOG.info("init router async responder count: {}", asyncResponderCount);
507-
asyncRouterResponder = Executors.newFixedThreadPool(
508-
asyncResponderCount, new AsyncThreadFactory("router async responder "));
533+
534+
int asyncResponderCount = configuration.getInt(DFS_ROUTER_ASYNC_RPC_RESPONDER_COUNT_KEY,
535+
DFS_ROUTER_ASYNCRPC_RESPONDER_COUNT_DEFAULT);
536+
if (routerAsyncResponderExecutor == null) {
537+
LOG.info("Initialize router async responder count: {}", asyncResponderCount);
538+
routerAsyncResponderExecutor = Executors.newFixedThreadPool(
539+
asyncResponderCount, new AsyncThreadFactory("Router Async Responder #"));
540+
}
541+
AsyncRpcProtocolPBUtil.setAsyncResponderExecutor(routerAsyncResponderExecutor);
542+
543+
if (routerDefaultAsyncHandlerExecutor == null) {
544+
LOG.info("init router async default executor handler count: {}", asyncHandlerCountDefault);
545+
routerDefaultAsyncHandlerExecutor = Executors.newFixedThreadPool(
546+
asyncHandlerCountDefault, new AsyncThreadFactory("Router Async Default Handler #"));
547+
}
548+
}
549+
550+
private void initNsAsyncHandlerCount() {
551+
String configNsHandler = conf.get(DFS_ROUTER_ASYNC_RPC_NS_HANDLER_COUNT_KEY,
552+
DFS_ROUTER_ASYNC_RPC_NS_HANDLER_COUNT_DEFAULT);
553+
if (StringUtils.isEmpty(configNsHandler)) {
554+
LOG.error(
555+
"The value of config key: {} is empty. Will use default conf.",
556+
DFS_ROUTER_ASYNC_RPC_NS_HANDLER_COUNT_KEY);
509557
}
510-
AsyncRpcProtocolPBUtil.setWorker(asyncRouterResponder);
558+
String[] nsHandlers = configNsHandler.split(",");
559+
for (String nsHandlerInfo : nsHandlers) {
560+
String[] nsHandlerItems = nsHandlerInfo.split(":");
561+
if (nsHandlerItems.length != 2 || StringUtils.isBlank(nsHandlerItems[0]) ||
562+
!StringUtils.isNumeric(nsHandlerItems[1])) {
563+
LOG.error("The config key: {} is incorrect! The value is {}.",
564+
DFS_ROUTER_ASYNC_RPC_NS_HANDLER_COUNT_KEY, nsHandlerInfo);
565+
continue;
566+
}
567+
nsAsyncHandlerCount.put(nsHandlerItems[0], Integer.parseInt(nsHandlerItems[1]));
568+
}
569+
}
570+
571+
private void initAsyncHandlerThreadPools4Ns(String nsId, int dedicatedHandlers) {
572+
asyncRouterHandlerExecutors.computeIfAbsent(nsId, id -> Executors.newFixedThreadPool(
573+
dedicatedHandlers, new AsyncThreadFactory("Router Async Handler for " + id + " #")));
511574
}
512575

513576
/**
@@ -2426,8 +2489,12 @@ public boolean isAsync() {
24262489
return this.enableAsync;
24272490
}
24282491

2429-
public Executor getAsyncRouterHandler() {
2430-
return asyncRouterHandler;
2492+
public Map<String, ExecutorService> getAsyncRouterHandlerExecutors() {
2493+
return asyncRouterHandlerExecutors;
2494+
}
2495+
2496+
public ExecutorService getRouterAsyncHandlerDefaultExecutor() {
2497+
return routerDefaultAsyncHandlerExecutor;
24312498
}
24322499

24332500
private static class AsyncThreadFactory implements ThreadFactory {
@@ -2439,8 +2506,10 @@ private static class AsyncThreadFactory implements ThreadFactory {
24392506
}
24402507

24412508
@Override
2442-
public Thread newThread(Runnable r) {
2443-
return new Thread(r, namePrefix + threadNumber.getAndIncrement());
2509+
public Thread newThread(@NonNull Runnable r) {
2510+
Thread thread = new Thread(r, namePrefix + threadNumber.getAndIncrement());
2511+
thread.setDaemon(true);
2512+
return thread;
24442513
}
24452514
}
24462515
}

hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/RouterAsyncRpcClient.java

Lines changed: 4 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -57,7 +57,6 @@
5757
import java.util.Map;
5858
import java.util.concurrent.Callable;
5959
import java.util.concurrent.CompletableFuture;
60-
import java.util.concurrent.Executor;
6160

6261
import static org.apache.hadoop.hdfs.server.federation.fairness.RouterRpcFairnessConstants.CONCURRENT_NS;
6362
import static org.apache.hadoop.hdfs.server.federation.router.async.utils.Async.warpCompletionException;
@@ -98,7 +97,6 @@ public class RouterAsyncRpcClient extends RouterRpcClient{
9897
private final ActiveNamenodeResolver namenodeResolver;
9998
/** Optional perf monitor. */
10099
private final RouterRpcMonitor rpcMonitor;
101-
private final Executor asyncRouterHandler;
102100

103101
/**
104102
* Create a router async RPC client to manage remote procedure calls to NNs.
@@ -108,17 +106,15 @@ public class RouterAsyncRpcClient extends RouterRpcClient{
108106
* @param resolver A NN resolver to determine the currently active NN in HA.
109107
* @param monitor Optional performance monitor.
110108
* @param routerStateIdContext the router state context object to hold the state ids for all
111-
* @param asyncRouterHandler async router handler
112109
* namespaces.
113110
*/
114111
public RouterAsyncRpcClient(Configuration conf,
115112
Router router, ActiveNamenodeResolver resolver, RouterRpcMonitor monitor,
116-
RouterStateIdContext routerStateIdContext, Executor asyncRouterHandler) {
113+
RouterStateIdContext routerStateIdContext) {
117114
super(conf, router, resolver, monitor, routerStateIdContext);
118115
this.router = router;
119116
this.namenodeResolver = resolver;
120117
this.rpcMonitor = monitor;
121-
this.asyncRouterHandler = asyncRouterHandler;
122118
}
123119

124120
/**
@@ -172,6 +168,7 @@ public Object invokeMethod(
172168
" with params " + Arrays.deepToString(params) + " from "
173169
+ router.getRouterId());
174170
}
171+
String nsid = namenodes.get(0).getNameserviceId();
175172
// transfer threadLocalContext to worker threads of executor.
176173
ThreadLocalContext threadLocalContext = new ThreadLocalContext();
177174
asyncComplete(null);
@@ -183,7 +180,8 @@ public Object invokeMethod(
183180
threadLocalContext.transfer();
184181
invokeMethodAsync(ugi, (List<FederationNamenodeContext>) namenodes,
185182
useObserver, protocol, method, params);
186-
}, asyncRouterHandler);
183+
}, router.getRpcServer().getAsyncRouterHandlerExecutors().getOrDefault(nsid,
184+
router.getRpcServer().getRouterAsyncHandlerDefaultExecutor()));
187185
return null;
188186
}
189187

0 commit comments

Comments
 (0)