1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.regionserver;
20
21 import org.apache.commons.logging.Log;
22 import org.apache.commons.logging.LogFactory;
23 import org.apache.hadoop.classification.InterfaceAudience;
24 import org.apache.hadoop.hbase.exceptions.LeaseException;
25 import org.apache.hadoop.hbase.util.HasThread;
26
27 import java.util.ConcurrentModificationException;
28 import java.util.HashMap;
29 import java.util.Map;
30 import java.util.concurrent.Delayed;
31 import java.util.concurrent.DelayQueue;
32 import java.util.concurrent.TimeUnit;
33
34 import java.io.IOException;
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55 @InterfaceAudience.Private
56 public class Leases extends HasThread {
57 private static final Log LOG = LogFactory.getLog(Leases.class.getName());
58 private final int leaseCheckFrequency;
59 private final DelayQueue<Lease> leaseQueue = new DelayQueue<Lease>();
60 protected final Map<String, Lease> leases = new HashMap<String, Lease>();
61 private volatile boolean stopRequested = false;
62
63
64
65
66
67
68
69 public Leases(final int leaseCheckFrequency) {
70 this.leaseCheckFrequency = leaseCheckFrequency;
71 setDaemon(true);
72 }
73
74
75
76
77 @Override
78 public void run() {
79 while (!stopRequested || (stopRequested && leaseQueue.size() > 0) ) {
80 Lease lease = null;
81 try {
82 lease = leaseQueue.poll(leaseCheckFrequency, TimeUnit.MILLISECONDS);
83 } catch (InterruptedException e) {
84 continue;
85 } catch (ConcurrentModificationException e) {
86 continue;
87 } catch (Throwable e) {
88 LOG.fatal("Unexpected exception killed leases thread", e);
89 break;
90 }
91 if (lease == null) {
92 continue;
93 }
94
95
96 if (lease.getListener() == null) {
97 LOG.error("lease listener is null for lease " + lease.getLeaseName());
98 } else {
99 lease.getListener().leaseExpired();
100 }
101 synchronized (leaseQueue) {
102 leases.remove(lease.getLeaseName());
103 }
104 }
105 close();
106 }
107
108
109
110
111
112
113
114
115 public void closeAfterLeasesExpire() {
116 this.stopRequested = true;
117 }
118
119
120
121
122
123 public void close() {
124 LOG.info(Thread.currentThread().getName() + " closing leases");
125 this.stopRequested = true;
126 synchronized (leaseQueue) {
127 leaseQueue.clear();
128 leases.clear();
129 leaseQueue.notifyAll();
130 }
131 LOG.info(Thread.currentThread().getName() + " closed leases");
132 }
133
134
135
136
137
138
139
140
141
142 public void createLease(String leaseName, int leaseTimeoutPeriod, final LeaseListener listener)
143 throws LeaseStillHeldException {
144 addLease(new Lease(leaseName, leaseTimeoutPeriod, 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.resetExpirationTime();
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.resetExpirationTime();
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 int leaseTimeoutPeriod;
243 private long expirationTime;
244
245 Lease(final String leaseName, int leaseTimeoutPeriod, LeaseListener listener) {
246 this.leaseName = leaseName;
247 this.listener = listener;
248 this.leaseTimeoutPeriod = leaseTimeoutPeriod;
249 this.expirationTime = 0;
250 }
251
252
253 public String getLeaseName() {
254 return leaseName;
255 }
256
257
258 public LeaseListener getListener() {
259 return this.listener;
260 }
261
262 @Override
263 public boolean equals(Object obj) {
264 if (this == obj) {
265 return true;
266 }
267 if (obj == null) {
268 return false;
269 }
270 if (getClass() != obj.getClass()) {
271 return false;
272 }
273 return this.hashCode() == obj.hashCode();
274 }
275
276 @Override
277 public int hashCode() {
278 return this.leaseName.hashCode();
279 }
280
281 public long getDelay(TimeUnit unit) {
282 return unit.convert(this.expirationTime - System.currentTimeMillis(),
283 TimeUnit.MILLISECONDS);
284 }
285
286 public int compareTo(Delayed o) {
287 long delta = this.getDelay(TimeUnit.MILLISECONDS) -
288 o.getDelay(TimeUnit.MILLISECONDS);
289
290 return this.equals(o) ? 0 : (delta > 0 ? 1 : -1);
291 }
292
293
294
295
296 public void resetExpirationTime() {
297 this.expirationTime = System.currentTimeMillis() + this.leaseTimeoutPeriod;
298 }
299 }
300 }