[
https://issues.apache.org/jira/browse/CASSANDRA-81?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Jonathan Ellis updated CASSANDRA-81:
------------------------------------
Attachment: 81-v5.patch
I cleaned up a couple things in CountFilter.
1. use this() for constructor overloading instead of pasting code
2. values.length == 1 is _not_ the same as !isSuper(). You could be slicing
supercolumns of a super CF, or you could be slicing normal columns of a
standard CF. (This is an easy mistake to make because the API is designed
poorly. But fixing that is a job for 0.4 I think.)
testGetRowSuperColumnOffsetCount errors out b/c of cleanup problems. When I
applied it after CASSANDRA-85 (which improves cleanup considerably), the test
fails. (Possibly my fault for sloppy conflict resolution.)
I suggest we apply 85 first to at least get to a reproducible state in the test.
> 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: 0.3
>
> Attachments: 81-v5.patch, fix_for_get_slice.patch,
> get_slice_fix_and_unit_tests_v2.patch, get_slice_fix_and_unit_tests_v3.patch,
> get_slice_fix_and_unit_tests_v4.patch, unit_tests_for_get_slice.patch
>
>
> 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.