1 /**
2 *
3 * Licensed to the Apache Software Foundation (ASF) under one
4 * or more contributor license agreements. See the NOTICE file
5 * distributed with this work for additional information
6 * regarding copyright ownership. The ASF licenses this file
7 * to you under the Apache License, Version 2.0 (the
8 * "License"); you may not use this file except in compliance
9 * with the License. You may obtain a copy of the License at
10 *
11 * http://www.apache.org/licenses/LICENSE-2.0
12 *
13 * Unless required by applicable law or agreed to in writing, software
14 * distributed under the License is distributed on an "AS IS" BASIS,
15 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16 * See the License for the specific language governing permissions and
17 * limitations under the License.
18 */
19
20 package org.apache.hadoop.hbase.util;
21
22 import java.util.concurrent.atomic.AtomicLong;
23
24 /**
25 * A simple barrier that can be used by classes that need to wait for some operations to
26 * finish before stopping/closing/etc. forever.
27 */
28 public class DrainBarrier {
29 /**
30 * Contains the number of outstanding operations, as well as flags.
31 * Initially, the number of operations is 1. Each beginOp increments, and endOp decrements it.
32 * beginOp does not proceed when it sees the draining flag. When stop is called, it atomically
33 * decrements the number of operations (the initial 1) and sets the draining flag. If stop did
34 * the decrement to zero, that means there are no more operations outstanding, so stop is done.
35 * Otherwise, stop blocks, and the endOp that decrements the count to 0 unblocks it.
36 */
37 private final AtomicLong valueAndFlags = new AtomicLong(inc(0));
38 private final static long DRAINING_FLAG = 0x1;
39 private final static int FLAG_BIT_COUNT = 1;
40
41 /**
42 * Tries to start an operation.
43 * @return false iff the stop is in progress, and the operation cannot be started.
44 */
45 public boolean beginOp() {
46 long oldValAndFlags;
47 do {
48 oldValAndFlags = valueAndFlags.get();
49 if (isDraining(oldValAndFlags)) return false;
50 } while (!valueAndFlags.compareAndSet(oldValAndFlags, inc(oldValAndFlags)));
51 return true;
52 }
53
54 /**
55 * Ends the operation. Unblocks the blocked caller of stop, if necessary.
56 */
57 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
58 justification="First, we do change the state before notify, 2nd, it doesn't even matter")
59 public void endOp() {
60 long oldValAndFlags;
61 do {
62 oldValAndFlags = valueAndFlags.get();
63 long unacceptableCount = isDraining(oldValAndFlags) ? 0 : 1;
64 if (getValue(oldValAndFlags) == unacceptableCount) {
65 throw new AssertionError("endOp called without corresponding beginOp call ("
66 + "the current count is " + unacceptableCount + ")");
67 }
68 } while (!valueAndFlags.compareAndSet(oldValAndFlags, dec(oldValAndFlags)));
69 if (getValue(oldValAndFlags) == 1) {
70 synchronized (this) { this.notifyAll(); }
71 }
72 }
73
74 /**
75 * Blocks new operations from starting, waits for the current ones to drain.
76 * If someone already called it, returns immediately, which is currently unavoidable as
77 * most of the users stop and close things right and left, and hope for the best.
78 * stopAndWaitForOpsOnce asserts instead.
79 * @throws InterruptedException the wait for operations has been interrupted.
80 */
81 public void stopAndDrainOps() throws InterruptedException {
82 stopAndDrainOps(true);
83 }
84
85 /**
86 * Blocks new operations from starting, waits for the current ones to drain.
87 * Can only be called once.
88 * @throws InterruptedException the wait for operations has been interrupted.
89 */
90 public void stopAndDrainOpsOnce() throws InterruptedException {
91 stopAndDrainOps(false);
92 }
93
94 /**
95 * @param ignoreRepeatedCalls If this is true and somebody already called stop, this method
96 * will return immediately if true; if this is false and somebody
97 * already called stop, it will assert.
98 */
99 // Justification for warnings - wait is not unconditional, and contrary to what WA_NOT_IN_LOOP
100 // description says we are not waiting on multiple conditions.
101 @edu.umd.cs.findbugs.annotations.SuppressWarnings({"UW_UNCOND_WAIT", "WA_NOT_IN_LOOP"})
102 private void stopAndDrainOps(boolean ignoreRepeatedCalls) throws InterruptedException {
103 long oldValAndFlags;
104 do {
105 oldValAndFlags = valueAndFlags.get();
106 if (isDraining(oldValAndFlags)) {
107 if (ignoreRepeatedCalls) return;
108 throw new AssertionError("stopAndWaitForOpsOnce called more than once");
109 }
110 } while (!valueAndFlags.compareAndSet(oldValAndFlags, dec(oldValAndFlags) | DRAINING_FLAG));
111 if (getValue(oldValAndFlags) == 1) return; // There were no operations outstanding.
112 synchronized (this) { this.wait(); }
113 }
114
115 // Helper methods.
116 private static final boolean isDraining(long valueAndFlags) {
117 return (valueAndFlags & DRAINING_FLAG) == DRAINING_FLAG;
118 }
119
120 private static final long getValue(long valueAndFlags) {
121 return valueAndFlags >> FLAG_BIT_COUNT;
122 }
123
124 private static final long inc(long valueAndFlags) {
125 return valueAndFlags + (1 << FLAG_BIT_COUNT); // Not checking for overflow.
126 }
127
128 private static final long dec(long valueAndFlags) {
129 return valueAndFlags - (1 << FLAG_BIT_COUNT); // Negative overflow checked outside.
130 }
131 }