Author: jbellis
Date: Thu Jul 22 03:30:12 2010
New Revision: 966492

URL: http://svn.apache.org/viewvc?rev=966492&view=rev
Log:
take advantage of row cache during range queries where possible.  patch by 
tjake and jbellis for CASSANDRA-1302

Modified:
    cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
    cassandra/trunk/src/java/org/apache/cassandra/db/RowIteratorFactory.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=966492&r1=966491&r2=966492&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java 
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java Thu 
Jul 22 03:30:12 2010
@@ -832,58 +832,69 @@ public class ColumnFamilyStore implement
             ColumnFamily cached = cacheRow(filter.key);
             if (cached == null)
                 return null;
-            
-            // special case slicing the entire row:
-            // we can skip the filter step entirely, and we can help out 
removeDeleted by re-caching the result
-            // if any tombstones have aged out since last time.  (This means 
that the row cache will treat gcBefore as
-            // max(gcBefore, all previous gcBefore), which is fine for 
correctness.)
-            //
-            // But, if the filter is asking for less columns than we have 
cached, we fall back to the slow path
-            // since we have to copy out a subset.
-            if (filter.filter instanceof SliceQueryFilter)
+
+            return filterColumnFamily(cached, filter, gcBefore);
+        }
+        finally
+        {
+            readStats_.addNano(System.nanoTime() - start);
+        }
+    }
+
+    /** filter a cached row, which will not be modified by the filter, but may 
be modified by throwing out
+     *  tombstones that are no longer relevant. */
+    ColumnFamily filterColumnFamily(ColumnFamily cached, QueryFilter filter, 
int gcBefore)
+    {
+        // special case slicing the entire row:
+        // we can skip the filter step entirely, and we can help out 
removeDeleted by re-caching the result
+        // if any tombstones have aged out since last time.  (This means that 
the row cache will treat gcBefore as
+        // max(gcBefore, all previous gcBefore), which is fine for 
correctness.)
+        //
+        // But, if the filter is asking for less columns than we have cached, 
we fall back to the slow path
+        // since we have to copy out a subset.
+        if (filter.filter instanceof SliceQueryFilter)
+        {
+            SliceQueryFilter sliceFilter = (SliceQueryFilter) filter.filter;
+            if (sliceFilter.start.length == 0 && sliceFilter.finish.length == 
0)
             {
-                SliceQueryFilter sliceFilter = (SliceQueryFilter) 
filter.filter;
-                if (sliceFilter.start.length == 0 && sliceFilter.finish.length 
== 0)
+                if (cached.isSuper() && filter.path.superColumnName != null)
                 {
-                    if (cached.isSuper() && filter.path.superColumnName != 
null)
+                    // subcolumns from named supercolumn
+                    IColumn sc = cached.getColumn(filter.path.superColumnName);
+                    if (sc == null || sliceFilter.count >= 
sc.getSubColumns().size())
                     {
-                        // subcolumns from named supercolumn
-                        IColumn sc = 
cached.getColumn(filter.path.superColumnName);
-                        if (sc == null || sliceFilter.count >= 
sc.getSubColumns().size())
-                        {
-                            ColumnFamily cf = cached.cloneMeShallow();
-                            if (sc != null)
-                                cf.addColumn(sc);
-                            return removeDeleted(cf, gcBefore);
-                        }
+                        ColumnFamily cf = cached.cloneMeShallow();
+                        if (sc != null)
+                            cf.addColumn(sc);
+                        return removeDeleted(cf, gcBefore);
                     }
-                    else
+                }
+                else
+                {
+                    // top-level columns
+                    if (sliceFilter.count >= cached.getColumnCount())
                     {
-                        // top-level columns
-                        if (sliceFilter.count >= cached.getColumnCount())
-                        {
-                            removeDeletedColumnsOnly(cached, gcBefore);
-                            return removeDeletedCF(cached, gcBefore);
-                        }
+                        removeDeletedColumnsOnly(cached, gcBefore);
+                        return removeDeletedCF(cached, gcBefore);
                     }
                 }
             }
-            
-            IColumnIterator ci = filter.getMemtableColumnIterator(cached, 
null, getComparator());
-            ColumnFamily cf = ci.getColumnFamily().cloneMeShallow();
-            filter.collectCollatedColumns(cf, ci, gcBefore);
-            // TODO this is necessary because when we collate supercolumns 
together, we don't check
-            // their subcolumns for relevance, so we need to do a second prune 
post facto here.
-            return cf.isSuper() ? removeDeleted(cf, gcBefore) : 
removeDeletedCF(cf, gcBefore);
         }
-        catch (IOException e)
+
+        IColumnIterator ci = filter.getMemtableColumnIterator(cached, null, 
getComparator());
+        ColumnFamily cf = null;
+        try
         {
-            throw new IOError(e);
+            cf = ci.getColumnFamily().cloneMeShallow();
         }
-        finally
+        catch (IOException e)
         {
-            readStats_.addNano(System.nanoTime() - start);
+            throw new IOError(e);
         }
+        filter.collectCollatedColumns(cf, ci, gcBefore);
+        // TODO this is necessary because when we collate supercolumns 
together, we don't check
+        // their subcolumns for relevance, so we need to do a second prune 
post facto here.
+        return cf.isSuper() ? removeDeleted(cf, gcBefore) : 
removeDeletedCF(cf, gcBefore);
     }
 
     private ColumnFamily getTopLevelColumns(QueryFilter filter, int gcBefore)
@@ -982,7 +993,7 @@ public class ColumnFamilyStore implement
         Collection<SSTableReader> sstables = new ArrayList<SSTableReader>();
         Iterables.addAll(sstables, ssTables_);
 
-        RowIterator iterator = RowIteratorFactory.getIterator(memtables, 
sstables, startWith, stopAt, filter, getComparator(), gcBefore);
+        RowIterator iterator = RowIteratorFactory.getIterator(memtables, 
sstables, startWith, stopAt, filter, getComparator(), this, gcBefore);
 
         try
         {

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/db/RowIteratorFactory.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/RowIteratorFactory.java?rev=966492&r1=966491&r2=966492&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/RowIteratorFactory.java 
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/RowIteratorFactory.java 
Thu Jul 22 03:30:12 2010
@@ -71,6 +71,7 @@ public class RowIteratorFactory
                                           final DecoratedKey stopAt,
                                           final QueryFilter filter,
                                           final AbstractType comparator,
+                                          final ColumnFamilyStore cfs,
                                           final int gcBefore)
     {
         // fetch data from current memtable, historical memtables, and 
SSTables in the correct order.
@@ -128,20 +129,32 @@ public class RowIteratorFactory
                 Comparator<IColumn> colComparator = 
QueryFilter.getColumnComparator(comparator);
                 Iterator<IColumn> colCollated = 
IteratorUtils.collatedIterator(colComparator, colIters);
 
-                ColumnFamily returnCF = firstMemtable.getColumnFamily(key);
-                // TODO this is a little subtle: the Memtable ColumnIterator 
has to be a shallow clone of the source CF,
-                // with deletion times set correctly, so we can use it as the 
"base" CF to add query results to.
-                // (for sstable ColumnIterators we do not care if it is a 
shallow clone or not.)
-                returnCF = returnCF == null ? 
ColumnFamily.create(firstMemtable.getTableName(), filter.getColumnFamilyName())
-                                            : returnCF.cloneMeShallow();
-
-                if (colCollated.hasNext())
+                ColumnFamily returnCF = null;
+                
+                // First check if this row is in the rowCache. If it is we can 
skip the rest
+                ColumnFamily cached = cfs.getRawCachedRow(key);
+                if (cached != null)
                 {
-                    filter.collectCollatedColumns(returnCF, colCollated, 
gcBefore);
+                    QueryFilter keyFilter = new QueryFilter(key, filter.path, 
filter.filter);
+                    returnCF = cfs.filterColumnFamily(cached, keyFilter, 
CompactionManager.getDefaultGCBefore());
                 }
                 else
                 {
-                    returnCF = null;
+                    returnCF = firstMemtable.getColumnFamily(key);            
+                    // TODO this is a little subtle: the Memtable 
ColumnIterator has to be a shallow clone of the source CF,
+                    // with deletion times set correctly, so we can use it as 
the "base" CF to add query results to.
+                    // (for sstable ColumnIterators we do not care if it is a 
shallow clone or not.)
+                    returnCF = returnCF == null ? 
ColumnFamily.create(firstMemtable.getTableName(), filter.getColumnFamilyName())
+                            : returnCF.cloneMeShallow();
+
+                    if (colCollated.hasNext())
+                    {
+                        filter.collectCollatedColumns(returnCF, colCollated, 
gcBefore);
+                    }
+                    else
+                    {
+                        returnCF = null;
+                    }
                 }
 
                 Row rv = new Row(key, returnCF);


Reply via email to