[
https://issues.apache.org/jira/browse/CASSANDRA-781?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12832827#action_12832827
]
bjc commented on CASSANDRA-781:
-------------------------------
I don't think my test is buggy. I realize that the range is start inclusive,
and it does pass a key that exists as start, but sets "done = True" if the
range scan returns less keys than requested. Since it passes "" as the
end/finish, this should return less keys than requested when you get to the
end, provided you ask for more than one key (which I do).
I think the last remaining problem is with the sorting! I bet that is why using
"" for my finish string doesn't work. Here's the problem I see now (transcript
followed by test):
I put 10 random keys in, ask for them back. They aren't sorted, so I sort them
and take the highest. I use that as start, and "" as finish. This should give
me one key back, but instead I get 10. Could it be that my columnfamily
definition is different than yours? Here's mine:
<ColumnFamily ColumnType="Super"
CompareWith="UTF8Type"
CompareSubcolumnsWith="UTF8Type"
Name="Super1"
RowsCached="1000"
KeysCachedFraction="0"
Comment="A column family with supercolumns, whose column
and subcolumn names are UTF8 strings"/>
In [17]: run test_bug_simple2.py
result1 before sorting
af37b718213b4219897ea1564ebc8900
f196ad5537294840b2de0a636202dbd2
7578ba38b66d4708a38663717e020959
b5266af926a647c3a1a4d2f62dfe952c
d729d5181bac42a48ac3e49d9700047e
4d58b6fbea214d0c9c7a9f288feba2d8
41df7aee7d674a75a4943d89153f9bde
4e121f95459e4f67a6cd3c06b2d078e7
99b2b03675a8413f94e60e3d1bbded8c
66121c5c863f4c1f804a46b8c2136fe9
result1 after sorting
41df7aee7d674a75a4943d89153f9bde
4d58b6fbea214d0c9c7a9f288feba2d8
4e121f95459e4f67a6cd3c06b2d078e7
66121c5c863f4c1f804a46b8c2136fe9
7578ba38b66d4708a38663717e020959
99b2b03675a8413f94e60e3d1bbded8c
af37b718213b4219897ea1564ebc8900
b5266af926a647c3a1a4d2f62dfe952c
d729d5181bac42a48ac3e49d9700047e
f196ad5537294840b2de0a636202dbd2
start f196ad5537294840b2de0a636202dbd2
result2
f196ad5537294840b2de0a636202dbd2
7578ba38b66d4708a38663717e020959
b5266af926a647c3a1a4d2f62dfe952c
d729d5181bac42a48ac3e49d9700047e
4d58b6fbea214d0c9c7a9f288feba2d8
41df7aee7d674a75a4943d89153f9bde
4e121f95459e4f67a6cd3c06b2d078e7
99b2b03675a8413f94e60e3d1bbded8c
66121c5c863f4c1f804a46b8c2136fe9
b8b290a864464271ad30df1bbab2f2b7
---------------------------------------------------------------------------
AssertionError Traceback (most recent call last)
/k/jack/bridge/test_bug_simple2.py in <module>()
54 for r in result2: print r.key
55
---> 56 assert len(result2) == 1
57
58
AssertionError:
WARNING: Failure executing file: <test_bug_simple2.py>
In [18]:
import uuid
from thrift import Thrift
from thrift.transport import TTransport
from thrift.transport import TSocket
from thrift.protocol.TBinaryProtocol import TBinaryProtocolAccelerated
import sys
sys.path.insert(0,'/usr/local/cassandra/interface/thrift/gen-py')
from cassandra import Cassandra
from cassandra.ttypes import *
socket = TSocket.TSocket("10.212.87.165", 9160)
transport = TTransport.TBufferedTransport(socket)
protocol = TBinaryProtocol.TBinaryProtocolAccelerated(transport)
client = Cassandra.Client(protocol)
transport.open()
ks = "Keyspace1"
cf = "Super1"
path = ColumnPath(cf, "foo", "is")
value = "cool"
for i in xrange(100):
key = uuid.uuid4().hex
client.insert(ks, key, path, value, 0, ConsistencyLevel.ONE)
parent = ColumnParent(column_family=cf)
slice_range = SliceRange(start="key", finish="key")
predicate = SlicePredicate(slice_range=slice_range)
result1 = client.get_range_slice(ks, parent, predicate, "", "", 10,
ConsistencyLevel.ONE)
print "result1 before sorting"
for r in result1: print r.key
def getkey(x): return x.key
print "result1 after sorting"
result1 = map(getkey, result1)
result1.sort()
for r in result1: print r
start = result1[-1]
print "start", start
result2 = client.get_range_slice(ks, parent, predicate, start, "", 10,
ConsistencyLevel.ONE)
print "result2"
for r in result2: print r.key
assert len(result2) == 1
> in a cluster, get_range_slice() does not return all the keys it should
> ----------------------------------------------------------------------
>
> Key: CASSANDRA-781
> URL: https://issues.apache.org/jira/browse/CASSANDRA-781
> Project: Cassandra
> Issue Type: Bug
> Affects Versions: 0.5
> Environment: Debian 5 lenny on EC2, Gentoo linux, Windows XP
> Reporter: bjc
> Assignee: Jonathan Ellis
> Fix For: 0.5, 0.6
>
> Attachments: 781.txt
>
>
> get_range_slice() does not return the same set of keys as get_key_range() in
> 0.5.0 final.
> I posted a program to reproduce the behavior:
> http://www.mail-archive.com/[email protected]/msg01474.html
> Apparently, you must have more than one node to get the behavior. Also, it
> may depend on the locations of the nodes on the ring.. I.e., if you don't
> generate enough keys randomly, then by chance they could all fall on the same
> host and you might not see the behavior, although I was able to get it to
> happen using only 2 nodes and 10 keys.
> Here are the other emails describing the issue:
> http://www.mail-archive.com/[email protected]/msg02423.html
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.