Author: jbellis
Date: Fri Jun  4 22:13:21 2010
New Revision: 951594

URL: http://svn.apache.org/viewvc?rev=951594&view=rev
Log:
optimize away copy of sliced row on memtable read path.  patch by mdennis; 
reviewed by jbellis for CASSANDRA-1046

Modified:
    cassandra/trunk/CHANGES.txt
    cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamily.java
    cassandra/trunk/src/java/org/apache/cassandra/db/Memtable.java

Modified: cassandra/trunk/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/CHANGES.txt?rev=951594&r1=951593&r2=951594&view=diff
==============================================================================
--- cassandra/trunk/CHANGES.txt (original)
+++ cassandra/trunk/CHANGES.txt Fri Jun  4 22:13:21 2010
@@ -23,6 +23,7 @@ dev
  * removed deprecated batch_insert and get_range_slice methods (CASSANDRA-1065)
  * add truncate thrift method (CASSANDRA-531)
  * http mini-interface using mx4j (CASSANDRA-1068)
+ * optimize away copy of sliced row on memtable read path (CASSANDRA-1046)
 
 
 0.6.3

Modified: cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamily.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamily.java?rev=951594&r1=951593&r2=951594&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamily.java 
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamily.java Fri Jun  
4 22:13:21 2010
@@ -18,10 +18,7 @@
 
 package org.apache.cassandra.db;
 
-import java.util.Collection;
-import java.util.Map;
-import java.util.Set;
-import java.util.SortedSet;
+import java.util.*;
 import java.util.concurrent.ConcurrentSkipListMap;
 import java.security.MessageDigest;
 import java.security.NoSuchAlgorithmException;
@@ -265,6 +262,11 @@ public class ColumnFamily implements ICo
         return columns.values();
     }
 
+    public Collection<IColumn> getReverseSortedColumns()
+    {
+        return columns.descendingMap().values();
+    }
+
     public Map<byte[], IColumn> getColumnsMap()
     {
         return columns;

Modified: cassandra/trunk/src/java/org/apache/cassandra/db/Memtable.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/Memtable.java?rev=951594&r1=951593&r2=951594&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/Memtable.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/Memtable.java Fri Jun  4 
22:13:21 2010
@@ -26,9 +26,10 @@ import java.util.concurrent.ExecutorServ
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.Condition;
 
+import com.google.common.collect.Iterators;
+import com.google.common.collect.PeekingIterator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.commons.lang.ArrayUtils;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.filter.*;
@@ -207,39 +208,24 @@ public class Memtable implements Compara
     public static IColumnIterator getSliceIterator(final DecoratedKey key, 
final ColumnFamily cf, SliceQueryFilter filter, AbstractType typeComparator)
     {
         assert cf != null;
-        Collection<IColumn> rawColumns = cf.getSortedColumns();
-        Collection<IColumn> filteredColumns = 
filter.applyPredicate(rawColumns);
+        final boolean isSuper = cf.isSuper();
+        final Collection<IColumn> filteredColumns = filter.reversed ? 
filter.applyPredicate(cf.getReverseSortedColumns()) : 
filter.applyPredicate(cf.getSortedColumns());
 
-        final IColumn columns[] = filteredColumns.toArray(new IColumn[0]);
-        // TODO if we are dealing with supercolumns, we need to clone them 
while we have the read lock since they can be modified later
-        if (filter.reversed)
-            ArrayUtils.reverse(columns);
-        IColumn startIColumn;
-        final boolean isStandard = !cf.isSuper();
-        if (isStandard)
-            startIColumn = new Column(filter.start);
-        else
-            startIColumn = new SuperColumn(filter.start, null, 
cf.getClockType(), cf.getReconciler()); // ok to not have subcolumnComparator 
since we won't be adding columns to this object
-
-        // can't use a ColumnComparatorFactory comparator since those compare 
on both name and time (and thus will fail to match
-        // our dummy column, since the time there is arbitrary).
+        // ok to not have subcolumnComparator since we won't be adding columns 
to this object
+        IColumn startColumn = isSuper ? new SuperColumn(filter.start, null, 
cf.getClockType(), cf.getReconciler()) :  new Column(filter.start);
         Comparator<IColumn> comparator = 
filter.getColumnComparator(typeComparator);
-        int index;
-        if (filter.start.length == 0 && filter.reversed)
-        {
-            /* scan from the largest column in descending order */
-            index = 0;
-        }
-        else
+
+        final PeekingIterator<IColumn> filteredIter = 
Iterators.peekingIterator(filteredColumns.iterator());
+        if (!filter.reversed || filter.start.length != 0)
         {
-            index = Arrays.binarySearch(columns, startIColumn, comparator);
+            while (filteredIter.hasNext() && 
comparator.compare(filteredIter.peek(), startColumn) < 0)
+            {
+                filteredIter.next();
+            }
         }
-        final int startIndex = index < 0 ? -(index + 1) : index;
 
         return new AbstractColumnIterator()
         {
-            private int curIndex_ = startIndex;
-
             public ColumnFamily getColumnFamily()
             {
                 return cf;
@@ -252,13 +238,14 @@ public class Memtable implements Compara
 
             public boolean hasNext()
             {
-                return curIndex_ < columns.length;
+                return filteredIter.hasNext();
             }
 
             public IColumn next()
             {
                 // clone supercolumns so caller can freely removeDeleted or 
otherwise mutate it
-                return isStandard ? columns[curIndex_++] : 
((SuperColumn)columns[curIndex_++]).cloneMe();
+                // TODO can't the callers that wish to mutate it clone it 
themselves?
+                return isSuper ? ((SuperColumn) filteredIter.next()).cloneMe() 
: filteredIter.next();
             }
         };
     }


Reply via email to