Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 11EF5200D34 for ; Fri, 20 Oct 2017 02:53:03 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 1079F1609EE; Fri, 20 Oct 2017 00:53:03 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id D8AD0160BED for ; Fri, 20 Oct 2017 02:53:00 +0200 (CEST) Received: (qmail 72962 invoked by uid 500); 20 Oct 2017 00:52:52 -0000 Mailing-List: contact common-commits-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list common-commits@hadoop.apache.org Received: (qmail 70954 invoked by uid 99); 20 Oct 2017 00:52:51 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 20 Oct 2017 00:52:51 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 7BDCEE00B7; Fri, 20 Oct 2017 00:52:49 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: inigoiri@apache.org To: common-commits@hadoop.apache.org Date: Fri, 20 Oct 2017 00:53:17 -0000 Message-Id: <40c6b3ec03374953b52c1c84a5dbd119@git.apache.org> In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [30/32] hadoop git commit: HDFS-12273. Federation UI. Contributed by Inigo Goiri. archived-at: Fri, 20 Oct 2017 00:53:03 -0000 HDFS-12273. Federation UI. Contributed by Inigo Goiri. (cherry picked from commit adbb2e00c7b85524fd43bd68895d49814c16680a) (cherry picked from commit 81601dac8ec7650bec14700b174910390a92fe1f) Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/27295eec Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/27295eec Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/27295eec Branch: refs/heads/branch-2 Commit: 27295eecdd5775c4fb6c899d515728a7e1972e43 Parents: 7d79598 Author: Inigo Goiri Authored: Thu Oct 5 17:26:43 2017 -0700 Committer: Inigo Goiri Committed: Thu Oct 19 17:39:19 2017 -0700 ---------------------------------------------------------------------- hadoop-hdfs-project/hadoop-hdfs/pom.xml | 3 + .../org/apache/hadoop/hdfs/DFSConfigKeys.java | 19 + .../federation/metrics/FederationMBean.java | 7 + .../federation/metrics/FederationMetrics.java | 25 +- .../resolver/MembershipNamenodeResolver.java | 23 ++ .../hdfs/server/federation/router/Router.java | 36 ++ .../federation/router/RouterHttpServer.java | 124 +++++++ .../federation/router/RouterRpcClient.java | 45 ++- .../federation/router/RouterRpcServer.java | 15 +- .../src/main/resources/hdfs-default.xml | 56 +++ .../main/webapps/router/federationhealth.html | 371 +++++++++++++++++++ .../src/main/webapps/router/federationhealth.js | 313 ++++++++++++++++ .../src/main/webapps/router/index.html | 24 ++ .../server/federation/RouterConfigBuilder.java | 13 + .../server/federation/RouterDFSCluster.java | 29 +- .../federation/metrics/TestMetricsBase.java | 1 + .../server/federation/router/TestRouter.java | 9 +- 17 files changed, 1102 insertions(+), 11 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hadoop/blob/27295eec/hadoop-hdfs-project/hadoop-hdfs/pom.xml ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs/pom.xml index 8ae3db8..154e4f5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml @@ -276,6 +276,9 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd"> + http://git-wip-us.apache.org/repos/asf/hadoop/blob/27295eec/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java index 3606e7a..3f967da 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java @@ -1141,6 +1141,25 @@ public class DFSConfigKeys extends CommonConfigurationKeys { FEDERATION_ROUTER_PREFIX + "admin.enable"; public static final boolean DFS_ROUTER_ADMIN_ENABLE_DEFAULT = true; + // HDFS Router-based federation web + public static final String DFS_ROUTER_HTTP_ENABLE = + FEDERATION_ROUTER_PREFIX + "http.enable"; + public static final boolean DFS_ROUTER_HTTP_ENABLE_DEFAULT = true; + public static final String DFS_ROUTER_HTTP_ADDRESS_KEY = + FEDERATION_ROUTER_PREFIX + "http-address"; + public static final int DFS_ROUTER_HTTP_PORT_DEFAULT = 50071; + public static final String DFS_ROUTER_HTTP_BIND_HOST_KEY = + FEDERATION_ROUTER_PREFIX + "http-bind-host"; + public static final String DFS_ROUTER_HTTP_ADDRESS_DEFAULT = + "0.0.0.0:" + DFS_ROUTER_HTTP_PORT_DEFAULT; + public static final String DFS_ROUTER_HTTPS_ADDRESS_KEY = + FEDERATION_ROUTER_PREFIX + "https-address"; + public static final int DFS_ROUTER_HTTPS_PORT_DEFAULT = 50072; + public static final String DFS_ROUTER_HTTPS_BIND_HOST_KEY = + FEDERATION_ROUTER_PREFIX + "https-bind-host"; + public static final String DFS_ROUTER_HTTPS_ADDRESS_DEFAULT = + "0.0.0.0:" + DFS_ROUTER_HTTPS_PORT_DEFAULT; + // dfs.client.retry confs are moved to HdfsClientConfigKeys.Retry @Deprecated public static final String DFS_CLIENT_RETRY_POLICY_ENABLED_KEY http://git-wip-us.apache.org/repos/asf/hadoop/blob/27295eec/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMBean.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMBean.java index 43efb3c..cb4245a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMBean.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMBean.java @@ -201,4 +201,11 @@ public interface FederationMBean { * @return Host and port of the router. */ String getBlockPoolId(); + + /** + * Get the current state of the router. + * + * @return String label for the current router state. + */ + String getRouterStatus(); } http://git-wip-us.apache.org/repos/asf/hadoop/blob/27295eec/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMetrics.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMetrics.java index 1e80256..7844a2e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMetrics.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMetrics.java @@ -21,6 +21,9 @@ import static org.apache.hadoop.util.Time.now; import java.io.IOException; import java.lang.reflect.Method; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.UnknownHostException; import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Arrays; @@ -34,6 +37,7 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.TimeUnit; import java.util.function.Function; import java.util.function.ToIntFunction; import java.util.function.ToLongFunction; @@ -83,6 +87,9 @@ public class FederationMetrics implements FederationMBean { /** Format for a date. */ private static final String DATE_FORMAT = "yyyy/MM/dd HH:mm:ss"; + /** Prevent holding the page from load too long. */ + private static final long TIME_OUT = TimeUnit.SECONDS.toMillis(1); + /** Router interface. */ private final Router router; @@ -353,8 +360,8 @@ public class FederationMetrics implements FederationMBean { final Map> info = new HashMap<>(); try { RouterRpcServer rpcServer = this.router.getRpcServer(); - DatanodeInfo[] live = - rpcServer.getDatanodeReport(DatanodeReportType.LIVE); + DatanodeInfo[] live = rpcServer.getDatanodeReport( + DatanodeReportType.LIVE, TIME_OUT); if (live.length > 0) { float totalDfsUsed = 0; @@ -446,7 +453,14 @@ public class FederationMetrics implements FederationMBean { @Override public String getHostAndPort() { - // TODO this should be the HTTP address + InetSocketAddress address = this.router.getHttpServerAddress(); + if (address != null) { + try { + String hostname = InetAddress.getLocalHost().getHostName(); + int port = address.getPort(); + return hostname + ":" + port; + } catch (UnknownHostException ignored) { } + } return "Unknown"; } @@ -479,6 +493,11 @@ public class FederationMetrics implements FederationMBean { } } + @Override + public String getRouterStatus() { + return "RUNNING"; + } + /** * Build a set of unique values found in all namespaces. * http://git-wip-us.apache.org/repos/asf/hadoop/blob/27295eec/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java index d974c78..0950cde 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java @@ -43,6 +43,7 @@ import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamespaceInfoR import org.apache.hadoop.hdfs.server.federation.store.protocol.NamenodeHeartbeatRequest; import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateNamenodeRegistrationRequest; import org.apache.hadoop.hdfs.server.federation.store.records.MembershipState; +import org.apache.hadoop.hdfs.server.federation.store.records.MembershipStats; import org.apache.hadoop.util.Time; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -222,6 +223,28 @@ public class MembershipNamenodeResolver report.getServiceAddress(), report.getLifelineAddress(), report.getWebAddress(), report.getState(), report.getSafemode()); + if (report.statsValid()) { + MembershipStats stats = MembershipStats.newInstance(); + stats.setNumOfFiles(report.getNumFiles()); + stats.setNumOfBlocks(report.getNumBlocks()); + stats.setNumOfBlocksMissing(report.getNumBlocksMissing()); + stats.setNumOfBlocksPendingReplication( + report.getNumOfBlocksPendingReplication()); + stats.setNumOfBlocksUnderReplicated( + report.getNumOfBlocksUnderReplicated()); + stats.setNumOfBlocksPendingDeletion( + report.getNumOfBlocksPendingDeletion()); + stats.setAvailableSpace(report.getAvailableSpace()); + stats.setTotalSpace(report.getTotalSpace()); + stats.setNumOfDecommissioningDatanodes( + report.getNumDecommissioningDatanodes()); + stats.setNumOfActiveDatanodes(report.getNumLiveDatanodes()); + stats.setNumOfDeadDatanodes(report.getNumDeadDatanodes()); + stats.setNumOfDecomActiveDatanodes(report.getNumDecomLiveDatanodes()); + stats.setNumOfDecomDeadDatanodes(report.getNumDecomDeadDatanodes()); + record.setStats(stats); + } + if (report.getState() != UNAVAILABLE) { // Set/update our last contact time record.setLastContact(Time.now()); http://git-wip-us.apache.org/repos/asf/hadoop/blob/27295eec/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java index 3ab5e2d..df5549c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java @@ -89,6 +89,9 @@ public class Router extends CompositeService { private RouterAdminServer adminServer; private InetSocketAddress adminAddress; + /** HTTP interface and web application. */ + private RouterHttpServer httpServer; + /** Interface with the State Store. */ private StateStoreService stateStore; @@ -169,6 +172,14 @@ public class Router extends CompositeService { } if (conf.getBoolean( + DFSConfigKeys.DFS_ROUTER_HTTP_ENABLE, + DFSConfigKeys.DFS_ROUTER_HTTP_ENABLE_DEFAULT)) { + // Create HTTP server + this.httpServer = createHttpServer(); + addService(this.httpServer); + } + + if (conf.getBoolean( DFSConfigKeys.DFS_ROUTER_HEARTBEAT_ENABLE, DFSConfigKeys.DFS_ROUTER_HEARTBEAT_ENABLE_DEFAULT)) { @@ -354,6 +365,31 @@ public class Router extends CompositeService { } ///////////////////////////////////////////////////////// + // HTTP server + ///////////////////////////////////////////////////////// + + /** + * Create an HTTP server for this Router. + * + * @return HTTP server for this Router. + */ + protected RouterHttpServer createHttpServer() { + return new RouterHttpServer(this); + } + + /** + * Get the current HTTP socket address for the router. + * + * @return InetSocketAddress HTTP address. + */ + public InetSocketAddress getHttpServerAddress() { + if (httpServer != null) { + return httpServer.getHttpAddress(); + } + return null; + } + + ///////////////////////////////////////////////////////// // Namenode heartbeat monitors ///////////////////////////////////////////////////////// http://git-wip-us.apache.org/repos/asf/hadoop/blob/27295eec/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterHttpServer.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterHttpServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterHttpServer.java new file mode 100644 index 0000000..046f0ba --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterHttpServer.java @@ -0,0 +1,124 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.server.federation.router; + +import java.net.InetSocketAddress; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.DFSUtil; +import org.apache.hadoop.hdfs.server.common.JspHelper; +import org.apache.hadoop.http.HttpServer2; +import org.apache.hadoop.service.AbstractService; + +/** + * Web interface for the {@link Router}. It exposes the Web UI and the WebHDFS + * methods from {@link RouterWebHdfsMethods}. + */ +public class RouterHttpServer extends AbstractService { + + protected static final String NAMENODE_ATTRIBUTE_KEY = "name.node"; + + + /** Configuration for the Router HTTP server. */ + private Configuration conf; + + /** Router using this HTTP server. */ + private final Router router; + + /** HTTP server. */ + private HttpServer2 httpServer; + + /** HTTP addresses. */ + private InetSocketAddress httpAddress; + private InetSocketAddress httpsAddress; + + + public RouterHttpServer(Router router) { + super(RouterHttpServer.class.getName()); + this.router = router; + } + + @Override + protected void serviceInit(Configuration configuration) throws Exception { + this.conf = configuration; + + // Get HTTP address + this.httpAddress = conf.getSocketAddr( + DFSConfigKeys.DFS_ROUTER_HTTP_BIND_HOST_KEY, + DFSConfigKeys.DFS_ROUTER_HTTP_ADDRESS_KEY, + DFSConfigKeys.DFS_ROUTER_HTTP_ADDRESS_DEFAULT, + DFSConfigKeys.DFS_ROUTER_HTTP_PORT_DEFAULT); + + // Get HTTPs address + this.httpsAddress = conf.getSocketAddr( + DFSConfigKeys.DFS_ROUTER_HTTPS_BIND_HOST_KEY, + DFSConfigKeys.DFS_ROUTER_HTTPS_ADDRESS_KEY, + DFSConfigKeys.DFS_ROUTER_HTTPS_ADDRESS_DEFAULT, + DFSConfigKeys.DFS_ROUTER_HTTPS_PORT_DEFAULT); + + super.serviceInit(conf); + } + + @Override + protected void serviceStart() throws Exception { + // Build and start server + String webApp = "router"; + HttpServer2.Builder builder = DFSUtil.httpServerTemplateForNNAndJN( + this.conf, this.httpAddress, this.httpsAddress, webApp, + DFSConfigKeys.DFS_NAMENODE_KERBEROS_INTERNAL_SPNEGO_PRINCIPAL_KEY, + DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY); + + this.httpServer = builder.build(); + + this.httpServer.setAttribute(NAMENODE_ATTRIBUTE_KEY, this.router); + this.httpServer.setAttribute(JspHelper.CURRENT_CONF, this.conf); + setupServlets(this.httpServer, this.conf); + + this.httpServer.start(); + + // The server port can be ephemeral... ensure we have the correct info + InetSocketAddress listenAddress = this.httpServer.getConnectorAddress(0); + if (listenAddress != null) { + this.httpAddress = new InetSocketAddress(this.httpAddress.getHostName(), + listenAddress.getPort()); + } + super.serviceStart(); + } + + @Override + protected void serviceStop() throws Exception { + if(this.httpServer != null) { + this.httpServer.stop(); + } + super.serviceStop(); + } + + private static void setupServlets( + HttpServer2 httpServer, Configuration conf) { + // TODO Add servlets for FSCK, etc + } + + public InetSocketAddress getHttpAddress() { + return this.httpAddress; + } + + public InetSocketAddress getHttpsAddress() { + return this.httpsAddress; + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/hadoop/blob/27295eec/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java index 5c33c2e..932295e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java @@ -35,11 +35,13 @@ import java.util.Map.Entry; import java.util.Set; import java.util.TreeMap; import java.util.concurrent.Callable; +import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -713,6 +715,7 @@ public class RouterRpcClient { * Re-throws exceptions generated by the remote RPC call as either * RemoteException or IOException. * + * @param The type of the remote location. * @param locations List of remote locations to call concurrently. * @param remoteMethod The remote method and parameters to invoke. * @param requireResponse If true an exception will be thrown if all calls do @@ -723,10 +726,35 @@ public class RouterRpcClient { * @throws IOException If requiredResponse=true and any of the calls throw an * exception. */ - @SuppressWarnings("unchecked") public Map invokeConcurrent( final Collection locations, final RemoteMethod method, boolean requireResponse, boolean standby) throws IOException { + return invokeConcurrent(locations, method, requireResponse, standby, -1); + } + + /** + * Invokes multiple concurrent proxy calls to different clients. Returns an + * array of results. + * + * Re-throws exceptions generated by the remote RPC call as either + * RemoteException or IOException. + * + * @param locations List of remote locations to call concurrently. + * @param remoteMethod The remote method and parameters to invoke. + * @param requireResponse If true an exception will be thrown if all calls do + * not complete. If false exceptions are ignored and all data results + * successfully received are returned. + * @param standby If the requests should go to the standby namenodes too. + * @param timeoutMs Timeout for each individual call. + * @return Result of invoking the method per subcluster: nsId -> result. + * @throws IOException If requiredResponse=true and any of the calls throw an + * exception. + */ + @SuppressWarnings("unchecked") + public Map invokeConcurrent( + final Collection locations, final RemoteMethod method, + boolean requireResponse, boolean standby, long timeOutMs) + throws IOException { final UserGroupInformation ugi = RouterRpcServer.getRemoteUser(); final Method m = method.getMethod(); @@ -782,7 +810,13 @@ public class RouterRpcClient { } try { - List> futures = executorService.invokeAll(callables); + List> futures = null; + if (timeOutMs > 0) { + futures = executorService.invokeAll( + callables, timeOutMs, TimeUnit.MILLISECONDS); + } else { + futures = executorService.invokeAll(callables); + } Map results = new TreeMap<>(); Map exceptions = new TreeMap<>(); for (int i=0; i future = futures.get(i); Object result = future.get(); results.put(location, result); + } catch (CancellationException ce) { + T loc = orderedLocations.get(i); + String msg = + "Invocation to \"" + loc + "\" for \"" + method + "\" timed out"; + LOG.error(msg); + IOException ioe = new IOException(msg); + exceptions.put(location, ioe); } catch (ExecutionException ex) { Throwable cause = ex.getCause(); LOG.debug("Canot execute {} in {}: {}", http://git-wip-us.apache.org/repos/asf/hadoop/blob/27295eec/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java index 650c6ab..4d3c237 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java @@ -1091,6 +1091,19 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol { public DatanodeInfo[] getDatanodeReport(DatanodeReportType type) throws IOException { checkOperation(OperationCategory.UNCHECKED); + return getDatanodeReport(type, 0); + } + + /** + * Get the datanode report with a timeout. + * @param type Type of the datanode. + * @param timeOutMs Time out for the reply in milliseconds. + * @return List of datanodes. + * @throws IOException If it cannot get the report. + */ + public DatanodeInfo[] getDatanodeReport( + DatanodeReportType type, long timeOutMs) throws IOException { + checkOperation(OperationCategory.UNCHECKED); Map datanodesMap = new LinkedHashMap<>(); RemoteMethod method = new RemoteMethod("getDatanodeReport", @@ -1098,7 +1111,7 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol { Set nss = namenodeResolver.getNamespaces(); Map results = - rpcClient.invokeConcurrent(nss, method, true, false); + rpcClient.invokeConcurrent(nss, method, true, false, timeOutMs); for (Entry entry : results.entrySet()) { FederationNamespaceInfo ns = entry.getKey(); DatanodeInfo[] result = (DatanodeInfo[]) entry.getValue(); http://git-wip-us.apache.org/repos/asf/hadoop/blob/27295eec/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml index 50ce6f2..8af7e4c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml @@ -4506,6 +4506,62 @@ + dfs.federation.router.http-address + 0.0.0.0:50071 + + HTTP address that handles the web requests to the Router. + The value of this property will take the form of router-host1:http-port. + + + + + dfs.federation.router.http-bind-host + + + The actual address the HTTP server will bind to. If this optional + address is set, it overrides only the hostname portion of + dfs.federation.router.http-address. This is useful for making the name + node listen on all interfaces by setting it to 0.0.0.0. + + + + + dfs.federation.router.https-address + 0.0.0.0:50072 + + HTTPS address that handles the web requests to the Router. + The value of this property will take the form of router-host1:https-port. + + + + + dfs.federation.router.https-bind-host + + + The actual address the HTTPS server will bind to. If this optional + address is set, it overrides only the hostname portion of + dfs.federation.router.https-address. This is useful for making the name + node listen on all interfaces by setting it to 0.0.0.0. + + + + + dfs.federation.router.http.enable + true + + If the HTTP service to handle client requests in the router is enabled. + + + + + dfs.federation.router.metrics.enable + true + + If the metrics service in the router is enabled. + + + + dfs.federation.router.file.resolver.client.class org.apache.hadoop.hdfs.server.federation.MockResolver http://git-wip-us.apache.org/repos/asf/hadoop/blob/27295eec/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/router/federationhealth.html ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/router/federationhealth.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/router/federationhealth.html new file mode 100644 index 0000000..3da5283 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/router/federationhealth.html @@ -0,0 +1,371 @@ + + + + + + + + +Router Information + + + + + +
+ +
+
+ +
+
+
+ +
+
+
+
+
+
+ +
+
+

Hadoop, {release-year-token}.

+
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + http://git-wip-us.apache.org/repos/asf/hadoop/blob/27295eec/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/router/federationhealth.js ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/router/federationhealth.js b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/router/federationhealth.js new file mode 100644 index 0000000..380f097 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/router/federationhealth.js @@ -0,0 +1,313 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +(function () { + "use strict"; + + dust.loadSource(dust.compile($('#tmpl-federationhealth').html(), 'federationhealth')); + dust.loadSource(dust.compile($('#tmpl-namenode').html(), 'namenode-info')); + dust.loadSource(dust.compile($('#tmpl-datanode').html(), 'datanode-info')); + dust.loadSource(dust.compile($('#tmpl-mounttable').html(), 'mounttable')); + + $.fn.dataTable.ext.order['ng-value'] = function (settings, col) + { + return this.api().column(col, {order:'index'} ).nodes().map(function (td, i) { + return $(td).attr('ng-value'); + }); + }; + + function load_overview() { + var BEANS = [ + {"name": "federation", "url": "/jmx?qry=Hadoop:service=Router,name=FederationState"} + ]; + + var HELPERS = { + 'helper_fs_max_objects': function (chunk, ctx, bodies, params) { + var o = ctx.current(); + if (o.MaxObjects > 0) { + chunk.write('(' + Math.round((o.FilesTotal + o.BlockTotal) / o.MaxObjects * 100) * 100 + ')%'); + } + }, + + 'helper_dir_status': function (chunk, ctx, bodies, params) { + var j = ctx.current(); + for (var i in j) { + chunk.write('' + i + '' + j[i] + '' + params.type + ''); + } + }, + + 'helper_date_tostring' : function (chunk, ctx, bodies, params) { + var value = dust.helpers.tap(params.value, chunk, ctx); + return chunk.write('' + new Date(Number(value)).toLocaleString()); + } + }; + + var data = {}; + + // Workarounds for the fact that JMXJsonServlet returns non-standard JSON strings + function workaround(nn) { + nn.NodeUsage = JSON.parse(nn.NodeUsage); + return nn; + } + + load_json( + BEANS, + guard_with_startup_progress(function(d) { + for (var k in d) { + data[k] = k === 'federation' ? workaround(d[k].beans[0]) : d[k].beans[0]; + } + render(); + }), + function (url, jqxhr, text, err) { + show_err_msg('

Failed to retrieve data from ' + url + ', cause: ' + err + '

'); + }); + + function render() { + var base = dust.makeBase(HELPERS); + dust.render('federationhealth', base.push(data), function(err, out) { + $('#tab-overview').html(out); + $('#ui-tabs a[href="#tab-overview"]').tab('show'); + }); + } + } + + function load_namenode_info() { + var HELPERS = { + 'helper_lastcontact_tostring' : function (chunk, ctx, bodies, params) { + var value = dust.helpers.tap(params.value, chunk, ctx); + return chunk.write('' + new Date(Date.now()-1000*Number(value))); + } + }; + + function workaround(r) { + function node_map_to_array(nodes) { + var res = []; + for (var n in nodes) { + var p = nodes[n]; + p.name = n; + res.push(p); + } + return res; + } + + function capitalise(string) { + return string.charAt(0).toUpperCase() + string.slice(1).toLowerCase(); + } + + function augment_namenodes(nodes) { + for (var i = 0, e = nodes.length; i < e; ++i) { + var n = nodes[i]; + n.usedPercentage = Math.round(n.used * 1.0 / n.totalSpace * 100); + n.title = "Unavailable"; + n.iconState = "down"; + if (n.isSafeMode === true) { + n.title = capitalise(n.state) + " (safe mode)" + n.iconState = "decommisioned"; + } else if (n.state === "ACTIVE") { + n.title = capitalise(n.state); + n.iconState = "alive"; + } else if (nodes[i].state === "STANDBY") { + n.title = capitalise(n.state); + n.iconState = "down-decommisioned"; + } else if (nodes[i].state === "UNAVAILABLE") { + n.title = capitalise(n.state); + n.iconState = "down"; + } + if (n.namenodeId === "null") { + n.namenodeId = ""; + } + } + } + + r.Nameservices = node_map_to_array(JSON.parse(r.Nameservices)); + augment_namenodes(r.Nameservices); + r.Namenodes = node_map_to_array(JSON.parse(r.Namenodes)); + augment_namenodes(r.Namenodes); + return r; + } + + $.get( + '/jmx?qry=Hadoop:service=Router,name=FederationState', + guard_with_startup_progress(function (resp) { + var data = workaround(resp.beans[0]); + var base = dust.makeBase(HELPERS); + dust.render('namenode-info', base.push(data), function(err, out) { + $('#tab-namenode').html(out); + $('#ui-tabs a[href="#tab-namenode"]').tab('show'); + }); + })).error(ajax_error_handler); + } + + // TODO Copied directly from dfshealth.js; is there a way to import this function? + function load_datanode_info() { + + var HELPERS = { + 'helper_relative_time' : function (chunk, ctx, bodies, params) { + var value = dust.helpers.tap(params.value, chunk, ctx); + return chunk.write(moment().subtract(Number(value), 'seconds').format('YYYY-MM-DD HH:mm:ss')); + }, + 'helper_usage_bar' : function (chunk, ctx, bodies, params) { + var value = dust.helpers.tap(params.value, chunk, ctx); + var v = Number(value); + var r = null; + if (v < 70) { + r = 'progress-bar-success'; + } else if (v < 85) { + r = 'progress-bar-warning'; + } else { + r = "progress-bar-danger"; + } + return chunk.write(r); + }, + }; + + function workaround(r) { + function node_map_to_array(nodes) { + var res = []; + for (var n in nodes) { + var p = nodes[n]; + p.name = n; + res.push(p); + } + return res; + } + + function augment_live_nodes(nodes) { + for (var i = 0, e = nodes.length; i < e; ++i) { + var n = nodes[i]; + n.usedPercentage = Math.round((n.used + n.nonDfsUsedSpace) * 1.0 / n.capacity * 100); + if (n.adminState === "In Service") { + n.state = "alive"; + } else if (nodes[i].adminState === "Decommission In Progress") { + n.state = "decommisioning"; + } else if (nodes[i].adminState === "Decommissioned") { + n.state = "decommissioned"; + } + } + } + + function augment_dead_nodes(nodes) { + for (var i = 0, e = nodes.length; i < e; ++i) { + if (nodes[i].decommissioned) { + nodes[i].state = "down-decommissioned"; + } else { + nodes[i].state = "down"; + } + } + } + + r.LiveNodes = node_map_to_array(JSON.parse(r.LiveNodes)); + augment_live_nodes(r.LiveNodes); + r.DeadNodes = node_map_to_array(JSON.parse(r.DeadNodes)); + augment_dead_nodes(r.DeadNodes); + r.DecomNodes = node_map_to_array(JSON.parse(r.DecomNodes)); + return r; + } + + $.get( + '/jmx?qry=Hadoop:service=NameNode,name=NameNodeInfo', + guard_with_startup_progress(function (resp) { + var data = workaround(resp.beans[0]); + var base = dust.makeBase(HELPERS); + dust.render('datanode-info', base.push(data), function(err, out) { + $('#tab-datanode').html(out); + $('#table-datanodes').dataTable( { + 'lengthMenu': [ [25, 50, 100, -1], [25, 50, 100, "All"] ], + 'columns': [ + { 'orderDataType': 'ng-value', 'searchable': true }, + { 'orderDataType': 'ng-value', 'type': 'numeric' }, + { 'orderDataType': 'ng-value', 'type': 'numeric' }, + { 'orderDataType': 'ng-value', 'type': 'numeric'} + ]}); + $('#ui-tabs a[href="#tab-datanode"]').tab('show'); + }); + })).error(ajax_error_handler); + } + + function load_mount_table() { + var HELPERS = {} + + function workaround(resource) { + resource.MountTable = JSON.parse(resource.MountTable) + return resource; + } + + $.get( + '/jmx?qry=Hadoop:service=Router,name=FederationState', + guard_with_startup_progress(function (resp) { + var data = workaround(resp.beans[0]); + var base = dust.makeBase(HELPERS); + dust.render('mounttable', base.push(data), function(err, out) { + $('#tab-mounttable').html(out); + $('#ui-tabs a[href="#tab-mounttable"]').tab('show'); + }); + })).error(ajax_error_handler); + } + + function toTitleCase(str) { + return str.replace(/\w\S*/g, function(txt){ + return txt.charAt(0).toUpperCase() + txt.substr(1).toLowerCase(); + }); + } + + function show_err_msg(msg) { + $('#alert-panel-body').html(msg); + $('#alert-panel').show(); + } + + function ajax_error_handler(url, jqxhr, text, err) { + show_err_msg('

Failed to retrieve data from ' + url + ', cause: ' + err + '

'); + } + + function guard_with_startup_progress(fn) { + return function() { + try { + fn.apply(this, arguments); + } catch (err) { + if (err instanceof TypeError) { + show_err_msg('Router error: ' + err); + } + } + }; + } + + function load_page() { + var hash = window.location.hash; + switch(hash) { + case "#tab-mounttable": + load_mount_table(); + break; + case "#tab-namenode": + load_namenode_info(); + break; + case "#tab-datanode": + load_datanode_info(); + break; + case "#tab-overview": + load_overview(); + break; + default: + window.location.hash = "tab-overview"; + break; + } + } + load_page(); + + $(window).bind('hashchange', function () { + load_page(); + }); +})(); \ No newline at end of file http://git-wip-us.apache.org/repos/asf/hadoop/blob/27295eec/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/router/index.html ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/router/index.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/router/index.html new file mode 100644 index 0000000..4ee04ac --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/router/index.html @@ -0,0 +1,24 @@ + + + + + +Hadoop Administration + + \ No newline at end of file http://git-wip-us.apache.org/repos/asf/hadoop/blob/27295eec/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java index 58ca1d1..88da77e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java @@ -29,6 +29,7 @@ public class RouterConfigBuilder { private boolean enableRpcServer = false; private boolean enableAdminServer = false; + private boolean enableHttpServer = false; private boolean enableHeartbeat = false; private boolean enableLocalHeartbeat = false; private boolean enableStateStore = false; @@ -45,6 +46,7 @@ public class RouterConfigBuilder { public RouterConfigBuilder all() { this.enableRpcServer = true; this.enableAdminServer = true; + this.enableHttpServer = true; this.enableHeartbeat = true; this.enableLocalHeartbeat = true; this.enableStateStore = true; @@ -67,6 +69,11 @@ public class RouterConfigBuilder { return this; } + public RouterConfigBuilder http(boolean enable) { + this.enableHttpServer = enable; + return this; + } + public RouterConfigBuilder heartbeat(boolean enable) { this.enableHeartbeat = enable; return this; @@ -90,6 +97,10 @@ public class RouterConfigBuilder { return this.admin(true); } + public RouterConfigBuilder http() { + return this.http(true); + } + public RouterConfigBuilder heartbeat() { return this.heartbeat(true); } @@ -108,6 +119,8 @@ public class RouterConfigBuilder { conf.setBoolean(DFSConfigKeys.DFS_ROUTER_RPC_ENABLE, this.enableRpcServer); conf.setBoolean(DFSConfigKeys.DFS_ROUTER_ADMIN_ENABLE, this.enableAdminServer); + conf.setBoolean(DFSConfigKeys.DFS_ROUTER_HTTP_ENABLE, + this.enableHttpServer); conf.setBoolean(DFSConfigKeys.DFS_ROUTER_HEARTBEAT_ENABLE, this.enableHeartbeat); conf.setBoolean(DFSConfigKeys.DFS_ROUTER_MONITOR_LOCAL_NAMENODE, http://git-wip-us.apache.org/repos/asf/hadoop/blob/27295eec/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java index e79674d..4ea0f19 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java @@ -33,6 +33,9 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_CACHE_TIME_TO_LIVE import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_DEFAULT_NAMESERVICE; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_HANDLER_COUNT_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_HEARTBEAT_INTERVAL_MS; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_HTTPS_ADDRESS_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_HTTP_ADDRESS_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_HTTP_BIND_HOST_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_MONITOR_NAMENODE; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_RPC_ADDRESS_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_RPC_BIND_HOST_KEY; @@ -134,6 +137,7 @@ public class RouterDFSCluster { private String nameserviceId; private String namenodeId; private int rpcPort; + private int httpPort; private DFSClient client; private Configuration conf; private RouterClient adminClient; @@ -164,6 +168,10 @@ public class RouterDFSCluster { return this.rpcPort; } + public int getHttpPort() { + return this.httpPort; + } + public FileContext getFileContext() { return this.fileContext; } @@ -183,6 +191,10 @@ public class RouterDFSCluster { this.fileContext = null; } } + InetSocketAddress httpAddress = router.getHttpServerAddress(); + if (httpAddress != null) { + this.httpPort = httpAddress.getPort(); + } } public FileSystem getFileSystem() throws IOException { @@ -399,14 +411,19 @@ public class RouterDFSCluster { conf.set(DFS_NAMENODE_RPC_ADDRESS_KEY + "." + suffix, "127.0.0.1:" + context.rpcPort); - conf.set(DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY + "." + suffix, - "127.0.0.1:" + context.servicePort); conf.set(DFS_NAMENODE_HTTP_ADDRESS_KEY + "." + suffix, "127.0.0.1:" + context.httpPort); conf.set(DFS_NAMENODE_RPC_BIND_HOST_KEY + "." + suffix, "0.0.0.0"); - conf.set(DFS_NAMENODE_SERVICE_RPC_BIND_HOST_KEY + "." + suffix, - "0.0.0.0"); + + // If the service port is enabled by default, we need to set them up + boolean servicePortEnabled = false; + if (servicePortEnabled) { + conf.set(DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY + "." + suffix, + "127.0.0.1:" + context.servicePort); + conf.set(DFS_NAMENODE_SERVICE_RPC_BIND_HOST_KEY + "." + suffix, + "0.0.0.0"); + } } } @@ -447,6 +464,10 @@ public class RouterDFSCluster { conf.set(DFS_ROUTER_ADMIN_ADDRESS_KEY, "127.0.0.1:0"); conf.set(DFS_ROUTER_ADMIN_BIND_HOST_KEY, "0.0.0.0"); + conf.set(DFS_ROUTER_HTTP_ADDRESS_KEY, "127.0.0.1:0"); + conf.set(DFS_ROUTER_HTTPS_ADDRESS_KEY, "127.0.0.1:0"); + conf.set(DFS_ROUTER_HTTP_BIND_HOST_KEY, "0.0.0.0"); + conf.set(DFS_ROUTER_DEFAULT_NAMESERVICE, nameservices.get(0)); conf.setLong(DFS_ROUTER_HEARTBEAT_INTERVAL_MS, heartbeatInterval); conf.setLong(DFS_ROUTER_CACHE_TIME_TO_LIVE_MS, cacheFlushInterval); http://git-wip-us.apache.org/repos/asf/hadoop/blob/27295eec/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/metrics/TestMetricsBase.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/metrics/TestMetricsBase.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/metrics/TestMetricsBase.java index bbcfbe8..b455177 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/metrics/TestMetricsBase.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/metrics/TestMetricsBase.java @@ -64,6 +64,7 @@ public class TestMetricsBase { routerConfig = new RouterConfigBuilder() .stateStore() .metrics() + .http() .build(); router = new Router(); router.init(routerConfig); http://git-wip-us.apache.org/repos/asf/hadoop/blob/27295eec/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java index e22be84..77d1698 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java @@ -24,6 +24,7 @@ import java.io.IOException; import java.net.URISyntaxException; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.server.federation.MockResolver; import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder; @@ -61,10 +62,13 @@ public class TestRouter { conf.set(DFSConfigKeys.DFS_ROUTER_RPC_ADDRESS_KEY, "127.0.0.1:0"); conf.set(DFSConfigKeys.DFS_ROUTER_ADMIN_ADDRESS_KEY, "127.0.0.1:0"); conf.set(DFSConfigKeys.DFS_ROUTER_ADMIN_BIND_HOST_KEY, "0.0.0.0"); + conf.set(DFSConfigKeys.DFS_ROUTER_HTTP_ADDRESS_KEY, "127.0.0.1:0"); + conf.set(DFSConfigKeys.DFS_ROUTER_HTTPS_ADDRESS_KEY, "127.0.0.1:0"); + conf.set(DFSConfigKeys.DFS_ROUTER_HTTP_BIND_HOST_KEY, "0.0.0.0"); // Simulate a co-located NN conf.set(DFSConfigKeys.DFS_NAMESERVICES, "ns0"); - conf.set("fs.defaultFS", "hdfs://" + "ns0"); + conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, "hdfs://" + "ns0"); conf.set(DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY + "." + "ns0", "127.0.0.1:0" + 0); conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY + "." + "ns0", @@ -104,6 +108,9 @@ public class TestRouter { // Admin only testRouterStartup(new RouterConfigBuilder(conf).admin().build()); + // Http only + testRouterStartup(new RouterConfigBuilder(conf).http().build()); + // Rpc only testRouterStartup(new RouterConfigBuilder(conf).rpc().build()); --------------------------------------------------------------------- To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org For additional commands, e-mail: common-commits-help@hadoop.apache.org