Make iterator-based read code the One True Path
-----------------------------------------------

                 Key: CASSANDRA-287
                 URL: https://issues.apache.org/jira/browse/CASSANDRA-287
             Project: Cassandra
          Issue Type: Bug
            Reporter: Jonathan Ellis
            Assignee: Jonathan Ellis
         Attachments: 
0001-CASSANDRA-287-r-m-unused-and-dangerous-RowReadComman.txt, 
0002-rename-lock_-sstableLock_.txt, 
0003-refactor-out-QueryFilter-SliceQueryFilter.txt, 
0004-replace-namesfilter-with-NamesQueryFilter.-mv-filter.txt, 
0005-nuke-timefilter.txt, 
0006-add-IdentityQueryFilter-finish-removing-IFilter.txt

Since CASSANDRA-172 we've had two read paths; the old, ad-hoc path based on the 
faulty assumption that we could skip checking older sstables if we got a hit 
earlier in the path (fixed in CASSANDRA-223 but still bearing the marks of its 
origin) and the new iterator-based path.

This makes all read operations go through the iterator path, which cleans 
things up enormously and sets the stage for CASSANDRA-139.

I introduce a new QueryFilter interface, which has 3 main methods:

    /**
     * returns an iterator that returns columns from the given memtable
     * matching the Filter criteria in sorted order.
     */
    public abstract ColumnIterator getMemColumnIterator(Memtable memtable);

    /**
     * returns an iterator that returns columns from the given SSTable
     * matching the Filter criteria in sorted order.
     */
    public abstract ColumnIterator getSSTableColumnIterator(SSTableReader 
sstable) throws IOException;

    /**
     * subcolumns of a supercolumn are unindexed, so to pick out parts of those 
we operate in-memory.
     * @param superColumn
     */
    public abstract void filterSuperColumn(SuperColumn superColumn);

The first two are for pulling out indexed top-level columns, from a memtable or 
an sstable, respectively.

If the query is on subcolumns of a supercolumn, which are unindexed, 
CFS.getColumnFamily does an indexed Name filter on the supercolumn, then asks 
filterSuperColumn on the primary QueryFilter to pick out the parts the user is 
requesting.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.

Reply via email to