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

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

Jonathan, are we right that you are mainly concerned about the inconsistency 
problem we were discussing earlier?

To repeat, the problem was that if a write has not been acknowledged, it may 
still have succeeded in the base columnfamily (Table.apply) but the trigger is 
stored durably (TriggerSlave.storeDanglingTrigger). It is because trigger 
execution is not part of log replay. So we can end up with data that never had 
the trigger fire.

As Stu said, using entity groups to solve this problem is suboptimal. We want 
to propose an improvement of our approach that also solves this problem but 
does not introduce the restrictions of entity groups. 

Our solution is to make the storage of "dangling" trigger at replicas also part 
of log replay. Whenever a replica (slave) node receives a write request, it 
will durably log that it has to fire a trigger in case of a failure of the 
update coordinator (master). In case this slave node fails, it will come back 
up replaying the logs, installing any data item, and also firing triggers.

Once again, our point is to avoid entity group whenever possible for the 
following two reasons. (1) Entity groups are restrictive because they require 
co-location of data. This is hard to achieve in practice and not all 
application can be implemented this. For example, it is not possible to 
implement Twitter as it would require to co-locate users with their followers 
ending up with a single, big entity group. (2) Even in case when you can use 
entity groups, you usually need to update redundant data stored in other entity 
groups as well. In this case, triggers may be used to keep redundant data 
consistent across entity groups.

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

        

Reply via email to