You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by li...@apache.org on 2013/06/04 20:20:56 UTC

svn commit: r1489543 - /hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/SizeBasedThrottler.java

Author: liyin
Date: Tue Jun  4 18:20:55 2013
New Revision: 1489543

URL: http://svn.apache.org/r1489543
Log:
[HBASE-6728] [89-fb] Improvements to SizeBasedThrottler

Author: adela

Summary:
- current implementation had all methods syncrhonized - which means
  synchronization overhead was always incurred - and in contrary to
  comments in code. Removing all synchronized modifiers from methods
  actually does what comments explain (and was probably initially
  intended)
- further - removed actual race condition (which happens only after we
  remove syncrhonized modifiers)

Test Plan: run TestSizeBasedThrottler

Reviewers: aaiyer, liyintang

Reviewed By: liyintang

CC: manukranthk, hbase-eng@, rshroff, shaneh

Differential Revision: https://phabricator.fb.com/D778193

Modified:
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/SizeBasedThrottler.java

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/SizeBasedThrottler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/SizeBasedThrottler.java?rev=1489543&r1=1489542&r2=1489543&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/SizeBasedThrottler.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/SizeBasedThrottler.java Tue Jun  4 18:20:55 2013
@@ -27,9 +27,7 @@ import java.util.concurrent.atomic.Atomi
  *     queues - you can submit object that is bigger than limit.
  *
  * This implementation introduces small costs in terms of
- * synchronization (no synchronization in most cases at all), but is
- * vulnerable to races. For details see documentation of
- * increase method.
+ * synchronization (no synchronization in most cases at all).
  */
 public class SizeBasedThrottler {
 
@@ -53,29 +51,28 @@ public class SizeBasedThrottler {
    * Blocks until internal counter is lower than threshold
    * and then increases value of internal counter.
    *
-   * THIS METHOD IS VULNERABLE TO RACES.
-   * It may happen that increment operation will
-   * succeed immediately, even if it should block. This happens when
-   * at least two threads call increase at the some moment. The decision
-   * whether to block is made at the beginning, without synchronization.
-   * If value of currentSize is lower than threshold at that time, call
-   * will succeed immediately. It is possible, that 2 threads will make
-   * decision not to block, even if one of them should block.
+   * Note that order in which increases will be granted is not guaranteed,
+   * i.e. some request can wait, while newer request are granted in the meantime
    *
    * @param delta increase internal counter by this value
    * @return new value of internal counter
    * @throws InterruptedException when interrupted during waiting
    */
-  public synchronized long increase(long delta) throws InterruptedException{
-    if (currentSize.get() >= threshold) {
-      synchronized (this) {
-        while (currentSize.get() >= threshold) {
-          wait();
+  public long increase(long delta) throws InterruptedException{
+    while (true) {
+      long size = currentSize.get();
+      if (size >= threshold) {
+        synchronized (this) {
+          while ((size = currentSize.get()) >= threshold) {
+            wait();
+          }
         }
       }
-    }
 
-    return currentSize.addAndGet(delta);
+      if (currentSize.compareAndSet(size, size + delta)) {
+        return size + delta;
+      }
+    }
   }
 
 
@@ -85,7 +82,7 @@ public class SizeBasedThrottler {
    * @param delta decrease internal counter by this value
    * @return new value of internal counter
    */
-  public synchronized long decrease(long delta) {
+  public long decrease(long delta) {
     final long newSize = currentSize.addAndGet(-delta);
 
     if (newSize < threshold && newSize + delta >= threshold) {
@@ -101,7 +98,7 @@ public class SizeBasedThrottler {
    *
    * @return current value of internal counter
    */
-  public synchronized long getCurrentValue(){
+  public long getCurrentValue(){
     return currentSize.get();
   }