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 import org.apache.hadoop.hbase.classification.InterfaceAudience;
25
26 /**
27 * A simple barrier that can be used by classes that need to wait for some operations to
28 * finish before stopping/closing/etc. forever.
29 */
30 @InterfaceAudience.Private
31 public class DrainBarrier {
32 /**
33 * Contains the number of outstanding operations, as well as flags.
34 * Initially, the number of operations is 1. Each beginOp increments, and endOp decrements it.
35 * beginOp does not proceed when it sees the draining flag. When stop is called, it atomically
36 * decrements the number of operations (the initial 1) and sets the draining flag. If stop did
37 * the decrement to zero, that means there are no more operations outstanding, so stop is done.
38 * Otherwise, stop blocks, and the endOp that decrements the count to 0 unblocks it.
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 * Tries to start an operation.
46 * @return false iff the stop is in progress, and the operation cannot be started.
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 * Ends the operation. Unblocks the blocked caller of stop, if necessary.
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 * Blocks new operations from starting, waits for the current ones to drain.
79 * If someone already called it, returns immediately, which is currently unavoidable as
80 * most of the users stop and close things right and left, and hope for the best.
81 * stopAndWaitForOpsOnce asserts instead.
82 * @throws InterruptedException the wait for operations has been interrupted.
83 */
84 public void stopAndDrainOps() throws InterruptedException {
85 stopAndDrainOps(true);
86 }
87
88 /**
89 * Blocks new operations from starting, waits for the current ones to drain.
90 * Can only be called once.
91 * @throws InterruptedException the wait for operations has been interrupted.
92 */
93 public void stopAndDrainOpsOnce() throws InterruptedException {
94 stopAndDrainOps(false);
95 }
96
97 /**
98 * @param ignoreRepeatedCalls If this is true and somebody already called stop, this method
99 * will return immediately if true; if this is false and somebody
100 * already called stop, it will assert.
101 */
102 // Justification for warnings - wait is not unconditional, and contrary to what WA_NOT_IN_LOOP
103 // description says we are not waiting on multiple conditions.
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; // There were no operations outstanding.
115 synchronized (this) { this.wait(); }
116 }
117
118 // Helper methods.
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); // Not checking for overflow.
129 }
130
131 private static final long dec(long valueAndFlags) {
132 return valueAndFlags - (1 << FLAG_BIT_COUNT); // Negative overflow checked outside.
133 }
134 }