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.logging.Log;
32 import org.apache.commons.logging.LogFactory;
33
34 import com.google.common.annotations.VisibleForTesting;
35 import com.google.common.collect.Lists;
36
37
38
39
40
41
42 public class TaskMonitor {
43 private static final Log LOG = LogFactory.getLog(TaskMonitor.class);
44
45
46
47 private static final long EXPIRATION_TIME = 60*1000;
48
49 @VisibleForTesting
50 static final int MAX_TASKS = 1000;
51
52 private static TaskMonitor instance;
53 private List<TaskAndWeakRefPair> tasks =
54 Lists.newArrayList();
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 if (tasks.size() > MAX_TASKS) {
77 purgeExpiredTasks();
78 }
79 return proxy;
80 }
81
82 public synchronized MonitoredRPCHandler createRPCStatus(String description) {
83 MonitoredRPCHandler stat = new MonitoredRPCHandlerImpl();
84 stat.setDescription(description);
85 MonitoredRPCHandler proxy = (MonitoredRPCHandler) Proxy.newProxyInstance(
86 stat.getClass().getClassLoader(),
87 new Class<?>[] { MonitoredRPCHandler.class },
88 new PassthroughInvocationHandler<MonitoredRPCHandler>(stat));
89 TaskAndWeakRefPair pair = new TaskAndWeakRefPair(stat, proxy);
90 tasks.add(pair);
91 if (tasks.size() > MAX_TASKS) {
92 purgeExpiredTasks();
93 }
94 return proxy;
95 }
96
97 private synchronized void purgeExpiredTasks() {
98 int size = 0;
99
100 for (Iterator<TaskAndWeakRefPair> it = tasks.iterator();
101 it.hasNext();) {
102 TaskAndWeakRefPair pair = it.next();
103 MonitoredTask stat = pair.get();
104
105 if (pair.isDead()) {
106
107
108 if (stat.getState() == MonitoredTaskImpl.State.RUNNING) {
109 LOG.warn("Status " + stat + " appears to have been leaked");
110 stat.cleanup();
111 }
112 }
113
114 if (canPurge(stat)) {
115 it.remove();
116 } else {
117 size++;
118 }
119 }
120
121 if (size > MAX_TASKS) {
122 LOG.warn("Too many actions in action monitor! Purging some.");
123 tasks = tasks.subList(size - MAX_TASKS, size);
124 }
125 }
126
127
128
129
130
131
132 public synchronized List<MonitoredTask> getTasks() {
133 purgeExpiredTasks();
134 ArrayList<MonitoredTask> ret = Lists.newArrayListWithCapacity(tasks.size());
135 for (TaskAndWeakRefPair pair : tasks) {
136 MonitoredTask t = pair.get();
137 ret.add(t.clone());
138 }
139 return ret;
140 }
141
142 private boolean canPurge(MonitoredTask stat) {
143 long cts = stat.getCompletionTimestamp();
144 return (cts > 0 && System.currentTimeMillis() - cts > EXPIRATION_TIME);
145 }
146
147
148 public void dumpAsText(PrintWriter out) {
149 long now = System.currentTimeMillis();
150
151 List<MonitoredTask> tasks = getTasks();
152 for (MonitoredTask task : tasks) {
153 out.println("Task: " + task.getDescription());
154 out.println("Status: " + task.getState() + ":" + task.getStatus());
155 long running = (now - task.getStartTime())/1000;
156 if (task.getCompletionTimestamp() != -1) {
157 long completed = (now - task.getCompletionTimestamp()) / 1000;
158 out.println("Completed " + completed + "s ago");
159 out.println("Ran for " +
160 (task.getCompletionTimestamp() - task.getStartTime())/1000
161 + "s");
162 } else {
163 out.println("Running for " + running + "s");
164 }
165 out.println();
166 }
167 }
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187 private static class TaskAndWeakRefPair {
188 private MonitoredTask impl;
189 private WeakReference<MonitoredTask> weakProxy;
190
191 public TaskAndWeakRefPair(MonitoredTask stat,
192 MonitoredTask proxy) {
193 this.impl = stat;
194 this.weakProxy = new WeakReference<MonitoredTask>(proxy);
195 }
196
197 public MonitoredTask get() {
198 return impl;
199 }
200
201 public boolean isDead() {
202 return weakProxy.get() == null;
203 }
204 }
205
206
207
208
209
210 private static class PassthroughInvocationHandler<T> implements InvocationHandler {
211 private T delegatee;
212
213 public PassthroughInvocationHandler(T delegatee) {
214 this.delegatee = delegatee;
215 }
216
217 @Override
218 public Object invoke(Object proxy, Method method, Object[] args)
219 throws Throwable {
220 return method.invoke(delegatee, args);
221 }
222 }
223 }