incubator-cassandra-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Jonathan Ellis <jbel...@gmail.com>
Subject Re: get_key_range() vs. get_range_slice() -- scan/counting errors
Date Mon, 08 Feb 2010 22:06:12 GMT
Can you create a ticket for this, please?  Thanks!

On Sat, Feb 6, 2010 at 7:05 PM, Jack Culpepper <jackculpepper@gmail.com> wrote:
> I did a bit more testing, and it does seem to be related to having two
> nodes. When I turn one node off and repeat the range scan, I get the
> same result, but if I start with only one node and do all the inserts
> and then a range scan, I get the correct count using
> get_range_slice().
>
> However, with two nodes there is a very easy way to replicate the
> problem. Just clear out your Test Keyspace and insert 1000 keys. For
> example, here I use pycassa to do that.
>
> if 1:
>    import pycassa
>    import uuid
>
>    client = pycassa.connect(["10.212.87.165:9160"])
>    cf_test = pycassa.ColumnFamily(client, "Test Keyspace", "Test
> Super", super=True)
>
>    for i in xrange(1000):
>        key = uuid.uuid4().hex
>        cf_test.insert(key, { 'params' : { 'is' : 'cool' }})
>        print key
>
> Hear me out before you argue that pycassa is the problem. I haven't
> actually done this using the raw thrift interface, but only the
> retrieval is problemic. You can run this code and pipe the output to a
> file to record all the keys that were inserted. Now use the regular
> thrift interface to try and get them back:
>
> if 1:
>    from thrift import Thrift
>    from thrift.transport import TTransport
>    from thrift.transport import TSocket
>    from thrift.protocol.TBinaryProtocol import TBinaryProtocolAccelerated
>    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()
>
>
>    column_parent = ColumnParent(column_family="Test Super")
>    slice_range = SliceRange(start="key", finish="key")
>    #slice_range = SliceRange(start="", finish="")
>    predicate = SlicePredicate(slice_range=slice_range)
>
>    done = False
>    seg = 1000
>    start = ""
>
>    while not done:
>        #result = client.get_key_range("Test Keyspace", "Test Super",
> start, "", seg, ConsistencyLevel.ONE)
>        result = client.get_range_slice("Test Keyspace",
> column_parent, predicate, start, "", seg, ConsistencyLevel.ONE)
>
>        if len(result) < seg: done = True
>        #else: start = result[seg-1]
>        else: start = result[seg-1].key
>
>
>        for r in result:
>            #print r
>            print r.key
>
> Using get_range_slice() I see only keys from
> 562ab7792af249be8e73ba2ace5a5888 to 9fd73cf2ab264571a5654c315ab6e93d,
> but with get_key_range() I see keys from
> 01b12cdae9464d1ab4cf2f89808883d9 to ffda307823ee43eeac590a3201b81962.
>
> That is, get_key_range() retrieves *all* the keys, but
> get_range_slice() does not. Thus, it seems unlikely that there is a
> problem with pycassa or the way I did my insertions if get_key_range()
> is able to work properly.
>
> I also just read through the "How to retrieve keys from Cassandra ?"
> thread. I agree with Jean-Denis Greze that it would be nice to have a
> method to retrieve all the keys at a particular node, instead of a
> range of keys.
>
> Jack
>
> On Sat, Feb 6, 2010 at 2:01 PM, Jack Culpepper <jackculpepper@gmail.com> wrote:
>> Well, from the output I included you can see that get_slice_range()
>> does not return any keys above
>> 9ffff14fd361b981faea6a04c5ef5699a96a8d6d, whereas get_key_range()
>> finds keys all the way up to ffffffa1b5e3aeb9ca92d4d848280093bdf49892.
>>
>> My program stops if either function ever returns less keys than
>> requested (1000 in this case).
>>
>> I have 2 nodes and a replication factor of 2, so both nodes should
>> have all the data, right?
>>
>> If I turn off one node and try the same test, I get the same result --
>> that is, get_key_range() finds many more key than get_slice_range(). I
>> haven't tested the case where I delete all the data, launch only a
>> single node and do all the inserts on a single node, and then compare
>> both methods. If you would like me to do that I can.
>>
>> Jack
>>
>> On Sat, Feb 6, 2010 at 10:16 AM, Jonathan Ellis <jbellis@gmail.com> wrote:
>>> It would help if you could narrow it down to "here are the keys I
>>> expect to see that I am not," especially if you can reproduce on a
>>> single-node cluster.
>>>
>>> On Sat, Feb 6, 2010 at 2:04 AM, Jack Culpepper <jackculpepper@gmail.com>
wrote:
>>>> Hi Jonathon,
>>>>
>>>> I am seeing a dramatic difference in the number of keys I can scan
>>>> when I use these two methods.
>>>>
>>>> The former (deprecated) method seems to return the correct result.
>>>> That is, it's on the right order of magnitude of around 500K, and if I
>>>> continue to insert keys via a separate process as I repeatedly count
>>>> them, the count grows. The recommended alternative, get_range_slice(),
>>>> returns far fewer keys and if I count repeatedly as I insert using a
>>>> separate process, the count bounces around erratically.
>>>>
>>>> I am using the python thrift interface against a two node setup. I am
>>>> running the current 0.5.0 release (just upgraded from rc1 since I saw
>>>> some other thrift bug was fixed). Here is my program (there are three
>>>> commented lines to switch from one method to the other):
>>>>
>>>> if sys.argv[1] == "count_things":
>>>>
>>>>    from thrift import Thrift
>>>>    from thrift.transport import TTransport
>>>>    from thrift.transport import TSocket
>>>>    from thrift.protocol.TBinaryProtocol import TBinaryProtocolAccelerated
>>>>    from cassandra import Cassandra
>>>>
>>>>    socket = TSocket.TSocket("10.212.230.176", 9160)
>>>>    transport = TTransport.TBufferedTransport(socket)
>>>>    protocol = TBinaryProtocol.TBinaryProtocolAccelerated(transport)
>>>>    client = Cassandra.Client(protocol)
>>>>
>>>>    transport.open()
>>>>
>>>>    column_parent = ColumnParent(column_family="thing")
>>>>    slice_range = SliceRange(start="key", finish="key")
>>>>    predicate = SlicePredicate(slice_range=slice_range)
>>>>
>>>>    done = False
>>>>    seg = 1000
>>>>    start = ""
>>>>
>>>>    while not done:
>>>>        #result = client.get_key_range("gg", "thing", start, "", seg,
>>>> ConsistencyLevel.ONE)
>>>>        result = client.get_range_slice("gg", column_parent,
>>>> predicate, start, "", seg, ConsistencyLevel.ONE)
>>>>
>>>>        if len(result) < seg: done = True
>>>>        #else: start = result[seg-1]
>>>>        else: start = result[seg-1].key
>>>>
>>>>        record_count += len(result)
>>>>
>>>>        t = now()
>>>>        dt = t - startTime
>>>>        record_per_sec = record_count / dt
>>>>        #print "\rstart %d now %d dt %d rec/s %.4f rec %d s %s f
>>>> %s"%(startTime,t,dt,record_per_sec,record_count,result[0],result[-1]),
>>>>        print "\rstart %d now %d dt %d rec/s %.4f rec %d s %s f
>>>> %s"%(startTime,t,dt,record_per_sec,record_count,result[0].key,result[-1].key),
>>>>    print
>>>>
>>>> An example of the output using get_range_slice(), without a concurrent
>>>> insertion process -- it counts 133674 keys.
>>>>
>>>> start 1265440888 now 1265441098 dt 210 rec/s 636.1996 rec 133674 s
>>>> 9f9dd2c0f043902f7f571942cfac3f6c28b82cec f
>>>> 9ffff14fd361b981faea6a04c5ef5699a96a8d6d
>>>>
>>>> Using get_key_range() I get 459351 keys, and the throughput is less:
>>>>
>>>> start 1265442143 now 1265443092 dt 948 rec/s 484.2775 rec 459351 s
>>>> ffce8099f808d10a09db471b04793315f555ccbd f
>>>> ffffffa1b5e3aeb9ca92d4d848280093bdf49892
>>>>
>>>> get_range_slice() seems to skip keys in each of the segments.
>>>>
>>>> The "thing" column family is a super column. There are no errors
>>>> reported to the log. The keys I am inserting are python generated
>>>> UUIDs:
>>>>
>>>> import uuid
>>>> key = uuid.uuid4().hex
>>>>
>>>> I'm not posting the program that inserts the data, but I can if that
>>>> would be help. Thanks very much,
>>>>
>>>> Jack
>>>>
>>>
>>
>

Mime
View raw message