View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.monitoring;
20  
21  import java.io.PrintWriter;
22  import java.lang.ref.WeakReference;
23  import java.lang.reflect.InvocationHandler;
24  import java.lang.reflect.Method;
25  import java.lang.reflect.Proxy;
26  import java.util.ArrayList;
27  import java.util.Iterator;
28  import java.util.List;
29  
30  import org.apache.commons.collections.buffer.CircularFifoBuffer;
31  import org.apache.commons.logging.Log;
32  import org.apache.commons.logging.LogFactory;
33  import org.apache.hadoop.conf.Configuration;
34  import org.apache.hadoop.hbase.HBaseConfiguration;
35  import org.apache.hadoop.hbase.classification.InterfaceAudience;
36  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
37  import org.apache.hadoop.hbase.util.Threads;
38  
39  import com.google.common.collect.Lists;
40  
41  /**
42   * Singleton which keeps track of tasks going on in this VM.
43   * A Task here is anything which takes more than a few seconds
44   * and the user might want to inquire about the status
45   */
46  @InterfaceAudience.Private
47  public class TaskMonitor {
48    private static final Log LOG = LogFactory.getLog(TaskMonitor.class);
49  
50    public static final String MAX_TASKS_KEY = "hbase.taskmonitor.max.tasks";
51    public static final int DEFAULT_MAX_TASKS = 1000;
52    public static final String RPC_WARN_TIME_KEY = "hbase.taskmonitor.rpc.warn.time";
53    public static final long DEFAULT_RPC_WARN_TIME = 0;
54    public static final String EXPIRATION_TIME_KEY = "hbase.taskmonitor.expiration.time";
55    public static final long DEFAULT_EXPIRATION_TIME = 60*1000;
56    public static final String MONITOR_INTERVAL_KEY = "hbase.taskmonitor.monitor.interval";
57    public static final long DEFAULT_MONITOR_INTERVAL = 10*1000;
58  
59    private static TaskMonitor instance;
60  
61    private final int maxTasks;
62    private final long rpcWarnTime;
63    private final long expirationTime;
64    private final CircularFifoBuffer tasks;
65    private final List<TaskAndWeakRefPair> rpcTasks;
66    private final long monitorInterval;
67    private Thread monitorThread;
68  
69    TaskMonitor(Configuration conf) {
70      maxTasks = conf.getInt(MAX_TASKS_KEY, DEFAULT_MAX_TASKS);
71      expirationTime = conf.getLong(EXPIRATION_TIME_KEY, DEFAULT_EXPIRATION_TIME);
72      rpcWarnTime = conf.getLong(RPC_WARN_TIME_KEY, DEFAULT_RPC_WARN_TIME);
73      tasks = new CircularFifoBuffer(maxTasks);
74      rpcTasks = Lists.newArrayList();
75      monitorInterval = conf.getLong(MONITOR_INTERVAL_KEY, DEFAULT_MONITOR_INTERVAL);
76      monitorThread = new Thread(new MonitorRunnable());
77      Threads.setDaemonThreadRunning(monitorThread, "Monitor thread for TaskMonitor");
78    }
79  
80    /**
81     * Get singleton instance.
82     * TODO this would be better off scoped to a single daemon
83     */
84    public static synchronized TaskMonitor get() {
85      if (instance == null) {
86        instance = new TaskMonitor(HBaseConfiguration.create());
87      }
88      return instance;
89    }
90    
91    public synchronized MonitoredTask createStatus(String description) {
92      MonitoredTask stat = new MonitoredTaskImpl();
93      stat.setDescription(description);
94      MonitoredTask proxy = (MonitoredTask) Proxy.newProxyInstance(
95          stat.getClass().getClassLoader(),
96          new Class<?>[] { MonitoredTask.class },
97          new PassthroughInvocationHandler<MonitoredTask>(stat));
98      TaskAndWeakRefPair pair = new TaskAndWeakRefPair(stat, proxy);
99      if (tasks.isFull()) {
100       purgeExpiredTasks();
101     }
102     tasks.add(pair);
103     return proxy;
104   }
105 
106   public synchronized MonitoredRPCHandler createRPCStatus(String description) {
107     MonitoredRPCHandler stat = new MonitoredRPCHandlerImpl();
108     stat.setDescription(description);
109     MonitoredRPCHandler proxy = (MonitoredRPCHandler) Proxy.newProxyInstance(
110         stat.getClass().getClassLoader(),
111         new Class<?>[] { MonitoredRPCHandler.class },
112         new PassthroughInvocationHandler<MonitoredRPCHandler>(stat));
113     TaskAndWeakRefPair pair = new TaskAndWeakRefPair(stat, proxy);
114     rpcTasks.add(pair);
115     return proxy;
116   }
117 
118   private synchronized void warnStuckTasks() {
119     if (rpcWarnTime > 0) {
120       final long now = EnvironmentEdgeManager.currentTime();
121       for (Iterator<TaskAndWeakRefPair> it = rpcTasks.iterator();
122           it.hasNext();) {
123         TaskAndWeakRefPair pair = it.next();
124         MonitoredTask stat = pair.get();
125         if ((stat.getState() == MonitoredTaskImpl.State.RUNNING) &&
126             (now >= stat.getWarnTime() + rpcWarnTime)) {
127           LOG.warn("Task may be stuck: " + stat);
128           stat.setWarnTime(now);
129         }
130       }
131     }
132   }
133 
134   private synchronized void purgeExpiredTasks() {
135     for (Iterator<TaskAndWeakRefPair> it = tasks.iterator();
136          it.hasNext();) {
137       TaskAndWeakRefPair pair = it.next();
138       MonitoredTask stat = pair.get();
139       
140       if (pair.isDead()) {
141         // The class who constructed this leaked it. So we can
142         // assume it's done.
143         if (stat.getState() == MonitoredTaskImpl.State.RUNNING) {
144           LOG.warn("Status " + stat + " appears to have been leaked");
145           stat.cleanup();
146         }
147       }
148       
149       if (canPurge(stat)) {
150         it.remove();
151       }
152     }
153   }
154 
155   /**
156    * Produces a list containing copies of the current state of all non-expired 
157    * MonitoredTasks handled by this TaskMonitor.
158    * @return A complete list of MonitoredTasks.
159    */
160   public synchronized List<MonitoredTask> getTasks() {
161     purgeExpiredTasks();
162     ArrayList<MonitoredTask> ret = Lists.newArrayListWithCapacity(tasks.size() + rpcTasks
163         .size());
164     for (Iterator<TaskAndWeakRefPair> it = tasks.iterator();
165          it.hasNext();) {
166       TaskAndWeakRefPair pair = it.next();
167       MonitoredTask t = pair.get();
168       ret.add(t.clone());
169     }
170     for (Iterator<TaskAndWeakRefPair> it = rpcTasks.iterator();
171          it.hasNext();) {
172       TaskAndWeakRefPair pair = it.next();
173       MonitoredTask t = pair.get();
174       ret.add(t.clone());
175     }
176     return ret;
177   }
178 
179   private boolean canPurge(MonitoredTask stat) {
180     long cts = stat.getCompletionTimestamp();
181     return (cts > 0 && EnvironmentEdgeManager.currentTime() - cts > expirationTime);
182   }
183 
184   public void dumpAsText(PrintWriter out) {
185     long now = EnvironmentEdgeManager.currentTime();
186     
187     List<MonitoredTask> tasks = getTasks();
188     for (MonitoredTask task : tasks) {
189       out.println("Task: " + task.getDescription());
190       out.println("Status: " + task.getState() + ":" + task.getStatus());
191       long running = (now - task.getStartTime())/1000;
192       if (task.getCompletionTimestamp() != -1) {
193         long completed = (now - task.getCompletionTimestamp()) / 1000;
194         out.println("Completed " + completed + "s ago");
195         out.println("Ran for " +
196             (task.getCompletionTimestamp() - task.getStartTime())/1000
197             + "s");
198       } else {
199         out.println("Running for " + running + "s");
200       }
201       out.println();
202     }
203   }
204 
205   public synchronized void shutdown() {
206     if (this.monitorThread != null) {
207       monitorThread.interrupt();
208     }
209   }
210 
211   /**
212    * This class encapsulates an object as well as a weak reference to a proxy
213    * that passes through calls to that object. In art form:
214    * <code>
215    *     Proxy  <------------------
216    *       |                       \
217    *       v                        \
218    * PassthroughInvocationHandler   |  weak reference
219    *       |                       /
220    * MonitoredTaskImpl            / 
221    *       |                     /
222    * StatAndWeakRefProxy  ------/
223    *
224    * Since we only return the Proxy to the creator of the MonitorableStatus,
225    * this means that they can leak that object, and we'll detect it
226    * since our weak reference will go null. But, we still have the actual
227    * object, so we can log it and display it as a leaked (incomplete) action.
228    */
229   private static class TaskAndWeakRefPair {
230     private MonitoredTask impl;
231     private WeakReference<MonitoredTask> weakProxy;
232     
233     public TaskAndWeakRefPair(MonitoredTask stat,
234         MonitoredTask proxy) {
235       this.impl = stat;
236       this.weakProxy = new WeakReference<MonitoredTask>(proxy);
237     }
238     
239     public MonitoredTask get() {
240       return impl;
241     }
242     
243     public boolean isDead() {
244       return weakProxy.get() == null;
245     }
246   }
247   
248   /**
249    * An InvocationHandler that simply passes through calls to the original 
250    * object.
251    */
252   private static class PassthroughInvocationHandler<T> implements InvocationHandler {
253     private T delegatee;
254     
255     public PassthroughInvocationHandler(T delegatee) {
256       this.delegatee = delegatee;
257     }
258 
259     @Override
260     public Object invoke(Object proxy, Method method, Object[] args)
261         throws Throwable {
262       return method.invoke(delegatee, args);
263     }    
264   }
265 
266   private class MonitorRunnable implements Runnable {
267     private boolean running = true;
268 
269     @Override
270     public void run() {
271       while (running) {
272         try {
273           Thread.sleep(monitorInterval);
274           if (tasks.isFull()) {
275             purgeExpiredTasks();
276           }
277           warnStuckTasks();
278         } catch (InterruptedException e) {
279           running = false;
280         }
281       }
282     }
283   }
284 }