-
Notifications
You must be signed in to change notification settings - Fork 3.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
gossip: connectivity logging is extremely spammy in 128 node cluster #30088
Comments
I think we can remove the every minute logging of gossip connectivity. The other logging only occurs when gossip connectivity changes. It shouldn't be changing frequently and we should figure out why it is. |
Yeah, I'll check out what's going on with gossip after the tpc-c runs are done, or more likely I'll just spin up a cluster using smaller nodes later on to focus on the gossip in a more cost effective way. However, there will always be a lot of logging triggered every time one or more nodes get restarted as the gossip network adjusts to the change(s) and stabilizes, which will certainly add up during an event like a rolling upgrade. |
On a 3 node cluster we gossip at ~2KB/sec. On a 9 node cluster we gossip at ~12KB/sec. On a 27 node cluster we gossip at ~55KB/sec. None of these numbers scale up to 37GB gossiped over 2 hours. I think there is something to investigate there. |
We both gossip the cluster-id as well as include it in every |
On a 3 node cluster, over the course of 1 minute we send/receive the following keys via gossip:
The second column is the number of times the key was sent/received. The third column is the size of the gossiped value times the number of times it was gossiped (providing an indication of what fraction of the gossiped bytes it is consuming). Note that the sizes above are only measuring |
We need to gossip the sentinel frequently, but there is no need to be gossiping the cluster-id and first-range so fast. |
The numbers I posted above were the sent/received gossip infos from a single node. The math inclined will notice that we sent/received 138 gossip-clients:1 keys. I looked into this a bit more and it turns out we sent 138 keys during that minute time frame. But something is off here because we are only supposed to send that key every 2 seconds so we should have gossiped it 60 times. A bit more instrumentation shows that every time |
I'm pretty sure the problem is due to how we update |
The previous handling of high water stamps left a gap during which an info could be resent over a connection multiple times. The general gossip send loop is to wait for a new info to be added to the local info store, to compute a delta between the local info store and the remote info store using the remote's high water stamps and to update the remote's high water stamps when the remote sends a reply on the connection. The problem here is that updating the remote's high water stamps when the remote sends a reply leaves a window where an info added to the local store would be sent multiple times (every time an additional info was added) until the remote replied. The fix is to keep track of a per connection high water stamp map and ratchet the contained stamps both when we receive a reply from the remote and when we send the remote an info. The result is a 30-40% reduction in gossip traffic (bytes sent/received) as measured on idle 3, 9 and 27 node clusters. See cockroachdb#30088 Release note: None
The previous handling of high water stamps left a gap during which an info could be resent over a connection multiple times. The general gossip send loop is to wait for a new info to be added to the local info store, to compute a delta between the local info store and the remote info store using the remote's high water stamps and to update the remote's high water stamps when the remote sends a reply on the connection. The problem here is that updating the remote's high water stamps when the remote sends a reply leaves a window where an info added to the local store would be sent multiple times (every time an additional info was added) until the remote replied. The fix is to keep track of a per connection high water stamp map and ratchet the contained stamps both when we receive a reply from the remote and when we send the remote an info. The result is a 30-40% reduction in gossip traffic (bytes sent/received) as measured on idle 3, 9 and 27 node clusters. See cockroachdb#30088 Release note: None
See #30126. I think that is an explanation for the gigabytes of gossip traffic. |
30106: gossip: remove unnecessary gossip traffic r=spencerkimball,a-robinson,bdarnell a=petermattis The previous handling of high water stamps left a gap during which an info could be resent over a connection multiple times. The general gossip send loop is to wait for a new info to be added to the local info store, to compute a delta between the local info store and the remote info store using the remote's high water stamps and to update the remote's high water stamps when the remote sends a reply on the connection. The problem here is that updating the remote's high water stamps when the remote sends a reply leaves a window where an info added to the local store would be sent multiple times (every time an additional info was added) until the remote replied. The fix is to keep track of a per connection high water stamp map and ratchet the contained stamps both when we receive a reply from the remote and when we send the remote an info. The result is a 30-40% reduction in gossip traffic (bytes sent/received) as measured on idle 3, 9 and 27 node clusters. See #30088 Release note: None Co-authored-by: Peter Mattis <[email protected]>
Make gossip.Request.{Addr,ClusterID} nullable and only send them on the first request over a connection. Remove gossip.Response.Addr which was unused. Only gossip the cluster ID once per server. The cluster ID never expires, so there was no need to be re-gossiping it frequently. Keep track of which high water stamps have been sent to the remote side of a connection and only send stamps that are newer than those already sent. This has a dramatic impact on gossip traffic as a cluster gets larger as usually only 1 or 2 stamps have changed on each gossip message which results in reducing an O(n) map size to a constant. In a local 27-node cluster, these changes reduce gossip traffic by ~60%. Fixes cockroachdb#30126 See cockroachdb#30088 Release note: None
The previous handling of high water stamps left a gap during which an info could be resent over a connection multiple times. The general gossip send loop is to wait for a new info to be added to the local info store, to compute a delta between the local info store and the remote info store using the remote's high water stamps and to update the remote's high water stamps when the remote sends a reply on the connection. The problem here is that updating the remote's high water stamps when the remote sends a reply leaves a window where an info added to the local store would be sent multiple times (every time an additional info was added) until the remote replied. The fix is to keep track of a per connection high water stamp map and ratchet the contained stamps both when we receive a reply from the remote and when we send the remote an info. The result is a 30-40% reduction in gossip traffic (bytes sent/received) as measured on idle 3, 9 and 27 node clusters. See cockroachdb#30088 Release note: None
Make gossip.Request.{Addr,ClusterID} nullable and only send them on the first request over a connection. Remove gossip.Response.Addr which was unused. Only gossip the cluster ID once per server. The cluster ID never expires, so there was no need to be re-gossiping it frequently. Keep track of which high water stamps have been sent to the remote side of a connection and only send stamps that are newer than those already sent. This has a dramatic impact on gossip traffic as a cluster gets larger as usually only 1 or 2 stamps have changed on each gossip message which results in reducing an O(n) map size to a constant. In a local 27-node cluster, these changes reduce gossip traffic by ~60%. Fixes cockroachdb#30126 See cockroachdb#30088 Release note: None
The previous handling of high water stamps left a gap during which an info could be resent over a connection multiple times. The general gossip send loop is to wait for a new info to be added to the local info store, to compute a delta between the local info store and the remote info store using the remote's high water stamps and to update the remote's high water stamps when the remote sends a reply on the connection. The problem here is that updating the remote's high water stamps when the remote sends a reply leaves a window where an info added to the local store would be sent multiple times (every time an additional info was added) until the remote replied. The fix is to keep track of a per connection high water stamp map and ratchet the contained stamps both when we receive a reply from the remote and when we send the remote an info. The result is a 30-40% reduction in gossip traffic (bytes sent/received) as measured on idle 3, 9 and 27 node clusters. See cockroachdb#30088 Release note: None
As discussed on cockroachdb#30088. It can be absolutely massive on large clusters, and isn't needed because we still log the connectivity info whenever it changes. Release note: None
30035: builder: upgrade to go1.11 r=bdarnell,raduberinde,nvanbenschoten a=benesch Go 1.11 brings us some nice performance improvements on single-node clusters. See #29827 for benchmark results. Fix #29827. Release note: None 30285: gossip: Don't include gossip connectivity in periodic status logs r=a-robinson a=a-robinson As discussed on #30088. It can be absolutely massive on large clusters, and isn't needed because we still log the connectivity info whenever it changes. Release note: None I still think that logging on every change in connectivity is going to be a mess in large clusters during rolling restarts even if gossip is otherwise perfectly stable, but this is useful regardless of whether we do anything about that. Co-authored-by: Nikhil Benesch <[email protected]> Co-authored-by: Alex Robinson <[email protected]>
As discussed on cockroachdb#30088. It can be absolutely massive on large clusters, and isn't needed because we still log the connectivity info whenever it changes. Release note: None
@a-robinson Do you think there is anything else to address here for 2.1? My understanding is that gossip stabilized on your 100+ node cluster and this spam just affected startup of the cluster. |
Unfortunately, that doesn't seem to be the case. I saved the latest log file from the cluster before taking it down, and all the log files have a lot of the connectivity logs even though no nodes were being added/removed around that time. Two nodes were down due to the bad_alloc issue, but they had been down for well over an hour before the time period these logs cover. Let me know if you'd like to see them for yourself. I imagine the Also, orthogonally from the thrashing, the logging during startup is excessive. We log every time we get new connectivity info that isn't identical to the last thing we logged, but during startup a node can get gossip info piece by piece, leading to log output like the below that keeps going on and on until all the nodes' info is received:
|
I've been tossing around the idea of rate limiting this logging to happen no more than once a minute, which would alleviate most of the pain. Would that still accomplish most of the original goal? I realize we've discovered some gossip problems lately, but I'm not really convinced that this information belongs in non-verbose logs. |
It's true that the gossip connectivity logs might be addressing yesterday's war. Perhaps we should re-add them to the once per minute logging, but only output the message if it has changed since the last log line. |
Fixes cockroachdb#30088 This will still probably result in logging it once a minute on large clusters due to the inherent instability caused by connection culling, but once a minute is a lot more bearable than the current state of things. Release note: None
30730: gossip: Log connectivity no more than once a minute r=a-robinson a=a-robinson Fixes #30088 This will still probably result in logging it once a minute on large clusters due to the inherent instability caused by connection culling, but once a minute is a lot more bearable than the current state of things. Release note: None Co-authored-by: Alex Robinson <[email protected]>
Fixes cockroachdb#30088 This will still probably result in logging it once a minute on large clusters due to the inherent instability caused by connection culling, but once a minute is a lot more bearable than the current state of things. Release note: None
Make gossip.Request.{Addr,ClusterID} nullable and only send them on the first request over a connection. Remove gossip.Response.Addr which was unused. Only gossip the cluster ID once per server. The cluster ID never expires, so there was no need to be re-gossiping it frequently. Keep track of which high water stamps have been sent to the remote side of a connection and only send stamps that are newer than those already sent. This has a dramatic impact on gossip traffic as a cluster gets larger as usually only 1 or 2 stamps have changed on each gossip message which results in reducing an O(n) map size to a constant. In a local 27-node cluster, these changes reduce gossip traffic by ~60%. Fixes cockroachdb#30126 See cockroachdb#30088 Release note: None
I spun up a 128 node cluster for tpc-c 50k testing (it was meant to be / will soon be 120, but my brain accidentally rounded up when creating it), and the gossip connectivity logs are overwhelming. First of all, they're huge, taking up almost my entire screen with just one log line:
Secondly, they're being logged a few times per second, making it impossible to read anything else out of the logs.
The second point is probably indicative of some amount of instability in the gossip network that should be investigated on its own right. However, even if it was only being logged once a minute as part of the gossip status, this would be quite annoying. The entire "gossip status" message literally doesn't fit on the screen of my 15" macbook:
Also, wow that's a lot of data being transmitted over gossip (the server indicates 25GB sent / 12GB received). I suspect a lot of it is the connectivity map being resent in its entirety over and over.
It'd be reasonable to say we should first understand/fix the fact that there are frequent changes happening in the gossip network. I'm still worried about how things will look after that, though.
@petermattis
The text was updated successfully, but these errors were encountered: