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.util;
21
22 import java.util.concurrent.atomic.AtomicLong;
23
24 import org.apache.hadoop.hbase.classification.InterfaceAudience;
25
26
27
28
29
30 @InterfaceAudience.Private
31 public class DrainBarrier {
32
33
34
35
36
37
38
39
40 private final AtomicLong valueAndFlags = new AtomicLong(inc(0));
41 private final static long DRAINING_FLAG = 0x1;
42 private final static int FLAG_BIT_COUNT = 1;
43
44
45
46
47
48 public boolean beginOp() {
49 long oldValAndFlags;
50 do {
51 oldValAndFlags = valueAndFlags.get();
52 if (isDraining(oldValAndFlags)) return false;
53 } while (!valueAndFlags.compareAndSet(oldValAndFlags, inc(oldValAndFlags)));
54 return true;
55 }
56
57
58
59
60 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
61 justification="First, we do change the state before notify, 2nd, it doesn't even matter")
62 public void endOp() {
63 long oldValAndFlags;
64 do {
65 oldValAndFlags = valueAndFlags.get();
66 long unacceptableCount = isDraining(oldValAndFlags) ? 0 : 1;
67 if (getValue(oldValAndFlags) == unacceptableCount) {
68 throw new AssertionError("endOp called without corresponding beginOp call ("
69 + "the current count is " + unacceptableCount + ")");
70 }
71 } while (!valueAndFlags.compareAndSet(oldValAndFlags, dec(oldValAndFlags)));
72 if (getValue(oldValAndFlags) == 1) {
73 synchronized (this) { this.notifyAll(); }
74 }
75 }
76
77
78
79
80
81
82
83
84 public void stopAndDrainOps() throws InterruptedException {
85 stopAndDrainOps(true);
86 }
87
88
89
90
91
92
93 public void stopAndDrainOpsOnce() throws InterruptedException {
94 stopAndDrainOps(false);
95 }
96
97
98
99
100
101
102
103
104 @edu.umd.cs.findbugs.annotations.SuppressWarnings({"UW_UNCOND_WAIT", "WA_NOT_IN_LOOP"})
105 private void stopAndDrainOps(boolean ignoreRepeatedCalls) throws InterruptedException {
106 long oldValAndFlags;
107 do {
108 oldValAndFlags = valueAndFlags.get();
109 if (isDraining(oldValAndFlags)) {
110 if (ignoreRepeatedCalls) return;
111 throw new AssertionError("stopAndWaitForOpsOnce called more than once");
112 }
113 } while (!valueAndFlags.compareAndSet(oldValAndFlags, dec(oldValAndFlags) | DRAINING_FLAG));
114 if (getValue(oldValAndFlags) == 1) return;
115 synchronized (this) { this.wait(); }
116 }
117
118
119 private static final boolean isDraining(long valueAndFlags) {
120 return (valueAndFlags & DRAINING_FLAG) == DRAINING_FLAG;
121 }
122
123 private static final long getValue(long valueAndFlags) {
124 return valueAndFlags >> FLAG_BIT_COUNT;
125 }
126
127 private static final long inc(long valueAndFlags) {
128 return valueAndFlags + (1 << FLAG_BIT_COUNT);
129 }
130
131 private static final long dec(long valueAndFlags) {
132 return valueAndFlags - (1 << FLAG_BIT_COUNT);
133 }
134 }