-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
Copy pathdirectory.go
347 lines (304 loc) · 11.8 KB
/
directory.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
// Copyright 2021 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 tenant
import (
"context"
"io"
"sync"
"time"
"github.com/cockroachdb/cockroach/pkg/util/grpcutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/errors"
"github.com/gogo/status"
)
//go:generate mockgen -package=tenant -destination=mocks_generated.go . DirectoryClient,Directory_WatchEndpointsClient
// dirOptions control the behavior of tenant.Directory.
type dirOptions struct {
deterministic bool
refreshDelay time.Duration
}
// DirOption defines an option that can be passed to tenant.Directory in order
// to control its behavior.
type DirOption func(opts *dirOptions)
// RefreshDelay specifies the minimum amount of time that must elapse between
// attempts to refresh endpoints for a given tenant after ReportFailure is called.
// This delay has the effect of throttling calls to directory server, in order to
// avoid overloading it.
//
// RefreshDelay defaults to 100ms. Use -1 to never throttle.
func RefreshDelay(delay time.Duration) func(opts *dirOptions) {
return func(opts *dirOptions) {
opts.refreshDelay = delay
}
}
// Directory tracks the network locations of SQL tenant processes. It is used by the
// sqlproxy to route incoming traffic to the correct backend process. Process
// information is populated and kept relatively up-to-date using a streaming watcher.
// However, since watchers deliver slightly stale information, the directory will also make
// direct server calls to fetch the latest information about a process that is not yet
// in the cache, or when a process is suspected to have failed. When a new tenant is
// created, or is resumed from suspension, this capability allows the directory
// to immediately return the IP address for the new process.
//
// All methods in the directory are thread-safe. Methods are intended to be
// called concurrently by many threads at once, and so locking is carefully
// designed to minimize contention. While a lock shared across tenants is used
// to synchronize access to shared in-memory data structures, each tenant also
// has its own locks that are used to synchronize per-tenant operations such as
// making directory server calls to fetch updated tenant information.
type Directory struct {
// ctl is the directory client instance used to make directory server calls.
ctl DirectoryClient
// stopper use used for graceful shutdown of the endpoint watcher.
stopper *stop.Stopper
// options control how the environment operates.
options dirOptions
// mut synchronizes access to the in-memory tenant entry caches. Take care
// to never hold this lock during directory server calls - it should only be used
// while adding and removing tenant entries to/from the caches.
mut struct {
syncutil.Mutex
// tenants is a cache of tenant entries. Each entry tracks available IP
// addresses for SQL processes for a given tenant. Entries may not be
// fully initialized.
tenants map[ID]*tenantEntry
}
}
// NewDirectory constructs a new Directory instance that tracks SQL tenant processes
// managed by a given Directory server. The given context
// is used for tracing endpoint watcher activity.
//
// NOTE: stopper.Stop must be called on the directory when it is no longer needed.
func NewDirectory(
ctx context.Context, stopper *stop.Stopper, client DirectoryClient, opts ...DirOption,
) (*Directory, error) {
dir := &Directory{ctl: client, stopper: stopper}
dir.mut.tenants = make(map[ID]*tenantEntry)
for _, opt := range opts {
opt(&dir.options)
}
if dir.options.refreshDelay == 0 {
// Default to a delay of 100ms between refresh attempts for a given tenant.
dir.options.refreshDelay = 100 * time.Millisecond
}
// Starts the endpoint watcher and then returns
if err := dir.watchEndpoints(ctx, stopper); err != nil {
return nil, err
}
return dir, nil
}
// EnsureTenantIP returns the IP address of one of the given tenant's SQL processes.
// If the tenant was just created or is suspended, such that there are no
// available processes, then EnsureTenantIP will trigger resumption of a new instance and
// block until the process is ready. If there are multiple processes for
// the tenant, then LookupTenantIPs will choose one of them (note that currently
// there is always at most one SQL process per tenant).
//
// If clusterName is non-empty, then an error is returned if no endpoints match the
// cluster name. This can be used to ensure that the incoming SQL connection "knows"
// some additional information about the tenant, such as the name of the
// cluster, before being allowed to connect.
func (d *Directory) EnsureTenantIP(
ctx context.Context, tenantID ID, clusterName string,
) (string, error) {
// Ensure that a directory entry has been created for this tenant.
entry, err := d.getEntry(ctx, tenantID, true /* allowCreate */)
if err != nil {
return "", err
}
// Check the cluster name matches, if specified.
if clusterName != "" && clusterName != entry.ClusterName {
log.Errorf(ctx, "cluster name %s doesn't match expected %s", clusterName, entry.ClusterName)
return "", errors.New("not found")
}
ctx, _ = d.stopper.WithCancelOnQuiesce(ctx)
ip, err := entry.ChooseEndpointIP(ctx, d.ctl, d.options.deterministic)
if err != nil {
if s, ok := status.FromError(err); ok && s.Message() == "not found" {
d.deleteEntry(tenantID)
}
}
return ip, err
}
// LookupTenantIPs returns the IP addresses for all available SQL processes for the
// given tenant. It returns an error if the tenant has not yet been created. If
// no processes are available for the tenant, LookupTenantIPs will return the empty
// set (unlike EnsureTenantIP).
func (d *Directory) LookupTenantIPs(ctx context.Context, tenantID ID) ([]string, error) {
// Ensure that a directory entry has been created for this tenant.
entry, err := d.getEntry(ctx, tenantID, false /* allowCreate */)
if err != nil {
return nil, err
}
if entry == nil {
return nil, errors.New("not found")
}
return entry.getEndpointIPs(), nil
}
// ReportFailure should be called when attempts to connect to a particular SQL
// tenant endpoint have failed. Since this could be due to a failed process,
// ReportFailure will attempt to refresh the cache with the latest information
// about available tenant processes.
// TODO(andyk): In the future, the ip parameter will be used to mark a
// particular endpoint as "unhealthy" so that it's less likely to be chosen. However,
// today there can be at most one endpoint for a given tenant, so it must always be
// chosen. Keep the parameter as a placeholder for the future.
func (d *Directory) ReportFailure(ctx context.Context, tenantID ID, ip string) error {
entry, err := d.getEntry(ctx, tenantID, false /* allowCreate */)
if err != nil {
return err
} else if entry == nil {
// If no tenant is in the cache, no-op.
return nil
}
// Refresh the entry in case there is a new endpoint IP address.
return entry.RefreshEndpoints(ctx, d.ctl)
}
// getEntry returns a directory entry for the given tenant. If the directory
// does not contain such an entry, then getEntry will create one if allowCreate
// is true. Otherwise, it returns nil. If an entry is returned, then getEntry
// ensures that it is fully initialized with tenant metadata. Obtaining this
// metadata requires making a separate directory server call;
// getEntry will block until that's complete.
func (d *Directory) getEntry(
ctx context.Context, tenantID ID, allowCreate bool,
) (*tenantEntry, error) {
entry := func() *tenantEntry {
// Acquire the directory lock just long enough to check the tenants map
// for the given tenant ID. Don't complete initialization while holding
// this lock, since that requires directory server calls.
d.mut.Lock()
defer d.mut.Unlock()
entry, ok := d.mut.tenants[tenantID]
if ok {
// Entry exists, so return it.
return entry
}
if !allowCreate {
// No entry, but not allowed to create one, so done.
return nil
}
// Create the tenant entry and enter it into the tenants map.
log.Infof(ctx, "creating directory entry for tenant %d", tenantID)
entry = &tenantEntry{TenantID: tenantID, RefreshDelay: d.options.refreshDelay}
d.mut.tenants[tenantID] = entry
return entry
}()
if entry == nil {
return nil, nil
}
// Initialize the entry now if not yet done.
err := entry.Initialize(ctx, d.ctl)
if err != nil {
// Remove the entry from the tenants map, since initialization failed.
d.mut.Lock()
defer d.mut.Unlock()
// Threads can race to add/remove entries, so ensure that right entry is
// removed.
existing, ok := d.mut.tenants[tenantID]
if ok && entry == existing {
delete(d.mut.tenants, tenantID)
log.Infof(ctx, "error initializing tenant %d: %v", tenantID, err)
}
return nil, err
}
return entry, nil
}
// deleteEntry removes the directory entry for the given tenant, if it exists.
func (d *Directory) deleteEntry(tenantID ID) {
d.mut.Lock()
defer d.mut.Unlock()
delete(d.mut.tenants, tenantID)
}
// watchEndpoints establishes a watcher that looks for changes to tenant endpoint addresses.
// Whenever tenant processes start or terminate, the watcher will get
// a notification and update the directory to reflect that change.
func (d *Directory) watchEndpoints(ctx context.Context, stopper *stop.Stopper) error {
req := WatchEndpointsRequest{}
var waitInit sync.WaitGroup
waitInit.Add(1)
err := stopper.RunAsyncTask(ctx, "watch-endpoints-client", func(ctx context.Context) {
var client Directory_WatchEndpointsClient
var err error
firstRun := true
ctx, _ = stopper.WithCancelOnQuiesce(ctx)
for {
if client == nil {
client, err = d.ctl.WatchEndpoints(ctx, &req)
if firstRun {
waitInit.Done()
firstRun = false
}
if err != nil {
if grpcutil.IsContextCanceled(err) {
break
}
log.Errorf(ctx, "err creating new watch endpoint client: %s", err)
time.Sleep(time.Second)
continue
}
}
// Read the next watcher event.
resp, err := client.Recv()
if err != nil {
if grpcutil.IsContextCanceled(err) {
break
}
if err != io.EOF {
log.Errorf(ctx, "err receiving stream events: %s", err)
time.Sleep(time.Second)
}
// Loop around and try a new call to get a client stream.
client = nil
continue
}
endpointIP := resp.Ip
if endpointIP == "" {
// Nothing needs to be done if there is no IP address specified.
continue
}
// Ensure that a directory entry exists for this tenant.
entry, err := d.getEntry(ctx, ID(resp.TenantId), true /* allowCreate */)
if err != nil {
if grpcutil.IsContextCanceled(err) {
break
}
// This should never happen.
log.Errorf(ctx, "ignoring error getting entry for tenant %d: %v", resp.TenantId, err)
continue
}
// For now, all we care about is the IP addresses of the tenant endpoint.
switch resp.Typ {
case ADDED, MODIFIED:
if entry.AddEndpointIP(endpointIP) {
log.Infof(ctx, "added IP address %s for tenant %d", endpointIP, resp.TenantId)
}
case DELETED:
if entry.RemoveEndpointIP(endpointIP) {
log.Infof(ctx, "deleted IP address %s for tenant %d", endpointIP, resp.TenantId)
}
}
}
})
if err != nil {
return err
}
// Block until the initial endpoint watcher client stream is constructed.
waitInit.Wait()
return err
}
// sleepContext sleeps for the given duration or until the given context is
// canceled, whichever comes first.
func sleepContext(ctx context.Context, delay time.Duration) {
select {
case <-ctx.Done():
case <-time.After(delay):
}
}