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 A024DDF2F for ; Fri, 29 Jun 2012 07:24:08 +0000 (UTC) Received: (qmail 88323 invoked by uid 500); 29 Jun 2012 07:24:06 -0000 Delivered-To: apmail-cassandra-user-archive@cassandra.apache.org Received: (qmail 88224 invoked by uid 500); 29 Jun 2012 07:24:05 -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 88202 invoked by uid 99); 29 Jun 2012 07:24:05 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 29 Jun 2012 07:24:05 +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 (nike.apache.org: domain of sylvain@datastax.com designates 209.85.220.172 as permitted sender) Received: from [209.85.220.172] (HELO mail-vc0-f172.google.com) (209.85.220.172) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 29 Jun 2012 07:23:59 +0000 Received: by vcqp1 with SMTP id p1so2293293vcq.31 for ; Fri, 29 Jun 2012 00:23:38 -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=FD8TMkxegc/A3ohEiKPt6ICZN2NH87fVMsLET2zUMtI=; b=GGY2H0t0FkIkzAEvaIcSKxyOkRxUaH1fLvzQMVPRJ50F0w9h19Jizv64bECDbMFRJ+ yaUPGTncNWb7FGACZVgtreJV8TTYrAERRjavDL+yRnSuTqOSrDhUpzoC0z/PYpmirE2U 1qgE5P8keIqns1MNXrjkMVOf4TjUvDfG75nZ1Ff6vsbzmj/V6syt/Td92Qc1lNf11jwe lRGQhNfLpLuxzdWNCpmWvjlkn+X2TxnEQgyu6EVB2VK/oTQQLYVa5nysta0g/huVmZj5 zWgGhAlwELkxjddYawCpfKBKtpW2nw56YSxI3NrpqjVGAqFu6iUV2zlq2zhmww47fB96 IZ1g== MIME-Version: 1.0 Received: by 10.52.19.133 with SMTP id f5mr209867vde.41.1340954617996; Fri, 29 Jun 2012 00:23:37 -0700 (PDT) Received: by 10.220.27.70 with HTTP; Fri, 29 Jun 2012 00:23:37 -0700 (PDT) In-Reply-To: References: Date: Fri, 29 Jun 2012 09:23:37 +0200 Message-ID: Subject: Re: 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: ALoCoQmG7kUUEsjPqfaCRZR+gnhkg3YIt5x4qftM/HzkIl5+kFcfMYhVj2LchUIKoM7SA/lfhW// On Thu, Jun 28, 2012 at 6:43 PM, Henning Kropp wrote: > I would like to use the BulkOutputFormat so. Is it likely to cause the be= low > exception? If so, why? Can it be fixed? It could, and if that only happens when you use it, then there is a good change this is where there is a problem. But I'll admit I don't know the BulkOutputFormat = well enough to be definitive. Would you mind opening a ticket on https://issues.apache.org/jira/browse/CASSANDRA will all those details ? -- Sylvain > > regards > > ________________________________ > Am 26.06.2012 17:02 schrieb Sylvain Lebresne : > On Tue, Jun 26, 2012 at 4:00 PM, Henning Kropp wrote: >> Thanks for the reply. Should have thought about looking into the log fil= es >> sooner. An AssertionError happens at execution. I haven't figured out ye= t >> 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.addColumn(ArrayBackedSo= rtedColumns.java:130) >> =A0 =A0 =A0 =A0at >> org.apache.cassandra.db.AbstractColumnContainer.addColumn(AbstractColumn= Container.java:107) >> =A0 =A0 =A0 =A0at >> org.apache.cassandra.db.AbstractColumnContainer.addColumn(AbstractColumn= Container.java:102) >> =A0 =A0 =A0 =A0at >> org.apache.cassandra.db.filter.SliceQueryFilter.collectReducedColumns(Sl= iceQueryFilter.java:141) >> =A0 =A0 =A0 =A0at >> org.apache.cassandra.db.filter.QueryFilter.collateColumns(QueryFilter.ja= va:139) >> =A0 =A0 =A0 =A0at >> org.apache.cassandra.db.CollationController.collectAllData(CollationCont= roller.java:283) >> =A0 =A0 =A0 =A0at >> org.apache.cassandra.db.CollationController.getTopLevelColumns(Collation= Controller.java:63) >> =A0 =A0 =A0 =A0at >> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamil= yStore.java:1321) >> =A0 =A0 =A0 =A0at >> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilySt= ore.java:1183) >> =A0 =A0 =A0 =A0at >> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilySt= ore.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(SliceFromReadCommand= .java:69) >> =A0 =A0 =A0 =A0at >> org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(= StorageProxy.java:816) >> =A0 =A0 =A0 =A0at >> org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageP= roxy.java:1250) >> =A0 =A0 =A0 =A0at >> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecuto= r.java:886) >> =A0 =A0 =A0 =A0at >> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.ja= va: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 st= ill >> lack a profound understanding of cassandras architecture to have a clue. >> And while client side filtering might seem like a valid option I am stil= l >> trying to get might head around a cassandra data model that would allow >> this. >> >> 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.SimpleStrategy'; >>> >>> 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 >>> the >>> column name: >>> >>> Column col =3D new Column(composite.serialize()); >>> >>> This all seems to work fine until I try to execute the following query >>> which >>> leads to a request timeout: >>> >>> SELECT datum FROM logs WHERE id=3D'861' and ref =3D 'raaf' and time > '= 3000'; >> >> 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 kn= ow >>> 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 o= f >>> the >>> column key? >> >> As described above, you can define a range on the second component, but >> then you >> won't be able to restrict on the 3rd component. >> >>> >>> Any thoughts? >>> >>> Thanks in advance and kind regards >>> Henning >>>