-
Notifications
You must be signed in to change notification settings - Fork 2.1k
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
Throttler: Use tmclient pool for CheckThrottler tabletmanager RPC #14979
Throttler: Use tmclient pool for CheckThrottler tabletmanager RPC #14979
Conversation
Review ChecklistHello reviewers! 👋 Please follow this checklist when reviewing this Pull Request. General
Tests
Documentation
New flags
If a workflow is added or modified:
Backward compatibility
|
5a3c2dc
to
067280a
Compare
Signed-off-by: Matt Lord <[email protected]>
067280a
to
a50e066
Compare
Codecov ReportAttention:
Additional details and impacted files@@ Coverage Diff @@
## main #14979 +/- ##
=======================================
Coverage 47.27% 47.27%
=======================================
Files 1138 1138
Lines 238842 238851 +9
=======================================
+ Hits 112907 112919 +12
+ Misses 117342 117337 -5
- Partials 8593 8595 +2 ☔ View full report in Codecov by Sentry. |
Signed-off-by: Matt Lord <[email protected]>
e947844
to
8a251a6
Compare
} | ||
defer closer.Close() | ||
|
||
if c == nil { |
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.
What are the cases where we don’t have a pooled dialer here?
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.
It's not that we don't have a pooled dialer, it's that dialPool returns a nil TabletManagerClient.client. I would not expect it, from reading the function, but it's a fail-safe to prevent RPC failures in unexpected scenarios.
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.
@mattlord but wouldn’t there have been an error then? Can the pooled dialer return nil but also not return an error?
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.
It would be unexpected, yes. It's a pointer that anyone could set to nil at any time though as it's a shared object. From looking at the code it (TabletManagerClient) could also exist in the map but the member (client) be nil as we don't check for that there.
What's so concerning about the failsafe here? I could add a comment that it should not generally happen.
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.
It’s that it seems very unexpected. In general on Go, you don’t nil check the return value if you have an error to indicate if it went right or wrong. If we’d do that everywhere, we would have very noisy code.
So if the fallback can’t happen, we should imho remove it since it’s useless code. It also for example means it can never be properly tested or covered either if it can never happen.
It’s also that the code being like this caused me to ask these questions which I think is a signal in itself too that it’s confusing 😃.
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.
@ajm188 what do you think? I see that this pattern came from this PR: #8368
I can only guess that the idea was we don't want to consider a pooled nil client value as an RPC level error here, but instead create a new dedicated tmclient and gRPC connection when needed for any reason so that the RPC should always succeed even when there's something odd going on with the tmclient pool? I wonder then if we shouldn't at least do this at the end of the dialPool function so that this nil value isn't returned yet again in the future:
@@ -164,7 +164,7 @@ func (client *grpcClient) dialPool(ctx context.Context, tablet *topodatapb.Table
client.rpcClientMap = make(map[string]chan *tmc)
}
c, ok := client.rpcClientMap[addr]
- if !ok {
+ if !ok || len(c) == 0 {
c = make(chan *tmc, concurrency)
client.rpcClientMap[addr] = c
client.mu.Unlock()
@@ -183,8 +183,12 @@ func (client *grpcClient) dialPool(ctx context.Context, tablet *topodatapb.Table
client.mu.Unlock()
}
+ // Get the least recently used/created cached client.
result := <-c
- c <- result
+ // If it's not nil, place it back at the end of the queue for future re-use.
+ if result != nil {
+ c <- result
+ }
return result.client, nil
}
I'm OK doing a minor refactor here, although it's unrelated to the issue this PR is meant to address.
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.
@mattlord yeah, I think we can tackle this separately for sure.
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.
@mattlord cached_client and client are two different paradigms (you linked a PR that added the cached client whereas this pr changes client), so i'm not sure i understand your question.
as to the difference, as i understood them at the time of #8368:
client
with normal (non-pooled) dialer opens a new gRPC connection for each RPC, and tears it down (along with the TCP overhead, etc.) at the end of each and every RPCclient
with a pooled dialer, on first dial to a given host:port, opens N connections to that host, shoves them onto a channel, and cycles through those conns one at a time for each RPC. no connections are closed, ever, until someone callsClose
on the TMC, at which point all connections are closed seriallycached_client
is slightly different from (2), in that it dials at most 1 connection per host:port, and the most recent N connections stay alive. if an N+1th host gets dialed, we close the oldest connection by use-time, and add the new connection to the cache
hopefully that helps clarify both this thread and @shlomi-noach's above
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.
To summarize further: this PR is not using (as I would describe it) the cached client. It is updating the plain (non-cached) client to use a pool of connections for the CheckThrottler RPC (which I agree is a good idea).
Separately, I'd like us to consider promoting the cached-client implementation to be the default over the non-cached (or, "oneshot") implementation.
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.
Thanks, @ajm188 ! For the cached client part, I meant that more generally — the tmclients that are cached in the buffered channel. But I agree that it’s not very precise/clear since we also have the cachedConn
implementation in cached_client.go
. I updated the wording in the PR to go from “cached client” to “tmclient pool” to be more clear. The linked PR #8368 is where you added this if c == nil {
pattern: https://github.com/vitessio/vitess/pull/8368/files#diff-c0aa163f88048bad1cd140049af3372d8f396784b4c93709fa269f82597a42bbR496-R500 which was being discussed/debated here.
c, err = poolDialer.dialPool(ctx, tablet) | ||
if err != nil { | ||
return nil, err | ||
} |
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 can see the same pattern is used in ExecuteFetchAsDba
and in ExecuteFetchAsApp
- so I'm assuming this is correct -- but still unsure -- I'm not sure I understand how this gets a pooled connections; and I don't see the pattern of "get from pool ; defer return to pool", so this does not read like a way of using a cached connection. Maybe ExecuteFetchAsApp
's implementation is likewise incorrect?
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.
You can see the implementation within this file:
vitess/go/vt/vttablet/grpctmclient/client.go
Lines 155 to 202 in a207a69
func (client *grpcClient) dialPool(ctx context.Context, tablet *topodatapb.Tablet) (tabletmanagerservicepb.TabletManagerClient, error) { | |
addr := netutil.JoinHostPort(tablet.Hostname, int32(tablet.PortMap["grpc"])) | |
opt, err := grpcclient.SecureDialOption(cert, key, ca, crl, name) | |
if err != nil { | |
return nil, err | |
} | |
client.mu.Lock() | |
if client.rpcClientMap == nil { | |
client.rpcClientMap = make(map[string]chan *tmc) | |
} | |
c, ok := client.rpcClientMap[addr] | |
if !ok { | |
c = make(chan *tmc, concurrency) | |
client.rpcClientMap[addr] = c | |
client.mu.Unlock() | |
for i := 0; i < cap(c); i++ { | |
cc, err := grpcclient.Dial(addr, grpcclient.FailFast(false), opt) | |
if err != nil { | |
return nil, err | |
} | |
c <- &tmc{ | |
cc: cc, | |
client: tabletmanagerservicepb.NewTabletManagerClient(cc), | |
} | |
} | |
} else { | |
client.mu.Unlock() | |
} | |
result := <-c | |
c <- result | |
return result.client, nil | |
} | |
// Close is part of the tmclient.TabletManagerClient interface. | |
func (client *grpcClient) Close() { | |
client.mu.Lock() | |
defer client.mu.Unlock() | |
for _, c := range client.rpcClientMap { | |
close(c) | |
for ch := range c { | |
ch.cc.Close() | |
} | |
} | |
client.rpcClientMap = nil | |
} |
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.
vs dial that was previously being used which creates a new tabletmanagerclient with a new gRPC connection each time:
vitess/go/vt/vttablet/grpctmclient/client.go
Lines 140 to 153 in a207a69
// dial returns a client to use | |
func (client *grpcClient) dial(ctx context.Context, tablet *topodatapb.Tablet) (tabletmanagerservicepb.TabletManagerClient, io.Closer, error) { | |
addr := netutil.JoinHostPort(tablet.Hostname, int32(tablet.PortMap["grpc"])) | |
opt, err := grpcclient.SecureDialOption(cert, key, ca, crl, name) | |
if err != nil { | |
return nil, nil, err | |
} | |
cc, err := grpcclient.Dial(addr, grpcclient.FailFast(false), opt) | |
if err != nil { | |
return nil, nil, err | |
} | |
return tabletmanagerservicepb.NewTabletManagerClient(cc), cc, nil | |
} |
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.
Thank you for finding and fixing this
@shlomi-noach since we switched from http to gRPC in v18, should this be back ported and included in a future v18 patch release? |
@deepthi good idea. I'll open a backport PR. |
…tessio#14979) Signed-off-by: Matt Lord <[email protected]>
|
Description
When the tablet throttler is enabled in a keyspace, the tablets within each shard make very frequent
CheckThrottler
RPC calls between themselves after moving from http to gRPC in #13514.The initial implementation created a new gRPC connection and dialed the other tablet on each
CheckThrottler
RPC call. Because this RPC is made so frequently, however, this was not practical from a performance perspective (CPU and network overhead along with feature/input latency).In this PR we instead leverage the existing tabletmanagerclient pooling, each tabletmanagerclient having its own gRPC connection, so that we re-use existing connections and avoid the overhead of constantly creating and destroying them on each RPC which caused a lot of tcp connection churn and related overhead.
Related Issue(s)
Checklist