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());
80 monitorThread.setDaemon(true);
81 monitorThread.setName("JvmPauseMonitor");
82 monitorThread.start();
83 }
84
85 public void stop() {
86 shouldRun = false;
87 monitorThread.interrupt();
88 try {
89 monitorThread.join();
90 } catch (InterruptedException e) {
91 Thread.currentThread().interrupt();
92 }
93 }
94
95 private String formatMessage(long extraSleepTime, Map<String, GcTimes> gcTimesAfterSleep,
96 Map<String, GcTimes> gcTimesBeforeSleep) {
97
98 Set<String> gcBeanNames = Sets.intersection(gcTimesAfterSleep.keySet(),
99 gcTimesBeforeSleep.keySet());
100 List<String> gcDiffs = Lists.newArrayList();
101 for (String name : gcBeanNames) {
102 GcTimes diff = gcTimesAfterSleep.get(name).subtract(gcTimesBeforeSleep.get(name));
103 if (diff.gcCount != 0) {
104 gcDiffs.add("GC pool '" + name + "' had collection(s): " + diff.toString());
105 }
106 }
107
108 String ret = "Detected pause in JVM or host machine (eg GC): " + "pause of approximately "
109 + extraSleepTime + "ms\n";
110 if (gcDiffs.isEmpty()) {
111 ret += "No GCs detected";
112 } else {
113 ret += Joiner.on("\n").join(gcDiffs);
114 }
115 return ret;
116 }
117
118 private Map<String, GcTimes> getGcTimes() {
119 Map<String, GcTimes> map = Maps.newHashMap();
120 List<GarbageCollectorMXBean> gcBeans = ManagementFactory.getGarbageCollectorMXBeans();
121 for (GarbageCollectorMXBean gcBean : gcBeans) {
122 map.put(gcBean.getName(), new GcTimes(gcBean));
123 }
124 return map;
125 }
126
127 private static class GcTimes {
128 private GcTimes(GarbageCollectorMXBean gcBean) {
129 gcCount = gcBean.getCollectionCount();
130 gcTimeMillis = gcBean.getCollectionTime();
131 }
132
133 private GcTimes(long count, long time) {
134 this.gcCount = count;
135 this.gcTimeMillis = time;
136 }
137
138 private GcTimes subtract(GcTimes other) {
139 return new GcTimes(this.gcCount - other.gcCount, this.gcTimeMillis - other.gcTimeMillis);
140 }
141
142 @Override
143 public String toString() {
144 return "count=" + gcCount + " time=" + gcTimeMillis + "ms";
145 }
146
147 private long gcCount;
148 private long gcTimeMillis;
149 }
150
151 private class Monitor implements Runnable {
152 @Override
153 public void run() {
154 Stopwatch sw = new Stopwatch();
155 Map<String, GcTimes> gcTimesBeforeSleep = getGcTimes();
156 while (shouldRun) {
157 sw.reset().start();
158 try {
159 Thread.sleep(SLEEP_INTERVAL_MS);
160 } catch (InterruptedException ie) {
161 return;
162 }
163 long extraSleepTime = sw.elapsedMillis() - SLEEP_INTERVAL_MS;
164 Map<String, GcTimes> gcTimesAfterSleep = getGcTimes();
165
166 if (extraSleepTime > warnThresholdMs) {
167 LOG.warn(formatMessage(extraSleepTime, gcTimesAfterSleep, gcTimesBeforeSleep));
168 } else if (extraSleepTime > infoThresholdMs) {
169 LOG.info(formatMessage(extraSleepTime, gcTimesAfterSleep, gcTimesBeforeSleep));
170 }
171
172 gcTimesBeforeSleep = gcTimesAfterSleep;
173 }
174 }
175 }
176
177
178
179
180
181
182
183
184 public static void main(String []args) throws Exception {
185 new JvmPauseMonitor(new Configuration()).start();
186 List<String> list = Lists.newArrayList();
187 int i = 0;
188 while (true) {
189 list.add(String.valueOf(i++));
190 }
191 }
192 }