cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jbel...@apache.org
Subject svn commit: r1022159 - in /cassandra/trunk/src/java/org/apache/cassandra/db: ColumnFamilyStore.java Table.java
Date Wed, 13 Oct 2010 16:02:59 GMT
Author: jbellis
Date: Wed Oct 13 16:02:58 2010
New Revision: 1022159

URL: http://svn.apache.org/viewvc?rev=1022159&view=rev
Log:
add Table.flusherLock javadoc

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

Modified: cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java?rev=1022159&r1=1022158&r2=1022159&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java Wed Oct 13 16:02:58
2010
@@ -549,10 +549,15 @@ public class ColumnFamilyStore implement
     /** flush the given memtable and swap in a new one for its CFS, if it hasn't been frozen
already.  threadsafe. */
     Future<?> maybeSwitchMemtable(Memtable oldMemtable, final boolean writeCommitLog)
     {
-        /**
-         *  If we can get the writelock, that means no new updates can come in and 
-         *  all ongoing updates to memtables have completed. We can get the tail
-         *  of the log and use it as the starting position for log replay on recovery.
+        /*
+         * If we can get the writelock, that means no new updates can come in and
+         * all ongoing updates to memtables have completed. We can get the tail
+         * of the log and use it as the starting position for log replay on recovery.
+         *
+         * This is why we Table.flusherLock needs to be global instead of per-Table:
+         * we need to schedule discardCompletedSegments calls in the same order as their
+         * contexts (commitlog position) were read, even though the flush executor
+         * is multithreaded.
          */
         Table.flusherLock.writeLock().lock();
         try

Modified: cassandra/trunk/src/java/org/apache/cassandra/db/Table.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/Table.java?rev=1022159&r1=1022158&r2=1022159&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/Table.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/Table.java Wed Oct 13 16:02:58 2010
@@ -55,7 +55,11 @@ public class Table
 
     private static final Logger logger = LoggerFactory.getLogger(Table.class);
     private static final String SNAPSHOT_SUBDIR_NAME = "snapshots";
-    /* accesses to CFS.memtable should acquire this for thread safety.  only switchMemtable
should aquire the writeLock. */
+
+    /**
+     * accesses to CFS.memtable should acquire this for thread safety.
+     * only Table.maybeSwitchMemtable should aquire the writeLock; see that method for the
full explanation.
+     */
     static final ReentrantReadWriteLock flusherLock = new ReentrantReadWriteLock(true);
 
     // This is a result of pushing down the point in time when storage directories get created.
 It used to happen in



Mime
View raw message