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

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

{quote}
IRC: it's fundamentally different to have triggers that leave things 
permanently inconsistent, vs triggers that are as exactly as eventually 
consistent as the write they are part of
{quote}
Agreed. I still see + or minuses for both approaches. 
The big minus for the replica level triggers is that no one really wants to get 
N triggers. Which is why this ticket is stuck in the mud. (see that gets ugly 
below) 

I see triggers as an event not a storage acknowledgement. For example, if I 
receive the coordinator-trigger  "WRITE QUORUM KEY X COLUMN Y VALUE Z SUCCESS". 
I can now now attempt to READ key X COLUMN Y and get VALUE Z (or later). This 
is enormously useful.


Now if I receive one replica-trigger. "WRITE KEY X COLUMN Y VALUE Z SUCCESS". 
Now what? First, I do not know the CL of the write. I really do not even know 
the replication factor unless it has in intelligence. Without this information 
I can not be sure if I can read WRITE KEY X COLUMN Y and get VALUE Z. Not 
useful.

(that gets ugly)
The replica-triggers is going to be is it's own fairly complex state machine 
and complex requirements.  
1) It has to be able to accept writes as fast as cassandra. Maybe 
replication_factor times as fast because all three triggers have to land on the 
same server to make any assessment about the state of the system.
2) It needs to buffer and hold triggers until it receive replication_factor of 
them
3) This just feels beastly. 

 

> 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

        

Reply via email to