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 EA224DCC6 for ; Tue, 26 Jun 2012 15:02:49 +0000 (UTC) Received: (qmail 3998 invoked by uid 500); 26 Jun 2012 15:02:47 -0000 Delivered-To: apmail-cassandra-user-archive@cassandra.apache.org Received: (qmail 3969 invoked by uid 500); 26 Jun 2012 15:02:47 -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 3961 invoked by uid 99); 26 Jun 2012 15:02:47 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 26 Jun 2012 15:02:47 +0000 X-ASF-Spam-Status: No, hits=-0.7 required=5.0 tests=FSL_RCVD_USER,RCVD_IN_DNSWL_LOW,SPF_PASS X-Spam-Check-By: apache.org Received-SPF: pass (athena.apache.org: domain of sylvain@datastax.com designates 209.85.214.172 as permitted sender) Received: from [209.85.214.172] (HELO mail-ob0-f172.google.com) (209.85.214.172) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 26 Jun 2012 15:02:43 +0000 Received: by obbwc20 with SMTP id wc20so9411459obb.31 for ; Tue, 26 Jun 2012 08:02:22 -0700 (PDT) X-Google-DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=google.com; s=20120113; h=mime-version:in-reply-to:references:date:message-id:subject:from:to :content-type:content-transfer-encoding:x-gm-message-state; bh=4YLl/RqdlzGKfHWijS6xriLR2ig/xxVRVKjiHtwv8oQ=; b=WlI34VOxKvGF5wlCaacXiB4wNBP9A/oGXr/o9p7omj1GPutZjFC4yT+F1XODlD0EJD Yn53QqNP4rFJTAlZJ/On2Os/khzEOVm53nqWgcmHFyDE+cLKdr4S2ViLzd00gfPD0Q04 FNmLgS04Ft5fQ3vjwZPP769jnhlgBrIFB2T5szj4KkOvDVZqLcjg9Z4v3GztIXfmSJny IwmnvEWdEAQpcaU7YBzxvARQqc5gk4/7Cq5rd85zNPye0zjGDBluW2+TYKjF1KlfhyPD C89AADxTYybn5XOrzuZmVJhVOP+ElqssMFdr96lhoqsp5E2txWaIy8lbcL7Aj4K4Ot2/ bJSA== MIME-Version: 1.0 Received: by 10.182.49.99 with SMTP id t3mr7777543obn.60.1340722942102; Tue, 26 Jun 2012 08:02:22 -0700 (PDT) Received: by 10.182.17.136 with HTTP; Tue, 26 Jun 2012 08:02:21 -0700 (PDT) In-Reply-To: References: Date: Tue, 26 Jun 2012 17:02:21 +0200 Message-ID: Subject: Re: Request Timeout with Composite Columns and CQL3 From: Sylvain Lebresne To: user@cassandra.apache.org Content-Type: text/plain; charset=ISO-8859-1 Content-Transfer-Encoding: quoted-printable X-Gm-Message-State: ALoCoQnXNQ/hwM+xxEbnOLUMoY0FcVQZtL4INJavpHLt8AkSLxLip5rzNPLYivQjFE7ozS/TE98N X-Virus-Checked: Checked by ClamAV on apache.org On Tue, Jun 26, 2012 at 4:00 PM, Henning Kropp wrote: > Thanks for the reply. Should have thought about looking into the log file= s sooner. An AssertionError happens at execution. I haven't figured out yet= why. Any input is very much appreciated: > > ERROR [ReadStage:1] 2012-06-26 15:49:54,481 AbstractCassandraDaemon.java = (line 134) Exception in thread Thread[ReadStage:1,5,main] > java.lang.AssertionError: Added column does not sort as the last column > =A0 =A0 =A0 =A0at org.apache.cassandra.db.ArrayBackedSortedColumns.addCol= umn(ArrayBackedSortedColumns.java:130) > =A0 =A0 =A0 =A0at org.apache.cassandra.db.AbstractColumnContainer.addColu= mn(AbstractColumnContainer.java:107) > =A0 =A0 =A0 =A0at org.apache.cassandra.db.AbstractColumnContainer.addColu= mn(AbstractColumnContainer.java:102) > =A0 =A0 =A0 =A0at org.apache.cassandra.db.filter.SliceQueryFilter.collect= ReducedColumns(SliceQueryFilter.java:141) > =A0 =A0 =A0 =A0at org.apache.cassandra.db.filter.QueryFilter.collateColum= ns(QueryFilter.java:139) > =A0 =A0 =A0 =A0at org.apache.cassandra.db.CollationController.collectAllD= ata(CollationController.java:283) > =A0 =A0 =A0 =A0at org.apache.cassandra.db.CollationController.getTopLevel= Columns(CollationController.java:63) > =A0 =A0 =A0 =A0at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelCo= lumns(ColumnFamilyStore.java:1321) > =A0 =A0 =A0 =A0at org.apache.cassandra.db.ColumnFamilyStore.getColumnFami= ly(ColumnFamilyStore.java:1183) > =A0 =A0 =A0 =A0at org.apache.cassandra.db.ColumnFamilyStore.getColumnFami= ly(ColumnFamilyStore.java:1118) > =A0 =A0 =A0 =A0at org.apache.cassandra.db.Table.getRow(Table.java:374) > =A0 =A0 =A0 =A0at org.apache.cassandra.db.SliceFromReadCommand.getRow(Sli= ceFromReadCommand.java:69) > =A0 =A0 =A0 =A0at org.apache.cassandra.service.StorageProxy$LocalReadRunn= able.runMayThrow(StorageProxy.java:816) > =A0 =A0 =A0 =A0at org.apache.cassandra.service.StorageProxy$DroppableRunn= able.run(StorageProxy.java:1250) > =A0 =A0 =A0 =A0at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(= ThreadPoolExecutor.java:886) > =A0 =A0 =A0 =A0at java.util.concurrent.ThreadPoolExecutor$Worker.run(Thre= adPoolExecutor.java:908) > =A0 =A0 =A0 =A0at java.lang.Thread.run(Thread.java:662) Obviously that shouldn't happen. You didn't happen to change the comparator for the column family or something like that from the hector side? Are you able to reproduce from a blank DB? -- Sylvain > > > BTW: I really would love to understand as of why the combined comparator = will not allow two ranges be specified for two key parts. Obviously I still= lack a profound understanding of cassandras architecture to have a clue. > And while client side filtering might seem like a valid option I am still= trying to get might head around a cassandra data model that would allow th= is. > > best regards > > ________________________________________ > Von: Sylvain Lebresne [sylvain@datastax.com] > Gesendet: Dienstag, 26. Juni 2012 10:21 > Bis: user@cassandra.apache.org > Betreff: Re: Request Timeout with Composite Columns and CQL3 > > On Mon, Jun 25, 2012 at 11:10 PM, Henning Kropp wrote: >> Hi, >> >> I am running into timeout issues using composite columns in cassandra 1.= 1.1 >> and cql 3. >> >> My keyspace and table is defined as the following: >> >> create keyspace bn_logs >> =A0 =A0 with strategy_options =3D [{replication_factor:1}] >> =A0 =A0 and placement_strategy =3D 'org.apache.cassandra.locator.SimpleS= trategy'; >> >> CREATE TABLE logs ( >> =A0 id text, >> =A0 ref text, >> =A0 time bigint, >> =A0 datum text, >> =A0 PRIMARY KEY(id, ref, time) >> ); >> >> I import some data to the table by using a combination of the thrift >> interface and the hector Composite.class by using its serialization as t= he >> column name: >> >> Column col =3D new Column(composite.serialize()); >> >> This all seems to work fine until I try to execute the following query w= hich >> leads to a request timeout: >> >> SELECT datum FROM logs WHERE id=3D'861' and ref =3D 'raaf' and time > '3= 000'; > > If it timeouts the likely reason is that this query selects more data > than the machine is able to fetch before the timeout. You can either > add a limit to the query, or increase the timeout. > If that doesn't seem to fix it, it might be worth checking the server > log to see if there isn't an error. > >> I really would like to figure out, why running this query on my laptop >> (single node, for development) will not finish. I also would like to kno= w if >> the following query would actually work >> >> SELECT datum FROM logs WHERE id=3D'861' and ref =3D 'raaf*' and time > '= 3000'; > > It won't. You can perform the following query: > > SELECT datum FROM logs WHERE id=3D'861' and ref =3D 'raaf'; > > which will select every datum whose ref starts with 'raaf', but then > you cannot restrict > the time parameter, so you will get ref where the time is <=3D 3000. Of > course you can > always filter client side if that is an option. > >> or how else there is a way to define a range for the second component of= the >> column key? > > As described above, you can define a range on the second component, but t= hen you > won't be able to restrict on the 3rd component. > >> >> Any thoughts? >> >> Thanks in advance and kind regards >> Henning >>