-
Notifications
You must be signed in to change notification settings - Fork 8.9k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
HDFS-17651.[ARR] Async handler executor isolation
- Loading branch information
1 parent
273673c
commit 1e07b5f
Showing
10 changed files
with
166 additions
and
65 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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,13 +58,15 @@ | |
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; | ||
|
@@ -72,6 +76,7 @@ | |
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; | ||
|
@@ -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,13 @@ 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 static Map<String, ExecutorService> routerAsyncHandlerExecutors = new ConcurrentHashMap<>(); | ||
private ExecutorService routerAsyncResponderExecutor; | ||
private static ExecutorService routerDefaultAsyncHandlerExecutor; | ||
private ExecutorService asyncRouterHandler; | ||
private ExecutorService asyncRouterResponder; | ||
|
||
|
||
/** | ||
* Construct a router RPC server. | ||
|
@@ -318,11 +331,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(); | ||
initAsyncThreadPool(conf); | ||
} | ||
// Override Hadoop Common IPC setting | ||
int readerQueueSize = this.conf.getInt(DFS_ROUTER_READER_QUEUE_SIZE_KEY, | ||
|
@@ -492,22 +505,76 @@ public RouterRpcServer(Configuration conf, Router router, | |
/** | ||
* Init router async handlers and router async responders. | ||
*/ | ||
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 initAsyncThreadPool(Configuration conf) { | ||
Check failure on line 508 in hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java ASF Cloudbees Jenkins ci-hadoop / Apache Yetushadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java#L508
|
||
LOG.info("Begin initialize asynchronous handler and responder thread pool."); | ||
initNsAsyncHandlerCount(); | ||
Set<String> allConfiguredNS = FederationUtil.getAllConfiguredNS(conf); | ||
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 = conf.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 ")); | ||
|
||
if (routerDefaultAsyncHandlerExecutor == null) { | ||
LOG.info("init router async default executor handler count: {}", asyncHandlerCountDefault); | ||
routerAsyncResponderExecutor = Executors.newFixedThreadPool( | ||
asyncHandlerCountDefault, new AsyncThreadFactory("Router Async Default Handler #")); | ||
} | ||
AsyncRpcProtocolPBUtil.setWorker(asyncRouterResponder); | ||
|
||
int asyncResponderCount = conf.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); | ||
} | ||
|
||
private void initNsAsyncHandlerCount() { | ||
String configNsHandler = conf.get(DFS_ROUTER_ASYNC_RPC_NS_HANDLER_COUNT_KEY, | ||
DFS_ROUTER_ASYNC_RPC_NS_HANDLER_COUNT_DEFAULT); | ||
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; | ||
} | ||
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])); | ||
} | ||
} | ||
|
||
private void initAsyncHandlerThreadPools4Ns(String nsId, int dedicatedHandlers) { | ||
routerAsyncHandlerExecutors.computeIfAbsent(nsId, id -> Executors.newFixedThreadPool( | ||
dedicatedHandlers, new AsyncThreadFactory("Router Async Handler for " + id + " #"))); | ||
} | ||
|
||
/** | ||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.