[jira] [Commented] (CASSANDRA-8844) Change Data Capture (CDC)
[ https://issues.apache.org/jira/browse/CASSANDRA-8844?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14522461#comment-14522461 ] Martin Kersten commented on CASSANDRA-8844: --- I would argue not for logs but for 'listener' queries for each table. If a client want to listen for a certain or all changes he is free to submit where clauses. So every time a changed row of that table fulfills the where clause the client listener gets notified. A client may issue many where clauses and would be able to query its active where clauses for a table. By removing all where clauses the listener will actually remove itself from the listener list for that table. One could even extend that by submitting real select statements working on only the currently change row. Maybe the listener may even add a timing setting allowing a node to aggregate multiple update events and send one single notification for multiple changed rows (if this makes sense for a system using hashing for partition / sharding). Since many clients may listen using the same where clause the performance would be manageable and not depending on how many clients are listening but how many different select statements where listened to by all clients. By being able to name the listening where clauses one could even generated named events where a combination of name and where clause makes it unique. An additional option is being able to add an additional value for each row containing the names of all event names associated with the listener select statements (queries). Pro: * Easy to understand * Easy to manage * Fine-tuning is possible (like a single client listens to only a single user or a list of particular users) * Lot of reuse capabilities (just query over the changed row(s) not all rows, grammar etc.) * Works on shared tables * Avoids maintaining disk logs * Only three operations are necessary to implement (add listener query, remove listener query and get all active listener queries). * Performance improvements possible by combining (almost) similar where clauses and by adding special cases (like where clauses for certain user IDs will result in a huge lists. * All in memory operation no disk writes necessary Contra: * Ensuring being notified exactly once might be challenging * Only works on changes about to happen afterwards and not changes happened in recent time. Change Data Capture (CDC) - Key: CASSANDRA-8844 URL: https://issues.apache.org/jira/browse/CASSANDRA-8844 Project: Cassandra Issue Type: New Feature Components: Core Reporter: Tupshin Harper Fix For: 3.x In databases, change data capture (CDC) is a set of software design patterns used to determine (and track) the data that has changed so that action can be taken using the changed data. Also, Change data capture (CDC) is an approach to data integration that is based on the identification, capture and delivery of the changes made to enterprise data sources. -Wikipedia As Cassandra is increasingly being used as the Source of Record (SoR) for mission critical data in large enterprises, it is increasingly being called upon to act as the central hub of traffic and data flow to other systems. In order to try to address the general need, we (cc [~brianmhess]), propose implementing a simple data logging mechanism to enable per-table CDC patterns. h2. The goals: # Use CQL as the primary ingestion mechanism, in order to leverage its Consistency Level semantics, and in order to treat it as the single reliable/durable SoR for the data. # To provide a mechanism for implementing good and reliable (deliver-at-least-once with possible mechanisms for deliver-exactly-once ) continuous semi-realtime feeds of mutations going into a Cassandra cluster. # To eliminate the developmental and operational burden of users so that they don't have to do dual writes to other systems. # For users that are currently doing batch export from a Cassandra system, give them the opportunity to make that realtime with a minimum of coding. h2. The mechanism: We propose a durable logging mechanism that functions similar to a commitlog, with the following nuances: - Takes place on every node, not just the coordinator, so RF number of copies are logged. - Separate log per table. - Per-table configuration. Only tables that are specified as CDC_LOG would do any logging. - Per DC. We are trying to keep the complexity to a minimum to make this an easy enhancement, but most likely use cases would prefer to only implement CDC logging in one (or a subset) of the DCs that are being replicated to - In the critical path of ConsistencyLevel acknowledgment. Just as with the commitlog, failure to write to the CDC log should fail
[jira] [Comment Edited] (CASSANDRA-8844) Change Data Capture (CDC)
[ https://issues.apache.org/jira/browse/CASSANDRA-8844?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14522461#comment-14522461 ] Martin Kersten edited comment on CASSANDRA-8844 at 4/30/15 11:29 PM: - I would argue not for logs but for 'listener' queries for each table. If a client want to listen for a certain or all changes he is free to submit where clauses. So every time a changed row of that table fulfills the where clause the client listener gets notified. A client may issue many where clauses and would be able to query its active where clauses for a table. By removing all where clauses the listener will actually remove itself from the listener list for that table. One could even extend that by submitting real select statements working on only the currently change row. Maybe the listener may even add a timing setting allowing a node to aggregate multiple update events and send one single notification for multiple changed rows (if this makes sense for a system using hashing for partition / sharding). Since many clients may listen using the same where clause the performance would be manageable and not depending on how many clients are listening but how many different select statements where listened to by all clients. By being able to name the listening where clauses one could even generated named events where a combination of name and where clause makes it unique. An additional option is being able to add an additional value for each row containing the names of all event names associated with the listener select statements (queries). Pro: * Easy to understand * Easy to manage * Fine-tuning is possible (like a single client listens to only a single user or a list of particular users) * Lot of reuse capabilities (just query over the changed row(s) not all rows, grammar etc.) * Works on shared tables * Avoids maintaining disk logs * Only three operations are necessary to implement (add listener query, remove listener query and get all active listener queries). * Performance improvements possible by combining (almost) similar where clauses and by adding special cases (like where clauses for certain user IDs will result in a huge lists. * All in memory operation no disk writes necessary Contra: * Ensuring being notified exactly once might be challenging * Only works on changes about to happen afterwards and not changes happened in recent time (no travel back in time like it would be possible while having change logs). was (Author: martin kersten): I would argue not for logs but for 'listener' queries for each table. If a client want to listen for a certain or all changes he is free to submit where clauses. So every time a changed row of that table fulfills the where clause the client listener gets notified. A client may issue many where clauses and would be able to query its active where clauses for a table. By removing all where clauses the listener will actually remove itself from the listener list for that table. One could even extend that by submitting real select statements working on only the currently change row. Maybe the listener may even add a timing setting allowing a node to aggregate multiple update events and send one single notification for multiple changed rows (if this makes sense for a system using hashing for partition / sharding). Since many clients may listen using the same where clause the performance would be manageable and not depending on how many clients are listening but how many different select statements where listened to by all clients. By being able to name the listening where clauses one could even generated named events where a combination of name and where clause makes it unique. An additional option is being able to add an additional value for each row containing the names of all event names associated with the listener select statements (queries). Pro: * Easy to understand * Easy to manage * Fine-tuning is possible (like a single client listens to only a single user or a list of particular users) * Lot of reuse capabilities (just query over the changed row(s) not all rows, grammar etc.) * Works on shared tables * Avoids maintaining disk logs * Only three operations are necessary to implement (add listener query, remove listener query and get all active listener queries). * Performance improvements possible by combining (almost) similar where clauses and by adding special cases (like where clauses for certain user IDs will result in a huge lists. * All in memory operation no disk writes necessary Contra: * Ensuring being notified exactly once might be challenging * Only works on changes about to happen afterwards and not changes happened in recent time. Change Data Capture (CDC) - Key: