From user-return-31529-apmail-cassandra-user-archive=cassandra.apache.org@cassandra.apache.org Wed Jan 30 23:03:19 2013 Return-Path: X-Original-To: apmail-cassandra-user-archive@www.apache.org Delivered-To: apmail-cassandra-user-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 62289EA3C for ; Wed, 30 Jan 2013 23:03:19 +0000 (UTC) Received: (qmail 85654 invoked by uid 500); 30 Jan 2013 23:03:16 -0000 Delivered-To: apmail-cassandra-user-archive@cassandra.apache.org Received: (qmail 85635 invoked by uid 500); 30 Jan 2013 23:03:16 -0000 Mailing-List: contact user-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@cassandra.apache.org Delivered-To: mailing list user@cassandra.apache.org Received: (qmail 85624 invoked by uid 99); 30 Jan 2013 23:03:16 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 30 Jan 2013 23:03:16 +0000 X-ASF-Spam-Status: No, hits=2.2 required=5.0 tests=HTML_MESSAGE,RCVD_IN_DNSWL_NONE,SPF_PASS X-Spam-Check-By: apache.org Received-SPF: pass (nike.apache.org: local policy) Received: from [208.113.200.5] (HELO homiemail-a48.g.dreamhost.com) (208.113.200.5) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 30 Jan 2013 23:03:08 +0000 Received: from homiemail-a48.g.dreamhost.com (localhost [127.0.0.1]) by homiemail-a48.g.dreamhost.com (Postfix) with ESMTP id A225D4F805B for ; Wed, 30 Jan 2013 15:02:46 -0800 (PST) DKIM-Signature: v=1; a=rsa-sha1; c=relaxed; d=thelastpickle.com; h=from :content-type:message-id:mime-version:subject:date:references:to :in-reply-to; s=thelastpickle.com; bh=icSCTgYzZ8PgfDY53kuu6ST/Iv 4=; b=TRLqZ20kuMfnvALAMLNkMJx2JdoM8LpjxINwttmY8NhXTw0TbVpFJfSFCo GBZGeJsMKII6NP1hoPBnqD3f1nvRp+4aEzPtIdeNLAvkx5jxyI0rimGPxJOyfzbm 0MY5KK8QA7uH4dMD8aQ6XMTXAOtE4Y6jc1B3JIvdKe6d7HdIU= Received: from [172.16.1.8] (unknown [203.86.207.101]) (using TLSv1 with cipher AES128-SHA (128/128 bits)) (No client certificate requested) (Authenticated sender: aaron@thelastpickle.com) by homiemail-a48.g.dreamhost.com (Postfix) with ESMTPSA id 9152D4F8057 for ; Wed, 30 Jan 2013 15:02:45 -0800 (PST) From: aaron morton Content-Type: multipart/alternative; boundary="Apple-Mail=_68E9879F-DD17-4D87-AA54-E8E56D65BEF1" Message-Id: Mime-Version: 1.0 (Mac OS X Mail 6.2 \(1499\)) Subject: Re: SStable Writer and composite key Date: Thu, 31 Jan 2013 12:02:43 +1300 References: <9BCDA0CDD97AB049903AEBDF966FA91811096F77@DB3PRD0710MB369.eurprd07.prod.outlook.com> To: user@cassandra.apache.org In-Reply-To: <9BCDA0CDD97AB049903AEBDF966FA91811096F77@DB3PRD0710MB369.eurprd07.prod.outlook.com> X-Mailer: Apple Mail (2.1499) X-Virus-Checked: Checked by ClamAV on apache.org --Apple-Mail=_68E9879F-DD17-4D87-AA54-E8E56D65BEF1 Content-Transfer-Encoding: quoted-printable Content-Type: text/plain; charset=windows-1252 This is what a row of your table will look like internally=85 ------------------- RowKey: id-value =3D> (column=3Ddate-value:request-value:, value=3D, = timestamp=3D1359586739456000) =3D> (column=3Ddate-value:request-value:data1, = value=3D64617461312d76616c7565, timestamp=3D1359586739456000) =3D> (column=3Ddate-value:request-value:data2, = value=3D64617461322d76616c7565, timestamp=3D1359586739456000) where id-value is the value of id column, and date-value is the=85. So you need to construct triples for each column value.=20 The first col name is the (date-value, request-value, empty)=20 The second col name is (date-value, request-value, "data1") and has the = value of data1.=20 Hope that helps.=20 ----------------- Aaron Morton Freelance Cassandra Developer New Zealand @aaronmorton http://www.thelastpickle.com On 30/01/2013, at 3:11 AM, POUGET Laurent = wrote: > Hi, > =20 > =20 > I have some trouble to request my data. I use = SSTableSimpleUnsortedWriter to write SSTable. Writing and Importing = works fine. > I think, I=92m misusing CompositeType.Builder with = SSTableSimpleUnsortedWriter. > Do you have any idea ? > =20 > Thanks > =20 > Here is my case : > =20 > /** > * CREATE STATEMENT > */ > =20 > CREATE TABLE raw_data ( > id text, > date text, > request text, > data1 text, > data2 text, > PRIMARY KEY (id, date, request) > ) WITH > bloom_filter_fp_chance=3D0.010000 AND > caching=3D'KEYS_ONLY' AND > comment=3D'' AND > dclocal_read_repair_chance=3D0.000000 AND > gc_grace_seconds=3D864000 AND > read_repair_chance=3D0.100000 AND > replicate_on_write=3D'true' AND > compaction=3D{'class': 'SizeTieredCompactionStrategy'} AND > compression=3D{'sstable_compression': 'SnappyCompressor'}; > =20 > /** > * JAVA CODE > */ > =20 > List> compositeList =3D new = ArrayList>(); > =20 > compositeList.add(UTF8Type.instance); > compositeList.add(UTF8Type.instance); > =20 > IPartitioner partitioner =3D StorageService.getPartitioner(); > dir =3D Directories.create(keyspace.getKeyspaceName(), = columnFamily.getName()).getDirectoryForNewSSTables(0); > =20 > simpleUnsortedWriter =3D new SSTableSimpleUnsortedWriter(dir, > = partitioner, keyspace.getKeyspaceName(), > = columnFamily.getName(), UTF8Type.instance, null, > 32); > =20 > CompositeType.Builder builderRequestDate =3D new = CompositeType.Builder( CompositeType.getInstance = (compositeList) ); > CompositeType.Builder builderUrl =3D new CompositeType.Builder( = CompositeType.getInstance(compositeList) ); > =20 > simpleUnsortedWriter.newRow(bytes(id)); > =20 > builderRequestDate.add(bytes("date")); > builderRequestDate.add(bytes("request")); > =20 > long timestamp =3D System.currentTimeMillis() * 1000; > =20 > simpleUnsortedWriter.addColumn(builderRequestDate.build(), > bytes(date), timestamp); > simpleUnsortedWriter.addColumn(builderUrl.build(), > bytes(request), timestamp); > =20 > simpleUnsortedWriter.addColumn(bytes("data1"), > bytes(data1), timestamp); > simpleUnsortedWriter.addColumn(bytes("data2"), > bytes(data2), timestamp); > =20 > simpleUnsortedWriter.close(); > =20 > =20 > ________________________________ > =20 > Laurent Pouget > Ing=E9nieur =E9tude et d=E9veloppement > Tel : 01.84.95.11.20 > =20 > Car & Boat Media > 22 Rue Joubert 75009 Paris > =20 > =20 > =20 > =20 > =20 --Apple-Mail=_68E9879F-DD17-4D87-AA54-E8E56D65BEF1 Content-Transfer-Encoding: quoted-printable Content-Type: text/html; charset=windows-1252 This is what a row of your = table will look like = internally=85

-------------------
RowKey= : id-value
=3D> (column=3Ddate-value:request-value:, = value=3D, timestamp=3D1359586739456000)
=3D> = (column=3Ddate-value:request-value:data1, value=3D64617461312d76616c7565, = timestamp=3D1359586739456000)
=3D> = (column=3Ddate-value:request-value:data2, value=3D64617461322d76616c7565, = timestamp=3D1359586739456000)

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

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. 

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=92m 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=3D0.010000 = AND
  caching=3D'KEYS_ONLY' = AND
  comment=3D'' AND
  = dclocal_read_repair_chance=3D0.000000 AND
  = gc_grace_seconds=3D864000 AND
  read_repair_chance=3D0.100000 = AND
  replicate_on_write=3D'true' = AND
  compaction=3D{'class': = 'SizeTieredCompactionStrategy'} AND
  = compression=3D{'sstable_compression': = 'SnappyCompressor'};
 
/**
* JAVA CODE
*/
 
List<AbstractType<?>> compositeList =3D new = ArrayList<AbstractType<?>>();
 
compositeList.add(UTF8Type.instance);
compositeList.add(UTF8Type.instance);
 
IPartitioner<?> = partitioner =3D = StorageService.getPartitioner();
dir =3D = Directories.create(keyspace.getKeyspaceName(), = columnFamily.getName()).getDirectoryForNewSSTables(0);
 
simpleUnsortedWriter =3D new = SSTableSimpleUnsortedWriter(dir,
          = ;            &= nbsp;           &nb= sp;            = ;            &= nbsp;   partitioner, = keyspace.getKeyspaceName(),
          = ;            &= nbsp;           &nb= sp;            = ;            &= nbsp;   columnFamily.getName(), UTF8Type.instance, = null,
          = ;            &= nbsp;           &nb= sp;            = ;            &= nbsp;   32);
 
CompositeType.Builder builderRequestDate =3D new = CompositeType.Builder( = CompositeType.getInstance       =             &n= bsp;   (compositeList) );
CompositeType.Builder = builderUrl =3D new CompositeType.Builder( = CompositeType.getInstance(compositeList) );
 
simpleUnsortedWriter.newRow(bytes(id));
          = ;            &= nbsp;       
builderRequestDate.add(bytes("date"));
builderRequestDate.add(bytes("request"));=
 
long timestamp =3D System.currentTimeMillis() * = 1000;
 
simpleUnsortedWriter.addColumn(builderRequestDate.build(),<= o:p>
          = ;            &= nbsp;        bytes(date), = timestamp);
simpleUnsortedWriter.addColumn(builderUrl.build(),
          = ;            &= nbsp;        bytes(request), = timestamp);
 
simpleUnsortedWriter.addColumn(bytes("data1"),
          = ;            &= nbsp;        bytes(data1), = timestamp);
simpleUnsortedWriter.addColumn(bytes("data2"),
          = ;            &= nbsp;        bytes(data2), = timestamp);
 
simpleUnsortedWriter.close();
 
 
________________________________
 
Laurent = Pouget
Ing=E9nieur =E9tude et = d=E9veloppement
Tel : 01.84.95.11.20
 
Car & Boat = Media
22 Rue Joubert  75009 = Paris
 
 
 
 
 

= --Apple-Mail=_68E9879F-DD17-4D87-AA54-E8E56D65BEF1--