HBASE-9630 Add thread which detects JVM pauses like HADOOP's (Liang Xie)

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1526026 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
ndimiduk 2013-09-24 22:11:09 +00:00
parent 49a29f6e94
commit b6ad121cb9
3 changed files with 206 additions and 0 deletions

View File

@ -199,6 +199,7 @@ import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
import org.apache.hadoop.hbase.util.HasThread;
import org.apache.hadoop.hbase.util.InfoServer;
import org.apache.hadoop.hbase.util.JvmPauseMonitor;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Sleeper;
import org.apache.hadoop.hbase.util.Strings;
@ -277,6 +278,7 @@ MasterServices, Server {
// RPC server for the HMaster
private final RpcServerInterface rpcServer;
private JvmPauseMonitor pauseMonitor;
// Set after we've called HBaseServer#openServer and ready to receive RPCs.
// Set back to false after we stop rpcServer. Used by tests.
private volatile boolean rpcServerOpen = false;
@ -456,6 +458,8 @@ MasterServices, Server {
this.zooKeeper = new ZooKeeperWatcher(conf, MASTER + ":" + isa.getPort(), this, true);
this.rpcServer.startThreads();
this.pauseMonitor = new JvmPauseMonitor(conf);
this.pauseMonitor.start();
// metrics interval: using the same property as region server.
this.msgInterval = conf.getInt("hbase.regionserver.msginterval", 3 * 1000);
@ -1235,6 +1239,9 @@ MasterServices, Server {
if (this.healthCheckChore != null) {
this.healthCheckChore.interrupt();
}
if (this.pauseMonitor != null) {
this.pauseMonitor.stop();
}
}
private static Thread getAndStartClusterStatusChore(HMaster master) {

View File

@ -201,6 +201,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.InfoServer;
import org.apache.hadoop.hbase.util.JvmPauseMonitor;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Sleeper;
import org.apache.hadoop.hbase.util.Strings;
@ -376,6 +377,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
// is name of the webapp and the attribute name used stuffing this instance
// into web context.
InfoServer infoServer;
private JvmPauseMonitor pauseMonitor;
/** region server process name */
public static final String REGIONSERVER = "regionserver";
@ -782,6 +784,8 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
// Setup RPC client for master communication
rpcClient = new RpcClient(conf, clusterId, new InetSocketAddress(
this.isa.getAddress(), 0));
this.pauseMonitor = new JvmPauseMonitor(conf);
pauseMonitor.start();
}
/**
@ -947,6 +951,9 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
}
this.rpcClient.stop();
this.leases.close();
if (this.pauseMonitor != null) {
this.pauseMonitor.stop();
}
if (!killed) {
join();

View File

@ -0,0 +1,192 @@
/*
* 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.lang.management.GarbageCollectorMXBean;
import java.lang.management.ManagementFactory;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.common.base.Stopwatch;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
/**
* Class which sets up a simple thread which runs in a loop sleeping
* for a short interval of time. If the sleep takes significantly longer
* than its target time, it implies that the JVM or host machine has
* paused processing, which may cause other problems. If such a pause is
* detected, the thread logs a message.
* The original JvmPauseMonitor is:
* ${hadoop-common-project}/hadoop-common/src/main/java/org/apache/hadoop/util/
* JvmPauseMonitor.java
* r1503806 | cmccabe | 2013-07-17 01:48:24 +0800 (Wed, 17 Jul 2013) | 1 line
* HADOOP-9618. thread which detects GC pauses(Todd Lipcon)
*/
@InterfaceAudience.Private
public class JvmPauseMonitor {
private static final Log LOG = LogFactory.getLog(JvmPauseMonitor.class);
/** The target sleep time */
private static final long SLEEP_INTERVAL_MS = 500;
/** log WARN if we detect a pause longer than this threshold */
private final long warnThresholdMs;
private static final String WARN_THRESHOLD_KEY =
"jvm.pause.warn-threshold.ms";
private static final long WARN_THRESHOLD_DEFAULT = 10000;
/** log INFO if we detect a pause longer than this threshold */
private final long infoThresholdMs;
private static final String INFO_THRESHOLD_KEY =
"jvm.pause.info-threshold.ms";
private static final long INFO_THRESHOLD_DEFAULT = 1000;
private Thread monitorThread;
private volatile boolean shouldRun = true;
public JvmPauseMonitor(Configuration conf) {
this.warnThresholdMs = conf.getLong(WARN_THRESHOLD_KEY, WARN_THRESHOLD_DEFAULT);
this.infoThresholdMs = conf.getLong(INFO_THRESHOLD_KEY, INFO_THRESHOLD_DEFAULT);
}
public void start() {
Preconditions.checkState(monitorThread == null, "Already started");
monitorThread = new Thread(new Monitor());
monitorThread.setDaemon(true);
monitorThread.setName("JvmPauseMonitor");
monitorThread.start();
}
public void stop() {
shouldRun = false;
monitorThread.interrupt();
try {
monitorThread.join();
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
}
}
private String formatMessage(long extraSleepTime, Map<String, GcTimes> gcTimesAfterSleep,
Map<String, GcTimes> gcTimesBeforeSleep) {
Set<String> gcBeanNames = Sets.intersection(gcTimesAfterSleep.keySet(),
gcTimesBeforeSleep.keySet());
List<String> gcDiffs = Lists.newArrayList();
for (String name : gcBeanNames) {
GcTimes diff = gcTimesAfterSleep.get(name).subtract(gcTimesBeforeSleep.get(name));
if (diff.gcCount != 0) {
gcDiffs.add("GC pool '" + name + "' had collection(s): " + diff.toString());
}
}
String ret = "Detected pause in JVM or host machine (eg GC): " + "pause of approximately "
+ extraSleepTime + "ms\n";
if (gcDiffs.isEmpty()) {
ret += "No GCs detected";
} else {
ret += Joiner.on("\n").join(gcDiffs);
}
return ret;
}
private Map<String, GcTimes> getGcTimes() {
Map<String, GcTimes> map = Maps.newHashMap();
List<GarbageCollectorMXBean> gcBeans = ManagementFactory.getGarbageCollectorMXBeans();
for (GarbageCollectorMXBean gcBean : gcBeans) {
map.put(gcBean.getName(), new GcTimes(gcBean));
}
return map;
}
private static class GcTimes {
private GcTimes(GarbageCollectorMXBean gcBean) {
gcCount = gcBean.getCollectionCount();
gcTimeMillis = gcBean.getCollectionTime();
}
private GcTimes(long count, long time) {
this.gcCount = count;
this.gcTimeMillis = time;
}
private GcTimes subtract(GcTimes other) {
return new GcTimes(this.gcCount - other.gcCount, this.gcTimeMillis - other.gcTimeMillis);
}
@Override
public String toString() {
return "count=" + gcCount + " time=" + gcTimeMillis + "ms";
}
private long gcCount;
private long gcTimeMillis;
}
private class Monitor implements Runnable {
@Override
public void run() {
Stopwatch sw = new Stopwatch();
Map<String, GcTimes> gcTimesBeforeSleep = getGcTimes();
while (shouldRun) {
sw.reset().start();
try {
Thread.sleep(SLEEP_INTERVAL_MS);
} catch (InterruptedException ie) {
return;
}
long extraSleepTime = sw.elapsedMillis() - SLEEP_INTERVAL_MS;
Map<String, GcTimes> gcTimesAfterSleep = getGcTimes();
if (extraSleepTime > warnThresholdMs) {
LOG.warn(formatMessage(extraSleepTime, gcTimesAfterSleep, gcTimesBeforeSleep));
} else if (extraSleepTime > infoThresholdMs) {
LOG.info(formatMessage(extraSleepTime, gcTimesAfterSleep, gcTimesBeforeSleep));
}
gcTimesBeforeSleep = gcTimesAfterSleep;
}
}
}
/**
* Simple 'main' to facilitate manual testing of the pause monitor.
*
* This main function just leaks memory into a list. Running this class
* with a 1GB heap will very quickly go into "GC hell" and result in
* log messages about the GC pauses.
*/
public static void main(String []args) throws Exception {
new JvmPauseMonitor(new Configuration()).start();
List<String> list = Lists.newArrayList();
int i = 0;
while (true) {
list.add(String.valueOf(i++));
}
}
}