incubator-cassandra-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From aaron morton <aa...@thelastpickle.com>
Subject Re: strange get_range_slices behaviour v0.6.1
Date Sun, 02 May 2010 12:19:33 GMT
He there, I'm still getting odd behavior with get_range_slices. I've  
created a JUNIT test that illustrates the case.
Could someone take a look and either let me know where my  
understanding is wrong or is this is a real issue?


I added the following to ColumnFamilyStoreTest.java


     private ColumnFamilyStore insertKey1Key2Key3() throws  
IOException, ExecutionException, InterruptedException
     {
         List<RowMutation> rms = new LinkedList<RowMutation>();
         RowMutation rm;
         rm = new RowMutation("Keyspace2", "key1".getBytes());
         rm.add(new QueryPath("Standard1", null,  
"Column1".getBytes()), "asdf".getBytes(), 0);
         rms.add(rm);

         rm = new RowMutation("Keyspace2", "key2".getBytes());
         rm.add(new QueryPath("Standard1", null,  
"Column1".getBytes()), "asdf".getBytes(), 0);
         rms.add(rm);

         rm = new RowMutation("Keyspace2", "key3".getBytes());
         rm.add(new QueryPath("Standard1", null,  
"Column1".getBytes()), "asdf".getBytes(), 0);
         rms.add(rm);
         return Util.writeColumnFamily(rms);
     }


     @Test
     public void testThreeKeyRangeAll() throws IOException,  
ExecutionException, InterruptedException
     {
         ColumnFamilyStore cfs = insertKey1Key2Key3();

         IPartitioner p = StorageService.getPartitioner();
         RangeSliceReply result =  
cfs.getRangeSlice(ArrayUtils.EMPTY_BYTE_ARRAY,
                                                    Util.range(p,  
"key1", "key3"),
                                                    10,
                                                    null,
                                                     
Arrays.asList("Column1".getBytes()));
         assertEquals(3, result.rows.size());
     }

     @Test
     public void testThreeKeyRangeSkip1() throws IOException,  
ExecutionException, InterruptedException
     {
         ColumnFamilyStore cfs = insertKey1Key2Key3();

         IPartitioner p = StorageService.getPartitioner();
         RangeSliceReply result =  
cfs.getRangeSlice(ArrayUtils.EMPTY_BYTE_ARRAY,
                                                    Util.range(p,  
"key2", "key3"),
                                                    10,
                                                    null,
                                                     
Arrays.asList("Column1".getBytes()));
         assertEquals(2, result.rows.size());
     }

Running this with "ant test" the partial output is....

     [junit] Testsuite: org.apache.cassandra.db.ColumnFamilyStoreTest
     [junit] Tests run: 7, Failures: 2, Errors: 0, Time elapsed: 1.405  
sec
     [junit]
     [junit] Testcase:  
testThreeKeyRangeAll(org.apache.cassandra.db.ColumnFamilyStoreTest):	 
FAILED
     [junit] expected:<3> but was:<2>
     [junit] junit.framework.AssertionFailedError: expected:<3> but  
was:<2>
     [junit] 	at  
org 
.apache 
.cassandra 
.db 
.ColumnFamilyStoreTest.testThreeKeyRangeAll(ColumnFamilyStoreTest.java: 
170)
     [junit]
     [junit]
     [junit] Testcase:  
testThreeKeyRangeSkip1(org.apache.cassandra.db.ColumnFamilyStoreTest):	 
FAILED
     [junit] expected:<2> but was:<1>
     [junit] junit.framework.AssertionFailedError: expected:<2> but  
was:<1>
     [junit] 	at  
org 
.apache 
.cassandra 
.db 
.ColumnFamilyStoreTest 
.testThreeKeyRangeSkip1(ColumnFamilyStoreTest.java:184)
     [junit]
     [junit]
     [junit] Test org.apache.cassandra.db.ColumnFamilyStoreTest FAILED


Any help appreciated.

Aaron


On 27 Apr 2010, at 09:38, aaron wrote:

>
> I've broken this case down further to some pyton code that works  
> against
> the thrift generated
> client and am still getting the same odd results. With keys obejct1,
> object2 and object3 an
> open ended get_range_slice starting with "object1" only returns  
> object1 and
> 2.
>
> I'm guessing that I've got something wrong or my expectation of how
> get_range_slice works
> is wrong, but I cannot see where I've gone wrong. Any help would be
> appreciated.
>
> They python code to add and read keys is below, assumes a  
> Cassandra.Client
> connection.
>
> import time
> from cassandra import Cassandra,ttypes
> from thrift import Thrift
> from thrift.protocol import TBinaryProtocol
> from thrift.transport import TSocket, TTransport
>
>
> def add_data(conn):
>
>    col_path = ttypes.ColumnPath(column_family="Standard1",
> column="col_name")
>    consistency = ttypes.ConsistencyLevel.QUORUM
>
>    for key in ["object1", "object2", "object3"]:
>        conn.insert("Keyspace1", key, col_path, "col_value",
>            int(time.time() * 1e6), consistency)
>    return
>
> def read_range(conn, start_key, end_key):
>
>    col_parent = ttypes.ColumnParent(column_family="Standard1")
>
>    predicate = ttypes.SlicePredicate(column_names=["col_name"])
>    range = ttypes.KeyRange(start_key=start_key, end_key=end_key,
> count=1000)
>    consistency = ttypes.ConsistencyLevel.QUORUM
>
>    return conn.get_range_slices("Keyspace1", col_parent,
>                predicate, range, consistency)
>
>
> Below is the result of calling read_range with different start  
> values. I've
> also included
> the debug log for each call, the line starting with "reading
> RangeSliceCommand" seems to
> show that key hash for "object2" is greater than "object3".
>
> #expect to return objects 1,2 and 3
>
> In [37]: cass_test.read_range(conn, "object1", "")
> Out[37]:
> [KeySlice 
> (columns 
> =[ColumnOrSuperColumn(column=Column(timestamp=1272315595268837,
> name='col_name', value='col_value'), super_column=None)],  
> key='object1'),
> KeySlice 
> (columns 
> =[ColumnOrSuperColumn(column=Column(timestamp=1272315595272693,
> name='col_name', value='col_value'), super_column=None)],  
> key='object3')]
>
> DEBUG 09:29:59,791 range_slice
> DEBUG 09:29:59,791 RangeSliceCommand{keyspace='Keyspace1',
> column_family='Standard1', super_column=null,
> predicate=SlicePredicate(column_names:[[B@257b40fe]),
> range=[121587881847328893689247922008234581399,0], max_keys=1000}
> DEBUG 09:29:59,791 Adding to restricted ranges
> [121587881847328893689247922008234581399,0] for
> (75349581786326521367945210761838448174,75349581786326521367945210761838448174 
> ]
> DEBUG 09:29:59,791 reading RangeSliceCommand{keyspace='Keyspace1',
> column_family='Standard1', super_column=null,
> predicate=SlicePredicate(column_names:[[B@257b40fe]),
> range=[121587881847328893689247922008234581399,0], max_keys=1000} from
> 1528@localhost/127.0.0.1
> DEBUG 09:29:59,791 Sending RangeSliceReply{rows=Row(key='object1',
> cf=ColumnFamily(Standard1
> [636f6c5f6e616d65:false:9@1272315595268837,])),Row(key='object3',
> cf=ColumnFamily(Standard1 [636f6c5f6e616d65:false: 
> 9@1272315595272693,]))}
> to 1528@localhost/127.0.0.1
> DEBUG 09:29:59,791 Processing response on a callback from
> 1528@localhost/127.0.0.1
> DEBUG 09:29:59,791 range slices read object1
> DEBUG 09:29:59,791 range slices read object3
>
>
> In [38]: cass_test.read_range(conn, "object2", "")
> Out[38]:
> [KeySlice 
> (columns 
> =[ColumnOrSuperColumn(column=Column(timestamp=1272315595271798,
> name='col_name', value='col_value'), super_column=None)],  
> key='object2'),
> KeySlice 
> (columns 
> =[ColumnOrSuperColumn(column=Column(timestamp=1272315595268837,
> name='col_name', value='col_value'), super_column=None)],  
> key='object1'),
> KeySlice 
> (columns 
> =[ColumnOrSuperColumn(column=Column(timestamp=1272315595272693,
> name='col_name', value='col_value'), super_column=None)],  
> key='object3')]
>
> DEBUG 09:34:48,133 range_slice
> DEBUG 09:34:48,133 RangeSliceCommand{keyspace='Keyspace1',
> column_family='Standard1', super_column=null,
> predicate=SlicePredicate(column_names:[[B@7966340c]),
> range=[28312518014678916505369931620527723964,0], max_keys=1000}
> DEBUG 09:34:48,133 Adding to restricted ranges
> [28312518014678916505369931620527723964,0] for
> (75349581786326521367945210761838448174,75349581786326521367945210761838448174 
> ]
> DEBUG 09:34:48,133 reading RangeSliceCommand{keyspace='Keyspace1',
> column_family='Standard1', super_column=null,
> predicate=SlicePredicate(column_names:[[B@7966340c]),
> range=[28312518014678916505369931620527723964,0], max_keys=1000} from
> 1810@localhost/127.0.0.1
> DEBUG 09:34:48,133 Sending RangeSliceReply{rows=Row(key='object2',
> cf=ColumnFamily(Standard1
> [636f6c5f6e616d65:false:9@1272315595271798,])),Row(key='object1',
> cf=ColumnFamily(Standard1
> [636f6c5f6e616d65:false:9@1272315595268837,])),Row(key='object3',
> cf=ColumnFamily(Standard1 [636f6c5f6e616d65:false: 
> 9@1272315595272693,]))}
> to 1810@localhost/127.0.0.1
> DEBUG 09:34:48,133 Processing response on a callback from
> 1810@localhost/127.0.0.1
> DEBUG 09:34:48,133 range slices read object2
> DEBUG 09:34:48,133 range slices read object1
> DEBUG 09:34:48,133 range slices read object3
>
>
> In [39]: cass_test.read_range(conn, "object3", "")
> Out[39]:
> [KeySlice 
> (columns 
> =[ColumnOrSuperColumn(column=Column(timestamp=1272315595272693,
> name='col_name', value='col_value'), super_column=None)],  
> key='object3')]
>
> DEBUG 09:35:26,090 range_slice
> DEBUG 09:35:26,090 RangeSliceCommand{keyspace='Keyspace1',
> column_family='Standard1', super_column=null,
> predicate=SlicePredicate(column_names:[[B@24e33e18]),
> range=[123092639156685888118746480803115294277,0], max_keys=1000}
> DEBUG 09:35:26,090 Adding to restricted ranges
> [123092639156685888118746480803115294277,0] for
> (75349581786326521367945210761838448174,75349581786326521367945210761838448174 
> ]
> DEBUG 09:35:26,090 reading RangeSliceCommand{keyspace='Keyspace1',
> column_family='Standard1', super_column=null,
> predicate=SlicePredicate(column_names:[[B@24e33e18]),
> range=[123092639156685888118746480803115294277,0], max_keys=1000} from
> 1847@localhost/127.0.0.1
> DEBUG 09:35:26,090 Sending RangeSliceReply{rows=Row(key='object3',
> cf=ColumnFamily(Standard1 [636f6c5f6e616d65:false: 
> 9@1272315595272693,]))}
> to 1847@localhost/127.0.0.1
> DEBUG 09:35:26,090 Processing response on a callback from
> 1847@localhost/127.0.0.1
> DEBUG 09:35:26,090 range slices read object3
>
>
>
> thanks
> Aaron
>
>
>
>
> On Sun, 25 Apr 2010 20:23:05 -0700, aaron <aaron@the-mortons.org>  
> wrote:
>> I've been looking at the get_range_slices feature and have found  
>> some odd
>> behaviour I do not understand. Basically the keys returned in a range
> query
>> do not match what I would expect to see. I think it may have  
>> something to
>> do with the ordering of keys that I don't know about, but I'm just
>> guessing.
>>
>> On Cassandra v 0.6.1, single node local install; RandomPartitioner.  
>> Using
>> Python and my own thin wrapper around the Thrift Python API.
>>
>> Step 1.
>>
>> Insert 3 keys into the "Standard 1" column family, called "object 1"
>> "object 2" and "object 3", each with a single column called 'name'  
>> with a
>> value like 'object1'
>>
>> Step 2.
>>
>> Do a get_range_slices call in the "Standard 1" CF, for column names
>> ["name"] with start_key "object1" and end_key "object3". I expect  
>> to see
>> three results, but I only see results for object1 and object2.  
>> Below are
>> the thrift types I'm passing into the Cassandra.Client object...
>>
>> - ColumnParent(column_family='Standard1', super_column=None)
>> - SlicePredicate(column_names=['name'], slice_range=None)
>> - KeyRange(end_key='object3', start_key='object1', count=4000,
>> end_token=None, start_token=None)
>>
>> and the output
>>
>>
> [KeySlice 
> (columns 
> =[ColumnOrSuperColumn(column=Column(timestamp=1272250258810439,
>> name='name', value='object1'), super_column=None)], key='object1'),
>>
> KeySlice 
> (columns 
> =[ColumnOrSuperColumn(column=Column(timestamp=1272250271620362,
>> name='name', value='object3'), super_column=None)], key='object3')]
>>
>> Step 3.
>>
>> Modify the get_range_slices call, so the start_key is object2. In  
>> this
> case
>> I expect to see 2 rows returned, but I get 3. Thrift args and  
>> return are
>> below...
>>
>> - ColumnParent(column_family='Standard1', super_column=None)
>> - SlicePredicate(column_names=['name'], slice_range=None)
>> - KeyRange(end_key='object3', start_key='object2', count=4000,
>> end_token=None, start_token=None)
>>
>> and the output
>>
>>
> [KeySlice 
> (columns 
> =[ColumnOrSuperColumn(column=Column(timestamp=1272250265190715,
>> name='name', value='object2'), super_column=None)], key='object2'),
>>
> KeySlice 
> (columns 
> =[ColumnOrSuperColumn(column=Column(timestamp=1272250258810439,
>> name='name', value='object1'), super_column=None)], key='object1'),
>>
> KeySlice 
> (columns 
> =[ColumnOrSuperColumn(column=Column(timestamp=1272250271620362,
>> name='name', value='object3'), super_column=None)], key='object3')]
>>
>>
>>
>> Can anyone explain these odd results? As I said I've got my own  
>> python
>> wrapper around the client, so I may be doing something wrong. But  
>> I've
>> pulled out the thrift objects and they go in and out of the thrift
>> Cassandra.Client, so I think I'm ok. (I have not noticed a systematic
>> problem with my wrapper).
>>
>> On a more general note, is there information on the sort order of  
>> keys
> when
>> using key ranges? I'm guessing the hash of the keys is compared and I
>> wondering if the hash's of the keys maintain the order of the  
>> original
>> values? Also I assume the order is byte order, rather than ascii or  
>> utf8.
>
>>
>> I was experimenting with the difference between column slicing and  
>> key
>> slicing. In my I could write the keys in as column names (they are in
>> buckets) as well and slice there first, then use the results to to  
>> make a
>> multi key get. I'm trying to support features like, get me all the  
>> data
>> where the key starts with "foo.bar".
>>
>> Thanks for the fun project.
>>
>> Aaron


Mime
View raw message