-
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
Better Broker Connection Management #15
Comments
As suggested by somebody at the review this afternoon, I have filed a ticket with Kafka asking why not all brokers seem to be returned from a "get all metadata" request. |
After some digging we discovered this is due to Kafka not querying ZK for the most recent data but instead each broker uses its local "belief"? of what the cluster looks like. The reason it is not querying ZK seems to be mainly due to the following patch https://issues.apache.org/jira/browse/KAFKA-901 where there was worries about overloading ZK in the event of failures. |
I'll just paste the mail I sent to Evan: I'm going to be working on Sarama and specifically #15, so I thought I'd consult you for a bit, since you worked on it the most. I looked at the code of the scala library and the way they seem to implement managing the brokers is as follows:
To clarify a bit: There's basically two broker pools, the initial/seed pool and the pool that it actually uses and connects to. The initial pool is used for metadata requests only and the brokers that it gets from the requests are those that are used for actually producing/consuming. Of course, these two pools can overlap. The initial pool is frozen. I think copying this behavior is the way to go. |
Thanks for looking into this @bouk :-) |
Nice work @bouk |
Good things we should definitely copy:
An issue I see with this design: if the subset of brokers provided by the user goes down, it fails to get metadata even if other connections are still up. Ideally the user should only have to provide two or three brokers for bootstrap, not the entire (potentially large) cluster. And follow-up questions: are broker connections maintained always, even when not in use? Is there any logic to deal with this efficiently, or is it just assumed that the cluster size is small enough for this to be negligible in terms of file-handles and performance? Is there any logic to deal with long TCP timeouts, or could a broker in a bad state cause the entire library to block for 2+ minutes trying to get metadata? |
(Linkedin runs some very large clusters, so I would be surprised if their official client simply kept all of those connections open all of the time.) |
An issue that seems related we encountered in the wild: We started a few clients running Sarama on a lot of servers. The next day, we went back and tested them and had left them running for a day. It seemed to still think it had the connection, but for some reason it couldn't send off messages and didn't crash when trying. Any ideas why this could happen @eapache? |
The user does provide only a few bootstrap brokers, so we'd have to make sure that at least one of those is online. It then finds the relevant brokers from that set. I saw that sarama currently requests the metadata on all topics and partitions, that is not needed, the scala implementation just gets the leaders of all the partitions for the topics it needs and connects to them if needed, leaving the connection open until the metadata is refreshed, it seems to close the connection until it needs to send something to it again. So to recap: if you want to send something to a certain topic, it checks if it knows the leaders (if it doesn't it gets them) then it checks whether it is connected to the leader of the partition it wants to send to (connects if it isn't) then it finally sends. So, it maintains the minimum number of connections needed. |
I'm not sure why this could happen. The only thing I can think of is that the far end went away, and so the TCP stack is blocking trying retransmissions, but assuming the cluster is still up then even if it bounced it should send a RST immediately. If you can reproduce, take a traffic dump and see what shows up. Alternatively, it could be some peculiar deadlock in the library, I don't know how you'd debug that. |
@bouk Not quite minimal, because if I get a leader, then use it, then no longer need it that connection never gets closed? Or does it close all open connections every time it refreshes metadata? Also curious: Sarama has maintained a fairly clean distinction between the Client object doing broker management, and the Producer/Consumer objects doing actual send/receives. Does the Scala library have the same separation, or do the Producer/Consumer classes have a bit more "say" in how brokers are managed, when connections are closed etc? |
@eapache We checked the Kafka log, and we could see lines like "lost connection to [..]", where [..] was the ip of the servers the producer ran on. Have you run into any behavior that would leave Kafka killing stale connections? It seems like weird behavior to me. |
It closes all connections of a topic each time that topic is refreshed. It refreshes topic by topic, or multiple at once if you send multiple messages at once (which sarama doesn't support yet I think). It seems the scala library does something else for producers and consumers, I'd have to dig a bit deeper into how Kafka works to understand how the consumer manages the connections. The producer does as I described though |
@sirupsen Don't know why Kafka would kill stale connections. TCP doesn't do keep-alives by default (but it can), so I guess it's possible that's one of their undocumented features is to kill apparently stale connections. Sarama should receive a RST and be able to safely reconnect in that case though. |
@bouk If I have two producers on different topics that happen to go to the same broker, does that end up as two connections in the Scala client then? One of the benefits of Sarama's current design is that the two described producers will share a connection. However, this means that if one of them closes all its connections to refresh metadata, it would drop the connection out from under the other producer. |
@sirupsen Kafka config page mentions "producer.purgatory.purge.interval.requests" and "fetch.purgatory.purge.interval.requests" so maybe it does something weird to handle misbehaving clients? There isn't a lot of detail on the documentation page. |
It does keep one connection per broker. Bouke On Tue, Sep 17, 2013 at 3:47 PM, Evan Huus [email protected] wrote:
|
The consumers in the scala library are zookeeper based, so they do something else entirely |
I implemented some of the changes bouk@0016f3d Note that a metadata request currently just stops when it can't connect to any of the seed brokers, I think that should be replaced by a 'back off' and retry just like when a leader is being elected. That would require an API change because it would need a new config for the client which says how long it should back off (or should we just make backing for election and waiting for a seed broker the same?) |
The great thing about go is that a new config doesn't count as a breaking API change since unspecified fields in a struct literal default to zero. However, with the current changes I feel like we're just exchanging one set of corner-case issues for another. I don't see a lot of actual benefit I'm afraid. |
@eapache what do you mean by "I feel like we're just exchanging one set of corner-case issues for another."? I feel like copying the 'reference' behavior is probably the best strategy. Note that my code currently hangs on the tests, this is because they expect that the client opens a connection to every broker it finds, which it does not |
@bouk The current sarama design has a set of cases where it doesn't behave well. The scala client's design has a different set of cases where it doesn't behave well. We can certainly mine the scala client for ideas, but blindly copying their design just means we're exchanging the issues with our design for the issues with their design. If we're going to do that, we need some sort of argument for why their issues are somehow "better" than the ones we've already got, otherwise it's just code churn for no reason. Ideally we can look at both versions and design something new that behaves well in all of the cases we've come up with, though that is obviously more work. I suspect such a design would involve explicitly reference-counting the brokers. |
@eapache Which issues do you think there are with the official approach? |
@sirupsen I've listed several already in previous comments: |
Sure, you could make it check the other known brokers but I think it's best to just go with the suggested behavior.
Yep that's a valid issue and I'm sure the scala lib doesn't just drop it, I'll have to consult it some more |
Why? What benefit in behaviour or complexity or speed or ... does it provide? Also I don't really see the scala lib as representing "suggested" behaviour. It's a reference implementation, sure, but the protocol supports half a dozen tricks it doesn't even make use of, so I want to stay away from just blindly copying it. |
The scala lib makes sure that a broker doesn't get closed while messages are being sent with a lock, just like sarama does already so that's OK |
This is a partial workaround for #15. Really, broker management should be completely rearchitected, but today is not the day for that.
This issue is rather stale, and a lot has changed, so here's an update: Several of the issues raised were fixed long ago. The following are no longer problems:
Several of the issues are low-priority enhancements that should probably be split into their own tickets:
The only remaining non-trivial design question that is still open here is regarding how we bounce broker connections, and I went into more depth on that in the following comment: #23 (comment) I'm leaving this ticket open to track that specific issue (specifically: the semantics of |
Lazy broker connections have been implemented, as has the sane bouncing of broker connections. |
Following up to #7, #9, #10, #13 our connection management still has problems.
First some things to know about Kafka that complicate the issue:
Now some of the issues with our current implementation:
Thoughts on potential future design:
leader
function shouldn't guarantee that the broker it returns is connected, only connecting.any
function should guarantee that the broker it returns is actually connected. Walking through bad brokers to find a good one could be slow, but connecting to all the brokers just to pick one and drop the others is gross. We could try to connect to one, and if it hasn't succeed in 5 seconds, start another, etc. That should cover the common case, since if it hasn't responded after 5 seconds it probably won't, and if it does then we'll drop it right away, oh well. More importantly, we won't spawn hundreds of connections if the first broker does respond normally.Open Questions:
any
, when should it be disconnected? TheextraBroker
field in client currently is a hack to work around this, and it still leaves one connection floating around for now reason. PresumablydisconnectBroker
could be used and made smarter? However, if a metadata-supplied broker is used, callingdisconnectBroker
will pull it out from underneath other code that could be using it as a leader. Do we need reference-counting?leader
calls would return the same broker and both callOpen
, the second one will block when we don't want it to. Do we need a second lock in the brokers? That gets really messy...The text was updated successfully, but these errors were encountered: