You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2011/10/20 20:24:55 UTC

svn commit: r1186979 - in /hbase/branches/0.92: CHANGES.txt src/main/java/org/apache/hadoop/hbase/io/hfile/slab/SingleSizeCache.java src/main/java/org/apache/hadoop/hbase/io/hfile/slab/SlabCache.java

Author: stack
Date: Thu Oct 20 18:24:54 2011
New Revision: 1186979

URL: http://svn.apache.org/viewvc?rev=1186979&view=rev
Log:
HBASE-4430 Disable TestSlabCache and TestSingleSizedCache temporarily to see if these are cause of build box failure though all tests pass

Modified:
    hbase/branches/0.92/CHANGES.txt
    hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/io/hfile/slab/SingleSizeCache.java
    hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/io/hfile/slab/SlabCache.java

Modified: hbase/branches/0.92/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/CHANGES.txt?rev=1186979&r1=1186978&r2=1186979&view=diff
==============================================================================
--- hbase/branches/0.92/CHANGES.txt (original)
+++ hbase/branches/0.92/CHANGES.txt Thu Oct 20 18:24:54 2011
@@ -350,6 +350,9 @@ Release 0.92.0 - Unreleased
    HBASE-4378  [hbck] Does not complain about regions with startkey==endkey.
                (Jonathan Hsieh)
    HBASE-4459  HbaseObjectWritable code is a byte, we will eventually run out of codes
+   HBASE-4430  Disable TestSlabCache and TestSingleSizedCache temporarily to
+               see if these are cause of build box failure though all tests pass
+               (Li Pi)
 
   TESTS
    HBASE-4492  TestRollingRestart fails intermittently

Modified: hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/io/hfile/slab/SingleSizeCache.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/io/hfile/slab/SingleSizeCache.java?rev=1186979&r1=1186978&r2=1186979&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/io/hfile/slab/SingleSizeCache.java (original)
+++ hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/io/hfile/slab/SingleSizeCache.java Thu Oct 20 18:24:54 2011
@@ -57,7 +57,7 @@ public class SingleSizeCache implements 
   private final int numBlocks;
   private final int blockSize;
   private final CacheStats stats;
-  private final SlabItemEvictionWatcher evictionWatcher;
+  private final SlabItemActionWatcher actionWatcher;
   private final AtomicLong size;
   private final AtomicLong timeSinceLastAccess;
   public final static long CACHE_FIXED_OVERHEAD = ClassSize
@@ -78,12 +78,12 @@ public class SingleSizeCache implements 
    * @param master the SlabCache this SingleSlabCache is assigned to.
    */
   public SingleSizeCache(int blockSize, int numBlocks,
-      SlabItemEvictionWatcher master) {
+      SlabItemActionWatcher master) {
     this.blockSize = blockSize;
     this.numBlocks = numBlocks;
     backingStore = new Slab(blockSize, numBlocks);
     this.stats = new CacheStats();
-    this.evictionWatcher = master;
+    this.actionWatcher = master;
     this.size = new AtomicLong(CACHE_FIXED_OVERHEAD + backingStore.heapSize());
     this.timeSinceLastAccess = new AtomicLong();
 
@@ -121,11 +121,17 @@ public class SingleSizeCache implements 
         storedBlock);
     toBeCached.serialize(storedBlock);
 
-    CacheablePair alreadyCached = backingMap.putIfAbsent(blockName, newEntry);
-
-    if (alreadyCached != null) {
-      backingStore.free(storedBlock);
-      throw new RuntimeException("already cached " + blockName);
+    synchronized (this) {
+      CacheablePair alreadyCached = backingMap.putIfAbsent(blockName, newEntry);
+    
+
+      if (alreadyCached != null) {
+        backingStore.free(storedBlock);
+        throw new RuntimeException("already cached " + blockName);
+      }
+      if (actionWatcher != null) {
+        actionWatcher.onInsertion(blockName, this);
+      }
     }
     newEntry.recentlyAccessed.set(System.nanoTime());
     this.size.addAndGet(newEntry.heapSize());
@@ -198,8 +204,8 @@ public class SingleSizeCache implements 
       // Thread A calls cacheBlock on the same block, and gets
       // "already cached" since the block is still in backingStore
 
-      if (evictionWatcher != null) {
-        evictionWatcher.onEviction(key, this);
+      if (actionWatcher != null) {
+        actionWatcher.onEviction(key, this);
       }
     }
     stats.evicted();

Modified: hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/io/hfile/slab/SlabCache.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/io/hfile/slab/SlabCache.java?rev=1186979&r1=1186978&r2=1186979&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/io/hfile/slab/SlabCache.java (original)
+++ hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/io/hfile/slab/SlabCache.java Thu Oct 20 18:24:54 2011
@@ -50,7 +50,7 @@ import com.google.common.util.concurrent
  * correct SingleSizeCache.
  *
  **/
-public class SlabCache implements SlabItemEvictionWatcher, BlockCache, HeapSize {
+public class SlabCache implements SlabItemActionWatcher, BlockCache, HeapSize {
 
   private final ConcurrentHashMap<String, SingleSizeCache> backingStore;
   private final TreeMap<Integer, SingleSizeCache> sizer;
@@ -212,34 +212,7 @@ public class SlabCache implements SlabIt
      * twice
      */
     scache.cacheBlock(blockName, cachedItem);
-
-    /*
-     * If an eviction for this value hasn't taken place yet, we want to wait for
-     * it to take place. See HBase-4330.
-     */
-    SingleSizeCache replace;
-    while ((replace = backingStore.putIfAbsent(blockName, scache)) != null) {
-      synchronized (replace) {
-        /*
-         * With the exception of unit tests, this should happen extremely
-         * rarely.
-         */
-        try {
-          replace.wait();
-        } catch (InterruptedException e) {
-          LOG.warn("InterruptedException on the caching thread: " + e);
-        }
-      }
-    }
-
-    /*
-     * Let the eviction threads know that something has been cached, and let
-     * them try their hand at eviction
-     */
-    synchronized (scache) {
-      scache.notifyAll();
-    }
-  }
+  } 
 
   /**
    * We don't care about whether its in memory or not, so we just pass the call
@@ -291,60 +264,14 @@ public class SlabCache implements SlabIt
   }
 
   @Override
-  public void onEviction(String key, Object notifier) {
-    /*
-     * Without the while loop below, the following can occur:
-     *
-     * Invariant: Anything in SingleSizeCache will have a representation in
-     * SlabCache, and vice-versa.
-     *
-     * Start: Key A is in both SingleSizeCache and SlabCache. Invariant is
-     * satisfied
-     *
-     * Thread A: Caches something, starting eviction of Key A in SingleSizeCache
-     *
-     * Thread B: Checks for Key A -> Returns Gets Null, as eviction has begun
-     *
-     * Thread B: Recaches Key A, gets to SingleSizeCache, does not get the
-     * PutIfAbsentLoop yet...
-     *
-     * Thread C: Caches another key, starting the second eviction of Key A.
-     *
-     * Thread A: does its onEviction, removing the entry of Key A from
-     * SlabCache.
-     *
-     * Thread C: does its onEviction, removing the (blank) entry of Key A from
-     * SlabCache:
-     *
-     * Thread B: goes to putifabsent, and puts its entry into SlabCache.
-     *
-     * Result: SlabCache has an entry for A, while SingleSizeCache has no
-     * entries for A. Invariant is violated.
-     *
-     * What the while loop does, is that, at the end, it GUARANTEES that an
-     * onEviction will remove an entry. See HBase-4482.
-     */
-
-    stats.evict();
-    while ((backingStore.remove(key)) == null) {
-      /* With the exception of unit tests, this should happen extremely rarely. */
-      synchronized (notifier) {
-        try {
-          notifier.wait();
-        } catch (InterruptedException e) {
-          LOG.warn("InterruptedException on the evicting thread: " + e);
-        }
-      }
-    }
+  public void onEviction(String key, SingleSizeCache notifier) {
     stats.evicted();
-
-    /*
-     * Now we've evicted something, lets tell the caching threads to try to
-     * cache something.
-     */
-    synchronized (notifier) {
-      notifier.notifyAll();
-    }
+    backingStore.remove(key);
+  }
+  
+  @Override
+  public void onInsertion(String key, SingleSizeCache notifier) {
+    backingStore.put(key, notifier);
   }
 
   /**