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/05/13 04:17:23 UTC

svn commit: r774165 - /incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Memtable.java

Author: jbellis
Date: Wed May 13 02:17:23 2009
New Revision: 774165

URL: http://svn.apache.org/viewvc?rev=774165&view=rev
Log:
to avoid adding work to terminating executor (which will raise an exception) we need to check for threshold violated _first_, and move puts to the new memtable once flush has started.  patch by jbellis; reviewed by Eric Evans for CASSANDRA-165

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

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Memtable.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Memtable.java?rev=774165&r1=774164&r2=774165&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Memtable.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Memtable.java Wed May 13 02:17:23 2009
@@ -194,14 +194,19 @@
      * the memtable. This version will respect the threshold and flush
      * the memtable to disk when the size exceeds the threshold.
     */
-    void put(String key, ColumnFamily columnFamily, CommitLog.CommitLogContext cLogCtx) throws IOException
+    public void put(String key, ColumnFamily columnFamily, CommitLog.CommitLogContext cLogCtx) throws IOException
     {
-        isDirty_ = true;
-        executor_.submit(new Putter(key, columnFamily));
         if (isThresholdViolated())
         {
             enqueueFlush(cLogCtx);
+            // retry the put on the new memtable
+            ColumnFamilyStore cfStore = Table.open(table_).getColumnFamilyStore(cfName_);
+            cfStore.apply(key, columnFamily, cLogCtx);
+            return;
         }
+
+        isDirty_ = true;
+        executor_.submit(new Putter(key, columnFamily));
     }
 
     /*