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