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 56B68E0F7 for ; Thu, 31 Jan 2013 14:53:32 +0000 (UTC) Received: (qmail 97367 invoked by uid 500); 31 Jan 2013 14:53:29 -0000 Delivered-To: apmail-cassandra-user-archive@cassandra.apache.org Received: (qmail 96964 invoked by uid 500); 31 Jan 2013 14:53:25 -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 96936 invoked by uid 99); 31 Jan 2013 14:53:24 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 31 Jan 2013 14:53:24 +0000 X-ASF-Spam-Status: No, hits=2.2 required=5.0 tests=HTML_MESSAGE,RCVD_IN_DNSWL_LOW,SPF_NEUTRAL X-Spam-Check-By: apache.org Received-SPF: neutral (athena.apache.org: local policy) Received: from [216.32.180.189] (HELO co1outboundpool.messaging.microsoft.com) (216.32.180.189) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 31 Jan 2013 14:53:18 +0000 Received: from mail187-co1-R.bigfish.com (10.243.78.197) by CO1EHSOBE033.bigfish.com (10.243.66.98) with Microsoft SMTP Server id 14.1.225.23; Thu, 31 Jan 2013 14:52:55 +0000 Received: from mail187-co1 (localhost [127.0.0.1]) by mail187-co1-R.bigfish.com (Postfix) with ESMTP id A69DD84004E for ; Thu, 31 Jan 2013 14:52:55 +0000 (UTC) X-Forefront-Antispam-Report: CIP:157.56.253.85;KIP:(null);UIP:(null);IPV:NLI;H:DB3PRD0710HT005.eurprd07.prod.outlook.com;RD:none;EFVD:NLI X-SpamScore: -4 X-BigFish: PS-4(zzbb2dI98dI9371Ic89bhc85dh4015Izz1ee6h1de0h1d18h1202h1e76h1d1ah1d2ahzz17326ah8275dh18c673h1954cbh8275bhz2dh2a8h668h839hd25hf0ah1288h12a5h12bdh137ah1441h1504h1537h153bh15d0h162dh1631h1758h18e1h1946h1155h) Received: from mail187-co1 (localhost.localdomain [127.0.0.1]) by mail187-co1 (MessageSwitch) id 1359643972862243_15235; Thu, 31 Jan 2013 14:52:52 +0000 (UTC) Received: from CO1EHSMHS014.bigfish.com (unknown [10.243.78.204]) by mail187-co1.bigfish.com (Postfix) with ESMTP id D032894005B for ; Thu, 31 Jan 2013 14:52:52 +0000 (UTC) Received: from DB3PRD0710HT005.eurprd07.prod.outlook.com (157.56.253.85) by CO1EHSMHS014.bigfish.com (10.243.66.24) with Microsoft SMTP Server (TLS) id 14.1.225.23; Thu, 31 Jan 2013 14:52:52 +0000 Received: from DB3PRD0710MB369.eurprd07.prod.outlook.com ([169.254.12.21]) by DB3PRD0710HT005.eurprd07.prod.outlook.com ([10.255.75.40]) with mapi id 14.16.0263.000; Thu, 31 Jan 2013 14:52:45 +0000 From: POUGET Laurent To: "user@cassandra.apache.org" Subject: RE: SStable Writer and composite key Thread-Topic: SStable Writer and composite key Thread-Index: Ac3+Ko95HGA53VXCRnuHGMJ1aTMJUgBE1l6AABeEzrAACWqAYA== Date: Thu, 31 Jan 2013 14:52:44 +0000 Message-ID: <9BCDA0CDD97AB049903AEBDF966FA918110ACE25@DB3PRD0710MB369.eurprd07.prod.outlook.com> References: <9BCDA0CDD97AB049903AEBDF966FA91811096F77@DB3PRD0710MB369.eurprd07.prod.outlook.com> <9BCDA0CDD97AB049903AEBDF966FA918110A9BF4@DB3PRD0710MB369.eurprd07.prod.outlook.com> In-Reply-To: <9BCDA0CDD97AB049903AEBDF966FA918110A9BF4@DB3PRD0710MB369.eurprd07.prod.outlook.com> Accept-Language: fr-FR, en-US Content-Language: fr-FR X-MS-Has-Attach: X-MS-TNEF-Correlator: x-originating-ip: [212.234.52.189] Content-Type: multipart/alternative; boundary="_000_9BCDA0CDD97AB049903AEBDF966FA918110ACE25DB3PRD0710MB369_" MIME-Version: 1.0 X-OriginatorOrg: carboatmedia.fr X-Virus-Checked: Checked by ClamAV on apache.org --_000_9BCDA0CDD97AB049903AEBDF966FA918110ACE25DB3PRD0710MB369_ Content-Type: text/plain; charset="iso-8859-1" Content-Transfer-Encoding: quoted-printable Ok, in order to solve : java.lang.AssertionError: Added column does not sor= t as the last column I just have to make a scrub with nodetools. Is there a programmatic way to scrub and compact new sstable on an active n= ode ? Thx, Laurent Pouget De : POUGET Laurent [mailto:laurent.pouget@carboatmedia.fr] Envoy=E9 : jeudi 31 janvier 2013 11:31 =C0 : user@cassandra.apache.org Objet : RE: SStable Writer and composite key Thanks, Import is ok now but : =B7 If I query my table with 1.2.0, I have an exception : java.lang.AssertionError: Added column does not sort as the last column at org.apache.cassandra.db.ArrayBackedSortedColumns.addColumn(ArrayB= ackedSortedColumns.java:131) at org.apache.cassandra.db.AbstractColumnContainer.addColumn(Abstrac= tColumnContainer.java:109) at org.apache.cassandra.db.AbstractColumnContainer.addColumn(Abstrac= tColumnContainer.java:104) at org.apache.cassandra.db.filter.SliceQueryFilter.collectReducedCol= umns(SliceQueryFilter.java:171) at org.apache.cassandra.db.filter.QueryFilter.collateColumns(QueryFi= lter.java:136) at org.apache.cassandra.db.filter.QueryFilter.collateOnDiskAtom(Quer= yFilter.java:84) at org.apache.cassandra.db.CollationController.collectAllData(Collat= ionController.java:286) at org.apache.cassandra.db.CollationController.getTopLevelColumns(Co= llationController.java:61) at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(Colu= mnFamilyStore.java:1362) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnF= amilyStore.java:1222) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnF= amilyStore.java:1134) at org.apache.cassandra.db.Table.getRow(Table.java:348) at org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromRead= Command.java:70) at org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMa= yThrow(StorageProxy.java:1048) at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(S= torageProxy.java:1506) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown So= urce) at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source= ) at java.lang.Thread.run(Unknown Source) With 1.2.1 data doesn't load : ERROR Corrupt sstable ///_logs-ib-4= 2=3D[TOC.txt, Summary.db, Index .db, Data.db, Filter.db, Digest.sha1, Statistics.db]; skipped java.io.EOFException at java.io.DataInputStream.readFully(Unknown Source) at java.io.DataInputStream.readUTF(Unknown Source) at java.io.DataInputStream.readUTF(Unknown Source) at org.apache.cassandra.io.sstable.SSTableMetadata$SSTableMetadataS= erializer.deserialize(SSTableMetadata.java:27 8) at org.apache.cassandra.io.sstable.SSTableMetadata$SSTableMetadataS= erializer.deserialize(SSTableMetadata.java:25 1) at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader= .java:166) at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader= .java:149) at org.apache.cassandra.io.sstable.SSTableReader$1.run(SSTableReade= r.java:238) at java.util.concurrent.Executors$RunnableAdapter.call(Unknown Sour= ce) at java.util.concurrent.FutureTask$Sync.innerRun(Unknown Source) at java.util.concurrent.FutureTask.run(Unknown Source) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown S= ource) at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Sourc= e) at java.lang.Thread.run(Unknown Source) Now with your advice, my java code looks like : ... simpleUnsortedWriter.newRow(bytes(id)); ByteBuffer date =3D bytes(_date); ByteBuffer request =3D bytes(_request); long timestamp =3D System.currentTimeMillis() * 1000; addColumn(date, request, "", "", timestamp); addColumn(date, request, "data1", data1, timestamp); addColumn(date, request, "data2", data2, timestamp); ... public void addColumn(ByteBuffer date, ByteBuffer request, String field, String value, long ti= mestamp) throws IOException { builder =3D compositeType.builder(); builder.add(date); builder.add(request); builder.add(bytes(field)); simpleUnsortedWriter.addColumn(builder.build(), bytes(value), timestamp); } De : aaron morton [mailto:aaron@thelastpickle.com] Envoy=E9 : jeudi 31 janvier 2013 00:03 =C0 : user@cassandra.apache.org Objet : Re: SStable Writer and composite key This is what a row of your table will look like internally... ------------------- RowKey: id-value =3D> (column=3Ddate-value:request-value:, value=3D, timestamp=3D13595867394= 56000) =3D> (column=3Ddate-value:request-value:data1, value=3D64617461312d76616c75= 65, timestamp=3D1359586739456000) =3D> (column=3Ddate-value:request-value:data2, value=3D64617461322d76616c75= 65, timestamp=3D1359586739456000) 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 val= ue 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 > wrote: Hi, I have some trouble to request my data. I use SSTableSimpleUnsortedWriter t= o write SSTable. Writing and Importing works fine. I think, I'm misusing CompositeType.Builder with SSTableSimpleUnsortedWrite= r. 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> compositeList =3D new ArrayList>(); 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, partitioner,= keyspace.getKeyspaceName(), columnFamily= .getName(), UTF8Type.instance, null, 32); CompositeType.Builder builderRequestDate =3D new CompositeType.Builder( Com= positeType.getInstance (compositeList) ); CompositeType.Builder builderUrl =3D new CompositeType.Builder( CompositeTy= pe.getInstance(compositeList) ); simpleUnsortedWriter.newRow(bytes(id)); builderRequestDate.add(bytes("date")); builderRequestDate.add(bytes("request")); long timestamp =3D 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=E9nieur =E9tude et d=E9veloppement Tel : 01.84.95.11.20 Car & Boat Media 22 Rue Joubert 75009 Paris --_000_9BCDA0CDD97AB049903AEBDF966FA918110ACE25DB3PRD0710MB369_ Content-Type: text/html; charset="iso-8859-1" Content-Transfer-Encoding: quoted-printable

Ok, in order to solve&nbs= p;: java.lang.AssertionError: Added column do= es not sort as the last column

 <= /p>

I just have to make a scr= ub with nodetools.

 <= /p>

Is there a programmatic w= ay to scrub and compact new sstable on an active node ?

 <= /p>

Thx,

 <= /p>

Laurent Pouget=

 <= /p>

 <= /p>

De : POUG= ET Laurent [mailto:laurent.pouget@carboatmedia.fr]
Envoy=E9 : jeudi 31 janvier 2013 11:31
=C0 : user@cassandra.apache.org
Objet : RE: SStable Writer and composite key
<= /p>

 

Thanks,=

 <= /p>

Import is ok now but = ;:

=B7      &nb= sp;  If I query my tab= le with 1.2.0, I have an exception :

 <= /span>

java.lang.AssertionError:= Added column does not sort as the last column

    &= nbsp;  at org.apache.cassandra.db.ArrayBackedSortedColumns.addColumn(A= rrayBackedSortedColumns.java:131)

    &= nbsp;  at org.apache.cassandra.db.AbstractColumnContainer.addColumn(Ab= stractColumnContainer.java:109)

    &= nbsp;  at org.apache.cassandra.db.AbstractColumnContainer.addColumn(Ab= stractColumnContainer.java:104)

    &= nbsp;  at org.apache.cassandra.db.filter.SliceQueryFilter.collectReduc= edColumns(SliceQueryFilter.java:171)

    &= nbsp;  at org.apache.cassandra.db.filter.QueryFilter.collateColumns(Qu= eryFilter.java:136)

    &= nbsp;  at org.apache.cassandra.db.filter.QueryFilter.collateOnDiskAtom= (QueryFilter.java:84)

    &= nbsp;  at org.apache.cassandra.db.CollationController.collectAllData(C= ollationController.java:286)

    &= nbsp;  at org.apache.cassandra.db.CollationController.getTopLevelColum= ns(CollationController.java:61)

    &= nbsp;  at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns= (ColumnFamilyStore.java:1362)

    &= nbsp;  at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(Co= lumnFamilyStore.java:1222)

    &= nbsp;  at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(Co= lumnFamilyStore.java:1134)

    &= nbsp;  at org.apache.cassandra.db.Table.getRow(Table.java:348)

    &= nbsp;  at org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFro= mReadCommand.java:70)

    &= nbsp;  at org.apache.cassandra.service.StorageProxy$LocalReadRunnable.= runMayThrow(StorageProxy.java:1048)

    &= nbsp;  at org.apache.cassandra.service.StorageProxy$DroppableRunnable.= run(StorageProxy.java:1506)

    &= nbsp;  at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unkno= wn Source)

    &= nbsp;  at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown S= ource)

    &= nbsp;  at java.lang.Thread.run(Unknown Source)

 <= /p>

With 1.2.1 data doesnR= 17;t load :

 <= /p>

ERROR  Corrupt sstab= le <path>/<keyspace_name>/<cf_name>/<cf_name>_logs-= ib-42=3D[TOC.txt, Summary.db, Index

.db, Data.db, Filter.db, = Digest.sha1, Statistics.db]; skipped

java.io.EOFException=

    &= nbsp;   at java.io.DataInputStream.readFully(Unknown Source)=

    &= nbsp;   at java.io.DataInputStream.readUTF(Unknown Source)

    &= nbsp;   at java.io.DataInputStream.readUTF(Unknown Source)

    &= nbsp;   at org.apache.cassandra.io.sstable.SSTableMetadata$SSTabl= eMetadataSerializer.deserialize(SSTableMetadata.java:27

8)

    &= nbsp;   at org.apache.cassandra.io.sstable.SSTableMetadata$SSTabl= eMetadataSerializer.deserialize(SSTableMetadata.java:25

1)

    &= nbsp;   at org.apache.cassandra.io.sstable.SSTableReader.open(SST= ableReader.java:166)

    &= nbsp;   at org.apache.cassandra.io.sstable.SSTableReader.open(SST= ableReader.java:149)

    &= nbsp;   at org.apache.cassandra.io.sstable.SSTableReader$1.run(SS= TableReader.java:238)

    &= nbsp;   at java.util.concurrent.Executors$RunnableAdapter.call(Un= known Source)

    &= nbsp;   at java.util.concurrent.FutureTask$Sync.innerRun(Unknown = Source)

    &= nbsp;   at java.util.concurrent.FutureTask.run(Unknown Source)

    &= nbsp;   at java.util.concurrent.ThreadPoolExecutor$Worker.runTask= (Unknown Source)

    &= nbsp;   at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unk= nown Source)

    &= nbsp;   at java.lang.Thread.run(Unknown Source)=

 <= /p>

 <= /p>

Now with your advice, my = java code looks like :

 <= /p>

=

 <= /p>

simpleUnsortedWriter.newRow(byte= s(id));

     &= nbsp;      

ByteBuffer date =3D bytes(_date);

ByteBuffer request =3D bytes(_request);

 

long timestamp =3D Syste= m.currentTimeMillis() * 1000;

     &= nbsp;      

addColumn(date, request, = "", = "", timestamp);

addColumn(date, request, = "data1", data1, timestamp);

addColumn(date, request, = "data2", data2, timestamp);<= /o:p>

 <= /p>

=

 <= /p>

public void addColumn(ByteBuffer date,

     &= nbsp;           &nbs= p;  ByteBuffer request, String field, String value, long timestamp)

     &= nbsp;           &nbs= p;  throws IOException {

     &= nbsp; = builder =3D = compositeType.builder();

     &= nbsp;      

     &= nbsp; = builder.add(date);

     &= nbsp; = builder.add(request);

     &= nbsp; = builder.add(bytes(field));

     &= nbsp;      

     &= nbsp; = simpleUnsortedWriter.addColumn(builder.build(),

     &= nbsp;           &nbs= p;  bytes(value), timestamp);

}

 <= /p>

 <= /p>

 <= /p>

 <= /p>

 <= /p>

De : aaro= n morton [mailto:aaron@thelastpi= ckle.com]
Envoy=E9 : jeudi 31 janvier 2013 00:03
=C0 : user@cassand= ra.apache.org
Objet : Re: SStable Writer and composite key
<= /p>

 

This is what a row of your table will look like inte= rnally…

 

-------------------

RowKey: id-value

=3D> (column=3Ddate-value:request-value:, value= =3D, timestamp=3D1359586739456000)

=3D> (column=3Ddate-value:request-value:data1, va= lue=3D64617461312d76616c7565, timestamp=3D1359586739456000)

=3D> (column=3Ddate-value:request-value:data2, va= lue=3D64617461322d76616c7565, timestamp=3D1359586739456000)

 

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

 

So you need to construct triples for each column val= ue. 

The first col name is the (date-value, request-value= , empty) 

The second col name is (date-value, request-value, &= quot;data1") and has the value of data1. 

 

Hope that helps. 

 

-----------------

Aaron Morton

Freelance Cassandra Devel= oper

New Zealand

 <= /p>

@aaronmorton

 

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

 

Hi,<= o:p>

 

 

I have some trouble to r= equest my data. I use SSTableSimpleUnsortedWriter to write SSTable. Writing= and Importing works fine.

I think, I’m misus= ing CompositeType.Builder with SSTableSimpleUnsortedWriter.

Do you have any idea ?

 

Thanks

 

Here is my case :

 

/**<= o:p>

* CREATE STATEMENT

*/

 

CREATE TABLE raw_data (<= /span>

  id text,

  date text,=

  request text,

  data1 text,

  data2 text,

  PRIMARY KEY (id, = date, request)

) WITH

  bloom_filter_fp_c= hance=3D0.010000 AND

  caching=3D'KEYS_O= NLY' AND

  comment=3D'' AND<= /span>

  dclocal_read_repa= ir_chance=3D0.000000 AND

  gc_grace_seconds= =3D864000 AND

  read_repair_chanc= e=3D0.100000 AND

  replicate_on_writ= e=3D'true' AND

  compaction=3D{'cl= ass': 'SizeTieredCompactionStrategy'} AND

  compression=3D{'s= stable_compression': 'SnappyCompressor'};

 

/**<= o:p>

* JAVA CODE

*/

 

List<AbstractType<= ?>> compositeList =3D new ArrayList<AbstractType<?>>();

 

compositeList.add(UTF8Ty= pe.instance);

compositeList.add(UTF8Ty= pe.instance);

 

IPartitioner<?> pa= rtitioner =3D StorageService.getPartitioner();

dir =3D Directories.crea= te(keyspace.getKeyspaceName(), columnFamily.getName()).getDirectoryForNewSS= Tables(0);

 

simpleUnsortedWriter =3D= new SSTableSimpleUnsortedWriter(dir,=

    =             &nb= sp;            =             &nb= sp;            =          partitioner, keyspace.getK= eyspaceName(),

    =             &nb= sp;            =             &nb= sp;            =          columnFamily.getName(), UT= F8Type.instance, null,

    =             &nb= sp;            =             &nb= sp;            =          32);

 

CompositeType.Builder bu= ilderRequestDate =3D new CompositeType.Builder( CompositeType.getInstance&n= bsp;            &nbs= p;         (compositeList) );

CompositeType.Builder bu= ilderUrl =3D new CompositeType.Builder( CompositeType.getInstance(composite= List) );

 

simpleUnsortedWriter.new= Row(bytes(id));

    =             &nb= sp;            =  

builderRequestDate.add(b= ytes("date"));

builderRequestDate.add(b= ytes("request"));

 

long timestamp =3D Syste= m.currentTimeMillis() * 1000;

 

simpleUnsortedWriter.add= Column(builderRequestDate.build(),

    =             &nb= sp;            =   bytes(date), timestamp);

simpleUnsortedWriter.add= Column(builderUrl.build(),

    =             &nb= sp;            =   bytes(request), timestamp);

 

simpleUnsortedWriter.add= Column(bytes("data1"),

    =             &nb= sp;            =   bytes(data1), timestamp);

simpleUnsortedWriter.add= Column(bytes("data2"),

    =             &nb= sp;            =   bytes(data2), timestamp);

 

simpleUnsortedWriter.clo= se();

 

 

________________________= ________

 

Laurent Pouget

Ing=E9nieur =E9tude et d= =E9veloppement

Tel : 01.84.95.11.20

 

Car & Boat Media

22 Rue Joubert  750= 09 Paris

 

 

 

 

 

 

--_000_9BCDA0CDD97AB049903AEBDF966FA918110ACE25DB3PRD0710MB369_--