You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Jonathan Ellis (JIRA)" <ji...@apache.org> on 2009/05/14 23:33:45 UTC

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

    [ 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.