You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by to...@apache.org on 2011/08/30 23:08:04 UTC

svn commit: r1163385 - in /hbase/trunk: ./ src/main/jamon/org/apache/hbase/tmpl/master/ src/main/jamon/org/apache/hbase/tmpl/regionserver/ src/main/java/org/apache/hadoop/hbase/master/ src/main/java/org/apache/hadoop/hbase/monitoring/ src/main/java/org...

Author: todd
Date: Tue Aug 30 21:08:04 2011
New Revision: 1163385

URL: http://svn.apache.org/viewvc?rev=1163385&view=rev
Log:
HBASE-4292  Add a debugging dump servlet to the master and regionserver

Added:
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/MasterDumpServlet.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/monitoring/LogMonitoring.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/monitoring/StateDumpServlet.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/RSDumpServlet.java
Modified:
    hbase/trunk/CHANGES.txt
    hbase/trunk/src/main/jamon/org/apache/hbase/tmpl/master/MasterStatusTmpl.jamon
    hbase/trunk/src/main/jamon/org/apache/hbase/tmpl/regionserver/RSStatusTmpl.jamon
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/monitoring/TaskMonitor.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/VersionInfo.java

Modified: hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/trunk/CHANGES.txt?rev=1163385&r1=1163384&r2=1163385&view=diff
==============================================================================
--- hbase/trunk/CHANGES.txt (original)
+++ hbase/trunk/CHANGES.txt Tue Aug 30 21:08:04 2011
@@ -485,6 +485,8 @@ Release 0.91.0 - Unreleased
    HBASE-4027  Enable direct byte buffers LruBlockCache (Li Pi)
    HBASE-4117  Slow Query Log and Client Operation Fingerprints
                (Riley Patterson)
+   HBASE-4292  Add a debugging dump servlet to the master and regionserver
+               (todd)
 
 Release 0.90.5 - Unreleased
 

Modified: hbase/trunk/src/main/jamon/org/apache/hbase/tmpl/master/MasterStatusTmpl.jamon
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/jamon/org/apache/hbase/tmpl/master/MasterStatusTmpl.jamon?rev=1163385&r1=1163384&r2=1163385&view=diff
==============================================================================
--- hbase/trunk/src/main/jamon/org/apache/hbase/tmpl/master/MasterStatusTmpl.jamon (original)
+++ hbase/trunk/src/main/jamon/org/apache/hbase/tmpl/master/MasterStatusTmpl.jamon Tue Aug 30 21:08:04 2011
@@ -51,7 +51,12 @@ org.apache.hadoop.hbase.HTableDescriptor
 <body>
 <a id="logo" href="http://wiki.apache.org/lucene-hadoop/Hbase"><img src="/static/hbase_logo_med.gif" alt="HBase Logo" title="HBase Logo" /></a>
 <h1 id="page_title">Master: <% master.getServerName().getHostname() %>:<% master.getServerName().getPort() %></h1>
-<p id="links_menu"><a href="/logs/">Local logs</a>, <a href="/stacks">Thread Dump</a>, <a href="/logLevel">Log Level</a></p>
+<p id="links_menu">
+  <a href="/logs/">Local logs</a>,
+  <a href="/stacks">Thread Dump</a>,
+  <a href="/logLevel">Log Level</a>,
+  <a href="/dump">Debug dump</a>
+</p>
 
 <!-- Various warnings that cluster admins should be aware of -->
 <%if JvmVersion.isBadJvmVersion() %>

Modified: hbase/trunk/src/main/jamon/org/apache/hbase/tmpl/regionserver/RSStatusTmpl.jamon
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/jamon/org/apache/hbase/tmpl/regionserver/RSStatusTmpl.jamon?rev=1163385&r1=1163384&r2=1163385&view=diff
==============================================================================
--- hbase/trunk/src/main/jamon/org/apache/hbase/tmpl/regionserver/RSStatusTmpl.jamon (original)
+++ hbase/trunk/src/main/jamon/org/apache/hbase/tmpl/regionserver/RSStatusTmpl.jamon Tue Aug 30 21:08:04 2011
@@ -56,7 +56,12 @@ org.apache.hadoop.hbase.HRegionInfo;
 <body>
 <a id="logo" href="http://wiki.apache.org/lucene-hadoop/Hbase"><img src="/static/hbase_logo_med.gif" alt="HBase Logo" title="HBase Logo" /></a>
 <h1 id="page_title">Region Server: <% serverInfo.getServerAddress().getHostname() %>:<% serverInfo.getServerAddress().getPort() %></h1>
-<p id="links_menu"><a href="/logs/">Local logs</a>, <a href="/stacks">Thread Dump</a>, <a href="/logLevel">Log Level</a></p>
+<p id="links_menu">
+  <a href="/logs/">Local logs</a>,
+  <a href="/stacks">Thread Dump</a>,
+  <a href="/logLevel">Log Level</a>,
+  <a href="/dump">Debug dump</a>
+</p>
 <hr id="head_rule" />
 
 <h2>Region Server Attributes</h2>
@@ -103,4 +108,4 @@ If region has both an empty start and an
 <p>Not serving regions</p>
 </%if>
 </body>
-</html>
\ No newline at end of file
+</html>

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java?rev=1163385&r1=1163384&r2=1163385&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java Tue Aug 30 21:08:04 2011
@@ -675,6 +675,7 @@ implements HMasterInterface, HMasterRegi
      String a = this.conf.get("hbase.master.info.bindAddress", "0.0.0.0");
      this.infoServer = new InfoServer(MASTER, a, port, false, this.conf);
      this.infoServer.addServlet("status", "/master-status", MasterStatusServlet.class);
+     this.infoServer.addServlet("dump", "/dump", MasterDumpServlet.class);
      this.infoServer.setAttribute(MASTER, this);
      this.infoServer.start();
     }

Added: hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/MasterDumpServlet.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/MasterDumpServlet.java?rev=1163385&view=auto
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/MasterDumpServlet.java (added)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/MasterDumpServlet.java Tue Aug 30 21:08:04 2011
@@ -0,0 +1,121 @@
+/**
+ * Copyright 2011 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.
+ */
+package org.apache.hadoop.hbase.master;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.PrintWriter;
+import java.util.Date;
+import java.util.Map;
+import java.util.NavigableMap;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HServerInfo;
+import org.apache.hadoop.hbase.HServerLoad;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.master.AssignmentManager.RegionState;
+import org.apache.hadoop.hbase.monitoring.LogMonitoring;
+import org.apache.hadoop.hbase.monitoring.StateDumpServlet;
+import org.apache.hadoop.hbase.monitoring.TaskMonitor;
+import org.apache.hadoop.util.ReflectionUtils;
+
+public class MasterDumpServlet extends StateDumpServlet {
+  private static final long serialVersionUID = 1L;
+  private static final String LINE =
+    "===========================================================";
+  
+  @Override
+  public void doGet(HttpServletRequest request, HttpServletResponse response)
+      throws IOException {
+    HMaster master = (HMaster) getServletContext().getAttribute(HMaster.MASTER);
+    assert master != null : "No Master in context!";
+
+    response.setContentType("text/plain");
+    OutputStream os = response.getOutputStream();
+    PrintWriter out = new PrintWriter(os);
+    
+    out.println("Master status for " + master.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\nServers:");
+    out.println(LINE);
+    dumpServers(master, out);
+    
+    out.println("\n\nRegions-in-transition:");
+    out.println(LINE);
+    dumpRIT(master, out);
+    
+    out.println("\n\nExecutors:");
+    out.println(LINE);
+    dumpExecutors(master.getExecutorService(), out);
+    
+    out.println("\n\nStacks:");
+    out.println(LINE);
+    ReflectionUtils.printThreadInfo(out, "");
+    
+    out.println("\n\nMaster configuration:");
+    out.println(LINE);
+    Configuration conf = master.getConfiguration();
+    out.flush();
+    conf.writeXml(os);
+    os.flush();
+    
+    out.println("\n\nRecent regionserver aborts:");
+    out.println(LINE);
+    master.getRegionServerFatalLogBuffer().dumpTo(out);
+    
+    out.println("\n\nLogs");
+    out.println(LINE);
+    long tailKb = getTailKbParam(request);
+    LogMonitoring.dumpTailOfLogs(out, tailKb);
+    
+    out.flush();
+  }
+  
+
+  private void dumpRIT(HMaster master, PrintWriter out) {
+    NavigableMap<String, RegionState> regionsInTransition =
+      master.getAssignmentManager().getRegionsInTransition();
+    for (Map.Entry<String, RegionState> e : regionsInTransition.entrySet()) {
+      String rid = e.getKey();
+      RegionState rs = e.getValue();
+      out.println("Region " + rid + ": " + rs.toDescriptiveString());
+    }
+  }
+
+  private void dumpServers(HMaster master, PrintWriter out) {
+    Map<ServerName, HServerLoad> servers =
+      master.getServerManager().getOnlineServers();
+    for (Map.Entry<ServerName, HServerLoad> e : servers.entrySet()) {
+      out.println(e.getKey() + ": " + e.getValue());
+    }
+  }
+}

Added: hbase/trunk/src/main/java/org/apache/hadoop/hbase/monitoring/LogMonitoring.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/monitoring/LogMonitoring.java?rev=1163385&view=auto
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/monitoring/LogMonitoring.java (added)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/monitoring/LogMonitoring.java Tue Aug 30 21:08:04 2011
@@ -0,0 +1,95 @@
+/**
+ * Copyright 2011 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.
+ */
+package org.apache.hadoop.hbase.monitoring;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.PrintWriter;
+import java.nio.channels.FileChannel;
+import java.util.Enumeration;
+import java.util.Set;
+
+import org.apache.hadoop.io.IOUtils;
+import org.apache.log4j.Appender;
+import org.apache.log4j.FileAppender;
+import org.apache.log4j.Logger;
+
+import com.google.common.collect.Sets;
+
+/**
+ * Utility functions for reading the log4j logs that are
+ * being written by HBase.
+ */
+public abstract class LogMonitoring {
+  public static Set<File> getActiveLogFiles() throws IOException {
+    Set<File> ret = Sets.newHashSet();
+    Appender a;
+    @SuppressWarnings("unchecked")
+    Enumeration<Appender> e = Logger.getRootLogger().getAllAppenders();
+    while (e.hasMoreElements()) {
+      a = e.nextElement();
+      if (a instanceof FileAppender) {
+        FileAppender fa = (FileAppender) a;
+        String filename = fa.getFile();
+        ret.add(new File(filename));
+      }
+    }
+    return ret;
+  }
+  
+
+  public static void dumpTailOfLogs(
+      PrintWriter out, long tailKb) throws IOException {
+    Set<File> logs = LogMonitoring.getActiveLogFiles();
+    for (File f : logs) {
+      out.println("+++++++++++++++++++++++++++++++");
+      out.println(f.getAbsolutePath());
+      out.println("+++++++++++++++++++++++++++++++");
+      try {
+        dumpTailOfLog(f, out, tailKb);
+      } catch (IOException ioe) {
+        out.println("Unable to dump log at " + f);
+        ioe.printStackTrace(out);
+      }
+      out.println("\n\n");
+    }
+  }
+
+  private static void dumpTailOfLog(File f, PrintWriter out, long tailKb)
+      throws IOException {
+    FileInputStream fis = new FileInputStream(f);
+    try {
+      FileChannel channel = fis.getChannel();
+      channel.position(Math.max(0, channel.size() - tailKb*1024));
+      BufferedReader r = new BufferedReader(
+          new InputStreamReader(fis));
+      r.readLine(); // skip the first partial line
+      String line;
+      while ((line = r.readLine()) != null) {
+        out.println(line);
+      }
+    } finally {
+      IOUtils.closeStream(fis);
+    }
+  }
+}

Added: hbase/trunk/src/main/java/org/apache/hadoop/hbase/monitoring/StateDumpServlet.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/monitoring/StateDumpServlet.java?rev=1163385&view=auto
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/monitoring/StateDumpServlet.java (added)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/monitoring/StateDumpServlet.java Tue Aug 30 21:08:04 2011
@@ -0,0 +1,62 @@
+/**
+ * Copyright 2011 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.
+ */
+package org.apache.hadoop.hbase.monitoring;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.Map;
+
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+
+import org.apache.hadoop.hbase.executor.ExecutorService;
+import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorStatus;
+import org.apache.hadoop.hbase.util.VersionInfo;
+
+public abstract class StateDumpServlet extends HttpServlet {
+  static final long DEFAULT_TAIL_KB = 100;
+  private static final long serialVersionUID = 1L;
+
+  protected void dumpVersionInfo(PrintWriter out) {
+    VersionInfo.writeTo(out);
+  
+    out.println("Hadoop " + org.apache.hadoop.util.VersionInfo.getVersion());
+    out.println("Subversion " + org.apache.hadoop.util.VersionInfo.getUrl() + " -r " +
+        org.apache.hadoop.util.VersionInfo.getRevision());
+    out.println("Compiled by " + org.apache.hadoop.util.VersionInfo.getUser() +
+        " on " + org.apache.hadoop.util.VersionInfo.getDate());
+  }
+
+  protected long getTailKbParam(HttpServletRequest request) {
+    String param = request.getParameter("tailkb");
+    if (param == null) {
+      return DEFAULT_TAIL_KB;
+    }
+    return Long.parseLong(param);
+  }
+
+  protected void dumpExecutors(ExecutorService service, PrintWriter out)
+      throws IOException {
+    Map<String, ExecutorStatus> statuses = service.getAllExecutorStatuses();
+    for (ExecutorStatus status : statuses.values()) {
+      status.dumpTo(out, "  ");
+    }
+  }
+}
\ No newline at end of file

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/monitoring/TaskMonitor.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/monitoring/TaskMonitor.java?rev=1163385&r1=1163384&r2=1163385&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/monitoring/TaskMonitor.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/monitoring/TaskMonitor.java Tue Aug 30 21:08:04 2011
@@ -19,6 +19,7 @@
  */
 package org.apache.hadoop.hbase.monitoring;
 
+import java.io.PrintWriter;
 import java.lang.ref.WeakReference;
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.Method;
@@ -120,6 +121,28 @@ public class TaskMonitor {
     return (cts > 0 && System.currentTimeMillis() - cts > EXPIRATION_TIME);
   }
   
+
+  public void dumpAsText(PrintWriter out) {
+    long now = System.currentTimeMillis();
+    
+    List<MonitoredTask> tasks = TaskMonitor.get().getTasks();
+    for (MonitoredTask task : tasks) {
+      out.println("Task: " + task.getDescription());
+      out.println("Status: " + task.getState() + ":" + task.getStatus());
+      long running = (now - task.getStartTime())/1000;
+      if (task.getCompletionTimestamp() != -1) {
+        long completed = (now - task.getCompletionTimestamp()) / 1000;
+        out.println("Completed " + completed + "s ago");
+        out.println("Ran for " +
+            (task.getCompletionTimestamp() - task.getStartTime())/1000
+            + "s");
+      } else {
+        out.println("Running for " + running + "s");
+      }
+      out.println();
+    }
+  }
+
   /**
    * This class encapsulates an object as well as a weak reference to a proxy
    * that passes through calls to that object. In art form:

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1163385&r1=1163384&r2=1163385&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Tue Aug 30 21:08:04 2011
@@ -1360,6 +1360,7 @@ public class HRegionServer implements HR
       try {
         this.infoServer = new InfoServer("regionserver", addr, port, false, this.conf);
         this.infoServer.addServlet("status", "/rs-status", RSStatusServlet.class);
+        this.infoServer.addServlet("dump", "/dump", RSDumpServlet.class);
         this.infoServer.setAttribute(REGIONSERVER, this);
         this.infoServer.start();
         break;
@@ -3003,6 +3004,10 @@ public class HRegionServer implements HR
   public Set<byte[]> getRegionsInTransitionInRS() {
     return this.regionsInTransitionInRS;
   }
+  
+  public ExecutorService getExecutorService() {
+    return service;
+  }
 
   //
   // Main program and support routines

Added: hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/RSDumpServlet.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/RSDumpServlet.java?rev=1163385&view=auto
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/RSDumpServlet.java (added)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/RSDumpServlet.java Tue Aug 30 21:08:04 2011
@@ -0,0 +1,85 @@
+/**
+ * Copyright 2011 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.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import java.io.IOException;
+import java.io.OutputStream;
+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.LogMonitoring;
+import org.apache.hadoop.hbase.monitoring.StateDumpServlet;
+import org.apache.hadoop.hbase.monitoring.TaskMonitor;
+import org.apache.hadoop.util.ReflectionUtils;
+
+public class RSDumpServlet extends StateDumpServlet {
+  private static final long serialVersionUID = 1L;
+  private static final String LINE =
+    "===========================================================";
+  
+  @Override
+  public void doGet(HttpServletRequest request, HttpServletResponse response)
+      throws IOException {
+    HRegionServer hrs = (HRegionServer)getServletContext().getAttribute(
+        HRegionServer.REGIONSERVER);
+    assert hrs != null : "No RS in context!";
+
+    response.setContentType("text/plain");
+    OutputStream os = response.getOutputStream();
+    PrintWriter out = new PrintWriter(os);
+    
+    out.println("Master status for " + hrs.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\nExecutors:");
+    out.println(LINE);
+    dumpExecutors(hrs.getExecutorService(), out);
+    
+    out.println("\n\nStacks:");
+    out.println(LINE);
+    ReflectionUtils.printThreadInfo(out, "");
+    
+    out.println("\n\nRS Configuration:");
+    out.println(LINE);
+    Configuration conf = hrs.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();
+  }  
+}

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/VersionInfo.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/VersionInfo.java?rev=1163385&r1=1163384&r2=1163385&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/VersionInfo.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/VersionInfo.java Tue Aug 30 21:08:04 2011
@@ -21,6 +21,8 @@
 package org.apache.hadoop.hbase.util;
 
 import org.apache.commons.logging.LogFactory;
+import java.io.PrintWriter;
+
 import org.apache.hadoop.hbase.VersionAnnotation;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.commons.logging.Log;
@@ -87,15 +89,27 @@ public class VersionInfo {
     return version != null ? version.url() : "Unknown";
   }
   
-  public static void logVersion(){
-	  LOG.info("HBase " + getVersion());
-	  LOG.info("Subversion " + getUrl() + " -r " + getRevision());
-	  LOG.info("Compiled by " + getUser() + " on " + getDate());
+  static String[] versionReport() {
+    return new String[] {
+      "HBase " + getVersion(),
+      "Subversion " + getUrl() + " -r " + getRevision(),
+      "Compiled by " + getUser() + " on " + getDate()
+      };
+  }
+
+  public static void writeTo(PrintWriter out) {
+    for (String line : versionReport()) {
+      out.println(line);
+    }
+  }
+
+  public static void logVersion() {
+    for (String line : versionReport()) {
+      LOG.info(line);
+    }
   }
 
   public static void main(String[] args) {
-    System.out.println("HBase " + getVersion());
-    System.out.println("Subversion " + getUrl() + " -r " + getRevision());
-    System.out.println("Compiled by " + getUser() + " on " + getDate());
+    logVersion();
   }
 }