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