Pārlūkot izejas kodu

HDFS-17657. The balancer service supports httpserver. (#7242) Contribtued by Zhaobo Huang.

Reviewed-by: Tao Li <tomscut@apache.org>
Signed-off-by: Shilun Fan <slfan1989@apache.org>
Zhaobo Huang 3 mēneši atpakaļ
vecāks
revīzija
0432761ac8
19 mainītis faili ar 702 papildinājumiem un 8 dzēšanām
  1. 1 1
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3HttpServer.java
  2. 1 1
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterHttpServer.java
  3. 4 0
      hadoop-hdfs-project/hadoop-hdfs/pom.xml
  4. 15 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  5. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  6. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeHttpServer.java
  7. 54 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
  8. 114 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/BalancerHttpServer.java
  9. 43 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/BalancerMXBean.java
  10. 25 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/package-info.java
  11. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
  12. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
  13. 54 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  14. 114 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/balancer/balancer.html
  15. 95 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/balancer/balancer.js
  16. 24 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/balancer/index.html
  17. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/balancer/robots.txt
  18. 100 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerHttpServer.java
  19. 51 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerService.java

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3HttpServer.java

@@ -52,7 +52,7 @@ class Nfs3HttpServer {
         NfsConfigKeys.NFS_HTTPS_ADDRESS_DEFAULT);
     InetSocketAddress httpsAddr = NetUtils.createSocketAddr(httpsAddrString);
 
-    HttpServer2.Builder builder = DFSUtil.httpServerTemplateForNNAndJN(conf,
+    HttpServer2.Builder builder = DFSUtil.getHttpServerTemplate(conf,
         httpAddr, httpsAddr, "nfs3",
         NfsConfigKeys.DFS_NFS_KERBEROS_PRINCIPAL_KEY,
         NfsConfigKeys.DFS_NFS_KEYTAB_FILE_KEY);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterHttpServer.java

@@ -82,7 +82,7 @@ public class RouterHttpServer extends AbstractService {
   protected void serviceStart() throws Exception {
     // Build and start server
     String webApp = "router";
-    HttpServer2.Builder builder = DFSUtil.httpServerTemplateForNNAndJN(
+    HttpServer2.Builder builder = DFSUtil.getHttpServerTemplate(
         this.conf, this.httpAddress, this.httpsAddress, webApp,
         RBFConfigKeys.DFS_ROUTER_KERBEROS_INTERNAL_SPNEGO_PRINCIPAL_KEY,
         RBFConfigKeys.DFS_ROUTER_KEYTAB_FILE_KEY);

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/pom.xml

@@ -283,6 +283,9 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
                 <copy file="${basedir}/src/main/webapps/proto-web.xml"
                       tofile="${project.build.directory}/webapps/journal/WEB-INF/web.xml"
                       filtering="true"/>
+                <copy file="${basedir}/src/main/webapps/proto-web.xml"
+                      tofile="${project.build.directory}/webapps/balancer/WEB-INF/web.xml"
+                      filtering="true"/>
                 <copy file="${basedir}/src/main/webapps/proto-web.xml"
                       tofile="${project.build.directory}/webapps/nfs3/WEB-INF/web.xml"
                       filtering="true"/>
@@ -428,6 +431,7 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
             <exclude>src/main/webapps/hdfs/robots.txt</exclude>
             <exclude>src/main/webapps/journal/robots.txt</exclude>
             <exclude>src/main/webapps/secondary/robots.txt</exclude>
+            <exclude>src/main/webapps/balancer/robots.txt</exclude>
             <exclude>src/contrib/**</exclude>
             <exclude>src/site/resources/images/*</exclude>
             <exclude>src/main/webapps/static/bootstrap-3.4.1/**</exclude>

+ 15 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -811,6 +811,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_BALANCER_ADDRESS_DEFAULT= "0.0.0.0:0";
   public static final String  DFS_BALANCER_KEYTAB_FILE_KEY = "dfs.balancer.keytab.file";
   public static final String  DFS_BALANCER_KERBEROS_PRINCIPAL_KEY = "dfs.balancer.kerberos.principal";
+  public static final String DFS_BALANCER_KERBEROS_INTERNAL_SPNEGO_PRINCIPAL_KEY =
+      "dfs.balancer.kerberos.internal.spnego.principal";
   public static final String  DFS_BALANCER_BLOCK_MOVE_TIMEOUT = "dfs.balancer.block-move.timeout";
   public static final int     DFS_BALANCER_BLOCK_MOVE_TIMEOUT_DEFAULT = 0;
   public static final String  DFS_BALANCER_MAX_NO_MOVE_INTERVAL_KEY = "dfs.balancer.max-no-move-interval";
@@ -821,6 +823,19 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final long    DFS_BALANCER_SERVICE_INTERVAL_DEFAULT = TimeUnit.MINUTES.toMillis(5); //5 mins
   public static final String  DFS_BALANCER_SERVICE_RETRIES_ON_EXCEPTION = "dfs.balancer.service.retries.on.exception";
   public static final int     DFS_BALANCER_SERVICE_RETRIES_ON_EXCEPTION_DEFAULT = 5;
+  public static final String DFS_BALANCER_HTTPSERVER_ENABLED_KEY =
+      "dfs.balancer.httpserver.enabled";
+  public static final Boolean DFS_BALANCER_HTTPSERVER_ENABLED_DEFAULT = false;
+  public static final String DFS_BALANCER_HTTP_ADDRESS_KEY = "dfs.balancer.http-address";
+  public static final int DFS_BALANCER_HTTP_PORT_DEFAULT = 8590;
+  public static final String DFS_BALANCER_HTTP_BIND_HOST_KEY = "dfs.balancer.http-bind-host";
+  public static final String DFS_BALANCER_HTTP_ADDRESS_DEFAULT =
+      "0.0.0.0:" + DFS_BALANCER_HTTP_PORT_DEFAULT;
+  public static final String DFS_BALANCER_HTTPS_ADDRESS_KEY = "dfs.balancer.https-address";
+  public static final int DFS_BALANCER_HTTPS_PORT_DEFAULT = 8591;
+  public static final String DFS_BALANCER_HTTPS_BIND_HOST_KEY = "dfs.balancer.https-bind-host";
+  public static final String DFS_BALANCER_HTTPS_ADDRESS_DEFAULT =
+      "0.0.0.0:" + DFS_BALANCER_HTTPS_PORT_DEFAULT;
 
   public static final String  DFS_MOVER_MOVEDWINWIDTH_KEY = "dfs.mover.movedWinWidth";
   public static final long    DFS_MOVER_MOVEDWINWIDTH_DEFAULT = 5400*1000L;

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java

@@ -1736,11 +1736,11 @@ public class DFSUtil {
   }
 
   /**
-   * Return a HttpServer.Builder that the journalnode / namenode / secondary
+   * Return a HttpServer.Builder that the journalnode / namenode / secondary / router / balancer
    * namenode can use to initialize their HTTP / HTTPS server.
    *
    */
-  public static HttpServer2.Builder httpServerTemplateForNNAndJN(
+  public static HttpServer2.Builder getHttpServerTemplate(
       Configuration conf, final InetSocketAddress httpAddr,
       final InetSocketAddress httpsAddr, String name, String spnegoUserNameKey,
       String spnegoKeytabFileKey) throws IOException {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeHttpServer.java

@@ -73,7 +73,7 @@ public class JournalNodeHttpServer {
       }
     }
 
-    HttpServer2.Builder builder = DFSUtil.httpServerTemplateForNNAndJN(conf,
+    HttpServer2.Builder builder = DFSUtil.getHttpServerTemplate(conf,
         httpAddr, httpsAddr, "journal",
         DFSConfigKeys.DFS_JOURNALNODE_KERBEROS_INTERNAL_SPNEGO_PRINCIPAL_KEY,
         DFSConfigKeys.DFS_JOURNALNODE_KEYTAB_FILE_KEY);

+ 54 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java

@@ -38,6 +38,8 @@ import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import javax.management.ObjectName;
+
 import org.apache.commons.lang3.builder.ToStringBuilder;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.source.JvmMetrics;
@@ -66,6 +68,7 @@ import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.metrics2.util.MBeans;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -76,6 +79,7 @@ import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 
 import org.apache.hadoop.util.Preconditions;
+import org.apache.hadoop.util.VersionInfo;
 
 /** <p>The balancer is a tool that balances disk space usage on an HDFS cluster
  * when some datanodes become full or when new empty nodes join the cluster.
@@ -180,7 +184,7 @@ import org.apache.hadoop.util.Preconditions;
  */
 
 @InterfaceAudience.Private
-public class Balancer {
+public class Balancer implements BalancerMXBean {
   static final Logger LOG = LoggerFactory.getLogger(Balancer.class);
 
   static final Path BALANCER_ID_PATH = new Path("/system/balancer.id");
@@ -241,6 +245,7 @@ public class Balancer {
   private final boolean sortTopNodes;
   private final int limitOverUtilizedNum;
   private final BalancerMetrics metrics;
+  private ObjectName balancerInfoBeanName;
 
   // all data node lists
   private final Collection<Source> overUtilized = new LinkedList<Source>();
@@ -377,6 +382,8 @@ public class Balancer {
         DFSConfigKeys.DFS_BLOCK_SIZE_KEY,
         DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT);
     this.metrics = BalancerMetrics.create(this);
+
+    registerBalancerMXBean();
   }
   
   private static long getCapacity(DatanodeStorageReport report, StorageType t) {
@@ -680,6 +687,13 @@ public class Balancer {
             left.getDatanodeInfo(), right.getDatanodeInfo());
   }
 
+  /**
+   * Register BalancerMXBean.
+   */
+  private void registerBalancerMXBean() {
+    balancerInfoBeanName = MBeans.register("Balancer", "BalancerInfo", this);
+  }
+
   /* reset all fields in a balancer preparing for the next iteration */
   void resetData(Configuration conf) {
     this.overUtilized.clear();
@@ -689,12 +703,32 @@ public class Balancer {
     this.policy.reset();
     this.dispatcher.reset(conf);
     DefaultMetricsSystem.removeSourceName(metrics.getName());
+    if (balancerInfoBeanName != null) {
+      MBeans.unregister(balancerInfoBeanName);
+      balancerInfoBeanName = null;
+    }
   }
 
   NameNodeConnector getNnc() {
     return nnc;
   }
 
+  @Override
+  public String getVersion() {
+    return VersionInfo.getVersion() + ", r" + VersionInfo.getRevision();
+  }
+
+  @Override
+  public String getSoftwareVersion() {
+    return VersionInfo.getVersion();
+  }
+
+  @Override
+  public String getCompileInfo() {
+    return VersionInfo.getDate() + " by " + VersionInfo.getUser() + " from "
+        + VersionInfo.getBranch();
+  }
+
   static class Result {
     private final ExitStatus exitStatus;
     private final long bytesLeftToMove;
@@ -860,6 +894,7 @@ public class Balancer {
         + "  NameNode");
     
     List<NameNodeConnector> connectors = Collections.emptyList();
+    BalancerHttpServer balancerHttpServer = startBalancerHttpServer(conf);
     try {
       connectors = NameNodeConnector.newNameNodeConnectors(namenodes, nsIds,
           Balancer.class.getSimpleName(), BALANCER_ID_PATH, conf,
@@ -872,6 +907,9 @@ public class Balancer {
           if (p.getBlockPools().size() == 0
               || p.getBlockPools().contains(nnc.getBlockpoolID())) {
             final Balancer b = new Balancer(nnc, p, conf);
+            if (balancerHttpServer != null) {
+              balancerHttpServer.setBalancerAttribute(b);
+            }
             final Result r = b.runOneIteration();
             r.print(iteration, nnc, System.out);
 
@@ -898,6 +936,9 @@ public class Balancer {
       for(NameNodeConnector nnc : connectors) {
         IOUtils.cleanupWithLogger(LOG, nnc);
       }
+      if (balancerHttpServer != null) {
+        balancerHttpServer.stop();
+      }
     }
     return ExitStatus.SUCCESS.getExitCode();
   }
@@ -969,6 +1010,18 @@ public class Balancer {
     serviceRunning = false;
   }
 
+  private static BalancerHttpServer startBalancerHttpServer(Configuration conf) throws IOException {
+    boolean httpServerEnabled = conf.getBoolean(DFSConfigKeys.DFS_BALANCER_HTTPSERVER_ENABLED_KEY,
+        DFSConfigKeys.DFS_BALANCER_HTTPSERVER_ENABLED_DEFAULT);
+    if (httpServerEnabled) {
+      BalancerHttpServer balancerHttpServer = new BalancerHttpServer(conf);
+      balancerHttpServer.start();
+      return balancerHttpServer;
+    } else {
+      return null;
+    }
+  }
+
   private static void checkKeytabAndInit(Configuration conf)
       throws IOException {
     if (conf.getBoolean(DFSConfigKeys.DFS_BALANCER_KEYTAB_ENABLED_KEY,

+ 114 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/BalancerHttpServer.java

@@ -0,0 +1,114 @@
+/**
+ * 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.balancer;
+
+import java.io.IOException;
+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.HttpConfig;
+import org.apache.hadoop.http.HttpServer2;
+import org.apache.hadoop.net.NetUtils;
+
+public class BalancerHttpServer {
+
+  private static final String BALANCER_ATTRIBUTE_KEY = "current.balancer";
+
+  private final Configuration conf;
+  private InetSocketAddress httpAddress;
+  private InetSocketAddress httpsAddress;
+  private HttpServer2 httpServer;
+
+  public BalancerHttpServer(Configuration conf) {
+    this.conf = conf;
+  }
+
+  public void start() throws IOException {
+    String webApp = "balancer";
+    // Get HTTP address
+    httpAddress = conf.getSocketAddr(DFSConfigKeys.DFS_BALANCER_HTTP_BIND_HOST_KEY,
+        DFSConfigKeys.DFS_BALANCER_HTTP_ADDRESS_KEY,
+        DFSConfigKeys.DFS_BALANCER_HTTP_ADDRESS_DEFAULT,
+        DFSConfigKeys.DFS_BALANCER_HTTP_PORT_DEFAULT);
+
+    // Get HTTPs address
+    httpsAddress = conf.getSocketAddr(DFSConfigKeys.DFS_BALANCER_HTTPS_BIND_HOST_KEY,
+        DFSConfigKeys.DFS_BALANCER_HTTPS_ADDRESS_KEY,
+        DFSConfigKeys.DFS_BALANCER_HTTPS_ADDRESS_DEFAULT,
+        DFSConfigKeys.DFS_BALANCER_HTTPS_PORT_DEFAULT);
+
+    HttpServer2.Builder builder =
+        DFSUtil.getHttpServerTemplate(conf, httpAddress, httpsAddress, webApp,
+            DFSConfigKeys.DFS_BALANCER_KERBEROS_INTERNAL_SPNEGO_PRINCIPAL_KEY,
+            DFSConfigKeys.DFS_BALANCER_KEYTAB_FILE_KEY);
+
+    final boolean xFrameEnabled = conf.getBoolean(DFSConfigKeys.DFS_XFRAME_OPTION_ENABLED,
+        DFSConfigKeys.DFS_XFRAME_OPTION_ENABLED_DEFAULT);
+
+    final String xFrameOptionValue = conf.getTrimmed(DFSConfigKeys.DFS_XFRAME_OPTION_VALUE,
+        DFSConfigKeys.DFS_XFRAME_OPTION_VALUE_DEFAULT);
+
+    builder.configureXFrame(xFrameEnabled).setXFrameOption(xFrameOptionValue);
+
+    httpServer = builder.build();
+    httpServer.setAttribute(JspHelper.CURRENT_CONF, conf);
+    httpServer.start();
+
+    HttpConfig.Policy policy = DFSUtil.getHttpPolicy(conf);
+    int connIdx = 0;
+    if (policy.isHttpEnabled()) {
+      httpAddress = httpServer.getConnectorAddress(connIdx++);
+      if (httpAddress != null) {
+        conf.set(DFSConfigKeys.DFS_BALANCER_HTTP_ADDRESS_KEY,
+            NetUtils.getHostPortString(httpAddress));
+      }
+    }
+    if (policy.isHttpsEnabled()) {
+      httpsAddress = httpServer.getConnectorAddress(connIdx);
+      if (httpsAddress != null) {
+        conf.set(DFSConfigKeys.DFS_BALANCER_HTTPS_ADDRESS_KEY,
+            NetUtils.getHostPortString(httpsAddress));
+      }
+    }
+  }
+
+  public void setBalancerAttribute(Balancer balancer) {
+    httpServer.setAttribute(BALANCER_ATTRIBUTE_KEY, balancer);
+  }
+
+  public void stop() throws IOException {
+    if (httpServer != null) {
+      try {
+        httpServer.stop();
+      } catch (Exception e) {
+        throw new IOException(e);
+      }
+    }
+  }
+
+  public InetSocketAddress getHttpAddress() {
+    return httpAddress;
+  }
+
+  public InetSocketAddress getHttpsAddress() {
+    return httpsAddress;
+  }
+}

+ 43 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/BalancerMXBean.java

@@ -0,0 +1,43 @@
+/**
+ * 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.balancer;
+
+public interface BalancerMXBean {
+
+  /**
+   * Gets the version of Hadoop.
+   *
+   * @return the version of Hadoop
+   */
+  String getVersion();
+
+  /**
+   * Get the version of software running on the Balancer.
+   *
+   * @return a string representing the version.
+   */
+  String getSoftwareVersion();
+
+  /**
+   * Get the compilation information which contains date, user and branch.
+   *
+   * @return the compilation information, as a JSON string.
+   */
+  String getCompileInfo();
+
+}

+ 25 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/package-info.java

@@ -0,0 +1,25 @@
+/**
+ * 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.
+ */
+
+/**
+ * The Balancer is a tool for balancing the data across
+ * the storage devices of an HDFS cluster.
+ * Over time, the data in the HDFS storage can become skewed,
+ * The Balancer moves data blocks between DataNodes to balance data distribution.
+ */
+package org.apache.hadoop.hdfs.server.balancer;

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java

@@ -149,7 +149,7 @@ public class NameNodeHttpServer {
       }
     }
 
-    HttpServer2.Builder builder = DFSUtil.httpServerTemplateForNNAndJN(conf,
+    HttpServer2.Builder builder = DFSUtil.getHttpServerTemplate(conf,
         httpAddr, httpsAddr, "hdfs",
         DFSConfigKeys.DFS_NAMENODE_KERBEROS_INTERNAL_SPNEGO_PRINCIPAL_KEY,
         DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java

@@ -474,7 +474,7 @@ public class SecondaryNameNode implements Runnable,
         DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTPS_ADDRESS_DEFAULT);
     InetSocketAddress httpsAddr = NetUtils.createSocketAddr(httpsAddrString);
 
-    HttpServer2.Builder builder = DFSUtil.httpServerTemplateForNNAndJN(conf,
+    HttpServer2.Builder builder = DFSUtil.getHttpServerTemplate(conf,
         httpAddr, httpsAddr, "secondary", DFSConfigKeys.
             DFS_SECONDARY_NAMENODE_KERBEROS_INTERNAL_SPNEGO_PRINCIPAL_KEY,
         DFSConfigKeys.DFS_SECONDARY_NAMENODE_KEYTAB_FILE_KEY);

+ 54 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -4114,6 +4114,60 @@
   </description>
 </property>
 
+  <property>
+    <name>dfs.balancer.kerberos.internal.spnego.principal</name>
+    <value></value>
+    <description>
+      Kerberos SPNEGO principal name used by the balancer.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.balancer.httpserver.enabled</name>
+    <value>false</value>
+    <description>
+      If true, balancer will start httpserver.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.balancer.http-address</name>
+    <value>0.0.0.0:8590</value>
+    <description>
+      The balancer http server address and port.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.balancer.http-bind-host</name>
+    <value></value>
+    <description>
+      The actual address the HTTP server will bind to. If this optional address
+      is set, it overrides only the hostname portion of
+      dfs.balancer.http-address. This is useful for making the Balancer
+      HTTP server listen on all interfaces by setting it to 0.0.0.0.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.balancer.https-address</name>
+    <value>0.0.0.0:8591</value>
+    <description>
+      The balancer HTTPS server address and port.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.balancer.https-bind-host</name>
+    <value></value>
+    <description>
+      The actual address the HTTP server will bind to. If this optional address
+      is set, it overrides only the hostname portion of
+      dfs.balancer.https-address. This is useful for making the Balancer
+      HTTP server listen on all interfaces by setting it to 0.0.0.0.
+    </description>
+  </property>
+
 <property>
   <name>dfs.http.client.retry.policy.enabled</name>
   <value>false</value>

+ 114 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/balancer/balancer.html

@@ -0,0 +1,114 @@
+<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Strict//EN"
+        "http://www.w3.org/TR/xhtml1/DTD/xhtml1-strict.dtd">
+<!--
+   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.
+-->
+<html xmlns="http://www.w3.org/1999/xhtml">
+<head>
+  <meta http-equiv="X-UA-Compatible" content="IE=edge" />
+  <link rel="stylesheet" type="text/css" href="/static/bootstrap-3.4.1/css/bootstrap.min.css" />
+  <link rel="stylesheet" type="text/css" href="/static/hadoop.css" />
+  <title>Balancer Information</title>
+</head>
+
+<body>
+<header class="navbar navbar-inverse bs-docs-nav" role="banner">
+  <div class="container">
+    <div class="navbar-header">
+      <div class="navbar-brand">Hadoop</div>
+    </div>
+
+    <ul class="nav navbar-nav" id="ui-tabs">
+      <li><a href="#tab-overview">Overview</a></li>
+      <li class="dropdown">
+        <a href="#" class="dropdown-toggle" data-toggle="dropdown">Utilities <b class="caret"></b></a>
+        <ul class="dropdown-menu">
+          <li><a href="logs/">Logs</a></li>
+          <li><a href="logLevel">Log Level</a></li>
+          <li><a href="jmx">Metrics</a></li>
+          <li><a href="conf">Configuration</a></li>
+          <li><a href="stacks">Process Thread Dump</a></li>
+        </ul>
+      </li>
+    </ul>
+  </div>
+</header>
+
+<div class="container">
+  <div id="alert-panel">
+    <div class="alert alert-danger">
+      <button type="button" class="close" onclick="$('#alert-panel').hide();">&times;</button>
+      <div class="alert-body" id="alert-panel-body"></div>
+    </div>
+  </div>
+
+  <div class="tab-content">
+    <div class="tab-pane" id="tab-overview"></div>
+  </div>
+
+  <div class="row">
+    <hr />
+    <div class="col-xs-2"><p>Hadoop, {release-year-token}.</p></div>
+  </div>
+</div>
+
+<script type="text/x-dust-template" id="tmpl-balancer">
+  {#balancer}
+  <div class="page-header">
+    <h1>Balancer on <small>{HostName}</small></h1>
+  </div>
+  <table class="table table-bordered table-striped">
+    <tr><th>Started:</th><td>{BalancerStartedTimeInMillis|date_tostring}</td></tr>
+    <tr><th>Version:</th><td>{Version}</td></tr>
+    <tr><th>Compiled:</th><td>{CompileInfo}</td></tr>
+  </table>
+  {/balancer}
+
+  <div class="page-header">
+    <h1>Balancer Status</h1>
+  </div>
+  <table class="table">
+    <thead>
+      <tr>
+        <th>Block Pool ID</th>
+        <th>BytesLeftToMove</th>
+        <th>NumOfOverUtilizedNodes</th>
+        <th>NumOfUnderUtilizedNodes</th>
+        <th>BytesMovedInCurrentRun</th>
+      </tr>
+    </thead>
+    {#balancers}
+    <tr>
+      <td>{BlockPoolID}</td>
+      <td>{BytesLeftToMove|fmt_bytes}</td>
+      <td>{NumOfOverUtilizedNodes}</td>
+      <td>{NumOfUnderUtilizedNodes}</td>
+      <td>{BytesMovedInCurrentRun|fmt_bytes}</td>
+    </tr>
+    {/balancers}
+  </table>
+</script>
+
+<script type="text/javascript" src="/static/jquery-3.6.0.min.js"></script>
+<script type="text/javascript" src="/static/bootstrap-3.4.1/js/bootstrap.min.js"></script>
+<script type="text/javascript" src="/static/moment.min.js"></script>
+<script type="text/javascript" src="/static/dust-full-2.0.0.min.js"></script>
+<script type="text/javascript" src="/static/dust-helpers-1.1.1.min.js"></script>
+<script type="text/javascript" src="/static/dfs-dust.js"></script>
+<script type="text/javascript" src="balancer.js"></script>
+
+</body>
+</html>

+ 95 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/balancer/balancer.js

@@ -0,0 +1,95 @@
+/**
+ * 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";
+
+    var data = {};
+
+    dust.loadSource(dust.compile($('#tmpl-balancer').html(), 'balancer'));
+
+    var BEANS = [
+        {"name": "mBeanServerInfo", "url": "/jmx?qry=JMImplementation:type=MBeanServerDelegate"},
+        {"name": "balancer", "url": "/jmx?qry=Hadoop:service=Balancer,name=BalancerInfo"},
+        {"name": "balancers", "url": "/jmx?qry=Hadoop:service=Balancer,name=Balancer-*"}
+    ];
+
+    load_json(BEANS, guard_with_startup_progress(function (d) {
+        for (var k in d) {
+            data[k] = k === 'balancers' ? workaround(d[k].beans) : d[k].beans[0];
+        }
+
+        extractMetrics();
+
+        render();
+    }), function (url, jqxhr, text, err) {
+        show_err_msg('<p>Failed to retrieve data from ' + url + ', cause: ' + err + '</p>');
+    });
+
+    function guard_with_startup_progress(fn) {
+        return function () {
+            try {
+                fn.apply(this, arguments);
+            } catch (err) {
+                if (err instanceof TypeError) {
+                    show_err_msg('Balancer error: ' + err);
+                }
+            }
+        };
+    }
+
+    function workaround(balancers) {
+        for (var balancer of balancers) {
+            var str = balancer['modelerType'];
+            var index = str.indexOf("-");
+            balancer['BlockPoolID'] = str.substring(index + 1);
+        }
+        return balancers;
+    }
+
+    function extractMetrics() {
+        var str = data['mBeanServerInfo']['MBeanServerId'];
+        var index = str.indexOf("_");
+        if (index !== -1) {
+            data['balancer']['HostName'] = str.substring(0, index);
+            data['balancer']['BalancerStartedTimeInMillis'] = str.substring(index + 1);
+        } else {
+            data['balancer']['HostName'] = "invalid data";
+            data['balancer']['BalancerStartedTimeInMillis'] = "invalid data";
+        }
+    }
+
+    var HELPERS = {
+        'helper_date_tostring': function (chunk, ctx, bodies, params) {
+            var value = dust.helpers.tap(params.value, chunk, ctx);
+            return chunk.write('' + moment(Number(value)).format('ddd MMM DD HH:mm:ss ZZ YYYY'));
+        }
+    };
+
+    function render() {
+        var base = dust.makeBase(HELPERS);
+        dust.render('balancer', base.push(data), function (err, out) {
+            $('#tab-overview').html(out);
+            $('#tab-overview').addClass('active');
+        });
+    }
+
+    function show_err_msg() {
+        $('#alert-panel-body').html("Failed to load balancer information");
+        $('#alert-panel').show();
+    }
+})();

+ 24 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/balancer/index.html

@@ -0,0 +1,24 @@
+<!--
+   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.
+-->
+<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Strict//EN"
+        "http://www.w3.org/TR/xhtml1/DTD/xhtml1-strict.dtd">
+<html xmlns="http://www.w3.org/1999/xhtml">
+<head>
+  <meta http-equiv="REFRESH" content="0;url=balancer.html" />
+  <title>Hadoop Administration</title>
+</head>
+</html>

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/balancer/robots.txt

@@ -0,0 +1,2 @@
+User-agent: *
+Disallow: /

+ 100 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerHttpServer.java

@@ -0,0 +1,100 @@
+/**
+ * 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.balancer;
+
+import java.io.File;
+import java.net.InetSocketAddress;
+import java.net.URL;
+import java.net.URLConnection;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.web.URLConnectionFactory;
+import org.apache.hadoop.http.HttpConfig;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.test.GenericTestUtils;
+
+import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
+
+public class TestBalancerHttpServer {
+  private static final String BASEDIR =
+      GenericTestUtils.getTempPath(TestBalancerHttpServer.class.getSimpleName());
+  private static String keystoresDir;
+  private static String sslConfDir;
+  private static Configuration conf;
+  private static URLConnectionFactory connectionFactory;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    conf = new Configuration();
+    conf.set(DFSConfigKeys.DFS_HTTP_POLICY_KEY, HttpConfig.Policy.HTTP_ONLY.name());
+    conf.set(DFSConfigKeys.DFS_BALANCER_HTTP_ADDRESS_KEY, "localhost:0");
+    conf.set(DFSConfigKeys.DFS_BALANCER_HTTPS_ADDRESS_KEY, "localhost:0");
+    File base = new File(BASEDIR);
+    FileUtil.fullyDelete(base);
+    base.mkdirs();
+    keystoresDir = new File(BASEDIR).getAbsolutePath();
+    sslConfDir = KeyStoreTestUtil.getClasspathDir(TestBalancerHttpServer.class);
+    KeyStoreTestUtil.setupSSLConfig(keystoresDir, sslConfDir, conf, false);
+    connectionFactory = URLConnectionFactory.newDefaultURLConnectionFactory(conf);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    FileUtil.fullyDelete(new File(BASEDIR));
+    KeyStoreTestUtil.cleanupSSLConfig(keystoresDir, sslConfDir);
+  }
+
+  @Test
+  public void testHttpServer() throws Exception {
+    BalancerHttpServer server = null;
+    try {
+      server = new BalancerHttpServer(conf);
+      server.start();
+      assertThat(checkConnection("http", server.getHttpAddress())).isTrue();
+      assertThat(checkConnection("https", server.getHttpsAddress())).isFalse();
+    } finally {
+      if (server != null) {
+        server.stop();
+      }
+    }
+  }
+
+  private boolean checkConnection(String scheme, InetSocketAddress address) {
+    if (address == null) {
+      return false;
+    }
+    try {
+      URL url = new URL(scheme + "://" + NetUtils.getHostPortString(address));
+      URLConnection conn = connectionFactory.openConnection(url);
+      conn.setConnectTimeout(5 * 1000);
+      conn.setReadTimeout(5 * 1000);
+      conn.connect();
+      conn.getContent();
+      return true;
+    } catch (Exception e) {
+      return false;
+    }
+  }
+}

+ 51 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerService.java

@@ -33,10 +33,16 @@ import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.MetricsAsserts;
 import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.VersionInfo;
+
 import org.junit.Test;
 
+import java.lang.management.ManagementFactory;
 import java.util.concurrent.TimeUnit;
 
+import javax.management.MBeanServer;
+import javax.management.ObjectName;
+
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -210,4 +216,49 @@ public class TestBalancerService {
       }
     }
   }
+
+  @Test(timeout = 60000)
+  public void testBalancerServiceMetrics() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    conf.setTimeDuration(DFSConfigKeys.DFS_BALANCER_SERVICE_INTERVAL_KEY, 5, TimeUnit.SECONDS);
+    TestBalancer.initConf(conf);
+    try {
+      setupCluster(conf);
+      TestBalancerWithHANameNodes.waitStoragesNoStale(cluster, client, 0);
+      long totalCapacity = addOneDataNode(conf); // make cluster imbalanced
+
+      Thread balancerThread = newBalancerService(conf, new String[] {"-asService"});
+      balancerThread.start();
+
+      MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+      GenericTestUtils.waitFor(() -> {
+        try {
+          ObjectName mxbeanName = new ObjectName("Hadoop:service=Balancer,name=BalancerInfo");
+          String version = (String) mbs.getAttribute(mxbeanName, "Version");
+          return version.contains(VersionInfo.getVersion()) &&
+              version.contains(VersionInfo.getRevision());
+        } catch (Exception e) {
+          return false;
+        }
+      }, 100, 10000);
+
+      TestBalancer.waitForBalancer(totalUsedSpace, totalCapacity, client, cluster,
+          BalancerParameters.DEFAULT);
+
+      cluster.triggerHeartbeats();
+      cluster.triggerBlockReports();
+
+      // add another empty datanode, wait for cluster become balance again
+      totalCapacity = addOneDataNode(conf);
+      TestBalancer.waitForBalancer(totalUsedSpace, totalCapacity, client, cluster,
+          BalancerParameters.DEFAULT);
+
+      Balancer.stop();
+      balancerThread.join();
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }