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
Sinks should accept differential batches #2808
Comments
So I took a first whack at this and ran into issues that will need a bit of a rethink. Summarizing my thoughts and questions for future context. Currently, we handle the problem of sinks writing data "exactly once" per updates wrt restarts by writing to a new instance of the storage (kafka topic / file / etc) every time we restart Materialize. This leads us to a fairly reasonable expectation that two kafka topics that were written to for the same sink should present updates in a consistent, deterministic order. The product thesis is that it would be really weird from a user point of view if Materialize presented a change log of updates at some time T, and then another new log at time T' after a restart and the original log had sequences of updates that were not present in the new log. I think its worth it to step back and ask:
Our strategy to try to achieve this was to use a TODO for me -- lets test this out empirically and try to see how nondeterministic this can get because we may have to set expectations with prospects using sinks sooner rather than later A little bit orthogonally, in this issue we want to have sinks receive and write down structured batches of data rather than streams of records because we can use the batches we already have to prevent doing additional computation to convert them into a stream and batches have nice properties that make it easier to move to a world where Materialize can track how far it wrote to a sink and not have to rewrite from scratch to brand new storage (kafka topic / file / etc) after every restart. Unfortunately, having sinks receive and write down structured batches is incompatible with our current approach of having all threads send their data to one writer because our batches are only currently safe for single threaded access. There's a couple of different engineering ways we could think about solving this, but I think before that we need to answer two product questions
I hope this is a clear summary please tell me if that's not the case |
Btw, I think we can get deterministic output at one worker, but with a more sophisticated sink operator. It would need to stash updates and consult progress information to know when it had received all the updates. If we want to do that we can, but it would be some new code to write. |
The implementation that I'm thinking of to do that looks pretty close to timely's |
Just to check, the thing you are "thinking to do" is a way to get deterministic output rather than a way to accept differential batches, is that right? If so, that should be fine for now, but it will be a bit off of peak performance. It means logic for each distinct timestamp, rather than retiring batches at a time, which was one of the steps differential made up from timely. I think at the moment it is not a big deal, but our sources (and perhaps soon our sinks) are the only operators that have these limitations. Imo the right answer is still parallel batch writing across workers. If the single-worker thing is here just to get determinism, we should feel free to relax that and just get determinism with multiple workers instead (perhaps the single-worker thing is here to make something else easy, like set-ups where we aren't allowed to use partitioned topics, maybe?). |
Yes that's right. Re: parallel writing and determinism I think there's 2 product questions:
Our actual state is 1.c 2.c but we aspire to 1.a 2.c at the moment. My understanding is that to get the performance benefits of writing down batches (not having to flatten data out to a collection) -- we need each thread to write independently, and to do that in a deterministic way we need 2.a? On the other hand, I think if we don't care about determinism then we can achieve parallel batch writing within a single partition topic or a multipartition topic? I'm not sure if this is an accurate description of the design space. |
Also Frank just so I'm clear, when you say "it will be a bit off from peak performance" do you mean that "it will be slightly off from peak performance" or do you mean "it will be far off from peak performance"? I read it as the latter but wanted to double check |
It would be far off in differential where we can do 1Mhz updates. I think we are running about 100hz at the moment in Materialized, so no problems at the moment. |
@nacrooks brought up a point that I hadn't considered -- if we write to multiple partitions its very hard (not quite impossible but afaik no one does this) to read from the partitions in a deterministic way. so even if we try really hard to write things down deterministically, I'm not sure there will be any benefit |
Strawman proposal: we stay at nondeterministic writes until a client asks us for deterministic writes, and we stay at a single partition kafka topic for now and in parallel we see if we can make writing down batches work w/ multiple threads writing to a single partition kafka topic in a nondeterministic, but still correct w/ respect to timestamps, way (I'm not entirely sure if thats possible) |
If it's going to be nondeterministic, we might as well parallelize by
creating a Kafka topic with n workers and allowing every worker to do its
own writes, no?
…On Thu, Apr 30, 2020 at 5:28 PM Ruchir Khaitan ***@***.***> wrote:
Strawman proposal: we stay at nondeterministic writes until a client asks
us for deterministic writes, and we stay at a single partition kafka topic
for now and in parallel we see if we can make writing down batches work w/
multiple threads writing to a single partition kafka topic in a
nondeterministic, but still correct w/ respect to timestamps, way (I'm not
entirely sure if thats possible)
—
You are receiving this because you were mentioned.
Reply to this email directly, view it on GitHub
<#2808 (comment)>,
or unsubscribe
<https://github.com/notifications/unsubscribe-auth/AAGXSIAIW3DLYLG36OJ56TLRPHUOHANCNFSM4MTB4IJA>
.
|
Just returning to this with some thoughts: I do think we'll want at least one code path that is "write to as many parts as there are workers", as this allows a more efficient, batch-driven implementation (one that avoids at least one copy of the data to sink it, and has fewer back-pressure issues as it reads out of shared batches at its own speed).
We do want our output to be semantically deterministic, but I'm not too bothered if it is non-deterministically ordered, as long as that order doesn't affect the semantics. We might want to make it ordered so that folks can pick out duplicates more easily (e.g., ordered by (timestamp, data)). I'm not sure we gain too much from non-determinism here other than a simpler sink operator (one that writes data as it arrives, rather than buffering it). If that's right, I think deterministic would be great.
I'm personally a fan of a |
Our sinks compact data that comes at them, which means that they have access to batch structured data. In addition, when we sink materialized inputs we already have the batch structure. Sinks currently ask for
Collection
structured data, which requires flattening batches down to owned records, and can represent a substantial resource requirement for the sink.We should re-write sinks so that they accept a stream of differential batches, rather than a
Collection
. The sink then needs to iterate through the contents of each batch, rather than through update tuples. The intended result is a reduction in memory use when batches already exist, and a reduction in latency and total work performed when sinking data from an existing arrangement.cc @ruchirK
The text was updated successfully, but these errors were encountered: