You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2019/03/18 02:08:15 UTC

[hbase] branch master updated: HBASE-21926 Profiler servlet

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

apurtell pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/master by this push:
     new 6cd78e8  HBASE-21926 Profiler servlet
6cd78e8 is described below

commit 6cd78e899fd37393eaa9f142facbee2d9a052dba
Author: Andrew Purtell <ap...@apache.org>
AuthorDate: Sun Mar 17 18:47:52 2019 -0700

    HBASE-21926 Profiler servlet
---
 bin/hbase                                          |   1 +
 .../org/apache/hadoop/hbase/http/HttpServer.java   |  24 +-
 .../hadoop/hbase/http/ProfileOutputServlet.java    |  58 ++++
 .../apache/hadoop/hbase/http/ProfileServlet.java   | 373 +++++++++++++++++++++
 .../org/apache/hadoop/hbase/util/ProcessUtils.java |  70 ++++
 .../src/main/resources/hbase-webapps/rest/rest.jsp |   1 +
 .../hbase/tmpl/master/MasterStatusTmpl.jamon       |   1 +
 .../hbase/tmpl/regionserver/RSStatusTmpl.jamon     |   1 +
 .../main/resources/hbase-webapps/master/header.jsp |   1 +
 .../hbase-webapps/regionserver/header.jsp          |   1 +
 .../main/resources/hbase-webapps/thrift/thrift.jsp |   1 +
 src/main/asciidoc/_chapters/profiler.adoc          |  98 ++++++
 src/main/asciidoc/book.adoc                        |   1 +
 13 files changed, 627 insertions(+), 4 deletions(-)

diff --git a/bin/hbase b/bin/hbase
index 59a3b69..1af2213 100755
--- a/bin/hbase
+++ b/bin/hbase
@@ -689,5 +689,6 @@ fi
 if [ "${HBASE_NOEXEC}" != "" ]; then
   "$JAVA" -Dproc_$COMMAND -XX:OnOutOfMemoryError="kill -9 %p" $HEAP_SETTINGS $HBASE_OPTS $CLASS "$@"
 else
+  export JVM_PID="$$"
   exec "$JAVA" -Dproc_$COMMAND -XX:OnOutOfMemoryError="kill -9 %p" $HEAP_SETTINGS $HBASE_OPTS $CLASS "$@"
 fi
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java
index af72ab8..e9bcfbb 100644
--- a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java
+++ b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java
@@ -26,6 +26,9 @@ import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Enumeration;
@@ -570,7 +573,7 @@ public class HttpServer implements FilterContainer {
       }
     }
 
-    addDefaultServlets();
+    addDefaultServlets(contexts);
 
     if (pathSpecs != null) {
       for (String path : pathSpecs) {
@@ -668,11 +671,10 @@ public class HttpServer implements FilterContainer {
   /**
    * Add default servlets.
    */
-  protected void addDefaultServlets() {
+  protected void addDefaultServlets(ContextHandlerCollection contexts) throws IOException {
     // set up default servlets
     addServlet("stacks", "/stacks", StackServlet.class);
     addServlet("logLevel", "/logLevel", LogLevel.Servlet.class);
-
     // Hadoop3 has moved completely to metrics2, and  dropped support for Metrics v1's
     // MetricsServlet (see HADOOP-12504).  We'll using reflection to load if against hadoop2.
     // Remove when we drop support for hbase on hadoop2.x.
@@ -682,9 +684,23 @@ public class HttpServer implements FilterContainer {
     } catch (Exception e) {
       // do nothing
     }
-
     addServlet("jmx", "/jmx", JMXJsonServlet.class);
     addServlet("conf", "/conf", ConfServlet.class);
+    final String asyncProfilerHome = ProfileServlet.getAsyncProfilerHome();
+    if (asyncProfilerHome != null && !asyncProfilerHome.trim().isEmpty()) {
+      addServlet("prof", "/prof", ProfileServlet.class);
+      Path tmpDir = Paths.get(ProfileServlet.OUTPUT_DIR);
+      if (Files.notExists(tmpDir)) {
+        Files.createDirectories(tmpDir);
+      }
+      ServletContextHandler genCtx = new ServletContextHandler(contexts, "/prof-output");
+      genCtx.addServlet(ProfileOutputServlet.class, "/*");
+      genCtx.setResourceBase(tmpDir.toAbsolutePath().toString());
+      genCtx.setDisplayName("prof-output");
+    } else {
+      LOG.info("ASYNC_PROFILER_HOME environment variable and async.profiler.home system property " +
+        "not specified. Disabling /prof endpoint.");
+    }
   }
 
   /**
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/ProfileOutputServlet.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/ProfileOutputServlet.java
new file mode 100644
index 0000000..670c3ac
--- /dev/null
+++ b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/ProfileOutputServlet.java
@@ -0,0 +1,58 @@
+/*
+ * 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.http;
+
+import java.io.File;
+import java.io.IOException;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.eclipse.jetty.servlet.DefaultServlet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Servlet to serve files generated by {@link ProfileServlet}
+ */
+@InterfaceAudience.Private
+public class ProfileOutputServlet extends DefaultServlet {
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LoggerFactory.getLogger(ProfileOutputServlet.class);
+  private static final int REFRESH_PERIOD = 2;
+
+  @Override
+  protected void doGet(final HttpServletRequest req, final HttpServletResponse resp)
+    throws ServletException, IOException {
+    String absoluteDiskPath = getServletContext().getRealPath(req.getPathInfo());
+    File requestedFile = new File(absoluteDiskPath);
+    // async-profiler version 1.4 writes 'Started [cpu] profiling' to output file when profiler is
+    // running which gets replaced by final output. If final output is not ready yet, the file size
+    // will be <100 bytes (in all modes).
+    if (requestedFile.length() < 100) {
+      LOG.info(requestedFile  + " is incomplete. Sending auto-refresh header.");
+      resp.setHeader("Refresh", REFRESH_PERIOD + "," + req.getRequestURI());
+      resp.getWriter().write("This page will be auto-refreshed every " + REFRESH_PERIOD +
+        " seconds until the output file is ready.");
+    } else {
+      super.doGet(req, resp);
+    }
+  }
+}
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/ProfileServlet.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/ProfileServlet.java
new file mode 100644
index 0000000..25255a0
--- /dev/null
+++ b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/ProfileServlet.java
@@ -0,0 +1,373 @@
+/*
+ * 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.http;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.hadoop.hbase.util.ProcessUtils;
+import org.apache.hbase.thirdparty.com.google.common.base.Joiner;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Servlet that runs async-profiler as web-endpoint.
+ * Following options from async-profiler can be specified as query paramater.
+ * //  -e event          profiling event: cpu|alloc|lock|cache-misses etc.
+ * //  -d duration       run profiling for 'duration' seconds (integer)
+ * //  -i interval       sampling interval in nanoseconds (long)
+ * //  -j jstackdepth    maximum Java stack depth (integer)
+ * //  -b bufsize        frame buffer size (long)
+ * //  -t                profile different threads separately
+ * //  -s                simple class names instead of FQN
+ * //  -o fmt[,fmt...]   output format: summary|traces|flat|collapsed|svg|tree|jfr
+ * //  --width px        SVG width pixels (integer)
+ * //  --height px       SVG frame height pixels (integer)
+ * //  --minwidth px     skip frames smaller than px (double)
+ * //  --reverse         generate stack-reversed FlameGraph / Call tree
+ * Example:
+ * - To collect 30 second CPU profile of current process (returns FlameGraph svg)
+ * curl "http://localhost:10002/prof"
+ * - To collect 1 minute CPU profile of current process and output in tree format (html)
+ * curl "http://localhost:10002/prof?output=tree&amp;duration=60"
+ * - To collect 30 second heap allocation profile of current process (returns FlameGraph svg)
+ * curl "http://localhost:10002/prof?event=alloc"
+ * - To collect lock contention profile of current process (returns FlameGraph svg)
+ * curl "http://localhost:10002/prof?event=lock"
+ * Following event types are supported (default is 'cpu') (NOTE: not all OS'es support all events)
+ * // Perf events:
+ * //    cpu
+ * //    page-faults
+ * //    context-switches
+ * //    cycles
+ * //    instructions
+ * //    cache-references
+ * //    cache-misses
+ * //    branches
+ * //    branch-misses
+ * //    bus-cycles
+ * //    L1-dcache-load-misses
+ * //    LLC-load-misses
+ * //    dTLB-load-misses
+ * //    mem:breakpoint
+ * //    trace:tracepoint
+ * // Java events:
+ * //    alloc
+ * //    lock
+ */
+@InterfaceAudience.Private
+public class ProfileServlet extends HttpServlet {
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LoggerFactory.getLogger(ProfileServlet.class);
+
+  private static final String ACCESS_CONTROL_ALLOW_METHODS = "Access-Control-Allow-Methods";
+  private static final String ALLOWED_METHODS = "GET";
+  private static final String ACCESS_CONTROL_ALLOW_ORIGIN = "Access-Control-Allow-Origin";
+  private static final String CONTENT_TYPE_TEXT = "text/plain; charset=utf-8";
+  private static final String ASYNC_PROFILER_HOME_ENV = "ASYNC_PROFILER_HOME";
+  private static final String ASYNC_PROFILER_HOME_SYSTEM_PROPERTY = "async.profiler.home";
+  private static final String PROFILER_SCRIPT = "/profiler.sh";
+  private static final int DEFAULT_DURATION_SECONDS = 10;
+  private static final AtomicInteger ID_GEN = new AtomicInteger(0);
+  static final String OUTPUT_DIR = System.getProperty("java.io.tmpdir") + "/prof-output";
+
+  enum Event {
+    CPU("cpu"),
+    ALLOC("alloc"),
+    LOCK("lock"),
+    PAGE_FAULTS("page-faults"),
+    CONTEXT_SWITCHES("context-switches"),
+    CYCLES("cycles"),
+    INSTRUCTIONS("instructions"),
+    CACHE_REFERENCES("cache-references"),
+    CACHE_MISSES("cache-misses"),
+    BRANCHES("branches"),
+    BRANCH_MISSES("branch-misses"),
+    BUS_CYCLES("bus-cycles"),
+    L1_DCACHE_LOAD_MISSES("L1-dcache-load-misses"),
+    LLC_LOAD_MISSES("LLC-load-misses"),
+    DTLB_LOAD_MISSES("dTLB-load-misses"),
+    MEM_BREAKPOINT("mem:breakpoint"),
+    TRACE_TRACEPOINT("trace:tracepoint"),;
+
+    private final String internalName;
+
+    Event(final String internalName) {
+      this.internalName = internalName;
+    }
+
+    public String getInternalName() {
+      return internalName;
+    }
+
+    public static Event fromInternalName(final String name) {
+      for (Event event : values()) {
+        if (event.getInternalName().equalsIgnoreCase(name)) {
+          return event;
+        }
+      }
+
+      return null;
+    }
+  }
+
+  enum Output {
+    SUMMARY,
+    TRACES,
+    FLAT,
+    COLLAPSED,
+    SVG,
+    TREE,
+    JFR
+  }
+
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "SE_TRANSIENT_FIELD_NOT_RESTORED",
+    justification = "This class is never serialized nor restored.")
+  private transient Lock profilerLock = new ReentrantLock();
+  private transient volatile Process process;
+  private String asyncProfilerHome;
+  private Integer pid;
+
+  public ProfileServlet() {
+    this.asyncProfilerHome = getAsyncProfilerHome();
+    this.pid = ProcessUtils.getPid();
+    LOG.info("Servlet process PID: " + pid + " asyncProfilerHome: " + asyncProfilerHome);
+  }
+
+  @Override
+  protected void doGet(final HttpServletRequest req, final HttpServletResponse resp)
+      throws IOException {
+    if (!HttpServer.isInstrumentationAccessAllowed(getServletContext(), req, resp)) {
+      resp.setStatus(HttpServletResponse.SC_UNAUTHORIZED);
+      setResponseHeader(resp);
+      resp.getWriter().write("Unauthorized: Instrumentation access is not allowed!");
+      return;
+    }
+
+    // make sure async profiler home is set
+    if (asyncProfilerHome == null || asyncProfilerHome.trim().isEmpty()) {
+      resp.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+      setResponseHeader(resp);
+      resp.getWriter().write("ASYNC_PROFILER_HOME env is not set.");
+      return;
+    }
+
+    // if pid is explicitly specified, use it else default to current process
+    pid = getInteger(req, "pid", pid);
+
+    // if pid is not specified in query param and if current process pid cannot be determined
+    if (pid == null) {
+      resp.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+      setResponseHeader(resp);
+      resp.getWriter().write(
+        "'pid' query parameter unspecified or unable to determine PID of current process.");
+      return;
+    }
+
+    final int duration = getInteger(req, "duration", DEFAULT_DURATION_SECONDS);
+    final Output output = getOutput(req);
+    final Event event = getEvent(req);
+    final Long interval = getLong(req, "interval");
+    final Integer jstackDepth = getInteger(req, "jstackdepth", null);
+    final Long bufsize = getLong(req, "bufsize");
+    final boolean thread = req.getParameterMap().containsKey("thread");
+    final boolean simple = req.getParameterMap().containsKey("simple");
+    final Integer width = getInteger(req, "width", null);
+    final Integer height = getInteger(req, "height", null);
+    final Double minwidth = getMinWidth(req);
+    final boolean reverse = req.getParameterMap().containsKey("reverse");
+
+    if (process == null || !process.isAlive()) {
+      try {
+        int lockTimeoutSecs = 3;
+        if (profilerLock.tryLock(lockTimeoutSecs, TimeUnit.SECONDS)) {
+          try {
+            File outputFile = new File(OUTPUT_DIR, "async-prof-pid-" + pid + "-" +
+              event.name().toLowerCase() + "-" + ID_GEN.incrementAndGet() + "." +
+              output.name().toLowerCase());
+            List<String> cmd = new ArrayList<>();
+            cmd.add(asyncProfilerHome + PROFILER_SCRIPT);
+            cmd.add("-e");
+            cmd.add(event.getInternalName());
+            cmd.add("-d");
+            cmd.add("" + duration);
+            cmd.add("-o");
+            cmd.add(output.name().toLowerCase());
+            cmd.add("-f");
+            cmd.add(outputFile.getAbsolutePath());
+            if (interval != null) {
+              cmd.add("-i");
+              cmd.add(interval.toString());
+            }
+            if (jstackDepth != null) {
+              cmd.add("-j");
+              cmd.add(jstackDepth.toString());
+            }
+            if (bufsize != null) {
+              cmd.add("-b");
+              cmd.add(bufsize.toString());
+            }
+            if (thread) {
+              cmd.add("-t");
+            }
+            if (simple) {
+              cmd.add("-s");
+            }
+            if (width != null) {
+              cmd.add("--width");
+              cmd.add(width.toString());
+            }
+            if (height != null) {
+              cmd.add("--height");
+              cmd.add(height.toString());
+            }
+            if (minwidth != null) {
+              cmd.add("--minwidth");
+              cmd.add(minwidth.toString());
+            }
+            if (reverse) {
+              cmd.add("--reverse");
+            }
+            cmd.add(pid.toString());
+            process = ProcessUtils.runCmdAsync(cmd);
+
+            // set response and set refresh header to output location
+            setResponseHeader(resp);
+            resp.setStatus(HttpServletResponse.SC_ACCEPTED);
+            String relativeUrl = "/prof-output/" + outputFile.getName();
+            resp.getWriter().write(
+              "Started [" + event.getInternalName() +
+              "] profiling. This page will automatically redirect to " +
+              relativeUrl + " after " + duration + " seconds.\n\ncommand:\n" +
+              Joiner.on(" ").join(cmd));
+
+            // to avoid auto-refresh by ProfileOutputServlet, refreshDelay can be specified
+            // via url param
+            int refreshDelay = getInteger(req, "refreshDelay", 0);
+
+            // instead of sending redirect, set auto-refresh so that browsers will refresh
+            // with redirected url
+            resp.setHeader("Refresh", (duration + refreshDelay) + ";" + relativeUrl);
+            resp.getWriter().flush();
+          } finally {
+            profilerLock.unlock();
+          }
+        } else {
+          setResponseHeader(resp);
+          resp.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+          resp.getWriter().write(
+            "Unable to acquire lock. Another instance of profiler might be running.");
+          LOG.warn("Unable to acquire lock in " + lockTimeoutSecs +
+            " seconds. Another instance of profiler might be running.");
+        }
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while acquiring profile lock.", e);
+        resp.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+      }
+    } else {
+      setResponseHeader(resp);
+      resp.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+      resp.getWriter().write("Another instance of profiler is already running.");
+    }
+  }
+
+  private Integer getInteger(final HttpServletRequest req, final String param,
+      final Integer defaultValue) {
+    final String value = req.getParameter(param);
+    if (value != null) {
+      try {
+        return Integer.valueOf(value);
+      } catch (NumberFormatException e) {
+        return defaultValue;
+      }
+    }
+    return defaultValue;
+  }
+
+  private Long getLong(final HttpServletRequest req, final String param) {
+    final String value = req.getParameter(param);
+    if (value != null) {
+      try {
+        return Long.valueOf(value);
+      } catch (NumberFormatException e) {
+        return null;
+      }
+    }
+    return null;
+  }
+
+  private Double getMinWidth(final HttpServletRequest req) {
+    final String value = req.getParameter("minwidth");
+    if (value != null) {
+      try {
+        return Double.valueOf(value);
+      } catch (NumberFormatException e) {
+        return null;
+      }
+    }
+    return null;
+  }
+
+  private Event getEvent(final HttpServletRequest req) {
+    final String eventArg = req.getParameter("event");
+    if (eventArg != null) {
+      Event event = Event.fromInternalName(eventArg);
+      return event == null ? Event.CPU : event;
+    }
+    return Event.CPU;
+  }
+
+  private Output getOutput(final HttpServletRequest req) {
+    final String outputArg = req.getParameter("output");
+    if (req.getParameter("output") != null) {
+      try {
+        return Output.valueOf(outputArg.trim().toUpperCase());
+      } catch (IllegalArgumentException e) {
+        return Output.SVG;
+      }
+    }
+    return Output.SVG;
+  }
+
+  private void setResponseHeader(final HttpServletResponse response) {
+    response.setHeader(ACCESS_CONTROL_ALLOW_METHODS, ALLOWED_METHODS);
+    response.setHeader(ACCESS_CONTROL_ALLOW_ORIGIN, "*");
+    response.setContentType(CONTENT_TYPE_TEXT);
+  }
+
+  static String getAsyncProfilerHome() {
+    String asyncProfilerHome = System.getenv(ASYNC_PROFILER_HOME_ENV);
+    // if ENV is not set, see if -Dasync.profiler.home=/path/to/async/profiler/home is set
+    if (asyncProfilerHome == null || asyncProfilerHome.trim().isEmpty()) {
+      asyncProfilerHome = System.getProperty(ASYNC_PROFILER_HOME_SYSTEM_PROPERTY);
+    }
+
+    return asyncProfilerHome;
+  }
+}
\ No newline at end of file
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/util/ProcessUtils.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/util/ProcessUtils.java
new file mode 100644
index 0000000..2d3835e
--- /dev/null
+++ b/hbase-http/src/main/java/org/apache/hadoop/hbase/util/ProcessUtils.java
@@ -0,0 +1,70 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.util.List;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Process related utilities.
+ */
+@InterfaceAudience.Private
+public final class ProcessUtils {
+  private static Logger LOG = LoggerFactory.getLogger(ProcessUtils.class);
+
+  private ProcessUtils() { }
+
+  public static Integer getPid() {
+    // JVM_PID is exported by bin/hbase run script
+    String pidStr = System.getenv("JVM_PID");
+
+    // in case if it is not set correctly used fallback from mxbean which is implementation specific
+    if (pidStr == null || pidStr.trim().isEmpty()) {
+      String name = ManagementFactory.getRuntimeMXBean().getName();
+      if (name != null) {
+        int idx = name.indexOf("@");
+        if (idx != -1) {
+          pidStr = name.substring(0, name.indexOf("@"));
+        }
+      }
+    }
+    try {
+      if (pidStr != null) {
+        return Integer.valueOf(pidStr);
+      }
+    } catch (NumberFormatException nfe) {
+      // ignore
+    }
+    return null;
+  }
+
+  public static Process runCmdAsync(List<String> cmd) {
+    try {
+      LOG.info("Running command async: " + cmd);
+      return new ProcessBuilder(cmd).inheritIO().start();
+    } catch (IOException ex) {
+      throw new IllegalStateException(ex);
+    }
+  }
+}
diff --git a/hbase-rest/src/main/resources/hbase-webapps/rest/rest.jsp b/hbase-rest/src/main/resources/hbase-webapps/rest/rest.jsp
index ba3c027..ed4e9c2 100644
--- a/hbase-rest/src/main/resources/hbase-webapps/rest/rest.jsp
+++ b/hbase-rest/src/main/resources/hbase-webapps/rest/rest.jsp
@@ -58,6 +58,7 @@ String listenPort = conf.get("hbase.rest.port", "8080");
                   <li><a href="/logs/">Local logs</a></li>
                   <li><a href="/logLevel">Log Level</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>
                   <% } %>
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
index da44052..f6ea764 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
@@ -156,6 +156,7 @@ AssignmentManager assignmentManager = master.getAssignmentManager();
                 <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>
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon
index 8eb362a..94fcecf 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon
@@ -113,6 +113,7 @@ org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
                 <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>
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/header.jsp b/hbase-server/src/main/resources/hbase-webapps/master/header.jsp
index 0df7099..1e02e2b 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/header.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/header.jsp
@@ -64,6 +64,7 @@
             <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>
             <% } %>
diff --git a/hbase-server/src/main/resources/hbase-webapps/regionserver/header.jsp b/hbase-server/src/main/resources/hbase-webapps/regionserver/header.jsp
index d61cf10..88ca515 100644
--- a/hbase-server/src/main/resources/hbase-webapps/regionserver/header.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/regionserver/header.jsp
@@ -56,6 +56,7 @@
             <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>
             <% } %>
diff --git a/hbase-thrift/src/main/resources/hbase-webapps/thrift/thrift.jsp b/hbase-thrift/src/main/resources/hbase-webapps/thrift/thrift.jsp
index b0064b4..432b0f4 100644
--- a/hbase-thrift/src/main/resources/hbase-webapps/thrift/thrift.jsp
+++ b/hbase-thrift/src/main/resources/hbase-webapps/thrift/thrift.jsp
@@ -66,6 +66,7 @@ String compact = conf.get("hbase.regionserver.thrift.compact", "false");
                 <li><a href="/logs/">Local logs</a></li>
                 <li><a href="/logLevel">Log Level</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>
                 <% } %>
diff --git a/src/main/asciidoc/_chapters/profiler.adoc b/src/main/asciidoc/_chapters/profiler.adoc
new file mode 100644
index 0000000..09cbccc
--- /dev/null
+++ b/src/main/asciidoc/_chapters/profiler.adoc
@@ -0,0 +1,98 @@
+////
+/**
+ *
+ * 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.
+ */
+////
+
+[[profiler]]
+= Profiler Servlet
+:doctype: book
+:numbered:
+:toc: left
+:icons: font
+:experimental:
+
+== Background
+
+HBASE-21926 introduced a new servlet that supports integrated profiling via async-profiler.
+
+== Prerequisites
+
+Go to https://github.com/jvm-profiling-tools/async-profiler, download a release appropriate for your platform, and install on every cluster host.
+
+Set `ASYNC_PROFILER_HOME` in the environment (put it in hbase-env.sh) to the root directory of the async-profiler install location, or pass it on the HBase daemon's command line as a system property as `-Dasync.profiler.home=/path/to/async-profiler`.
+
+== Usage
+
+Once the prerequisites are satisfied, access to async-profiler is available by way of the HBase UI or direct interaction with the infoserver.
+
+Examples:
+
+* To collect 30 second CPU profile of current process (returns FlameGraph svg)
+  `curl http://localhost:16030/prof`
+* To collect 1 minute CPU profile of current process and output in tree format (html)
+  `curl http://localhost:16030/prof?output=tree&duration=60`
+* To collect 30 second heap allocation profile of current process (returns FlameGraph svg)
+  `curl http://localhost:16030/prof?event=alloc`
+* To collect lock contention profile of current process (returns FlameGraph svg)
+  `curl http://localhost:16030/prof?event=lock`
+
+The following event types are supported by async-profiler. Use the 'event' parameter to specify. Default is 'cpu'. Not all operating systems will support all types.
+
+Perf events:
+
+* cpu
+* page-faults
+* context-switches
+* cycles
+* instructions
+* cache-references
+* cache-misses
+* branches
+* branch-misses
+* bus-cycles
+* L1-dcache-load-misses
+* LLC-load-misses
+* dTLB-load-misses
+
+Java events:
+
+* alloc
+* lock
+
+The following output formats are supported. Use the 'output' parameter to specify. Default is 'flamegraph'.
+
+Output formats:
+
+* summary: A dump of basic profiling statistics.
+* traces: Call traces.
+* flat: Flat profile (top N hot methods).
+* collapsed: Collapsed call traces in the format used by FlameGraph script. This is a collection of call stacks, where each line is a semicolon separated list of frames followed by a counter.
+* svg: FlameGraph in SVG format.
+* tree: Call tree in HTML format.
+* jfr: Call traces in Java Flight Recorder format.
+
+The 'duration' parameter specifies how long to collect trace data before generating output, specified in seconds. The default is 10 seconds.
+
+== UI
+
+In the UI, there is a new entry 'Profiler' in the top menu that will run the default action, which is to profile the CPU usage of the local process for thirty seconds and then produce FlameGraph SVG output.
+
+== Notes
+
+The query parameter `pid` can be used to specify the process id of a specific process to be profiled. If this parameter is missing the local process in which the infoserver is embedded will be profiled. Profile targets that are not JVMs might work but is not specifically supported. There are security implications. Access to the infoserver should be appropriately restricted.
diff --git a/src/main/asciidoc/book.adoc b/src/main/asciidoc/book.adoc
index edfbcce..51a2800 100644
--- a/src/main/asciidoc/book.adoc
+++ b/src/main/asciidoc/book.adoc
@@ -73,6 +73,7 @@ include::_chapters/thrift_filter_language.adoc[]
 include::_chapters/spark.adoc[]
 include::_chapters/cp.adoc[]
 include::_chapters/performance.adoc[]
+include::_chapters/profiler.adoc[]
 include::_chapters/troubleshooting.adoc[]
 include::_chapters/case_studies.adoc[]
 include::_chapters/ops_mgt.adoc[]