RE: Problem using sstableloader with SSTableSimpleUnsortedWriter and a composite key

2013-06-30 Thread Peer, Oded
Thank you Aaaron!

Your blog post helped me understand how a row with a compound key is stored and 
this helped me understand how to create the sstable files.
For anyone who needs it this is how it works:

In Cassandra-cli the row looks like this:
RowKey: 5
=> (column=10:created, value=013f84be6288, timestamp=137232163700)

>From this we see that the row key is a single Long value "5", and it has one 
>composite column "10:created" with a timestamp value.
Thus the code should look like this:

   File directory = new File( System.getProperty( "output" ) );
   IPartitioner partitioner = new Murmur3Partitioner();
   String keyspace = "test_keyspace";
   String columnFamily = "test_table";
   List> compositeList = new ArrayList>();
   compositeList.add( LongType.instance );
   compositeList.add( LongType.instance );
   CompositeType compositeType = CompositeType.getInstance( compositeList );
   SSTableSimpleUnsortedWriter sstableWriter = new SSTableSimpleUnsortedWriter(
  directory,
  partitioner,
  keyspace,
  columnFamily,
  compositeType,
  null,
  64 );
   long timestamp = 1372321637000L;
   long nanotimestamp = timestamp * 1000;
   long k1 = 5L;
   long k2 = 10L;
   sstableWriter.newRow( bytes( k1 ) );
   sstableWriter.addColumn( compositeType.builder().add( bytes( k2 ) ).add( 
bytes( "created" ) ).build(), bytes( timestamp ), nanotimestamp );
   sstableWriter.close();





Re: Problem using sstableloader with SSTableSimpleUnsortedWriter and a composite key

2013-06-27 Thread aaron morton
>   create table test_table ( k1 bigint, k2 bigint, created timestamp, PRIMARY 
> KEY (k1, k2) ) with compaction = { 'class' : 'LeveledCompactionStrategy' };
When CQL creates a non COMPACT STORAGE 
(http://www.datastax.com/docs/1.2/cql_cli/cql/CREATE_TABLE#using-compact-storage)
 table it uses composite types in places you do not expect. 

This post will give you a bit of an idea 
http://thelastpickle.com/2013/01/11/primary-keys-in-cql/
You can also use the cassandra-cli to look at the internal table def for your 
CQL 3 table

I think in your example the Columns are composites of (Int, UTF8Type) and the 
key will be a plain int (may be a composite am not sure right now). 

Hope that helps. 

-
Aaron Morton
Freelance Cassandra Consultant
New Zealand

@aaronmorton
http://www.thelastpickle.com

On 27/06/2013, at 8:19 PM, "Peer, Oded"  wrote:

> Hi,
> 
> I am using Cassandra 1.2.5. I built a cluster of 2 data centers with 3 nodes 
> in each data center.
> I created a key space and table with a composite key:
>   create keyspace test_keyspace WITH replication = {'class': 
> 'NetworkTopologyStrategy', 'DC1' : 1, 'DC2' : 1};
>   create table test_table ( k1 bigint, k2 bigint, created timestamp, PRIMARY 
> KEY (k1, k2) ) with compaction = { 'class' : 'LeveledCompactionStrategy' };
> I then tried to load data to the table using sstableloader, which uses input 
> created via SSTableSimpleUnsortedWriter using the following code:
> 
>   File directory = new File( System.getProperty( "output" ) );
>   IPartitioner partitioner = new Murmur3Partitioner();
>   String keyspace = "test_keyspace";
>   String columnFamily = "test_table";
>   List> compositeList = new ArrayList>();
>   compositeList.add( LongType.instance );
>   compositeList.add( LongType.instance );
>   CompositeType compositeType = CompositeType.getInstance( compositeList );
>   SSTableSimpleUnsortedWriter sstableWriter = new SSTableSimpleUnsortedWriter(
>  directory,
>  partitioner,
>  keyspace,
>  columnFamily,
>  compositeType,
>  null,
>  64 );
>   long timestamp = 1372321637000L;
>   long nanotimestamp = timestamp * 1000;
>   sstableWriter.newRow( compositeType.builder().add( bytes( 1L ) ).add( 
> bytes( 1L ) ).build() );
>   sstableWriter.addColumn( bytes( "created" ), bytes( timestamp ), 
> nanotimestamp );
>   sstableWriter.close();
>   System.exit( 0 );
> 
> I then load the sstable files using the command "sstableloader -d node1 -v 
> -debug test_keyspace/test_table/"
> The command ends without any indication of a problem, but the table remains 
> empty.
> I see an exception in one of the nodes system.log:
> java.lang.RuntimeException: java.lang.IllegalArgumentException
>at 
> org.apache.cassandra.service.RangeSliceVerbHandler.doVerb(RangeSliceVerbHandler.java:64)
>at 
> org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:56)
>at 
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:895)
>at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:918)
>at java.lang.Thread.run(Thread.java:662)
> Caused by: java.lang.IllegalArgumentException
>at java.nio.Buffer.limit(Buffer.java:247)
>at 
> org.apache.cassandra.db.marshal.AbstractCompositeType.getBytes(AbstractCompositeType.java:51)
>at 
> org.apache.cassandra.db.marshal.AbstractCompositeType.getWithShortLength(AbstractCompositeType.java:60)
>at 
> org.apache.cassandra.db.marshal.AbstractCompositeType.split(AbstractCompositeType.java:126)
>at 
> org.apache.cassandra.db.filter.ColumnCounter$GroupByPrefix.count(ColumnCounter.java:96)
>at 
> org.apache.cassandra.db.filter.SliceQueryFilter.collectReducedColumns(SliceQueryFilter.java:164)
>at 
> org.apache.cassandra.db.filter.QueryFilter.collateColumns(QueryFilter.java:136)
>at 
> org.apache.cassandra.db.filter.QueryFilter.collateOnDiskAtom(QueryFilter.java:84)
>at 
> org.apache.cassandra.db.RowIteratorFactory$2.getReduced(RowIteratorFactory.java:106)
>at 
> org.apache.cassandra.db.RowIteratorFactory$2.getReduced(RowIteratorFactory.java:79)
>at 
> org.apache.cassandra.utils.MergeIterator$ManyToOne.consume(MergeIterator.java:114)
>at 
> org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:97)
>at 
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
>at 
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
>at 
> org.apache.cassandra.db.ColumnFamilyStore$3.computeNext(ColumnFamilyStore.java:1399)
>at 
> org.apache.cassandra.db.ColumnFamilyStore$3.computeNext(ColumnFamilyStore.java:1395)
>at 
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
>at 
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
>

Problem using sstableloader with SSTableSimpleUnsortedWriter and a composite key

2013-06-27 Thread Peer, Oded
Hi,

I am using Cassandra 1.2.5. I built a cluster of 2 data centers with 3 nodes in 
each data center.
I created a key space and table with a composite key:
   create keyspace test_keyspace WITH replication = {'class': 
'NetworkTopologyStrategy', 'DC1' : 1, 'DC2' : 1};
   create table test_table ( k1 bigint, k2 bigint, created timestamp, PRIMARY 
KEY (k1, k2) ) with compaction = { 'class' : 'LeveledCompactionStrategy' };
I then tried to load data to the table using sstableloader, which uses input 
created via SSTableSimpleUnsortedWriter using the following code:

   File directory = new File( System.getProperty( "output" ) );
   IPartitioner partitioner = new Murmur3Partitioner();
   String keyspace = "test_keyspace";
   String columnFamily = "test_table";
   List> compositeList = new ArrayList>();
   compositeList.add( LongType.instance );
   compositeList.add( LongType.instance );
   CompositeType compositeType = CompositeType.getInstance( compositeList );
   SSTableSimpleUnsortedWriter sstableWriter = new SSTableSimpleUnsortedWriter(
  directory,
  partitioner,
  keyspace,
  columnFamily,
  compositeType,
  null,
  64 );
   long timestamp = 1372321637000L;
   long nanotimestamp = timestamp * 1000;
   sstableWriter.newRow( compositeType.builder().add( bytes( 1L ) ).add( bytes( 
1L ) ).build() );
   sstableWriter.addColumn( bytes( "created" ), bytes( timestamp ), 
nanotimestamp );
   sstableWriter.close();
   System.exit( 0 );

I then load the sstable files using the command "sstableloader -d node1 -v 
-debug test_keyspace/test_table/"
The command ends without any indication of a problem, but the table remains 
empty.
I see an exception in one of the nodes system.log:
java.lang.RuntimeException: java.lang.IllegalArgumentException
at 
org.apache.cassandra.service.RangeSliceVerbHandler.doVerb(RangeSliceVerbHandler.java:64)
at 
org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:56)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:895)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:918)
at java.lang.Thread.run(Thread.java:662)
Caused by: java.lang.IllegalArgumentException
at java.nio.Buffer.limit(Buffer.java:247)
at 
org.apache.cassandra.db.marshal.AbstractCompositeType.getBytes(AbstractCompositeType.java:51)
at 
org.apache.cassandra.db.marshal.AbstractCompositeType.getWithShortLength(AbstractCompositeType.java:60)
at 
org.apache.cassandra.db.marshal.AbstractCompositeType.split(AbstractCompositeType.java:126)
at 
org.apache.cassandra.db.filter.ColumnCounter$GroupByPrefix.count(ColumnCounter.java:96)
at 
org.apache.cassandra.db.filter.SliceQueryFilter.collectReducedColumns(SliceQueryFilter.java:164)
at 
org.apache.cassandra.db.filter.QueryFilter.collateColumns(QueryFilter.java:136)
at 
org.apache.cassandra.db.filter.QueryFilter.collateOnDiskAtom(QueryFilter.java:84)
at 
org.apache.cassandra.db.RowIteratorFactory$2.getReduced(RowIteratorFactory.java:106)
at 
org.apache.cassandra.db.RowIteratorFactory$2.getReduced(RowIteratorFactory.java:79)
at 
org.apache.cassandra.utils.MergeIterator$ManyToOne.consume(MergeIterator.java:114)
at 
org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:97)
at 
com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
at 
com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
at 
org.apache.cassandra.db.ColumnFamilyStore$3.computeNext(ColumnFamilyStore.java:1399)
at 
org.apache.cassandra.db.ColumnFamilyStore$3.computeNext(ColumnFamilyStore.java:1395)
at 
com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
at 
com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
at 
org.apache.cassandra.db.ColumnFamilyStore.filter(ColumnFamilyStore.java:1466)
at 
org.apache.cassandra.db.ColumnFamilyStore.getRangeSlice(ColumnFamilyStore.java:1443)
at 
org.apache.cassandra.service.RangeSliceVerbHandler.executeLocally(RangeSliceVerbHandler.java:46)
at 
org.apache.cassandra.service.RangeSliceVerbHandler.doVerb(RangeSliceVerbHandler.java:58)
... 4 more

Am I using the CompositeType and SSTableSimpleUnsortedWriter correctly?