You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Maxim Grinev (JIRA)" <ji...@apache.org> on 2010/07/23 16:32:50 UTC

[jira] Created: (CASSANDRA-1311) Support (asynchronous) triggers

Support (asynchronous) triggers
-------------------------------

                 Key: CASSANDRA-1311
                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1311
             Project: Cassandra
          Issue Type: New Feature
          Components: Contrib
            Reporter: Maxim Grinev


Asynchronous trigger is a basic mechanism to implement various use cases of asynchronous execution of application code at database side. For example to support indexes and materialized views, online analytics, push-based data propagation.

Please find the motivation, triggers description and list of applications:
http://maxgrinev.com/2010/07/23/extending-cassandra-with-asynchronous-triggers/

An example of using triggers for indexing:
http://maxgrinev.com/2010/07/23/managing-indexes-in-cassandra-using-async-triggers/

Implementation details are attached.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (CASSANDRA-1311) Support (asynchronous) triggers

Posted by "Maxim Grinev (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-1311?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12898935#action_12898935 ] 

Maxim Grinev commented on CASSANDRA-1311:
-----------------------------------------

> An interesting benefit provided by CASSANDRA-1016 is that it has a pre-execution step, allowing the Plugin/Trigger to perform a read if it needs to (for example, to update a compound index which the mutation does not contain all columns for).

Nothing prevents you from querying Cassandra from within a trigger. It is not atomic as you mentioned. Having all information ready in the mutation will be atomic (and faster because you avoid the query). But once again you can still query in the same way as you update the data from within a trigger.

> Support (asynchronous) triggers
> -------------------------------
>
>                 Key: CASSANDRA-1311
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1311
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Contrib
>            Reporter: Maxim Grinev
>             Fix For: 0.8
>
>         Attachments: HOWTO-PatchAndRunTriggerExample-update1.txt, HOWTO-PatchAndRunTriggerExample.txt, ImplementationDetails-update1.pdf, ImplementationDetails.pdf, trunk-967053.txt, trunk-984391-update1.txt
>
>
> Asynchronous triggers is a basic mechanism to implement various use cases of asynchronous execution of application code at database side. For example to support indexes and materialized views, online analytics, push-based data propagation.
> Please find the motivation, triggers description and list of applications:
> http://maxgrinev.com/2010/07/23/extending-cassandra-with-asynchronous-triggers/
> An example of using triggers for indexing:
> http://maxgrinev.com/2010/07/23/managing-indexes-in-cassandra-using-async-triggers/
> Implementation details are attached.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (CASSANDRA-1311) Support (asynchronous) triggers

Posted by "David Erickson (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-1311?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

David Erickson updated CASSANDRA-1311:
--------------------------------------

    Comment: was deleted

(was: A quick glance at the implementation looks like its doing Slave hashing based only on its IP address, would this cause problems with hosts running multiple Cassandra instances on a single IP address?)

> Support (asynchronous) triggers
> -------------------------------
>
>                 Key: CASSANDRA-1311
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1311
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Contrib
>            Reporter: Maxim Grinev
>             Fix For: 0.8
>
>         Attachments: HOWTO-PatchAndRunTriggerExample-update1.txt, HOWTO-PatchAndRunTriggerExample.txt, ImplementationDetails-update1.pdf, ImplementationDetails.pdf, trunk-967053.txt, trunk-984391-update1.txt
>
>
> Asynchronous triggers is a basic mechanism to implement various use cases of asynchronous execution of application code at database side. For example to support indexes and materialized views, online analytics, push-based data propagation.
> Please find the motivation, triggers description and list of applications:
> http://maxgrinev.com/2010/07/23/extending-cassandra-with-asynchronous-triggers/
> An example of using triggers for indexing:
> http://maxgrinev.com/2010/07/23/managing-indexes-in-cassandra-using-async-triggers/
> Implementation details are attached.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (CASSANDRA-1311) Support (asynchronous) triggers

Posted by "Maxim Grinev (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-1311?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Maxim Grinev updated CASSANDRA-1311:
------------------------------------

    Comment: was deleted

(was: patch for SVN revision 967053 (July 23))

> Support (asynchronous) triggers
> -------------------------------
>
>                 Key: CASSANDRA-1311
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1311
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Contrib
>            Reporter: Maxim Grinev
>
> Asynchronous trigger is a basic mechanism to implement various use cases of asynchronous execution of application code at database side. For example to support indexes and materialized views, online analytics, push-based data propagation.
> Please find the motivation, triggers description and list of applications:
> http://maxgrinev.com/2010/07/23/extending-cassandra-with-asynchronous-triggers/
> An example of using triggers for indexing:
> http://maxgrinev.com/2010/07/23/managing-indexes-in-cassandra-using-async-triggers/
> Implementation details are attached.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (CASSANDRA-1311) Support (asynchronous) triggers

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-1311?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12898744#action_12898744 ] 

Jonathan Ellis commented on CASSANDRA-1311:
-------------------------------------------

You already can't have multiple Cassandra instances on a single IP.

> Support (asynchronous) triggers
> -------------------------------
>
>                 Key: CASSANDRA-1311
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1311
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Contrib
>            Reporter: Maxim Grinev
>             Fix For: 0.8
>
>         Attachments: HOWTO-PatchAndRunTriggerExample-update1.txt, HOWTO-PatchAndRunTriggerExample.txt, ImplementationDetails-update1.pdf, ImplementationDetails.pdf, trunk-967053.txt, trunk-984391-update1.txt
>
>
> Asynchronous triggers is a basic mechanism to implement various use cases of asynchronous execution of application code at database side. For example to support indexes and materialized views, online analytics, push-based data propagation.
> Please find the motivation, triggers description and list of applications:
> http://maxgrinev.com/2010/07/23/extending-cassandra-with-asynchronous-triggers/
> An example of using triggers for indexing:
> http://maxgrinev.com/2010/07/23/managing-indexes-in-cassandra-using-async-triggers/
> Implementation details are attached.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Issue Comment Edited: (CASSANDRA-1311) Support (asynchronous) triggers

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-1311?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12898744#action_12898744 ] 

Jonathan Ellis edited comment on CASSANDRA-1311 at 8/15/10 8:17 PM:
--------------------------------------------------------------------

bq. A quick glance at the implementation looks like its doing Slave hashing based only on its IP address, would this cause problems with hosts running multiple Cassandra instances on a single IP address?

You already can't have multiple Cassandra instances on a single IP.

[the original comment I replied to was deleted, so to avoid losing context, I've edited mine to include it.]

      was (Author: jbellis):
    You already can't have multiple Cassandra instances on a single IP.
  
> Support (asynchronous) triggers
> -------------------------------
>
>                 Key: CASSANDRA-1311
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1311
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Contrib
>            Reporter: Maxim Grinev
>             Fix For: 0.8
>
>         Attachments: HOWTO-PatchAndRunTriggerExample-update1.txt, HOWTO-PatchAndRunTriggerExample.txt, ImplementationDetails-update1.pdf, ImplementationDetails.pdf, trunk-967053.txt, trunk-984391-update1.txt
>
>
> Asynchronous triggers is a basic mechanism to implement various use cases of asynchronous execution of application code at database side. For example to support indexes and materialized views, online analytics, push-based data propagation.
> Please find the motivation, triggers description and list of applications:
> http://maxgrinev.com/2010/07/23/extending-cassandra-with-asynchronous-triggers/
> An example of using triggers for indexing:
> http://maxgrinev.com/2010/07/23/managing-indexes-in-cassandra-using-async-triggers/
> Implementation details are attached.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (CASSANDRA-1311) Support (asynchronous) triggers

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-1311?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12898378#action_12898378 ] 

Stu Hood commented on CASSANDRA-1311:
-------------------------------------

An interesting benefit provided by CASSANDRA-1016 is that it has a pre-execution step, allowing the Plugin/Trigger to perform a read if it needs to (for example, to update a compound index which the mutation does not contain all columns for).

Since making the read-then-write step atomic is out of the question, the benefit is questionable (something I think we missed when we were considering 1016). Requiring the user to use mutations containing all fields the trigger might need may be reasonable. Example: for updating the zipcode in an index of "country-zipcode", it would be up to the user to include both fields in the mutation, meaning that they might have to perform a read from their base data first.

> Support (asynchronous) triggers
> -------------------------------
>
>                 Key: CASSANDRA-1311
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1311
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Contrib
>            Reporter: Maxim Grinev
>             Fix For: 0.8
>
>         Attachments: HOWTO-PatchAndRunTriggerExample-update1.txt, HOWTO-PatchAndRunTriggerExample.txt, ImplementationDetails-update1.pdf, ImplementationDetails.pdf, trunk-967053.txt, trunk-984391-update1.txt
>
>
> Asynchronous triggers is a basic mechanism to implement various use cases of asynchronous execution of application code at database side. For example to support indexes and materialized views, online analytics, push-based data propagation.
> Please find the motivation, triggers description and list of applications:
> http://maxgrinev.com/2010/07/23/extending-cassandra-with-asynchronous-triggers/
> An example of using triggers for indexing:
> http://maxgrinev.com/2010/07/23/managing-indexes-in-cassandra-using-async-triggers/
> Implementation details are attached.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (CASSANDRA-1311) Support (asynchronous) triggers

Posted by "Martin Hentschel (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-1311?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Martin Hentschel updated CASSANDRA-1311:
----------------------------------------

    Attachment: trunk-984391-update2.txt

I fixed the points Stu mentioned:

*  untangled findTriggers
*  using MD5 instead of Arrays.hashCode
*  storing dangling triggers in the System Table instead of Java collections

I want to note that these fixes have introduce some performance penalties.  In our benchmarks we observed a performance loss of 22%.  According to Maxim who has discussed this point with Stu at the summit, this is still fine.  The System Table provides additional durability guarantees instead.

> Support (asynchronous) triggers
> -------------------------------
>
>                 Key: CASSANDRA-1311
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1311
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Contrib
>            Reporter: Maxim Grinev
>             Fix For: 0.8
>
>         Attachments: HOWTO-PatchAndRunTriggerExample-update1.txt, HOWTO-PatchAndRunTriggerExample.txt, ImplementationDetails-update1.pdf, ImplementationDetails.pdf, trunk-967053.txt, trunk-984391-update1.txt, trunk-984391-update2.txt
>
>
> Asynchronous triggers is a basic mechanism to implement various use cases of asynchronous execution of application code at database side. For example to support indexes and materialized views, online analytics, push-based data propagation.
> Please find the motivation, triggers description and list of applications:
> http://maxgrinev.com/2010/07/23/extending-cassandra-with-asynchronous-triggers/
> An example of using triggers for indexing:
> http://maxgrinev.com/2010/07/23/managing-indexes-in-cassandra-using-async-triggers/
> Implementation details are attached.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (CASSANDRA-1311) Support (asynchronous) triggers

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-1311?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Jonathan Ellis updated CASSANDRA-1311:
--------------------------------------

    Fix Version/s: 0.8

> Support (asynchronous) triggers
> -------------------------------
>
>                 Key: CASSANDRA-1311
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1311
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Contrib
>            Reporter: Maxim Grinev
>             Fix For: 0.8
>
>         Attachments: ImplementationDetails.pdf, trunk-967053.txt
>
>
> Asynchronous triggers is a basic mechanism to implement various use cases of asynchronous execution of application code at database side. For example to support indexes and materialized views, online analytics, push-based data propagation.
> Please find the motivation, triggers description and list of applications:
> http://maxgrinev.com/2010/07/23/extending-cassandra-with-asynchronous-triggers/
> An example of using triggers for indexing:
> http://maxgrinev.com/2010/07/23/managing-indexes-in-cassandra-using-async-triggers/
> Implementation details are attached.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (CASSANDRA-1311) Support (asynchronous) triggers

Posted by "Martin Hentschel (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-1311?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Martin Hentschel updated CASSANDRA-1311:
----------------------------------------

    Attachment:     (was: HOWTO-PatchAndRunTriggerExample.txt)

> Support (asynchronous) triggers
> -------------------------------
>
>                 Key: CASSANDRA-1311
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1311
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Contrib
>            Reporter: Maxim Grinev
>             Fix For: 0.8
>
>         Attachments: ImplementationDetails.pdf, trunk-967053.txt
>
>
> Asynchronous triggers is a basic mechanism to implement various use cases of asynchronous execution of application code at database side. For example to support indexes and materialized views, online analytics, push-based data propagation.
> Please find the motivation, triggers description and list of applications:
> http://maxgrinev.com/2010/07/23/extending-cassandra-with-asynchronous-triggers/
> An example of using triggers for indexing:
> http://maxgrinev.com/2010/07/23/managing-indexes-in-cassandra-using-async-triggers/
> Implementation details are attached.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (CASSANDRA-1311) Support (asynchronous) triggers

Posted by "Martin Hentschel (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-1311?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Martin Hentschel updated CASSANDRA-1311:
----------------------------------------

    Attachment: trunk-984391-update1.txt
                ImplementationDetails-update1.pdf
                HOWTO-PatchAndRunTriggerExample-update1.txt

Uploaded a new patch, improvements to the last release:

* Updated to Cassandra revision 984391
* Reduced network traffic of notification messages (by bundling of notifications and sending only hash values)
* Faster storage of "dangling" triggers at a slave node (using Java collections instead of Cassandra system tables because we don't need additional durability guarantees)

> Support (asynchronous) triggers
> -------------------------------
>
>                 Key: CASSANDRA-1311
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1311
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Contrib
>            Reporter: Maxim Grinev
>             Fix For: 0.8
>
>         Attachments: HOWTO-PatchAndRunTriggerExample-update1.txt, HOWTO-PatchAndRunTriggerExample.txt, ImplementationDetails-update1.pdf, ImplementationDetails.pdf, trunk-967053.txt, trunk-984391-update1.txt
>
>
> Asynchronous triggers is a basic mechanism to implement various use cases of asynchronous execution of application code at database side. For example to support indexes and materialized views, online analytics, push-based data propagation.
> Please find the motivation, triggers description and list of applications:
> http://maxgrinev.com/2010/07/23/extending-cassandra-with-asynchronous-triggers/
> An example of using triggers for indexing:
> http://maxgrinev.com/2010/07/23/managing-indexes-in-cassandra-using-async-triggers/
> Implementation details are attached.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (CASSANDRA-1311) Support (asynchronous) triggers

Posted by "Maxim Grinev (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-1311?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Maxim Grinev updated CASSANDRA-1311:
------------------------------------

    Description: 
Asynchronous triggers is a basic mechanism to implement various use cases of asynchronous execution of application code at database side. For example to support indexes and materialized views, online analytics, push-based data propagation.

Please find the motivation, triggers description and list of applications:
http://maxgrinev.com/2010/07/23/extending-cassandra-with-asynchronous-triggers/

An example of using triggers for indexing:
http://maxgrinev.com/2010/07/23/managing-indexes-in-cassandra-using-async-triggers/

Implementation details are attached.

  was:
Asynchronous trigger is a basic mechanism to implement various use cases of asynchronous execution of application code at database side. For example to support indexes and materialized views, online analytics, push-based data propagation.

Please find the motivation, triggers description and list of applications:
http://maxgrinev.com/2010/07/23/extending-cassandra-with-asynchronous-triggers/

An example of using triggers for indexing:
http://maxgrinev.com/2010/07/23/managing-indexes-in-cassandra-using-async-triggers/

Implementation details are attached.


> Support (asynchronous) triggers
> -------------------------------
>
>                 Key: CASSANDRA-1311
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1311
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Contrib
>            Reporter: Maxim Grinev
>         Attachments: ImplementationDetails.pdf, trunk-967053.txt
>
>
> Asynchronous triggers is a basic mechanism to implement various use cases of asynchronous execution of application code at database side. For example to support indexes and materialized views, online analytics, push-based data propagation.
> Please find the motivation, triggers description and list of applications:
> http://maxgrinev.com/2010/07/23/extending-cassandra-with-asynchronous-triggers/
> An example of using triggers for indexing:
> http://maxgrinev.com/2010/07/23/managing-indexes-in-cassandra-using-async-triggers/
> Implementation details are attached.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (CASSANDRA-1311) Support (asynchronous) triggers

Posted by "Maxim Grinev (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-1311?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Maxim Grinev updated CASSANDRA-1311:
------------------------------------

    Attachment: ImplementationDetails.pdf

> Support (asynchronous) triggers
> -------------------------------
>
>                 Key: CASSANDRA-1311
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1311
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Contrib
>            Reporter: Maxim Grinev
>         Attachments: ImplementationDetails.pdf, trunk-967053.txt
>
>
> Asynchronous trigger is a basic mechanism to implement various use cases of asynchronous execution of application code at database side. For example to support indexes and materialized views, online analytics, push-based data propagation.
> Please find the motivation, triggers description and list of applications:
> http://maxgrinev.com/2010/07/23/extending-cassandra-with-asynchronous-triggers/
> An example of using triggers for indexing:
> http://maxgrinev.com/2010/07/23/managing-indexes-in-cassandra-using-async-triggers/
> Implementation details are attached.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (CASSANDRA-1311) Support (asynchronous) triggers

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-1311?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12891636#action_12891636 ] 

Jonathan Ellis commented on CASSANDRA-1311:
-------------------------------------------

also, the specific application of indexing is better dealt with by CASSANDRA-749 -- pushing it down into "native" logic means we can updating multiple indexes from a single commitlog entry, ensuring the indexes are consistent with the "master" row.

> Support (asynchronous) triggers
> -------------------------------
>
>                 Key: CASSANDRA-1311
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1311
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Contrib
>            Reporter: Maxim Grinev
>         Attachments: ImplementationDetails.pdf, trunk-967053.txt
>
>
> Asynchronous triggers is a basic mechanism to implement various use cases of asynchronous execution of application code at database side. For example to support indexes and materialized views, online analytics, push-based data propagation.
> Please find the motivation, triggers description and list of applications:
> http://maxgrinev.com/2010/07/23/extending-cassandra-with-asynchronous-triggers/
> An example of using triggers for indexing:
> http://maxgrinev.com/2010/07/23/managing-indexes-in-cassandra-using-async-triggers/
> Implementation details are attached.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (CASSANDRA-1311) Support (asynchronous) triggers

Posted by "Maxim Grinev (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-1311?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12898932#action_12898932 ] 

Maxim Grinev commented on CASSANDRA-1311:
-----------------------------------------

>> Faster storage of "dangling" triggers at a slave node

> I mentioned to Maxim at the summit that I think persisting the dangling triggers to the system tables is a good idea...

Thanks for your comments. We agree with all the points and will send a new patch soon.


> Support (asynchronous) triggers
> -------------------------------
>
>                 Key: CASSANDRA-1311
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1311
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Contrib
>            Reporter: Maxim Grinev
>             Fix For: 0.8
>
>         Attachments: HOWTO-PatchAndRunTriggerExample-update1.txt, HOWTO-PatchAndRunTriggerExample.txt, ImplementationDetails-update1.pdf, ImplementationDetails.pdf, trunk-967053.txt, trunk-984391-update1.txt
>
>
> Asynchronous triggers is a basic mechanism to implement various use cases of asynchronous execution of application code at database side. For example to support indexes and materialized views, online analytics, push-based data propagation.
> Please find the motivation, triggers description and list of applications:
> http://maxgrinev.com/2010/07/23/extending-cassandra-with-asynchronous-triggers/
> An example of using triggers for indexing:
> http://maxgrinev.com/2010/07/23/managing-indexes-in-cassandra-using-async-triggers/
> Implementation details are attached.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (CASSANDRA-1311) Support (asynchronous) triggers

Posted by "Martin Hentschel (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-1311?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Martin Hentschel updated CASSANDRA-1311:
----------------------------------------

    Attachment: HOWTO-PatchAndRunTriggerExample.txt

> Support (asynchronous) triggers
> -------------------------------
>
>                 Key: CASSANDRA-1311
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1311
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Contrib
>            Reporter: Maxim Grinev
>             Fix For: 0.8
>
>         Attachments: HOWTO-PatchAndRunTriggerExample.txt, ImplementationDetails.pdf, trunk-967053.txt
>
>
> Asynchronous triggers is a basic mechanism to implement various use cases of asynchronous execution of application code at database side. For example to support indexes and materialized views, online analytics, push-based data propagation.
> Please find the motivation, triggers description and list of applications:
> http://maxgrinev.com/2010/07/23/extending-cassandra-with-asynchronous-triggers/
> An example of using triggers for indexing:
> http://maxgrinev.com/2010/07/23/managing-indexes-in-cassandra-using-async-triggers/
> Implementation details are attached.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Issue Comment Edited: (CASSANDRA-1311) Support (asynchronous) triggers

Posted by "Martin Hentschel (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-1311?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12897281#action_12897281 ] 

Martin Hentschel edited comment on CASSANDRA-1311 at 8/11/10 10:21 AM:
-----------------------------------------------------------------------

Uploaded a new patch, improvements to the last release:

* Updated to Cassandra revision 984391
* Reduced network traffic of notification messages (by bundling notifications and sending only hash values)
* Faster storage of "dangling" triggers at a slave node (using Java collections instead of Cassandra system tables because we don't need additional durability guarantees)

      was (Author: hemartin):
    Uploaded a new patch, improvements to the last release:

* Updated to Cassandra revision 984391
* Reduced network traffic of notification messages (by bundling of notifications and sending only hash values)
* Faster storage of "dangling" triggers at a slave node (using Java collections instead of Cassandra system tables because we don't need additional durability guarantees)
  
> Support (asynchronous) triggers
> -------------------------------
>
>                 Key: CASSANDRA-1311
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1311
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Contrib
>            Reporter: Maxim Grinev
>             Fix For: 0.8
>
>         Attachments: HOWTO-PatchAndRunTriggerExample-update1.txt, HOWTO-PatchAndRunTriggerExample.txt, ImplementationDetails-update1.pdf, ImplementationDetails.pdf, trunk-967053.txt, trunk-984391-update1.txt
>
>
> Asynchronous triggers is a basic mechanism to implement various use cases of asynchronous execution of application code at database side. For example to support indexes and materialized views, online analytics, push-based data propagation.
> Please find the motivation, triggers description and list of applications:
> http://maxgrinev.com/2010/07/23/extending-cassandra-with-asynchronous-triggers/
> An example of using triggers for indexing:
> http://maxgrinev.com/2010/07/23/managing-indexes-in-cassandra-using-async-triggers/
> Implementation details are attached.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (CASSANDRA-1311) Support (asynchronous) triggers

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-1311?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12891630#action_12891630 ] 

Jonathan Ellis commented on CASSANDRA-1311:
-------------------------------------------

Is this different from CASSANDRA-1016?

> Support (asynchronous) triggers
> -------------------------------
>
>                 Key: CASSANDRA-1311
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1311
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Contrib
>            Reporter: Maxim Grinev
>         Attachments: ImplementationDetails.pdf, trunk-967053.txt
>
>
> Asynchronous triggers is a basic mechanism to implement various use cases of asynchronous execution of application code at database side. For example to support indexes and materialized views, online analytics, push-based data propagation.
> Please find the motivation, triggers description and list of applications:
> http://maxgrinev.com/2010/07/23/extending-cassandra-with-asynchronous-triggers/
> An example of using triggers for indexing:
> http://maxgrinev.com/2010/07/23/managing-indexes-in-cassandra-using-async-triggers/
> Implementation details are attached.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (CASSANDRA-1311) Support (asynchronous) triggers

Posted by "Martin Hentschel (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-1311?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Martin Hentschel updated CASSANDRA-1311:
----------------------------------------

    Attachment: HOWTO-PatchAndRunTriggerExample.txt

> Support (asynchronous) triggers
> -------------------------------
>
>                 Key: CASSANDRA-1311
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1311
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Contrib
>            Reporter: Maxim Grinev
>             Fix For: 0.8
>
>         Attachments: HOWTO-PatchAndRunTriggerExample.txt, ImplementationDetails.pdf, trunk-967053.txt
>
>
> Asynchronous triggers is a basic mechanism to implement various use cases of asynchronous execution of application code at database side. For example to support indexes and materialized views, online analytics, push-based data propagation.
> Please find the motivation, triggers description and list of applications:
> http://maxgrinev.com/2010/07/23/extending-cassandra-with-asynchronous-triggers/
> An example of using triggers for indexing:
> http://maxgrinev.com/2010/07/23/managing-indexes-in-cassandra-using-async-triggers/
> Implementation details are attached.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (CASSANDRA-1311) Support (asynchronous) triggers

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-1311?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12898362#action_12898362 ] 

Stu Hood commented on CASSANDRA-1311:
-------------------------------------

> Faster storage of "dangling" triggers at a slave node
I mentioned to Maxim at the summit that I think persisting the dangling triggers to the system tables is a good idea, because it preserves the guarantee that a full power loss to the cluster does not lose data. I imagine that it should be possible to gain the performance back somehow, since persisting to disk only requires additional writes (and should only read in node-failure cases).

Otherwise, I really like the architecture of this change, and only have minor suggestions:
* findTriggers is much too deeply nested: also, I'd like to see the bufferSize logic contained there moved into a maybeDrainNotificationBuffer method, or something.
* I'm not sure Arrays.hashCode is a strong enough indicator of uniqueness: you might want to switch to MD5
* TriggerSlave.deletedTriggers looks fragile (needs to expire old deletions), but switching back to using the system table would resolve that

> Support (asynchronous) triggers
> -------------------------------
>
>                 Key: CASSANDRA-1311
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1311
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Contrib
>            Reporter: Maxim Grinev
>             Fix For: 0.8
>
>         Attachments: HOWTO-PatchAndRunTriggerExample-update1.txt, HOWTO-PatchAndRunTriggerExample.txt, ImplementationDetails-update1.pdf, ImplementationDetails.pdf, trunk-967053.txt, trunk-984391-update1.txt
>
>
> Asynchronous triggers is a basic mechanism to implement various use cases of asynchronous execution of application code at database side. For example to support indexes and materialized views, online analytics, push-based data propagation.
> Please find the motivation, triggers description and list of applications:
> http://maxgrinev.com/2010/07/23/extending-cassandra-with-asynchronous-triggers/
> An example of using triggers for indexing:
> http://maxgrinev.com/2010/07/23/managing-indexes-in-cassandra-using-async-triggers/
> Implementation details are attached.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (CASSANDRA-1311) Support (asynchronous) triggers

Posted by "Maxim Grinev (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-1311?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Maxim Grinev updated CASSANDRA-1311:
------------------------------------

    Attachment: trunk-967053.txt

Patch for SVN revision 967053 (July 23)

> Support (asynchronous) triggers
> -------------------------------
>
>                 Key: CASSANDRA-1311
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1311
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Contrib
>            Reporter: Maxim Grinev
>         Attachments: trunk-967053.txt
>
>
> Asynchronous trigger is a basic mechanism to implement various use cases of asynchronous execution of application code at database side. For example to support indexes and materialized views, online analytics, push-based data propagation.
> Please find the motivation, triggers description and list of applications:
> http://maxgrinev.com/2010/07/23/extending-cassandra-with-asynchronous-triggers/
> An example of using triggers for indexing:
> http://maxgrinev.com/2010/07/23/managing-indexes-in-cassandra-using-async-triggers/
> Implementation details are attached.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (CASSANDRA-1311) Support (asynchronous) triggers

Posted by "Maxim Grinev (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-1311?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12892251#action_12892251 ] 

Maxim Grinev commented on CASSANDRA-1311:
-----------------------------------------

We studied the discussion of [CASSANDRA-749|https://issues.apache.org/jira/browse/CASSANDRA-749]

1) CASSANDRA-749 is about local vs. distributed secondary indexes.
You decided to start with local indexes but stated that both approaches have their advantages and disadvantages.
Our triggers allow to implement distributed secondary indexes.  So, as concerns indexing, triggers complement CASSANDRA-749 with distributed indexes.

2) Stu Hood proposed to support view ([https://issues.apache.org/jira/browse/CASSANDRA-749?focusedCommentId=12829403&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12829403]) as a general mechanism for 'advanced' indexing. 

Our triggers is a mechanism to implement exactly this view approach.

Stu put the advantates really nicely so I just cite him:
 - "But views are considerably more powerful, since you can store any item in the key or value for the view."
-  "Also, a view is more conducive to duplication of data, which we prefer in Cassandra: rather than having secondary indexes pointing to the one true copy of the data, you can duplicate that data in a view if you'd like, and have it be lazily/eagerly updated serverside."

Moreover, support for duplicate data allows to map basic SQL operations to Cassandra's data model as described in [http://maxgrinev.com/2010/07/12/do-you-really-need-sql-to-do-it-all-in-cassandra/]

Also, as a general mechanism, triggers/views can be used for other applications such as online analytics or workflow-like (push) data propagation.

> Support (asynchronous) triggers
> -------------------------------
>
>                 Key: CASSANDRA-1311
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1311
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Contrib
>            Reporter: Maxim Grinev
>             Fix For: 0.8
>
>         Attachments: HOWTO-PatchAndRunTriggerExample.txt, ImplementationDetails.pdf, trunk-967053.txt
>
>
> Asynchronous triggers is a basic mechanism to implement various use cases of asynchronous execution of application code at database side. For example to support indexes and materialized views, online analytics, push-based data propagation.
> Please find the motivation, triggers description and list of applications:
> http://maxgrinev.com/2010/07/23/extending-cassandra-with-asynchronous-triggers/
> An example of using triggers for indexing:
> http://maxgrinev.com/2010/07/23/managing-indexes-in-cassandra-using-async-triggers/
> Implementation details are attached.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (CASSANDRA-1311) Support (asynchronous) triggers

Posted by "David Erickson (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-1311?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12898741#action_12898741 ] 

David Erickson commented on CASSANDRA-1311:
-------------------------------------------

A quick glance at the implementation looks like its doing Slave hashing based only on its IP address, would this cause problems with hosts running multiple Cassandra instances on a single IP address?

> Support (asynchronous) triggers
> -------------------------------
>
>                 Key: CASSANDRA-1311
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1311
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Contrib
>            Reporter: Maxim Grinev
>             Fix For: 0.8
>
>         Attachments: HOWTO-PatchAndRunTriggerExample-update1.txt, HOWTO-PatchAndRunTriggerExample.txt, ImplementationDetails-update1.pdf, ImplementationDetails.pdf, trunk-967053.txt, trunk-984391-update1.txt
>
>
> Asynchronous triggers is a basic mechanism to implement various use cases of asynchronous execution of application code at database side. For example to support indexes and materialized views, online analytics, push-based data propagation.
> Please find the motivation, triggers description and list of applications:
> http://maxgrinev.com/2010/07/23/extending-cassandra-with-asynchronous-triggers/
> An example of using triggers for indexing:
> http://maxgrinev.com/2010/07/23/managing-indexes-in-cassandra-using-async-triggers/
> Implementation details are attached.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (CASSANDRA-1311) Support (asynchronous) triggers

Posted by "Martin Hentschel (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-1311?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12891653#action_12891653 ] 

Martin Hentschel commented on CASSANDRA-1311:
---------------------------------------------

Thanks for referring to CASSANDRA-1016. We were not aware of it and it is similar to our work. We think the main differences of our work to CASSANDRA-1016 are the following:

* Triggers are executed asynchronously (Stu Hood mentioned it as [lazy execution|https://issues.apache.org/jira/browse/CASSANDRA-749?focusedCommentId=12829403&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#action_12829403]).
* We provide a fail-over mechanism. If a node, which was responsible for executing a trigger, goes down, replicas will take over to ensure proper execution. We implemented at-least-once semantics of triggers.
* Triggers are set on column families, instead of the whole database (which might have been implemented by 1016 as well now).

Asynchronous means that triggers will be executed after the client has received the acknowledgment. Hence the response time of a user request is improved (as opposed to CASSANDRA-1016, which we think implements synchronous execution of triggers). The downside of it is that indexes might be out of sync with the base data sometimes. We think that this is fine for many use cases where Cassandra is the database of choice (as explained in our blog posts).

> Support (asynchronous) triggers
> -------------------------------
>
>                 Key: CASSANDRA-1311
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1311
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Contrib
>            Reporter: Maxim Grinev
>         Attachments: ImplementationDetails.pdf, trunk-967053.txt
>
>
> Asynchronous triggers is a basic mechanism to implement various use cases of asynchronous execution of application code at database side. For example to support indexes and materialized views, online analytics, push-based data propagation.
> Please find the motivation, triggers description and list of applications:
> http://maxgrinev.com/2010/07/23/extending-cassandra-with-asynchronous-triggers/
> An example of using triggers for indexing:
> http://maxgrinev.com/2010/07/23/managing-indexes-in-cassandra-using-async-triggers/
> Implementation details are attached.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.