Skip to content

Commit

Permalink
admission: support non-blocking {Store,}WorkQueue.Admit()
Browse files Browse the repository at this point in the history
Part of cockroachdb#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:

  // 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].
  - In WorkQueue ordering -- for replicated writes below-raft, we ignore
    CreateTime/epoch-LIFO, and instead sort by priority and within a
    priority, sort by log position.
- 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:

    // 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:

  // 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 cockroachdb#95637.
[^2]: See kvflowcontrol.Handle.{ReturnTokensUpto,DeductTokensFor}
      introduced in cockroachdb#95637. Token deductions and returns are tied to
      raft log positions.
[^3]: See raftlog.EntryEncoding{Standard,Sideloaded}WithAC introduced in
      cockroachdb#95748.
[^4]: See kvflowcontrolpb.RaftAdmissionMeta introduced in cockroachdb#95637.
        message RaftAdmissionMeta {
          int32 admission_priority = ...;
          int64 admission_create_time = ...;
          int32 admission_origin_node = ...;
        }

Release note: None
  • Loading branch information
irfansharif committed Mar 9, 2023
1 parent c705d32 commit 842ee1a
Show file tree
Hide file tree
Showing 32 changed files with 1,864 additions and 163 deletions.
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 CreateTime when
// ordering work. Within a given <tenant,priority,range>, admission takes
// place in raft log order (i.e. entries with lower terms get admitted first,
// or lower indexes within the same term). This 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're effectively ignoring 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 based on 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 "sequence number"
// that tracks observed log positions. This sequence number 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 forgetful about this tracking (for internal
// GC reasons) since at wt 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 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 the sequencer in pkg/util/admission.
// [^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
2 changes: 1 addition & 1 deletion pkg/server/node.go
Original file line number Diff line number Diff line change
Expand Up @@ -919,7 +919,7 @@ func (n *Node) GetPebbleMetrics() []admission.StoreMetrics {
diskStats = s
}
metrics = append(metrics, admission.StoreMetrics{
StoreID: int32(store.StoreID()),
StoreID: store.StoreID(),
Metrics: m.Metrics,
WriteStallCount: m.WriteStallCount,
DiskStats: diskStats})
Expand Down
5 changes: 5 additions & 0 deletions pkg/util/admission/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -15,8 +15,10 @@ go_library(
"kv_slot_adjuster.go",
"pacer.go",
"scheduler_latency_listener.go",
"sequencer.go",
"sql_cpu_overload_indicator.go",
"store_token_estimation.go",
"testing_knobs.go",
"tokens_linear_model.go",
"work_queue.go",
],
Expand Down Expand Up @@ -53,7 +55,9 @@ go_test(
"elastic_cpu_work_queue_test.go",
"granter_test.go",
"io_load_listener_test.go",
"replicated_write_admission_test.go",
"scheduler_latency_listener_test.go",
"sequencer_test.go",
"store_token_estimation_test.go",
"tokens_linear_model_test.go",
"work_queue_test.go",
Expand All @@ -67,6 +71,7 @@ go_test(
"//pkg/testutils/datapathutils",
"//pkg/testutils/echotest",
"//pkg/util/admission/admissionpb",
"//pkg/util/humanizeutil",
"//pkg/util/leaktest",
"//pkg/util/log",
"//pkg/util/metric",
Expand Down
7 changes: 4 additions & 3 deletions pkg/util/admission/admission.go
Original file line number Diff line number Diff line change
Expand Up @@ -522,8 +522,9 @@ func workKindString(workKind WorkKind) string {
// all of these when StoreWorkQueue.AdmittedWorkDone is called, so that these
// cumulative values are mutually consistent.
type storeAdmissionStats struct {
// Total requests that called AdmittedWorkDone or BypassedWorkDone.
admittedCount uint64
// Total requests that called {Admitted,Bypassed}WorkDone, or in the case of
// replicated writes, the requests that called Admit.
workCount uint64
// Sum of StoreWorkDoneInfo.WriteBytes.
//
// TODO(sumeer): writeAccountedBytes and ingestedAccountedBytes are not
Expand All @@ -548,7 +549,7 @@ type storeAdmissionStats struct {
// (e.g. for logging).
aux struct {
// These bypassed numbers are already included in the corresponding
// {admittedCount, writeAccountedBytes, ingestedAccountedBytes}.
// {workCount, writeAccountedBytes, ingestedAccountedBytes}.
bypassedCount uint64
writeBypassedAccountedBytes uint64
ingestedBypassedAccountedBytes uint64
Expand Down
2 changes: 1 addition & 1 deletion pkg/util/admission/elastic_cpu_work_queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,7 @@ func (e *ElasticCPUWorkQueue) Admit(
if duration > MaxElasticCPUDuration {
duration = MaxElasticCPUDuration
}
info.requestedCount = duration.Nanoseconds()
info.RequestedCount = duration.Nanoseconds()
enabled, err := e.workQueue.Admit(ctx, info)
if err != nil {
return nil, err
Expand Down
2 changes: 1 addition & 1 deletion pkg/util/admission/elastic_cpu_work_queue_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -161,7 +161,7 @@ func (t *testElasticCPUInternalWorkQueue) Admit(
_ context.Context, info WorkInfo,
) (enabled bool, err error) {
if !t.disabled {
t.buf.WriteString(fmt.Sprintf("admitted=%s ", time.Duration(info.requestedCount)))
t.buf.WriteString(fmt.Sprintf("admitted=%s ", time.Duration(info.RequestedCount)))
}
return !t.disabled, nil
}
Expand Down
11 changes: 6 additions & 5 deletions pkg/util/admission/grant_coordinator.go
Original file line number Diff line number Diff line change
Expand Up @@ -113,7 +113,7 @@ func (sgc *StoreGrantCoordinators) SetPebbleMetricsProvider(
if unsafeGc, ok := sgc.gcMap.Load(int64(m.StoreID)); ok {
gc := (*GrantCoordinator)(unsafeGc)
gc.pebbleMetricsTick(ctx, m)
iotc.UpdateIOThreshold(roachpb.StoreID(m.StoreID), gc.ioLoadListener.ioThreshold)
iotc.UpdateIOThreshold(m.StoreID, gc.ioLoadListener.ioThreshold)
} else {
log.Warningf(ctx,
"seeing metrics for unknown storeID %d", m.StoreID)
Expand All @@ -135,7 +135,7 @@ func (sgc *StoreGrantCoordinators) SetPebbleMetricsProvider(
}()
}

func (sgc *StoreGrantCoordinators) initGrantCoordinator(storeID int32) *GrantCoordinator {
func (sgc *StoreGrantCoordinators) initGrantCoordinator(storeID roachpb.StoreID) *GrantCoordinator {
coord := &GrantCoordinator{
settings: sgc.settings,
useGrantChains: false,
Expand Down Expand Up @@ -168,7 +168,7 @@ func (sgc *StoreGrantCoordinators) initGrantCoordinator(storeID int32) *GrantCoo
},
}

storeReq := sgc.makeStoreRequesterFunc(sgc.ambientCtx, granters, sgc.settings, sgc.workQueueMetrics, opts)
storeReq := sgc.makeStoreRequesterFunc(sgc.ambientCtx, storeID, granters, sgc.settings, sgc.workQueueMetrics, opts, nil)
coord.queues[KVWork] = storeReq
requesters := storeReq.getRequesters()
kvg.regularRequester = requesters[admissionpb.RegularWorkClass]
Expand Down Expand Up @@ -336,8 +336,9 @@ type makeRequesterFunc func(
metrics *WorkQueueMetrics, opts workQueueOptions) requester

type makeStoreRequesterFunc func(
_ log.AmbientContext, granters [admissionpb.NumWorkClasses]granterWithStoreWriteDone,
settings *cluster.Settings, metrics *WorkQueueMetrics, opts workQueueOptions) storeRequester
_ log.AmbientContext, storeID roachpb.StoreID, granters [admissionpb.NumWorkClasses]granterWithStoreWriteDone,
settings *cluster.Settings, metrics *WorkQueueMetrics, opts workQueueOptions, knobs *TestingKnobs,
) storeRequester

// NewGrantCoordinators constructs GrantCoordinators and WorkQueues for a
// regular cluster node. Caller is responsible for:
Expand Down
2 changes: 1 addition & 1 deletion pkg/util/admission/granter.go
Original file line number Diff line number Diff line change
Expand Up @@ -594,7 +594,7 @@ type IOThresholdConsumer interface {

// StoreMetrics are the metrics for a store.
type StoreMetrics struct {
StoreID int32
StoreID roachpb.StoreID
*pebble.Metrics
WriteStallCount int64
// Optional.
Expand Down
11 changes: 6 additions & 5 deletions pkg/util/admission/granter_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -109,8 +109,9 @@ func TestGranterBasic(t *testing.T) {
storeCoordinators := &StoreGrantCoordinators{
settings: settings,
makeStoreRequesterFunc: func(
ambientCtx log.AmbientContext, granters [admissionpb.NumWorkClasses]granterWithStoreWriteDone,
settings *cluster.Settings, metrics *WorkQueueMetrics, opts workQueueOptions) storeRequester {
ambientCtx log.AmbientContext, _ roachpb.StoreID, granters [admissionpb.NumWorkClasses]granterWithStoreWriteDone,
settings *cluster.Settings, metrics *WorkQueueMetrics, opts workQueueOptions, knobs *TestingKnobs,
) storeRequester {
makeTestRequester := func(wc admissionpb.WorkClass) *testRequester {
req := &testRequester{
workKind: KVWork,
Expand Down Expand Up @@ -273,8 +274,8 @@ func TestStoreCoordinators(t *testing.T) {
opts := Options{
makeRequesterFunc: makeRequesterFunc,
makeStoreRequesterFunc: func(
ctx log.AmbientContext, granters [admissionpb.NumWorkClasses]granterWithStoreWriteDone,
settings *cluster.Settings, metrics *WorkQueueMetrics, opts workQueueOptions) storeRequester {
ctx log.AmbientContext, _ roachpb.StoreID, granters [admissionpb.NumWorkClasses]granterWithStoreWriteDone,
settings *cluster.Settings, metrics *WorkQueueMetrics, opts workQueueOptions, _ *TestingKnobs) storeRequester {
reqReg := makeRequesterFunc(ctx, KVWork, granters[admissionpb.RegularWorkClass], settings, metrics, opts)
reqElastic := makeRequesterFunc(ctx, KVWork, granters[admissionpb.ElasticWorkClass], settings, metrics, opts)
str := &storeTestRequester{}
Expand Down Expand Up @@ -444,7 +445,7 @@ func (m *testMetricsProvider) setMetricsForStores(stores []int32, metrics pebble
m.metrics = m.metrics[:0]
for _, s := range stores {
m.metrics = append(m.metrics, StoreMetrics{
StoreID: s,
StoreID: roachpb.StoreID(s),
Metrics: &metrics,
})
}
Expand Down
Loading

0 comments on commit 842ee1a

Please sign in to comment.