1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.monitoring;
21
22 import java.io.PrintWriter;
23 import java.lang.ref.WeakReference;
24 import java.lang.reflect.InvocationHandler;
25 import java.lang.reflect.Method;
26 import java.lang.reflect.Proxy;
27 import java.util.ArrayList;
28 import java.util.Iterator;
29 import java.util.List;
30
31 import org.apache.commons.collections.buffer.CircularFifoBuffer;
32 import org.apache.commons.logging.Log;
33 import org.apache.commons.logging.LogFactory;
34
35 import com.google.common.annotations.VisibleForTesting;
36 import com.google.common.collect.Lists;
37
38
39
40
41
42
43 public class TaskMonitor {
44 private static final Log LOG = LogFactory.getLog(TaskMonitor.class);
45
46
47
48 private static final long EXPIRATION_TIME = 60*1000;
49
50 @VisibleForTesting
51 static final int MAX_TASKS = 1000;
52
53 private static TaskMonitor instance;
54 private CircularFifoBuffer tasks = new CircularFifoBuffer(MAX_TASKS);
55
56
57
58
59
60 public static synchronized TaskMonitor get() {
61 if (instance == null) {
62 instance = new TaskMonitor();
63 }
64 return instance;
65 }
66
67 public synchronized MonitoredTask createStatus(String description) {
68 MonitoredTask stat = new MonitoredTaskImpl();
69 stat.setDescription(description);
70 MonitoredTask proxy = (MonitoredTask) Proxy.newProxyInstance(
71 stat.getClass().getClassLoader(),
72 new Class<?>[] { MonitoredTask.class },
73 new PassthroughInvocationHandler<MonitoredTask>(stat));
74 TaskAndWeakRefPair pair = new TaskAndWeakRefPair(stat, proxy);
75 tasks.add(pair);
76 return proxy;
77 }
78
79 public synchronized MonitoredRPCHandler createRPCStatus(String description) {
80 MonitoredRPCHandler stat = new MonitoredRPCHandlerImpl();
81 stat.setDescription(description);
82 MonitoredRPCHandler proxy = (MonitoredRPCHandler) Proxy.newProxyInstance(
83 stat.getClass().getClassLoader(),
84 new Class<?>[] { MonitoredRPCHandler.class },
85 new PassthroughInvocationHandler<MonitoredRPCHandler>(stat));
86 TaskAndWeakRefPair pair = new TaskAndWeakRefPair(stat, proxy);
87 tasks.add(pair);
88 return proxy;
89 }
90
91 private synchronized void purgeExpiredTasks() {
92 for (Iterator<TaskAndWeakRefPair> it = tasks.iterator();
93 it.hasNext();) {
94 TaskAndWeakRefPair pair = it.next();
95 MonitoredTask stat = pair.get();
96
97 if (pair.isDead()) {
98
99
100 if (stat.getState() == MonitoredTaskImpl.State.RUNNING) {
101 LOG.warn("Status " + stat + " appears to have been leaked");
102 stat.cleanup();
103 }
104 }
105
106 if (canPurge(stat)) {
107 it.remove();
108 }
109 }
110 }
111
112
113
114
115
116
117 public synchronized List<MonitoredTask> getTasks() {
118 purgeExpiredTasks();
119 ArrayList<MonitoredTask> ret = Lists.newArrayListWithCapacity(tasks.size());
120 for (Iterator<TaskAndWeakRefPair> it = tasks.iterator();
121 it.hasNext();) {
122 TaskAndWeakRefPair pair = it.next();
123 MonitoredTask t = pair.get();
124 ret.add(t.clone());
125 }
126 return ret;
127 }
128
129 private boolean canPurge(MonitoredTask stat) {
130 long cts = stat.getCompletionTimestamp();
131 return (cts > 0 && System.currentTimeMillis() - cts > EXPIRATION_TIME);
132 }
133
134
135 public void dumpAsText(PrintWriter out) {
136 long now = System.currentTimeMillis();
137
138 List<MonitoredTask> tasks = getTasks();
139 for (MonitoredTask task : tasks) {
140 out.println("Task: " + task.getDescription());
141 out.println("Status: " + task.getState() + ":" + task.getStatus());
142 long running = (now - task.getStartTime())/1000;
143 if (task.getCompletionTimestamp() != -1) {
144 long completed = (now - task.getCompletionTimestamp()) / 1000;
145 out.println("Completed " + completed + "s ago");
146 out.println("Ran for " +
147 (task.getCompletionTimestamp() - task.getStartTime())/1000
148 + "s");
149 } else {
150 out.println("Running for " + running + "s");
151 }
152 out.println();
153 }
154 }
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174 private static class TaskAndWeakRefPair {
175 private MonitoredTask impl;
176 private WeakReference<MonitoredTask> weakProxy;
177
178 public TaskAndWeakRefPair(MonitoredTask stat,
179 MonitoredTask proxy) {
180 this.impl = stat;
181 this.weakProxy = new WeakReference<MonitoredTask>(proxy);
182 }
183
184 public MonitoredTask get() {
185 return impl;
186 }
187
188 public boolean isDead() {
189 return weakProxy.get() == null;
190 }
191 }
192
193
194
195
196
197 private static class PassthroughInvocationHandler<T> implements InvocationHandler {
198 private T delegatee;
199
200 public PassthroughInvocationHandler(T delegatee) {
201 this.delegatee = delegatee;
202 }
203
204 @Override
205 public Object invoke(Object proxy, Method method, Object[] args)
206 throws Throwable {
207 return method.invoke(delegatee, args);
208 }
209 }
210 }