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 B8DF5979B for ; Fri, 30 Dec 2011 09:44:59 +0000 (UTC) Received: (qmail 61635 invoked by uid 500); 30 Dec 2011 09:44:57 -0000 Delivered-To: apmail-cassandra-user-archive@cassandra.apache.org Received: (qmail 61521 invoked by uid 500); 30 Dec 2011 09:44:46 -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 61505 invoked by uid 99); 30 Dec 2011 09:44:42 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 30 Dec 2011 09:44:42 +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 (nike.apache.org: domain of watcherfr@gmail.com designates 209.85.212.172 as permitted sender) Received: from [209.85.212.172] (HELO mail-wi0-f172.google.com) (209.85.212.172) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 30 Dec 2011 09:44:34 +0000 Received: by wibhj6 with SMTP id hj6so8511306wib.31 for ; Fri, 30 Dec 2011 01:44:13 -0800 (PST) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=gamma; h=mime-version:in-reply-to:references:date:message-id:subject:from:to :content-type; bh=98PWQKo/YbD5JFQ5I3poyKnaXLpFxCtIQQO1qR7qovQ=; b=hbMQysTJje5ojMOIIAqhDNJCwbZBU4pHKAta0zbTSI7YsA20DvTI3Y7QUSKGKNK/iA DNi4FUfs1jL3m0g0gAMrhHUVVFHCvUsffK9DIj4ea1YFq9X5cfuFMqMQgoPtrcSZyXEj 3pvAiGoLQ0xxgtiWOx+1Ma2XamI8/s2vmcMDE= MIME-Version: 1.0 Received: by 10.180.106.165 with SMTP id gv5mr84469866wib.18.1325238253826; Fri, 30 Dec 2011 01:44:13 -0800 (PST) Received: by 10.180.107.228 with HTTP; Fri, 30 Dec 2011 01:44:13 -0800 (PST) In-Reply-To: References: Date: Fri, 30 Dec 2011 10:44:13 +0100 Message-ID: Subject: Re: Retrieve all composite columns from a row, whose composite name's first component matches from a list of Integers From: Philippe To: user@cassandra.apache.org Content-Type: multipart/alternative; boundary=e89a8f3bad475cb2d704b54c13cd X-Virus-Checked: Checked by ClamAV on apache.org --e89a8f3bad475cb2d704b54c13cd Content-Type: text/plain; charset=ISO-8859-1 Content-Transfer-Encoding: quoted-printable I currently have scf[c1][sc1]=3Dvalue scf[c1][sc2]=3Dvalue ... scf[c2][sc1]=3Dvalue scf[c2][sc2]=3Dvalue scf[c2][sc3]=3Dvalue scf[c2][sc4]=3Dvalue 99% of the time, I do multiget super slices: for multiple keys, I query for columns explicitly c1,c2,c10,c12 1% of the time, I do a multigetrange superslice where for multiple keys, I query for a range of super columns As Tyler said, it can be done by specifying supercolumns in the slice predicate, it will implicitly return all its columns. I use Hector and it works great. Now interestingly enough, column names sc1, sc2, sc3 are in fact home-made composite columns. I could and would switch to full composite columns because I am fishing for every drop of performance I can. However, I would need "Letting multiget_slice accept multiple SlicePredicates per key could also accomplish this." Can anyone on the dev team comment on doing this ? Is it a no-no ? Thanks 2011/12/29 Edward Capriolo > Hum... > > Do you have this? > scf [b][1][a]=3Dvalue > scf [b][1][x]=3Dvalue > scf [b][7][b]=3Dvalue > > and you want to slice: > scf [b][1][*] > > Which would result in > > scf [b][1][a]=3Dvalue > scf [b][1][x]=3Dvalue > > ? > > The composite version of this would be: > cf [b][1:a]=3Dvalue > cf [b][1:x]=3Dvalue > cf [b][7:b]=3Dvalue > > I am not sure exactly what you are doing because A SlicePredicate > takes either a list of columns or a SliceRange. A ColumnPath takes a > Single SuperColumn. > > I do not see how this is done with Columns or SuperColumns. Maybe you > can provide a code snippet and/or some sample data? > > On 12/29/11, Aditya wrote: > > @Edward: Perhaps you missed to notice that I need to always retrieve 'a= ll > > columns' under the supercolumn at any time.. and as per my query > > requirements if I use composite columns instead of supercolumns then it > is > > impossible to do wildcard queries like the ones asked in this thread's > > headline but which is much easier to do through the use of supercolumns= . > > > > On Thu, Dec 29, 2011 at 11:06 PM, Edward Capriolo > > wrote: > > > >> The use case in question was: Only accessing some columns. > >> > >> Even if that is not the case: > >> > >> SuperColumns: 1 extra level of nesting > >> Composite Colunns: Arbitrary levels of nesting > >> > >> SuperColumns: More overhead (space on disk) then using your own > delimiter > >> '_' > >> SuperColumns: Likely going to be replaced in future c* version behind > >> the scenes by composite columns anyway > >> SuperColumns: Usually an afterthought for API developers, (support for > >> them comes "later") > >> SuperColumns: Almost always utilized incorrectly by users, users speak > >> of '10%' performance gains after they switch away from them. > >> > >> There are some (a small % of cases) where SuperColumns are a better > >> choice, but this is rare. With composites and concatenating columns > >> they have no great purpose any more, (bad analogy coming!) like a > >> mechanical type writer. > >> > >> On 12/29/11, Philippe wrote: > >> > Would you stand by that statement in case all colums inside the supe= r > >> > column need to be read? Why? > >> > > >> > Thanks > >> > Le 28 d=E9c. 2011 19:26, "Edward Capriolo" a > >> =E9crit : > >> > > >> >> Super columns have the same fundamental problem and perform worse i= n > >> >> general. So switching from composites to super columns is NEVER a > good > >> >> idea. > >> >> > >> >> > >> >> On Wed, Dec 28, 2011 at 1:19 PM, Aditya wrote: > >> >> > >> >>> Since I have around 20 items to query, I guess making 20 queries t= o > >> >>> retrieve activities by all followies on all of those 20 columns > would > >> too > >> >>> inefficient, so to take the advantage of more efficient queries, a= re > >> >>> supercolumns recommended for this case ? Anyways, in case I use > >> >>> supercolumns, I need to retrieve the entire supercolumn at any poi= nt > >> >>> of > >> >>> time & I am writing subcolumn(s) to the supercolumn at different > times > >> >>> not > >> >>> at once. > >> >>> > >> >>> On Wed, Dec 28, 2011 at 8:07 PM, Edward Capriolo > >> >>> wrote: > >> >>> > >> >>>> You need to execute one get slice operation for each item id or i= f > >> >>>> the > >> >>>> row is not large , you can try one large get slice on the entire > row > >> and > >> >>>> deal with the results client side. > >> >>>> > >> >>>> If you try method 1 When doing slices on composites you can set t= he > >> >>>> start inclusive or exclusive values to get only the column you wa= nt > >> and > >> >>>> not > >> >>>> some extra columns up to slice range size. > >> >>>> > >> >>>> > >> >>>> On Tuesday, December 27, 2011, Aditya wrote: > >> >>>> > I need to store data of all activities by user's followies in > >> >>>> > single > >> >>>> row. I am trying to do that making use of composite column names > in a > >> >>>> single user specific row named 'rowX'. > >> >>>> > On any activity by a user's followie on an item, a column is > stored > >> in > >> >>>> 'rowX'. The column has a composite type column name made up of > >> >>>> itemId+userId (which makes it unique col. name) in rowX. (& colum= n > >> value > >> >>>> contains the activity data related to that item by that followie) > >> >>>> > > >> >>>> > Now I want to retrieve activity by all users on a list of items= . > So > >> I > >> >>>> need to retrieve all composite columns with composite's first > >> component > >> >>>> matching the itemId. Is it possible to do such a query to > Cassandra ? > >> I > >> >>>> am > >> >>>> using Hector. > >> >>>> > >> >>> > >> >>> > >> >> > >> > > >> > > > --e89a8f3bad475cb2d704b54c13cd Content-Type: text/html; charset=ISO-8859-1 Content-Transfer-Encoding: quoted-printable I currently have=A0
scf[c1][sc1]=3Dvalue
scf[c1][sc2]=3Dvalue
...=
scf[c2][sc1]=3Dvalue
scf[c2][= sc2]=3Dvalue
scf[c2][sc3]=3Dvalue
scf[c2][sc4]=3Dvalue
=

99% of the time, I do multiget super slices: for multiple keys, = I query for columns=A0explicitly=A0c1,c2,c10,c12
1% of the time, = I do a multigetrange superslice where for multiple keys, I query for a rang= e of super columns
As Tyler said, it can be done by specifying supercolumns in the slice = predicate, it will=A0implicitly=A0return all its columns. I use Hector and = it works great.

Now interestingly enough, column n= ames sc1, sc2, sc3 are in fact home-made composite columns.

I could and would switch to full composite columns beca= use I am fishing for every drop of performance I can. However, I would need= "Letting multiget_slice accept= multiple SlicePredicates per key could also accomplish this."<= /div>
Can anyone on the dev team comm= ent on doing this ? Is it a no-no ?

Thanks

2011/12/29 Edward Capriolo = <edlinuxguru@= gmail.com>
Hum...

Do you have this?
scf [b][1][a]=3Dvalue
scf [b][1][x]=3Dvalue
scf [b][7][b]=3Dvalue

and you want to slice:
scf [b][1][*]

Which would result in

scf [b][1][a]=3Dvalue
scf [b][1][x]=3Dvalue

?

The composite version of this would be:
cf [b][1:a]=3Dvalue
cf [b][1:x]=3Dvalue
cf [b][7:b]=3Dvalue

I am not sure exactly what you are doing because A SlicePredicate
takes either a list of columns or a SliceRange. A ColumnPath takes a
Single SuperColumn.

I do not see how this is done with Columns or SuperColumns. Maybe you
can provide a code snippet and/or some sample data?

On 12/29/11, Aditya <adynnn@gmail.co= m> wrote:
> @Edward: Perhaps you missed to notice that I need to always retrieve &= #39;all
> columns' under the supercolumn at any time.. and as per my query > requirements if I use composite columns instead of supercolumns then i= t is
> impossible to do wildcard queries like the ones asked in this thread&#= 39;s
> headline but which is much easier to do through the use of supercolumn= s.
>
> On Thu, Dec 29, 2011 at 11:06 PM, Edward Capriolo
> <edlinuxguru@gmail.com= >wrote:
>
>> The use case in question was: Only accessing some columns.
>>
>> Even if that is not the case:
>>
>> SuperColumns: 1 extra level of nesting
>> Composite Colunns: Arbitrary levels of nesting
>>
>> SuperColumns: More overhead (space on disk) then using your own de= limiter
>> '_'
>> SuperColumns: Likely going to be replaced in future c* version beh= ind
>> the scenes by composite columns anyway
>> SuperColumns: Usually an afterthought for API developers, (support= for
>> them comes "later")
>> SuperColumns: Almost always utilized incorrectly by users, users s= peak
>> of '10%' performance gains after they switch away from the= m.
>>
>> There are some (a small % of cases) where SuperColumns are a bette= r
>> choice, but this is rare. With composites and concatenating column= s
>> they have no great purpose any more, (bad analogy coming!) like a<= br> >> mechanical type writer.
>>
>> On 12/29/11, Philippe <w= atcherfr@gmail.com> wrote:
>> > Would you stand by that statement in case all colums inside t= he super
>> > column need to be read? =A0Why?
>> >
>> > Thanks
>> > Le 28 d=E9c. 2011 19:26, "Edward Capriolo" <edlinuxguru@gmail.com> a
>> =E9crit :
>> >
>> >> Super columns have the same fundamental problem and perfo= rm worse in
>> >> general. So switching from composites to super columns is= NEVER a good
>> >> idea.
>> >>
>> >>
>> >> On Wed, Dec 28, 2011 at 1:19 PM, Aditya <adynnn@gmail.com> wrote:
>> >>
>> >>> Since I have around 20 items to query, I guess making= 20 queries to
>> >>> retrieve activities by all followies on all of those = 20 columns would
>> too
>> >>> inefficient, so to take the advantage of more efficie= nt queries, are
>> >>> supercolumns recommended for this case ? Anyways, in = case I use
>> >>> supercolumns, I need to retrieve the entire supercolu= mn at any point
>> >>> of
>> >>> time & I am writing subcolumn(s) to the supercolu= mn at different times
>> >>> not
>> >>> at once.
>> >>>
>> >>> On Wed, Dec 28, 2011 at 8:07 PM, Edward Capriolo
>> >>> <edlinuxg= uru@gmail.com>wrote:
>> >>>
>> >>>> You need to execute one get slice operation for e= ach item id or if
>> >>>> the
>> >>>> row is not large , you can try one large get slic= e on the entire row
>> and
>> >>>> deal with the results client side.
>> >>>>
>> >>>> If you try method 1 When doing slices on composit= es you can set the
>> >>>> start inclusive or exclusive values to get only t= he column you want
>> and
>> >>>> not
>> >>>> some extra columns up to slice range size.
>> >>>>
>> >>>>
>> >>>> On Tuesday, December 27, 2011, Aditya <adynnn@gmail.com> wrote:
>> >>>> > I need to store data of all activities by us= er's followies in
>> >>>> > single
>> >>>> row. I am trying to do that making use of composi= te column names in a
>> >>>> single user specific row named 'rowX'. >> >>>> > On any activity by a user's followie on = an item, a column is stored
>> in
>> >>>> 'rowX'. The column has a composite type c= olumn name made up of
>> >>>> itemId+userId (which makes it unique col. name) i= n rowX. (& column
>> value
>> >>>> contains the activity data related to that item b= y that followie)
>> >>>> >
>> >>>> > Now I want to retrieve activity by all users= on a list of items. So
>> I
>> >>>> need to retrieve all composite columns with compo= site's first
>> component
>> >>>> matching the itemId. Is it possible to do such a = query to Cassandra ?
>> I
>> >>>> am
>> >>>> using Hector.
>> >>>>
>> >>>
>> >>>
>> >>
>> >
>>
>

--e89a8f3bad475cb2d704b54c13cd--