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 java.text.SimpleDateFormat;
22 import java.util.Date;
23 import java.util.Map;
24 import java.util.concurrent.ConcurrentHashMap;
25
26 import org.apache.commons.logging.Log;
27 import org.apache.commons.logging.LogFactory;
28 import org.apache.hadoop.conf.Configuration;
29 import org.apache.hadoop.hbase.HConstants;
30 import org.apache.hadoop.hbase.ScheduledChore;
31 import org.apache.hadoop.hbase.Stoppable;
32 import org.apache.hadoop.hbase.classification.InterfaceAudience;
33 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
34 import org.apache.hadoop.hbase.util.NonceKey;
35
36
37
38
39
40 @InterfaceAudience.Private
41 public class ServerNonceManager {
42 public static final String HASH_NONCE_GRACE_PERIOD_KEY = "hbase.server.hashNonce.gracePeriod";
43 private static final Log LOG = LogFactory.getLog(ServerNonceManager.class);
44
45
46
47 private int conflictWaitIterationMs = 30000;
48
49 private static final SimpleDateFormat tsFormat = new SimpleDateFormat("HH:mm:ss.SSS");
50
51
52 private static class OperationContext {
53 static final int DONT_PROCEED = 0;
54 static final int PROCEED = 1;
55 static final int WAIT = 2;
56
57
58 private long data = 0;
59 private static final long STATE_BITS = 3;
60 private static final long WAITING_BIT = 4;
61 private static final long ALL_FLAG_BITS = WAITING_BIT | STATE_BITS;
62
63 private volatile long mvcc;
64
65 @Override
66 public String toString() {
67 return "[state " + getState() + ", hasWait " + hasWait() + ", activity "
68 + tsFormat.format(new Date(getActivityTime())) + "]";
69 }
70
71 public OperationContext() {
72 setState(WAIT);
73 reportActivity();
74 }
75
76 public void setState(int state) {
77 this.data = (this.data & ~STATE_BITS) | state;
78 }
79
80 public int getState() {
81 return (int)(this.data & STATE_BITS);
82 }
83
84 public void setHasWait() {
85 this.data = this.data | WAITING_BIT;
86 }
87
88 public boolean hasWait() {
89 return (this.data & WAITING_BIT) == WAITING_BIT;
90 }
91
92 public void reportActivity() {
93 long now = EnvironmentEdgeManager.currentTime();
94 this.data = (this.data & ALL_FLAG_BITS) | (now << 3);
95 }
96
97 public boolean isExpired(long minRelevantTime) {
98 return getActivityTime() < (minRelevantTime & (~0l >>> 3));
99 }
100
101 private long getActivityTime() {
102 return this.data >>> 3;
103 }
104
105 public void setMvcc(long mvcc) {
106 this.mvcc = mvcc;
107 }
108
109 public long getMvcc() {
110 return this.mvcc;
111 }
112 }
113
114
115
116
117
118
119
120
121
122 private ConcurrentHashMap<NonceKey, OperationContext> nonces =
123 new ConcurrentHashMap<NonceKey, OperationContext>();
124
125 private int deleteNonceGracePeriod;
126
127 public ServerNonceManager(Configuration conf) {
128
129 deleteNonceGracePeriod = conf.getInt(HASH_NONCE_GRACE_PERIOD_KEY, 30 * 60 * 1000);
130 if (deleteNonceGracePeriod < 60 * 1000) {
131 LOG.warn("Nonce grace period " + deleteNonceGracePeriod
132 + " is less than a minute; might be too small to be useful");
133 }
134 }
135
136 public void setConflictWaitIterationMs(int conflictWaitIterationMs) {
137 this.conflictWaitIterationMs = conflictWaitIterationMs;
138 }
139
140
141
142
143
144
145
146
147
148 public boolean startOperation(long group, long nonce, Stoppable stoppable)
149 throws InterruptedException {
150 if (nonce == HConstants.NO_NONCE) return true;
151 NonceKey nk = new NonceKey(group, nonce);
152 OperationContext ctx = new OperationContext();
153 while (true) {
154 OperationContext oldResult = nonces.putIfAbsent(nk, ctx);
155 if (oldResult == null) return true;
156
157
158 synchronized (oldResult) {
159 int oldState = oldResult.getState();
160 LOG.debug("Conflict detected by nonce: " + nk + ", " + oldResult);
161 if (oldState != OperationContext.WAIT) {
162 return oldState == OperationContext.PROCEED;
163 }
164 oldResult.setHasWait();
165 oldResult.wait(this.conflictWaitIterationMs);
166 if (stoppable.isStopped()) {
167 throw new InterruptedException("Server stopped");
168 }
169 }
170 }
171 }
172
173
174
175
176
177
178
179 public void endOperation(long group, long nonce, boolean success) {
180 if (nonce == HConstants.NO_NONCE) return;
181 NonceKey nk = new NonceKey(group, nonce);
182 OperationContext newResult = nonces.get(nk);
183 assert newResult != null;
184 synchronized (newResult) {
185 assert newResult.getState() == OperationContext.WAIT;
186
187 newResult.setState(success ? OperationContext.DONT_PROCEED : OperationContext.PROCEED);
188 if (success) {
189 newResult.reportActivity();
190 } else {
191 OperationContext val = nonces.remove(nk);
192 assert val == newResult;
193 }
194 if (newResult.hasWait()) {
195 LOG.debug("Conflict with running op ended: " + nk + ", " + newResult);
196 newResult.notifyAll();
197 }
198 }
199 }
200
201
202
203
204
205
206
207 public void addMvccToOperationContext(long group, long nonce, long mvcc) {
208 if (nonce == HConstants.NO_NONCE) {
209 return;
210 }
211 NonceKey nk = new NonceKey(group, nonce);
212 OperationContext result = nonces.get(nk);
213 assert result != null;
214 synchronized (result) {
215 result.setMvcc(mvcc);
216 }
217 }
218
219
220
221
222
223
224
225 public long getMvccFromOperationContext(long group, long nonce) {
226 if (nonce == HConstants.NO_NONCE) {
227 return Long.MAX_VALUE;
228 }
229 NonceKey nk = new NonceKey(group, nonce);
230 OperationContext result = nonces.get(nk);
231 return result == null ? Long.MAX_VALUE : result.getMvcc();
232 }
233
234
235
236
237
238
239
240 public void reportOperationFromWal(long group, long nonce, long writeTime) {
241 if (nonce == HConstants.NO_NONCE) return;
242
243 long now = EnvironmentEdgeManager.currentTime();
244 if (now > writeTime + (deleteNonceGracePeriod * 1.5)) return;
245 OperationContext newResult = new OperationContext();
246 newResult.setState(OperationContext.DONT_PROCEED);
247 NonceKey nk = new NonceKey(group, nonce);
248 OperationContext oldResult = nonces.putIfAbsent(nk, newResult);
249 if (oldResult != null) {
250
251
252 LOG.warn("Nonce collision during WAL recovery: " + nk
253 + ", " + oldResult + " with " + newResult);
254 }
255 }
256
257
258
259
260
261
262 public ScheduledChore createCleanupScheduledChore(Stoppable stoppable) {
263
264 return new ScheduledChore("nonceCleaner", stoppable, deleteNonceGracePeriod / 5) {
265 @Override
266 protected void chore() {
267 cleanUpOldNonces();
268 }
269 };
270 }
271
272 private void cleanUpOldNonces() {
273 long cutoff = EnvironmentEdgeManager.currentTime() - deleteNonceGracePeriod;
274 for (Map.Entry<NonceKey, OperationContext> entry : nonces.entrySet()) {
275 OperationContext oc = entry.getValue();
276 if (!oc.isExpired(cutoff)) continue;
277 synchronized (oc) {
278 if (oc.getState() == OperationContext.WAIT || !oc.isExpired(cutoff)) continue;
279 nonces.remove(entry.getKey());
280 }
281 }
282 }
283 }