View Javadoc

1   /*
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
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   * Class which sets up a simple thread which runs in a loop sleeping
40   * for a short interval of time. If the sleep takes significantly longer
41   * than its target time, it implies that the JVM or host machine has
42   * paused processing, which may cause other problems. If such a pause is
43   * detected, the thread logs a message.
44   * The original JvmPauseMonitor is:
45   * ${hadoop-common-project}/hadoop-common/src/main/java/org/apache/hadoop/util/
46   * JvmPauseMonitor.java
47   * r1503806 | cmccabe | 2013-07-17 01:48:24 +0800 (Wed, 17 Jul 2013) | 1 line
48   * HADOOP-9618.  thread which detects GC pauses(Todd Lipcon)
49   */
50  @InterfaceAudience.Private
51  public class JvmPauseMonitor {
52    private static final Log LOG = LogFactory.getLog(JvmPauseMonitor.class);
53  
54    /** The target sleep time */
55    private static final long SLEEP_INTERVAL_MS = 500;
56    
57    /** log WARN if we detect a pause longer than this threshold */
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    /** log INFO if we detect a pause longer than this threshold */
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    * Simple 'main' to facilitate manual testing of the pause monitor.
178    * 
179    * This main function just leaks memory into a list. Running this class
180    * with a 1GB heap will very quickly go into "GC hell" and result in
181    * log messages about the GC pauses.
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 }