-
Notifications
You must be signed in to change notification settings - Fork 1.8k
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
New producer design [beta] #132
Conversation
@wvanbergen interesting, the test failure here is exactly the same heisenbug that we've been seeing with the old producer - this quite effectively rules out a producer bug :) |
Interesting. So probably it's in the test then. I don't understand what could be wrong with it though; it's very straightforward. |
After letting it sit for a while and doing a little additional cleanup, I'm starting to feel more comfortable with this design. |
} | ||
|
||
// Errors is the output channel back to the user. If you do not read from this channel, | ||
// the Producer may deadlock. It is suggested that you send messages and read errors in a select statement. |
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.
may
-> will
since consuming those events is a pretty hard requirement:
- Mention it very clearly in the package docs; or
- Take a function as argument in the constructor which is used to send error to; or
- Do your own leaky select logic, and log errors that are dropped.
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.
Good catch, I've clarified that.
251499f
to
928e671
Compare
I am testing your new producer and it seems to work just fine. I am also unable to reproduce #150. So is it possible for your producer to send more than I will continue my testing tomorrow. |
I believe #150 is still a theoretical problem with this implementation, though it may be somewhat more difficult to trigger due to unrelated changes. |
If it's possible to trigger that issue, do you have any workaround for producer-ng? I will be incredibly thankful if you can fix that for a new producer! :) |
@wkuranowski I just pushed a fix to this branch; I didn't test it, but if my understanding is correct it should do the right thing. |
Thank you very much!!! Patch looks good. I will test it tomorrow. After reading your code I have a question. Is it possible (under some rare conditions) to build request bigger than MaxRequestSize in buildRequest() method? |
The |
Compression CompressionCodec // The type of compression to use on messages (defaults to no compression). | ||
FlushMsgCount int // The number of messages needed to trigger a flush. | ||
FlushFrequency time.Duration // If this amount of time elapses without a flush, one will be queued. | ||
FlushByteCount int // If this many bytes of messages are accumulated, a flush will be triggered. |
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 assume the message count and byte count are on a per partition basis, right? Do we want to make that explicit in the docs as well?
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.
Per-broker.
6aeb38f
to
126822c
Compare
@eapache I am producing messages to the input channel from multiples goroutines. I see that this channel is not buffered. Do you consider adding a buffer to that channel or should I stop producing from multiple goroutines? :) |
The goroutine consuming from that channel is very fast, so even producing from multiple goroutines shouldn't block much if at all. Do you have a benchmark showing this to be a problem? |
I didn't run any experiments myself, but I'd be surprised if buffering didn't have notable effect. Is it easy to get a blocking profile from some of your experiments? Alternatively do you have a testing setup here that I could play with? I'd be quite interested to sit down for a walk through although this week we have a remote coworker visiting Ottawa so I'd rather not divert my attention too much. Would sometime next week work for you? @snormore interested to join as well? |
There is a vagrantfile in this repo: you can |
Quick adhoc profiling suggests that (both with and without buffering) the cost of the channel operations is basically noise compared to the other things that we have to do (CRC calculations, writing data to the socket, optional compression, etc). |
I'd be interested in a walk-through for sure! |
Here are two blocking profiles of running your scratch app in vagrant for 1 minute. The first is with ChannelBufferSize = 0 and the second with ChannelBufferSize = 10. At first sight it seems like they are largely the same, but if you look closer there are notable differences in the amount of time the various goroutines spend blocking. I suspect the total blocking time is largely the same because the duration is fixed, but I wonder if there is a difference in the total throughput between the two cases. Might be worth adding some stats in that regard to the demo. Here are the tweaks I made to gather the profiles https://github.com/mkobetic/scratch/compare/flags.
|
The current version of My fix was simply
|
Thanks @ORBAT, I had assumed a typed nil (empty slice) would still compare as nil when put in an interface, but I guess not. Fixed now (and updated tests to check it as well). |
Also fix one bug found by the tests
Also simplify an if/else into just an if as suggested by golint.
and generate a new partitioner for each topic
they are no longer necessary
This is what the scala client defaults to
Worked through the happens-before graph a little more formally and found some (admittedly unlikely) cases where shutdown would fail/panic/hang/whatever. This version *should* be correct.
A typed nil still compares non-nil when stuffed into an interface. Thanks to Tom (ORBAT) for pointing this out.
select { | ||
case msg = <-p.retries: | ||
case p.input <- buf[0]: | ||
buf = buf[1:] |
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.
Might be better to preallocate some reasonably sized buf and do this with copy(buf, buf[1:])
, if we just reslice buf the append below is guaranteed to reallocate the buffer every time since it's appending past the end.
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.
append
grows the backing slice to some sane multiple of the current number of elements (not the textbook 2x but something like that) when it needs to realloc, so I'm not sure what the concern is - not sure how we could end up reallocating every time?
For optimal performance we could use https://github.com/eapache/queue/ but given that this isn't a common path (only being used during leader election failover) I didn't bother.
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.
Sorry, I didn't realize that append preallocates capacity. So my "reallocate every time" is clearly wrong. Shifting rather than reslicing would keep the active buffer region from creeping upwards and would probably reduce the number of reallocations still, but if we are talking infrequent reallocations then the costs of the frequent copying may well outweigh the cost of the reallocations so this definitely makes my suggestion a lot less compelling.
Why is there concurrency in around This feels like a flawed design which effect throughput. Now all routines that want to publish something are blocked and processed one by one. Shouldn't the currency at the level of broker workers? |
@eapache Thanks for the response. The |
@wvanbergen @burke @sirupsen @graemej and whoever else might be interested.
This is an idea for a different API for the producer that is somewhat more channel/goroutine based. I was thinking about alternative architectures after reviewing the PR which adds the channels returning failed/successful messages. I started playing with things (after also reviewing the original multiproducer PRs) and came up with this.
It ended up being basically a complete rewrite, so reviewing the diff won't be too helpful, just read the new
producer.go
straight. It's very much a work-in-progress, I'm more interested in architectural thoughts right now than code nits. It seems to pass the simple tests, but that's primarily accidental, shrug.Misc. notes on things that might cause questions:
select
on those two channels to avoid blocking.SimpleProducer
uses theAckSuccesses
flag to effectively imitate that.Broker
object to put on the wire. Not sure if this will impact performance noticeably, but it means that if any intermediate step blocks (i.e. a metadata request for a new leader for a topic) then all other messages not directly impacted will continue processing, which is nice (assuming you configure a large enough channel buffer).Key
/Value
elements to bytes until the very last moment. This seems cleaner, and also potentially simplifies the addition of streaming messages which somebody asked for recently.leaderDispatcher
logic for this).