[
https://issues.apache.org/jira/browse/CASSANDRA-1311?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12976307#action_12976307
]
Martin Hentschel commented on CASSANDRA-1311:
---------------------------------------------
No, wait, there seems to be a misconception here.
Whenever there is a write to a column family for which a trigger has been
defined, that trigger is guaranteed to be executed at least once. For that, we
use the same mechanics as any replicated write to a column family. In
Cassandra, a write is not acknowledged if the write quorum cannot be met. If a
write is acknowledged though, the client can be sure the write is durable. The
same holds for our trigger framework: if the write has been acknowledged, the
client can be sure the trigger will be executed (at least once). As we see it,
there is no window for permanent inconsistency. If you could provide us with
an example of such a window, we would be more than happy to think about it and
make changes to our code as necessary.
Having at-least-once semantics requires triggers to be implemented
idempotently. Exactly-once semantics of trigger execution would only allow
triggers to not be implemented idempotently. This, of course, has advantages
but does not mean that triggers are executed in a more reliable/guaranteed way.
> 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.
-
You can reply to this email to add a comment to the issue online.