1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.util;
19
20 import java.lang.management.GarbageCollectorMXBean;
21 import java.lang.management.ManagementFactory;
22 import java.util.List;
23 import java.util.Map;
24 import java.util.Set;
25
26 import org.apache.commons.logging.Log;
27 import org.apache.commons.logging.LogFactory;
28 import org.apache.hadoop.hbase.classification.InterfaceAudience;
29 import org.apache.hadoop.conf.Configuration;
30
31 import com.google.common.base.Joiner;
32 import com.google.common.base.Preconditions;
33 import com.google.common.base.Stopwatch;
34 import com.google.common.collect.Lists;
35 import com.google.common.collect.Maps;
36 import com.google.common.collect.Sets;
37
38
39
40
41
42
43
44
45
46
47
48
49
50 @InterfaceAudience.Private
51 public class JvmPauseMonitor {
52 private static final Log LOG = LogFactory.getLog(JvmPauseMonitor.class);
53
54
55 private static final long SLEEP_INTERVAL_MS = 500;
56
57
58 private final long warnThresholdMs;
59 private static final String WARN_THRESHOLD_KEY =
60 "jvm.pause.warn-threshold.ms";
61 private static final long WARN_THRESHOLD_DEFAULT = 10000;
62
63
64 private final long infoThresholdMs;
65 private static final String INFO_THRESHOLD_KEY =
66 "jvm.pause.info-threshold.ms";
67 private static final long INFO_THRESHOLD_DEFAULT = 1000;
68
69 private Thread monitorThread;
70 private volatile boolean shouldRun = true;
71
72 public JvmPauseMonitor(Configuration conf) {
73 this.warnThresholdMs = conf.getLong(WARN_THRESHOLD_KEY, WARN_THRESHOLD_DEFAULT);
74 this.infoThresholdMs = conf.getLong(INFO_THRESHOLD_KEY, INFO_THRESHOLD_DEFAULT);
75 }
76
77 public void start() {
78 Preconditions.checkState(monitorThread == null, "Already started");
79 monitorThread = new Thread(new Monitor(), "JvmPauseMonitor");
80 monitorThread.setDaemon(true);
81 monitorThread.start();
82 }
83
84 public void stop() {
85 shouldRun = false;
86 monitorThread.interrupt();
87 try {
88 monitorThread.join();
89 } catch (InterruptedException e) {
90 Thread.currentThread().interrupt();
91 }
92 }
93
94 private String formatMessage(long extraSleepTime, Map<String, GcTimes> gcTimesAfterSleep,
95 Map<String, GcTimes> gcTimesBeforeSleep) {
96
97 Set<String> gcBeanNames = Sets.intersection(gcTimesAfterSleep.keySet(),
98 gcTimesBeforeSleep.keySet());
99 List<String> gcDiffs = Lists.newArrayList();
100 for (String name : gcBeanNames) {
101 GcTimes diff = gcTimesAfterSleep.get(name).subtract(gcTimesBeforeSleep.get(name));
102 if (diff.gcCount != 0) {
103 gcDiffs.add("GC pool '" + name + "' had collection(s): " + diff.toString());
104 }
105 }
106
107 String ret = "Detected pause in JVM or host machine (eg GC): " + "pause of approximately "
108 + extraSleepTime + "ms\n";
109 if (gcDiffs.isEmpty()) {
110 ret += "No GCs detected";
111 } else {
112 ret += Joiner.on("\n").join(gcDiffs);
113 }
114 return ret;
115 }
116
117 private Map<String, GcTimes> getGcTimes() {
118 Map<String, GcTimes> map = Maps.newHashMap();
119 List<GarbageCollectorMXBean> gcBeans = ManagementFactory.getGarbageCollectorMXBeans();
120 for (GarbageCollectorMXBean gcBean : gcBeans) {
121 map.put(gcBean.getName(), new GcTimes(gcBean));
122 }
123 return map;
124 }
125
126 private static class GcTimes {
127 private GcTimes(GarbageCollectorMXBean gcBean) {
128 gcCount = gcBean.getCollectionCount();
129 gcTimeMillis = gcBean.getCollectionTime();
130 }
131
132 private GcTimes(long count, long time) {
133 this.gcCount = count;
134 this.gcTimeMillis = time;
135 }
136
137 private GcTimes subtract(GcTimes other) {
138 return new GcTimes(this.gcCount - other.gcCount, this.gcTimeMillis - other.gcTimeMillis);
139 }
140
141 @Override
142 public String toString() {
143 return "count=" + gcCount + " time=" + gcTimeMillis + "ms";
144 }
145
146 private long gcCount;
147 private long gcTimeMillis;
148 }
149
150 private class Monitor implements Runnable {
151 @Override
152 public void run() {
153 Stopwatch sw = new Stopwatch();
154 Map<String, GcTimes> gcTimesBeforeSleep = getGcTimes();
155 while (shouldRun) {
156 sw.reset().start();
157 try {
158 Thread.sleep(SLEEP_INTERVAL_MS);
159 } catch (InterruptedException ie) {
160 return;
161 }
162 long extraSleepTime = sw.elapsedMillis() - SLEEP_INTERVAL_MS;
163 Map<String, GcTimes> gcTimesAfterSleep = getGcTimes();
164
165 if (extraSleepTime > warnThresholdMs) {
166 LOG.warn(formatMessage(extraSleepTime, gcTimesAfterSleep, gcTimesBeforeSleep));
167 } else if (extraSleepTime > infoThresholdMs) {
168 LOG.info(formatMessage(extraSleepTime, gcTimesAfterSleep, gcTimesBeforeSleep));
169 }
170
171 gcTimesBeforeSleep = gcTimesAfterSleep;
172 }
173 }
174 }
175
176
177
178
179
180
181
182
183 public static void main(String []args) throws Exception {
184 new JvmPauseMonitor(new Configuration()).start();
185 List<String> list = Lists.newArrayList();
186 int i = 0;
187 while (true) {
188 list.add(String.valueOf(i++));
189 }
190 }
191 }