You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jb...@apache.org on 2009/09/18 21:02:59 UTC

svn commit: r816747 - /incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java

Author: jbellis
Date: Fri Sep 18 19:02:58 2009
New Revision: 816747

URL: http://svn.apache.org/viewvc?rev=816747&view=rev
Log:
r/m memtableLock; it is redundant wrt flusherLock
patch by jbellis; reviewed by junrao for CASSANDRA-444

Modified:
    incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java?rev=816747&r1=816746&r2=816747&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java Fri Sep 18 19:02:58 2009
@@ -68,11 +68,6 @@
 
     /* active memtable associated with this ColumnFamilyStore. */
     private Memtable memtable_;
-    // this lock is to
-    // make sure we don't perform puts on a memtable that is queued for flush.
-    // (or conversely, flush a memtable that is mid-put.)
-    // gets may be safely performed on a flushing ("frozen") memtable.
-    private ReentrantReadWriteLock memtableLock_ = new ReentrantReadWriteLock(true);
 
     // TODO binarymemtable ops are not threadsafe (do they need to be?)
     private AtomicReference<BinaryMemtable> binaryMemtable_;
@@ -406,34 +401,20 @@
 
     /**
      * Insert/Update the column family for this key.
+     * Caller is responsible for acquiring Table.flusherLock!
      * param @ lock - lock that needs to be used.
      * param @ key - key for update/insert
      * param @ columnFamily - columnFamily changes
      */
-    Memtable apply(String key, ColumnFamily columnFamily)
-            throws IOException
+    Memtable apply(String key, ColumnFamily columnFamily) throws IOException
     {
         long start = System.currentTimeMillis();
-        Memtable initialMemtable = getMemtableThreadSafe();
-        boolean isFlush = false;
-        
-        if (initialMemtable.isThresholdViolated())
-        {
-            isFlush = true;
-        }
 
-        memtableLock_.readLock().lock();
-        try
-        {
-            initialMemtable.put(key, columnFamily);
-        }
-        finally
-        {
-            memtableLock_.readLock().unlock();
-        }
+        boolean flushRequested = memtable_.isThresholdViolated();
+        memtable_.put(key, columnFamily);
         writeStats_.add(System.currentTimeMillis() - start);
         
-        return isFlush ? initialMemtable : null;
+        return flushRequested ? memtable_ : null;
     }
 
     /*
@@ -977,27 +958,27 @@
      */
     private Memtable getMemtableThreadSafe()
     {
-        memtableLock_.readLock().lock();
+        Table.flusherLock_.readLock().lock();
         try
         {
             return memtable_;
         }
         finally
         {
-            memtableLock_.readLock().unlock();
+            Table.flusherLock_.readLock().unlock();
         }
     }
 
     public Iterator<String> memtableKeyIterator() throws ExecutionException, InterruptedException
     {
-        memtableLock_.readLock().lock();
+        Table.flusherLock_.readLock().lock();
         try
         {
              return memtable_.getKeyIterator();
         }
         finally
         {
-            memtableLock_.readLock().unlock();
+            Table.flusherLock_.readLock().unlock();
         }
     }
 
@@ -1016,10 +997,11 @@
     {
         return readStats_.mean();
     }
-    
+
+    // TODO this actually isn't a good meature of pending tasks
     public int getPendingTasks()
     {
-        return memtableLock_.getQueueLength();
+        return Table.flusherLock_.getQueueLength();
     }
 
     /**
@@ -1082,7 +1064,7 @@
             ColumnIterator iter;
         
             /* add the current memtable */
-            memtableLock_.readLock().lock();
+            Table.flusherLock_.readLock().lock();
             try
             {
                 iter = filter.getMemColumnIterator(memtable_, getComparator());
@@ -1090,7 +1072,7 @@
             }
             finally
             {
-                memtableLock_.readLock().unlock();            
+                Table.flusherLock_.readLock().unlock();
             }        
             iterators.add(iter);