-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
connector.go
694 lines (633 loc) · 22 KB
/
connector.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
// Copyright 2020 The Cockroach Authors.
//
// Licensed as a CockroachDB Enterprise file under the Cockroach Community
// License (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt
// Package kvtenantccl provides utilities required by SQL-only tenant processes
// in order to interact with the key-value layer.
package kvtenantccl
import (
"context"
"io"
"math/rand"
"sort"
"time"
"github.com/cockroachdb/cockroach/pkg/config"
"github.com/cockroachdb/cockroach/pkg/config/zonepb"
"github.com/cockroachdb/cockroach/pkg/gossip"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvtenant"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangecache"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/rpc"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/spanconfig"
"github.com/cockroachdb/cockroach/pkg/util/contextutil"
"github.com/cockroachdb/cockroach/pkg/util/grpcutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/retry"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/syncutil/singleflight"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/errors/errorspb"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
)
func init() {
kvtenant.Factory = connectorFactory{}
}
// Connector mediates the communication of cluster-wide state to sandboxed
// SQL-only tenant processes through a restricted interface.
//
// A Connector is instantiated inside a tenant's SQL process and is seeded with
// a set of one or more network addresses that reference existing KV nodes in
// the host cluster (or a load-balancer which fans out to some/all KV nodes). On
// startup, it establishes contact with one of these nodes to learn about the
// topology of the cluster and bootstrap the rest of SQL <-> KV network
// communication.
//
// The Connector communicates with the host cluster through the roachpb.Internal
// API.
//
// See below for the Connector's roles.
type Connector struct {
log.AmbientContext
tenantID roachpb.TenantID
rpcContext *rpc.Context
rpcRetryOptions retry.Options
rpcDialTimeout time.Duration // for testing
rpcDial singleflight.Group
defaultZoneCfg *zonepb.ZoneConfig
addrs []string
mu struct {
syncutil.RWMutex
client *client
nodeDescs map[roachpb.NodeID]*roachpb.NodeDescriptor
systemConfig *config.SystemConfig
systemConfigChannels map[chan<- struct{}]struct{}
}
settingsMu struct {
syncutil.Mutex
allTenantOverrides map[string]settings.EncodedValue
specificOverrides map[string]settings.EncodedValue
// notifyCh receives an event when there are changes to overrides.
notifyCh chan struct{}
}
}
// client represents an RPC client that proxies to a KV instance.
type client struct {
roachpb.InternalClient
serverpb.StatusClient
}
// Connector is capable of providing information on each of the KV nodes in the
// cluster in the form of NodeDescriptors. This obviates the need for SQL-only
// tenant processes to join the cluster-wide gossip network.
var _ kvcoord.NodeDescStore = (*Connector)(nil)
// Connector is capable of providing Range addressing information in the form of
// RangeDescriptors through delegated RangeLookup requests. This is necessary
// because SQL-only tenants are restricted from reading Range Metadata keys
// directly. Instead, the RangeLookup requests are proxied through existing KV
// nodes while being subject to additional validation (e.g. is the Range being
// requested owned by the requesting tenant?).
var _ rangecache.RangeDescriptorDB = (*Connector)(nil)
// Connector is capable of providing a filtered view of the SystemConfig
// containing only information applicable to secondary tenants. This obviates
// the need for SQL-only tenant processes to join the cluster-wide gossip
// network.
var _ config.SystemConfigProvider = (*Connector)(nil)
// Connector is capable of finding debug information about the current
// tenant within the cluster. This is necessary for things such as
// debug zip and range reports.
var _ serverpb.TenantStatusServer = (*Connector)(nil)
// Connector is capable of accessing span configurations for secondary tenants.
var _ spanconfig.KVAccessor = (*Connector)(nil)
// Reporter is capable of generating span configuration conformance reports for
// secondary tenants.
var _ spanconfig.Reporter = (*Connector)(nil)
// NewConnector creates a new Connector.
// NOTE: Calling Start will set cfg.RPCContext.ClusterID.
func NewConnector(cfg kvtenant.ConnectorConfig, addrs []string) *Connector {
cfg.AmbientCtx.AddLogTag("tenant-connector", nil)
if cfg.TenantID.IsSystem() {
panic("TenantID not set")
}
c := &Connector{
tenantID: cfg.TenantID,
AmbientContext: cfg.AmbientCtx,
rpcContext: cfg.RPCContext,
rpcRetryOptions: cfg.RPCRetryOptions,
defaultZoneCfg: cfg.DefaultZoneConfig,
addrs: addrs,
}
c.mu.nodeDescs = make(map[roachpb.NodeID]*roachpb.NodeDescriptor)
c.mu.systemConfigChannels = make(map[chan<- struct{}]struct{})
c.settingsMu.allTenantOverrides = make(map[string]settings.EncodedValue)
c.settingsMu.specificOverrides = make(map[string]settings.EncodedValue)
return c
}
// connectorFactory implements kvtenant.ConnectorFactory.
type connectorFactory struct{}
func (connectorFactory) NewConnector(
cfg kvtenant.ConnectorConfig, addrs []string,
) (kvtenant.Connector, error) {
return NewConnector(cfg, addrs), nil
}
// Start launches the connector's worker thread and waits for it to successfully
// connect to a KV node. Start returns once the connector has determined the
// cluster's ID and set Connector.rpcContext.ClusterID.
func (c *Connector) Start(ctx context.Context) error {
gossipStartupCh := make(chan struct{})
settingsStartupCh := make(chan struct{})
bgCtx := c.AnnotateCtx(context.Background())
if err := c.rpcContext.Stopper.RunAsyncTask(bgCtx, "connector-gossip", func(ctx context.Context) {
ctx = c.AnnotateCtx(ctx)
ctx, cancel := c.rpcContext.Stopper.WithCancelOnQuiesce(ctx)
defer cancel()
c.runGossipSubscription(ctx, gossipStartupCh)
}); err != nil {
return err
}
if err := c.rpcContext.Stopper.RunAsyncTask(bgCtx, "connector-settings", func(ctx context.Context) {
ctx = c.AnnotateCtx(ctx)
ctx, cancel := c.rpcContext.Stopper.WithCancelOnQuiesce(ctx)
defer cancel()
c.runTenantSettingsSubscription(ctx, settingsStartupCh)
}); err != nil {
return err
}
// Block until we receive the first GossipSubscription event and the initial
// setting overrides.
for gossipStartupCh != nil || settingsStartupCh != nil {
select {
case <-gossipStartupCh:
log.Infof(ctx, "kv connector gossip subscription started")
gossipStartupCh = nil
case <-settingsStartupCh:
log.Infof(ctx, "kv connector tenant settings started")
settingsStartupCh = nil
case <-ctx.Done():
return ctx.Err()
}
}
return nil
}
// runGossipSubscription listens for gossip subscription events. It closes the
// given channel once the ClusterID gossip key has been handled.
// Exits when the context is done.
func (c *Connector) runGossipSubscription(ctx context.Context, startupCh chan struct{}) {
for ctx.Err() == nil {
client, err := c.getClient(ctx)
if err != nil {
continue
}
stream, err := client.GossipSubscription(ctx, &roachpb.GossipSubscriptionRequest{
Patterns: gossipSubsPatterns,
})
if err != nil {
log.Warningf(ctx, "error issuing GossipSubscription RPC: %v", err)
c.tryForgetClient(ctx, client)
continue
}
for {
e, err := stream.Recv()
if err != nil {
if err == io.EOF {
break
}
// Soft RPC error. Drop client and retry.
log.Warningf(ctx, "error consuming GossipSubscription RPC: %v", err)
c.tryForgetClient(ctx, client)
break
}
if e.Error != nil {
// Hard logical error. We expect io.EOF next.
log.Errorf(ctx, "error consuming GossipSubscription RPC: %v", e.Error)
continue
}
handler, ok := gossipSubsHandlers[e.PatternMatched]
if !ok {
log.Errorf(ctx, "unknown GossipSubscription pattern: %q", e.PatternMatched)
continue
}
handler(c, ctx, e.Key, e.Content)
// Signal that startup is complete once the ClusterID gossip key has
// been handled.
if startupCh != nil && e.PatternMatched == gossip.KeyClusterID {
close(startupCh)
startupCh = nil
}
}
}
}
var gossipSubsHandlers = map[string]func(*Connector, context.Context, string, roachpb.Value){
// Subscribe to the ClusterID update.
gossip.KeyClusterID: (*Connector).updateClusterID,
// Subscribe to all *NodeDescriptor updates.
gossip.MakePrefixPattern(gossip.KeyNodeDescPrefix): (*Connector).updateNodeAddress,
// Subscribe to a filtered view of *SystemConfig updates.
gossip.KeyDeprecatedSystemConfig: (*Connector).updateSystemConfig,
}
var gossipSubsPatterns = func() []string {
patterns := make([]string, 0, len(gossipSubsHandlers))
for pattern := range gossipSubsHandlers {
patterns = append(patterns, pattern)
}
sort.Strings(patterns)
return patterns
}()
// updateClusterID handles updates to the "ClusterID" gossip key, and sets the
// rpcContext so that it's available to other code running in the tenant.
func (c *Connector) updateClusterID(ctx context.Context, key string, content roachpb.Value) {
bytes, err := content.GetBytes()
if err != nil {
log.Errorf(ctx, "invalid ClusterID value: %v", content.RawBytes)
return
}
clusterID, err := uuid.FromBytes(bytes)
if err != nil {
log.Errorf(ctx, "invalid ClusterID value: %v", content.RawBytes)
return
}
c.rpcContext.StorageClusterID.Set(ctx, clusterID)
}
// updateNodeAddress handles updates to "node" gossip keys, performing the
// corresponding update to the Connector's cached NodeDescriptor set.
func (c *Connector) updateNodeAddress(ctx context.Context, key string, content roachpb.Value) {
desc := new(roachpb.NodeDescriptor)
if err := content.GetProto(desc); err != nil {
log.Errorf(ctx, "could not unmarshal node descriptor: %v", err)
return
}
// TODO(nvanbenschoten): this doesn't handle NodeDescriptor removal from the
// gossip network. As it turns out, neither does Gossip.updateNodeAddress.
// There is some logic in Gossip.updateNodeAddress that attempts to remove
// replaced network addresses, but that logic has been dead since 5bce267.
// Other than that, gossip callbacks are not invoked on info expiration, so
// nothing ever removes them from Gossip.nodeDescs. Fix this.
c.mu.Lock()
defer c.mu.Unlock()
c.mu.nodeDescs[desc.NodeID] = desc
}
// GetNodeDescriptor implements the kvcoord.NodeDescStore interface.
func (c *Connector) GetNodeDescriptor(nodeID roachpb.NodeID) (*roachpb.NodeDescriptor, error) {
c.mu.RLock()
defer c.mu.RUnlock()
desc, ok := c.mu.nodeDescs[nodeID]
if !ok {
return nil, errors.Errorf("unable to look up descriptor for n%d", nodeID)
}
return desc, nil
}
// updateSystemConfig handles updates to a filtered view of the "system-db"
// gossip key, performing the corresponding update to the Connector's cached
// SystemConfig.
func (c *Connector) updateSystemConfig(ctx context.Context, key string, content roachpb.Value) {
cfg := config.NewSystemConfig(c.defaultZoneCfg)
if err := content.GetProto(&cfg.SystemConfigEntries); err != nil {
log.Errorf(ctx, "could not unmarshal system config: %v", err)
return
}
c.mu.Lock()
defer c.mu.Unlock()
c.mu.systemConfig = cfg
for c := range c.mu.systemConfigChannels {
select {
case c <- struct{}{}:
default:
}
}
}
// GetSystemConfig implements the config.SystemConfigProvider interface.
func (c *Connector) GetSystemConfig() *config.SystemConfig {
// TODO(nvanbenschoten): we need to wait in `(*Connector).Start()` until the
// system config is populated. As is, there's a small chance that we return
// nil, which SQL does not handle.
c.mu.RLock()
defer c.mu.RUnlock()
return c.mu.systemConfig
}
// RegisterSystemConfigChannel implements the config.SystemConfigProvider
// interface.
func (c *Connector) RegisterSystemConfigChannel() (_ <-chan struct{}, unregister func()) {
// Create channel that receives new system config notifications. The channel
// has a size of 1 to prevent connector from having to block on it.
ch := make(chan struct{}, 1)
c.mu.Lock()
defer c.mu.Unlock()
c.mu.systemConfigChannels[ch] = struct{}{}
// Notify the channel right away if we have a config.
if c.mu.systemConfig != nil {
ch <- struct{}{}
}
return ch, func() {
c.mu.Lock()
defer c.mu.Unlock()
delete(c.mu.systemConfigChannels, ch)
}
}
// RangeLookup implements the kvcoord.RangeDescriptorDB interface.
func (c *Connector) RangeLookup(
ctx context.Context, key roachpb.RKey, rc rangecache.RangeLookupConsistency, useReverseScan bool,
) ([]roachpb.RangeDescriptor, []roachpb.RangeDescriptor, error) {
// Proxy range lookup requests through the Internal service.
ctx = c.AnnotateCtx(ctx)
for ctx.Err() == nil {
client, err := c.getClient(ctx)
if err != nil {
continue
}
resp, err := client.RangeLookup(ctx, &roachpb.RangeLookupRequest{
Key: key,
// See the comment on (*kvcoord.DistSender).RangeLookup or kv.RangeLookup
// for more discussion on the choice of ReadConsistency and its
// implications.
ReadConsistency: rc,
PrefetchNum: kvcoord.RangeLookupPrefetchCount,
PrefetchReverse: useReverseScan,
})
if err != nil {
log.Warningf(ctx, "error issuing RangeLookup RPC: %v", err)
if grpcutil.IsAuthError(err) {
// Authentication or authorization error. Propagate.
return nil, nil, err
}
// Soft RPC error. Drop client and retry.
c.tryForgetClient(ctx, client)
continue
}
if resp.Error != nil {
// Hard logical error. Propagate.
return nil, nil, resp.Error.GoError()
}
return resp.Descriptors, resp.PrefetchedDescriptors, nil
}
return nil, nil, ctx.Err()
}
// Regions implements the serverpb.TenantStatusServer interface
func (c *Connector) Regions(
ctx context.Context, req *serverpb.RegionsRequest,
) (resp *serverpb.RegionsResponse, _ error) {
if err := c.withClient(ctx, func(ctx context.Context, c *client) error {
var err error
resp, err = c.Regions(ctx, req)
return err
}); err != nil {
return nil, err
}
return resp, nil
}
// NodeLocality implements the serverpb.TenantStatusServer interface
func (c *Connector) NodeLocality(
ctx context.Context, req *serverpb.NodeLocalityRequest,
) (resp *serverpb.NodeLocalityResponse, err error) {
if err := c.withClient(ctx, func(ctx context.Context, c *client) error {
var err error
resp, err = c.NodeLocality(ctx, req)
return err
}); err != nil {
return nil, err
}
return resp, nil
}
// TenantRanges implements the serverpb.TenantStatusServer interface
func (c *Connector) TenantRanges(
ctx context.Context, req *serverpb.TenantRangesRequest,
) (resp *serverpb.TenantRangesResponse, _ error) {
if err := c.withClient(ctx, func(ctx context.Context, c *client) error {
var err error
resp, err = c.TenantRanges(ctx, req)
return err
}); err != nil {
return nil, err
}
return resp, nil
}
// FirstRange implements the kvcoord.RangeDescriptorDB interface.
func (c *Connector) FirstRange() (*roachpb.RangeDescriptor, error) {
return nil, status.Error(codes.Unauthenticated, "kvtenant.Proxy does not have access to FirstRange")
}
// TokenBucket implements the kvtenant.TokenBucketProvider interface.
func (c *Connector) TokenBucket(
ctx context.Context, in *roachpb.TokenBucketRequest,
) (*roachpb.TokenBucketResponse, error) {
// Proxy token bucket requests through the Internal service.
ctx = c.AnnotateCtx(ctx)
for ctx.Err() == nil {
client, err := c.getClient(ctx)
if err != nil {
continue
}
resp, err := client.TokenBucket(ctx, in)
if err != nil {
log.Warningf(ctx, "error issuing TokenBucket RPC: %v", err)
if grpcutil.IsAuthError(err) {
// Authentication or authorization error. Propagate.
return nil, err
}
// Soft RPC error. Drop client and retry.
c.tryForgetClient(ctx, client)
continue
}
if resp.Error != (errorspb.EncodedError{}) {
// Hard logical error. Propagate.
return nil, errors.DecodeError(ctx, resp.Error)
}
return resp, nil
}
return nil, ctx.Err()
}
// GetSpanConfigRecords implements the spanconfig.KVAccessor interface.
func (c *Connector) GetSpanConfigRecords(
ctx context.Context, targets []spanconfig.Target,
) (records []spanconfig.Record, _ error) {
if err := c.withClient(ctx, func(ctx context.Context, c *client) error {
resp, err := c.GetSpanConfigs(ctx, &roachpb.GetSpanConfigsRequest{
Targets: spanconfig.TargetsToProtos(targets),
})
if err != nil {
return err
}
records, err = spanconfig.EntriesToRecords(resp.SpanConfigEntries)
if err != nil {
return err
}
return nil
}); err != nil {
return nil, err
}
return records, nil
}
// UpdateSpanConfigRecords implements the spanconfig.KVAccessor
// interface.
func (c *Connector) UpdateSpanConfigRecords(
ctx context.Context,
toDelete []spanconfig.Target,
toUpsert []spanconfig.Record,
minCommitTS, maxCommitTS hlc.Timestamp,
) error {
return c.withClient(ctx, func(ctx context.Context, c *client) error {
resp, err := c.UpdateSpanConfigs(ctx, &roachpb.UpdateSpanConfigsRequest{
ToDelete: spanconfig.TargetsToProtos(toDelete),
ToUpsert: spanconfig.RecordsToEntries(toUpsert),
MinCommitTimestamp: minCommitTS,
MaxCommitTimestamp: maxCommitTS,
})
if err != nil {
return err
}
if resp.Error.IsSet() {
// Logical error; propagate as such.
return errors.DecodeError(ctx, resp.Error)
}
return nil
})
}
// SpanConfigConformance implements the spanconfig.Reporter interface.
func (c *Connector) SpanConfigConformance(
ctx context.Context, spans []roachpb.Span,
) (roachpb.SpanConfigConformanceReport, error) {
var report roachpb.SpanConfigConformanceReport
if err := c.withClient(ctx, func(ctx context.Context, c *client) error {
resp, err := c.SpanConfigConformance(ctx, &roachpb.SpanConfigConformanceRequest{
Spans: spans,
})
if err != nil {
return err
}
report = resp.Report
return nil
}); err != nil {
return roachpb.SpanConfigConformanceReport{}, err
}
return report, nil
}
// GetAllSystemSpanConfigsThatApply implements the spanconfig.KVAccessor
// interface.
func (c *Connector) GetAllSystemSpanConfigsThatApply(
ctx context.Context, id roachpb.TenantID,
) ([]roachpb.SpanConfig, error) {
var spanConfigs []roachpb.SpanConfig
if err := c.withClient(ctx, func(ctx context.Context, c *client) error {
var err error
resp, err := c.GetAllSystemSpanConfigsThatApply(
ctx, &roachpb.GetAllSystemSpanConfigsThatApplyRequest{
TenantID: id,
})
if err != nil {
return err
}
spanConfigs = resp.SpanConfigs
return nil
}); err != nil {
return nil, err
}
return spanConfigs, nil
}
// WithTxn implements the spanconfig.KVAccessor interface.
func (c *Connector) WithTxn(context.Context, *kv.Txn) spanconfig.KVAccessor {
panic("not applicable")
}
// withClient is a convenience wrapper that executes the given closure while
// papering over InternalClient retrieval errors.
func (c *Connector) withClient(
ctx context.Context, f func(ctx context.Context, c *client) error,
) error {
ctx = c.AnnotateCtx(ctx)
for ctx.Err() == nil {
c, err := c.getClient(ctx)
if err != nil {
continue
}
return f(ctx, c)
}
return ctx.Err()
}
// getClient returns the singleton InternalClient if one is currently active. If
// not, the method attempts to dial one of the configured addresses. The method
// blocks until either a connection is successfully established or the provided
// context is canceled.
func (c *Connector) getClient(ctx context.Context) (*client, error) {
c.mu.RLock()
if client := c.mu.client; client != nil {
c.mu.RUnlock()
return client, nil
}
ch, _ := c.rpcDial.DoChan("dial", func() (interface{}, error) {
dialCtx := c.AnnotateCtx(context.Background())
dialCtx, cancel := c.rpcContext.Stopper.WithCancelOnQuiesce(dialCtx)
defer cancel()
var client *client
err := c.rpcContext.Stopper.RunTaskWithErr(dialCtx, "kvtenant.Connector: dial",
func(ctx context.Context) error {
var err error
client, err = c.dialAddrs(ctx)
return err
})
if err != nil {
return nil, err
}
c.mu.Lock()
defer c.mu.Unlock()
c.mu.client = client
return client, nil
})
c.mu.RUnlock()
select {
case res := <-ch:
if res.Err != nil {
return nil, res.Err
}
return res.Val.(*client), nil
case <-ctx.Done():
return nil, ctx.Err()
}
}
// dialAddrs attempts to dial each of the configured addresses in a retry loop.
// The method will only return a non-nil error on context cancellation.
func (c *Connector) dialAddrs(ctx context.Context) (*client, error) {
for r := retry.StartWithCtx(ctx, c.rpcRetryOptions); r.Next(); {
// Try each address on each retry iteration (in random order).
for _, i := range rand.Perm(len(c.addrs)) {
addr := c.addrs[i]
conn, err := c.dialAddr(ctx, addr)
if err != nil {
log.Warningf(ctx, "error dialing tenant KV address %s: %v", addr, err)
continue
}
return &client{
InternalClient: roachpb.NewInternalClient(conn),
StatusClient: serverpb.NewStatusClient(conn),
}, nil
}
}
return nil, ctx.Err()
}
func (c *Connector) dialAddr(ctx context.Context, addr string) (conn *grpc.ClientConn, err error) {
if c.rpcDialTimeout == 0 {
return c.rpcContext.GRPCUnvalidatedDial(addr).Connect(ctx)
}
err = contextutil.RunWithTimeout(ctx, "dial addr", c.rpcDialTimeout, func(ctx context.Context) error {
conn, err = c.rpcContext.GRPCUnvalidatedDial(addr).Connect(ctx)
return err
})
return conn, err
}
func (c *Connector) tryForgetClient(ctx context.Context, client roachpb.InternalClient) {
if ctx.Err() != nil {
// Error (may be) due to context. Don't forget client.
return
}
// Compare-and-swap to avoid thrashing.
c.mu.Lock()
defer c.mu.Unlock()
if c.mu.client == client {
c.mu.client = nil
}
}