-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
transport.go
381 lines (338 loc) · 12.4 KB
/
transport.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
// Copyright 2016 The Cockroach Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
// implied. See the License for the specific language governing
// permissions and limitations under the License. See the AUTHORS file
// for names of contributors.
package kv
import (
"context"
"sort"
"time"
"github.com/cockroachdb/cockroach/pkg/internal/client"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/rpc"
"github.com/cockroachdb/cockroach/pkg/rpc/nodedialer"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/opentracing/opentracing-go"
"github.com/pkg/errors"
)
// A SendOptions structure describes the algorithm for sending RPCs to one or
// more replicas, depending on error conditions and how many successful
// responses are required.
type SendOptions struct {
metrics *DistSenderMetrics
}
type batchClient struct {
replica roachpb.ReplicaDescriptor
healthy bool
retryable bool
deadline time.Time
}
// TransportFactory encapsulates all interaction with the RPC
// subsystem, allowing it to be mocked out for testing. The factory
// function returns a Transport object which is used to send requests
// to one or more replicas in the slice.
//
// In addition to actually sending RPCs, the transport is responsible
// for ordering replicas in accordance with SendOptions.Ordering and
// transport-specific knowledge such as connection health or latency.
//
// TODO(bdarnell): clean up this crufty interface; it was extracted
// verbatim from the non-abstracted code.
type TransportFactory func(SendOptions, *nodedialer.Dialer, ReplicaSlice) (Transport, error)
// Transport objects can send RPCs to one or more replicas of a range.
// All calls to Transport methods are made from a single thread, so
// Transports are not required to be thread-safe.
type Transport interface {
// IsExhausted returns true if there are no more replicas to try.
IsExhausted() bool
// SendNext synchronously sends the BatchRequest rpc to the next replica.
// May panic if the transport is exhausted.
//
// SendNext is also in charge of importing the remotely collected spans (if
// any) into the local trace.
SendNext(context.Context, roachpb.BatchRequest) (*roachpb.BatchResponse, error)
// NextInternalClient returns the InternalClient to use for making RPC
// calls. Returns a context.Context which should be used when making RPC
// calls on the returned server (This context is annotated to mark this
// request as in-process and bypass ctx.Peer checks).
NextInternalClient(context.Context) (context.Context, roachpb.InternalClient, error)
// NextReplica returns the replica descriptor of the replica to be tried in
// the next call to SendNext. MoveToFront will cause the return value to
// change. Returns a zero value if the transport is exhausted.
NextReplica() roachpb.ReplicaDescriptor
// MoveToFront locates the specified replica and moves it to the
// front of the ordering of replicas to try. If the replica has
// already been tried, it will be retried. If the specified replica
// can't be found, this is a noop.
MoveToFront(roachpb.ReplicaDescriptor)
}
// grpcTransportFactoryImpl is the default TransportFactory, using GRPC.
// Do not use this directly - use grpcTransportFactory instead.
//
// During race builds, we wrap this to hold on to and read all obtained
// requests in a tight loop, exposing data races; see transport_race.go.
func grpcTransportFactoryImpl(
opts SendOptions, nodeDialer *nodedialer.Dialer, replicas ReplicaSlice,
) (Transport, error) {
clients := make([]batchClient, 0, len(replicas))
for _, replica := range replicas {
healthy := nodeDialer.ConnHealth(replica.NodeID) == nil
clients = append(clients, batchClient{
replica: replica.ReplicaDescriptor,
healthy: healthy,
})
}
// Put known-healthy clients first.
splitHealthy(clients)
return &grpcTransport{
opts: opts,
nodeDialer: nodeDialer,
orderedClients: clients,
}, nil
}
type grpcTransport struct {
opts SendOptions
nodeDialer *nodedialer.Dialer
clientIndex int
orderedClients []batchClient
}
// IsExhausted returns false if there are any untried replicas remaining. If
// there are none, it attempts to resurrect replicas which were tried but
// failed with a retryable error. If any where resurrected, returns false;
// true otherwise.
func (gt *grpcTransport) IsExhausted() bool {
if gt.clientIndex < len(gt.orderedClients) {
return false
}
return !gt.maybeResurrectRetryablesLocked()
}
// maybeResurrectRetryablesLocked moves already-tried replicas which
// experienced a retryable error (currently this means a
// NotLeaseHolderError) into a newly-active state so that they can be
// retried. Returns true if any replicas were moved to active.
func (gt *grpcTransport) maybeResurrectRetryablesLocked() bool {
var resurrect []batchClient
for i := 0; i < gt.clientIndex; i++ {
if c := gt.orderedClients[i]; c.retryable && timeutil.Since(c.deadline) >= 0 {
resurrect = append(resurrect, c)
}
}
for _, c := range resurrect {
gt.moveToFrontLocked(c.replica)
}
return len(resurrect) > 0
}
// SendNext invokes the specified RPC on the supplied client when the
// client is ready. On success, the reply is sent on the channel;
// otherwise an error is sent.
func (gt *grpcTransport) SendNext(
ctx context.Context, ba roachpb.BatchRequest,
) (*roachpb.BatchResponse, error) {
client := gt.orderedClients[gt.clientIndex]
ctx, iface, err := gt.NextInternalClient(ctx)
if err != nil {
return nil, err
}
ba.Replica = client.replica
reply, err := gt.sendBatch(ctx, client.replica.NodeID, iface, ba)
// NotLeaseHolderErrors can be retried.
var retryable bool
if reply != nil && reply.Error != nil {
// TODO(spencer): pass the lease expiration when setting the state
// to set a more efficient deadline for retrying this replica.
if _, ok := reply.Error.GetDetail().(*roachpb.NotLeaseHolderError); ok {
retryable = true
}
}
gt.setState(client.replica, retryable)
return reply, err
}
// NB: nodeID is unused, but accessible in stack traces.
func (gt *grpcTransport) sendBatch(
ctx context.Context, nodeID roachpb.NodeID, iface roachpb.InternalClient, ba roachpb.BatchRequest,
) (*roachpb.BatchResponse, error) {
// Bail out early if the context is already canceled. (GRPC will
// detect this pretty quickly, but the first check of the context
// in the local server comes pretty late)
if ctx.Err() != nil {
return nil, errors.Wrap(ctx.Err(), "aborted before batch send")
}
gt.opts.metrics.SentCount.Inc(1)
if rpc.IsLocal(iface) {
gt.opts.metrics.LocalSentCount.Inc(1)
}
reply, err := iface.Batch(ctx, &ba)
// If we queried a remote node, perform extra validation and
// import trace spans.
if reply != nil && !rpc.IsLocal(iface) {
for i := range reply.Responses {
if err := reply.Responses[i].GetInner().Verify(ba.Requests[i].GetInner()); err != nil {
log.Error(ctx, err)
}
}
// Import the remotely collected spans, if any.
if len(reply.CollectedSpans) != 0 {
span := opentracing.SpanFromContext(ctx)
if span == nil {
return nil, errors.Errorf(
"trying to ingest remote spans but there is no recording span set up")
}
if err := tracing.ImportRemoteSpans(span, reply.CollectedSpans); err != nil {
return nil, errors.Wrap(err, "error ingesting remote spans")
}
}
}
return reply, err
}
// NextInternalClient returns the next InternalClient to use for performing
// RPCs.
func (gt *grpcTransport) NextInternalClient(
ctx context.Context,
) (context.Context, roachpb.InternalClient, error) {
client := gt.orderedClients[gt.clientIndex]
gt.clientIndex++
return gt.nodeDialer.DialInternalClient(ctx, client.replica.NodeID)
}
func (gt *grpcTransport) NextReplica() roachpb.ReplicaDescriptor {
if gt.IsExhausted() {
return roachpb.ReplicaDescriptor{}
}
return gt.orderedClients[gt.clientIndex].replica
}
func (gt *grpcTransport) MoveToFront(replica roachpb.ReplicaDescriptor) {
gt.moveToFrontLocked(replica)
}
func (gt *grpcTransport) moveToFrontLocked(replica roachpb.ReplicaDescriptor) {
for i := range gt.orderedClients {
if gt.orderedClients[i].replica == replica {
// Clear the retryable bit as this replica is being made
// available.
gt.orderedClients[i].retryable = false
gt.orderedClients[i].deadline = time.Time{}
// If we've already processed the replica, decrement the current
// index before we swap.
if i < gt.clientIndex {
gt.clientIndex--
}
// Swap the client representing this replica to the front.
gt.orderedClients[i], gt.orderedClients[gt.clientIndex] =
gt.orderedClients[gt.clientIndex], gt.orderedClients[i]
return
}
}
}
// NB: this method's callers may have a reference to the client they wish to
// mutate, but the clients reside in a slice which is shuffled via
// MoveToFront, making it unsafe to mutate the client through a reference to
// the slice.
func (gt *grpcTransport) setState(replica roachpb.ReplicaDescriptor, retryable bool) {
for i := range gt.orderedClients {
if gt.orderedClients[i].replica == replica {
gt.orderedClients[i].retryable = retryable
if retryable {
gt.orderedClients[i].deadline = timeutil.Now().Add(time.Second)
}
break
}
}
}
// splitHealthy splits the provided client slice into healthy clients and
// unhealthy clients, based on their connection state. Healthy clients will
// be rearranged first in the slice, and unhealthy clients will be rearranged
// last. Within these two groups, the rearrangement will be stable. The function
// will then return the number of healthy clients.
func splitHealthy(clients []batchClient) int {
var nHealthy int
sort.Stable(byHealth(clients))
for _, client := range clients {
if client.healthy {
nHealthy++
}
}
return nHealthy
}
// byHealth sorts a slice of batchClients by their health with healthy first.
type byHealth []batchClient
func (h byHealth) Len() int { return len(h) }
func (h byHealth) Swap(i, j int) { h[i], h[j] = h[j], h[i] }
func (h byHealth) Less(i, j int) bool { return h[i].healthy && !h[j].healthy }
// SenderTransportFactory wraps a client.Sender for use as a KV
// Transport. This is useful for tests that want to use DistSender
// without a full RPC stack.
func SenderTransportFactory(tracer opentracing.Tracer, sender client.Sender) TransportFactory {
return func(
_ SendOptions, _ *nodedialer.Dialer, replicas ReplicaSlice,
) (Transport, error) {
// Always send to the first replica.
replica := replicas[0].ReplicaDescriptor
return &senderTransport{tracer, sender, replica, false}, nil
}
}
type senderTransport struct {
tracer opentracing.Tracer
sender client.Sender
replica roachpb.ReplicaDescriptor
called bool
}
func (s *senderTransport) IsExhausted() bool {
return s.called
}
func (s *senderTransport) SendNext(
ctx context.Context, ba roachpb.BatchRequest,
) (*roachpb.BatchResponse, error) {
if s.called {
panic("called an exhausted transport")
}
s.called = true
ctx, cleanup := tracing.EnsureContext(ctx, s.tracer, "node" /* name */)
defer cleanup()
ba.Replica = s.replica
log.Event(ctx, ba.String())
br, pErr := s.sender.Send(ctx, ba)
if br == nil {
br = &roachpb.BatchResponse{}
}
if br.Error != nil {
panic(roachpb.ErrorUnexpectedlySet(s.sender, br))
}
br.Error = pErr
if pErr != nil {
log.Event(ctx, "error: "+pErr.String())
}
// Import the remotely collected spans, if any.
if len(br.CollectedSpans) != 0 {
span := opentracing.SpanFromContext(ctx)
if span == nil {
panic("trying to ingest remote spans but there is no recording span set up")
}
if err := tracing.ImportRemoteSpans(span, br.CollectedSpans); err != nil {
panic(err)
}
}
return br, nil
}
func (s *senderTransport) NextInternalClient(
ctx context.Context,
) (context.Context, roachpb.InternalClient, error) {
panic("unimplemented")
}
func (s *senderTransport) NextReplica() roachpb.ReplicaDescriptor {
if s.IsExhausted() {
return roachpb.ReplicaDescriptor{}
}
return s.replica
}
func (s *senderTransport) MoveToFront(replica roachpb.ReplicaDescriptor) {
}