-
Notifications
You must be signed in to change notification settings - Fork 9.2k
HDFS-15417. RBF: Get the datanode report from cache for federation WebHDFS operations #2080
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
a1ad88e
3369ba7
712835a
5372347
66b2b33
801216b
6bbbf46
33f104b
8970a0e
7eeb2b5
f4d4b9a
dfb6811
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,8 @@ | |
| 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.metrics.NamenodeBeanMetrics.DN_REPORT_CACHE_EXPIRE; | ||
| import static org.apache.hadoop.hdfs.server.federation.metrics.NamenodeBeanMetrics.DN_REPORT_CACHE_EXPIRE_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; | ||
|
|
@@ -41,7 +43,21 @@ | |
| import java.util.Map; | ||
| import java.util.Map.Entry; | ||
| import java.util.Set; | ||
|
|
||
| import java.util.concurrent.Callable; | ||
| import java.util.concurrent.ExecutionException; | ||
| import java.util.concurrent.Executors; | ||
| import java.util.concurrent.LinkedBlockingQueue; | ||
| import java.util.concurrent.ThreadFactory; | ||
| import java.util.concurrent.ThreadPoolExecutor; | ||
| import java.util.concurrent.TimeUnit; | ||
|
|
||
| import com.google.common.cache.CacheBuilder; | ||
| import com.google.common.cache.CacheLoader; | ||
| import com.google.common.cache.LoadingCache; | ||
| import com.google.common.util.concurrent.ListenableFuture; | ||
| import com.google.common.util.concurrent.ListeningExecutorService; | ||
| import com.google.common.util.concurrent.MoreExecutors; | ||
| import com.google.common.util.concurrent.ThreadFactoryBuilder; | ||
| import org.apache.hadoop.conf.Configuration; | ||
| import org.apache.hadoop.crypto.CryptoProtocolVersion; | ||
| import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries; | ||
|
|
@@ -219,6 +235,9 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol, | |
| private static final ThreadLocal<UserGroupInformation> CUR_USER = | ||
| new ThreadLocal<>(); | ||
|
|
||
| /** DN type -> full DN report. */ | ||
| private final LoadingCache<DatanodeReportType, DatanodeInfo[]> dnCache; | ||
|
|
||
| /** | ||
| * Construct a router RPC server. | ||
| * | ||
|
|
@@ -361,6 +380,23 @@ public RouterRpcServer(Configuration configuration, Router router, | |
| this.nnProto = new RouterNamenodeProtocol(this); | ||
| this.clientProto = new RouterClientProtocol(conf, this); | ||
| this.routerProto = new RouterUserProtocol(this); | ||
|
|
||
| long dnCacheExpire = conf.getTimeDuration( | ||
| DN_REPORT_CACHE_EXPIRE, | ||
| DN_REPORT_CACHE_EXPIRE_DEFAULT, TimeUnit.MILLISECONDS); | ||
| this.dnCache = CacheBuilder.newBuilder() | ||
| .build(new DatanodeReportCacheLoader()); | ||
|
|
||
| // Actively refresh the dn cache in a configured interval | ||
| Executors | ||
goiri marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| .newSingleThreadScheduledExecutor() | ||
| .scheduleWithFixedDelay(() -> this.dnCache | ||
| .asMap() | ||
| .keySet() | ||
| .parallelStream() | ||
| .forEach((key) -> this.dnCache.refresh(key)), | ||
| 0, | ||
| dnCacheExpire, TimeUnit.MILLISECONDS); | ||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -868,6 +904,50 @@ public DatanodeInfo[] getDatanodeReport(DatanodeReportType type) | |
| return clientProto.getDatanodeReport(type); | ||
| } | ||
|
|
||
| /** | ||
| * Get the datanode report from cache. | ||
| * | ||
| * @param type Type of the datanode. | ||
| * @return List of datanodes. | ||
| * @throws IOException If it cannot get the report. | ||
| */ | ||
| public DatanodeInfo[] getCachedDatanodeReport(DatanodeReportType type) | ||
|
||
| throws IOException { | ||
| try { | ||
| DatanodeInfo[] dns = this.dnCache.get(type); | ||
| if (dns == null) { | ||
| LOG.debug("Get null DN report from cache"); | ||
| dns = getCachedDatanodeReportImpl(type); | ||
| this.dnCache.put(type, dns); | ||
| } | ||
| return dns; | ||
| } catch (ExecutionException e) { | ||
| LOG.error("Cannot get the DN report for {}", type, e); | ||
| Throwable cause = e.getCause(); | ||
| if (cause instanceof IOException) { | ||
| throw (IOException) cause; | ||
| } else { | ||
| throw new IOException(cause); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| private DatanodeInfo[] getCachedDatanodeReportImpl | ||
| (final DatanodeReportType type) throws IOException{ | ||
|
||
| // We need to get the DNs as a privileged user | ||
| UserGroupInformation loginUser = UserGroupInformation.getLoginUser(); | ||
| RouterRpcServer.setCurrentUser(loginUser); | ||
|
|
||
| try { | ||
| DatanodeInfo[] dns = clientProto.getDatanodeReport(type); | ||
| LOG.debug("Refresh cached DN report with {} datanodes", dns.length); | ||
| return dns; | ||
| } finally { | ||
| // Reset ugi to remote user for remaining operations. | ||
| RouterRpcServer.resetCurrentUser(); | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Get the datanode report with a timeout. | ||
| * @param type Type of the datanode. | ||
|
|
@@ -1748,4 +1828,58 @@ public void refreshSuperUserGroupsConfiguration() throws IOException { | |
| public String[] getGroupsForUser(String user) throws IOException { | ||
| return routerProto.getGroupsForUser(user); | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Deals with loading datanode report into the cache and refresh. | ||
| */ | ||
| private class DatanodeReportCacheLoader | ||
| extends CacheLoader<DatanodeReportType, DatanodeInfo[]> { | ||
|
|
||
| private ListeningExecutorService executorService; | ||
|
|
||
| DatanodeReportCacheLoader() { | ||
| ThreadFactory threadFactory = new ThreadFactoryBuilder() | ||
NickyYe marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| .setNameFormat("DatanodeReport-Cache-Reload") | ||
| .setDaemon(true) | ||
| .build(); | ||
|
|
||
| // Only use 1 thread to refresh cache. | ||
| // With coreThreadCount == maxThreadCount we effectively | ||
| // create a fixed size thread pool. As allowCoreThreadTimeOut | ||
| // has been set, all threads will die after 60 seconds of non use. | ||
| ThreadPoolExecutor parentExecutor = new ThreadPoolExecutor( | ||
| 1, | ||
| 1, | ||
| 60, | ||
| TimeUnit.SECONDS, | ||
| new LinkedBlockingQueue<Runnable>(), | ||
| threadFactory); | ||
| parentExecutor.allowCoreThreadTimeOut(true); | ||
| executorService = MoreExecutors.listeningDecorator(parentExecutor); | ||
| } | ||
|
|
||
| @Override | ||
| public DatanodeInfo[] load(DatanodeReportType type) throws Exception { | ||
| return getCachedDatanodeReportImpl(type); | ||
| } | ||
|
|
||
| /** | ||
| * Override the reload method to provide an asynchronous implementation, | ||
| * so that the query will not be slowed down by the cache refresh. It | ||
| * will return the old cache value and schedule a background refresh. | ||
| */ | ||
| @Override | ||
| public ListenableFuture<DatanodeInfo[]> reload( | ||
| final DatanodeReportType type, DatanodeInfo[] oldValue) | ||
| throws Exception { | ||
| ListenableFuture<DatanodeInfo[]> listenableFuture = | ||
|
||
| executorService.submit(new Callable<DatanodeInfo[]>() { | ||
| @Override | ||
| public DatanodeInfo[] call() throws Exception { | ||
| return load(type); | ||
| } | ||
| }); | ||
| return listenableFuture; | ||
| } | ||
| } | ||
| } | ||
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.
This will cause compilation error. Also I'm wondering whether it makes sense to move the DN cache logic from
NamenodeBeanMetricsto here and have the former to depend on this. This way we don't have to keep two copies of cache.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.
Thanks for catching this. I was working on an old branch, now it is fixed. I prefer to move the NamenodeBeanMetrics logic by a separate change. Let's make this change cohesive. NamenodeBeanMetrics cached report needs a different API or at least some extra work since it needs a String.