2525import static org .apache .hadoop .hdfs .server .federation .router .RBFConfigKeys .DFS_ROUTER_ASYNC_RPC_HANDLER_COUNT_KEY ;
2626import static org .apache .hadoop .hdfs .server .federation .router .RBFConfigKeys .DFS_ROUTER_ASYNC_RPC_NS_HANDLER_COUNT_DEFAULT ;
2727import 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_QUEUE_SIZE ;
29+ import static org .apache .hadoop .hdfs .server .federation .router .RBFConfigKeys .DFS_ROUTER_ASYNC_RPC_QUEUE_SIZE_DEFAULT ;
2830import static org .apache .hadoop .hdfs .server .federation .router .RBFConfigKeys .DFS_ROUTER_ASYNC_RPC_RESPONDER_COUNT_KEY ;
2931import static org .apache .hadoop .hdfs .server .federation .router .RBFConfigKeys .DFS_ROUTER_FEDERATION_RENAME_OPTION ;
3032import static org .apache .hadoop .hdfs .server .federation .router .RBFConfigKeys .DFS_ROUTER_FEDERATION_RENAME_OPTION_DEFAULT ;
5860import java .util .ArrayList ;
5961import java .util .Collection ;
6062import java .util .EnumSet ;
61- import java .util .HashSet ;
6263import java .util .Iterator ;
6364import java .util .LinkedHashMap ;
6465import java .util .LinkedHashSet ;
7071import java .util .concurrent .ExecutionException ;
7172import java .util .concurrent .ExecutorService ;
7273import java .util .concurrent .Executors ;
74+ import java .util .concurrent .LinkedBlockingQueue ;
7375import java .util .concurrent .ThreadFactory ;
76+ import java .util .concurrent .ThreadPoolExecutor ;
7477import java .util .concurrent .TimeUnit ;
7578import java .util .concurrent .atomic .AtomicInteger ;
7679import java .util .stream .Collectors ;
@@ -292,11 +295,10 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol,
292295 private RouterRenameOption routerRenameOption ;
293296 /** Schedule the router federation rename jobs. */
294297 private BalanceProcedureScheduler fedRenameScheduler ;
295- private boolean enableAsync ;
296- private Map <String , Integer > nsAsyncHandlerCount = new ConcurrentHashMap <>();
297- private Map < String , ExecutorService > asyncRouterHandlerExecutors = new ConcurrentHashMap <>() ;
298+ private final boolean enableAsync ;
299+ private final Map <String , ThreadPoolExecutor > asyncRouterHandlerExecutors = new ConcurrentHashMap <>();
300+ private ThreadPoolExecutor routerDefaultAsyncHandlerExecutor ;
298301 private ExecutorService routerAsyncResponderExecutor ;
299- private ExecutorService routerDefaultAsyncHandlerExecutor ;
300302
301303 /**
302304 * Construct a router RPC server.
@@ -504,31 +506,38 @@ public RouterRpcServer(Configuration conf, Router router,
504506 */
505507 public void initAsyncThreadPools (Configuration configuration ) {
506508 LOG .info ("Begin initialize asynchronous handler and responder thread pool." );
507- initNsAsyncHandlerCount ( );
509+ Map < String , Integer > nsAsyncHandlerCount = parseNsAsyncHandlerCount ( configuration );
508510 Set <String > allConfiguredNS = FederationUtil .getAllConfiguredNS (configuration );
509- Set <String > unassignedNS = new HashSet <>();
510511 allConfiguredNS .add (CONCURRENT_NS );
511512
513+ int asyncQueueSize = configuration .getInt (DFS_ROUTER_ASYNC_RPC_QUEUE_SIZE ,
514+ DFS_ROUTER_ASYNC_RPC_QUEUE_SIZE_DEFAULT );
515+ if (asyncQueueSize <= 1 ) {
516+ throw new IllegalArgumentException ("Async queue size must be greater than 1" );
517+ }
518+ int asyncHandlerCountDefault = configuration .getInt (DFS_ROUTER_ASYNC_RPC_HANDLER_COUNT_KEY ,
519+ DFS_ROUTER_ASYNC_RPC_HANDLER_COUNT_DEFAULT );
512520 for (String nsId : allConfiguredNS ) {
513521 int dedicatedHandlers = nsAsyncHandlerCount .getOrDefault (nsId , 0 );
514- LOG .info ("Dedicated handlers {} for ns {} " , dedicatedHandlers , nsId );
522+ if (dedicatedHandlers <= 0 ) {
523+ dedicatedHandlers = asyncHandlerCountDefault ;
524+ LOG .info ("Use default async handler count {} for ns {}." , asyncHandlerCountDefault , nsId );
525+ } else {
526+ LOG .info ("Dedicated handlers {} for ns {} " , dedicatedHandlers , nsId );
527+ }
528+
515529 if (dedicatedHandlers > 0 ) {
516- initAsyncHandlerThreadPools4Ns (nsId , dedicatedHandlers );
530+ int finalDedicatedHandlers = dedicatedHandlers ;
531+ asyncRouterHandlerExecutors .computeIfAbsent (nsId ,
532+ id -> initAsyncHandlerThreadPools4Ns (id , asyncQueueSize , finalDedicatedHandlers ));
517533 LOG .info ("Assigned {} async handlers to nsId {} " , dedicatedHandlers , nsId );
518- } else {
519- unassignedNS .add (nsId );
520534 }
521535 }
522536
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- }
537+ if (routerDefaultAsyncHandlerExecutor == null ) {
538+ LOG .info ("init router async default executor handler count: {}" , asyncHandlerCountDefault );
539+ routerDefaultAsyncHandlerExecutor = initAsyncHandlerThreadPools4Ns (
540+ "default" , asyncQueueSize , asyncHandlerCountDefault );
532541 }
533542
534543 int asyncResponderCount = configuration .getInt (DFS_ROUTER_ASYNC_RPC_RESPONDER_COUNT_KEY ,
@@ -539,17 +548,35 @@ public void initAsyncThreadPools(Configuration configuration) {
539548 asyncResponderCount , new AsyncThreadFactory ("Router Async Responder #" ));
540549 }
541550 AsyncRpcProtocolPBUtil .setAsyncResponderExecutor (routerAsyncResponderExecutor );
551+ }
542552
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 #" ));
553+ private ThreadPoolExecutor initAsyncHandlerThreadPools4Ns (String ns , int asyncQueueSize , int handlerCount ) {
554+ LinkedBlockingQueue <Runnable > queue = new LinkedBlockingQueue <>(asyncQueueSize );
555+ return new ThreadPoolExecutor (handlerCount , handlerCount ,
556+ 0L , TimeUnit .MILLISECONDS , queue ,
557+ new AsyncThreadFactory ("Router Async Handler for " + ns + " #" ));
558+ }
559+
560+ /**
561+ * Returns the asynchronous executor for the specified namespace.
562+ * If no executor is configured for the given namespace ID, returns the default executor.
563+ *
564+ * @param nsId the namespace identifier
565+ * @return the corresponding ExecutorService
566+ */
567+ public ThreadPoolExecutor getAsyncExecutorForNamespace (String nsId ) {
568+ ThreadPoolExecutor executorService = asyncRouterHandlerExecutors .getOrDefault (
569+ nsId , routerDefaultAsyncHandlerExecutor );
570+ if (rpcMonitor != null ) {
571+ rpcMonitor .recordAsyncQueueSize (nsId , executorService .getQueue ().size ());
547572 }
573+ return executorService ;
548574 }
549575
550- private void initNsAsyncHandlerCount ( ) {
576+ private Map < String , Integer > parseNsAsyncHandlerCount ( Configuration conf ) {
551577 String configNsHandler = conf .get (DFS_ROUTER_ASYNC_RPC_NS_HANDLER_COUNT_KEY ,
552578 DFS_ROUTER_ASYNC_RPC_NS_HANDLER_COUNT_DEFAULT );
579+ Map <String , Integer > nsAsyncHandlerCount = new ConcurrentHashMap <>();
553580 if (StringUtils .isEmpty (configNsHandler )) {
554581 LOG .error (
555582 "The value of config key: {} is empty. Will use default conf." ,
@@ -566,11 +593,7 @@ private void initNsAsyncHandlerCount() {
566593 }
567594 nsAsyncHandlerCount .put (nsHandlerItems [0 ], Integer .parseInt (nsHandlerItems [1 ]));
568595 }
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 + " #" )));
596+ return nsAsyncHandlerCount ;
574597 }
575598
576599 /**
@@ -2489,14 +2512,6 @@ public boolean isAsync() {
24892512 return this .enableAsync ;
24902513 }
24912514
2492- public Map <String , ExecutorService > getAsyncRouterHandlerExecutors () {
2493- return asyncRouterHandlerExecutors ;
2494- }
2495-
2496- public ExecutorService getRouterAsyncHandlerDefaultExecutor () {
2497- return routerDefaultAsyncHandlerExecutor ;
2498- }
2499-
25002515 private static class AsyncThreadFactory implements ThreadFactory {
25012516 private final String namePrefix ;
25022517 private final AtomicInteger threadNumber = new AtomicInteger (1 );
0 commit comments