-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
kvflowcontrol.go
137 lines (126 loc) · 6.8 KB
/
kvflowcontrol.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
// Copyright 2023 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
// Package kvflowcontrol provides flow control for replication traffic in KV.
// It's part of the integration layer between KV and admission control.
package kvflowcontrol
import (
"time"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb"
)
// Stream models the stream over which we replicate data traffic, the
// transmission for which we regulate using flow control. It's segmented by the
// specific store the traffic is bound for and the tenant driving it. Despite
// the underlying link/transport being shared across tenants, modeling streams
// on a per-tenant basis helps provide inter-tenant isolation.
type Stream struct {
TenantID roachpb.TenantID
StoreID roachpb.StoreID
}
// Tokens represent the finite capacity of a given stream, expressed in bytes
// for data we're looking to replicate. Use of replication streams are
// predicated on tokens being available.
type Tokens uint64
// Controller provides flow control for replication traffic in KV, held at the
// node-level.
type Controller interface {
// Admit seeks admission to replicate data of a given priority (regardless
// of size) and create-time over the specified streams. This is a blocking
// operation; requests wait until there are flow tokens available.
Admit(admissionpb.WorkPriority, time.Time, ...Stream)
// DeductTokens deducts (without blocking) flow tokens for transmission over
// the given stream, for work with the given priority. This is non-blocking.
// Requests are expected to have been Admit()-ed first.
DeductTokens(admissionpb.WorkPriority, Tokens, Stream) bool
// ReturnTokens returns flow tokens for the given stream. These tokens are
// expected to have been deducted earlier with a specific priority; that
// same priority is what's specified here.
ReturnTokens(admissionpb.WorkPriority, Tokens, Stream)
// TODO(irfansharif): We might need the ability to "disable" specific
// streams/corresponding token buckets when there are failures or
// replication to a specific store is paused due to follower-pausing.
// That'll have to show up between the Handler and the Controller somehow.
}
// Handle is used to interface with replication flow control; it's typically
// backed by a node-level Controller. Handles are held on replicas initiating
// replication traffic, i.e. are both the leaseholder and raft leader, and
// manage multiple Streams (one per active replica) underneath.
//
// When replicating log entries, these replicas choose the log position
// (term+index) the data is to end up at, and use this handle to track the token
// deductions on a per log position basis. Later when freeing up tokens
// (typically after being informed of said log entries being admitted on the
// receiving end of the stream), we do so by specifying the log position up to
// which we free up deducted tokens (see kvflowcontrolpb.AdmittedRaftLogEntries
// for more details).
type Handle interface {
// Admit seeks admission to replicate data of a given priority (regardless
// of size) and create time. This is a blocking operation; requests wait
// until there are flow tokens available.
Admit(admissionpb.WorkPriority, time.Time)
// DeductTokensFor deducts flow tokens for replicating data of a given
// priority to members of the raft group, and tracks it with respect to the
// specific raft log position it's expecting it to end up in. This is
// non-blocking. Requests are assumed to have been Admit()-ed first.
DeductTokensFor(admissionpb.WorkPriority, kvflowcontrolpb.RaftLogPosition, Tokens)
// DeductedTokensUpto returns the highest log position for which we've
// deducted flow tokens for the given stream.
DeductedTokensUpto(Stream) kvflowcontrolpb.RaftLogPosition
// ReturnTokensUpto returns all previously deducted tokens of a given
// priority for all log positions less than or equal to the one specified.
// It does for the specific stream. Once returned, subsequent attempts to
// return tokens upto the same position or lower are no-ops.
ReturnTokensUpto(admissionpb.WorkPriority, kvflowcontrolpb.RaftLogPosition, Stream)
// TrackLowWater is used to set a low-water mark for a given replication
// stream. Tokens held below this position are returned back to the
// underlying Controller, regardless of priority. All subsequent returns at
// that position or lower are ignored.
//
// NB: This is used when a replica on the other end of a stream gets caught
// up via snapshot (say, after a log truncation), where we then don't expect
// dispatches for the individual AdmittedRaftLogEntries between what it
// admitted last and its latest RaftLogPosition. Another use is during
// successive lease changes (out and back) within the same raft term -- we
// want to both free up tokens from when we lost the lease, and also ensure
// that attempts to return them (on hearing about AdmittedRaftLogEntries
// replicated under the earlier lease), we discard the attempts.
TrackLowWater(kvflowcontrolpb.RaftLogPosition, Stream)
// Close closes the handle and returns all held tokens back to the
// underlying controller. Typically used when the replica loses its lease
// and/or raft leadership, or ends up getting GC-ed (if it's being
// rebalanced, merged away, etc).
Close()
}
// Dispatch is used to dispatch information about admitted raft log entries to
// specific nodes and read pending dispatches.
type Dispatch interface {
DispatchWriter
DispatchReader
}
// DispatchWriter is used to dispatch information about admitted raft log
// entries to specific nodes (typically where said entries originated, where
// flow tokens were deducted and waiting to be returned).
type DispatchWriter interface {
Dispatch(roachpb.NodeID, kvflowcontrolpb.AdmittedRaftLogEntries)
}
// DispatchReader is used to read pending dispatches. It's used in the raft
// transport layer when looking to piggyback information on traffic already
// bound to specific nodes. It's also used when timely dispatching (read:
// piggybacking) has not taken place.
//
// NB: PendingDispatchFor is expected to remove dispatches from the pending
// list. If the GRPC stream we're sending it over happens to break, we drop
// these dispatches. The node waiting these dispatches is expected to react to
// the stream breaking by freeing up all held tokens.
type DispatchReader interface {
PendingDispatch() []roachpb.NodeID
PendingDispatchFor(roachpb.NodeID) []kvflowcontrolpb.AdmittedRaftLogEntries
}