-
Notifications
You must be signed in to change notification settings - Fork 9.2k
HDFS-17651.[ARR] Async handler executor isolation #7244
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
Changes from 5 commits
1d4508b
3f5e749
fdfedcd
75d09e7
cc3606e
ac837e2
7cfe81a
ca2d9cd
75c56c9
3ceefb3
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,6 +18,16 @@ | |
| package org.apache.hadoop.hdfs.server.federation.router; | ||
|
|
||
| import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION; | ||
| import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNCRPC_RESPONDER_COUNT_DEFAULT; | ||
| import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNC_RPC_ENABLE_DEFAULT; | ||
| import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNC_RPC_ENABLE_KEY; | ||
| import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNC_RPC_HANDLER_COUNT_DEFAULT; | ||
| import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNC_RPC_HANDLER_COUNT_KEY; | ||
| import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNC_RPC_NS_HANDLER_COUNT_DEFAULT; | ||
| import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNC_RPC_NS_HANDLER_COUNT_KEY; | ||
| import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNC_RPC_RESPONDER_COUNT_KEY; | ||
| import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_FEDERATION_RENAME_OPTION; | ||
| import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_FEDERATION_RENAME_OPTION_DEFAULT; | ||
| import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_HANDLER_COUNT_DEFAULT; | ||
| import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_HANDLER_COUNT_KEY; | ||
| import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_HANDLER_QUEUE_SIZE_DEFAULT; | ||
|
|
@@ -26,16 +36,8 @@ | |
| import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_READER_COUNT_KEY; | ||
| import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_READER_QUEUE_SIZE_DEFAULT; | ||
| import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_READER_QUEUE_SIZE_KEY; | ||
| import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_RPC_ASYNC_HANDLER_COUNT; | ||
| import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_RPC_ASYNC_HANDLER_COUNT_DEFAULT; | ||
| import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_RPC_ASYNC_RESPONDER_COUNT; | ||
| import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_RPC_ASYNC_RESPONDER_COUNT_DEFAULT; | ||
| import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_RPC_ENABLE_ASYNC; | ||
| import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_RPC_ENABLE_ASYNC_DEFAULT; | ||
| import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DN_REPORT_CACHE_EXPIRE; | ||
| import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DN_REPORT_CACHE_EXPIRE_MS_DEFAULT; | ||
| import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_FEDERATION_RENAME_OPTION; | ||
| import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_FEDERATION_RENAME_OPTION_DEFAULT; | ||
| import static org.apache.hadoop.hdfs.server.federation.router.RouterFederationRename.RouterRenameOption; | ||
| import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncApply; | ||
| import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncCatch; | ||
|
|
@@ -56,22 +58,24 @@ | |
| import java.util.ArrayList; | ||
| import java.util.Collection; | ||
| import java.util.EnumSet; | ||
| import java.util.HashSet; | ||
| import java.util.Iterator; | ||
| import java.util.LinkedHashMap; | ||
| import java.util.LinkedHashSet; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.Map.Entry; | ||
| import java.util.Set; | ||
| import java.util.concurrent.ConcurrentHashMap; | ||
| import java.util.concurrent.ExecutionException; | ||
| import java.util.concurrent.Executor; | ||
| import java.util.concurrent.ExecutorService; | ||
| import java.util.concurrent.Executors; | ||
| import java.util.concurrent.ThreadFactory; | ||
| import java.util.concurrent.TimeUnit; | ||
| import java.util.concurrent.atomic.AtomicInteger; | ||
| import java.util.stream.Collectors; | ||
|
|
||
| import org.apache.commons.lang3.StringUtils; | ||
| import org.apache.hadoop.fs.Path; | ||
| import org.apache.hadoop.hdfs.HAUtil; | ||
| import org.apache.hadoop.hdfs.protocol.UnresolvedPathException; | ||
|
|
@@ -209,6 +213,7 @@ | |
| import org.apache.hadoop.tools.protocolPB.GetUserMappingsProtocolPB; | ||
| import org.apache.hadoop.tools.protocolPB.GetUserMappingsProtocolServerSideTranslatorPB; | ||
| import org.apache.hadoop.util.ReflectionUtils; | ||
| import org.checkerframework.checker.nullness.qual.NonNull; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
|
|
@@ -228,8 +233,9 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol, | |
|
|
||
| private static final Logger LOG = | ||
| LoggerFactory.getLogger(RouterRpcServer.class); | ||
| private ExecutorService asyncRouterHandler; | ||
| private ExecutorService asyncRouterResponder; | ||
|
|
||
| /** Name service keyword to identify fan-out calls. */ | ||
| public static final String CONCURRENT_NS = "concurrent"; | ||
|
|
||
| /** Configuration for the RPC server. */ | ||
| private Configuration conf; | ||
|
|
@@ -287,6 +293,10 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol, | |
| /** Schedule the router federation rename jobs. */ | ||
| private BalanceProcedureScheduler fedRenameScheduler; | ||
| private boolean enableAsync; | ||
| private Map<String, Integer> nsAsyncHandlerCount = new ConcurrentHashMap<>(); | ||
| private Map<String, ExecutorService> asyncRouterHandlerExecutors = new ConcurrentHashMap<>(); | ||
| private ExecutorService routerAsyncResponderExecutor; | ||
| private ExecutorService routerDefaultAsyncHandlerExecutor; | ||
|
|
||
| /** | ||
| * Construct a router RPC server. | ||
|
|
@@ -318,11 +328,11 @@ public RouterRpcServer(Configuration conf, Router router, | |
| int handlerQueueSize = this.conf.getInt(DFS_ROUTER_HANDLER_QUEUE_SIZE_KEY, | ||
| DFS_ROUTER_HANDLER_QUEUE_SIZE_DEFAULT); | ||
|
|
||
| this.enableAsync = conf.getBoolean(DFS_ROUTER_RPC_ENABLE_ASYNC, | ||
| DFS_ROUTER_RPC_ENABLE_ASYNC_DEFAULT); | ||
| LOG.info("Router enable async {}", this.enableAsync); | ||
| this.enableAsync = conf.getBoolean(DFS_ROUTER_ASYNC_RPC_ENABLE_KEY, | ||
| DFS_ROUTER_ASYNC_RPC_ENABLE_DEFAULT); | ||
| LOG.info("Router enable async rpc: {}", this.enableAsync); | ||
| if (this.enableAsync) { | ||
| initAsyncThreadPool(); | ||
| initAsyncThreadPools(conf); | ||
| } | ||
| // Override Hadoop Common IPC setting | ||
| int readerQueueSize = this.conf.getInt(DFS_ROUTER_READER_QUEUE_SIZE_KEY, | ||
|
|
@@ -446,8 +456,7 @@ public RouterRpcServer(Configuration conf, Router router, | |
| // Create the client | ||
| if (this.enableAsync) { | ||
| this.rpcClient = new RouterAsyncRpcClient(this.conf, this.router, | ||
| this.namenodeResolver, this.rpcMonitor, | ||
| routerStateIdContext, asyncRouterHandler); | ||
| this.namenodeResolver, this.rpcMonitor, routerStateIdContext); | ||
| this.clientProto = new RouterAsyncClientProtocol(conf, this); | ||
| this.nnProto = new RouterAsyncNamenodeProtocol(this); | ||
| this.routerProto = new RouterAsyncUserProtocol(this); | ||
|
|
@@ -491,23 +500,78 @@ public RouterRpcServer(Configuration conf, Router router, | |
|
|
||
| /** | ||
| * Init router async handlers and router async responders. | ||
| * @param configuration the configuration. | ||
| */ | ||
| public void initAsyncThreadPool() { | ||
| int asyncHandlerCount = conf.getInt(DFS_ROUTER_RPC_ASYNC_HANDLER_COUNT, | ||
| DFS_ROUTER_RPC_ASYNC_HANDLER_COUNT_DEFAULT); | ||
| int asyncResponderCount = conf.getInt(DFS_ROUTER_RPC_ASYNC_RESPONDER_COUNT, | ||
| DFS_ROUTER_RPC_ASYNC_RESPONDER_COUNT_DEFAULT); | ||
| if (asyncRouterHandler == null) { | ||
| LOG.info("init router async handler count: {}", asyncHandlerCount); | ||
| asyncRouterHandler = Executors.newFixedThreadPool( | ||
| asyncHandlerCount, new AsyncThreadFactory("router async handler ")); | ||
| public void initAsyncThreadPools(Configuration configuration) { | ||
| LOG.info("Begin initialize asynchronous handler and responder thread pool."); | ||
| initNsAsyncHandlerCount(); | ||
| Set<String> allConfiguredNS = FederationUtil.getAllConfiguredNS(configuration); | ||
| Set<String> unassignedNS = new HashSet<>(); | ||
| allConfiguredNS.add(CONCURRENT_NS); | ||
|
|
||
| for (String nsId : allConfiguredNS) { | ||
| int dedicatedHandlers = nsAsyncHandlerCount.getOrDefault(nsId, 0); | ||
| LOG.info("Dedicated handlers {} for ns {} ", dedicatedHandlers, nsId); | ||
| if (dedicatedHandlers > 0) { | ||
| initAsyncHandlerThreadPools4Ns(nsId, dedicatedHandlers); | ||
| LOG.info("Assigned {} async handlers to nsId {} ", dedicatedHandlers, nsId); | ||
| } else { | ||
| unassignedNS.add(nsId); | ||
| } | ||
| } | ||
|
|
||
| int asyncHandlerCountDefault = configuration.getInt(DFS_ROUTER_ASYNC_RPC_HANDLER_COUNT_KEY, | ||
| DFS_ROUTER_ASYNC_RPC_HANDLER_COUNT_DEFAULT); | ||
|
|
||
| if (!unassignedNS.isEmpty()) { | ||
| LOG.warn("Async handler unassigned ns: {}", unassignedNS); | ||
| LOG.info("Use default async handler count {} for unassigned ns.", asyncHandlerCountDefault); | ||
| for (String nsId : unassignedNS) { | ||
| initAsyncHandlerThreadPools4Ns(nsId, asyncHandlerCountDefault); | ||
| } | ||
| } | ||
| if (asyncRouterResponder == null) { | ||
| LOG.info("init router async responder count: {}", asyncResponderCount); | ||
| asyncRouterResponder = Executors.newFixedThreadPool( | ||
| asyncResponderCount, new AsyncThreadFactory("router async responder ")); | ||
|
|
||
| int asyncResponderCount = configuration.getInt(DFS_ROUTER_ASYNC_RPC_RESPONDER_COUNT_KEY, | ||
| DFS_ROUTER_ASYNCRPC_RESPONDER_COUNT_DEFAULT); | ||
| if (routerAsyncResponderExecutor == null) { | ||
| LOG.info("Initialize router async responder count: {}", asyncResponderCount); | ||
| routerAsyncResponderExecutor = Executors.newFixedThreadPool( | ||
| asyncResponderCount, new AsyncThreadFactory("Router Async Responder #")); | ||
| } | ||
| AsyncRpcProtocolPBUtil.setAsyncResponderExecutor(routerAsyncResponderExecutor); | ||
|
|
||
| if (routerDefaultAsyncHandlerExecutor == null) { | ||
| LOG.info("init router async default executor handler count: {}", asyncHandlerCountDefault); | ||
| routerDefaultAsyncHandlerExecutor = Executors.newFixedThreadPool( | ||
| asyncHandlerCountDefault, new AsyncThreadFactory("Router Async Default Handler #")); | ||
| } | ||
| } | ||
|
|
||
| private void initNsAsyncHandlerCount() { | ||
| String configNsHandler = conf.get(DFS_ROUTER_ASYNC_RPC_NS_HANDLER_COUNT_KEY, | ||
| DFS_ROUTER_ASYNC_RPC_NS_HANDLER_COUNT_DEFAULT); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I don't prefer the format of this value, but no better idea now. ^_^
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks Sir, yes. Actually I don't like this config too, but if we do not use this format , we may introduce more configs. like: |
||
| if (StringUtils.isEmpty(configNsHandler)) { | ||
| LOG.error( | ||
| "The config key: {} is incorrect! The value is empty.", | ||
| DFS_ROUTER_ASYNC_RPC_NS_HANDLER_COUNT_KEY); | ||
| configNsHandler = DFS_ROUTER_ASYNC_RPC_NS_HANDLER_COUNT_DEFAULT; | ||
|
||
| } | ||
| AsyncRpcProtocolPBUtil.setWorker(asyncRouterResponder); | ||
| String[] nsHandlers = configNsHandler.split(","); | ||
| for (String nsHandlerInfo : nsHandlers) { | ||
| String[] nsHandlerItems = nsHandlerInfo.split(":"); | ||
| if (nsHandlerItems.length != 2 || StringUtils.isBlank(nsHandlerItems[0]) || | ||
| !StringUtils.isNumeric(nsHandlerItems[1])) { | ||
| LOG.error("The config key: {} is incorrect! The value is {}.", | ||
| DFS_ROUTER_ASYNC_RPC_NS_HANDLER_COUNT_KEY, nsHandlerInfo); | ||
| continue; | ||
| } | ||
| nsAsyncHandlerCount.put(nsHandlerItems[0], Integer.parseInt(nsHandlerItems[1])); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Will it involve some unexpected action if
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. |
||
| } | ||
| } | ||
|
|
||
| private void initAsyncHandlerThreadPools4Ns(String nsId, int dedicatedHandlers) { | ||
| asyncRouterHandlerExecutors.computeIfAbsent(nsId, id -> Executors.newFixedThreadPool( | ||
| dedicatedHandlers, new AsyncThreadFactory("Router Async Handler for " + id + " #"))); | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -2426,8 +2490,12 @@ public boolean isAsync() { | |
| return this.enableAsync; | ||
| } | ||
|
|
||
| public Executor getAsyncRouterHandler() { | ||
| return asyncRouterHandler; | ||
| public Map<String, ExecutorService> getAsyncRouterHandlerExecutors() { | ||
| return asyncRouterHandlerExecutors; | ||
| } | ||
|
|
||
| public ExecutorService getRouterAsyncHandlerDefaultExecutor() { | ||
| return routerDefaultAsyncHandlerExecutor; | ||
| } | ||
|
|
||
| private static class AsyncThreadFactory implements ThreadFactory { | ||
|
|
@@ -2439,8 +2507,10 @@ private static class AsyncThreadFactory implements ThreadFactory { | |
| } | ||
|
|
||
| @Override | ||
| public Thread newThread(Runnable r) { | ||
| return new Thread(r, namePrefix + threadNumber.getAndIncrement()); | ||
| public Thread newThread(@NonNull Runnable r) { | ||
| Thread thread = new Thread(r, namePrefix + threadNumber.getAndIncrement()); | ||
| thread.setDaemon(true); | ||
| return thread; | ||
| } | ||
| } | ||
| } | ||

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.
Leave empty too as above comment.
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.