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