[ 
https://issues.apache.org/jira/browse/CASSANDRA-81?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12699032#action_12699032
 ] 

Sandeep Tata commented on CASSANDRA-81:
---------------------------------------

There's an another, more subtle problem with this bug. Even with start=0, the 
columns returned are not guaranteed to be in order.

Suppose that you:

write col2
write col3

flush -- the memtable is now empty, the ssTable contains col2 and col3
write col1
write col4
write col5

get_slice(table, colfam, 0, 3):

You should get col1, col2, col3 --> not col1, col4, col5 (from current memtable 
alone)


> get_slice ignores the "start" parameter
> ---------------------------------------
>
>                 Key: CASSANDRA-81
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-81
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: trunk
>         Environment: all
>            Reporter: Sandeep Tata
>            Assignee: Sandeep Tata
>             Fix For: trunk
>
>
> get_slice(string tablename, string key, string columnFamily_column, i32 
> start, i32 count) is expected is return all columns starting at offset 
> "start" subject to a maximum of "count" columns. The current code does not do 
> this.
> Example interaction:
> ./Cassandra-remote insert 'Table1' 'key' 'DATA:c1' 'val1' 1
> None
> ./Cassandra-remote insert 'Table1' 'key' 'DATA:c2' 'val2' 1
> None
> ./Cassandra-remote insert 'Table1' 'key' 'DATA:c3' 'val3' 1
> None
> ./Cassandra-remote get_slice 'Table1' 'key' 'DATA'  0 2
> [ {'columnName': 'c1', 'value': 'val1', 'timestamp': 1},
>   {'columnName': 'c2', 'value': 'val2', 'timestamp': 1}]
> ./Cassandra-remote get_slice 'Table1' 'key' 'DATA'  1 2
> [ {'columnName': 'c1', 'value': 'val1', 'timestamp': 1},
>   {'columnName': 'c2', 'value': 'val2', 'timestamp': 1}]]  <<---- Same as 
> prev! "start" ignored
> ./Cassandra-remote  get_slice 'Table1' 'key' 'DATA'  0 1
> [{'columnName': 'c1', 'value': 'val1', 'timestamp': 1}]
> ./Cassandra-remote get_slice 'Table1' 'key' 'DATA'  2 1
> [{'columnName': 'c1', 'value': 'val1', 'timestamp': 1}]    <<---- Same as 
> prev! "start" ignored

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