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 2011/10/01 17:04:01 UTC

svn commit: r1178007 - in /cassandra/branches/cassandra-1.0.0: CHANGES.txt src/java/org/apache/cassandra/db/CollationController.java src/java/org/apache/cassandra/db/ColumnFamilyStore.java

Author: jbellis
Date: Sat Oct  1 15:04:00 2011
New Revision: 1178007

URL: http://svn.apache.org/viewvc?rev=1178007&view=rev
Log:
use correct ISortedColumns for time-optimized reads
patch by jbellis; reviewed by slebresne for CASSANDRA-3289

Modified:
    cassandra/branches/cassandra-1.0.0/CHANGES.txt
    cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/db/CollationController.java
    cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/db/ColumnFamilyStore.java

Modified: cassandra/branches/cassandra-1.0.0/CHANGES.txt
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0.0/CHANGES.txt?rev=1178007&r1=1178006&r2=1178007&view=diff
==============================================================================
--- cassandra/branches/cassandra-1.0.0/CHANGES.txt (original)
+++ cassandra/branches/cassandra-1.0.0/CHANGES.txt Sat Oct  1 15:04:00 2011
@@ -1,8 +1,3 @@
-1.0.0-final
- * ignore any CF ids sent by client for adding CF/KS (CASSANDRA-3288)
- * remove obsolete hints on first startup (CASSANDRA-3291)
-
-
 1.0.0-rc2
  * Log a meaningful warning when a node receives a message for a repair session
    that doesn't exist anymore (CASSANDRA-3256)
@@ -17,6 +12,10 @@
  * fix full queue scenario for ParallelCompactionIterator (CASSANDRA-3270)
  * fix bootstrap process (CASSANDRA-3285)
  * CLI documentation change for ColumnFamily `compression_options` (CASSANDRA-3282)
+ * ignore any CF ids sent by client for adding CF/KS (CASSANDRA-3288)
+ * remove obsolete hints on first startup (CASSANDRA-3291)
+ * use correct ISortedColumns for time-optimized reads (CASSANDRA-3289)
+
 
 1.0.0-rc1
  * Update CQL to generate microsecond timestamps by default (CASSANDRA-3227)

Modified: cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/db/CollationController.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/db/CollationController.java?rev=1178007&r1=1178006&r2=1178007&view=diff
==============================================================================
--- cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/db/CollationController.java (original)
+++ cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/db/CollationController.java Sat Oct  1 15:04:00 2011
@@ -42,16 +42,16 @@ public class CollationController
     private static Logger logger = LoggerFactory.getLogger(CollationController.class);
 
     private final ColumnFamilyStore cfs;
-    private final ISortedColumns.Factory factory;
+    private final boolean mutableColumns;
     private final QueryFilter filter;
     private final int gcBefore;
 
     private int sstablesIterated = 0;
 
-    public CollationController(ColumnFamilyStore cfs, ISortedColumns.Factory factory, QueryFilter filter, int gcBefore)
+    public CollationController(ColumnFamilyStore cfs, boolean mutableColumns, QueryFilter filter, int gcBefore)
     {
         this.cfs = cfs;
-        this.factory = factory;
+        this.mutableColumns = mutableColumns;
         this.filter = filter;
         this.gcBefore = gcBefore;
     }
@@ -72,6 +72,9 @@ public class CollationController
     {
         logger.debug("collectTimeOrderedData");
 
+        ISortedColumns.Factory factory = mutableColumns
+                                       ? ThreadSafeSortedColumns.factory()
+                                       : TreeMapBackedSortedColumns.factory();
         ColumnFamily container = ColumnFamily.create(cfs.metadata, factory, filter.filter.isReversed());
         List<IColumnIterator> iterators = new ArrayList<IColumnIterator>();
         ColumnFamilyStore.ViewFragment view = cfs.markReferenced(filter.key);
@@ -184,6 +187,9 @@ public class CollationController
     private ColumnFamily collectAllData()
     {
         logger.debug("collectAllData");
+        ISortedColumns.Factory factory = mutableColumns
+                                       ? ThreadSafeSortedColumns.factory()
+                                       : ArrayBackedSortedColumns.factory();
         List<IColumnIterator> iterators = new ArrayList<IColumnIterator>();
         ColumnFamily returnCF = ColumnFamily.create(cfs.metadata, factory, filter.filter.isReversed());
 

Modified: cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/db/ColumnFamilyStore.java?rev=1178007&r1=1178006&r2=1178007&view=diff
==============================================================================
--- cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/db/ColumnFamilyStore.java (original)
+++ cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/db/ColumnFamilyStore.java Sat Oct  1 15:04:00 2011
@@ -1106,12 +1106,12 @@ public class ColumnFamilyStore implement
      */
     public ColumnFamily getColumnFamily(QueryFilter filter)
     {
-        return getColumnFamily(filter, gcBefore(), ThreadSafeSortedColumns.factory());
+        return getColumnFamily(filter, gcBefore());
     }
 
     public ColumnFamily getColumnFamily(QueryFilter filter, ISortedColumns.Factory factory)
     {
-        return getColumnFamily(filter, gcBefore(), factory);
+        return getColumnFamily(filter, gcBefore());
     }
 
     public int gcBefore()
@@ -1125,7 +1125,7 @@ public class ColumnFamilyStore implement
         if ((cached = rowCache.get(key)) == null)
         {
             // We force ThreadSafeSortedColumns because cached row will be accessed concurrently
-            cached = getTopLevelColumns(QueryFilter.getIdentityFilter(key, new QueryPath(columnFamily)), Integer.MIN_VALUE, ThreadSafeSortedColumns.factory());
+            cached = getTopLevelColumns(QueryFilter.getIdentityFilter(key, new QueryPath(columnFamily)), Integer.MIN_VALUE, true);
             if (cached == null)
                 return null;
 
@@ -1135,7 +1135,7 @@ public class ColumnFamilyStore implement
         return cached;
     }
 
-    private ColumnFamily getColumnFamily(QueryFilter filter, int gcBefore, ISortedColumns.Factory factory)
+    private ColumnFamily getColumnFamily(QueryFilter filter, int gcBefore)
     {
         assert columnFamily.equals(filter.getColumnFamilyName()) : filter.getColumnFamilyName();
 
@@ -1144,7 +1144,7 @@ public class ColumnFamilyStore implement
         {
             if (rowCache.getCapacity() == 0)
             {
-                ColumnFamily cf = getTopLevelColumns(filter, gcBefore, factory);
+                ColumnFamily cf = getTopLevelColumns(filter, gcBefore, false);
 
                 if (cf == null)
                     return null;
@@ -1291,9 +1291,9 @@ public class ColumnFamilyStore implement
         return new ViewFragment(sstables, Iterables.concat(Collections.singleton(view.memtable), view.memtablesPendingFlush));
     }
 
-    private ColumnFamily getTopLevelColumns(QueryFilter filter, int gcBefore, ISortedColumns.Factory factory)
+    private ColumnFamily getTopLevelColumns(QueryFilter filter, int gcBefore, boolean forCache)
     {
-        CollationController controller = new CollationController(this, factory, filter, gcBefore);
+        CollationController controller = new CollationController(this, forCache, filter, gcBefore);
         ColumnFamily columns = controller.getTopLevelColumns();
         recentSSTablesPerRead.add(controller.getSstablesIterated());
         sstablesPerRead.add(controller.getSstablesIterated());