diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/DrainBarrier.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/DrainBarrier.java index 2c42acc..1ee142f 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/DrainBarrier.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/DrainBarrier.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.util; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.hbase.classification.InterfaceAudience; @@ -42,6 +43,11 @@ public class DrainBarrier { private final static int FLAG_BIT_COUNT = 1; /** + * Blocks until the count of outstanding operations becomes zero under draining. + */ + private final CountDownLatch drainLatch = new CountDownLatch(1); + + /** * Tries to start an operation. * @return false iff the stop is in progress, and the operation cannot be started. */ @@ -57,8 +63,6 @@ public class DrainBarrier { /** * Ends the operation. Unblocks the blocked caller of stop, if necessary. */ - @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY", - justification="First, we do change the state before notify, 2nd, it doesn't even matter") public void endOp() { long oldValAndFlags; do { @@ -70,7 +74,7 @@ public class DrainBarrier { } } while (!valueAndFlags.compareAndSet(oldValAndFlags, dec(oldValAndFlags))); if (getValue(oldValAndFlags) == 1) { - synchronized (this) { this.notifyAll(); } + drainLatch.countDown(); } } @@ -99,9 +103,6 @@ public class DrainBarrier { * will return immediately if true; if this is false and somebody * already called stop, it will assert. */ - // Justification for warnings - wait is not unconditional, and contrary to what WA_NOT_IN_LOOP - // description says we are not waiting on multiple conditions. - @edu.umd.cs.findbugs.annotations.SuppressWarnings({"UW_UNCOND_WAIT", "WA_NOT_IN_LOOP"}) private void stopAndDrainOps(boolean ignoreRepeatedCalls) throws InterruptedException { long oldValAndFlags; do { @@ -112,7 +113,7 @@ public class DrainBarrier { } } while (!valueAndFlags.compareAndSet(oldValAndFlags, dec(oldValAndFlags) | DRAINING_FLAG)); if (getValue(oldValAndFlags) == 1) return; // There were no operations outstanding. - synchronized (this) { this.wait(); } + drainLatch.await(); } // Helper methods. diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestDrainBarrier.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestDrainBarrier.java index 4542cbd..6099f74 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestDrainBarrier.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestDrainBarrier.java @@ -43,12 +43,16 @@ public class TestDrainBarrier { @Test public void testUnmatchedEndAssert() throws Exception { + boolean assertionErrorThrown; + DrainBarrier barrier = new DrainBarrier(); try { barrier.endOp(); - fail("Should have asserted"); + assertionErrorThrown = false; } catch (AssertionError e) { + assertionErrorThrown = true; } + assertTrue("Should have asserted", assertionErrorThrown); barrier.beginOp(); barrier.beginOp(); @@ -56,9 +60,11 @@ public class TestDrainBarrier { barrier.endOp(); try { barrier.endOp(); - fail("Should have asserted"); + assertionErrorThrown = false; } catch (AssertionError e) { + assertionErrorThrown = true; } + assertTrue("Should have asserted", assertionErrorThrown); } @Test @@ -104,13 +110,17 @@ public class TestDrainBarrier { @Test public void testMultipleStopOnceAssert() throws Exception { + boolean assertionErrorThrown; + DrainBarrier barrier = new DrainBarrier(); barrier.stopAndDrainOpsOnce(); try { barrier.stopAndDrainOpsOnce(); - fail("Should have asserted"); + assertionErrorThrown = false; } catch (AssertionError e) { + assertionErrorThrown = true; } + assertTrue("Should have asserted", assertionErrorThrown); } @Test