cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Joshua McKenzie (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-8844) Change Data Capture (CDC)
Date Mon, 18 Apr 2016 20:26:29 GMT

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

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

bq. Descriptor parsed id mismatch error doesn't look right. The replay position specifies
from which id (and position within that id) we should replay. In addition to (parts of) the
file with the same id, this includes all files with higher ids. Mismatch is normal.
The logical flow should be identical to trunk. It only returns at that point if {{CommitLogReadHandler.shouldStopOnError}}
returns true, which {{CommitLogReplayer.shouldStopOnError}} doesn't ever do. It's either permissable(not
in this case), ignored, or we throw from CLR.

bq. Mutation before offset check compares file position with logical segment position and
is only valid for uncompressed files.
My understanding of this code - the before offset check is against [reader.getFilePointer|https://github.com/apache/cassandra/compare/trunk...josh-mckenzie:8844_review#diff-9fe0bd988c4fc47a022f589f5ad72b09R232],
which in the compressed and encrypted case is from [syncSegment.input|https://github.com/apache/cassandra/compare/trunk...josh-mckenzie:8844_review#diff-9fe0bd988c4fc47a022f589f5ad72b09R196].
That consists of a [FileSegmentInputStream|https://github.com/josh-mckenzie/cassandra/blob/8844_review/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentReader.java#L290]
wrapped around the uncompressed buffer in the compressed case, so the raw sentinel checking
in Mutation skipping logic should correctly apply as though it were a normal uncompressed
file as the {{getFilePointer}} calls in {{FileSegmentInputStream}} (and {{EncryptedFileSegmentInputStream}}
for that matter) take their offsets into account. As CommitLogReaderTest.testReadFromMidpoint
passes on test-compression and validates that the replayed indexes after the passed in offset
are the correct numeric value, I'd be surprised if that sentinel check didn't work as I tried
to explicitly test for that. Added comments surrounding this fact to the comparison for Mutation
skipping. Also - please let me know if there's something I'm missing or if I've misread this
code.

bq. shouldSkipSegment JavaDoc should make it clear which kind of position it needs (it is
used correctly).
Added clarification that it's a logical position

bq. prepReader is only called for pre-2.1 segments. JavaDoc does not say so. I don't think
we want this in the handler interface, inline it at its one use site.
Documented in javadoc. Since globalPosition exists inside CommitLogReplayer, I've left it
in the interface for now as I don't see a need to move globalPosition into the reader.
bq. statusTracker.flagError isn't a very fitting name for what is actually a termination request.
Renamed to requestTermination.

bq. flagError and return is inconsistent with the rest in readSection. It should also return
regardless of the shouldStop result as there's nothing meaningful to be done with the rest
of the section...
I'd actually prefer to revert this delta and consider opening a follow-up ticket with this
change if we consider it an improvement. There's enough going into this CDC refactor that
my primary goal is to keep the reading logic itself untouched.

bq. segmentId confuses that it would be the one used later. We should rename this to segmentIdFromFilename.
Good call. Changed.

bq. tolerateErrorsInSection &=: I don't think it was intended for the value to depend
on previous iterations.
Given the implementation of [tolerateSegmentErrors|https://github.com/josh-mckenzie/cassandra/blob/8844_review/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentReader.java#L229]
I'm inclined to agree. That being said, this is also another change I'd prefer to create as
a follow-up ticket rather than changing behavior of non-CDC related things on this ticket.

I'll address your further feedback in another update. The above changes are pushed.

> 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)

Mime
View raw message