Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
131505: storeliveness: add SupportManager metrics r=nvanbenschoten a=miraradeva

This commit adds 9 new metrics for the Store Liveness `SupportManager`:

- `storeliveness.heartbeat.successes`: Number of Store Liveness heartbeats sent out by the Store Liveness Support Manager.

- `storeliveness.heartbeat.failures`: Number of Store Liveness heartbeats that failed to be sent out by the Store Liveness Support Manager.

- `storeliveness.message_handle.successes`: Number of incoming Store Liveness messages handled by the Store Liveness Support Manager.

- `storeliveness.message_handle.failures`: Number of incoming Store Liveness messages that failed to be handled by the Store Liveness Support Manager.

- `storeliveness.support_withdraw.successes`: Number of times the Store Liveness Support Manager has successfully withdrawn support for another store.

- `storeliveness.support_withdraw.failures`: Number of times the Store Liveness Support Manager has encountered an error while withdrawing support for another store.

- `storeliveness.support_from.stores`: Number of stores that the Store Liveness Support Manager is requesting support from by sending heartbeats.

- `storeliveness.transport.receive-queue-size`: Number of pending incoming messages from the Store Liveness Transport.

- `storeliveness.transport.receive-queue-bytes`: Total byte size of pending incoming messages from Store Liveness Transport.

Fixes: cockroachdb#125067

Release note: None

131642: *: remove the `txn.restarts.writetoooldmulti` metric. r=nvanbenschoten a=nicktrav

The `txn.restarts.writetoooldmulti` metric became a subset of the `writetooold` metric in cockroachdb#11941. This effectively rendered it obsolete.

Remove the metric.

Fix cockroachdb#119413.

Epic: CRDB-37617.

Release note (ui change): The "Write Too Old (multiple)" metric was rolled into the "Write Too Old" error in a previous release. The metric has now been removed altogether.

131657: CODEOWNERS: update server/application_api/ to o11y r=kyle-a-wong a=kyle-a-wong

Informs: cockroachdb#131630
Release note: None

Co-authored-by: Mira Radeva <[email protected]>
Co-authored-by: Nick Travers <[email protected]>
Co-authored-by: Kyle Wong <[email protected]>
  • Loading branch information
4 people committed Oct 1, 2024
4 parents 07657c1 + f095444 + 48f0004 + 49fd97e commit 5e8869e
Show file tree
Hide file tree
Showing 24 changed files with 274 additions and 84 deletions.
1 change: 1 addition & 0 deletions .github/CODEOWNERS
Validating CODEOWNERS rules …
Original file line number Diff line number Diff line change
Expand Up @@ -181,6 +181,7 @@
/pkg/server/admin*.go @cockroachdb/obs-prs @cockroachdb/server-prs
/pkg/server/api_v2*.go @cockroachdb/obs-prs @cockroachdb/server-prs
/pkg/server/api_v2_auth*.go @cockroachdb/obs-prs @cockroachdb/server-prs @cockroachdb/prodsec @cockroachdb/product-security
/pkg/server/application_api/ @cockroachdb/obs-prs
/pkg/server/authentication*.go @cockroachdb/prodsec @cockroachdb/server-prs @cockroachdb/product-security
/pkg/server/clock_monotonicity.go @cockroachdb/kv-prs
/pkg/server/combined_statement_stats*.go @cockroachdb/obs-prs
Expand Down
11 changes: 10 additions & 1 deletion docs/generated/metrics/metrics.html
Original file line number Diff line number Diff line change
Expand Up @@ -771,6 +771,16 @@
<tr><td>STORAGE</td><td>storage.wal.fsync.latency</td><td>The write ahead log fsync latency</td><td>Fsync Latency</td><td>HISTOGRAM</td><td>NANOSECONDS</td><td>AVG</td><td>NONE</td></tr>
<tr><td>STORAGE</td><td>storage.write-stall-nanos</td><td>Total write stall duration in nanos</td><td>Nanoseconds</td><td>GAUGE</td><td>NANOSECONDS</td><td>AVG</td><td>NONE</td></tr>
<tr><td>STORAGE</td><td>storage.write-stalls</td><td>Number of instances of intentional write stalls to backpressure incoming writes</td><td>Events</td><td>GAUGE</td><td>COUNT</td><td>AVG</td><td>NONE</td></tr>
<tr><td>STORAGE</td><td>storeliveness.heartbeat.failures</td><td>Number of Store Liveness heartbeats that failed to be sent out by the Store Liveness Support Manager</td><td>Heartbeats</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>STORAGE</td><td>storeliveness.heartbeat.successes</td><td>Number of Store Liveness heartbeats sent out by the Store Liveness Support Manager</td><td>Heartbeats</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>STORAGE</td><td>storeliveness.message_handle.failures</td><td>Number of incoming Store Liveness messages that failed to be handled by the Store Liveness Support Manager</td><td>Messages</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>STORAGE</td><td>storeliveness.message_handle.successes</td><td>Number of incoming Store Liveness messages handled by the Store Liveness Support Manager</td><td>Messages</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>STORAGE</td><td>storeliveness.support_for.stores</td><td>Number of stores that the Store Liveness Support Manager has ever provided support for</td><td>Stores</td><td>GAUGE</td><td>COUNT</td><td>AVG</td><td>NONE</td></tr>
<tr><td>STORAGE</td><td>storeliveness.support_from.stores</td><td>Number of stores that the Store Liveness Support Manager is requesting support from by sending heartbeats</td><td>Stores</td><td>GAUGE</td><td>COUNT</td><td>AVG</td><td>NONE</td></tr>
<tr><td>STORAGE</td><td>storeliveness.support_withdraw.failures</td><td>Number of times the Store Liveness Support Manager has encountered an error while withdrawing support for another store</td><td>Support Withdrawals</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>STORAGE</td><td>storeliveness.support_withdraw.successes</td><td>Number of times the Store Liveness Support Manager has successfully withdrawn support for another store</td><td>Support Withdrawals</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>STORAGE</td><td>storeliveness.transport.receive-queue-bytes</td><td>Total byte size of pending incoming messages from Store Liveness Transport</td><td>Bytes</td><td>GAUGE</td><td>BYTES</td><td>AVG</td><td>NONE</td></tr>
<tr><td>STORAGE</td><td>storeliveness.transport.receive-queue-size</td><td>Number of pending incoming messages from the Store Liveness Transport</td><td>Messages</td><td>GAUGE</td><td>COUNT</td><td>AVG</td><td>NONE</td></tr>
<tr><td>STORAGE</td><td>storeliveness.transport.receive_dropped</td><td>Number of Store Liveness messages dropped by the Store Liveness Transport on the receiver side</td><td>Messages</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>STORAGE</td><td>storeliveness.transport.received</td><td>Number of Store Liveness messages received by the Store Liveness Transport</td><td>Messages</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>STORAGE</td><td>storeliveness.transport.send-queue-bytes</td><td>Total byte size of pending outgoing messages in all Store Liveness Transport per-store send queues</td><td>Bytes</td><td>GAUGE</td><td>BYTES</td><td>AVG</td><td>NONE</td></tr>
Expand Down Expand Up @@ -1780,7 +1790,6 @@
<tr><td>APPLICATION</td><td>txn.restarts.txnpush</td><td>Number of restarts due to a transaction push failure</td><td>Restarted Transactions</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>APPLICATION</td><td>txn.restarts.unknown</td><td>Number of restarts due to a unknown reasons</td><td>Restarted Transactions</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>APPLICATION</td><td>txn.restarts.writetooold</td><td>Number of restarts due to a concurrent writer committing first</td><td>Restarted Transactions</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>APPLICATION</td><td>txn.restarts.writetoooldmulti</td><td>Number of restarts due to multiple concurrent writers committing first</td><td>Restarted Transactions</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>APPLICATION</td><td>txn.rollbacks.async.failed</td><td>Number of KV transaction that failed to send abort asynchronously which is not always retried</td><td>KV Transactions</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>APPLICATION</td><td>txn.rollbacks.failed</td><td>Number of KV transaction that failed to send final abort</td><td>KV Transactions</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>SERVER</td><td>build.timestamp</td><td>Build information</td><td>Build Time</td><td>GAUGE</td><td>TIMESTAMP_SEC</td><td>AVG</td><td>NONE</td></tr>
Expand Down
12 changes: 0 additions & 12 deletions monitoring/grafana-dashboards/by-cluster/distributed.json
Original file line number Diff line number Diff line change
Expand Up @@ -505,18 +505,6 @@
"legendFormat": "Write Too Old",
"refId": "A"
},
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
"exemplar": true,
"expr": "sum(rate(txn_restarts_writetoooldmulti{cluster=\"$cluster\",job=\"cockroachdb\",instance=~\"$node\"}[$__rate_interval]))",
"hide": false,
"interval": "",
"legendFormat": "Write Too Old (multiple)",
"refId": "B"
},
{
"datasource": {
"type": "prometheus",
Expand Down
12 changes: 0 additions & 12 deletions monitoring/grafana-dashboards/by-roachtest/distributed.json
Original file line number Diff line number Diff line change
Expand Up @@ -520,18 +520,6 @@
"legendFormat": "Write Too Old",
"refId": "A"
},
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
"exemplar": true,
"expr": "sum(rate(txn_restarts_writetoooldmulti{test_run_id=~\"$test_run_id\", test_name=~\"$test_name\",job=\"cockroachdb\",instance=~\"$node\"}[$__rate_interval]))",
"hide": false,
"interval": "",
"legendFormat": "Write Too Old (multiple)",
"refId": "B"
},
{
"datasource": {
"type": "prometheus",
Expand Down
3 changes: 1 addition & 2 deletions monitoring/splunk-dashboard/sql.xml
Original file line number Diff line number Diff line change
Expand Up @@ -843,7 +843,6 @@ sum(abort) as abort</query>
<query>| mstats
rate_sum(txn_restarts_asyncwritefailure) as async_failure,
rate_sum(txn_restarts_writetooold_count) as write_too_old,
rate_sum(txn_restarts_writetoooldmulti_count) as write_too_old_multi
rate_sum(txn_restarts_commitdeadlineexceeded_count) as commit_deadline_exceeded ,
rate_sum(txn_restarts_readwithinuncertainty_count) as read_within_uncertainty_interval,
rate_sum(txn_restarts_txnaborted_count) as aborted ,
Expand Down Expand Up @@ -1169,4 +1168,4 @@ span=10s
</chart>
</panel>
</row>
</form>
</form>
12 changes: 0 additions & 12 deletions pkg/cmd/roachprod/grafana/configs/distributed.json
Original file line number Diff line number Diff line change
Expand Up @@ -505,18 +505,6 @@
"legendFormat": "Write Too Old",
"refId": "A"
},
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
"exemplar": true,
"expr": "sum(rate(txn_restarts_writetoooldmulti{cluster=\"$cluster\",job=\"cockroachdb\",instance=~\"$node\"}[$__rate_interval]))",
"hide": false,
"interval": "",
"legendFormat": "Write Too Old (multiple)",
"refId": "B"
},
{
"datasource": {
"type": "prometheus",
Expand Down
7 changes: 0 additions & 7 deletions pkg/kv/kvclient/kvcoord/txn_coord_sender.go
Original file line number Diff line number Diff line change
Expand Up @@ -842,13 +842,6 @@ func (tc *TxnCoordSender) handleRetryableErrLocked(ctx context.Context, pErr *kv

case *kvpb.WriteTooOldError:
tc.metrics.RestartsWriteTooOld.Inc()
// Also increment the deprecated "txn.restarts.writetoooldmulti" metric.
// We preserve this for one release to avoid confusion in mixed-version
// clusters.
// TODO(nvanbenschoten): remove this when compatibility with v23.2 is no
// longer needed. At the same time, also remove the corresponding metric
// from the "Transaction Restarts" graph in the SQL Dashboard.
tc.metrics.RestartsWriteTooOldMulti.Inc()

case *kvpb.ReadWithinUncertaintyIntervalError:
tc.metrics.RestartsReadWithinUncertainty.Inc()
Expand Down
16 changes: 0 additions & 16 deletions pkg/kv/kvclient/kvcoord/txn_metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,6 @@ type TxnMetrics struct {

// Counts of restart types.
RestartsWriteTooOld telemetry.CounterWithMetric
RestartsWriteTooOldMulti telemetry.CounterWithMetric
RestartsSerializable telemetry.CounterWithMetric
RestartsAsyncWriteFailure telemetry.CounterWithMetric
RestartsCommitDeadlineExceeded telemetry.CounterWithMetric
Expand Down Expand Up @@ -201,26 +200,12 @@ var (
// the WriteTooOld flag is set on the Transaction, which causes EndTxn to
// return a/ TransactionRetryError with RETRY_WRITE_TOO_OLD. These are
// captured as txn.restarts.writetooold.
//
// If the Store's retried operation generates a second WriteTooOldError
// (indicating a conflict with a third transaction with a higher timestamp
// than the one that caused the first WriteTooOldError), the store doesn't
// retry again, and the WriteTooOldError will be returned up the stack to be
// retried at this level. These are captured as
// txn.restarts.writetoooldmulti. This path is inefficient, and if it turns
// out to be common we may want to do something about it.
metaRestartsWriteTooOld = metric.Metadata{
Name: "txn.restarts.writetooold",
Help: "Number of restarts due to a concurrent writer committing first",
Measurement: "Restarted Transactions",
Unit: metric.Unit_COUNT,
}
metaRestartsWriteTooOldMulti = metric.Metadata{
Name: "txn.restarts.writetoooldmulti",
Help: "Number of restarts due to multiple concurrent writers committing first",
Measurement: "Restarted Transactions",
Unit: metric.Unit_COUNT,
}
metaRestartsSerializable = metric.Metadata{
Name: "txn.restarts.serializable",
Help: "Number of restarts due to a forwarded commit timestamp and isolation=SERIALIZABLE",
Expand Down Expand Up @@ -318,7 +303,6 @@ func MakeTxnMetrics(histogramWindow time.Duration) TxnMetrics {
BucketConfig: metric.Count1KBuckets,
}),
RestartsWriteTooOld: telemetry.NewCounterWithMetric(metaRestartsWriteTooOld),
RestartsWriteTooOldMulti: telemetry.NewCounterWithMetric(metaRestartsWriteTooOldMulti),
RestartsSerializable: telemetry.NewCounterWithMetric(metaRestartsSerializable),
RestartsAsyncWriteFailure: telemetry.NewCounterWithMetric(metaRestartsAsyncWriteFailure),
RestartsCommitDeadlineExceeded: telemetry.NewCounterWithMetric(metaRestartsCommitDeadlineExceeded),
Expand Down
2 changes: 2 additions & 0 deletions pkg/kv/kvserver/store.go
Original file line number Diff line number Diff line change
Expand Up @@ -2187,6 +2187,7 @@ func (s *Store) Start(ctx context.Context, stopper *stop.Stopper) error {
)
s.metrics.registry.AddMetricStruct(s.recoveryMgr.Metrics())

// Create the Store Liveness SupportManager.
livenessInterval, heartbeatInterval := s.cfg.StoreLivenessDurations()
supportGracePeriod := s.cfg.RPCContext.StoreLivenessWithdrawalGracePeriod()
options := storeliveness.NewOptions(livenessInterval, heartbeatInterval, supportGracePeriod)
Expand All @@ -2196,6 +2197,7 @@ func (s *Store) Start(ctx context.Context, stopper *stop.Stopper) error {
)
s.cfg.StoreLivenessTransport.ListenMessages(s.StoreID(), sm)
s.storeLiveness = sm
s.metrics.registry.AddMetricStruct(sm.Metrics())
if err = sm.Start(ctx); err != nil {
return errors.Wrap(err, "starting store liveness")
}
Expand Down
100 changes: 100 additions & 0 deletions pkg/kv/kvserver/storeliveness/metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,36 @@ func newTransportMetrics() *TransportMetrics {
}
}

// SupportManagerMetrics includes all Store Liveness SupportManager metrics.
type SupportManagerMetrics struct {
HeartbeatSuccesses *metric.Counter
HeartbeatFailures *metric.Counter
MessageHandleSuccesses *metric.Counter
MessageHandleFailures *metric.Counter
SupportWithdrawSuccesses *metric.Counter
SupportWithdrawFailures *metric.Counter
SupportFromStores *metric.Gauge
SupportForStores *metric.Gauge

ReceiveQueueSize *metric.Gauge
ReceiveQueueBytes *metric.Gauge
}

func newSupportManagerMetrics() *SupportManagerMetrics {
return &SupportManagerMetrics{
HeartbeatSuccesses: metric.NewCounter(metaHeartbeatSuccesses),
HeartbeatFailures: metric.NewCounter(metaHeartbeatFailures),
MessageHandleSuccesses: metric.NewCounter(metaMessageHandleSuccesses),
MessageHandleFailures: metric.NewCounter(metaMessageHandleFailures),
SupportWithdrawSuccesses: metric.NewCounter(metaSupportWithdrawSuccesses),
SupportWithdrawFailures: metric.NewCounter(metaSupportWithdrawFailures),
SupportFromStores: metric.NewGauge(metaSupportFromStores),
SupportForStores: metric.NewGauge(metaSupportForStores),
ReceiveQueueSize: metric.NewGauge(metaReceiveQueueSize),
ReceiveQueueBytes: metric.NewGauge(metaReceiveQueueBytes),
}
}

var (
metaSendQueueSize = metric.Metadata{
Name: "storeliveness.transport.send-queue-size",
Expand Down Expand Up @@ -86,4 +116,74 @@ var (
Measurement: "Messages",
Unit: metric.Unit_COUNT,
}
metaHeartbeatSuccesses = metric.Metadata{
Name: "storeliveness.heartbeat.successes",
Help: "Number of Store Liveness heartbeats sent out by the " +
"Store Liveness Support Manager",
Measurement: "Heartbeats",
Unit: metric.Unit_COUNT,
}
metaHeartbeatFailures = metric.Metadata{
Name: "storeliveness.heartbeat.failures",
Help: "Number of Store Liveness heartbeats that failed to be sent out by the " +
"Store Liveness Support Manager",
Measurement: "Heartbeats",
Unit: metric.Unit_COUNT,
}
metaMessageHandleSuccesses = metric.Metadata{
Name: "storeliveness.message_handle.successes",
Help: "Number of incoming Store Liveness messages handled by the " +
"Store Liveness Support Manager",
Measurement: "Messages",
Unit: metric.Unit_COUNT,
}
metaMessageHandleFailures = metric.Metadata{
Name: "storeliveness.message_handle.failures",
Help: "Number of incoming Store Liveness messages that failed to be handled by the " +
"Store Liveness Support Manager",
Measurement: "Messages",
Unit: metric.Unit_COUNT,
}
metaSupportWithdrawSuccesses = metric.Metadata{
Name: "storeliveness.support_withdraw.successes",
Help: "Number of times the Store Liveness Support Manager has successfully withdrawn " +
"support for another store",
Measurement: "Support Withdrawals",
Unit: metric.Unit_COUNT,
}
metaSupportWithdrawFailures = metric.Metadata{
Name: "storeliveness.support_withdraw.failures",
Help: "Number of times the Store Liveness Support Manager has encountered an error " +
"while withdrawing support for another store",
Measurement: "Support Withdrawals",
Unit: metric.Unit_COUNT,
}
metaSupportFromStores = metric.Metadata{
Name: "storeliveness.support_from.stores",
Help: "Number of stores that the Store Liveness Support Manager is requesting " +
"support from by sending heartbeats",
Measurement: "Stores",
Unit: metric.Unit_COUNT,
}
metaSupportForStores = metric.Metadata{
Name: "storeliveness.support_for.stores",
Help: "Number of stores that the Store Liveness Support Manager has ever " +
"provided support for",
Measurement: "Stores",
Unit: metric.Unit_COUNT,
}
metaReceiveQueueSize = metric.Metadata{
Name: "storeliveness.transport.receive-queue-size",
Help: "Number of pending incoming messages from the " +
"Store Liveness Transport",
Measurement: "Messages",
Unit: metric.Unit_COUNT,
}
metaReceiveQueueBytes = metric.Metadata{
Name: "storeliveness.transport.receive-queue-bytes",
Help: "Total byte size of pending incoming messages from " +
"Store Liveness Transport",
Measurement: "Bytes",
Unit: metric.Unit_BYTES,
}
)
5 changes: 4 additions & 1 deletion pkg/kv/kvserver/storeliveness/requester_state.go
Original file line number Diff line number Diff line change
Expand Up @@ -155,7 +155,8 @@ func (rsh *requesterStateHandler) getSupportFrom(id slpb.StoreIdent) (slpb.Suppo
}

// addStore adds a store to the requesterState.supportFrom map, if not present.
func (rsh *requesterStateHandler) addStore(id slpb.StoreIdent) {
// The function returns a boolean indicating whether the store was added.
func (rsh *requesterStateHandler) addStore(id slpb.StoreIdent) bool {
// Adding a store doesn't require persisting anything to disk, so it doesn't
// need to go through the full checkOut/checkIn process. However, we still
// check out the update to ensure that there are no concurrent updates.
Expand All @@ -171,7 +172,9 @@ func (rsh *requesterStateHandler) addStore(id slpb.StoreIdent) {
// be removed immediately after adding.
rs.recentlyQueried.Store(active)
rsh.requesterState.supportFrom[id] = &rs
return true
}
return false
}

// markIdleStores marks all stores in the requesterState.supportFrom map as
Expand Down
15 changes: 15 additions & 0 deletions pkg/kv/kvserver/storeliveness/store_liveness_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -135,6 +135,21 @@ func TestStoreLiveness(t *testing.T) {
strings.Join(sortedSupportMap, "\n"),
)

case "debug-metrics":
return fmt.Sprintf(
"HeartbeatSuccess: %d, HeartbeatFailure: %d\n"+
"MessageHandleSuccess: %d, MessageHandleFailure: %d\n"+
"SupportWithdrawSuccess: %d, SupportWithdrawFailure: %d\n"+
"SupportFromStores: %d, SupportForStores: %d",
sm.metrics.HeartbeatSuccesses.Count(),
sm.metrics.HeartbeatFailures.Count(),
sm.metrics.MessageHandleSuccesses.Count(),
sm.metrics.MessageHandleFailures.Count(),
sm.metrics.SupportWithdrawSuccesses.Count(),
sm.metrics.SupportWithdrawFailures.Count(),
sm.metrics.SupportFromStores.Value(),
sm.metrics.SupportForStores.Value(),
)
default:
return fmt.Sprintf("unknown command: %s", d.Cmd)
}
Expand Down
Loading

0 comments on commit 5e8869e

Please sign in to comment.