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.regionserver;
21
22 import org.apache.commons.logging.Log;
23 import org.apache.commons.logging.LogFactory;
24 import org.apache.hadoop.hbase.util.HasThread;
25
26 import java.util.ConcurrentModificationException;
27 import java.util.HashMap;
28 import java.util.Map;
29 import java.util.concurrent.Delayed;
30 import java.util.concurrent.DelayQueue;
31 import java.util.concurrent.TimeUnit;
32
33 import java.io.IOException;
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54 public class Leases extends HasThread {
55 private static final Log LOG = LogFactory.getLog(Leases.class.getName());
56 private final int leasePeriod;
57 private final int leaseCheckFrequency;
58 private volatile DelayQueue<Lease> leaseQueue = new DelayQueue<Lease>();
59 protected final Map<String, Lease> leases = new HashMap<String, Lease>();
60 private volatile boolean stopRequested = false;
61
62
63
64
65
66
67
68
69 public Leases(final int leasePeriod, final int leaseCheckFrequency) {
70 this.leasePeriod = leasePeriod;
71 this.leaseCheckFrequency = leaseCheckFrequency;
72 setDaemon(true);
73 }
74
75
76
77
78 @Override
79 public void run() {
80 while (!stopRequested || (stopRequested && leaseQueue.size() > 0) ) {
81 Lease lease = null;
82 try {
83 lease = leaseQueue.poll(leaseCheckFrequency, TimeUnit.MILLISECONDS);
84 } catch (InterruptedException e) {
85 continue;
86 } catch (ConcurrentModificationException e) {
87 continue;
88 } catch (Throwable e) {
89 LOG.fatal("Unexpected exception killed leases thread", e);
90 break;
91 }
92 if (lease == null) {
93 continue;
94 }
95
96
97 if (lease.getListener() == null) {
98 LOG.error("lease listener is null for lease " + lease.getLeaseName());
99 } else {
100 lease.getListener().leaseExpired();
101 }
102 synchronized (leaseQueue) {
103 leases.remove(lease.getLeaseName());
104 }
105 }
106 close();
107 }
108
109
110
111
112
113
114
115
116 public void closeAfterLeasesExpire() {
117 this.stopRequested = true;
118 }
119
120
121
122
123
124 public void close() {
125 LOG.info(Thread.currentThread().getName() + " closing leases");
126 this.stopRequested = true;
127 synchronized (leaseQueue) {
128 leaseQueue.clear();
129 leases.clear();
130 leaseQueue.notifyAll();
131 }
132 LOG.info(Thread.currentThread().getName() + " closed leases");
133 }
134
135
136
137
138
139
140
141
142 public void createLease(String leaseName, final LeaseListener listener)
143 throws LeaseStillHeldException {
144 addLease(new Lease(leaseName, listener));
145 }
146
147
148
149
150
151
152 public void addLease(final Lease lease) throws LeaseStillHeldException {
153 if (this.stopRequested) {
154 return;
155 }
156 lease.setExpirationTime(System.currentTimeMillis() + this.leasePeriod);
157 synchronized (leaseQueue) {
158 if (leases.containsKey(lease.getLeaseName())) {
159 throw new LeaseStillHeldException(lease.getLeaseName());
160 }
161 leases.put(lease.getLeaseName(), lease);
162 leaseQueue.add(lease);
163 }
164 }
165
166
167
168
169
170 @SuppressWarnings("serial")
171 public static class LeaseStillHeldException extends IOException {
172 private final String leaseName;
173
174
175
176
177 public LeaseStillHeldException(final String name) {
178 this.leaseName = name;
179 }
180
181
182 public String getName() {
183 return this.leaseName;
184 }
185 }
186
187
188
189
190
191
192
193 public void renewLease(final String leaseName) throws LeaseException {
194 synchronized (leaseQueue) {
195 Lease lease = leases.get(leaseName);
196
197
198
199 if (lease == null || !leaseQueue.remove(lease)) {
200 throw new LeaseException("lease '" + leaseName +
201 "' does not exist or has already expired");
202 }
203 lease.setExpirationTime(System.currentTimeMillis() + leasePeriod);
204 leaseQueue.add(lease);
205 }
206 }
207
208
209
210
211
212
213 public void cancelLease(final String leaseName) throws LeaseException {
214 removeLease(leaseName);
215 }
216
217
218
219
220
221
222
223
224
225
226 Lease removeLease(final String leaseName) throws LeaseException {
227 Lease lease = null;
228 synchronized (leaseQueue) {
229 lease = leases.remove(leaseName);
230 if (lease == null) {
231 throw new LeaseException("lease '" + leaseName + "' does not exist");
232 }
233 leaseQueue.remove(lease);
234 }
235 return lease;
236 }
237
238
239 static class Lease implements Delayed {
240 private final String leaseName;
241 private final LeaseListener listener;
242 private long expirationTime;
243
244 Lease(final String leaseName, LeaseListener listener) {
245 this(leaseName, listener, 0);
246 }
247
248 Lease(final String leaseName, LeaseListener listener, long expirationTime) {
249 this.leaseName = leaseName;
250 this.listener = listener;
251 this.expirationTime = expirationTime;
252 }
253
254
255 public String getLeaseName() {
256 return leaseName;
257 }
258
259
260 public LeaseListener getListener() {
261 return this.listener;
262 }
263
264 @Override
265 public boolean equals(Object obj) {
266 if (this == obj) {
267 return true;
268 }
269 if (obj == null) {
270 return false;
271 }
272 if (getClass() != obj.getClass()) {
273 return false;
274 }
275 return this.hashCode() == ((Lease) obj).hashCode();
276 }
277
278 @Override
279 public int hashCode() {
280 return this.leaseName.hashCode();
281 }
282
283 public long getDelay(TimeUnit unit) {
284 return unit.convert(this.expirationTime - System.currentTimeMillis(),
285 TimeUnit.MILLISECONDS);
286 }
287
288 public int compareTo(Delayed o) {
289 long delta = this.getDelay(TimeUnit.MILLISECONDS) -
290 o.getDelay(TimeUnit.MILLISECONDS);
291
292 return this.equals(o) ? 0 : (delta > 0 ? 1 : -1);
293 }
294
295
296 public void setExpirationTime(long expirationTime) {
297 this.expirationTime = expirationTime;
298 }
299 }
300 }