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