HBASE-13710 Remove dependency on Hadoop's ReflectionUtil.

* copies ReflectionUtils.logThreadInfo and needed private methods from Hadoop
  branch-2, fixes minor issues specific to our use.
* updates HttpServer's use of RU.newInstance to use the HBase version.
  Side effect: previously, FilterInitializer instances that happened to also
  implement Configurable would have setConfiguration called. Such uses should
  instead rely on the mandatory FilterInitializer.initFilter method call.
This commit is contained in:
Sean Busbey 2015-05-19 00:06:29 -05:00 committed by Sean Busbey
parent eeb11b5327
commit 353b046d6c
2 changed files with 100 additions and 3 deletions

View File

@ -18,8 +18,17 @@
*/
package org.apache.hadoop.hbase.util;
import java.io.ByteArrayOutputStream;
import java.io.PrintStream;
import java.io.UnsupportedEncodingException;
import java.lang.management.ManagementFactory;
import java.lang.management.ThreadInfo;
import java.lang.management.ThreadMXBean;
import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException;
import java.nio.charset.Charset;
import org.apache.commons.logging.Log;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
@ -89,4 +98,93 @@ public class ReflectionUtils {
throw new UnsupportedOperationException(
"Unable to find suitable constructor for class " + type.getName());
}
/* synchronized on ReflectionUtils.class */
private static long previousLogTime = 0;
private static final ThreadMXBean threadBean = ManagementFactory.getThreadMXBean();
/**
* Log the current thread stacks at INFO level.
* @param log the logger that logs the stack trace
* @param title a descriptive title for the call stacks
* @param minInterval the minimum time from the last
*/
public static void logThreadInfo(Log log,
String title,
long minInterval) {
boolean dumpStack = false;
if (log.isInfoEnabled()) {
synchronized (ReflectionUtils.class) {
long now = System.currentTimeMillis();
if (now - previousLogTime >= minInterval * 1000) {
previousLogTime = now;
dumpStack = true;
}
}
if (dumpStack) {
try {
ByteArrayOutputStream buffer = new ByteArrayOutputStream();
printThreadInfo(new PrintStream(buffer, false, "UTF-8"), title);
log.info(buffer.toString(Charset.defaultCharset().name()));
} catch (UnsupportedEncodingException ignored) {
log.warn("Could not write thread info about '" + title +
"' due to a string encoding issue.");
}
}
}
}
/**
* Print all of the thread's information and stack traces.
*
* @param stream the stream to
* @param title a string title for the stack trace
*/
private static void printThreadInfo(PrintStream stream,
String title) {
final int STACK_DEPTH = 20;
boolean contention = threadBean.isThreadContentionMonitoringEnabled();
long[] threadIds = threadBean.getAllThreadIds();
stream.println("Process Thread Dump: " + title);
stream.println(threadIds.length + " active threads");
for (long tid: threadIds) {
ThreadInfo info = threadBean.getThreadInfo(tid, STACK_DEPTH);
if (info == null) {
stream.println(" Inactive");
continue;
}
stream.println("Thread " +
getTaskName(info.getThreadId(),
info.getThreadName()) + ":");
Thread.State state = info.getThreadState();
stream.println(" State: " + state);
stream.println(" Blocked count: " + info.getBlockedCount());
stream.println(" Waited count: " + info.getWaitedCount());
if (contention) {
stream.println(" Blocked time: " + info.getBlockedTime());
stream.println(" Waited time: " + info.getWaitedTime());
}
if (state == Thread.State.WAITING) {
stream.println(" Waiting on " + info.getLockName());
} else if (state == Thread.State.BLOCKED) {
stream.println(" Blocked on " + info.getLockName());
stream.println(" Blocked by " +
getTaskName(info.getLockOwnerId(),
info.getLockOwnerName()));
}
stream.println(" Stack:");
for (StackTraceElement frame: info.getStackTrace()) {
stream.println(" " + frame.toString());
}
}
stream.flush();
}
private static String getTaskName(long id, String name) {
if (name == null) {
return Long.toString(id);
}
return id + " (" + name + ")";
}
}

View File

@ -57,12 +57,12 @@ import org.apache.hadoop.hbase.http.conf.ConfServlet;
import org.apache.hadoop.hbase.http.jmx.JMXJsonServlet;
import org.apache.hadoop.hbase.http.log.LogLevel;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.ReflectionUtils;
import org.apache.hadoop.metrics.MetricsServlet;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
import org.apache.hadoop.security.authorize.AccessControlList;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.Shell;
import org.mortbay.io.Buffer;
import org.mortbay.jetty.Connector;
@ -617,8 +617,7 @@ public class HttpServer implements FilterContainer {
FilterInitializer[] initializers = new FilterInitializer[classes.length];
for(int i = 0; i < classes.length; i++) {
initializers[i] = (FilterInitializer)ReflectionUtils.newInstance(
classes[i], conf);
initializers[i] = (FilterInitializer)ReflectionUtils.newInstance(classes[i]);
}
return initializers;
}