001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.util; 019 020import java.lang.management.GarbageCollectorMXBean; 021import java.lang.management.ManagementFactory; 022import java.util.List; 023import java.util.Map; 024import java.util.Set; 025import java.util.concurrent.TimeUnit; 026import org.apache.hadoop.conf.Configuration; 027import org.apache.hadoop.hbase.metrics.JvmPauseMonitorSource; 028import org.apache.yetus.audience.InterfaceAudience; 029import org.slf4j.Logger; 030import org.slf4j.LoggerFactory; 031 032import org.apache.hbase.thirdparty.com.google.common.base.Joiner; 033import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; 034import org.apache.hbase.thirdparty.com.google.common.base.Stopwatch; 035import org.apache.hbase.thirdparty.com.google.common.collect.Lists; 036import org.apache.hbase.thirdparty.com.google.common.collect.Maps; 037import org.apache.hbase.thirdparty.com.google.common.collect.Sets; 038 039/** 040 * Class which sets up a simple thread which runs in a loop sleeping for a short interval of time. 041 * If the sleep takes significantly longer than its target time, it implies that the JVM or host 042 * machine has paused processing, which may cause other problems. If such a pause is detected, the 043 * thread logs a message. The original JvmPauseMonitor is: 044 * ${hadoop-common-project}/hadoop-common/src/main/java/org/apache/hadoop/util/ JvmPauseMonitor.java 045 * r1503806 | cmccabe | 2013-07-17 01:48:24 +0800 (Wed, 17 Jul 2013) | 1 line HADOOP-9618. thread 046 * which detects GC pauses(Todd Lipcon) 047 */ 048@InterfaceAudience.Private 049public class JvmPauseMonitor { 050 private static final Logger LOG = LoggerFactory.getLogger(JvmPauseMonitor.class); 051 052 /** The target sleep time */ 053 private static final long SLEEP_INTERVAL_MS = 500; 054 055 /** log WARN if we detect a pause longer than this threshold */ 056 private final long warnThresholdMs; 057 public static final String WARN_THRESHOLD_KEY = "jvm.pause.warn-threshold.ms"; 058 private static final long WARN_THRESHOLD_DEFAULT = 10000; 059 060 /** log INFO if we detect a pause longer than this threshold */ 061 private final long infoThresholdMs; 062 public static final String INFO_THRESHOLD_KEY = "jvm.pause.info-threshold.ms"; 063 private static final long INFO_THRESHOLD_DEFAULT = 1000; 064 065 private Thread monitorThread; 066 private volatile boolean shouldRun = true; 067 private JvmPauseMonitorSource metricsSource; 068 069 public JvmPauseMonitor(Configuration conf) { 070 this(conf, null); 071 } 072 073 public JvmPauseMonitor(Configuration conf, JvmPauseMonitorSource metricsSource) { 074 this.warnThresholdMs = conf.getLong(WARN_THRESHOLD_KEY, WARN_THRESHOLD_DEFAULT); 075 this.infoThresholdMs = conf.getLong(INFO_THRESHOLD_KEY, INFO_THRESHOLD_DEFAULT); 076 this.metricsSource = metricsSource; 077 } 078 079 public void start() { 080 Preconditions.checkState(monitorThread == null, "Already started"); 081 monitorThread = new Thread(new Monitor(), "JvmPauseMonitor"); 082 monitorThread.setDaemon(true); 083 monitorThread.start(); 084 } 085 086 public void stop() { 087 shouldRun = false; 088 monitorThread.interrupt(); 089 try { 090 monitorThread.join(); 091 } catch (InterruptedException e) { 092 Thread.currentThread().interrupt(); 093 } 094 } 095 096 private String formatMessage(long extraSleepTime, List<String> gcDiffs) { 097 String ret = "Detected pause in JVM or host machine (eg GC): " + "pause of approximately " 098 + extraSleepTime + "ms\n"; 099 if (gcDiffs.isEmpty()) { 100 ret += "No GCs detected"; 101 } else { 102 ret += Joiner.on("\n").join(gcDiffs); 103 } 104 return ret; 105 } 106 107 private Map<String, GcTimes> getGcTimes() { 108 Map<String, GcTimes> map = Maps.newHashMap(); 109 List<GarbageCollectorMXBean> gcBeans = ManagementFactory.getGarbageCollectorMXBeans(); 110 for (GarbageCollectorMXBean gcBean : gcBeans) { 111 map.put(gcBean.getName(), new GcTimes(gcBean)); 112 } 113 return map; 114 } 115 116 private static class GcTimes { 117 private GcTimes(GarbageCollectorMXBean gcBean) { 118 gcCount = gcBean.getCollectionCount(); 119 gcTimeMillis = gcBean.getCollectionTime(); 120 } 121 122 private GcTimes(long count, long time) { 123 this.gcCount = count; 124 this.gcTimeMillis = time; 125 } 126 127 private GcTimes subtract(GcTimes other) { 128 return new GcTimes(this.gcCount - other.gcCount, this.gcTimeMillis - other.gcTimeMillis); 129 } 130 131 @Override 132 public String toString() { 133 return "count=" + gcCount + " time=" + gcTimeMillis + "ms"; 134 } 135 136 private long gcCount; 137 private long gcTimeMillis; 138 } 139 140 private class Monitor implements Runnable { 141 @Override 142 public void run() { 143 Stopwatch sw = Stopwatch.createUnstarted(); 144 Map<String, GcTimes> gcTimesBeforeSleep = getGcTimes(); 145 while (shouldRun) { 146 sw.reset().start(); 147 try { 148 Thread.sleep(SLEEP_INTERVAL_MS); 149 } catch (InterruptedException ie) { 150 return; 151 } 152 153 long extraSleepTime = sw.elapsed(TimeUnit.MILLISECONDS) - SLEEP_INTERVAL_MS; 154 Map<String, GcTimes> gcTimesAfterSleep = getGcTimes(); 155 156 if (extraSleepTime > infoThresholdMs) { 157 Set<String> gcBeanNames = 158 Sets.intersection(gcTimesAfterSleep.keySet(), gcTimesBeforeSleep.keySet()); 159 List<String> gcDiffs = Lists.newArrayList(); 160 for (String name : gcBeanNames) { 161 GcTimes diff = gcTimesAfterSleep.get(name).subtract(gcTimesBeforeSleep.get(name)); 162 if (diff.gcCount != 0) { 163 gcDiffs.add("GC pool '" + name + "' had collection(s): " + diff.toString()); 164 } 165 } 166 167 updateMetrics(extraSleepTime, !gcDiffs.isEmpty()); 168 169 if (extraSleepTime > warnThresholdMs) { 170 LOG.warn(formatMessage(extraSleepTime, gcDiffs)); 171 } else { 172 LOG.info(formatMessage(extraSleepTime, gcDiffs)); 173 } 174 } 175 gcTimesBeforeSleep = gcTimesAfterSleep; 176 } 177 } 178 } 179 180 public void updateMetrics(long sleepTime, boolean gcDetected) { 181 if (metricsSource != null) { 182 if (sleepTime > warnThresholdMs) { 183 metricsSource.incWarnThresholdExceeded(1); 184 } else { 185 metricsSource.incInfoThresholdExceeded(1); 186 } 187 if (gcDetected) { 188 metricsSource.updatePauseTimeWithGc(sleepTime); 189 } else { 190 metricsSource.updatePauseTimeWithoutGc(sleepTime); 191 } 192 } 193 } 194 195 public JvmPauseMonitorSource getMetricsSource() { 196 return metricsSource; 197 } 198 199 public void setMetricsSource(JvmPauseMonitorSource metricsSource) { 200 this.metricsSource = metricsSource; 201 } 202 203 /** 204 * Simple 'main' to facilitate manual testing of the pause monitor. This main function just leaks 205 * memory into a list. Running this class with a 1GB heap will very quickly go into "GC hell" and 206 * result in log messages about the GC pauses. 207 */ 208 public static void main(String[] args) throws Exception { 209 new JvmPauseMonitor(new Configuration()).start(); 210 List<String> list = Lists.newArrayList(); 211 int i = 0; 212 while (true) { 213 list.add(String.valueOf(i++)); 214 } 215 } 216}