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: SStable Writer and composite key
Date Wed, 30 Jan 2013 23:02:43 GMT
This is what a row of your table will look like internally…

-------------------
RowKey: id-value
=> (column=date-value:request-value:, value=, timestamp=1359586739456000)
=> (column=date-value:request-value:data1, value=64617461312d76616c7565, timestamp=1359586739456000)
=> (column=date-value:request-value:data2, value=64617461322d76616c7565, timestamp=1359586739456000)

where id-value is the value of id column, and date-value is the….

So you need to construct triples for each column value. 
The first col name is the (date-value, request-value, empty) 
The second col name is (date-value, request-value, "data1") and has the value of data1. 

Hope that helps. 

-----------------
Aaron Morton
Freelance Cassandra Developer
New Zealand

@aaronmorton
http://www.thelastpickle.com

On 30/01/2013, at 3:11 AM, POUGET Laurent <laurent.pouget@carboatmedia.fr> wrote:

> Hi,
>  
>  
> I have some trouble to request my data. I use SSTableSimpleUnsortedWriter to write SSTable.
Writing and Importing works fine.
> I think, I’m misusing CompositeType.Builder with SSTableSimpleUnsortedWriter.
> Do you have any idea ?
>  
> Thanks
>  
> Here is my case :
>  
> /**
> * CREATE STATEMENT
> */
>  
> CREATE TABLE raw_data (
>   id text,
>   date text,
>   request text,
>   data1 text,
>   data2 text,
>   PRIMARY KEY (id, date, request)
> ) WITH
>   bloom_filter_fp_chance=0.010000 AND
>   caching='KEYS_ONLY' AND
>   comment='' AND
>   dclocal_read_repair_chance=0.000000 AND
>   gc_grace_seconds=864000 AND
>   read_repair_chance=0.100000 AND
>   replicate_on_write='true' AND
>   compaction={'class': 'SizeTieredCompactionStrategy'} AND
>   compression={'sstable_compression': 'SnappyCompressor'};
>  
> /**
> * JAVA CODE
> */
>  
> List<AbstractType<?>> compositeList = new ArrayList<AbstractType<?>>();
>  
> compositeList.add(UTF8Type.instance);
> compositeList.add(UTF8Type.instance);
>  
> IPartitioner<?> partitioner = StorageService.getPartitioner();
> dir = Directories.create(keyspace.getKeyspaceName(), columnFamily.getName()).getDirectoryForNewSSTables(0);
>  
> simpleUnsortedWriter = new SSTableSimpleUnsortedWriter(dir,
>                                                                partitioner, keyspace.getKeyspaceName(),
>                                                                columnFamily.getName(),
UTF8Type.instance, null,
>                                                                32);
>  
> CompositeType.Builder builderRequestDate = new CompositeType.Builder( CompositeType.getInstance
                      (compositeList) );
> CompositeType.Builder builderUrl = new CompositeType.Builder( CompositeType.getInstance(compositeList)
);
>  
> simpleUnsortedWriter.newRow(bytes(id));
>                               
> builderRequestDate.add(bytes("date"));
> builderRequestDate.add(bytes("request"));
>  
> long timestamp = System.currentTimeMillis() * 1000;
>  
> simpleUnsortedWriter.addColumn(builderRequestDate.build(),
>                                bytes(date), timestamp);
> simpleUnsortedWriter.addColumn(builderUrl.build(),
>                                bytes(request), timestamp);
>  
> simpleUnsortedWriter.addColumn(bytes("data1"),
>                                bytes(data1), timestamp);
> simpleUnsortedWriter.addColumn(bytes("data2"),
>                                bytes(data2), timestamp);
>  
> simpleUnsortedWriter.close();
>  
>  
> ________________________________
>  
> Laurent Pouget
> Ingénieur étude et développement
> Tel : 01.84.95.11.20
>  
> Car & Boat Media
> 22 Rue Joubert  75009 Paris
>  
>  
>  
>  
>  


Mime
View raw message