Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
97599: admission: support non-blocking {Store,}WorkQueue.Admit() r=irfansharif a=irfansharif

Part of #95563. For end-to-end flow control of replicated writes, we want to enable below-raft admission control through the following API on kvadmission.Controller:
```go
  // AdmitRaftEntry informs admission control of a raft log entry being
  // written to storage (for the given tenant, the specific range, and
  // on the named store).
  AdmitRaftEntry(
    context.Context, roachpb.TenantID,
    roachpb.StoreID, roachpb.RangeID, raftpb.Entry.
  )
```
This serves as the integration point for log entries received below raft right as they're being written to stable storage. It's a non-blocking interface since we're below-raft and in the raft.Ready() loop. What it effectively does is enqueues a "virtual" work item in the underlying StoreWorkQueue mediating all store IO. This virtual work item is what later gets dequeued once the IO granter informs the work queue of newly available IO tokens. When enqueueing the virtual work item, we still update the StoreWorkQueue's physically-accounted-for bytes since the actual write is not deferred, and timely statistic updates improves accuracy for the underlying linear models (that map between accounted-for writes and observed L0 growth, using it to inform IO token generation rates).

For each of the arguments above:
- The roachpb.TenantID is plumbed to find the right tenant heap to queue it under (for inter-tenant isolation).
- The roachpb.StoreID to find the right store work queue on multi-store nodes. We'll also use the StoreID when informing the origin node of log entries being admitted[^1].
- We pass in the roachpb.RangeID on behalf of which work is being admitted. This, along side the raftpb.Entry.{Term,Index} for the replicated write uniquely identifies where the write is to end up. We use these identifiers to return flow tokens on the origin node[^1][^2].
- For standard work queue ordering, our work item needs to include the CreateTime and AdmissionPriority, details that are passed down using dedicated raft log entry encodings[^3][^4] as part of the raftpb.Entry parameter above.
  - Since the raftpb.Entry encodes within it its origin node[^4], it will be used post-admission to dispatch flow tokens to the right node. This integration is left to future PRs.

We use the above to populate the following fields on a per-(replicated write)work basis:
```go
    // ReplicatedWorkInfo groups everything needed to admit replicated
    // writes, done so asynchronously below-raft as part of replication
    // admission control.
    type ReplicatedWorkInfo struct {
      RangeID roachpb.RangeID
      Origin roachpb.NodeID
      LogPosition LogPosition
      Ingested bool
    }
```
Since admission is happening below-raft where the size of the write is known, we no longer need per-work estimates for upfront IO token deductions. Since admission is asynchronous, we also don't use the AdmittedWorkDone interface which was to make token adjustments (without blocking) given the upfront estimates. We still want to intercept the exact point when some write work gets admitted in order to inform the origin node so it can release flow tokens. We do so through the following interface satisfied by the StoreWorkQueue:
```go
  // onAdmittedReplicatedWork is used to intercept the
  // point-of-admission for replicated writes.
  type onAdmittedReplicatedWork interface {
    admittedReplicatedWork(
      tenantID roachpb.TenantID,
      pri admissionpb.WorkPriority,
      rwi ReplicatedWorkInfo,
      requestedTokens int64,
    )
  }
```

[^1]: See kvflowcontrolpb.AdmittedRaftLogEntries introduced in #95637.
[^2]: See kvflowcontrol.Handle.{ReturnTokensUpto,DeductTokensFor} introduced in #95637. Token deductions and returns are tied to raft log positions.
[^3]: See raftlog.EntryEncoding{Standard,Sideloaded}WithAC introduced in #95748.
[^4]: See kvflowcontrolpb.RaftAdmissionMeta introduced in #95637.

Release note: None


98419: clusterversion: add a gate for new system privileges r=jayshrivastava a=rafiss

A 22.2/23.1 mixed version cluster cannot handle new system privileges well. This commit gates their usage and adds a test.

Without this gate, the included test would fail and users would not be able to log in to nodes running on the old binary.

Epic: None
Release note: None

98495: settingswatcher: version guard support for clusters bootstrapped at old versions r=JeffSwenson a=JeffSwenson

When a cluster is bootstrapping, the sql server is initialized before the cluster version is populated in the DB. Previously, the version guard utility was unable to handle this state if the version is older than the maxVersion used to initialize the version guard. Now, the versionGuard handles this bootstrapping state by falling back on the in-memory cluster version.

Part of #94843

Release note: none

Co-authored-by: irfan sharif <[email protected]>
Co-authored-by: Rafi Shamim <[email protected]>
Co-authored-by: Jeff <[email protected]>
  • Loading branch information
4 people committed Mar 13, 2023
4 parents c31c1ac + c550113 + fc50c99 + 506a3ef commit 2d63378
Show file tree
Hide file tree
Showing 50 changed files with 2,001 additions and 213 deletions.
2 changes: 1 addition & 1 deletion docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -299,4 +299,4 @@ trace.opentelemetry.collector string address of an OpenTelemetry trace collecto
trace.snapshot.rate duration 0s if non-zero, interval at which background trace snapshots are captured
trace.span_registry.enabled boolean true if set, ongoing traces can be seen at https://<ui>/#/debug/tracez
trace.zipkin.collector string the address of a Zipkin instance to receive traces, as <host>:<port>. If no port is specified, 9411 will be used.
version version 1000022.2-76 set the active cluster version in the format '<major>.<minor>'
version version 1000022.2-78 set the active cluster version in the format '<major>.<minor>'
2 changes: 1 addition & 1 deletion docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -240,6 +240,6 @@
<tr><td><div id="setting-trace-snapshot-rate" class="anchored"><code>trace.snapshot.rate</code></div></td><td>duration</td><td><code>0s</code></td><td>if non-zero, interval at which background trace snapshots are captured</td></tr>
<tr><td><div id="setting-trace-span-registry-enabled" class="anchored"><code>trace.span_registry.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>if set, ongoing traces can be seen at https://&lt;ui&gt;/#/debug/tracez</td></tr>
<tr><td><div id="setting-trace-zipkin-collector" class="anchored"><code>trace.zipkin.collector</code></div></td><td>string</td><td><code></code></td><td>the address of a Zipkin instance to receive traces, as &lt;host&gt;:&lt;port&gt;. If no port is specified, 9411 will be used.</td></tr>
<tr><td><div id="setting-version" class="anchored"><code>version</code></div></td><td>version</td><td><code>1000022.2-76</code></td><td>set the active cluster version in the format &#39;&lt;major&gt;.&lt;minor&gt;&#39;</td></tr>
<tr><td><div id="setting-version" class="anchored"><code>version</code></div></td><td>version</td><td><code>1000022.2-78</code></td><td>set the active cluster version in the format &#39;&lt;major&gt;.&lt;minor&gt;&#39;</td></tr>
</tbody>
</table>
4 changes: 4 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -218,6 +218,7 @@ ALL_TESTS = [
"//pkg/kv/kvserver/kvflowcontrol/kvflowcontroller:kvflowcontroller_test",
"//pkg/kv/kvserver/kvflowcontrol/kvflowdispatch:kvflowdispatch_test",
"//pkg/kv/kvserver/kvflowcontrol/kvflowhandle:kvflowhandle_test",
"//pkg/kv/kvserver/kvflowcontrol/kvflowsequencer:kvflowsequencer_test",
"//pkg/kv/kvserver/kvflowcontrol/kvflowsimulator:kvflowsimulator_test",
"//pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker:kvflowtokentracker_test",
"//pkg/kv/kvserver/kvstorage:kvstorage_test",
Expand Down Expand Up @@ -1267,6 +1268,8 @@ GO_TARGETS = [
"//pkg/kv/kvserver/kvflowcontrol/kvflowdispatch:kvflowdispatch_test",
"//pkg/kv/kvserver/kvflowcontrol/kvflowhandle:kvflowhandle",
"//pkg/kv/kvserver/kvflowcontrol/kvflowhandle:kvflowhandle_test",
"//pkg/kv/kvserver/kvflowcontrol/kvflowsequencer:kvflowsequencer",
"//pkg/kv/kvserver/kvflowcontrol/kvflowsequencer:kvflowsequencer_test",
"//pkg/kv/kvserver/kvflowcontrol/kvflowsimulator:kvflowsimulator_test",
"//pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker:kvflowtokentracker",
"//pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker:kvflowtokentracker_test",
Expand Down Expand Up @@ -2694,6 +2697,7 @@ GET_X_DATA_TARGETS = [
"//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb:get_x_data",
"//pkg/kv/kvserver/kvflowcontrol/kvflowdispatch:get_x_data",
"//pkg/kv/kvserver/kvflowcontrol/kvflowhandle:get_x_data",
"//pkg/kv/kvserver/kvflowcontrol/kvflowsequencer:get_x_data",
"//pkg/kv/kvserver/kvflowcontrol/kvflowsimulator:get_x_data",
"//pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker:get_x_data",
"//pkg/kv/kvserver/kvserverbase:get_x_data",
Expand Down
2 changes: 1 addition & 1 deletion pkg/cli/testdata/declarative-rules/deprules
Original file line number Diff line number Diff line change
@@ -1,6 +1,6 @@
dep
----
debug declarative-print-rules 1000022.2-76 dep
debug declarative-print-rules 1000022.2-78 dep
deprules
----
- name: 'CheckConstraint transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED'
Expand Down
2 changes: 1 addition & 1 deletion pkg/cli/testdata/declarative-rules/oprules
Original file line number Diff line number Diff line change
@@ -1,6 +1,6 @@
op
----
debug declarative-print-rules 1000022.2-76 op
debug declarative-print-rules 1000022.2-78 op
rules
----
[]
10 changes: 10 additions & 0 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -473,6 +473,12 @@ const (
// has been backfilled.
V23_1ExternalConnectionsTableOwnerIDColumnBackfilled

// V23_1AllowNewSystemPrivileges is the version at which we allow the new
// MODIFYSQLCLUSTERSETTING abd VIEWJOB system privileges to be used.
// Note: After v23.1 is released, we won't need to version gate these anymore,
// since we've made mixed-version clusters tolerate new privileges.
V23_1AllowNewSystemPrivileges

// *************************************************
// Step (1): Add new versions here.
// Do not add new versions to a patch release.
Expand Down Expand Up @@ -818,6 +824,10 @@ var rawVersionsSingleton = keyedVersions{
Key: V23_1ExternalConnectionsTableOwnerIDColumnBackfilled,
Version: roachpb.Version{Major: 22, Minor: 2, Internal: 76},
},
{
Key: V23_1AllowNewSystemPrivileges,
Version: roachpb.Version{Major: 22, Minor: 2, Internal: 78},
},

// *************************************************
// Step (2): Add new versions here.
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/kvadmission/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@ go_library(
visibility = ["//visibility:public"],
deps = [
"//pkg/kv/kvpb",
"//pkg/kv/kvserver/raftlog",
"//pkg/roachpb",
"//pkg/settings",
"//pkg/settings/cluster",
Expand Down
63 changes: 59 additions & 4 deletions pkg/kv/kvserver/kvadmission/kvadmission.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"time"

"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftlog"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
Expand Down Expand Up @@ -120,7 +121,7 @@ type Controller interface {
FollowerStoreWriteBytes(roachpb.StoreID, FollowerStoreWriteBytes)
// AdmitRaftEntry informs admission control of a raft log entry being
// written to storage.
AdmitRaftEntry(roachpb.TenantID, roachpb.StoreID, roachpb.RangeID, raftpb.Entry)
AdmitRaftEntry(context.Context, roachpb.TenantID, roachpb.StoreID, roachpb.RangeID, raftpb.Entry)
}

// TenantWeightProvider can be periodically asked to provide the tenant
Expand Down Expand Up @@ -244,7 +245,7 @@ func (n *controllerImpl) AdmitKVWork(
if err != nil {
return Handle{}, err
}
admissionEnabled = storeWorkHandle.AdmissionEnabled()
admissionEnabled = storeWorkHandle.UseAdmittedWorkDone()
if admissionEnabled {
defer func() {
if retErr != nil {
Expand Down Expand Up @@ -409,9 +410,63 @@ func (n *controllerImpl) FollowerStoreWriteBytes(

// AdmitRaftEntry implements the Controller interface.
func (n *controllerImpl) AdmitRaftEntry(
roachpb.TenantID, roachpb.StoreID, roachpb.RangeID, raftpb.Entry,
ctx context.Context,
tenantID roachpb.TenantID,
storeID roachpb.StoreID,
rangeID roachpb.RangeID,
entry raftpb.Entry,
) {
panic("unimplemented")
typ, err := raftlog.EncodingOf(entry)
if err != nil {
log.Errorf(ctx, "unable to determine raft command encoding: %v", err)
return
}
if !typ.UsesAdmissionControl() {
return // nothing to do
}
meta, err := raftlog.DecodeRaftAdmissionMeta(entry.Data)
if err != nil {
log.Errorf(ctx, "unable to decode raft command admission data: %v", err)
return
}

storeAdmissionQ := n.storeGrantCoords.TryGetQueueForStore(int32(storeID))
if storeAdmissionQ == nil {
log.Errorf(ctx, "unable to find queue for store: %s", storeID)
return // nothing to do
}

if len(entry.Data) == 0 {
log.Fatal(ctx, "found (unexpected) empty raft command for below-raft admission")
}
wi := admission.WorkInfo{
TenantID: tenantID,
Priority: admissionpb.WorkPriority(meta.AdmissionPriority),
CreateTime: meta.AdmissionCreateTime,
BypassAdmission: false,
RequestedCount: int64(len(entry.Data)),
}
wi.ReplicatedWorkInfo = admission.ReplicatedWorkInfo{
Enabled: true,
RangeID: rangeID,
Origin: meta.AdmissionOriginNode,
LogPosition: admission.LogPosition{
Term: entry.Term,
Index: entry.Index,
},
Ingested: typ.IsSideloaded(),
}

handle, err := storeAdmissionQ.Admit(ctx, admission.StoreWriteWorkInfo{
WorkInfo: wi,
})
if err != nil {
log.Errorf(ctx, "error while admitting to store admission queue: %v", err)
return
}
if handle.UseAdmittedWorkDone() {
log.Fatalf(ctx, "unexpected handle.UseAdmittedWorkDone")
}
}

// FollowerStoreWriteBytes captures stats about writes done to a store by a
Expand Down
113 changes: 113 additions & 0 deletions pkg/kv/kvserver/kvflowcontrol/doc.go
Original file line number Diff line number Diff line change
Expand Up @@ -327,6 +327,105 @@ package kvflowcontrol
// it can transition into the mode described in I3a where we deduct/block for
// flow tokens for subsequent quorum writes.
//
// I12. How does this interact with epoch-LIFO? Or CreateTime ordering
// generally?
// - Background: Epoch-LIFO tries to reduce lower percentile admission queueing
// delay (at the expense of higher percentile delay) by switching between the
// standard CreateTime-based FIFO ordering to LIFO-within-an-epoch. Work
// submitted by a transaction with CreateTime T is slotted into the epoch
// number with time interval [E, E+100ms) where T is contained in this
// interval. The work will start executing after the epoch is closed, at
// ~E+100ms. This increases transaction latency by at most ~100ms. When the
// epoch closes, all nodes start executing the transactions in that epoch in
// LIFO order, and will implicitly have the same set of competing
// transactions[^10], a set that stays unchanged until the next epoch closes.
// And by the time the next epoch closes, and the current epoch's transactions
// are deprioritized, 100ms will have elapsed, which is selected to be long
// enough for most of these now-admitted to have finished their work.
// - We switch to LIFO-within-an-epoch once we start observing that the
// maximum queuing delay for work within a <tenant,priority> starts
// exceeding the ~100ms we'd add with epoch-LIFO.
// - The idea is we have bottleneck resources that cause delays without
// bound with if work keeps accumulating, and other kinds of bottlenecks
// where delays aren't increasing without bound. We're also relying on work
// bottlenecked on epoch-LIFO not being able to issue more work.
// - For below-raft work queue ordering, we effectively ignore the "true"
// CreateTime when ordering work. Within a given <tenant,priority,range>,
// we instead want admission takes place in raft log order (i.e. entries with
// lower terms to get admitted first, or lower indexes within the same term).
// This lets us simplifies token returns which happen by specifying a prefix
// up to which we want to release flow tokens for a given priority[^11].
// - NB: Regarding "admission takes place in raft log order", we could
// implement this differently. We introduced log-position based ordering to
// simplify the implementation of token returns where we release tokens by
// specifying the log position up-to-which we want to release held
// tokens[^12]. But with additional tracking in the below-raft work queues,
// if we know that work W2 with log position L2 got admitted, and
// corresponded to F flow token deductions at the origin, and we also know
// that work W1 with log position L1 is currently queued, also corresponding
// to F flow token deductions at the origin, we could inform the origin node
// to return flow tokens up to L1 and still get what we want -- a return of
// F flow tokens when each work gets admitted.
// - We effectively ignore "true" CreateTime since flow token deductions at
// the sender aren't tied to CreateTime in the way they're tied to the
// issuing tenant or the work class. So while we still want priority-based
// ordering to release regular flow tokens before elastic ones, releasing
// flow tokens for work with lower CreateTimes does not actually promote
// doing older work first since the physical work below-raft is already done
// before (asynchronous) admission, and the token returns don't unblock work
// from some given epoch. Below-raft ordering by "true" CreateTime is moot.
// - Note that for WorkQueue orderings, we have (i) fair sharing through
// tenantID+weight, (ii) strict prioritization, (iii) and sequencing of work
// within a <tenant,priority>, using CreateTime. For below-raft work
// queue ordering where we want to admit in roughly log position order, we
// then (ab)use the CreateTime sequencing by combining each work's true
// CreateTime with its log position[^13], to get a monotonic "sequencing
// timestamp" that tracks observed log positions. This sequencing timestamp is
// kept close to the maximum observed CreateTime within a replication stream,
// which also lets us generate cluster-wide FIFO ordering as follows.
// - We re-assign CreateTime in a way that, with a high probability, matches
// log position order. We can be imprecise/forgetful about this tracking
// since at worst we might over-admit slightly.
// - To operate within cluster-wide FIFO ordering, we want to order by
// "true" CreateTime when comparing work across different ranges. Writes for
// a single range, as observed by a given store below-raft (follower or
// otherwise) travel along a single stream. Consider the case where a single
// store S3 receives replication traffic for two ranges R1 and R2,
// originating from two separate nodes N1 and N2. If N1 is issuing writes
// with strictly older CreateTimes, when returning flow tokens we should
// prefer N1. By keeping these sequence numbers close to "true" CreateTimes,
// we'd be favoring N1 without introducing bias for replication streams with
// shorter/longer raft logs[^12][^14].
// - We could improve cluster-wide FIFO properties by introducing a
// WorkQueue-like data structure that simply orders by "true" CreateTime when
// acquiring flow tokens above raft.
// - Could we then use this above-raft ordering to implement epoch-LIFO?
// Cluster-wide we want to admit work within a given epoch, which here
// entails issuing replication traffic for work slotted in a given epoch.
// - Fan-in effects to consider for epoch-LIFO, assuming below-raft orderings
// are as described above (i.e. log-position based).
// - When there's no epoch-LIFO happening, we have cluster-wide FIFO
// ordering within a <tenant,priority> pair as described above.
// - When epoch-LIFO is happening across all senders issuing replication
// traffic to a given receiver store, we're seeing work within the same
// epoch, but we'll be returning flow tokens to nodes issuing work with
// older CreateTimes. So defeating the LIFO in epoch-LIFO, but we're still
// completing work slotted into some given epoch.
// - When epoch-LIFO is happening only on a subset of the senders issuing
// replication traffic, we'll again be returning flow tokens to nodes
// issuing work with older CreateTimes. This is undefined.
// - It's strange that different nodes can admit work from "different
// epochs"[^10]. What are we to do below-raft, when deciding where to
// return flow tokens back to, since it's all for the same
// <tenant,priority>? Maybe we need to pass down whether the work was
// admitted at the sender with LIFO/FIFO, and return flow tokens in
// {LIFO,FIFO} order across all nodes that issued {LIFO,FIFO} work? Or
// also pass down the enqueuing timestamp, so we have a good sense
// below-raft on whether this work is past the epoch expiration and
// should be deprioritized.
// - Because the fan-in effects of epoch-LIFO are not well understood (by this
// author at least), we just disable it below-raft.
//
// ---
//
// [^1]: kvserverpb.RaftMessageRequest is the unit of what's sent
Expand Down Expand Up @@ -373,6 +472,20 @@ package kvflowcontrol
// machine application get significantly behind due to local scheduling
// reasons by using the same goroutine to do both async raft log writes
// and state machine application.
// [^10]: This relies on partially synchronized clocks without requiring
// explicit coordination.
// - Isn't the decision to start using epoch-LIFO a node-local one, based
// on node-local max queuing latency for a <tenant,priority>? So what
// happens when work for a transaction is operating across multiple
// nodes, some using epoch-LIFO and some not?
// Is this why we use the max queuing latency as the trigger to switch
// into epoch-LIFO? All queued work for an epoch E across all nodes, if
// still queued after ~100ms, will trigger epoch-LIFO everywhere.
// [^11]: See the implementation for kvflowcontrol.Dispatch.
// [^12]: See UpToRaftLogPosition in AdmittedRaftLogEntries.
// [^13]: See kvflowsequencer.Sequencer and its use in kvflowhandle.Handle.
// [^14]: See the high_create_time_low_position_different_range test case for
// TestReplicatedWriteAdmission.
//
// TODO(irfansharif): These descriptions are too high-level, imprecise and
// possibly wrong. Fix that. After implementing these interfaces and integrating
Expand Down
14 changes: 11 additions & 3 deletions pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go
Original file line number Diff line number Diff line change
Expand Up @@ -98,8 +98,13 @@ type Handle interface {
// work with given priority along connected streams. The deduction is
// tracked with respect to the specific raft log position it's expecting it
// to end up in, log positions that monotonically increase. Requests are
// assumed to have been Admit()-ed first.
DeductTokensFor(context.Context, admissionpb.WorkPriority, kvflowcontrolpb.RaftLogPosition, Tokens)
// assumed to have been Admit()-ed first. The returned time.Time parameter
// is to be used as the work item's CreateTime when enqueueing in IO
// admission queues.
DeductTokensFor(
context.Context, admissionpb.WorkPriority, time.Time,
kvflowcontrolpb.RaftLogPosition, Tokens,
) time.Time
// 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
Expand All @@ -110,7 +115,10 @@ type Handle interface {
// the same raft term -- we want to both free up tokens from when we lost
// the lease, and also ensure we discard attempts to return them (on hearing
// about AdmittedRaftLogEntries replicated under the earlier lease).
ReturnTokensUpto(context.Context, admissionpb.WorkPriority, kvflowcontrolpb.RaftLogPosition, Stream)
ReturnTokensUpto(
context.Context, admissionpb.WorkPriority,
kvflowcontrolpb.RaftLogPosition, Stream,
)
// ConnectStream connects a stream (typically pointing to an active member
// of the raft group) to the handle. Subsequent calls to Admit() will block
// until flow tokens are available for the stream, or for it to be
Expand Down
2 changes: 2 additions & 0 deletions pkg/kv/kvserver/kvflowcontrol/kvflowhandle/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ go_library(
"//pkg/base",
"//pkg/kv/kvserver/kvflowcontrol",
"//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb",
"//pkg/kv/kvserver/kvflowcontrol/kvflowsequencer",
"//pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker",
"//pkg/util/admission/admissionpb",
"//pkg/util/hlc",
Expand All @@ -39,6 +40,7 @@ go_test(
"//pkg/util/leaktest",
"//pkg/util/log",
"//pkg/util/metric",
"//pkg/util/timeutil",
"@com_github_stretchr_testify//require",
],
)
Expand Down
Loading

0 comments on commit 2d63378

Please sign in to comment.