-
Notifications
You must be signed in to change notification settings - Fork 3.9k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
RFC: Change Data Capture #17535
RFC: Change Data Capture #17535
Conversation
6149daf
to
8ac9bc9
Compare
This generally looks good to me. I'd like to see some discussion of latency here. There are a lot of things that can cause delays, especially for in-order delivery. How quickly can changes make it through this pipeline? If Kafka and/or the ChangeAggregator can't keep up, what are the failure modes? There should be circuit breakers to ensure a clean break instead of endless buffering or spinning on catch-up scans. Review status: 0 of 1 files reviewed at latest revision, 8 unresolved discussions, all commit checks successful. docs/RFCS/change_feeds_distsql.md, line 43 at r1 (raw file):
Is watching a database equivalent to separate watches on every table in the DB, or are transactions combined across all the tables they touch? Can a watched-set include multiple tables (but not an entire DB)? docs/RFCS/change_feeds_distsql.md, line 53 at r1 (raw file):
Maybe I'm overthinking this (given the example of rethinkdb), but putting the entire old row here (as opposed to just the PK) seems unnecessarily expensive. We may want to consider making this optional. docs/RFCS/change_feeds_distsql.md, line 68 at r1 (raw file):
When can you be sure you have everything for a transaction id? docs/RFCS/change_feeds_distsql.md, line 190 at r1 (raw file):
If the client requested out-of-order updates, this part is skipped and a single failed ChangeFeed doesn't block everything else, right? docs/RFCS/change_feeds_distsql.md, line 209 at r1 (raw file):
s/the/any/ ? docs/RFCS/change_feeds_distsql.md, line 335 at r1 (raw file):
This is true of triggers with effects outside the database; triggers that write to other tables in the same database are typically exactly-once. docs/RFCS/change_feeds_distsql.md, line 412 at r1 (raw file):
This is not the terminology we normally use. The result of writing an intent goes to the coordinator, not the range holding the transaction record. docs/RFCS/change_feeds_distsql.md, line 461 at r1 (raw file):
What exactly does "registering dependencies" mean here? What part of the range is responsible for doing this? It sounds like the ChangeFeed operation must periodically try to scan its whole range to trigger intent resolution for any extant intents. Comments from Reviewable |
Review status: 0 of 1 files reviewed at latest revision, 10 unresolved discussions, all commit checks successful. docs/RFCS/change_feeds_distsql.md, line 42 at r1 (raw file):
You're forgetting one here: by geo-partitioning prefix. docs/RFCS/change_feeds_distsql.md, line 109 at r1 (raw file):
There's some black magic needed here to ensure that a retried txn (either server-side or client-side) only causes one change feed to be created, with the last one. Perhaps delay the creation of the change feed to the moment the SQL transaction commits? Comments from Reviewable |
Looks good! I didn't read the appendix thoroughly.
@bdarnell afaict latency is really about the slowest close notification. Once a timestamp is closed, the latency shouldn't be much more than what we typically expect of DistSQL. #16838 specifically discusses cases in which you want to cut down your latency (listening for a handful of keys as opposed to full-blown CDC) by making use of the fact that updates are always ordered when restricted to a single key. Reviewed 1 of 1 files at r1. docs/RFCS/change_feeds_distsql.md, line 53 at r1 (raw file): Previously, bdarnell (Ben Darnell) wrote…
docs/RFCS/change_feeds_distsql.md, line 68 at r1 (raw file): Previously, bdarnell (Ben Darnell) wrote…
Streaming by transaction is a good bit less trivial than streaming by row. To decide that a transaction is finished, the consumer of the various primitives must be notified of the successful Can you make to without the txn ID and simply group by timestamp? The only downside is that you might be seeing multiple transactions' writes at the same timestamp (in which case they won't overlap), but maybe that's good enough. Please motivate the need for transactional grouping -- it's a bunch of extra work and if we can avoid it at all, I'm pretty sure we should. docs/RFCS/change_feeds_distsql.md, line 71 at r1 (raw file):
What do you win here? You don't have to sort when the txn timestamp closes out on all potentially written to ranges? Anything else? docs/RFCS/change_feeds_distsql.md, line 109 at r1 (raw file): Previously, knz (kena) wrote…
The BEGIN TRANSACTION; -- read-only
SELECT bunch_of_stuff();
SELECT transaction_timestamp(); -- or whatever the right method is
COMMIT;
MAKE MAGIC CHANGEFEED AS OF SYSTEM TIME timestamp_from_above(); I think that's roughly what's envisioned above, but @arjunravinarayan should clarify. docs/RFCS/change_feeds_distsql.md, line 113 at r1 (raw file):
Can you elaborate? I thought Kafka topics shard and you could have 10 Cockroach nodes each pushing into one shard. As written, it sounds like we need one topic per producer (= CockroachDB processor?) docs/RFCS/change_feeds_distsql.md, line 117 at r1 (raw file):
[citation needed] docs/RFCS/change_feeds_distsql.md, line 153 at r1 (raw file):
You're talking to a replica, so you've looked up its RangeID, and there's no need to put that into every message. docs/RFCS/change_feeds_distsql.md, line 158 at r1 (raw file):
This isn't sufficient, see the comment in #16593 (comment) It'll be more difficult to guarantee that a closed-out timestamp has no intents under it that are committed but not resolved. I think the onus here is on #16838 (RFC: Storage-Level Primitive for Change Feeds) to make that happen, but it doesn't hurt to point that out (or just remove the parens). Also affects next subsection. docs/RFCS/change_feeds_distsql.md, line 249 at r1 (raw file):
nit: no dot docs/RFCS/change_feeds_distsql.md, line 461 at r1 (raw file): Previously, bdarnell (Ben Darnell) wrote…
That or the Replica must maintain an (unreplicated, i.e. in aux storage, and consistent enough) index of timestamp -> intent. We could make sure we only write that index when the range is being watched (doing a single scan when the feed is set up, which we'll have to do at least once anyway). I think making that happen is more a requirement we'll want to impose on #16593 (comment) than this PR. Sure looks a bit unpleasant, though it seems necessary. Comments from Reviewable |
Review status: all files reviewed at latest revision, 23 unresolved discussions, all commit checks successful. docs/RFCS/change_feeds_distsql.md, line 20 at r1 (raw file):
docs/RFCS/change_feeds_distsql.md, line 29 at r1 (raw file):
I wouldn't describe these as chaos events. Perhaps "maintenance events". docs/RFCS/change_feeds_distsql.md, line 32 at r1 (raw file):
docs/RFCS/change_feeds_distsql.md, line 36 at r1 (raw file):
Better to include a link to the RFC directly (when it is available). docs/RFCS/change_feeds_distsql.md, line 43 at r1 (raw file): Previously, bdarnell (Ben Darnell) wrote…
If I watch for transactions on a single table, do I see all of the mutations in the transaction even if they occur on other tables or only the mutations that touch the table I'm watching? docs/RFCS/change_feeds_distsql.md, line 54 at r1 (raw file):
Does the docs/RFCS/change_feeds_distsql.md, line 84 at r1 (raw file):
docs/RFCS/change_feeds_distsql.md, line 189 at r1 (raw file):
Is there 1 Rather than be a complete downer with this comment, I'll suggest (without having put much thought into it) that we can support N Comments from Reviewable |
This RFC proposes a mechanism for subscribing to changes to a table or database. It will support the Apache Kafka publisher API. It will make use of a Replica level subsystem "ChangeFeed", the design of which is described in cockroachdb#16838.
8ac9bc9
to
97e3e11
Compare
Thanks for the reviews, everyone! There's a lot to digest here, so I've only addressed some of the comments. For the rest, I need some time to fully work through before posting answers, which I will do in a day or two and re-up the RFC. Review status: 0 of 1 files reviewed at latest revision, 23 unresolved discussions, some commit checks pending. docs/RFCS/change_feeds_distsql.md, line 20 at r1 (raw file): Previously, petermattis (Peter Mattis) wrote…
Correct. Done. docs/RFCS/change_feeds_distsql.md, line 29 at r1 (raw file): Previously, petermattis (Peter Mattis) wrote…
Done. docs/RFCS/change_feeds_distsql.md, line 32 at r1 (raw file): Previously, petermattis (Peter Mattis) wrote…
Done. docs/RFCS/change_feeds_distsql.md, line 36 at r1 (raw file): Previously, petermattis (Peter Mattis) wrote…
Made into a link to the PR for now, so that at least its a real link. Will update to the RFC when merged. docs/RFCS/change_feeds_distsql.md, line 42 at r1 (raw file): Previously, knz (kena) wrote…
I do not follow this point. Could you please elaborate? docs/RFCS/change_feeds_distsql.md, line 43 at r1 (raw file): Previously, petermattis (Peter Mattis) wrote…
I have not thought seriously about which answer to any of these questions would be the best behavior. On thinking about it, I believe that all the variations are possible, with various tradeoffs. I will add a section that addresses these tradeoffs soon. Thank you. docs/RFCS/change_feeds_distsql.md, line 53 at r1 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
Just the PK is an elegant default solution. Tracking the complete old row should definitely be optional. I will add this. docs/RFCS/change_feeds_distsql.md, line 54 at r1 (raw file): Previously, petermattis (Peter Mattis) wrote…
No it does not, as currently envisioned. Good point. I will document this. docs/RFCS/change_feeds_distsql.md, line 71 at r1 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
Added an answer to your question in the test. docs/RFCS/change_feeds_distsql.md, line 84 at r1 (raw file): Previously, petermattis (Peter Mattis) wrote…
Done. docs/RFCS/change_feeds_distsql.md, line 109 at r1 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
@tschottdorf is correct. Amended. docs/RFCS/change_feeds_distsql.md, line 113 at r1 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
An error. Rewritten. The intention is for there to be multiple changefeeds writing to the same topic. docs/RFCS/change_feeds_distsql.md, line 117 at r1 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
Done. docs/RFCS/change_feeds_distsql.md, line 153 at r1 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
Done. docs/RFCS/change_feeds_distsql.md, line 158 at r1 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
Removed the parens. Added some text to the next subsection; PTAL. docs/RFCS/change_feeds_distsql.md, line 189 at r1 (raw file): Previously, petermattis (Peter Mattis) wrote…
The intention is to have a docs/RFCS/change_feeds_distsql.md, line 190 at r1 (raw file): Previously, bdarnell (Ben Darnell) wrote…
Correct. Added. docs/RFCS/change_feeds_distsql.md, line 209 at r1 (raw file): Previously, bdarnell (Ben Darnell) wrote…
Done. docs/RFCS/change_feeds_distsql.md, line 249 at r1 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
Done. docs/RFCS/change_feeds_distsql.md, line 335 at r1 (raw file): Previously, bdarnell (Ben Darnell) wrote…
Done. docs/RFCS/change_feeds_distsql.md, line 412 at r1 (raw file): Previously, bdarnell (Ben Darnell) wrote…
Great! This was the intention of writing the appendix - to make sure I had the story straight. I will carefully go over this with someone and update this. docs/RFCS/change_feeds_distsql.md, line 461 at r1 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
Yes, I'm not sure there's a way to avoid that. Comments from Reviewable |
Review status: 0 of 1 files reviewed at latest revision, 23 unresolved discussions, all commit checks successful. docs/RFCS/change_feeds_distsql.md, line 42 at r1 (raw file): Previously, arjunravinarayan (Arjun Narayan) wrote…
If the user does geo-partitioning they will typically want to route change events that touch rows from a region to a separate collector than that of rows for another region. docs/RFCS/change_feeds_distsql.md, line 109 at r1 (raw file): Previously, arjunravinarayan (Arjun Narayan) wrote…
This means that clients cannot expect to start seeing events on the change feed until the txn has committed. Perhaps this can be highlighted too. Comments from Reviewable |
Review status: 0 of 1 files reviewed at latest revision, 23 unresolved discussions, all commit checks successful. docs/RFCS/change_feeds_distsql.md, line 42 at r1 (raw file): Previously, knz (kena) wrote…
(Data sovereignty concerns etc) Comments from Reviewable |
Review status: 0 of 1 files reviewed at latest revision, 18 unresolved discussions, all commit checks successful. docs/RFCS/change_feeds_distsql.md, line 189 at r1 (raw file): Previously, arjunravinarayan (Arjun Narayan) wrote…
Ok, I'm missing something fundamental about how this would work. If you have 2 Also, with multiple Comments from Reviewable |
Yes, my question was trying to get a little more specific about the "slowest close notification". How frequently do we intend to send close notifications, and how are our actual close notifications distributed around that target? Full table-level CDC is less latency-sensitive than other potential uses of change feeds, but we should try to get a handle on what that latency would be, and also the catch-up effects (it's one thing if a single slow close notification delays the close notification of a downstream consumer but the data is otherwise flowing, but if a slow close notification blocks the data flow as well (as I think it does in this proposal), it's more severe since the CDC process will consume more resources while it catches up. This will need to be rate limited to avoid interfering with OLTP traffic, but we also need to consider what happens when the CDC feed is unable to catch up) Review status: 0 of 1 files reviewed at latest revision, 17 unresolved discussions, all commit checks successful. docs/RFCS/change_feeds_distsql.md, line 43 at r1 (raw file): Previously, arjunravinarayan (Arjun Narayan) wrote…
This is where more concrete examples in the motivation section would be useful (and not just deferring to #16838 - some of those examples would definitely not use CDC; we should be clear about which woulds are intended to and let that guide which tradeoffs we make). docs/RFCS/change_feeds_distsql.md, line 68 at r1 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
On second thought, I don't think you'd need to do anything tricky to figure out when you have everything for a transaction. If you're already producing a feed in timestamp order (which is itself expensive! I think that to minimize catch-up effects we should encourage consumers of CDC to use out-of-order feeds and understand close notifications), you just need to buffer one timestamp's worth of data to split it into transactions. The only costly change is that you'd need to do something about 1PC transactions. docs/RFCS/change_feeds_distsql.md, line 77 at r2 (raw file):
The summary proposes to offer both at-least-once and at-most-once options; this buffering would only be required in at-most-once modes. (Which use cases would use each mode?) docs/RFCS/change_feeds_distsql.md, line 122 at r2 (raw file):
s/EEE/FEE/ Comments from Reviewable |
Review status: 0 of 1 files reviewed at latest revision, 24 unresolved discussions, all commit checks successful. docs/RFCS/change_feeds_distsql.md, line 95 at r2 (raw file):
Is this required by Kafka or other major consumers? Why is it worth the effort? docs/RFCS/change_feeds_distsql.md, line 136 at r2 (raw file):
#16838 includes waiting on updates for individual rows or small spans as the first motivating use case. That may just be chance, but did you check up on what the feature requests we've gotten were looking for? I'd think that allowing for filtering on a contiguous range of primary keys may be worthwhile, but would like to see what folks are asking for either way. docs/RFCS/change_feeds_distsql.md, line 163 at r2 (raw file):
If the starting timestamp has to be newer than the current timestamp, then how will the What if the timestamp chosen here isn't newer than the leaseholder's current timestamp? I think this will need a little more thought. docs/RFCS/change_feeds_distsql.md, line 207 at r2 (raw file):
A back-of-the-envelope calculation on how much buffering this will involve would be helpful. buffering 10+ seconds of all the writes on a large table or database seems as though it could get quite large. We'll probably need memory accounting here (although perhaps that's already implied). docs/RFCS/change_feeds_distsql.md, line 231 at r2 (raw file):
cc @benesch docs/RFCS/change_feeds_distsql.md, line 280 at r2 (raw file):
So what will the behavior be for 1.2 then? Everything will have to restart when a processor dies? docs/RFCS/change_feeds_distsql.md, line 440 at r2 (raw file):
As with Ben's comment above, Comments from Reviewable |
A close notification for time For a CDC that simply outputs everything to some other MVCC-aware storage without any aggregation at the txn level, I think close notifications don't block anything -- each processor receiving change feed updates simply relays everything back up the chain; close notifications only serve to limit the amount of work that has to be done should the feed disconnect. It's trickier if you're trying to group emitted updates into complete per-timestamp write sets (i.e. emit all writes at ts T at the same time, or emit all writes of a txn at the same time) because you do have to wait for all possible sources of such values to close out for any given timestamp you wish to report. Reviewed 1 of 1 files at r2. docs/RFCS/change_feeds_distsql.md, line 109 at r1 (raw file): Previously, knz (kena) wrote…
What do you mean, @knz? docs/RFCS/change_feeds_distsql.md, line 207 at r2 (raw file): Previously, a-robinson (Alex Robinson) wrote…
I think auxiliary storage will be at work here, though it'd be good to point that out. Comments from Reviewable |
docs/RFCS/change_feeds_distsql.md, line 20 at r2 (raw file):
Clarifying question - how would you support at-least-once at the database level if you only support at most once at the single-range primitive level? Comments from Reviewable |
Review status: all files reviewed at latest revision, 25 unresolved discussions, all commit checks successful. docs/RFCS/change_feeds_distsql.md, line 77 at r2 (raw file): Previously, bdarnell (Ben Darnell) wrote…
One use case I'm aware of is stream processing of IoT data, where you don't need every single ping in order to get interesting insights. docs/RFCS/change_feeds_distsql.md, line 95 at r2 (raw file): Previously, a-robinson (Alex Robinson) wrote…
Is this basically saying that we want to preserve commit ordering? If we want to be able to replay transaction logs, it's an important feature, but based on my understanding of Kafka, it's not a requirement. docs/RFCS/change_feeds_distsql.md, line 300 at r2 (raw file):
On this note, for 1.2, I think we should try to punt complexity to Kafka as much as possible. Also, PostgreSQL has this functionality for progress tracking that is actually user-facing / configurable. https://www.postgresql.org/docs/9.5/static/replication-origins.html docs/RFCS/change_feeds_distsql.md, line 320 at r2 (raw file):
Do we also want to support becoming a data sink - ingesting data from Kafka? docs/RFCS/change_feeds_distsql.md, line 343 at r2 (raw file):
Could you clarify 7 then with the expected behavior with the current implementation? Comments from Reviewable |
reminder: don't forget to rebase if you plan to continue work on this, and pick up the new RFC sections + naming convention. |
docs/RFCS/change_feeds_distsql.md, line 117 at r1 (raw file):
Another consideration here from Spencer is we eventually might want to support CRDB<>CRDB streams. Just something to keep in mind. Comments from Reviewable |
docs/RFCS/change_feeds_distsql.md, line 113 at r1 (raw file):
Correct me if I'm wrong, but won't this make ordering incredibly difficult? Comments from Reviewable |
This RFC is incomplete. Is is merged in `draft` status. This RFC proposes a mechanism for subscribing to changes to a set of key ranges starting at an initial timestamp. It is for use by Cockroach-internal higher-level systems such as distributed SQL, change data capture, or a Kafka producer endpoint. A "sister RFC" detailing these higher-level systems is in cockroachdb#17535. See also cockroachdb#19222 for a related follower-reads RFC.
This RFC is incomplete. Is is merged in `draft` status. This RFC proposes a mechanism for subscribing to changes to a set of key ranges starting at an initial timestamp. It is for use by Cockroach-internal higher-level systems such as distributed SQL, change data capture, or a Kafka producer endpoint. A "sister RFC" detailing these higher-level systems is in cockroachdb#17535. See also cockroachdb#19222 for a related follower-reads RFC.
This RFC proposes a mechanism for subscribing to changes to a table or
database. It will support the Apache Kafka publisher API. It will make
use of a Replica level subsystem "ChangeFeed", the design of which is
described in #16838.