You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Sylvain Lebresne (JIRA)" <ji...@apache.org> on 2013/11/28 14:03:37 UTC

[jira] [Commented] (CASSANDRA-6412) Custom creation and merge functions for user-defined column types

    [ https://issues.apache.org/jira/browse/CASSANDRA-6412?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13834794#comment-13834794 ] 

Sylvain Lebresne commented on CASSANDRA-6412:
---------------------------------------------

This is interesting, thanks.

The one generic problem that comes to mind is deletes. Deletes don't commute with other operations and so can't be supported (not in a useful way at least).  This is problem we have with counters too, but that doesn't make it less of a problem imho (in particular, that's one of the reason why counters are segregated to their own tables; does that mean we'd need to do do the same here?).

Anyway, while I think this delete problem is a major one, in the spirit of still discussing the patch itself, I'd like to suggest a slightly different design for this that the one of the patch, that I think would require quite a bit less changes.

There is 2 parts to this approach:
# the ability to have custom, value-based resolution between 2 cells (min and max only care about that).
# the cramming of multiple values into one (candleStick does that on top of 1)).

Trying to support both at once imo complicate things a bit. In practice, we already know how to do 2) and we should reuse that. Namely, assuming we do have 1), candlestick could be implemented client side with almost no additional support from Cassandra. Namely one could either:
* use a good old fashioned map.
* probably even nicer in this case, use a user type (CASSANDRA-5590). Which for a candlestick could be
{noformat}
CREATE TYPE candlestick (open float, close float, low float, high float)
{noformat}

Of course, if you do that, adding a new value to the candlestick looks slightly less nice but we can add support for custom
functions (which is useful in its own right anyway and is relatively simple if we're just talking of functions transforming values),
so one could imagine to be able to write (reusing the example from the patch):
{noformat}
INSERT INTO stocks (symbol,month,day,value) VALUES ('GOOG', '2013-10', 01, candleStick_value(887.00))
{noformat}

So really, we just need the first part, the ability to have custom reslution between cells. And to do that, I'd rather not add a new type of cell (i.e. no CrdtColumn).  Typically, we could pull out cell resolution from the Column class and move it to ColumnFamily. At which point we have the CFMedata ojbect and can just do something along the lines of:
{noformat}
Column reconciled = metadata.getResolver(column.name()).reconcile(column, oldColumn);
{noformat}
and then all we have to do is to be able to register cell resolvers that would just have this reconcile method, no CrdtColumn or CrdtColumnType, no specific new types...

For the CQL syntax, I could suggest something along the lines of:
{noformat}
CREATE TABLE myTable (
    key int PRIMARY KEY,
    max float WITH RESOLVER 'my.company.MaxResolver',
    min float WITH RESOLVER 'my.company.MinResolver',
)
{noformat}
which, applied to the candlestick example could be:
{noformat}
CREATE TYPE candlestick (
    open float WITH RESOLVER 'my.company.KeepFirstResolver',
    close float,
    low float WITH RESOLVER 'my.company.MinResolver',
    high float WITH RESOLVER 'my.company.MaxResolver'
)

CREATE TABLE stocks (
    symbol text,
    month text,
    day int,
    value candelstick,
    PRIMARY KEY ((symbol,month))
)
{noformat}
And of course, we can decide if we want to provide some basic resolvers like min/max out of the box.

All that said, I do think the delete problem is somewhat of a blocker (I haven't come to peace with it at least).


> Custom creation and merge functions for user-defined column types
> -----------------------------------------------------------------
>
>                 Key: CASSANDRA-6412
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-6412
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Nicolas Favre-Felix
>
> This is a proposal for a new feature, mapping custom types to Cassandra columns.
> These types would provide a creation function and a merge function, to be implemented in Java by the user.
> This feature relates to the concept of CRDTs; the proposal is to replicate "operations" on these types during write, to apply these operations internally during merge (Column.reconcile), and to also merge their values on read.
> The following operations are made possible without reading back any data:
> * MIN or MAX(value) for a column
> * First value for a column
> * Count Distinct
> * HyperLogLog
> * Count-Min
> And any composition of these too, e.g. a Candlestick type includes first, last, min, and max.
> The merge operations exposed by these types need to be commutative; this is the case for many functions used in analytics.
> This feature is incomplete without some integration with CASSANDRA-4775 (Counters 2.0) which provides a Read-Modify-Write implementation for distributed counters. Integrating custom creation and merge functions with new counters would let users implement complex CRDTs in Cassandra, including:
> * Averages & related (sum of squares, standard deviation)
> * Graphs
> * Sets
> * Custom registers (even with vector clocks)
> I have a working prototype with implementations for min, max, and Candlestick at https://github.com/acunu/cassandra/tree/crdts - I'd appreciate any feedback on the design and interfaces.



--
This message was sent by Atlassian JIRA
(v6.1#6144)