Author: jbellis
Date: Wed Aug 25 13:55:58 2010
New Revision: 989110

URL: http://svn.apache.org/viewvc?rev=989110&view=rev
Log:
clean up column iterators, primarily replacing sstable w/ metadata in 
parameters.  patch by jbellis

Modified:
    cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
    
cassandra/trunk/src/java/org/apache/cassandra/db/columniterator/IndexedSliceReader.java
    
cassandra/trunk/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java
    
cassandra/trunk/src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java
    
cassandra/trunk/src/java/org/apache/cassandra/db/columniterator/SimpleSliceReader.java
    cassandra/trunk/src/java/org/apache/cassandra/db/filter/IFilter.java
    
cassandra/trunk/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java
    cassandra/trunk/src/java/org/apache/cassandra/db/filter/QueryFilter.java
    
cassandra/trunk/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
    cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTable.java
    cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableScanner.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=989110&r1=989109&r2=989110&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java 
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java Wed 
Aug 25 13:55:58 2010
@@ -127,7 +127,7 @@ public class ColumnFamilyStore implement
     /* active memtable associated with this ColumnFamilyStore. */
     private Memtable memtable;
 
-    private final Map<byte[], ColumnFamilyStore> indexedColumns;
+    private final SortedMap<byte[], ColumnFamilyStore> indexedColumns;
 
     // TODO binarymemtable ops are not threadsafe (do they need to be?)
     private AtomicReference<BinaryMemtable> binaryMemtable;
@@ -138,7 +138,7 @@ public class ColumnFamilyStore implement
     private LatencyTracker readStats = new LatencyTracker();
     private LatencyTracker writeStats = new LatencyTracker();
 
-    final CFMetaData metadata;
+    public final CFMetaData metadata;
     
     private ColumnFamilyStore(String table, String columnFamilyName, 
IPartitioner partitioner, int generation, CFMetaData metadata)
     {
@@ -189,7 +189,7 @@ public class ColumnFamilyStore implement
         ssTables = new SSTableTracker(table, columnFamilyName);
         ssTables.add(sstables);
 
-        indexedColumns = new TreeMap<byte[], 
ColumnFamilyStore>(BytesType.instance);
+        indexedColumns = new TreeMap<byte[], 
ColumnFamilyStore>(getComparator());
         for (Map.Entry<byte[], ColumnDefinition> entry : 
metadata.column_metadata.entrySet())
         {
             byte[] column = entry.getKey();
@@ -895,7 +895,7 @@ public class ColumnFamilyStore implement
         return getColumnFamily(filter, gcBefore());
     }
 
-    private int gcBefore()
+    public int gcBefore()
     {
         return (int) (System.currentTimeMillis() / 1000) - 
metadata.gcGraceSeconds;
     }
@@ -1599,9 +1599,9 @@ public class ColumnFamilyStore implement
         return (double) falseCount / (trueCount + falseCount);
     }
 
-    public Set<byte[]> getIndexedColumns()
+    public SortedSet<byte[]> getIndexedColumns()
     {
-        return indexedColumns.keySet();
+        return (SortedSet<byte[]>) indexedColumns.keySet();
     }
 
     public ColumnFamilyStore getIndexedColumnFamilyStore(byte[] column)

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/db/columniterator/IndexedSliceReader.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/columniterator/IndexedSliceReader.java?rev=989110&r1=989109&r2=989110&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/db/columniterator/IndexedSliceReader.java
 (original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/db/columniterator/IndexedSliceReader.java
 Wed Aug 25 13:55:58 2010
@@ -29,6 +29,7 @@ import java.util.List;
 
 import com.google.common.collect.AbstractIterator;
 
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.IColumn;
@@ -57,19 +58,19 @@ class IndexedSliceReader extends Abstrac
     private Deque<IColumn> blockColumns = new ArrayDeque<IColumn>();
     private AbstractType comparator;
 
-    public IndexedSliceReader(SSTableReader sstable, FileDataInput input, 
byte[] startColumn, byte[] finishColumn, boolean reversed)
+    public IndexedSliceReader(CFMetaData metadata, FileDataInput input, byte[] 
startColumn, byte[] finishColumn, boolean reversed)
     {
         this.file = input;
         this.startColumn = startColumn;
         this.finishColumn = finishColumn;
         this.reversed = reversed;
-        comparator = sstable.getColumnComparator();
+        comparator = metadata.comparator;
         try
         {
             IndexHelper.skipBloomFilter(file);
             indexes = IndexHelper.deserializeIndex(file);
 
-            emptyColumnFamily = 
ColumnFamily.serializer().deserializeFromSSTableNoColumns(sstable.createColumnFamily(),
 file);
+            emptyColumnFamily = 
ColumnFamily.serializer().deserializeFromSSTableNoColumns(ColumnFamily.create(metadata),
 file);
             fetcher = indexes == null ? new SimpleBlockFetcher() : new 
IndexedBlockFetcher();
         }
         catch (IOException e)

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java?rev=989110&r1=989109&r2=989110&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java
 (original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java
 Wed Aug 25 13:55:58 2010
@@ -27,6 +27,7 @@ import java.util.*;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.DecoratedKey;
@@ -65,7 +66,7 @@ public class SSTableNamesIterator extend
                                                              
FBUtilities.readShortByteArray(file));
             assert keyInDisk.equals(key) : String.format("%s != %s in %s", 
keyInDisk, key, file.getPath());
             SSTableReader.readRowSize(file, sstable.getDescriptor());
-            read(sstable, file);
+            read(sstable.metadata, file);
         }
         catch (IOException e)
         {
@@ -84,7 +85,7 @@ public class SSTableNamesIterator extend
         }
     }
 
-    public SSTableNamesIterator(SSTableReader sstable, FileDataInput file, 
DecoratedKey key, SortedSet<byte[]> columns)
+    public SSTableNamesIterator(CFMetaData metadata, FileDataInput file, 
DecoratedKey key, SortedSet<byte[]> columns)
     {
         assert columns != null;
         this.columns = columns;
@@ -92,7 +93,7 @@ public class SSTableNamesIterator extend
 
         try
         {
-            read(sstable, file);
+            read(metadata, file);
         }
         catch (IOException ioe)
         {
@@ -100,7 +101,7 @@ public class SSTableNamesIterator extend
         }
     }
 
-    private void read(SSTableReader sstable, FileDataInput file)
+    private void read(CFMetaData metadata, FileDataInput file)
     throws IOException
     {
 
@@ -111,7 +112,7 @@ public class SSTableNamesIterator extend
 
         // we can stop early if bloom filter says none of the columns actually 
exist -- but,
         // we can't stop before initializing the cf above, in case there's a 
relevant tombstone
-        cf = 
ColumnFamily.serializer().deserializeFromSSTableNoColumns(sstable.createColumnFamily(),
 file);
+        cf = 
ColumnFamily.serializer().deserializeFromSSTableNoColumns(ColumnFamily.create(metadata),
 file);
 
         List<byte[]> filteredColumnNames = new 
ArrayList<byte[]>(columns.size());
         for (byte[] name : columns)
@@ -127,7 +128,7 @@ public class SSTableNamesIterator extend
         if (indexList == null)
             readSimpleColumns(file, columns, filteredColumnNames);
         else
-            readIndexedColumns(sstable, file, columns, filteredColumnNames, 
indexList);
+            readIndexedColumns(metadata, file, columns, filteredColumnNames, 
indexList);
 
         // create an iterator view of the columns we read
         iter = cf.getSortedColumns().iterator();
@@ -149,13 +150,13 @@ public class SSTableNamesIterator extend
         }
     }
 
-    private void readIndexedColumns(SSTableReader sstable, FileDataInput file, 
SortedSet<byte[]> columnNames, List<byte[]> filteredColumnNames, 
List<IndexHelper.IndexInfo> indexList)
+    private void readIndexedColumns(CFMetaData metadata, FileDataInput file, 
SortedSet<byte[]> columnNames, List<byte[]> filteredColumnNames, 
List<IndexHelper.IndexInfo> indexList)
     throws IOException
     {
         file.readInt(); // column count
 
         /* get the various column ranges we have to read */
-        AbstractType comparator = sstable.getColumnComparator();
+        AbstractType comparator = metadata.comparator;
         SortedSet<IndexHelper.IndexInfo> ranges = new 
TreeSet<IndexHelper.IndexInfo>(IndexHelper.getComparator(comparator));
         for (byte[] name : filteredColumnNames)
         {

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java?rev=989110&r1=989109&r2=989110&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java
 (original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java
 Wed Aug 25 13:55:58 2010
@@ -24,6 +24,7 @@ package org.apache.cassandra.db.columnit
 import java.io.IOError;
 import java.io.IOException;
 
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.IColumn;
@@ -63,12 +64,12 @@ public class SSTableSliceIterator implem
             throw new IOError(e);
         }
 
-        reader = createReader(sstable, fileToClose, startColumn, finishColumn, 
reversed);
+        reader = createReader(sstable.metadata, fileToClose, startColumn, 
finishColumn, reversed);
     }
 
     /**
      * An iterator for a slice within an SSTable
-     * @param sstable The SSTable to iterate over
+     * @param metadata Metadata for the CFS we are reading from
      * @param file Optional parameter that input is read from.  If null is 
passed, this class creates an appropriate one automatically.
      * If this class creates, it will close the underlying file when #close() 
is called.
      * If a caller passes a non-null argument, this class will NOT close the 
underlying file when the iterator is closed (i.e. the caller is responsible for 
closing the file)
@@ -78,18 +79,18 @@ public class SSTableSliceIterator implem
      * @param finishColumn The end of the slice
      * @param reversed Results are returned in reverse order iff reversed is 
true.
      */
-    public SSTableSliceIterator(SSTableReader sstable, FileDataInput file, 
DecoratedKey key, byte[] startColumn, byte[] finishColumn, boolean reversed)
+    public SSTableSliceIterator(CFMetaData metadata, FileDataInput file, 
DecoratedKey key, byte[] startColumn, byte[] finishColumn, boolean reversed)
     {
         this.key = key;
         fileToClose = null;
-        reader = createReader(sstable, file, startColumn, finishColumn, 
reversed);
+        reader = createReader(metadata, file, startColumn, finishColumn, 
reversed);
     }
 
-    private static IColumnIterator createReader(SSTableReader ssTable, 
FileDataInput file, byte[] startColumn, byte[] finishColumn, boolean reversed)
+    private static IColumnIterator createReader(CFMetaData metadata, 
FileDataInput file, byte[] startColumn, byte[] finishColumn, boolean reversed)
     {
         return startColumn.length == 0 && !reversed
-                 ? new SimpleSliceReader(ssTable, file, finishColumn)
-                 : new IndexedSliceReader(ssTable, file, startColumn, 
finishColumn, reversed);
+                 ? new SimpleSliceReader(metadata, file, finishColumn)
+                 : new IndexedSliceReader(metadata, file, startColumn, 
finishColumn, reversed);
     }
 
     public DecoratedKey getKey()

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/db/columniterator/SimpleSliceReader.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/columniterator/SimpleSliceReader.java?rev=989110&r1=989109&r2=989110&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/db/columniterator/SimpleSliceReader.java
 (original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/db/columniterator/SimpleSliceReader.java
 Wed Aug 25 13:55:58 2010
@@ -26,6 +26,7 @@ import java.io.IOException;
 
 import com.google.common.collect.AbstractIterator;
 
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.IColumn;
@@ -45,17 +46,17 @@ class SimpleSliceReader extends Abstract
     private int i;
     private FileMark mark;
 
-    public SimpleSliceReader(SSTableReader sstable, FileDataInput input, 
byte[] finishColumn)
+    public SimpleSliceReader(CFMetaData metadata, FileDataInput input, byte[] 
finishColumn)
     {
         this.file = input;
         this.finishColumn = finishColumn;
-        comparator = sstable.getColumnComparator();
+        comparator = metadata.comparator;
         try
         {
             IndexHelper.skipBloomFilter(file);
             IndexHelper.skipIndex(file);
 
-            emptyColumnFamily = 
ColumnFamily.serializer().deserializeFromSSTableNoColumns(sstable.createColumnFamily(),
 file);
+            emptyColumnFamily = 
ColumnFamily.serializer().deserializeFromSSTableNoColumns(ColumnFamily.create(metadata),
 file);
             columns = file.readInt();
             mark = file.mark();
         }

Modified: cassandra/trunk/src/java/org/apache/cassandra/db/filter/IFilter.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/filter/IFilter.java?rev=989110&r1=989109&r2=989110&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/filter/IFilter.java 
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/filter/IFilter.java Wed 
Aug 25 13:55:58 2010
@@ -23,6 +23,7 @@ package org.apache.cassandra.db.filter;
 import java.util.Comparator;
 import java.util.Iterator;
 
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.columniterator.IColumnIterator;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -47,12 +48,11 @@ public interface IFilter
     /**
      * Get an iterator that returns columns from the given SSTable using the 
opened file
      * matching the Filter criteria in sorted order.
-     * @param sstable SSTable we are reading from
+     * @param metadata
      * @param file Already opened file data input, saves us opening another one
      * @param key The key of the row we are about to iterate over
-     * @param dataStart Used to seek to the start of the data
      */
-    public abstract IColumnIterator getSSTableColumnIterator(SSTableReader 
sstable, FileDataInput file, DecoratedKey key, long dataStart);
+    public abstract IColumnIterator getSSTableColumnIterator(CFMetaData 
metadata, FileDataInput file, DecoratedKey key);
 
     /**
      * returns an iterator that returns columns from the given SSTable

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java?rev=989110&r1=989109&r2=989110&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java 
(original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java 
Wed Aug 25 13:55:58 2010
@@ -23,6 +23,7 @@ package org.apache.cassandra.db.filter;
 
 import java.util.*;
 
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.columniterator.IColumnIterator;
 import org.apache.cassandra.db.columniterator.SSTableNamesIterator;
 import org.apache.cassandra.io.sstable.SSTableReader;
@@ -68,9 +69,9 @@ public class NamesQueryFilter implements
         return new SSTableNamesIterator(sstable, key, columns);
     }
     
-    public IColumnIterator getSSTableColumnIterator(SSTableReader sstable, 
FileDataInput file, DecoratedKey key, long dataStart)
+    public IColumnIterator getSSTableColumnIterator(CFMetaData metadata, 
FileDataInput file, DecoratedKey key)
     {
-        return new SSTableNamesIterator(sstable, file, key, columns);
+        return new SSTableNamesIterator(metadata, file, key, columns);
     }
 
     public SuperColumn filterSuperColumn(SuperColumn superColumn, int gcBefore)

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/db/filter/QueryFilter.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/filter/QueryFilter.java?rev=989110&r1=989109&r2=989110&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/filter/QueryFilter.java 
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/filter/QueryFilter.java 
Wed Aug 25 13:55:58 2010
@@ -77,11 +77,11 @@ public class QueryFilter
         return superFilter.getSSTableColumnIterator(sstable, key);
     }
 
-    public IColumnIterator getSSTableColumnIterator(SSTableReader sstable, 
FileDataInput file, DecoratedKey key, long dataStart)
+    public IColumnIterator getSSTableColumnIterator(SSTableReader sstable, 
FileDataInput file, DecoratedKey key)
     {
         if (path.superColumnName == null)
-            return filter.getSSTableColumnIterator(sstable, file, key, 
dataStart);
-        return superFilter.getSSTableColumnIterator(sstable, file, key, 
dataStart);
+            return filter.getSSTableColumnIterator(sstable.metadata, file, 
key);
+        return superFilter.getSSTableColumnIterator(sstable.metadata, file, 
key);
     }
 
     public static Comparator<IColumn> getColumnComparator(final AbstractType 
comparator)

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java?rev=989110&r1=989109&r2=989110&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java 
(original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java 
Wed Aug 25 13:55:58 2010
@@ -29,6 +29,7 @@ import org.apache.commons.collections.co
 import org.apache.commons.collections.iterators.ReverseListIterator;
 import org.apache.commons.collections.IteratorUtils;
 
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.columniterator.IColumnIterator;
 import org.apache.cassandra.db.columniterator.SSTableSliceIterator;
 import org.apache.cassandra.io.sstable.SSTableReader;
@@ -64,9 +65,9 @@ public class SliceQueryFilter implements
         return new SSTableSliceIterator(sstable, key, start, finish, reversed);
     }
     
-    public IColumnIterator getSSTableColumnIterator(SSTableReader sstable, 
FileDataInput file, DecoratedKey key, long dataStart)
+    public IColumnIterator getSSTableColumnIterator(CFMetaData metadata, 
FileDataInput file, DecoratedKey key)
     {
-        return new SSTableSliceIterator(sstable, file, key, start, finish, 
reversed);
+        return new SSTableSliceIterator(metadata, file, key, start, finish, 
reversed);
     }
 
     public SuperColumn filterSuperColumn(SuperColumn superColumn, int gcBefore)

Modified: cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTable.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTable.java?rev=989110&r1=989109&r2=989110&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTable.java 
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTable.java Wed 
Aug 25 13:55:58 2010
@@ -58,7 +58,7 @@ public abstract class SSTable
     public static final String COMPONENT_COMPACTED = "Compacted";
 
     protected final Descriptor desc;
-    protected final CFMetaData metadata;
+    public final CFMetaData metadata;
     protected final IPartitioner partitioner;
 
     public static final String TEMPFILE_MARKER = "tmp";

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java?rev=989110&r1=989109&r2=989110&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java 
(original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java 
Wed Aug 25 13:55:58 2010
@@ -181,7 +181,7 @@ public class SSTableScanner implements I
                 }
                 else
                 {
-                    return row = filter.getSSTableColumnIterator(sstable, 
file, key, dataStart);
+                    return row = filter.getSSTableColumnIterator(sstable, 
file, key);
                 }
             }
             catch (IOException e)


Reply via email to