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 2B9CFFDDA for ; Thu, 30 May 2013 20:35:07 +0000 (UTC) Received: (qmail 16439 invoked by uid 500); 30 May 2013 20:35:04 -0000 Delivered-To: apmail-cassandra-user-archive@cassandra.apache.org Received: (qmail 16376 invoked by uid 500); 30 May 2013 20:35:04 -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 16368 invoked by uid 99); 30 May 2013 20:35:04 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 30 May 2013 20:35:04 +0000 X-ASF-Spam-Status: No, hits=1.5 required=5.0 tests=HTML_MESSAGE,RCVD_IN_DNSWL_LOW,SPF_PASS X-Spam-Check-By: apache.org Received-SPF: pass (athena.apache.org: domain of daniel@djmorton.com designates 209.85.214.179 as permitted sender) Received: from [209.85.214.179] (HELO mail-ob0-f179.google.com) (209.85.214.179) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 30 May 2013 20:35:00 +0000 Received: by mail-ob0-f179.google.com with SMTP id xk17so1525340obc.38 for ; Thu, 30 May 2013 13:34:39 -0700 (PDT) X-Google-DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=google.com; s=20120113; h=mime-version:x-originating-ip:in-reply-to:references:date :message-id:subject:from:to:content-type:x-gm-message-state; bh=FlbkSczGhK+jKf6ZvgtT1ncM1UI9B5/wSB/HoLBscMQ=; b=I/4AKoo0CwwHtrDkUJDfu8f0UML0nhFkJ5A69L8g5vg819QT37oaVRIEY8ZM3VywXi 0ZQU78WCEZ3mg9Lz2tZn+hHLqtFiNpvTThX/lmdmW5tvNYA8GsgCMyzJpU/2vw2lcDwB szDPItF0GY9FuqaWnB0bpVNVtVS/gPNv3SOk/BjWUhT5A2jDB7RxoDgUkWcwPA/yPYlo Cycy82CtgZdeEPxToizrMZr/W/ZRPUKjz4RW/TMUZN4GzZxIKxlUMrcDUwV+FnSvwkMR h8XqCuZ69IT5rrvoykIgtNSkOEcEEdQIsm0V4vNpvv8PA2jYS9zQCDdrRmr2O7tVi94t dfsA== MIME-Version: 1.0 X-Received: by 10.182.129.42 with SMTP id nt10mr4671389obb.54.1369946079255; Thu, 30 May 2013 13:34:39 -0700 (PDT) Received: by 10.76.35.37 with HTTP; Thu, 30 May 2013 13:34:39 -0700 (PDT) X-Originating-IP: [69.17.173.206] In-Reply-To: References: Date: Thu, 30 May 2013 16:34:39 -0400 Message-ID: Subject: Re: Bulk loading into CQL3 Composite Columns From: Daniel Morton To: user@cassandra.apache.org Content-Type: multipart/alternative; boundary=e89a8fb1fbfe6abcc204ddf56d63 X-Gm-Message-State: ALoCoQnXnpm8q7JDsaV/BBsCBXHxccwwDQOof2/IpjQM9aDdCd20+wtDrxW+toA5jNj3C8M6c4Hs X-Virus-Checked: Checked by ClamAV on apache.org --e89a8fb1fbfe6abcc204ddf56d63 Content-Type: text/plain; charset=ISO-8859-1 Hi Edward... Thanks for the pointer. I will use that going forward. Daniel Morton On Thu, May 30, 2013 at 4:09 PM, Edward Capriolo wrote: > You should probably be using system.nanoTime() not > system.currentTimeInMillis(). The user is free to set the timestamp to > whatever they like but nano-time is the standard (it is what the cli uses, > and what cql will use) > > > On Thu, May 30, 2013 at 3:33 PM, Keith Wright wrote: > >> Sorry, typo in code sample, should be: >> >> ssTableWriter.newRow(StringSerializer.get().toByteBuffer("20101201")); >> Composite columnComposite = new Composite(); >> columnComposite.setComponent(0,5,IntegerSerializer.get()); >> columnComposite.setComponent(1,10,IntegerSerializer.get()); >> >> ssTableWriter.addColumn( CompositeSerializer.get().toByteBuffer(columnComposite), null, System.currentTimeMillis() ); >> >> From: Keith Wright >> Date: Thursday, May 30, 2013 3:32 PM >> To: "user@cassandra.apache.org" >> Subject: Re: Bulk loading into CQL3 Composite Columns >> >> You do not want to repeat the first item of your primary key again. If >> you recall, in CQL3 a primary key as defined below indicates that the row >> key is the first item (key) and then the column names are composites of >> val1,val2. Although I don't see why you need val2 as part of the primary >> key in this case. In any event, you would do something like this (although >> I've never tested passing a null value): >> >> ssTableWriter.newRow(StringSerializer.get().toByteBuffer("20101201")); >> Composite columnComposite = new Composite(); >> columnComposite(0,5,IntegerSerializer.get()); >> columnComposite(0,10,IntegerSerializer.get()); >> ssTableWriter.addColumn( >> CompositeSerializer.get().toByteBuffer(columnComposite), >> null, >> System.currentTimeMillis() >> ); >> >> From: Daniel Morton >> Reply-To: "user@cassandra.apache.org" >> Date: Thursday, May 30, 2013 1:06 PM >> To: "user@cassandra.apache.org" >> Subject: Bulk loading into CQL3 Composite Columns >> >> Hi All. I am trying to bulk load some data into a CQL3 table using the >> sstableloader utility and I am having some difficulty figuring out how to >> use the SSTableSimpleUnsortedWriter with composite columns. >> >> I have created this simple contrived table for testing: >> >> create table test (key varchar, val1 int, val2 int, primary key (key, >> val1, val2)); >> >> Loosely following the bulk loading example in the docs, I have >> constructed the following method to create my temporary SSTables. >> >> public static void main(String[] args) throws Exception { >> final List> compositeTypes = new ArrayList<>(); >> compositeTypes.add(UTF8Type.instance); >> compositeTypes.add(IntegerType.instance); >> compositeTypes.add(IntegerType.instance); >> final CompositeType compType = >> CompositeType.getInstance(compositeTypes); >> SSTableSimpleUnsortedWriter ssTableWriter = >> new SSTableSimpleUnsortedWriter( >> new File("/tmp/cassandra_bulk/bigdata/test"), >> new Murmur3Partitioner() , >> "bigdata", >> "test", >> compType, >> null, >> 128); >> >> final Builder builder = >> new CompositeType.Builder(compType); >> >> builder.add(bytes("20101201")); >> builder.add(bytes(5)); >> builder.add(bytes(10)); >> >> ssTableWriter.newRow(bytes("20101201")); >> ssTableWriter.addColumn( >> builder.build(), >> ByteBuffer.allocate(0), >> System.currentTimeMillis() >> ); >> >> ssTableWriter.close(); >> } >> >> When I execute this method and load the data using sstableloader, if I do >> a 'SELECT * FROM test' in cqlsh, I get the results: >> >> key | val1 | val2 >> ---------------------------- >> 20101201 | '20101201' | 5 >> >> And the error: Failed to decode value '20101201' (for column 'val1') as >> int. >> >> The error I get makes sense, as apparently it tried to place the key >> value into the val1 column. From this error, I then assumed that the key >> value should not be part of the composite type when the row is added, so I >> removed the UTF8Type from the composite type, and only added the two >> integer values through the builder, but when I repeat the select with that >> data loaded, Cassandra throws an ArrayIndexOutOfBoundsException in the >> ColumnGroupMap class. >> >> Can anyone offer any advice on the correct way to insert data via the >> bulk loading process into CQL3 tables with composite columns? Does the >> fact that I am not inserting a value for the columns make a difference? >> For my particular use case, all I care about is the values in the column >> names themselves (and the associated sorting that goes with them). >> >> Any info or help anyone could provide would be very much appreciated. >> >> Regards, >> >> Daniel Morton >> > > --e89a8fb1fbfe6abcc204ddf56d63 Content-Type: text/html; charset=ISO-8859-1 Content-Transfer-Encoding: quoted-printable
Hi Edward... Thanks for the pointer. =A0I will use that go= ing forward.

Daniel Morton


On Thu, May 30, 2013 at 4:09 PM= , Edward Capriolo <edlinuxguru@gmail.com> wrote:
You should probably be usin= g system.nanoTime() not system.currentTimeInMillis(). The user is free to s= et the timestamp to whatever they like but nano-time is the standard (it is= what the cli uses, and what cql will use)

On Thu, May 30, 2013 at 3:33 PM, Keith Wri= ght <kwright@nanigans.com> wrote:
Sorry, typo in code sample, shou= ld be:

ssTableWriter.newRow(StringSerializer.get().toByte= Buffer("20101201"));
Composite columnComposite =3D new = Composite();
columnComposite.setComponent(0,5,IntegerSerializer.g= et());
columnComposite.setComponent(1,10,IntegerSerializer.get());
= ssTableWriter.addColumn(=A0CompositeSerializer.get().toByteBuffer(columnCom= posite),=A0null,=A0System.currentTimeMillis()=A0);

From: Keith Wright <kwright@nanigans.com>=
Date: Thursday, May 30, 2013 3= :32 PM
To: "user@cassandra.apache.org" = <user@cas= sandra.apache.org>
Subject: Re: Bulk loading into CQL= 3 Composite Columns

You do not wan= t to repeat the first item of your primary key again. =A0If you recall, in = CQL3 a primary key as defined below indicates that the row key is the first= item (key) and then the column names are composites of val1,val2. =A0Altho= ugh I don't see why you need val2 as part of the primary key in this case. =A0In any event, yo= u would do something like this (although I've never tested passing a nu= ll value):

ssTableWriter.newRow(StringSerializer.g= et().toByteBuffer("20101201"));
Composite columnComposite =3D new Composite();
columnComposi= te(0,5,IntegerSerializer.get());
columnComposite(0,10,Intege= rSerializer.get());
ssTableWriter.addColumn(
CompositeSerializer.get().toByteBuf= fer(columnComposite),
null,
System.currentTimeMillis()
);

From: Daniel Morton <daniel@djmorton.com>Reply-To: "user@cassandra.apache.org= " <u= ser@cassandra.apache.org>
Date: Thursday, May 30, 2013 1:06 P= M
To: "user@cassandra.apache.org&qu= ot; <user= @cassandra.apache.org>
Subject: Bulk loading into CQL3 Com= posite Columns

Hi A= ll. =A0I am trying to bulk load some data into a CQL3 table using the sstab= leloader utility and I am having some difficulty figuring out how to use th= e SSTableSimpleUnsortedWriter with composite columns.

I have created this simple contrived table for testing:=

create table test (key varchar, val1 int, val2 in= t, primary key (key, val1, val2));

Loosely followi= ng the bulk loading example in the docs, I have constructed the following m= ethod to create my temporary SSTables.

public static void main(String[] args) throws Exception= {
=A0 =A0final List<AbstractType<?>> compositeTypes = =3D new ArrayList<>();
=A0 =A0compositeTypes.add(UTF8Type.i= nstance);
=A0 =A0compositeTypes.add(IntegerType.instance);
=A0 =A0comp= ositeTypes.add(IntegerType.instance);
=A0 =A0final CompositeType compType =3D=A0
=A0 =A0 =A0 CompositeType.getInstance(compositeTypes);
=A0 =A0SSTableS= impleUnsortedWriter ssTableWriter =3D=A0
=A0 =A0 =A0 new SSTableS= impleUnsortedWriter(
= =A0 =A0 =A0 =A0 =A0new File("/tmp/cassandra_bulk/bigdata/test"),= =A0
=A0 =A0 =A0 =A0 =A0new Murmur3Partitioner() ,=A0
=A0 =A0 =A0= =A0 =A0"bigdata",=A0
=A0 =A0 =A0 =A0 =A0"test&quo= t;,=A0
=A0 =A0 =A0 =A0 =A0compType,=A0
=A0 =A0 =A0 =A0 = =A0null,=A0
=A0 =A0 =A0 =A0 =A0128);

=A0 =A0final Builder builder =3D=A0
=A0 =A0 = =A0 new CompositeType.Builder(compType);

=A0 =A0bu= ilder.add(bytes("20101201"));
=A0 =A0builder.add(bytes(= 5));
=A0 =A0builder.add(bytes(10));

=A0 =A0ssTableWriter.newRow(bytes("20101201")= );
=A0 =A0ssTableWrit= er.addColumn(
=A0 =A0 =A0 =A0 =A0builder.build(),=A0
= =A0 =A0 =A0 =A0 =A0ByteBuffer.allocate(0),=A0
=A0 =A0 =A0 =A0 =A0System.currentTimeMillis()
=A0 =A0);
=A0 =A0
=A0 = =A0ssTableWriter.close();
}

When I execu= te this method and load the data using sstableloader, if I do a 'SELECT= * FROM test' in cqlsh, I get the results:

key =A0 =A0 =A0| val1 =A0 =A0 =A0 | val2
----= ------------------------
20101201 | '20101201' | 5
<= div>
And the error: =A0Failed to decode value '20101201&#= 39; (for column 'val1') as int.

The error I get makes sense, as apparently it tried to = place the key value into the val1 column. =A0From this error, I then assume= d that the key value should not be part of the composite type when the row = is added, so I removed the UTF8Type from the composite type, and only added the two integer values through the builder, but when = I repeat the select with that data loaded, Cassandra throws an ArrayIndexOu= tOfBoundsException in the ColumnGroupMap class.

Can anyone offer any advice on the correct way to insert data via the bulk = loading process into CQL3 tables with composite columns? =A0Does the fact t= hat I am not inserting a value for the columns make a difference? =A0For my= particular use case, all I care about is the values in the column names themselves (and the associated sor= ting that goes with them).

Any info or help anyone= could provide would be very much appreciated.

Regards,

Daniel Morton


--e89a8fb1fbfe6abcc204ddf56d63--