You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ni...@apache.org on 2021/09/25 01:49:16 UTC

[hbase] branch HBASE-25714 updated: HBASE-26044 Add CompactionServer Web UI (#3694)

This is an automated email from the ASF dual-hosted git repository.

niuyulin pushed a commit to branch HBASE-25714
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/HBASE-25714 by this push:
     new 8769e3e  HBASE-26044 Add CompactionServer Web UI (#3694)
8769e3e is described below

commit 8769e3e093e0a1bc1310de98d483fecd89267e69
Author: niuyulin <yu...@gmail.com>
AuthorDate: Fri Sep 24 20:48:31 2021 -0500

    HBASE-26044 Add CompactionServer Web UI (#3694)
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
---
 hbase-server/pom.xml                               |   2 +
 .../hbase/tmpl/compactionserver/CSStatusTmpl.jamon | 199 +++++++++++++++++++++
 .../compactionserver/CompactionTasksTmpl.jamon     |  71 ++++++++
 .../org/apache/hadoop/hbase/AbstractServer.java    |  45 +++++
 .../hbase/compactionserver/CSDumpServlet.java      |  93 ++++++++++
 .../hbase/compactionserver/CSStatusServlet.java    |  55 ++++++
 .../hbase/compactionserver/CompactionTask.java     |   2 +-
 .../compactionserver/CompactionThreadManager.java  |   2 +-
 .../hbase/compactionserver/HCompactionServer.java  |  48 +++++
 .../hadoop/hbase/regionserver/HRegionServer.java   |  38 +---
 .../hadoop/hbase/regionserver/RSRpcServices.java   |   3 +-
 .../compactionserver/compactionserver.jsp          |  20 +++
 .../hbase-webapps/compactionserver/footer.jsp      |  30 ++++
 .../hbase-webapps/compactionserver/header.jsp      |  65 +++++++
 .../hbase-webapps/compactionserver/index.html      |  20 +++
 15 files changed, 655 insertions(+), 38 deletions(-)

diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index 8588357..ce05ca8 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -173,6 +173,8 @@
                 <jspcompiler uriroot="${src.webapps}/master" outputdir="${generated.sources}/java" package="org.apache.hadoop.hbase.generated.master" webxml="${build.webapps}/master/WEB-INF/web.xml"/>
                 <mkdir dir="${build.webapps}/regionserver/WEB-INF"/>
                 <jspcompiler uriroot="${src.webapps}/regionserver" outputdir="${generated.sources}/java" package="org.apache.hadoop.hbase.generated.regionserver" webxml="${build.webapps}/regionserver/WEB-INF/web.xml"/>
+                <mkdir dir="${build.webapps}/compactionserver/WEB-INF"/>
+                <jspcompiler uriroot="${src.webapps}/compactionserver" outputdir="${generated.sources}/java" package="org.apache.hadoop.hbase.generated.compactionserver" webxml="${build.webapps}/compactionserver/WEB-INF/web.xml"/>
               </target>
             </configuration>
             <goals>
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/compactionserver/CSStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/compactionserver/CSStatusTmpl.jamon
new file mode 100644
index 0000000..0529b4b
--- /dev/null
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/compactionserver/CSStatusTmpl.jamon
@@ -0,0 +1,199 @@
+<%doc>
+
+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.
+</%doc>
+<%args>
+HCompactionServer compactionServer;
+String filter = "general";
+String format = "html";
+String bcn = "";
+String bcv = "";
+</%args>
+<%import>
+java.util.*;
+org.apache.hadoop.hbase.client.RegionInfo;
+org.apache.hadoop.hbase.ServerName;
+org.apache.hadoop.hbase.HBaseConfiguration;
+org.apache.hadoop.hbase.io.hfile.CacheConfig;
+org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo;
+org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
+org.apache.hadoop.hbase.compactionserver.HCompactionServer;
+
+</%import>
+
+<%java>
+ServerName serverName = compactionServer.getServerName();
+</%java>
+
+<%class>
+  public String formatZKString() {
+    StringBuilder quorums = new StringBuilder();
+    String zkQuorum = compactionServer.getZooKeeper().getQuorum();
+
+    if (null == zkQuorum) {
+      return quorums.toString();
+    }
+
+    String[] zks = zkQuorum.split(",");
+
+    if (zks.length == 0) {
+      return quorums.toString();
+    }
+
+    for(int i = 0; i < zks.length; ++i) {
+      quorums.append(zks[i].trim());
+
+      if (i != (zks.length - 1)) {
+        quorums.append("<br/>");
+      }
+    }
+    return quorums.toString();
+  }
+</%class>
+
+<!DOCTYPE html>
+<?xml version="1.0" encoding="UTF-8" ?>
+<html lang="en">
+  <head>
+    <meta charset="utf-8">
+    <title>HBase Compaction Server: CSStatusServlet</title>
+    <meta name="viewport" content="width=device-width, initial-scale=1.0">
+    <meta name="description" content="">
+
+    <link href="/static/css/bootstrap.min.css" rel="stylesheet">
+    <link href="/static/css/bootstrap-theme.min.css" rel="stylesheet">
+    <link href="/static/css/hbase.css" rel="stylesheet">
+  </head>
+
+  <body>
+
+    <div class="navbar  navbar-fixed-top navbar-default">
+        <div class="container-fluid">
+            <div class="navbar-header">
+                <button type="button" class="navbar-toggle" data-toggle="collapse" data-target=".navbar-collapse">
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a class="navbar-brand" href="/cm-status"><img src="/static/hbase_logo_small.png" alt="HBase Logo"/></a>
+            </div>
+            <div class="collapse navbar-collapse">
+                <ul class="nav navbar-nav">
+                <li class="active"><a href="/">Home</a></li>
+                <!-- <li><a href="/processRS.jsp">Process Metrics</a></li> -->
+                <li><a href="/logs/">Local Logs</a></li>
+                <li><a href="/logLevel">Log Level</a></li>
+                <li><a href="/dump">Debug Dump</a></li>
+                <li><a href="/jmx">Metrics Dump</a></li>
+                <li><a href="/prof">Profiler</a></li>
+                <%if HBaseConfiguration.isShowConfInServlet()%>
+                <li><a href="/conf">HBase Configuration</a></li>
+                </%if>
+                </ul>
+            </div>
+        </div>
+    </div>
+
+  <div class="container-fluid content">
+    <div class="row inner_header">
+        <div class="page-header">
+            <h1>CompactionServer <small><% serverName %></small></h1>
+        </div>
+    </div>
+
+    <div class="row">
+      <section>
+        <& ../common/TaskMonitorTmpl; filter = filter &>
+      </section>
+
+      <section>
+        <h2><a name="tasks">Running Compaction Tasks</a></h2>
+        <& CompactionTasksTmpl; compactionServer= compactionServer&>
+      </section>
+      <section>
+        <h2>Software Attributes</h2>
+        <table id="attributes_table" class="table table-striped">
+           <tr>
+                  <th>Attribute Name</th>
+                  <th>Value</th>
+                  <th>Description</th>
+           </tr>
+           <tr>
+                  <td>HBase Version</td>
+                  <td><% org.apache.hadoop.hbase.util.VersionInfo.getVersion() %>, revision=<% org.apache.hadoop.hbase.util.VersionInfo.getRevision() %></td>
+                  <td>HBase version and revision</td>
+           </tr>
+           <tr>
+                  <td>HBase Compiled</td>
+                  <td><% org.apache.hadoop.hbase.util.VersionInfo.getDate() %>, <% org.apache.hadoop.hbase.util.VersionInfo.getUser() %></td>
+                  <td>When HBase version was compiled and by whom</td>
+           </tr>
+           <tr>
+                  <td>HBase Source Checksum</td>
+                  <td><% org.apache.hadoop.hbase.util.VersionInfo.getSrcChecksum() %></td>
+                  <td>HBase source MD5 checksum</td>
+           </tr>
+           <tr>
+                  <td>Hadoop Version</td>
+                  <td><% org.apache.hadoop.util.VersionInfo.getVersion() %>, revision=<% org.apache.hadoop.util.VersionInfo.getRevision() %></td>
+                  <td>Hadoop version and revision</td>
+           </tr>
+           <tr>
+                  <td>Hadoop Compiled</td>
+                  <td><% org.apache.hadoop.util.VersionInfo.getDate() %>, <% org.apache.hadoop.util.VersionInfo.getUser() %></td>
+                  <td>When Hadoop version was compiled and by whom</td>
+           </tr>
+           <tr>
+                  <td>Hadoop Source Checksum</td>
+                  <td><% org.apache.hadoop.util.VersionInfo.getSrcChecksum() %></td>
+                  <td>Hadoop source MD5 checksum</td>
+           </tr>
+           <tr>
+                  <td>ZooKeeper Client Version</td>
+                  <td><% org.apache.zookeeper.Version.getVersion() %>, revision=<% org.apache.zookeeper.Version.getRevisionHash() %></td>
+                  <td>ZooKeeper client version and revision</td>
+           </tr>
+           <tr>
+                  <td>ZooKeeper Client Compiled</td>
+                  <td><% org.apache.zookeeper.Version.getBuildDate() %></td>
+                  <td>When ZooKeeper client version was compiled</td>
+           </tr>
+           <tr>
+                  <td>ZooKeeper Quorum</td>
+                  <%escape #n>
+                  <td><% formatZKString() %></td>
+                  <td>Addresses of all registered ZK servers</td>
+            </tr>
+            <tr>
+                  <td>CS Start Time</td>
+                  <td><% new Date(compactionServer.getStartcode()) %></td>
+                  <td>Date stamp of when this compaction server was started</td>
+            </tr>
+        </table>
+        </section>
+    </div>
+  </div>
+<script src="/static/js/jquery.min.js" type="text/javascript"></script>
+<script src="/static/js/jquery.tablesorter.min.js" type="text/javascript"></script>
+<script src="/static/js/bootstrap.min.js" type="text/javascript"></script>
+<script src="/static/js/tab.js" type="text/javascript"></script>
+<script>
+
+</script>
+</body>
+</html>
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/compactionserver/CompactionTasksTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/compactionserver/CompactionTasksTmpl.jamon
new file mode 100644
index 0000000..478f9e2
--- /dev/null
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/compactionserver/CompactionTasksTmpl.jamon
@@ -0,0 +1,71 @@
+<%doc>
+Copyright The Apache Software Foundation
+
+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.
+ </%doc>
+
+<%args>
+HCompactionServer compactionServer;
+</%args>
+
+<%import>
+        java.util.*;
+        java.util.concurrent.ConcurrentHashMap;
+        org.apache.hadoop.hbase.compactionserver.*;
+        org.apache.hadoop.hbase.master.HMaster;
+        org.apache.hadoop.hbase.procedure2.util.StringUtils;
+        org.apache.hadoop.hbase.CompactionServerMetrics;
+        org.apache.hadoop.hbase.ServerName;
+        org.apache.hadoop.hbase.Size;
+        org.apache.hadoop.hbase.util.VersionInfo;
+        org.apache.hadoop.hbase.util.Pair;
+        org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
+</%import>
+
+
+<%java>
+ConcurrentHashMap<String, CompactionTask> tasks = compactionServer.getCompactionThreadManager().getRunningCompactionTasks();
+</%java>
+<table class="table table-striped">
+    <tr>
+        <th>CompactionTaskName</th>
+        <th>Num. Compacting Cells</th>
+        <th>Num. Compacted Cells</th>
+        <th>Compaction Progress</th>
+    </tr>
+    <%if (tasks != null && tasks.size() > 0)%>
+        <%for String taskName : tasks.keySet() %>
+        <%java>
+            CompactionTask compactionTask = tasks.get(taskName);
+            String percentDone = "";
+            long compactingCells = compactionTask.getStore().getCompactionProgress().getTotalCompactingKVs();
+            long compactedCells = compactionTask.getStore().getCompactionProgress().getCurrentCompactedKvs();
+            if (compactingCells > 0) {
+               percentDone = String.format("%.2f", 100 *
+                 ((float) compactedCells / compactingCells)) + "%";
+            }
+        </%java>
+            <tr>
+                <td><% taskName %></td>
+                <td><% compactingCells %></td>
+                <td><% compactedCells %></td>
+                <td><% percentDone %></td>
+            </tr>
+        </%for>
+    </%if>
+    <tr><td>Total: <% (tasks != null) ? tasks.size() : 0 %></td></tr>
+</table>
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/AbstractServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/AbstractServer.java
index e59d8b7..38e216f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/AbstractServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/AbstractServer.java
@@ -25,8 +25,10 @@ import java.io.IOException;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
+import java.net.BindException;
 import java.net.InetSocketAddress;
 import java.util.concurrent.atomic.AtomicBoolean;
+import javax.servlet.http.HttpServlet;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -35,6 +37,7 @@ import org.apache.hadoop.hbase.client.ClusterConnectionFactory;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.fs.HFileSystem;
+import org.apache.hadoop.hbase.http.InfoServer;
 import org.apache.hadoop.hbase.ipc.RpcClient;
 import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
 import org.apache.hadoop.hbase.master.HMaster;
@@ -114,6 +117,12 @@ public abstract class AbstractServer extends Thread implements Server {
   protected MasterAddressTracker masterAddressTracker;
   // Cluster Status Tracker
   protected ClusterStatusTracker clusterStatusTracker;
+
+  // Info server. Default access so can be used by unit tests. REGIONSERVER/COMPACTIONSERVER
+  // is name of the webapp and the attribute name used stuffing this instance
+  // into web context.
+  protected InfoServer infoServer;
+
   /**
    * Setup our cluster connection if not already initialized.
    */
@@ -149,6 +158,13 @@ public abstract class AbstractServer extends Thread implements Server {
   }
 
   /**
+   * @return time stamp in millis of when this region server was started
+   */
+  public long getStartcode() {
+    return this.startcode;
+  }
+
+  /**
    * Bring up connection to zk ensemble and then wait until a master for this cluster and then after
    * that, wait until cluster 'up' flag has been set. This is the order in which master does things.
    * <p>
@@ -436,6 +452,10 @@ public abstract class AbstractServer extends Thread implements Server {
 
   protected abstract String getProcessName();
 
+  public InfoServer getInfoServer() {
+    return infoServer;
+  }
+
   public ServerType getServerType() {
     String processName = getProcessName();
     if (processName.equals(MASTER)) {
@@ -449,4 +469,29 @@ public abstract class AbstractServer extends Thread implements Server {
     }
     return ServerType.ReplicationServer;
   }
+
+  protected abstract void configureInfoServer();
+
+  protected abstract Class<? extends HttpServlet> getDumpServlet();
+
+  protected void tryCreateInfoServer(int port, String addr, boolean auto) throws IOException {
+    while (true) {
+      try {
+        this.infoServer = new InfoServer(getProcessName(), addr, port, false, this.conf);
+        infoServer.addPrivilegedServlet("dump", "/dump", getDumpServlet());
+        configureInfoServer();
+        this.infoServer.start();
+        break;
+      } catch (BindException e) {
+        if (!auto) {
+          // auto bind disabled throw BindException
+          LOG.error("Failed binding http info server to port: " + port);
+          throw e;
+        }
+        // auto bind enabled, try to use another port
+        LOG.info("Failed binding http info server to port: " + port);
+        port++;
+      }
+    }
+  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/compactionserver/CSDumpServlet.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/compactionserver/CSDumpServlet.java
new file mode 100644
index 0000000..7c9dace
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/compactionserver/CSDumpServlet.java
@@ -0,0 +1,93 @@
+/**
+ *
+ * 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.hbase.compactionserver;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.PrintStream;
+import java.io.PrintWriter;
+import java.util.Date;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.monitoring.StateDumpServlet;
+import org.apache.hadoop.hbase.monitoring.TaskMonitor;
+import org.apache.hadoop.hbase.util.LogMonitoring;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Private
+public class CSDumpServlet extends StateDumpServlet {
+  private static final long serialVersionUID = 1L;
+  private static final String LINE =
+    "===========================================================";
+
+  @Override
+  public void doGet(HttpServletRequest request, HttpServletResponse response)
+    throws IOException {
+    HCompactionServer cs = (HCompactionServer)getServletContext().getAttribute(
+      HCompactionServer.COMPACTIONSERVER);
+    assert cs != null : "No CM in context!";
+
+    response.setContentType("text/plain");
+
+    if (!cs.isOnline()) {
+      response.getWriter().write("The CompactionServer is initializing!");
+      response.getWriter().close();
+      return;
+    }
+
+    OutputStream os = response.getOutputStream();
+    try (PrintWriter out = new PrintWriter(os)) {
+
+      out.println("CompactionServer status for " + cs.getServerName()
+        + " as of " + new Date());
+
+      out.println("\n\nVersion Info:");
+      out.println(LINE);
+      dumpVersionInfo(out);
+
+      out.println("\n\nTasks:");
+      out.println(LINE);
+      TaskMonitor.get().dumpAsText(out);
+
+      out.println("\n\nStacks:");
+      out.println(LINE);
+      PrintStream ps = new PrintStream(response.getOutputStream(), false, "UTF-8");
+      Threads.printThreadInfo(ps, "");
+      ps.flush();
+
+      out.println("\n\nCM Configuration:");
+      out.println(LINE);
+      Configuration conf = cs.getConfiguration();
+      out.flush();
+      conf.writeXml(os);
+      os.flush();
+
+      out.println("\n\nLogs");
+      out.println(LINE);
+      long tailKb = getTailKbParam(request);
+      LogMonitoring.dumpTailOfLogs(out, tailKb);
+
+      out.flush();
+    }
+  }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/compactionserver/CSStatusServlet.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/compactionserver/CSStatusServlet.java
new file mode 100644
index 0000000..cae86de
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/compactionserver/CSStatusServlet.java
@@ -0,0 +1,55 @@
+/**
+ *
+ * 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.hbase.compactionserver;
+
+import java.io.IOException;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.hadoop.hbase.tmpl.compactionserver.CSStatusTmpl;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+public class CSStatusServlet extends HttpServlet {
+  private static Logger LOG = LoggerFactory.getLogger(CSStatusServlet.class);
+  private static final long serialVersionUID = 1L;
+
+  @Override
+  protected void doGet(HttpServletRequest req, HttpServletResponse resp)
+    throws ServletException, IOException {
+    HCompactionServer cs =
+      (HCompactionServer) getServletContext().getAttribute(HCompactionServer.COMPACTIONSERVER);
+    assert cs != null : "No CM in context!";
+    resp.setContentType("text/html");
+
+    if (!cs.isOnline()) {
+      resp.getWriter().write("The CompactionServer is initializing!");
+      resp.getWriter().close();
+      return;
+    }
+
+    CSStatusTmpl tmpl = new CSStatusTmpl();
+    tmpl.render(resp.getWriter(), cs);
+  }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/compactionserver/CompactionTask.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/compactionserver/CompactionTask.java
index 05e350f..19babb0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/compactionserver/CompactionTask.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/compactionserver/CompactionTask.java
@@ -127,7 +127,7 @@ public final class CompactionTask {
     return compactionContext;
   }
 
-  HStore getStore() {
+  public HStore getStore() {
     return store;
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/compactionserver/CompactionThreadManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/compactionserver/CompactionThreadManager.java
index 9d67e29..92b23bd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/compactionserver/CompactionThreadManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/compactionserver/CompactionThreadManager.java
@@ -376,7 +376,7 @@ public class CompactionThreadManager implements ThroughputControllerService {
     return server.getAsyncClusterConnection().getRegionServerAdmin(sn);
   }
 
-  ConcurrentHashMap<String, CompactionTask> getRunningCompactionTasks() {
+  public ConcurrentHashMap<String, CompactionTask> getRunningCompactionTasks() {
     return runningCompactionTasks;
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/compactionserver/HCompactionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/compactionserver/HCompactionServer.java
index 578925f..3a76bc5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/compactionserver/HCompactionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/compactionserver/HCompactionServer.java
@@ -18,9 +18,11 @@
 package org.apache.hadoop.hbase.compactionserver;
 
 import java.io.IOException;
+import java.net.InetAddress;
 import java.util.Collection;
 import java.util.List;
 import java.util.concurrent.atomic.LongAdder;
+import javax.servlet.http.HttpServlet;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.AbstractServer;
@@ -39,6 +41,7 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
 import org.apache.hadoop.hbase.security.SecurityConstants;
 import org.apache.hadoop.hbase.security.Superusers;
 import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.util.Addressing;
 import org.apache.hadoop.hbase.util.Sleeper;
 import org.apache.hadoop.hbase.util.VersionInfo;
 import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
@@ -84,6 +87,10 @@ public class HCompactionServer extends AbstractServer implements RegionCoprocess
     return choreService;
   }
 
+  public CompactionThreadManager getCompactionThreadManager() {
+    return compactionThreadManager;
+  }
+
   protected final CSRpcServices rpcServices;
 
   // Stub to do compaction server status calls against the master.
@@ -136,6 +143,42 @@ public class HCompactionServer extends AbstractServer implements RegionCoprocess
     this.choreService = new ChoreService(getName(), true);
     this.compactionThreadManager = new CompactionThreadManager(conf, this);
     this.rpcServices.start();
+    putUpWebUI();
+  }
+
+  // Web UI
+  private void putUpWebUI() throws IOException {
+    int port = this.conf.getInt(HConstants.COMPACTION_SERVER_INFO_PORT,
+      HConstants.DEFAULT_COMPACTION_SERVER_INFOPORT);
+    String addr = this.conf.get("hbase.compaction.server.info.bindAddress", "0.0.0.0");
+    // -1 is for disabling info server
+    if (port < 0) {
+      return;
+    }
+    if (!Addressing.isLocalAddress(InetAddress.getByName(addr))) {
+      String msg = "Failed to start http info server. Address " + addr
+          + " does not belong to this host. Correct configuration parameter: "
+          + "hbase.compaction.server.info.bindAddress";
+      LOG.error(msg);
+      throw new IOException(msg);
+    }
+
+    // check if auto port bind enabled
+    boolean auto = this.conf.getBoolean(HConstants.COMPACTION_SERVER_INFO_PORT_AUTO, false);
+    tryCreateInfoServer(port, addr, auto);
+    port = this.infoServer.getPort();
+    conf.setInt(HConstants.COMPACTION_SERVER_INFO_PORT, port);
+  }
+
+  @Override
+  protected Class<? extends HttpServlet> getDumpServlet() {
+    return CSDumpServlet.class;
+  }
+
+  @Override
+  protected void configureInfoServer() {
+    infoServer.addServlet("cs-status", "/cs-status", CSStatusServlet.class);
+    infoServer.setAttribute(COMPACTIONSERVER, this);
   }
 
   @Override
@@ -168,6 +211,11 @@ public class HCompactionServer extends AbstractServer implements RegionCoprocess
     serverLoad.setTotalNumberOfRequests(requestCount.sum());
     serverLoad.setReportStartTime(reportStartTime);
     serverLoad.setReportEndTime(reportEndTime);
+    if (this.infoServer != null) {
+      serverLoad.setInfoServerPort(this.infoServer.getPort());
+    } else {
+      serverLoad.setInfoServerPort(-1);
+    }
     return serverLoad.build();
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index f5e82df..54e13bd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -27,7 +27,6 @@ import java.io.IOException;
 import java.lang.management.MemoryType;
 import java.lang.management.MemoryUsage;
 import java.lang.reflect.Constructor;
-import java.net.BindException;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.time.Duration;
@@ -361,10 +360,6 @@ public class HRegionServer extends AbstractServer implements
 
   private UncaughtExceptionHandler uncaughtExceptionHandler;
 
-  // Info server. Default access so can be used by unit tests. REGIONSERVER
-  // is name of the webapp and the attribute name used stuffing this instance
-  // into web context.
-  protected InfoServer infoServer;
   private JvmPauseMonitor pauseMonitor;
 
   /** region server process name */
@@ -748,11 +743,13 @@ public class HRegionServer extends AbstractServer implements
     return new RSRpcServices(this);
   }
 
+  @Override
   protected void configureInfoServer() {
     infoServer.addUnprivilegedServlet("rs-status", "/rs-status", RSStatusServlet.class);
     infoServer.setAttribute(REGIONSERVER, this);
   }
 
+  @Override
   protected Class<? extends HttpServlet> getDumpServlet() {
     return RSDumpServlet.class;
   }
@@ -2110,24 +2107,7 @@ public class HRegionServer extends AbstractServer implements
     }
     // check if auto port bind enabled
     boolean auto = this.conf.getBoolean(HConstants.REGIONSERVER_INFO_PORT_AUTO, false);
-    while (true) {
-      try {
-        this.infoServer = new InfoServer(getProcessName(), addr, port, false, this.conf);
-        infoServer.addPrivilegedServlet("dump", "/dump", getDumpServlet());
-        configureInfoServer();
-        this.infoServer.start();
-        break;
-      } catch (BindException e) {
-        if (!auto) {
-          // auto bind disabled throw BindException
-          LOG.error("Failed binding http info server to port: " + port);
-          throw e;
-        }
-        // auto bind enabled, try to use another port
-        LOG.info("Failed binding http info server to port: " + port);
-        port++;
-      }
-    }
+    tryCreateInfoServer(port, addr, auto);
     port = this.infoServer.getPort();
     conf.setInt(HConstants.REGIONSERVER_INFO_PORT, port);
     int masterInfoPort = conf.getInt(HConstants.MASTER_INFO_PORT,
@@ -2728,11 +2708,6 @@ public class HRegionServer extends AbstractServer implements
     }
   }
 
-  /** @return the info server */
-  public InfoServer getInfoServer() {
-    return infoServer;
-  }
-
   @Override
   public boolean isStopping() {
     return this.stopping;
@@ -2802,13 +2777,6 @@ public class HRegionServer extends AbstractServer implements
     return sortedRegions;
   }
 
-  /**
-   * @return time stamp in millis of when this region server was started
-   */
-  public long getStartcode() {
-    return this.startcode;
-  }
-
   /** @return reference to FlushRequester */
   @Override
   public FlushRequester getFlushRequester() {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 53658ee..768d430 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -1812,7 +1812,8 @@ public class RSRpcServices extends AbstractRpcServices implements
       throw new ServiceException(ie);
     }
     requestCount.increment();
-    int infoPort = regionServer.infoServer != null ? regionServer.infoServer.getPort() : -1;
+    int infoPort =
+        regionServer.getInfoServer() != null ? regionServer.getInfoServer().getPort() : -1;
     return ResponseConverter.buildGetServerInfoResponse(regionServer.getServerName(), infoPort);
   }
 
diff --git a/hbase-server/src/main/resources/hbase-webapps/compactionserver/compactionserver.jsp b/hbase-server/src/main/resources/hbase-webapps/compactionserver/compactionserver.jsp
new file mode 100644
index 0000000..ee63075
--- /dev/null
+++ b/hbase-server/src/main/resources/hbase-webapps/compactionserver/compactionserver.jsp
@@ -0,0 +1,20 @@
+<%--
+/**
+ * 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.
+ */
+--%>
+<meta HTTP-EQUIV="REFRESH" content="0;url=/rs-status"/>
diff --git a/hbase-server/src/main/resources/hbase-webapps/compactionserver/footer.jsp b/hbase-server/src/main/resources/hbase-webapps/compactionserver/footer.jsp
new file mode 100644
index 0000000..25bfb32
--- /dev/null
+++ b/hbase-server/src/main/resources/hbase-webapps/compactionserver/footer.jsp
@@ -0,0 +1,30 @@
+<%--
+/**
+* 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.
+*/
+--%>
+    <script src="/static/js/jquery.min.js" type="text/javascript"></script>
+    <script src="/static/js/bootstrap.min.js" type="text/javascript"></script>
+    <script src="/static/js/tab.js" type="text/javascript"></script>
+    <script type="text/javascript">
+      $(document).ready(function() {
+        $('div.navbar li.active').removeClass('active');
+        $('a[href="' + location.pathname + '"]').closest('li').addClass('active');
+      });
+    </script>
+  </body>
+</html>
diff --git a/hbase-server/src/main/resources/hbase-webapps/compactionserver/header.jsp b/hbase-server/src/main/resources/hbase-webapps/compactionserver/header.jsp
new file mode 100644
index 0000000..5a4b21f
--- /dev/null
+++ b/hbase-server/src/main/resources/hbase-webapps/compactionserver/header.jsp
@@ -0,0 +1,65 @@
+<%--
+/**
+* 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.
+*/
+--%>
+<%@ page contentType="text/html;charset=UTF-8"
+  import="org.apache.hadoop.hbase.HBaseConfiguration"
+%>
+<!DOCTYPE html>
+<?xml version="1.0" encoding="UTF-8" ?>
+<html xmlns="http://www.w3.org/1999/xhtml">
+  <head>
+    <meta charset="utf-8">
+    <title><%= request.getParameter("pageTitle")%></title>
+    <meta name="viewport" content="width=device-width, initial-scale=1.0">
+    <meta name="description" content="">
+    <meta name="author" content="">
+
+    <link href="/static/css/bootstrap.min.css" rel="stylesheet">
+    <link href="/static/css/bootstrap-theme.min.css" rel="stylesheet">
+    <link href="/static/css/hbase.css" rel="stylesheet">
+  </head>
+  <body>
+    <div class="navbar  navbar-fixed-top navbar-default">
+      <div class="container-fluid">
+        <div class="navbar-header">
+          <button type="button" class="navbar-toggle" data-toggle="collapse"
+                  data-target=".navbar-collapse">
+            <span class="icon-bar"></span>
+            <span class="icon-bar"></span>
+            <span class="icon-bar"></span>
+          </button>
+          <a class="navbar-brand" href="/cs-status">
+            <img src="/static/hbase_logo_small.png" alt="HBase Logo"/>
+          </a>
+        </div>
+        <div class="collapse navbar-collapse">
+          <ul class="nav navbar-nav">
+            <li><a href="/cs-status">Home</a></li>
+            <li><a href="/logs/">Local Logs</a></li>
+            <li><a href="/logLevel">Log Level</a></li>
+            <li><a href="/dump">Debug Dump</a></li>
+            <li><a href="/jmx">Metrics Dump</a></li>
+            <li><a href="/prof">Profiler</a></li>
+            <% if (HBaseConfiguration.isShowConfInServlet()) { %>
+            <li><a href="/conf">HBase Configuration</a></li>
+            <% } %>
+          </ul>
+        </div><!--/.nav-collapse -->
+      </div>
+    </div>
diff --git a/hbase-server/src/main/resources/hbase-webapps/compactionserver/index.html b/hbase-server/src/main/resources/hbase-webapps/compactionserver/index.html
new file mode 100644
index 0000000..8f31e9d
--- /dev/null
+++ b/hbase-server/src/main/resources/hbase-webapps/compactionserver/index.html
@@ -0,0 +1,20 @@
+<!--
+/**
+ * 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.
+ */
+-->
+<meta HTTP-EQUIV="REFRESH" content="0;url=/cs-status"/>