[ https://issues.apache.org/jira/browse/CASSANDRA-1311?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13137492#comment-13137492 ]
Jonathan Ellis edited comment on CASSANDRA-1311 at 10/27/11 8:27 PM: --------------------------------------------------------------------- Potential crack smoking ahead: What if we had a "distributed commitlog" for coordinator-based triggers? Triggers would be allowed to turn a write into a batch, or a batch into a modified batch. Triggers would *not* be allowed to do chains of non-idempotent logic (i.e. anything that can't be represented as a batch). Then coordinator would: - write the expanded batch (as serialized blob) to distributed commitlog (keyed by coordinator node id+; column name some kind of uuid) with CL equal to requested for original update - apply the batch normally - delete the batch DCL blob After restarting, the node (or its replacement) should check its commitlog row and re-apply any unfinished batches. + In practice this means something like id-per-hour, since Cassandra deals poorly with rows with an unbounded number of tombstones was (Author: jbellis): Potential crack smoking ahead: What if we had a "distributed commitlog" for coordinator-based triggers? Triggers would be allowed to turn a write into a batch, or a batch into a modified batch. Triggers would *not* be allowed to do chains of non-idempotent logic (i.e. anything that can't be represented as a batch). Then coordinator would: - write the expanded batch (as serialized blob) to distributed commitlog (keyed by coordinator node id+; column name some kind of uuid) with CL equal to requested for original update - apply the batch normally - delete the batch After restarting, the node (or its replacement) should check its commitlog row and re-apply any unfinished batches. + In practice this means something like id-per-hour, since Cassandra deals poorly with rows with an unbounded number of tombstones > Triggers > -------- > > Key: CASSANDRA-1311 > URL: https://issues.apache.org/jira/browse/CASSANDRA-1311 > Project: Cassandra > Issue Type: New Feature > 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. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira