You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@cassandra.apache.org by Ravikumar Govindarajan <ra...@gmail.com> on 2013/09/10 15:21:35 UTC

Composite Column Grouping

I have been faced with a problem of grouping composites on the second-part.

Lets say my CF contains this


TimeSeriesCF
                       key:                            UserID
                       composite-col-name:    TimeUUID:PKID

Some sample data

UserID = XYZ
                                 Time:PKID
               Col-Name1 = 200:1000
               Col-Name2 = 201:1001
               Col-Name3 = 202:1000
               Col-Name4 = 203:1000
               Col-Name5 = 204:1002

Whenever a time-series query is issued, it should return the following in
time-desc order.

UserID = XYZ
              Col-Name5 = 204:1002
              Col-Name4 = 203:1000
              Col-Name2 = 201:1001

Is something like this possible in Cassandra? Is there a different way to
design and achieve the same objective?

--
Ravi

Re: Composite Column Grouping

Posted by "Laing, Michael" <mi...@nytimes.com>.
Here's a slightly better version and a python script. -ml

-- put this in <file> and run using 'cqlsh -f <file>

DROP KEYSPACE latest;

CREATE KEYSPACE latest WITH replication = {
    'class': 'SimpleStrategy',
    'replication_factor' : 1
};

USE latest;

CREATE TABLE time_series (
    bucket_userid text, -- bucket is the beginning of a datetime span
concatenated with a shard designator
    user_id text,
    pkid text,
    timeuuid text,
    colname text,
    PRIMARY KEY (bucket_userid, timeuuid)
);

UPDATE time_series
SET
    user_id = 'XYZ',
    pkid = '1000',
    colname = 'Col-Name-1'
WHERE
    bucket_userid = '2013-09-11T05:15-0_XYZ' AND
    timeuuid='200'
;
UPDATE time_series
SET
    user_id = 'XYZ',
    pkid = '1001',
    colname = 'Col-Name-2'
WHERE
    bucket_userid = '2013-09-11T05:15-1_XYZ' AND
    timeuuid='201'
;
UPDATE time_series
SET
    user_id = 'XYZ',
    pkid = '1000',
    colname = 'Col-Name-3'
WHERE
    bucket_userid = '2013-09-11T05:15-0_XYZ' AND
    timeuuid='202'
;
UPDATE time_series
SET
    user_id = 'XYZ',
    pkid = '1000',
    colname = 'Col-Name-4'
WHERE
    bucket_userid = '2013-09-11T05:30-1_XYZ' AND
    timeuuid='203'
;
UPDATE time_series
SET
    user_id = 'XYZ',
    pkid = '1002',
    colname = 'Col-Name-5'
WHERE
    bucket_userid = '2013-09-11T05:30-0_XYZ' AND
    timeuuid='204'
;

-- This query assumes that the 'current' span is 2013-09-11T05:30 and I am
interested in this span and the previous one.

SELECT * FROM time_series
WHERE bucket_userid IN ( -- go back as many spans as you need to, all
shards in each span (cartesian product)
    '2013-09-11T05:15-0_XYZ',
    '2013-09-11T05:15-1_XYZ',
    '2013-09-11T05:30-0_XYZ',
    '2013-09-11T05:30-1_XYZ'
) -- you could add a range condition on timeuuid to further restrict the
results
ORDER BY timeuuid DESC;

-- returns:
-- bucket_userid          | timeuuid | colname    | pkid | user_id
--------------------------+----------+------------+------+---------
-- 2013-09-11T05:30-0_XYZ |      204 | Col-Name-5 | 1002 |     XYZ
-- 2013-09-11T05:30-1_XYZ |      203 | Col-Name-4 | 1000 |     XYZ
-- 2013-09-11T05:15-0_XYZ |      202 | Col-Name-3 | 1000 |     XYZ
-- 2013-09-11T05:15-1_XYZ |      201 | Col-Name-2 | 1001 |     XYZ
-- 2013-09-11T05:15-0_XYZ |      200 | Col-Name-1 | 1000 |     XYZ

-- do a stable purge on pkid to get the result


python script:

#!/usr/bin/env python
# -*- coding: utf-8 -*-
import logging
logging.basicConfig()

import cassandra
from cassandra.cluster import Cluster
from cassandra.query import SimpleStatement

cql_cluster = Cluster()
cql_session = cql_cluster.connect()
cql_session.set_keyspace('latest')

select_stmt = """
    SELECT * FROM time_series
    WHERE bucket_userid IN ( -- go back as many spans as you need to, all
shards in each span (cartesian product)
        '2013-09-11T05:15-0_XYZ',
        '2013-09-11T05:15-1_XYZ',
        '2013-09-11T05:30-0_XYZ',
        '2013-09-11T05:30-1_XYZ'
    )
    ORDER BY timeuuid DESC;
"""

query = SimpleStatement(select_stmt)
rows = cql_session.execute(query)

pkids = set()
for row in rows:
    if row.pkid in pkids:
        continue
    else:
        print row.user_id, row.timeuuid, row.colname, row.pkid
        pkids.add(row.pkid)

# prints:

# XYZ 204 Col-Name-5 1002
# XYZ 203 Col-Name-4 1000
# XYZ 201 Col-Name-2 1001


On Wed, Sep 11, 2013 at 6:13 AM, Laing, Michael
<mi...@nytimes.com>wrote:

> Then you can do this. I handle millions of entries this way and it works
> well if you are mostly interested in recent activity.
>
> If you need to span all activity then you can use a separate table to
> maintain the 'latest'. This table should also be sharded as entries will be
> 'hot'. Sharding will spread the heat and the tombstones (compaction load)
> around the cluster.
>
> -ml
>
> -- put this in <file> and run using 'cqlsh -f <file>
>
> DROP KEYSPACE latest;
>
> CREATE KEYSPACE latest WITH replication = {
>     'class': 'SimpleStrategy',
>     'replication_factor' : 1
> };
>
> USE latest;
>
> CREATE TABLE time_series (
>     bucket_userid text, -- bucket is the beginning of a datetime span
> concatenated with a shard designator
>     pkid text,
>     timeuuid text,
>     colname text,
>     PRIMARY KEY (bucket_userid, timeuuid)
> );
>
> -- the example table is using 15 minute bucket spans and 2 shards for
> illustration (you would usually use more shards)
> -- adjust these appropriately for your application
>
> UPDATE time_series SET pkid = '1000', colname = 'Col-Name-1' where
> bucket_userid = '2013-09-11T05:15-0_XYZ' AND timeuuid='200';
> UPDATE time_series SET pkid = '1001', colname = 'Col-Name-2' where
> bucket_userid = '2013-09-11T05:15-1_XYZ' AND timeuuid='201';
> UPDATE time_series SET pkid = '1000', colname = 'Col-Name-3' where
> bucket_userid = '2013-09-11T05:15-0_XYZ' AND timeuuid='202';
> UPDATE time_series SET pkid = '1000', colname = 'Col-Name-4' where
> bucket_userid = '2013-09-11T05:30-1_XYZ' AND timeuuid='203';
> UPDATE time_series SET pkid = '1002', colname = 'Col-Name-5' where
> bucket_userid = '2013-09-11T05:30-0_XYZ' AND timeuuid='204';
>
> -- This query assumes that the 'current' span is 2013-09-11T05:30 and I am
> interested in this span and the previous one.
>
> SELECT * FROM time_series
> WHERE bucket_userid in ( -- go back as many spans as you need to, all
> shards in each span (cartesian product)
>     '2013-09-11T05:15-0_XYZ',
>     '2013-09-11T05:15-1_XYZ',
>     '2013-09-11T05:30-0_XYZ',
>     '2013-09-11T05:30-1_XYZ'
> )
> ORDER BY timeuuid DESC;
>
> -- returns:
> -- bucket_userid          | timeuuid | colname    | pkid
> --------------------------+----------+------------+------
> -- 2013-09-11T05:30-0_XYZ |      204 | Col-Name-5 | 1002
> -- 2013-09-11T05:30-1_XYZ |      203 | Col-Name-4 | 1000
> -- 2013-09-11T05:15-0_XYZ |      202 | Col-Name-3 | 1000
> -- 2013-09-11T05:15-1_XYZ |      201 | Col-Name-2 | 1001
> -- 2013-09-11T05:15-0_XYZ |      200 | Col-Name-1 | 1000
>
> -- do a stable purge on pkid to get the result.
>
>
> On Wed, Sep 11, 2013 at 1:01 AM, Ravikumar Govindarajan <
> ravikumar.govindarajan@gmail.com> wrote:
>
>> Thanks Michael,
>>
>> But I cannot sort the rows in memory, as the number of columns will be
>> quite huge.
>>
>> From the python script above:
>>    select_stmt = "select * from time_series where userid = 'XYZ'"
>>
>> This would return me many hundreds of thousands of columns. I need to go
>> in time-series order using ranges [Pagination queries].
>>
>>
>> On Wed, Sep 11, 2013 at 7:06 AM, Laing, Michael <
>> michael.laing@nytimes.com> wrote:
>>
>>> If you have set up the table as described in my previous message, you
>>> could run this python snippet to return the desired result:
>>>
>>> #!/usr/bin/env python
>>> # -*- coding: utf-8 -*-
>>> import logging
>>> logging.basicConfig()
>>>
>>> from operator import itemgetter
>>>
>>> import cassandra
>>> from cassandra.cluster import Cluster
>>> from cassandra.query import SimpleStatement
>>>
>>> cql_cluster = Cluster()
>>> cql_session = cql_cluster.connect()
>>> cql_session.set_keyspace('latest')
>>>
>>> select_stmt = "select * from time_series where userid = 'XYZ'"
>>> query = SimpleStatement(select_stmt)
>>> rows = cql_session.execute(query)
>>>
>>> results = []
>>> for row in rows:
>>>     max_time = max(row.colname.keys())
>>>     results.append((row.userid, row.pkid, max_time,
>>> row.colname[max_time]))
>>>
>>> sorted_results = sorted(results, key=itemgetter(2), reverse=True)
>>> for result in sorted_results: print result
>>>
>>> # prints:
>>>
>>> # (u'XYZ', u'1002', u'204', u'Col-Name-5')
>>> # (u'XYZ', u'1000', u'203', u'Col-Name-4')
>>> # (u'XYZ', u'1001', u'201', u'Col-Name-2')
>>>
>>>
>>>
>>> On Tue, Sep 10, 2013 at 6:32 PM, Laing, Michael <
>>> michael.laing@nytimes.com> wrote:
>>>
>>>> You could try this. C* doesn't do it all for you, but it will
>>>> efficiently get you the right data.
>>>>
>>>> -ml
>>>>
>>>> -- put this in <file> and run using 'cqlsh -f <file>
>>>>
>>>> DROP KEYSPACE latest;
>>>>
>>>> CREATE KEYSPACE latest WITH replication = {
>>>>     'class': 'SimpleStrategy',
>>>>     'replication_factor' : 1
>>>> };
>>>>
>>>> USE latest;
>>>>
>>>> CREATE TABLE time_series (
>>>>     userid text,
>>>>     pkid text,
>>>>     colname map<text, text>,
>>>>     PRIMARY KEY (userid, pkid)
>>>> );
>>>>
>>>> UPDATE time_series SET colname = colname + {'200':'Col-Name-1'} WHERE
>>>> userid = 'XYZ' AND pkid = '1000';
>>>> UPDATE time_series SET colname = colname +
>>>> {'201':'Col-Name-2'} WHERE userid = 'XYZ' AND pkid = '1001';
>>>> UPDATE time_series SET colname = colname +
>>>> {'202':'Col-Name-3'} WHERE userid = 'XYZ' AND pkid = '1000';
>>>> UPDATE time_series SET colname = colname +
>>>> {'203':'Col-Name-4'} WHERE userid = 'XYZ' AND pkid = '1000';
>>>> UPDATE time_series SET colname = colname +
>>>> {'204':'Col-Name-5'} WHERE userid = 'XYZ' AND pkid = '1002';
>>>>
>>>> SELECT * FROM time_series WHERE userid = 'XYZ';
>>>>
>>>> -- returns:
>>>> -- userid | pkid | colname
>>>>
>>>> ----------+------+-----------------------------------------------------------------
>>>> --    XYZ | 1000 | {'200': 'Col-Name-1', '202': 'Col-Name-3', '203':
>>>> 'Col-Name-4'}
>>>> --    XYZ | 1001 |                                           {'201':
>>>> 'Col-Name-2'}
>>>> --    XYZ | 1002 |                                           {'204':
>>>> 'Col-Name-5'}
>>>>
>>>> -- use an app to pop off the latest key/value from the map for each
>>>> row, then sort by key desc.
>>>>
>>>>
>>>> On Tue, Sep 10, 2013 at 9:21 AM, Ravikumar Govindarajan <
>>>> ravikumar.govindarajan@gmail.com> wrote:
>>>>
>>>>> I have been faced with a problem of grouping composites on the
>>>>> second-part.
>>>>>
>>>>> Lets say my CF contains this
>>>>>
>>>>>
>>>>> TimeSeriesCF
>>>>>                        key:                            UserID
>>>>>                        composite-col-name:    TimeUUID:PKID
>>>>>
>>>>> Some sample data
>>>>>
>>>>> UserID = XYZ
>>>>>                                  Time:PKID
>>>>>                Col-Name1 = 200:1000
>>>>>                Col-Name2 = 201:1001
>>>>>                Col-Name3 = 202:1000
>>>>>                Col-Name4 = 203:1000
>>>>>                Col-Name5 = 204:1002
>>>>>
>>>>> Whenever a time-series query is issued, it should return the following
>>>>> in time-desc order.
>>>>>
>>>>> UserID = XYZ
>>>>>               Col-Name5 = 204:1002
>>>>>               Col-Name4 = 203:1000
>>>>>               Col-Name2 = 201:1001
>>>>>
>>>>> Is something like this possible in Cassandra? Is there a different way
>>>>> to design and achieve the same objective?
>>>>>
>>>>> --
>>>>> Ravi
>>>>>
>>>>>
>>>>
>>>>
>>>
>>
>

Re: Composite Column Grouping

Posted by "Laing, Michael" <mi...@nytimes.com>.
Then you can do this. I handle millions of entries this way and it works
well if you are mostly interested in recent activity.

If you need to span all activity then you can use a separate table to
maintain the 'latest'. This table should also be sharded as entries will be
'hot'. Sharding will spread the heat and the tombstones (compaction load)
around the cluster.

-ml

-- put this in <file> and run using 'cqlsh -f <file>

DROP KEYSPACE latest;

CREATE KEYSPACE latest WITH replication = {
    'class': 'SimpleStrategy',
    'replication_factor' : 1
};

USE latest;

CREATE TABLE time_series (
    bucket_userid text, -- bucket is the beginning of a datetime span
concatenated with a shard designator
    pkid text,
    timeuuid text,
    colname text,
    PRIMARY KEY (bucket_userid, timeuuid)
);

-- the example table is using 15 minute bucket spans and 2 shards for
illustration (you would usually use more shards)
-- adjust these appropriately for your application

UPDATE time_series SET pkid = '1000', colname = 'Col-Name-1' where
bucket_userid = '2013-09-11T05:15-0_XYZ' AND timeuuid='200';
UPDATE time_series SET pkid = '1001', colname = 'Col-Name-2' where
bucket_userid = '2013-09-11T05:15-1_XYZ' AND timeuuid='201';
UPDATE time_series SET pkid = '1000', colname = 'Col-Name-3' where
bucket_userid = '2013-09-11T05:15-0_XYZ' AND timeuuid='202';
UPDATE time_series SET pkid = '1000', colname = 'Col-Name-4' where
bucket_userid = '2013-09-11T05:30-1_XYZ' AND timeuuid='203';
UPDATE time_series SET pkid = '1002', colname = 'Col-Name-5' where
bucket_userid = '2013-09-11T05:30-0_XYZ' AND timeuuid='204';

-- This query assumes that the 'current' span is 2013-09-11T05:30 and I am
interested in this span and the previous one.

SELECT * FROM time_series
WHERE bucket_userid in ( -- go back as many spans as you need to, all
shards in each span (cartesian product)
    '2013-09-11T05:15-0_XYZ',
    '2013-09-11T05:15-1_XYZ',
    '2013-09-11T05:30-0_XYZ',
    '2013-09-11T05:30-1_XYZ'
)
ORDER BY timeuuid DESC;

-- returns:
-- bucket_userid          | timeuuid | colname    | pkid
--------------------------+----------+------------+------
-- 2013-09-11T05:30-0_XYZ |      204 | Col-Name-5 | 1002
-- 2013-09-11T05:30-1_XYZ |      203 | Col-Name-4 | 1000
-- 2013-09-11T05:15-0_XYZ |      202 | Col-Name-3 | 1000
-- 2013-09-11T05:15-1_XYZ |      201 | Col-Name-2 | 1001
-- 2013-09-11T05:15-0_XYZ |      200 | Col-Name-1 | 1000

-- do a stable purge on pkid to get the result.


On Wed, Sep 11, 2013 at 1:01 AM, Ravikumar Govindarajan <
ravikumar.govindarajan@gmail.com> wrote:

> Thanks Michael,
>
> But I cannot sort the rows in memory, as the number of columns will be
> quite huge.
>
> From the python script above:
>    select_stmt = "select * from time_series where userid = 'XYZ'"
>
> This would return me many hundreds of thousands of columns. I need to go
> in time-series order using ranges [Pagination queries].
>
>
> On Wed, Sep 11, 2013 at 7:06 AM, Laing, Michael <michael.laing@nytimes.com
> > wrote:
>
>> If you have set up the table as described in my previous message, you
>> could run this python snippet to return the desired result:
>>
>> #!/usr/bin/env python
>> # -*- coding: utf-8 -*-
>> import logging
>> logging.basicConfig()
>>
>> from operator import itemgetter
>>
>> import cassandra
>> from cassandra.cluster import Cluster
>> from cassandra.query import SimpleStatement
>>
>> cql_cluster = Cluster()
>> cql_session = cql_cluster.connect()
>> cql_session.set_keyspace('latest')
>>
>> select_stmt = "select * from time_series where userid = 'XYZ'"
>> query = SimpleStatement(select_stmt)
>> rows = cql_session.execute(query)
>>
>> results = []
>> for row in rows:
>>     max_time = max(row.colname.keys())
>>     results.append((row.userid, row.pkid, max_time,
>> row.colname[max_time]))
>>
>> sorted_results = sorted(results, key=itemgetter(2), reverse=True)
>> for result in sorted_results: print result
>>
>> # prints:
>>
>> # (u'XYZ', u'1002', u'204', u'Col-Name-5')
>> # (u'XYZ', u'1000', u'203', u'Col-Name-4')
>> # (u'XYZ', u'1001', u'201', u'Col-Name-2')
>>
>>
>>
>> On Tue, Sep 10, 2013 at 6:32 PM, Laing, Michael <
>> michael.laing@nytimes.com> wrote:
>>
>>> You could try this. C* doesn't do it all for you, but it will
>>> efficiently get you the right data.
>>>
>>> -ml
>>>
>>> -- put this in <file> and run using 'cqlsh -f <file>
>>>
>>> DROP KEYSPACE latest;
>>>
>>> CREATE KEYSPACE latest WITH replication = {
>>>     'class': 'SimpleStrategy',
>>>     'replication_factor' : 1
>>> };
>>>
>>> USE latest;
>>>
>>> CREATE TABLE time_series (
>>>     userid text,
>>>     pkid text,
>>>     colname map<text, text>,
>>>     PRIMARY KEY (userid, pkid)
>>> );
>>>
>>> UPDATE time_series SET colname = colname + {'200':'Col-Name-1'} WHERE
>>> userid = 'XYZ' AND pkid = '1000';
>>> UPDATE time_series SET colname = colname +
>>> {'201':'Col-Name-2'} WHERE userid = 'XYZ' AND pkid = '1001';
>>> UPDATE time_series SET colname = colname +
>>> {'202':'Col-Name-3'} WHERE userid = 'XYZ' AND pkid = '1000';
>>> UPDATE time_series SET colname = colname +
>>> {'203':'Col-Name-4'} WHERE userid = 'XYZ' AND pkid = '1000';
>>> UPDATE time_series SET colname = colname +
>>> {'204':'Col-Name-5'} WHERE userid = 'XYZ' AND pkid = '1002';
>>>
>>> SELECT * FROM time_series WHERE userid = 'XYZ';
>>>
>>> -- returns:
>>> -- userid | pkid | colname
>>>
>>> ----------+------+-----------------------------------------------------------------
>>> --    XYZ | 1000 | {'200': 'Col-Name-1', '202': 'Col-Name-3', '203':
>>> 'Col-Name-4'}
>>> --    XYZ | 1001 |                                           {'201':
>>> 'Col-Name-2'}
>>> --    XYZ | 1002 |                                           {'204':
>>> 'Col-Name-5'}
>>>
>>> -- use an app to pop off the latest key/value from the map for each row,
>>> then sort by key desc.
>>>
>>>
>>> On Tue, Sep 10, 2013 at 9:21 AM, Ravikumar Govindarajan <
>>> ravikumar.govindarajan@gmail.com> wrote:
>>>
>>>> I have been faced with a problem of grouping composites on the
>>>> second-part.
>>>>
>>>> Lets say my CF contains this
>>>>
>>>>
>>>> TimeSeriesCF
>>>>                        key:                            UserID
>>>>                        composite-col-name:    TimeUUID:PKID
>>>>
>>>> Some sample data
>>>>
>>>> UserID = XYZ
>>>>                                  Time:PKID
>>>>                Col-Name1 = 200:1000
>>>>                Col-Name2 = 201:1001
>>>>                Col-Name3 = 202:1000
>>>>                Col-Name4 = 203:1000
>>>>                Col-Name5 = 204:1002
>>>>
>>>> Whenever a time-series query is issued, it should return the following
>>>> in time-desc order.
>>>>
>>>> UserID = XYZ
>>>>               Col-Name5 = 204:1002
>>>>               Col-Name4 = 203:1000
>>>>               Col-Name2 = 201:1001
>>>>
>>>> Is something like this possible in Cassandra? Is there a different way
>>>> to design and achieve the same objective?
>>>>
>>>> --
>>>> Ravi
>>>>
>>>>
>>>
>>>
>>
>

Re: Composite Column Grouping

Posted by Ravikumar Govindarajan <ra...@gmail.com>.
Thanks Michael,

But I cannot sort the rows in memory, as the number of columns will be
quite huge.

>From the python script above:
   select_stmt = "select * from time_series where userid = 'XYZ'"

This would return me many hundreds of thousands of columns. I need to go in
time-series order using ranges [Pagination queries].


On Wed, Sep 11, 2013 at 7:06 AM, Laing, Michael
<mi...@nytimes.com>wrote:

> If you have set up the table as described in my previous message, you
> could run this python snippet to return the desired result:
>
> #!/usr/bin/env python
> # -*- coding: utf-8 -*-
> import logging
> logging.basicConfig()
>
> from operator import itemgetter
>
> import cassandra
> from cassandra.cluster import Cluster
> from cassandra.query import SimpleStatement
>
> cql_cluster = Cluster()
> cql_session = cql_cluster.connect()
> cql_session.set_keyspace('latest')
>
> select_stmt = "select * from time_series where userid = 'XYZ'"
> query = SimpleStatement(select_stmt)
> rows = cql_session.execute(query)
>
> results = []
> for row in rows:
>     max_time = max(row.colname.keys())
>     results.append((row.userid, row.pkid, max_time, row.colname[max_time]))
>
> sorted_results = sorted(results, key=itemgetter(2), reverse=True)
> for result in sorted_results: print result
>
> # prints:
>
> # (u'XYZ', u'1002', u'204', u'Col-Name-5')
> # (u'XYZ', u'1000', u'203', u'Col-Name-4')
> # (u'XYZ', u'1001', u'201', u'Col-Name-2')
>
>
>
> On Tue, Sep 10, 2013 at 6:32 PM, Laing, Michael <michael.laing@nytimes.com
> > wrote:
>
>> You could try this. C* doesn't do it all for you, but it will efficiently
>> get you the right data.
>>
>> -ml
>>
>> -- put this in <file> and run using 'cqlsh -f <file>
>>
>> DROP KEYSPACE latest;
>>
>> CREATE KEYSPACE latest WITH replication = {
>>     'class': 'SimpleStrategy',
>>     'replication_factor' : 1
>> };
>>
>> USE latest;
>>
>> CREATE TABLE time_series (
>>     userid text,
>>     pkid text,
>>     colname map<text, text>,
>>     PRIMARY KEY (userid, pkid)
>> );
>>
>> UPDATE time_series SET colname = colname + {'200':'Col-Name-1'} WHERE
>> userid = 'XYZ' AND pkid = '1000';
>> UPDATE time_series SET colname = colname +
>> {'201':'Col-Name-2'} WHERE userid = 'XYZ' AND pkid = '1001';
>> UPDATE time_series SET colname = colname +
>> {'202':'Col-Name-3'} WHERE userid = 'XYZ' AND pkid = '1000';
>> UPDATE time_series SET colname = colname +
>> {'203':'Col-Name-4'} WHERE userid = 'XYZ' AND pkid = '1000';
>> UPDATE time_series SET colname = colname +
>> {'204':'Col-Name-5'} WHERE userid = 'XYZ' AND pkid = '1002';
>>
>> SELECT * FROM time_series WHERE userid = 'XYZ';
>>
>> -- returns:
>> -- userid | pkid | colname
>>
>> ----------+------+-----------------------------------------------------------------
>> --    XYZ | 1000 | {'200': 'Col-Name-1', '202': 'Col-Name-3', '203':
>> 'Col-Name-4'}
>> --    XYZ | 1001 |                                           {'201':
>> 'Col-Name-2'}
>> --    XYZ | 1002 |                                           {'204':
>> 'Col-Name-5'}
>>
>> -- use an app to pop off the latest key/value from the map for each row,
>> then sort by key desc.
>>
>>
>> On Tue, Sep 10, 2013 at 9:21 AM, Ravikumar Govindarajan <
>> ravikumar.govindarajan@gmail.com> wrote:
>>
>>> I have been faced with a problem of grouping composites on the
>>> second-part.
>>>
>>> Lets say my CF contains this
>>>
>>>
>>> TimeSeriesCF
>>>                        key:                            UserID
>>>                        composite-col-name:    TimeUUID:PKID
>>>
>>> Some sample data
>>>
>>> UserID = XYZ
>>>                                  Time:PKID
>>>                Col-Name1 = 200:1000
>>>                Col-Name2 = 201:1001
>>>                Col-Name3 = 202:1000
>>>                Col-Name4 = 203:1000
>>>                Col-Name5 = 204:1002
>>>
>>> Whenever a time-series query is issued, it should return the following
>>> in time-desc order.
>>>
>>> UserID = XYZ
>>>               Col-Name5 = 204:1002
>>>               Col-Name4 = 203:1000
>>>               Col-Name2 = 201:1001
>>>
>>> Is something like this possible in Cassandra? Is there a different way
>>> to design and achieve the same objective?
>>>
>>> --
>>> Ravi
>>>
>>>
>>
>>
>

Re: Composite Column Grouping

Posted by "Laing, Michael" <mi...@nytimes.com>.
If you have set up the table as described in my previous message, you could
run this python snippet to return the desired result:

#!/usr/bin/env python
# -*- coding: utf-8 -*-
import logging
logging.basicConfig()

from operator import itemgetter

import cassandra
from cassandra.cluster import Cluster
from cassandra.query import SimpleStatement

cql_cluster = Cluster()
cql_session = cql_cluster.connect()
cql_session.set_keyspace('latest')

select_stmt = "select * from time_series where userid = 'XYZ'"
query = SimpleStatement(select_stmt)
rows = cql_session.execute(query)

results = []
for row in rows:
    max_time = max(row.colname.keys())
    results.append((row.userid, row.pkid, max_time, row.colname[max_time]))

sorted_results = sorted(results, key=itemgetter(2), reverse=True)
for result in sorted_results: print result

# prints:

# (u'XYZ', u'1002', u'204', u'Col-Name-5')
# (u'XYZ', u'1000', u'203', u'Col-Name-4')
# (u'XYZ', u'1001', u'201', u'Col-Name-2')



On Tue, Sep 10, 2013 at 6:32 PM, Laing, Michael
<mi...@nytimes.com>wrote:

> You could try this. C* doesn't do it all for you, but it will efficiently
> get you the right data.
>
> -ml
>
> -- put this in <file> and run using 'cqlsh -f <file>
>
> DROP KEYSPACE latest;
>
> CREATE KEYSPACE latest WITH replication = {
>     'class': 'SimpleStrategy',
>     'replication_factor' : 1
> };
>
> USE latest;
>
> CREATE TABLE time_series (
>     userid text,
>     pkid text,
>     colname map<text, text>,
>     PRIMARY KEY (userid, pkid)
> );
>
> UPDATE time_series SET colname = colname + {'200':'Col-Name-1'} WHERE
> userid = 'XYZ' AND pkid = '1000';
> UPDATE time_series SET colname = colname +
> {'201':'Col-Name-2'} WHERE userid = 'XYZ' AND pkid = '1001';
> UPDATE time_series SET colname = colname +
> {'202':'Col-Name-3'} WHERE userid = 'XYZ' AND pkid = '1000';
> UPDATE time_series SET colname = colname +
> {'203':'Col-Name-4'} WHERE userid = 'XYZ' AND pkid = '1000';
> UPDATE time_series SET colname = colname +
> {'204':'Col-Name-5'} WHERE userid = 'XYZ' AND pkid = '1002';
>
> SELECT * FROM time_series WHERE userid = 'XYZ';
>
> -- returns:
> -- userid | pkid | colname
>
> ----------+------+-----------------------------------------------------------------
> --    XYZ | 1000 | {'200': 'Col-Name-1', '202': 'Col-Name-3', '203':
> 'Col-Name-4'}
> --    XYZ | 1001 |                                           {'201':
> 'Col-Name-2'}
> --    XYZ | 1002 |                                           {'204':
> 'Col-Name-5'}
>
> -- use an app to pop off the latest key/value from the map for each row,
> then sort by key desc.
>
>
> On Tue, Sep 10, 2013 at 9:21 AM, Ravikumar Govindarajan <
> ravikumar.govindarajan@gmail.com> wrote:
>
>> I have been faced with a problem of grouping composites on the
>> second-part.
>>
>> Lets say my CF contains this
>>
>>
>> TimeSeriesCF
>>                        key:                            UserID
>>                        composite-col-name:    TimeUUID:PKID
>>
>> Some sample data
>>
>> UserID = XYZ
>>                                  Time:PKID
>>                Col-Name1 = 200:1000
>>                Col-Name2 = 201:1001
>>                Col-Name3 = 202:1000
>>                Col-Name4 = 203:1000
>>                Col-Name5 = 204:1002
>>
>> Whenever a time-series query is issued, it should return the following in
>> time-desc order.
>>
>> UserID = XYZ
>>               Col-Name5 = 204:1002
>>               Col-Name4 = 203:1000
>>               Col-Name2 = 201:1001
>>
>> Is something like this possible in Cassandra? Is there a different way to
>> design and achieve the same objective?
>>
>> --
>> Ravi
>>
>>
>
>

Re: Composite Column Grouping

Posted by "Laing, Michael" <mi...@nytimes.com>.
You could try this. C* doesn't do it all for you, but it will efficiently
get you the right data.

-ml

-- put this in <file> and run using 'cqlsh -f <file>

DROP KEYSPACE latest;

CREATE KEYSPACE latest WITH replication = {
    'class': 'SimpleStrategy',
    'replication_factor' : 1
};

USE latest;

CREATE TABLE time_series (
    userid text,
    pkid text,
    colname map<text, text>,
    PRIMARY KEY (userid, pkid)
);

UPDATE time_series SET colname = colname + {'200':'Col-Name-1'} WHERE
userid = 'XYZ' AND pkid = '1000';
UPDATE time_series SET colname = colname +
{'201':'Col-Name-2'} WHERE userid = 'XYZ' AND pkid = '1001';
UPDATE time_series SET colname = colname +
{'202':'Col-Name-3'} WHERE userid = 'XYZ' AND pkid = '1000';
UPDATE time_series SET colname = colname +
{'203':'Col-Name-4'} WHERE userid = 'XYZ' AND pkid = '1000';
UPDATE time_series SET colname = colname +
{'204':'Col-Name-5'} WHERE userid = 'XYZ' AND pkid = '1002';

SELECT * FROM time_series WHERE userid = 'XYZ';

-- returns:
-- userid | pkid | colname
----------+------+-----------------------------------------------------------------
--    XYZ | 1000 | {'200': 'Col-Name-1', '202': 'Col-Name-3', '203':
'Col-Name-4'}
--    XYZ | 1001 |                                           {'201':
'Col-Name-2'}
--    XYZ | 1002 |                                           {'204':
'Col-Name-5'}

-- use an app to pop off the latest key/value from the map for each row,
then sort by key desc.


On Tue, Sep 10, 2013 at 9:21 AM, Ravikumar Govindarajan <
ravikumar.govindarajan@gmail.com> wrote:

> I have been faced with a problem of grouping composites on the second-part.
>
> Lets say my CF contains this
>
>
> TimeSeriesCF
>                        key:                            UserID
>                        composite-col-name:    TimeUUID:PKID
>
> Some sample data
>
> UserID = XYZ
>                                  Time:PKID
>                Col-Name1 = 200:1000
>                Col-Name2 = 201:1001
>                Col-Name3 = 202:1000
>                Col-Name4 = 203:1000
>                Col-Name5 = 204:1002
>
> Whenever a time-series query is issued, it should return the following in
> time-desc order.
>
> UserID = XYZ
>               Col-Name5 = 204:1002
>               Col-Name4 = 203:1000
>               Col-Name2 = 201:1001
>
> Is something like this possible in Cassandra? Is there a different way to
> design and achieve the same objective?
>
> --
> Ravi
>
>