You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Jun Rao (JIRA)" <ji...@apache.org> on 2009/05/13 21:40:46 UTC

[jira] Created: (CASSANDRA-172) A improved and more general version of get_slice

A improved and more general version of get_slice
------------------------------------------------

                 Key: CASSANDRA-172
                 URL: https://issues.apache.org/jira/browse/CASSANDRA-172
             Project: Cassandra
          Issue Type: New Feature
            Reporter: Jun Rao
            Assignee: Jun Rao


Today, get_slice has to scan through all columns in every memtable and sstable to get a slice of columns. This becomes inefficient when the number of columns in a row is large. We need a more efficient API.



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


[jira] Commented: (CASSANDRA-172) A improved and more general version of get_slice

Posted by "Jun Rao (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-172?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12711937#action_12711937 ] 

Jun Rao commented on CASSANDRA-172:
-----------------------------------

Overall, 172-addendum-v2 looks fine. Below is a new version of ReducingIterator.computeNext() that I wrote. It's a bit shorter than your version.

    protected T computeNext()
    {
        if (last == null && !source.hasNext())
            return endOfData();

        boolean keyChanged = false;
        while (true)
        {
            if (keyChanged)
                break;
            if (last != null)
                reduce(last);
            if (!source.hasNext())
            {
                last = null;
                break;
            }
            T current = source.next();
            if (last != null && !getKey(current).equals(getKey(last)))
                keyChanged = true;
            last = current;
        }
        return getReduced();
    }

A couple of other minor comments:
* You need to add the following after line 1673 in CFS.java to reverse the comparator for descending order.
            if (!isAscending)
            {
                comparator = new ReverseComparator(comparator);
            }
* SSTableColumnIterator.computeNext(), log the catched IOException.


> A improved and more general version of get_slice
> ------------------------------------------------
>
>                 Key: CASSANDRA-172
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-172
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jun Rao
>            Assignee: Jun Rao
>             Fix For: 0.4
>
>         Attachments: 172-addendum-SF.patch, 172-addendum-v2.patch, 172-addendum.patch, get_slice_from.patchv1, get_slice_from.patchv2, get_slice_from.patchv3
>
>
> Today, get_slice has to scan through all columns in every memtable and sstable to get a slice of columns. This becomes inefficient when the number of columns in a row is large. We need a more efficient API.

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


[jira] Updated: (CASSANDRA-172) A improved and more general version of get_slice

Posted by "Jun Rao (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-172?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Jun Rao updated CASSANDRA-172:
------------------------------

    Attachment: get_slice_from.patchv1

Propose a new API:
  list<column_t> get_slice_from(1:string tablename, 2:string key, 3:string columnFamily_column, 4:bool isAscending, 5:i32 start, 6:i32 count)

get_slice_from gets a specified number of columns starting from a specified starting column. For example, 
   get_slice_from(Table1, row1, CF1:col5, true, 0, 5) 
returns the first 5 columns in CF1 in row1 whose names are larger than or equal to col5. 

Because get_slice_from specifies a starting column, column index can be exploited to reduce the amount of data scanned in sstables.


Submit a patch. In this patch, get_slice_from is only supported on Standard CF sorted by Name.

Overview of the patch
ColumnFamilyStore : a new method getSliceFrom that does a multi-way merge among memtables and sstables.

ColumnIterator: interface for iterating through columns from a starting column; both memtable and sstable have an implementation of that interface

SliceFromReadCommand: a new type of read command

SequenceFile: a new ColumnSliceBlockReader that skips to an initial data block using the column index and starts iterating the blocks after or before the initial block


> A improved and more general version of get_slice
> ------------------------------------------------
>
>                 Key: CASSANDRA-172
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-172
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jun Rao
>            Assignee: Jun Rao
>         Attachments: get_slice_from.patchv1
>
>
> Today, get_slice has to scan through all columns in every memtable and sstable to get a slice of columns. This becomes inefficient when the number of columns in a row is large. We need a more efficient API.

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


[jira] Commented: (CASSANDRA-172) A improved and more general version of get_slice

Posted by "Jun Rao (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-172?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12711409#action_12711409 ] 

Jun Rao commented on CASSANDRA-172:
-----------------------------------

172-addendum-SF.patch looks fine to me.


> A improved and more general version of get_slice
> ------------------------------------------------
>
>                 Key: CASSANDRA-172
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-172
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jun Rao
>            Assignee: Jun Rao
>             Fix For: 0.4
>
>         Attachments: 172-addendum-SF.patch, 172-addendum.patch, get_slice_from.patchv1, get_slice_from.patchv2, get_slice_from.patchv3
>
>
> Today, get_slice has to scan through all columns in every memtable and sstable to get a slice of columns. This becomes inefficient when the number of columns in a row is large. We need a more efficient API.

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


[jira] Updated: (CASSANDRA-172) A improved and more general version of get_slice

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-172?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Jonathan Ellis updated CASSANDRA-172:
-------------------------------------

    Attachment: 172-addendum-SF.patch

addendum to SF part of the patch.  

> A improved and more general version of get_slice
> ------------------------------------------------
>
>                 Key: CASSANDRA-172
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-172
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jun Rao
>            Assignee: Jun Rao
>             Fix For: 0.4
>
>         Attachments: 172-addendum-SF.patch, 172-addendum.patch, get_slice_from.patchv1, get_slice_from.patchv2, get_slice_from.patchv3
>
>
> Today, get_slice has to scan through all columns in every memtable and sstable to get a slice of columns. This becomes inefficient when the number of columns in a row is large. We need a more efficient API.

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


[jira] Updated: (CASSANDRA-172) A improved and more general version of get_slice

Posted by "Jun Rao (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-172?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Jun Rao updated CASSANDRA-172:
------------------------------

    Attachment: get_slice_from.patchv2

Attache patch v2.

1. Renamed <start, count> to <offset, limit>. Also set the default value of offset to 0 since that is likely the common usage. In general, the offset parameter allows one to efficiently implement scrolling to an arbitrary positition of the returned results. This kind of functionality is more efficient if provided on the server side.

3. On second thought, I don't think the descending ordering necessarily introduces extra disk seeks. This is because each SSTable is always accessed one block at a time, independent of the ordering. A block is always asked sequentially and therfore, there is likely just one seek for the whole block. Across blocks, there are likely seeks, again independent of the ordering. This is because we access all SSTables interleavingly. So, keep the ordering option since it makes this api more general.

4. Switched from CF iterator to Column iterator and reused the CF template for all columns iterated.

5. ColumnSliceBlockReader still returns CF + columns. The CF deserialization overhead is amortized among a block of columns and shouldn't be too much. This api can stay simple this way.

Addressed the rest of the comments and rebased the code.


> A improved and more general version of get_slice
> ------------------------------------------------
>
>                 Key: CASSANDRA-172
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-172
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jun Rao
>            Assignee: Jun Rao
>             Fix For: 0.4
>
>         Attachments: get_slice_from.patchv1, get_slice_from.patchv2
>
>
> Today, get_slice has to scan through all columns in every memtable and sstable to get a slice of columns. This becomes inefficient when the number of columns in a row is large. We need a more efficient API.

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


[jira] Commented: (CASSANDRA-172) A improved and more general version of get_slice

Posted by "Jun Rao (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-172?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12711193#action_12711193 ] 

Jun Rao commented on CASSANDRA-172:
-----------------------------------

The logic in CFS.getSliceFrom in the addendum is not quite right. Here is a counter example. Suppose that we did the following:
insert (col1, val1, 1)
flush
delete(col1,           2)
flush
insert(col1, val3,  3)

Suppose that we call get_slice_from(col1, isAscending=true, count=1). In CFS.getSliceFrom, we will get 3 iterators, each of them iterates a version of col1. The problem is that depending on the ordering that PQ pops those versions (which is arbitrary), you get different liveColumns. If version 2 (the deletion) gets iterated first , liveColumn is 0 in the end, which is incorrect (even though the version is added to returnCF correctly).


> A improved and more general version of get_slice
> ------------------------------------------------
>
>                 Key: CASSANDRA-172
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-172
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jun Rao
>            Assignee: Jun Rao
>             Fix For: 0.4
>
>         Attachments: 172-addendum.patch, get_slice_from.patchv1, get_slice_from.patchv2, get_slice_from.patchv3
>
>
> Today, get_slice has to scan through all columns in every memtable and sstable to get a slice of columns. This becomes inefficient when the number of columns in a row is large. We need a more efficient API.

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


[jira] Updated: (CASSANDRA-172) A improved and more general version of get_slice

Posted by "Michael Greene (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-172?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Michael Greene updated CASSANDRA-172:
-------------------------------------

    Component/s: Core

> A improved and more general version of get_slice
> ------------------------------------------------
>
>                 Key: CASSANDRA-172
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-172
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Jun Rao
>            Assignee: Jun Rao
>             Fix For: 0.4
>
>         Attachments: 172-addendum-SF.patch, 172-addendum-v2.patch, 172-addendum.patch, get_slice_from.patchv1, get_slice_from.patchv2, get_slice_from.patchv3
>
>
> Today, get_slice has to scan through all columns in every memtable and sstable to get a slice of columns. This becomes inefficient when the number of columns in a row is large. We need a more efficient API.

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


[jira] Commented: (CASSANDRA-172) A improved and more general version of get_slice

Posted by "Jun Rao (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-172?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12711300#action_12711300 ] 

Jun Rao commented on CASSANDRA-172:
-----------------------------------

The code you added doesn't work either. Suppose you have following:
insert (col1, val1, 1)
flush
delete(col1, 2)

and version 1 is iterated first, liveColumns is 1, which is incorrect.

In general, I don't think that you can write any code that works correctly by just increasing liveColumn. This is because you could see an obsolete version first and need be prepared to take it back later. You can fix the problem by having logic for both increasing and decreasing liveColumn. Then, the code is no longer much simpler than the my version. 


> A improved and more general version of get_slice
> ------------------------------------------------
>
>                 Key: CASSANDRA-172
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-172
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jun Rao
>            Assignee: Jun Rao
>             Fix For: 0.4
>
>         Attachments: 172-addendum-SF.patch, 172-addendum.patch, get_slice_from.patchv1, get_slice_from.patchv2, get_slice_from.patchv3
>
>
> Today, get_slice has to scan through all columns in every memtable and sstable to get a slice of columns. This becomes inefficient when the number of columns in a row is large. We need a more efficient API.

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


[jira] Commented: (CASSANDRA-172) A improved and more general version of get_slice

Posted by "Jun Rao (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-172?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12710006#action_12710006 ] 

Jun Rao commented on CASSANDRA-172:
-----------------------------------

1. Regarding the start parameter in the api. 
One of the applications of this api is to support an index scan query. Then, (start, count) is the equivalent of (offset,limit) in mysql for this query. Without this, the client may have to fetch more data simply to throw some of them away. I suggest that we keep start and set its default value to 0. What do you think?

2. Regarding the terminology "block" used in SSTable.
The ColumnSliceBlockReader returned in SSTable is actually returning a block of columns at a time. This is because the column index within a row is a block index.

3. Regarding the isAscending parameter. 
I agree that going backwards could have worse performance. I can take that out. The implication is that if an application needs to scan an index in both directions, two separate indexes may have to be created.

4. Regarding Cursor and ColumnIterator returning a CF, instead of a Column.
The reason that ColumnIterator returns a CF containing a single column is to deal with the possibility that a CF itself can be marked deleted. If ColumnIterator only returns Column, then we need to deal with CF deletion separately and this is likely not as clean as the current approach.

Agree with the rest of the comments. Will submit a new patch.


> A improved and more general version of get_slice
> ------------------------------------------------
>
>                 Key: CASSANDRA-172
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-172
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jun Rao
>            Assignee: Jun Rao
>             Fix For: 0.4
>
>         Attachments: get_slice_from.patchv1
>
>
> Today, get_slice has to scan through all columns in every memtable and sstable to get a slice of columns. This becomes inefficient when the number of columns in a row is large. We need a more efficient API.

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


[jira] Commented: (CASSANDRA-172) A improved and more general version of get_slice

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-172?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12710458#action_12710458 ] 

Jonathan Ellis commented on CASSANDRA-172:
------------------------------------------

Okay, that makes sense, although I repeat my suggestion to call these ColumnGroups, not blocks.  I think seek-per-group is reasonable to avoid the pain of creating a second CF to order in reverse, so +1 for descending functionalty.

The more I think about the CF wrapper though the more I think you don't need it.  Just update the CF tombstones as you collect iterators from the CFs, and use addColumn as I suggested.  This will also fix the bug where you don't update tombstones at all if there are no columns found.

Style notes:

prefer while (true) {} to do { } while (true); both loop infinitely but the first you don't have to skip to the bottom to see what the loop condition is.

prefer declaring variables close to where they are used (e.g. IColumn retColumn in the iterator code); the more limited the scope of a var is, the easier it is to see what it's used for.

> A improved and more general version of get_slice
> ------------------------------------------------
>
>                 Key: CASSANDRA-172
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-172
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jun Rao
>            Assignee: Jun Rao
>             Fix For: 0.4
>
>         Attachments: get_slice_from.patchv1, get_slice_from.patchv2
>
>
> Today, get_slice has to scan through all columns in every memtable and sstable to get a slice of columns. This becomes inefficient when the number of columns in a row is large. We need a more efficient API.

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


[jira] Commented: (CASSANDRA-172) A improved and more general version of get_slice

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-172?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12709599#action_12709599 ] 

Jonathan Ellis commented on CASSANDRA-172:
------------------------------------------

ambitious patch. :)  i think it is going in the right direction.

minor stuff:

indentation is wonky in places.  4 spaces is the standard.

since we're using the column from CF_column to seek, let's drop the `start` int.  It's confusing to have two offsets in one api call.

`Block` is used in SSTable to mean the entire set of columns per key, and here you are using to mean just one column at a time, correct?  Can we use a different term in your new Reader?  (Just Column would be fine IMO.)

let's use consistent naming w/ existing code, e.g. bufOut vs outBuf in SF.  (i agree that outBuf is better, feel free to convert the existing code to that in a separate patch first.)

your Reader should inherit BufferedReader not just Reader.  buffer size of at least 128KB.  reading that much at once is basically free after the seek penalty.

getUnflushedMemtables never returns null, you can skip that check.

do you have any other plans for cleaning up SF?  it's a mess and it makes me sad to add code there :)

major stuff:

isAscending should not go in, it's going to do vast amounts of seeks when going backwards and that's what we absolutely want to avoid for performance.  What we should do instead is allow specifying asc/desc sort in the CF definition.  i suggest taking out the isAsc stuff from this patch and doing the CF change in a separate ticket.

(I'm not sure if BufferedRandomAccessFile is smart enough to avoid actually hitting disk if it's told to seek to a position it's already buffered.  But just in case we should probably put the seek in init() to emphasize that we're only forcing a seek once.)

Cursor & ColumnIterator  should deal in  Columns (can it be generalized to SC just by using IColumn?  i think it can) not CFs each holding one column.  Then use CF.addColumn as you iterate. and do a final resolve&RD once at the end.  this will make all the resolve stuff go away since addColumn will Just Work.

the Memtable code needs updating now that we're not using an executor.  (see CASSANDRA-165).  basically, accesses to the active MT should go through a lock via CFS.  historical ones can be accessed directly w/o lock.

> A improved and more general version of get_slice
> ------------------------------------------------
>
>                 Key: CASSANDRA-172
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-172
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jun Rao
>            Assignee: Jun Rao
>             Fix For: 0.4
>
>         Attachments: get_slice_from.patchv1
>
>
> Today, get_slice has to scan through all columns in every memtable and sstable to get a slice of columns. This becomes inefficient when the number of columns in a row is large. We need a more efficient API.

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


[jira] Commented: (CASSANDRA-172) A improved and more general version of get_slice

Posted by "Jun Rao (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-172?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12710491#action_12710491 ] 

Jun Rao commented on CASSANDRA-172:
-----------------------------------

For the CF wrapper, I need to resolve a column at a time so that I know when I have enough columns to return (since a column could be deleted). I am not exactly sure how to apply your addColumn suggestion, since there is no place to call a single final resolve.

If all iterators are empty, an empty CF will be returned. I don't really need to update the tombstone in the CF unless for read-repair considerations. However, the current read-repair logic doesn't deal with CF removal.


> A improved and more general version of get_slice
> ------------------------------------------------
>
>                 Key: CASSANDRA-172
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-172
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jun Rao
>            Assignee: Jun Rao
>             Fix For: 0.4
>
>         Attachments: get_slice_from.patchv1, get_slice_from.patchv2
>
>
> Today, get_slice has to scan through all columns in every memtable and sstable to get a slice of columns. This becomes inefficient when the number of columns in a row is large. We need a more efficient API.

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


[jira] Commented: (CASSANDRA-172) A improved and more general version of get_slice

Posted by "Jun Rao (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-172?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12710441#action_12710441 ] 

Jun Rao commented on CASSANDRA-172:
-----------------------------------

3. Within an SSTable, for a given (key,CF) pair, all columns are stored contiguously in a segment. The column index is created roughly as follows. Everytime the size of the accumulated columns reaches a threshold, a column index entry is created, which has the column name and an offset within the segment. I call this a (column) block index. In the code, instead of reading all columns from an SSTable at once, I read the columns a block at a time, using the column index. This is more efficient if the number of columns in an SSTable is large (that's the key difference between this api and get_slice). Depending on how many columns are needed in this api, multiple column blocks will need to be fetched from a sinlge SSTable. Does that explain things?


> A improved and more general version of get_slice
> ------------------------------------------------
>
>                 Key: CASSANDRA-172
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-172
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jun Rao
>            Assignee: Jun Rao
>             Fix For: 0.4
>
>         Attachments: get_slice_from.patchv1, get_slice_from.patchv2
>
>
> Today, get_slice has to scan through all columns in every memtable and sstable to get a slice of columns. This becomes inefficient when the number of columns in a row is large. We need a more efficient API.

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


[jira] Commented: (CASSANDRA-172) A improved and more general version of get_slice

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-172?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12710434#action_12710434 ] 

Jonathan Ellis commented on CASSANDRA-172:
------------------------------------------

1. still a strong -1 on this

3. My understanding is that there can only be one instance of a key per sstable (b/c of how flush works) and for each key you have first the column index, then the columns serialized contiguously (b/c of how CF.serializerWithIndexes works).  Therefore all columns for a given key/CF pair are in a single block (group of keys that are indexed together as the term is used in the header of SSTable).  So (a) how can we need multiple seeks to read columns from a single key, and (b) isn't this tremendously overcomplicated?  I thought this was fixing the "must deserialize all of the CF at once" problem but on second look it mostly does not.  What am I missing?  Are we using the word block to mean different things as I initially thought?

4/5.  The CF wrapper still seems like the opposite of simple here to me but let us see how 3 shakes out.

> A improved and more general version of get_slice
> ------------------------------------------------
>
>                 Key: CASSANDRA-172
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-172
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jun Rao
>            Assignee: Jun Rao
>             Fix For: 0.4
>
>         Attachments: get_slice_from.patchv1, get_slice_from.patchv2
>
>
> Today, get_slice has to scan through all columns in every memtable and sstable to get a slice of columns. This becomes inefficient when the number of columns in a row is large. We need a more efficient API.

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


[jira] Updated: (CASSANDRA-172) A improved and more general version of get_slice

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-172?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Jonathan Ellis updated CASSANDRA-172:
-------------------------------------

    Attachment: 172-addendum-v2.patch

You are right, my way is not much of an improvement in that what it gains from a simpler loop it loses in duplicating too much of the addColumn internals.

So here is a third approach.  I add a ReducingIterator that is in charge of merging runs of "equal" objects into a single one.  (This is also useful in getKeyRange, and probably other places).  Then we have our reduction be an addColumn, and post-reduce all we have to do is iterate the results and check if the merged column is deleted.

I also use AbstractIterator from google collections (http://google-collections.googlecode.com/svn/trunk/javadoc/com/google/common/collect/AbstractIterator.html) to make defining "real" iterators from the SF easier (and similarly in FileStruct).  You can grab a copy here: http://google-collections.googlecode.com/files/google-collect-1.0-rc1.zip

This works out to be slightly more verbose (in part to our damned brace-on-new-line policy) but I think it's more maintainable overall since each step in the process is now easily understood at a glance.

> A improved and more general version of get_slice
> ------------------------------------------------
>
>                 Key: CASSANDRA-172
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-172
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jun Rao
>            Assignee: Jun Rao
>             Fix For: 0.4
>
>         Attachments: 172-addendum-SF.patch, 172-addendum-v2.patch, 172-addendum.patch, get_slice_from.patchv1, get_slice_from.patchv2, get_slice_from.patchv3
>
>
> Today, get_slice has to scan through all columns in every memtable and sstable to get a slice of columns. This becomes inefficient when the number of columns in a row is large. We need a more efficient API.

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


[jira] Updated: (CASSANDRA-172) A improved and more general version of get_slice

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-172?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Jonathan Ellis updated CASSANDRA-172:
-------------------------------------

    Attachment: 172-addendum.patch

Here is an addendum to v3.  Primarily it cleans up the memtable iterator and getSliceFrom.  It also merges Cursor functionally (basically, just getCurrent) into ColumnIterator.  What do you think?

I will look at the SF code more closely now.

> A improved and more general version of get_slice
> ------------------------------------------------
>
>                 Key: CASSANDRA-172
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-172
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jun Rao
>            Assignee: Jun Rao
>             Fix For: 0.4
>
>         Attachments: 172-addendum.patch, get_slice_from.patchv1, get_slice_from.patchv2, get_slice_from.patchv3
>
>
> Today, get_slice has to scan through all columns in every memtable and sstable to get a slice of columns. This becomes inefficient when the number of columns in a row is large. We need a more efficient API.

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


[jira] Commented: (CASSANDRA-172) A improved and more general version of get_slice

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-172?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12712135#action_12712135 ] 

Jonathan Ellis commented on CASSANDRA-172:
------------------------------------------

thanks, that computeNext is better.

good point on the comparator.  i also improved the tests to catch that.

the default UncaughtExceptionHandler will log the exception, so logging it a second time would be redundant (and logging instead of throwing is bad for testing).

the comparator bug reminds me that we still have a bug wrt descending queries from the client's perspective, but I think that can be tackled separately (in CASSANDRA-196).  I'll commit what we have here in the mean time.

> A improved and more general version of get_slice
> ------------------------------------------------
>
>                 Key: CASSANDRA-172
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-172
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jun Rao
>            Assignee: Jun Rao
>             Fix For: 0.4
>
>         Attachments: 172-addendum-SF.patch, 172-addendum-v2.patch, 172-addendum.patch, get_slice_from.patchv1, get_slice_from.patchv2, get_slice_from.patchv3
>
>
> Today, get_slice has to scan through all columns in every memtable and sstable to get a slice of columns. This becomes inefficient when the number of columns in a row is large. We need a more efficient API.

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


[jira] Updated: (CASSANDRA-172) A improved and more general version of get_slice

Posted by "Jun Rao (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-172?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Jun Rao updated CASSANDRA-172:
------------------------------

    Attachment: get_slice_from.patchv3

Attach patch v3.  Avoided calling CF.resolve. Took out offset from api. Changed coding style in a few places. Renamed ColumnSliceBlockReader to ColumnGroupReader.

Regarding the issue of including an offset in the API. Is your main concern that the api becomes less clear? The code complexity to support offset is minimal.

> A improved and more general version of get_slice
> ------------------------------------------------
>
>                 Key: CASSANDRA-172
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-172
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jun Rao
>            Assignee: Jun Rao
>             Fix For: 0.4
>
>         Attachments: get_slice_from.patchv1, get_slice_from.patchv2, get_slice_from.patchv3
>
>
> Today, get_slice has to scan through all columns in every memtable and sstable to get a slice of columns. This becomes inefficient when the number of columns in a row is large. We need a more efficient API.

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


[jira] Commented: (CASSANDRA-172) A improved and more general version of get_slice

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-172?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12711230#action_12711230 ] 

Jonathan Ellis commented on CASSANDRA-172:
------------------------------------------

Good catch.  I think we can fix that by changing the liveColumns update to

                    IColumn oldColumn = returnCF.getColumn(column.name());
                    // increment "live" count (the one that will be seen by the client) if we either
                    // (a) add a live column for the first time, or (b) replace a tombstone column w/ a live one.
                    if (!column.isMarkedForDelete() 
                        && (oldColumn == null || (oldColumn.isMarkedForDelete() && column.timestamp() >= oldColumn.timestamp())))
                    {
                        liveColumns++;
                    }

better?

> A improved and more general version of get_slice
> ------------------------------------------------
>
>                 Key: CASSANDRA-172
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-172
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jun Rao
>            Assignee: Jun Rao
>             Fix For: 0.4
>
>         Attachments: 172-addendum-SF.patch, 172-addendum.patch, get_slice_from.patchv1, get_slice_from.patchv2, get_slice_from.patchv3
>
>
> Today, get_slice has to scan through all columns in every memtable and sstable to get a slice of columns. This becomes inefficient when the number of columns in a row is large. We need a more efficient API.

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


[jira] Commented: (CASSANDRA-172) A improved and more general version of get_slice

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-172?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12710008#action_12710008 ] 

Jonathan Ellis commented on CASSANDRA-172:
------------------------------------------

1. By "fetch more data" you mean "fetch exactly one extra column" because you pass the last column returned as the start column for the next "page," right?  I think that is better than having a separate offset parameter (especially since new columns could have been inserted, and you are less likely to get "wrong" results when specifying a column name than an offset).

2. ok

3. Right, two separate indexes is definitely more in keeping with "the zen of cassandra" :)

4. That's true, which is an additional point in favor of doing CFs separately.  You have to go through some contortions to return CF information along w/ each column block which is clunky as well as inefficient (from re-copying out the CF wrapper and deserializing repeatedly).  It will definitely be more performant to handle the CFs and IColumns separately and I think overall it will end up simpler too.  If you like you can try making this change last and submit both versions and if I am wrong we'll go with the CF wrapper in the iterator.

> A improved and more general version of get_slice
> ------------------------------------------------
>
>                 Key: CASSANDRA-172
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-172
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jun Rao
>            Assignee: Jun Rao
>             Fix For: 0.4
>
>         Attachments: get_slice_from.patchv1
>
>
> Today, get_slice has to scan through all columns in every memtable and sstable to get a slice of columns. This becomes inefficient when the number of columns in a row is large. We need a more efficient API.

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


[jira] Commented: (CASSANDRA-172) A improved and more general version of get_slice

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-172?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12711162#action_12711162 ] 

Jonathan Ellis commented on CASSANDRA-172:
------------------------------------------

(the other objection is that it seems like an open invitation to use the API inefficiently, instead of  seeking to the real offset explicitly.)

> A improved and more general version of get_slice
> ------------------------------------------------
>
>                 Key: CASSANDRA-172
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-172
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jun Rao
>            Assignee: Jun Rao
>             Fix For: 0.4
>
>         Attachments: 172-addendum.patch, get_slice_from.patchv1, get_slice_from.patchv2, get_slice_from.patchv3
>
>
> Today, get_slice has to scan through all columns in every memtable and sstable to get a slice of columns. This becomes inefficient when the number of columns in a row is large. We need a more efficient API.

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


[jira] Commented: (CASSANDRA-172) A improved and more general version of get_slice

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-172?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12710500#action_12710500 ] 

Jonathan Ellis commented on CASSANDRA-172:
------------------------------------------

> For the CF wrapper, I need to resolve a column at a time so that I know when I have enough columns to return (since a column could be deleted). 

It's O(N^2) to do a "for 1..N: L.add(cf2); resolve(L)" so that really needs to be avoided.  (even just removeDeleted is O(N) so don't nest that either.)  SC removeDeleted in particular does a ton of allocations, it's not intended to be called repeatedly.

Since we are only concerned about counting top-level IColumns (even if this is called on a super CF) then manually resolving is easy: "if column is live: cf.addColumn(column)"

> I am not exactly sure how to apply your addColumn suggestion, since there is no place to call a single final resolve. 

I don't understand the dillemma.  building up a CF as described and then "return removeDeleted(cf)" takes care of it.  There is no resolving to do this way.

> If all iterators are empty, an empty CF will be returned. I don't really need to update the tombstone in the CF unless for read-repair considerations. However, the current read-repair logic doesn't deal with CF removal. 

Better to preserve correct behavior re CF tombstones when returning a CF than be surprised by it later, especially when there is no pain in doing so.  No?


> A improved and more general version of get_slice
> ------------------------------------------------
>
>                 Key: CASSANDRA-172
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-172
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jun Rao
>            Assignee: Jun Rao
>             Fix For: 0.4
>
>         Attachments: get_slice_from.patchv1, get_slice_from.patchv2
>
>
> Today, get_slice has to scan through all columns in every memtable and sstable to get a slice of columns. This becomes inefficient when the number of columns in a row is large. We need a more efficient API.

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


[jira] Commented: (CASSANDRA-172) A improved and more general version of get_slice

Posted by "Hudson (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-172?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12712416#action_12712416 ] 

Hudson commented on CASSANDRA-172:
----------------------------------

Integrated in Cassandra #85 (See [http://hudson.zones.apache.org/hudson/job/Cassandra/85/])
    Add get_slice_from functionality using column indexes for efficiency.
Patch by Jun Rao and jbellis for 


> A improved and more general version of get_slice
> ------------------------------------------------
>
>                 Key: CASSANDRA-172
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-172
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jun Rao
>            Assignee: Jun Rao
>             Fix For: 0.4
>
>         Attachments: 172-addendum-SF.patch, 172-addendum-v2.patch, 172-addendum.patch, get_slice_from.patchv1, get_slice_from.patchv2, get_slice_from.patchv3
>
>
> Today, get_slice has to scan through all columns in every memtable and sstable to get a slice of columns. This becomes inefficient when the number of columns in a row is large. We need a more efficient API.

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


[jira] Commented: (CASSANDRA-172) A improved and more general version of get_slice

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-172?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12710877#action_12710877 ] 

Jonathan Ellis commented on CASSANDRA-172:
------------------------------------------

(oh, and yes, my objection to the offset code is primarily an API cleanliness one.)

> A improved and more general version of get_slice
> ------------------------------------------------
>
>                 Key: CASSANDRA-172
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-172
>             Project: Cassandra
>          Issue Type: New Feature
>            Reporter: Jun Rao
>            Assignee: Jun Rao
>             Fix For: 0.4
>
>         Attachments: 172-addendum.patch, get_slice_from.patchv1, get_slice_from.patchv2, get_slice_from.patchv3
>
>
> Today, get_slice has to scan through all columns in every memtable and sstable to get a slice of columns. This becomes inefficient when the number of columns in a row is large. We need a more efficient API.

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