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

Joshua McKenzie commented on CASSANDRA-8844:
--------------------------------------------

I am impressed (and bothered) at how much I missed the forest for the trees on 
that one - I refactored out the {{CommitLogReplayer}} behavior quite awhile 
before adding the segment/offset skipping logic in the CommitLogReader for CDC 
and it never clicked that I was just duplicating the existing CommitLogReplayer 
globalPosition skip. I better understand where the confusion on our discussion 
(and your reading of the code) stemmed from.

Pushed a commit that does the following:
* Moved {{CommitLogReplayer}} skip logic into {{CommitLogReader}}
* Unified on minPosition in {{CommitLogReader}} rather than old startPosition
* Removed superfluous interface methods
* Tidied up and commented various read* methods in CommitLogReader
* Commented CommitLogSegment.nextId to clarify that we rely on it for correct 
ordering between multiple CLSM
* Revised static initializer in CommitLogSegment to take CDC log location into 
account on idBase determination
* Added comment in CommitLog reinforcing the need for the above

The fact that none of us caught the idBase determination in CommitLogSegment's 
init makes me wary, and I agree with you that this needs further testing. Where 
are we with that [~mambocab]?

Regarding the DirectorySizeCalculator, while I much prefer the elegance of your 
one-liner

# I like to avoid changing code that's battle-tested and working during an 
unrelated refactor
# it's a micro-optimzation in a part of the code that's not critical path and 
where the delta will be on the order of microseconds for the average case 
(though a large simplification and reduction in code as well, so I'd do it for 
that alone), and
# the benchmarking results of testing that on both win10 and linux had some 
surprises in store:

{noformat}
Windows, skylake, SSD:
   DirectorySizeCalculator
      [java] Result: 31.061 ¦(99.9%) 0.287 ms/op [Average]
      [java]   Statistics: (min, avg, max) = (30.861, 31.061, 33.028), stdev = 
0.430
      [java]   Confidence interval (99.9%): [30.774, 31.349]
   One liner:
      [java] Result: 116.941 ¦(99.9%) 1.238 ms/op [Average]
      [java]   Statistics: (min, avg, max) = (115.163, 116.941, 124.950), stdev 
= 1.854
      [java]   Confidence interval (99.9%): [115.703, 118.179]
Linux, haswell, SSD:
   DirectorySizeCalculator
      [java] Result: 76.765 ±(99.9%) 0.876 ms/op [Average]
      [java]   Statistics: (min, avg, max) = (75.586, 76.765, 81.744), stdev = 
1.311
      [java]   Confidence interval (99.9%): [75.889, 77.641]
   One liner:
      [java] Result: 57.608 ±(99.9%) 0.889 ms/op [Average]
      [java]   Statistics: (min, avg, max) = (56.365, 57.608, 61.697), stdev = 
1.330
      [java]   Confidence interval (99.9%): [56.719, 58.497]
{noformat}

I think that makes a strong case for us having a platform independent 
implementation of this and doing this in a follow-up ticket.

I also haven't done anything about CommitLogSegmentPosition's name yet. I don't 
have really strong feelings on it but am leaning towards {{CommitLogPosition}}.

Re-ran CI since we've made quite a few minor tweaks/refactors throughout, and 
there's a small amount (14 failures) of house-cleaning left to do on the tests. 
I'll start digging into that tomorrow.

> Change Data Capture (CDC)
> -------------------------
>
>                 Key: CASSANDRA-8844
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-8844
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Coordination, Local Write-Read Paths
>            Reporter: Tupshin Harper
>            Assignee: Joshua McKenzie
>            Priority: Critical
>             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 that node's write. If 
> that means the requested consistency level was not met, then clients *should* 
> experience UnavailableExceptions.
> - Be written in a Row-centric manner such that it is easy for consumers to 
> reconstitute rows atomically.
> - Written in a simple format designed to be consumed *directly* by daemons 
> written in non JVM languages
> h2. Nice-to-haves
> I strongly suspect that the following features will be asked for, but I also 
> believe that they can be deferred for a subsequent release, and to guage 
> actual interest.
> - Multiple logs per table. This would make it easy to have multiple 
> "subscribers" to a single table's changes. A workaround would be to create a 
> forking daemon listener, but that's not a great answer.
> - Log filtering. Being able to apply filters, including UDF-based filters 
> would make Casandra a much more versatile feeder into other systems, and 
> again, reduce complexity that would otherwise need to be built into the 
> daemons.
> h2. Format and Consumption
> - Cassandra would only write to the CDC log, and never delete from it. 
> - Cleaning up consumed logfiles would be the client daemon's responibility
> - Logfile size should probably be configurable.
> - Logfiles should be named with a predictable naming schema, making it 
> triivial to process them in order.
> - Daemons should be able to checkpoint their work, and resume from where they 
> left off. This means they would have to leave some file artifact in the CDC 
> log's directory.
> - A sophisticated daemon should be able to be written that could 
> -- Catch up, in written-order, even when it is multiple logfiles behind in 
> processing
> -- Be able to continuously "tail" the most recent logfile and get 
> low-latency(ms?) access to the data as it is written.
> h2. Alternate approach
> In order to make consuming a change log easy and efficient to do with low 
> latency, the following could supplement the approach outlined above
> - Instead of writing to a logfile, by default, Cassandra could expose a 
> socket for a daemon to connect to, and from which it could pull each row.
> - Cassandra would have a limited buffer for storing rows, should the listener 
> become backlogged, but it would immediately spill to disk in that case, never 
> incurring large in-memory costs.
> h2. Additional consumption possibility
> With all of the above, still relevant:
> - instead (or in addition to) using the other logging mechanisms, use CQL 
> transport itself as a logger.
> - Extend the CQL protoocol slightly so that rows of data can be return to a 
> listener that didn't explicit make a query, but instead registered itself 
> with Cassandra as a listener for a particular event type, and in this case, 
> the event type would be anything that would otherwise go to a CDC log.
> - If there is no listener for the event type associated with that log, or if 
> that listener gets backlogged, the rows will again spill to the persistent 
> storage.
> h2. Possible Syntax
> {code:sql}
> CREATE TABLE ... WITH CDC LOG
> {code}
> Pros: No syntax extesions
> Cons: doesn't make it easy to capture the various permutations (i'm happy to 
> be proven wrong) of per-dc logging. also, the hypothetical multiple logs per 
> table would break this
> {code:sql}
> CREATE CDC_LOG mylog ON mytable WHERE MyUdf(mycol1, mycol2) = 5 with 
> DCs={'dc1','dc3'}
> {code}
> Pros: Expressive and allows for easy DDL management of all aspects of CDC
> Cons: Syntax additions. Added complexity, partly for features that might not 
> be implemented



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to