hi,

i am using version 0.7-rc2 and pelops-c642967 from java.  when i try
to export all the data in a column family i don't get all of the data that
was inserted.  i suspect that this points to an underlying problem with 
the get_range_slices method.

i can reproduce the problem just using the command line interface
as follows:

1) create a 2 node cluster using the default cassandra.yml with these 
changes: 

* set the initial token for the 1st node:
initial_token: 0

* set the initial token for the 2nd node:
initial_token: 85070591730234615865843651857942052864

*comment out:
#listen_address: localhost
#rpc_address: localhost

*set the seeds (change ip's)
seeds:
   - 17.224.109.80
   - 17.224.109.81

2) start both nodes

3) put these commands in a file called schema.txt 

create keyspace Test with replication_factor = 1 and placement_strategy = 
'org.apache.cassandra.locator.SimpleStrategy';
use Test;
create column family Configs with column_type = 'Standard' and comparator = 
'UTF8Type';

and load them with the command line tool like this:

./bin/cassandra-cli --host 17.224.109.80 < schema.txt

4) put these commands in a file called config.txt

use Test;
set Configs['row-a']['key-a'] = '1';
set Configs['row-a']['key-b'] = '2';  
set Configs['row-a']['key-c'] = '3';
set Configs['row-b']['key-a'] = '4';
set Configs['row-b']['key-b'] = '5';
set Configs['row-b']['key-c'] = '6';
set Configs['row-c']['key-a'] = '7';
set Configs['row-c']['key-b'] = '8';
set Configs['row-c']['key-c'] = '9';

and load them with the command line tool like this:

./bin/cassandra-cli --host 17.224.109.80 < config.txt

5)  now start the command line tool and try to list the Configs column family:

./bin/cassandra-cli --host 17.224.109.80
Connected to: "Test Cluster" on 17.224.109.80/9160
Welcome to cassandra CLI.

Type 'help;' or '?' for help. Type 'quit;' or 'exit;' to quit.
[defa...@unknown] use Test;
Authenticated to keyspace: Test
[defa...@test] list Configs;
Using default limit of 100

0 Row Returned.

[defa...@test] get Configs['row-a'];
=> (column=key-a, value=31, timestamp=1292970451049000)
=> (column=key-b, value=32, timestamp=1292970451099000)
=> (column=key-c, value=33, timestamp=1292970451104000)
Returned 3 results.
[defa...@test] get Configs['row-b'];
=> (column=key-a, value=34, timestamp=1292970451108000)
=> (column=key-b, value=35, timestamp=1292970451111000)
=> (column=key-c, value=36, timestamp=1292970451116000)
Returned 3 results.
[defa...@test] get Configs['row-c'];
=> (column=key-a, value=37, timestamp=1292970451120000)
=> (column=key-b, value=38, timestamp=1292970451123000)
=> (column=key-c, value=39, timestamp=1292970451128000)
Returned 3 results.

note: that the 'list Configs' command doesn't return any results,
but when you 'get' a particular row the data is there.

note: when only using a single node everything works as 
expected

note: that if you try to combine schema.txt and config.txt and
load all at once the first data point fails to load and the following
error shows up in the 2nd node's logs:

 INFO [FlushWriter:1] 2010-12-21 12:30:25,701 Memtable.java (line 155) Writing 
memtable-sch...@390276053(2184 bytes, 3 operations)
ERROR [MutationStage:1] 2010-12-21 12:30:25,718 RowMutationVerbHandler.java 
(line 83) Error in row mutation
org.apache.cassandra.db.UnserializableColumnFamilyException: Couldn't find 
cfId=1000
        at 
org.apache.cassandra.db.ColumnFamilySerializer.deserialize(ColumnFamilySerializer.java:117)
        at 
org.apache.cassandra.db.RowMutationSerializer.defreezeTheMaps(RowMutation.java:383)
        at 
org.apache.cassandra.db.RowMutationSerializer.deserialize(RowMutation.java:393)
        at 
org.apache.cassandra.db.RowMutationSerializer.deserialize(RowMutation.java:351)
        at 
org.apache.cassandra.db.RowMutationVerbHandler.doVerb(RowMutationVerbHandler.java:52)
        at 
org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:63)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
        at java.lang.Thread.run(Thread.java:636)
 INFO [FlushWriter:1] 2010-12-21 12:30:25,925 Memtable.java (line 162) 
Completed flushing /var/lib/cassandra/data/system/Schema-e-1-Data.db (2472 byte

has anyone else seen anything similar or is there something that i'm missing?

thanks,
-mike

Reply via email to