You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Edward Capriolo (JIRA)" <ji...@apache.org> on 2011/09/02 20:57:26 UTC

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

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

Edward Capriolo commented on CASSANDRA-1311:
--------------------------------------------

I think the work to pick a trigger master is really awesome. Great job, hard problem to solve. I think different people want different things from triggers. 

The 'internal developers' might want to be able to use triggers to build secondary indexes. They want triggers to happen close to the storage layer in most cases.

The 'external users' might want to be able to use triggers to replicate data to another system. They want triggers to be on the coordinating node.  

What I am proposing is we recognize this and implement two types of triggers. The trigger for 'external users' is the easiest and should be done first.

ExternalTriggers have a PreHook and PostHook. External triggers happen on the coordinating node. For the post hook, if an operation succeeds at the client specified consistency level a success_trigger fires. If it does not succeed a failed_trigger fires.

The weirdness in external triggers comes from a write timing out on some but not all nodes. That write could succeed on some nodes but not on all. Read repair and hinted handoff could eventually fix this data. This would result in the failed trigger firing but the write eventually succeeding. I argue that this behaviour is undefined. Clients are "supposed" to replay failed writes until success. 

Now the fun part, since we can fire a trigger on both success and failure we could theoretically deal with the above weirdness by implementing a guaranteed hinted handoff trigger. This would be done by using the PostHook failed_trigger to delay and then retry the write. 

> 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: 1.1
>
>         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.
For more information on JIRA, see: http://www.atlassian.com/software/jira