cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "bjc (JIRA)" <j...@apache.org>
Subject [jira] Commented: (CASSANDRA-781) in a cluster, get_range_slice() does not return all the keys it should
Date Thu, 11 Feb 2010 06:53:28 GMT

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

bjc commented on CASSANDRA-781:
-------------------------------

Almost there I think. I found that the keys were not being returned in sorted order as they
were before, so my trick of taking the last key in a limited range, and using that to start
the next limited range did not work. So, I modified the test to sort the keys, then take the
last one. When I did this I found another bug: when there are fewer keys in the specified
range, duplicates are returned. Also, when I played around with the start and end for the
range the server starting giving AssertionErrors:

ERROR 06:25:41,032 Internal error processing get_range_slice
java.lang.AssertionError: [125358492461525499902293558181143752059,1244525150549
22950280650433865080672503]
        at org.apache.cassandra.dht.Bounds.<init>(Bounds.java:26)
        at org.apache.cassandra.dht.Bounds.<init>(Bounds.java:18)
        at org.apache.cassandra.thrift.CassandraServer.get_range_slice(Cassandra
Server.java:558)
        at org.apache.cassandra.thrift.Cassandra$Processor$get_range_slice.proce
ss(Cassandra.java:1189)
        at org.apache.cassandra.thrift.Cassandra$Processor.process(Cassandra.jav
a:984)
        at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadP
oolServer.java:253)
        at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExec
utor.java:886)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor
.java:908)
        at java.lang.Thread.run(Thread.java:619)


The modified test (below) gets stuck in an infinite loop. If you run it in ipython, you can
control-c to get back to the interpretor, then type "result" to look at it's contents. It
is a sorted list of keys. Look at the last key.. Here is the transcript showing the last two
keys from result:

 'ff8bfa30777f455695bf934ac7cfedac',
 'ffb701ea740646b9955f0e339f8e3ee2']

In [70]: result2 = client.get_range_slice(ks, cparent, p, start, "", seg, cl)

In [71]: len(result2)
Out[71]: 1000

In [72]: result3 = client.get_range_slice(ks, cparent, p, start, "", seg, cl)

In [73]: len(result3)
Out[73]: 1000

In [74]: start
Out[74]: 'ffb701ea740646b9955f0e339f8e3ee2'

In [75]: result4 = client.get_range_slice(ks, cparent, p, start, start, seg, cl)

In [76]: len(result4)
Out[76]: 1000

In [77]: result5 = client.get_range_slice(ks, cparent, p, start, start, seg, cl)

In [78]: len(result5)
Out[78]: 1

In [79]: 

That can't be right. Here is the latest test..

import sys
import time
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 *

num_keys = 10000

socket = TSocket.TSocket("10.212.87.165", 9160)
transport = TTransport.TBufferedTransport(socket)
protocol = TBinaryProtocol.TBinaryProtocolAccelerated(transport)
client = Cassandra.Client(protocol)

ks = "Keyspace1"
cf = "Super1"
cl = ConsistencyLevel.ONE

d = {}
    
transport.open()
    
if 1:
    ## insert keys using the raw thrift interface
    cpath = ColumnPath(cf, "foo", "is")
    value = "cool"

    for i in xrange(num_keys):
        ts = time.time()
        key = uuid.uuid4().hex
        client.insert(ks, key, cpath, value, ts, cl)
        d[key] = 1

else:
    ## insert keys using pycassa!
    import pycassa

    client = pycassa.connect(["10.212.87.165:9160"])
    cf_test = pycassa.ColumnFamily(client, ks, cf, super=True)

    for i in xrange(num_keys):
        key = uuid.uuid4().hex
        cf_test.insert(key, { 'params' : { 'is' : 'cool' }})
        d[key] = 1


cparent = ColumnParent(column_family=cf)
slice_range = SliceRange(start="key", finish="key")
p = SlicePredicate(slice_range=slice_range)

done = False
seg = 1000
start = ""

## do a scan using either get_key_range() (deprecated) or get_range_slice()
## for every key returned that is in the dictionary, mark it as found
while not done:
    print "start", start
    result = client.get_range_slice(ks, cparent, p, start, "", seg, cl)

    def getkey(x):
        return x.key
    result = map(getkey, result)   
    result.sort()

    for r in result:
        if d.has_key(r): 
            d[r] = 0

    if len(result) < seg: done = True
    else: start = result[seg-1]

cpath = ColumnPath(column_family=cf, super_column='foo')

## get, remove all the keys
## print all the keys that were not marked 0
for k in d:
    result = client.get(ks, k, cpath, cl)
    #print result

    if d[k] == 1: 
        print k, "not marked 0"
    #else:
    #    print k, "was marked 0!"

    ts = time.time()
    client.remove(ks, k, cpath, ts, cl)




BTW, this time around my nodetool worked perfectly! When I first brought the two nodes up,
they selected keys that were too close, and one node ended up with all the load. So I ran
loadbalance, and it worked great! That was really awesome. The only thing I noticed was a
single key that should have been found returned a NotFoundException. I'll keep an eye on this
one, too. Best,

Jack

> 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/cassandra-dev@incubator.apache.org/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/cassandra-user@incubator.apache.org/msg02423.html

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


Mime
View raw message