-
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: Storage-Level Primitive for Change Feeds #16838
Conversation
This looks pretty good, although it's leaning heavily on the follower-read RFC which still has a lot of unresolved questions, and I'm not sure the answers there are necessarily compatible with the needs of this RFC. For transactional writes, the change feed event is emitted when the intent is resolved, instead of when the original write happens. For follower reads, we've discussed implementing close timestamps via the timestamp cache or something like it, with the semantics that "no more writes with a timestamp less than T will be accepted". However, there could still be unresolved intents with timestamps less than T. For change feeds, we rewrite that "no new events with timestamp less than T will be emitted", so we must resolve all intents before we can advance the close timestamp. Review status: 0 of 1 files reviewed at latest revision, 7 unresolved discussions, some commit checks failed. docs/RFCS/change_feeds_primitive.md, line 55 at r1 (raw file):
This is one of the coolest features of CouchDB :) docs/RFCS/change_feeds_primitive.md, line 64 at r1 (raw file):
s/CDT/CDC/g docs/RFCS/change_feeds_primitive.md, line 82 at r1 (raw file):
What does "poll" mean here? docs/RFCS/change_feeds_primitive.md, line 96 at r1 (raw file):
There's also the "closed timestamp" event described above. Can consumers assume that for a given key, timestamps are monotonic? Can they make any assumptions about timestamps across keys (other than what is given by closed timestamps)? Is docs/RFCS/change_feeds_primitive.md, line 105 at r1 (raw file):
This will be a new top-level RPC (peer to Batch), and not a member of RequestUnion, right? (Otherwise I don't see how you stream responses) docs/RFCS/change_feeds_primitive.md, line 223 at r1 (raw file):
A third performance concern is that when single-row watches exist, there will likely be a lot of them active at once. This will have a memory cost, and could have a CPU cost as every write might need to be matched against many watchers. docs/RFCS/change_feeds_primitive.md, line 233 at r1 (raw file):
A single ChangeFeed operation with a base timestamp far in the past could also be a problem. It needs to process all events since its base timestamp before it can switch into streaming mode and get its first closed timestamp update. This could happen in a loop if the clients can't keep up. We may want to place limits on how far back ChangeFeeds can go (on the other hand, if a ChangeFeed is rejected, the client will probably fall back to doing a SELECT which may be even more expensive). Comments from Reviewable |
Reviewed 1 of 1 files at r1. docs/RFCS/change_feeds_primitive.md, line 25 at r1 (raw file):
It should also return a failure condition, if the timestamp is too far back in time and it cannot stream all changes from that timestamp because the GC queue has kicked in. docs/RFCS/change_feeds_primitive.md, line 64 at r1 (raw file):
CDC is the preferred acronym (Change Data Capture) docs/RFCS/change_feeds_primitive.md, line 101 at r1 (raw file):
Definitely think the format will be much simpler, although doesn't need to be specced out in this document. docs/RFCS/change_feeds_primitive.md, line 118 at r1 (raw file):
I don't understand the sentence inside the parenthesis, it needs elaboration. docs/RFCS/change_feeds_primitive.md, line 136 at r1 (raw file):
Instead of backpressure, we should consider triggering an early replica split if we're under heavy load as well. It's an easier solution. docs/RFCS/change_feeds_primitive.md, line 146 at r1 (raw file):
How about: We need a command C defined as follows: given a snapshot S1, a key span [K], and and a base timestamp T1 < current time T2, docs/RFCS/change_feeds_primitive.md, line 150 at r1 (raw file):
I don't get this argument, perhaps it needs more elaboration, with a little bit more formality. Would applying the WriteBatches (Put K1t1, Delete K1t2, Put K1t3, Delete K1t4) in any order give the right answer? What happens if we apply them in the order (Delete K1t4, Delete K1t2, Put K1t3, Put K1t1)? docs/RFCS/change_feeds_primitive.md, line 172 at r1 (raw file):
So far we have not covered how close notifications are propagated to followers and become part of the regular Raft commands. That would be required to make this happen. docs/RFCS/change_feeds_primitive.md, line 181 at r1 (raw file):
Streams need to be heartbeating their close notifications anyway, so this may not even be required, although it's a nice-to-have. docs/RFCS/change_feeds_primitive.md, line 217 at r1 (raw file):
of course. ;) Comments from Reviewable |
I'm very happy to see thought being put into this already. I believe CDC is going to be the first user and am mildly anxious about the proposed mechanism because we don't have the companion document describing CDC that is needed to understand the complete system. Review status: all files reviewed at latest revision, 25 unresolved discussions, some commit checks failed. docs/RFCS/change_feeds_primitive.md, line 17 at r1 (raw file):
The first sentence sounds incomplete to me. Or perhaps just abrupt. docs/RFCS/change_feeds_primitive.md, line 28 at r1 (raw file):
Sentence needs a period. docs/RFCS/change_feeds_primitive.md, line 53 at r1 (raw file):
Should add why companies tend to ask for it (e.g. to move data into an analytics platform). docs/RFCS/change_feeds_primitive.md, line 78 at r1 (raw file):
Is the term "close notifications" taken from somewhere? I'm finding it confusing when reading this doc as I think of closing a stream which isn't analogous. docs/RFCS/change_feeds_primitive.md, line 112 at r1 (raw file):
How is buffering going to work here? What happens if the docs/RFCS/change_feeds_primitive.md, line 115 at r1 (raw file):
This sentence was confusing to me as written. The snapshot is used to retrieve the updates that were made between a base timestamp and the time when the snapshot was taken (which is also when the feed was registered). My question about buffering becomes more serious here. I assume we're going to send these old changes to the feed first, but doing so might take 10s of seconds (or longer), during which time we'd have to buffer the "live" changes. Buffering those changes in memory feels problematic. Buffering them to disk feels complicated. docs/RFCS/change_feeds_primitive.md, line 134 at r1 (raw file):
The close notification is expected to be O(10s), right? This protocol will result in the changes for the past 10s getting resent. Am I missing something here? I think this needs to be called out explicitly in the doc. docs/RFCS/change_feeds_primitive.md, line 254 at r1 (raw file):
Does the proposed mechanism provide the correct building block for Change Data Capture? In particular, what interface does CDC want to present? A stream of key updates? A stream of row updates? A stream of committed transactions? A stream of key or row updates seems feasible with this mechanism. I don't see how a stream of committed transactions would be accomplished. I realize that the design of CDC is outside of the scope of this RFC, but it would be a shame to implement this mechanism and then discover that certain CDC requirements make it unusable. Comments from Reviewable |
Review status: all files reviewed at latest revision, 30 unresolved discussions, some commit checks failed. docs/RFCS/change_feeds_primitive.md, line 41 at r1 (raw file):
I'm sure you're aware, but there's a very real discussion to be had about prioritization here. Change data feeds are only useful to a user if the rest of the system is good enough for them to use it as their primary data store :) docs/RFCS/change_feeds_primitive.md, line 106 at r1 (raw file):
So if, at a higher level, a client wants to watch multiple key spans from the same range, it'll have to issue multiple docs/RFCS/change_feeds_primitive.md, line 136 at r1 (raw file):
When you say "if they find themselves unable to keep up", do you mean if the client isn't processing items off the stream quickly enough? Is that easy to detect in grpc? docs/RFCS/change_feeds_primitive.md, line 150 at r1 (raw file): Previously, arjunravinarayan (Arjun Narayan) wrote…
I don't see why the question of whether the WriteBatches could be reapplied to RocksDB is important here. Shouldn't we only care about what clients see, which I was curious about at first but seems to be addressed by the option described in the next paragraph? docs/RFCS/change_feeds_primitive.md, line 157 at r1 (raw file):
This sentence makes it sound like clients could request ascending order to avoid waiting, which seems backwards. However, I'm not seeing why ascending order on a per-key basis would require waiting during steady-state operation. A replica shouldn't be applying an update to a key until all after all prior updates to that key have already been applied, so it seems as though there wouldn't be waiting required to just emit updates as they're applied. docs/RFCS/change_feeds_primitive.md, line 176 at r1 (raw file):
"straightforward to entity"? Comments from Reviewable |
Reviewed 1 of 1 files at r2. docs/RFCS/change_feeds_primitive.md, line 41 at r1 (raw file): Previously, a-robinson (Alex Robinson) wrote…
To make it completely clear, I (and I believe Toby as well) have been working on change feeds on Fridays. I'm working on it now because I'm on vacation :) docs/RFCS/change_feeds_primitive.md, line 53 at r1 (raw file): Previously, petermattis (Peter Mattis) wrote…
But also replication to cold storage/legacy caches. docs/RFCS/change_feeds_primitive.md, line 78 at r1 (raw file): Previously, petermattis (Peter Mattis) wrote…
It was cribbed from the skunkworks materialized views RFC (which cribbed it from Timely Dataflow). It needs definition in this doc. Here's something to steal:
docs/RFCS/change_feeds_primitive.md, line 96 at r1 (raw file): Previously, bdarnell (Ben Darnell) wrote…
It is my understanding that Cockroach timestamps can be treated as monotonic across all keys. docs/RFCS/change_feeds_primitive.md, line 106 at r1 (raw file): Previously, a-robinson (Alex Robinson) wrote…
I've been thinking about watching entire ranges. Any filtering that involves spans-with-holes can happen downstream, to keep things simple. docs/RFCS/change_feeds_primitive.md, line 112 at r1 (raw file): Previously, petermattis (Peter Mattis) wrote…
I think we should drop the docs/RFCS/change_feeds_primitive.md, line 115 at r1 (raw file): Previously, petermattis (Peter Mattis) wrote…
We can send changes out of order (making buffering the receiver's problem), we just can't send close notifications until we are sure of the ordering. If the receiver requires a total ordering, it is the receivers job to buffer, sort, and wait for close notifications before doing any work with the stream. The first consumer of these change feeds will be a DistSQL processor, and DistSQL processors have access to sorted External Storage, and they can do exactly this. You should not think of the change feed sender as doing anything memory intensive, ever. If you are, then this document is misleading and needs fixing. docs/RFCS/change_feeds_primitive.md, line 134 at r1 (raw file): Previously, petermattis (Peter Mattis) wrote…
Close notifications for a running stream are expected to trail by ~10 seconds. This doesn't mean the tuples are resent. Here's how it would work: docs/RFCS/change_feeds_primitive.md, line 136 at r1 (raw file): Previously, a-robinson (Alex Robinson) wrote…
Yes. A client that is unable to process the stream quickly needs to do something, most likely binary split itself into two clients, which process one half of the stream each. This will be covered in the sister RFC. docs/RFCS/change_feeds_primitive.md, line 157 at r1 (raw file): Previously, a-robinson (Alex Robinson) wrote…
I don't think clients should ever be able to request ascending order at this level of abstraction. If we want to service clients who have this need, we just need a simple DistSQL processor that buffers rows, sorts them, and waits for close notifications before sending them on in-order. The DistSQL processor sits in between this change feed and the end-client. docs/RFCS/change_feeds_primitive.md, line 223 at r1 (raw file): Previously, bdarnell (Ben Darnell) wrote…
Yes. It's an open question how many CDCs will come out of a replica. I think of there being a single CDC out of every range, for the entire range, and that getting mirrored, filtered, etc to multiple "real" consumers. docs/RFCS/change_feeds_primitive.md, line 233 at r1 (raw file): Previously, bdarnell (Ben Darnell) wrote…
You can only truly solve this load issue with follower reads. docs/RFCS/change_feeds_primitive.md, line 254 at r1 (raw file): Previously, petermattis (Peter Mattis) wrote…
A stream of row updates is what people really want, at least from the perspective of a) what RethinkDB provided, and b) what materialized views needs. docs/RFCS/change_feeds_primitive.md, line 104 at r2 (raw file):
But do note that this second close notification is completely moot, as docs/RFCS/change_feeds_primitive.md, line 111 at r2 (raw file):
No these should never happen. docs/RFCS/change_feeds_primitive.md, line 155 at r2 (raw file):
sentence needs fixing, missing a comma or something. Comments from Reviewable |
@petermattis I will work on finishing up the companion RFC that more thoroughly describes CDCs and send it out soon. Review status: all files reviewed at latest revision, 32 unresolved discussions, all commit checks successful. Comments from Reviewable |
Yep, the design of follower reads would be influenced by this (this is partially the motivation in putting this RFC out so early). An alternative is serving change feeds from the lease holders only, but it would be good to avoid it. Review status: 0 of 1 files reviewed at latest revision, 34 unresolved discussions. docs/RFCS/change_feeds_primitive.md, line 17 at r1 (raw file): Previously, petermattis (Peter Mattis) wrote…
Done. docs/RFCS/change_feeds_primitive.md, line 25 at r1 (raw file): Previously, arjunravinarayan (Arjun Narayan) wrote…
I thought that was clear, but added a few words. docs/RFCS/change_feeds_primitive.md, line 28 at r1 (raw file): Previously, petermattis (Peter Mattis) wrote…
Done. docs/RFCS/change_feeds_primitive.md, line 41 at r1 (raw file): Previously, a-robinson (Alex Robinson) wrote…
Oh, yeah, this feature isn't on any current roadmap afaict. May be worth pointing that out explicitly. docs/RFCS/change_feeds_primitive.md, line 53 at r1 (raw file): Previously, petermattis (Peter Mattis) wrote…
Done. docs/RFCS/change_feeds_primitive.md, line 64 at r1 (raw file): Previously, arjunravinarayan (Arjun Narayan) wrote…
Thanks, not sure what CDT stands for. docs/RFCS/change_feeds_primitive.md, line 78 at r1 (raw file): Previously, petermattis (Peter Mattis) wrote…
I suppose the Naiad papers (for example https://cs.stanford.edu/~matei/courses/2015/6.S897/readings/naiad.pdf) but it's not really named explicitly there. I'm open to suggestions. docs/RFCS/change_feeds_primitive.md, line 82 at r1 (raw file): Previously, bdarnell (Ben Darnell) wrote…
Just "operate on". Changed. docs/RFCS/change_feeds_primitive.md, line 96 at r1 (raw file): Previously, bdarnell (Ben Darnell) wrote…
I expanded a little bit on the guarantees (TL;DR none given) docs/RFCS/change_feeds_primitive.md, line 105 at r1 (raw file): Previously, bdarnell (Ben Darnell) wrote…
Yep, made that clear. docs/RFCS/change_feeds_primitive.md, line 106 at r1 (raw file):
That also makes things hard, because now you're always moving a lot of data, even if you don't care about anything but a single key. Hence my initial reduction to one key range, which covers both that case and CDC. docs/RFCS/change_feeds_primitive.md, line 112 at r1 (raw file):
docs/RFCS/change_feeds_primitive.md, line 115 at r1 (raw file): Previously, petermattis (Peter Mattis) wrote…
No, you don't guarantee any order, so you only have to hold back close notifications while you're not "caught up". In effect, this pushes this problem to the caller, but I think of those like DistSQL processors with temporary storage, etc, so they're equipped to do this sort of thing well already. docs/RFCS/change_feeds_primitive.md, line 118 at r1 (raw file):
docs/RFCS/change_feeds_primitive.md, line 134 at r1 (raw file): Previously, petermattis (Peter Mattis) wrote…
Yes, a stream is only exactly-once until it is disrupted. The purpose of close notifications is to checkpoint work efficiently (well, as efficiently as it gets, anyway). Note that the caller isn't some user; it's some subsystem that knows that it has to keep enough state to filter duplicates that could result from duplications. I added a "semantics" section above. docs/RFCS/change_feeds_primitive.md, line 136 at r1 (raw file): Previously, arjunravinarayan (Arjun Narayan) wrote…
Done. docs/RFCS/change_feeds_primitive.md, line 136 at r1 (raw file): Previously, a-robinson (Alex Robinson) wrote…
I don't know if it's easy enough to detect in gRPC, but you could for example use a buffered channel on which you put updates, and if that channel fills up, the stream cancels (with some more lenient mechanism during the initial catchup). docs/RFCS/change_feeds_primitive.md, line 146 at r1 (raw file):
docs/RFCS/change_feeds_primitive.md, line 150 at r1 (raw file):
I also reworked the paragraph under this thread to be clearer about ordering. docs/RFCS/change_feeds_primitive.md, line 157 at r1 (raw file): Previously, arjunravinarayan (Arjun Narayan) wrote…
No, we can't wait for close notifications for the Github example, it needs to be snappier than that. I reworked this by adding a docs/RFCS/change_feeds_primitive.md, line 172 at r1 (raw file): Previously, arjunravinarayan (Arjun Narayan) wrote…
That's because I think it should be discussed in the context of follower reads (though as Ben pointed out, the close notifications are more complicated than follower read notifications). docs/RFCS/change_feeds_primitive.md, line 176 at r1 (raw file): Previously, a-robinson (Alex Robinson) wrote…
Done. docs/RFCS/change_feeds_primitive.md, line 181 at r1 (raw file): Previously, arjunravinarayan (Arjun Narayan) wrote…
What do you mean by "heartbeating a close notification"? docs/RFCS/change_feeds_primitive.md, line 223 at r1 (raw file): Previously, bdarnell (Ben Darnell) wrote…
Added. docs/RFCS/change_feeds_primitive.md, line 233 at r1 (raw file): Previously, bdarnell (Ben Darnell) wrote…
Done. docs/RFCS/change_feeds_primitive.md, line 254 at r1 (raw file): Previously, petermattis (Peter Mattis) wrote…
That's the paragraph below in "Alternative designs". I agree that getting a rough idea of CDC's requirements would be good. Comments from Reviewable |
BTW, I'm not sure CDC will be the "first user" here. I think the Github example is also something that's really interesting for developers. We have someone every other week asking about polling stuff like this efficiently, and there just isn't a good way to do it so far. We can make single-key watchers work much more easily than full-blown CDC. Review status: 0 of 1 files reviewed at latest revision, 27 unresolved discussions, some commit checks pending. docs/RFCS/change_feeds_primitive.md, line 64 at r1 (raw file): Previously, bdarnell (Ben Darnell) wrote…
Done. docs/RFCS/change_feeds_primitive.md, line 104 at r2 (raw file): Previously, arjunravinarayan (Arjun Narayan) wrote…
Reworked these examples, and added reason for why clients should be expected to handle close notifications out of order (TL;DR don't expect them in practice, but really don't want to commit to guaranteeing it seeing that it's unclear where they're even gonna come from). docs/RFCS/change_feeds_primitive.md, line 111 at r2 (raw file): Previously, arjunravinarayan (Arjun Narayan) wrote…
Yeah, that was bad wording. "anomalies" referred to the stuff above the.. well, real anomaly. Since removed. docs/RFCS/change_feeds_primitive.md, line 155 at r2 (raw file): Previously, arjunravinarayan (Arjun Narayan) wrote…
Done. Comments from Reviewable |
A CDC can be the first user, but with a DistSQL layer in between that does reorderings. You should note more explicitly that a "real" user-facing CDC will have that layer in-between (since some readers assumed the opposite). Review status: 0 of 1 files reviewed at latest revision, 23 unresolved discussions, some commit checks pending. docs/RFCS/change_feeds_primitive.md, line 78 at r1 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
The Naiad paper is not great at defining things in easily digestible words. I recommend you just copy the definition I pasted above, with a citation. docs/RFCS/change_feeds_primitive.md, line 118 at r1 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
Great! docs/RFCS/change_feeds_primitive.md, line 134 at r1 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
I would suggest not ever using the words 'exactly-once', if only to avoid triggering some drive-by reader. It's like saying we are a 'CA' system in CAP. I know what you mean, but it's just asking for trouble. :) docs/RFCS/change_feeds_primitive.md, line 157 at r1 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
But a client is free to do something with out-of-order events that aren't yet closed! Like a chat application can just deliver messages eagerly, out of order. I guess I see the point of the As a tangent, I'd like to note that this is why Naiad is so fast: in practice, it turns out that most relational operators can do a lot of things with out-of-order updates before they get close notifications, such that when the close notification finally arrives, they have O(1) or O(log n) work remaining. For a long dataflow pipeline, this dramatically reduces tail latencies at each layer of the dataflow graph, vastly improving end-to-end query processing time. This is why Naiad is faster than vanilla dataflow (i.e. DistSQL) even if you throw out all the incremental computation stateful-differential computation stuff. docs/RFCS/change_feeds_primitive.md, line 181 at r1 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
You need close notifications for timestamps even if there is no activity on the range. If I am a client who needs to interleave changes in-order from two different ranges, I need to know if timestamp t is closed from range 1, even if range 1 has no activity, because I can't ship an update at time docs/RFCS/change_feeds_primitive.md, line 104 at r2 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
I think Comments from Reviewable |
Heh, I likely overstated my belief, though I was basing that on reading the roadmap tea leaves. Let me restate: CDC appears to be a frequently requested requirement for enterprise integration. It is likely to be coming in the 1.2/1.3 time frame and we should, as pessimistic engineers, assume 1.2 until we learn otherwise. Yes, there are a whole lot of questions to be answered about CDC. I've been imagining it was a kafka feed of committed transactions. Hopefully that isn't too far off. Review status: 0 of 1 files reviewed at latest revision, 21 unresolved discussions, all commit checks successful. docs/RFCS/change_feeds_primitive.md, line 78 at r1 (raw file): Previously, arjunravinarayan (Arjun Narayan) wrote…
Perhaps we could use the term "checkpoint notification". docs/RFCS/change_feeds_primitive.md, line 145 at r3 (raw file):
Curious what simplification this provides. Performing the check on the stream doesn't seem onerous. Comments from Reviewable |
Review status: 0 of 1 files reviewed at latest revision, 17 unresolved discussions, all commit checks successful. docs/RFCS/change_feeds_primitive.md, line 78 at r1 (raw file): Previously, petermattis (Peter Mattis) wrote…
@arjunravinarayan didn't copy the whole definition (after all, this RFC doesn't care about the receiver of its events - that's your RFC, and 2. doesn't really fit here), but I specialized it to docs/RFCS/change_feeds_primitive.md, line 134 at r1 (raw file): Previously, arjunravinarayan (Arjun Narayan) wrote…
It's better than at-least-once until you reconnect, but yes, best to avoid that phrase. Updated. docs/RFCS/change_feeds_primitive.md, line 181 at r1 (raw file): Previously, arjunravinarayan (Arjun Narayan) wrote…
Ah, I somehow misread this as heartbeats the other way around - from client to Replica. I removed this sentence - this sort of thing really fits into your RFC much better. docs/RFCS/change_feeds_primitive.md, line 104 at r2 (raw file): Previously, arjunravinarayan (Arjun Narayan) wrote…
Happy to bikeshed, though I'm not interested in taking the lead 😅 docs/RFCS/change_feeds_primitive.md, line 145 at r3 (raw file): Previously, petermattis (Peter Mattis) wrote…
Between reconnects, you may be talking to completely different replicas. Comments from Reviewable |
Review status: 0 of 1 files reviewed at latest revision, 18 unresolved discussions, some commit checks pending. docs/RFCS/change_feeds_primitive.md, line 112 at r1 (raw file):
Followers for some ranges are leaders for others. I think serving change feeds from followers is going to be of limited use in minimizing the production impact. Ultimately, I think we're going to need to allow operators to classify consumers of these feeds as critical or non-critical, and for critical feeds we apply backpressure to writes. (For non-critical feeds, maybe we skip the backfill step or allow it to discard some data) docs/RFCS/change_feeds_primitive.md, line 56 at r3 (raw file):
Another variation of this use case is migrating a table from one cockroach cluster to another (for example, between a hosted service and a self-managed cluster). Our regular replication can't handle replication between clusters; something like this could be used to move tables between clusters with minimal (but non-zero) downtime. Comments from Reviewable |
Review status: 0 of 1 files reviewed at latest revision, 17 unresolved discussions, some commit checks pending. docs/RFCS/change_feeds_primitive.md, line 78 at r1 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
Looks good to me! docs/RFCS/change_feeds_primitive.md, line 112 at r1 (raw file): Previously, bdarnell (Ben Darnell) wrote…
Followers are leaders for other ranges, but almost every workload is skewed in some fashion, such that some specific hot ranges are the bottlenecks at any given point in time. docs/RFCS/change_feeds_primitive.md, line 104 at r2 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
How about Comments from Reviewable |
Review status: 0 of 1 files reviewed at latest revision, 17 unresolved discussions, all commit checks successful. docs/RFCS/change_feeds_primitive.md, line 112 at r1 (raw file): Previously, arjunravinarayan (Arjun Narayan) wrote…
Maybe this should've been written down more explicitly (I think @arjunravinarayan will probably dwell on it since I think I heard it from him) I was picturing that in a serious CDC setup we would have, say, ZoneConfig rules that designate some set of nodes to primarily push out changes (and/or deal with updating the views), so they would prefer to remain followers and avoid data they're not CDC'ing as much as they can. For the single-key watcher case, there's still some benefit to having this on the followers. Better watchers/3 on each node than all on the lease holder; perhaps only a few ranges are watcher-hot. But it's hard to argue that that would be a general case, so if that was the only use case, probably easier to just serve change feeds from the lease holder only. docs/RFCS/change_feeds_primitive.md, line 104 at r2 (raw file): Previously, arjunravinarayan (Arjun Narayan) wrote…
Done. docs/RFCS/change_feeds_primitive.md, line 56 at r3 (raw file): Previously, bdarnell (Ben Darnell) wrote…
Added. Comments from Reviewable |
Review status: 0 of 1 files reviewed at latest revision, 18 unresolved discussions, some commit checks failed. docs/RFCS/change_feeds_primitive.md, line 324 at r5 (raw file):
Could we use the Raft log index to reduce duplicate entries when a feed reconnects? This would only help with rebalancing-based reconnects, not splits, but rebalances happen much more frequently. We'd have to include the Raft log index (and perhaps range ID) along with the timestamp. There might be other downsides to this approach that I'm not thinking of. I was pondering this because, if I squint, the Comments from Reviewable |
Review status: 0 of 1 files reviewed at latest revision, 18 unresolved discussions, some commit checks failed. docs/RFCS/change_feeds_primitive.md, line 324 at r5 (raw file): Previously, petermattis (Peter Mattis) wrote…
The main downside is that raft logs are truncated frequently, so we can't provide much of a guarantee with them (unless we want the truncation queue to consider this and apply backpressure for active change feeds). I think it's a good idea for close notifications to include both an application-readable timestamp and an opaque blob to be passed back on reconnections. Comments from Reviewable |
A question: how feasible would it be to stream the pair Reviewed 1 of 1 files at r4, 1 of 1 files at r5. Comments from Reviewable |
On a follower, retrieving the old value is always going to be fairly expensive since the old value probably won't be in cache and will have to go all the way to disk. On the leader, because most UPDATEs read the old value first, it would probably be a cache hit. Even with a cache hit, there's a nontrivial cost to processing this. It's (probably) cheaper than if the consumer of the feed would just do queries for the old values anyway, but it would be even better if consumers that need the old value maintained their own cache. Review status: all files reviewed at latest revision, 17 unresolved discussions, some commit checks failed. Comments from Reviewable |
Review status: 0 of 1 files reviewed at latest revision, 17 unresolved discussions. docs/RFCS/change_feeds_primitive.md, line 324 at r5 (raw file): Previously, bdarnell (Ben Darnell) wrote…
Passing the log index back sounds good to me, though it's more effective to do it on every emitted event (which is the version I wrote down as an optimization). That makes catch-up a lot less scary, I think. In some distant future in which log truncation is less replicated, we could even keep the log around for longer if we are serving feeds. Comments from Reviewable |
Review status: 0 of 1 files reviewed at latest revision, 18 unresolved discussions, all commit checks successful. docs/RFCS/change_feeds_primitive.md, line 266 at r6 (raw file):
👍 Comments from Reviewable |
Review status: 0 of 1 files reviewed at latest revision, 19 unresolved discussions, all commit checks successful. docs/RFCS/change_feeds_primitive.md, line 252 at r6 (raw file):
Does "largest" here imply that the client is doing a lexicographic comparison, or is it just using the most recent? If so, the token should be a version number followed by the raft log index, encoded with our ordered key encoding. (if there's no lexicographic comparison it should probably be a protobuf) Comments from Reviewable |
Review status: 0 of 1 files reviewed at latest revision, 19 unresolved discussions, all commit checks successful. docs/RFCS/change_feeds_primitive.md, line 252 at r6 (raw file): Previously, bdarnell (Ben Darnell) wrote…
Largest as defined by some total order (basically highest log index). I don't see how these would arrive out of order anyway, but I like cementing the fact that we don't have to think about it. From the clients perspective, I think an opaque blob (int or lexicographical bytes) will do. Comments from Reviewable |
docs/RFCS/change_feeds_primitive.md, line 105 at r6 (raw file):
I think we also need the transaction Comments from Reviewable |
Review status: 0 of 1 files reviewed at latest revision, 14 unresolved discussions, all commit checks successful. docs/RFCS/change_feeds_primitive.md, line 105 at r6 (raw file): Previously, arjunravinarayan (Arjun Narayan) wrote…
The UUID naively can't be reconstructed from the WriteBatch because it isn't always sent (1PC transactions), so additional effort will be necessary. We have the ID in most WriteBatches when intents are written, so it could mostly be recovered, but we'll definitely have to add it in the 1PC path. What's the use case that requires the txn id? Comments from Reviewable |
docs/RFCS/change_feeds_primitive.md, line 105 at r6 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
If we want to group changes at the client-facing CDC by transaction rather than by row, we need the transaction UUID. Comments from Reviewable |
1PC transactions are by definition single-range operations, so we never need to correlate them with anything else. It may not matter that the original transaction id has been lost; we can just synthesize a new UUID for any write batch that applies without a transaction id. (although it's possible to expose the transaction id through sql, so it's probably better to plumb it through the evaluation stack if we can) |
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.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I am missing something in this RFC: operational concerns.
An administrator must be able to monitor which change trackers have been configured on each node, be able to measure how fast they appear to be, how much they are "lagging behind", etc. this means all these trackers must be aptly named, and registered in some concurrently inspectable data structure.
Also be able to disconnect them from "outside" of the app/client that defined them, or at least present a textual hint about how to disconnect them by other means.
Also report these things in cockroach zip
and debug endpoints.
docs/RFCS/change_feeds_primitive.md
Outdated
duplicates and miss updates. | ||
|
||
This is a toy implementation that already makes for a good demo and allows a | ||
similar toy implementation that uses [streaming SQL][streamsql] once it's |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Need to clarify somewhere what this "streaming SQL" feature is that you're referring to multiple times.
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.
reminder: don't forget to rebase if you plan to continue work on this, and pick up the new RFC sections + naming convention. |
Does reovery ChangeFeeds's history data from the rocksdb's snapshot needs to scanning the entire range? if there are multiple ChangeFeeds that need to be restored at the same time, it may take a long time to scan. MVCC-level |
@coder0718 yes, raft GC is controlled by (one of the) replicas.
The initial setup for the changefeed would require scanning the whole
range, though we can use iterators that skip over parts of the MVCC
keyspace that aren't relevant for the requested base timestamp (and once
the feed is running, we hopefully can rely on the Raft log "forever").
On Tue, Oct 31, 2017 at 3:43 AM coder0718 ***@***.***> wrote:
Is it possible to control the raft log's GC time?
if you can save the last several hours of the log, you can recover history
data from the raft log in most cases.
Does reovery ChangeFeeds's history data from the rocksdb's snapshot needs
to scanning the entire range? this means, if there are multiple ChangeFeeds
that need to be restored at the same time, it may take a long time to scan.
------------------------------
MVCC-level
We add an MVCC command that, given a snapshot, a key span and a base
timestamp, synthesizes events for all newer-than-base-timestamp versions of
keys contained in the span.
Note that data is laid out in timestamp-descending per-key order, which is
the opposite of the naively expected order. However, during the catch-up
phase, the stream may emit events in any order, and so there is no problem.
A reverse scan could be used instead and would result in forward order,
albeit perhaps at a slight performance cost and little added benefit.
Note: There is significant overlap with NewMVCCIncrementalIterator, which
is used in incremental Backup/Restore.
—
You are receiving this because you authored the thread.
Reply to this email directly, view it on GitHub
<#16838 (comment)>,
or mute the thread
<https://github.com/notifications/unsubscribe-auth/AE135LTTBULeLo36Q45NPfzQtqZFjjK3ks5sxs-JgaJpZM4OLcFo>
.
--
…-- Tobias
|
I was thinking that I should rebase this (to pick up the new naming convention) and merge it as a draft. Or is it preferable to just leave it open? |
I'd be fine with merging it as a draft given that we don't expect to touch it for a few months. |
LGTM with nit Reviewed 2 of 2 files at r7. docs/RFCS/20170613_change_feeds_storage_primitive.md, line 5 at r7 (raw file):
update this field Comments from Reviewable |
TFTR! Review status: 0 of 1 files reviewed at latest revision, 15 unresolved discussions, some commit checks pending. docs/RFCS/20170613_change_feeds_storage_primitive.md, line 105 at r6 (raw file): Previously, arjunravinarayan (Arjun Narayan) wrote…
I added this to "Unresolved Questions". It seems impossible to support, though? We don't store TxnIDs with writes and it would be prohibitive to do so, so at the very least, you have hard to overcome problems when setting up the stream. docs/RFCS/20170613_change_feeds_storage_primitive.md, line 5 at r7 (raw file): Previously, knz (kena) wrote…
Done. 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 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 will be prepared by @arjunravinarayan.
NB: This isn't on the roadmap, but we thought it would be good to get the ball rolling.
cc @arjunravinarayan