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

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

bq. We have published a technical report about the idea of integrating 
asynchronous triggers into Cassandra [1]

As described above in my reaction to the patch, the problem with this is that 
it relies on client-side retries to avoid durable inconsistencies:

bq. Given the protocol in Figure 4, the failure handling of the master is 
straight-forward. On reception of a write request,
if the master fails before sending the acknowledgement to the application 
(anywhere before step M5 in Figure 4), the task might or might not have been 
executed. Because the application will not get a response, the application 
retries the write request (at a dierent node now). Therefore, the trigger will 
be executed at least once.

This is a problem for several reasons.  First, it pushes part of the complexity 
back to the client; second, because relying on the client to never crash itself 
is a poor assumption, and third, because there *is* no separate client when 
using the StorageProxy API, as some users do to avoid the extra latency and 
overhead of Thrift.

bq. 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.

I'm less a fan of this than of my crack-smoking commitlog (CSCL for short).  
Partly because it spreads the complexity outside the coordinator, and partly 
because it's a less-general tool -- the CSCL also solves the inconsistency 
problem for batches that do *not* involve triggers as well.

bq. To guarantee exactly-once semantics of trigger execution, we have build a 
system that departs significantly from this patch [2].

I stopped reading when I got to "We implemented a two-phase commit protocol in 
order to execute distributed transactions." 2PC has well-known problems with 
coordinator failure. 
(http://en.wikipedia.org/wiki/Two-phase_commit_protocol#Disadvantages)

                
> 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.2
>
>         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


Reply via email to