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