Skip to content

Commit

Permalink
Merge #95905
Browse files Browse the repository at this point in the history
95905: kvflowcontroller: implement kvflowcontrol.Controller r=irfansharif a=irfansharif

Part of #95563.

kvflowcontroller.Controller is a concrete implementation of the kvflowcontrol.Controller interface. It provides flow control for replication traffic in KV and is typically held at the node-level. Internally it maintain flow token buckets for {regular,elastic} work traveling along each kvflowcontrol.Stream. When work tries to Admit(), the controller blocks until there are flow tokens available the specified stream/work class.

For each stream we maintain two flow token buckets, one for each work class (regular and elastic). To provide coarse grained prioritization, we want the following properties:
- When elastic tokens are deducted/returned, they only affect the elastic bucket. It cannot cause blocking for regular work (which would be a form of priority inversion).
- When regular tokens get deducted, they deduct from both the regular and elastic buckets. We do want regular work to starve out elastic work. If sufficient regular work has not yet been logically admitted, and other regular work is waiting for it, we don't want elastic work to be able to go through.

To get a sense of what this ends up looking like, consider the following datadriven snippet:

    history
    ----
                       regular |  elastic
                        +16MiB |  +8.0MiB
    ======================================
     -1.0MiB regular    +15MiB |  +7.0MiB
     -7.0MiB regular   +8.0MiB |      +0B (elastic blocked)
     -2.0MiB regular   +6.0MiB |      +0B (elastic blocked)
     -2.0MiB regular   +4.0MiB |      +0B (elastic blocked)
     +2.0MiB regular   +6.0MiB |      +0B (elastic blocked)
     -2.0MiB regular   +4.0MiB |      +0B (elastic blocked)
     +4.0MiB regular   +8.0MiB |      +0B (elastic blocked)
     +2.0MiB regular    +10MiB |  +2.0MiB
     -2.0MiB elastic    +10MiB |      +0B (elastic blocked)
     -2.0MiB regular   +8.0MiB |  -2.0MiB (elastic blocked)
     +2.0MiB regular    +10MiB |      +0B (elastic blocked)
     +2.0MiB elastic    +10MiB |  +2.0MiB
     +6.0MiB regular    +16MiB |  +8.0MiB

The kvflowcontrol.Controller implementation is furnished with the following metrics:
```
- kvadmission.flow_controller.{regular,elastic}_tokens_{available,deducted,returned}
- kvadmission.flow_controller.{regular,elastic}_requests_{waiting,admitted,errored}
- kvadmission.flow_controller.{regular,elastic}_wait_duration
- kvadmission.flow_controller.{regular,elastic}_{,blocked_}stream_count
```
This commit also introduces a deterministic simulator to understand the kind of traffic shaping this flow controller does. The simulator is able to spit out each of the metrics above. To do all this, we added a utility asciitsdb library that comes 'batteries-included' -- it integrates with our metrics registry, is able to scrape relevant metrics, and spit out ascii plots for metric values, all of which we use in tests added in this package. It looks like the following:

    # Note again the mirroring between token returns which immediately
    # allows admission, followed by token deductions.
    plot start=15s end=30s
    kvadmission.flow_controller.regular_tokens_{deducted,returned} unit=MiB rate=true
    ----
     2.1 ┤       ╭╮
     2.0 ┤      ╭╮│         ╭╮          ╭╮╮
     1.9 ┤      │╰╮         ││╮         │││
     1.7 ┤     ╭╯╯│        ╭╯╰╮        ╭╯╰╮
     1.6 ┤     ││ │╮       │╯ │        │╯ │
     1.4 ┤     │╯ ││      ╭╯  │       ╭╯  │╮
     1.3 ┤    ╭╯  ╰╮      │   ╰╮      │╯  ││
     1.1 ┤    │╯   │╮    ╭╯    │     ╭╯   ╰╮
     1.0 ┤   ╭╯    ││    │╯    │     │╯    │╮
     0.9 ┤   │╯    ╰╮   ╭╯     │╮   ╭╯     ││   ╭
     0.7 ┤  ╭╯      │   │      ╰╮  ╭╯      ││  ╭╯
     0.6 ┤ ╭╯╯      │╮ ╭╯       │  │╯      ╰╮  │╯
     0.4 ┤ │╯       ││╭╯        │ ╭╯        │╮╭╯
     0.3 ┤╭╯        ╰─╯│        ╰─╯         │╭╯
     0.1 ┼╯╯         │╭╯         ╰╯         ╰╯╯
    -0.0 ┼╯          ╰╯
          rate(regular_tokens_{deducted,returned}) (MiB)

    # So we're still admitting work choppily, and observe corresponding
    # deductions in the waiting request rate. But given the open-loop
    # thread above, the # of waiting request is still growing
    # unboundedly.
    plot start=15s end=30s
    kvadmission.flow_controller.regular_requests_{admitted,waiting} unit=reqs/s rate=true
    kvadmission.flow_controller.regular_requests_waiting unit=reqs/s
    ----
    ----
     10.7 ┤       ╭╮
      9.9 ┼╮      ││  ╭╮                  ╭╮  ╭╮
      9.2 ┤╰╮    ╭╯│  │╰╮    ╭─╮  ╭╮     ╭╯│  ││
      8.4 ┤ │    │ ╰╮ │ │   ╭╯ │ ╭╯╰╮   ╭╯ │  │╰╮
      7.7 ┤ │   ╭╯  │ │ ╰╮  │  │ │  │   │  ╰╮╭╯ │
      6.9 ┤ ╰╮  │   │╭╯  │ ╭╯  ╰╮│  ╰╮ ╭╯   ││  ╰╮
      6.1 ┤  ╰╮╭╯   ││   ╰╮│    ╭╯   │ │    ││   ╰
      5.4 ┤   ││    ╰│    │╯    │    ╰╮╯    ╭╯
      4.6 ┤   ╰╮    ╭╯    ╰╮    │    ╭╰╮    │╮
      3.9 ┤   ╭╰╮   ││   ╭╯│    │╮   │ │    ││   ╭
      3.1 ┤  ╭╯ ╰╮  │╰╮  │ ╰╮  ╭╯│  ╭╯ ╰╮  ╭╯│  ╭╯
      2.3 ┤ ╭╯   │  │ │ ╭╯  ╰╮ │ │ ╭╯   ╰╮ │ ╰╮╭╯
      1.6 ┤ │    ╰╮╭╯ │ │    │ │ ╰╮│     │ │  ││
      0.8 ┤╭╯     ││  │╭╯    ╰─╯  ╰╯     ╰╮│  ╰╯
      0.1 ┼╯      ││  ╰╯                  ╰╯
     -0.7 ┤       ╰╯
           rate(regular_requests_{admitted,waiting}) (reqs/s)

Release note: None


Co-authored-by: irfan sharif <irfanmahmoudsharif@gmail.com>
  • Loading branch information
craig[bot] and irfansharif committed Feb 8, 2023
2 parents 6ca0b8c + c16ca44 commit 0ab83da
Show file tree
Hide file tree
Showing 23 changed files with 2,549 additions and 73 deletions.
10 changes: 10 additions & 0 deletions DEPS.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -6043,6 +6043,16 @@ def go_deps():
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/mjibson/esc/com_github_mjibson_esc-v0.2.0.zip",
],
)
go_repository(
name = "com_github_mkungla_bexp_v3",
build_file_proto_mode = "disable_global",
importpath = "github.com/mkungla/bexp/v3",
sha256 = "903a932e83da8be3426e29c1484d79f814a825b2c2743c36d43b054910a9f886",
strip_prefix = "github.com/mkungla/bexp/v3@v3.0.1",
urls = [
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/mkungla/bexp/v3/com_github_mkungla_bexp_v3-v3.0.1.zip",
],
)
go_repository(
name = "com_github_mmatczuk_go_generics",
build_file_proto_mode = "disable_global",
Expand Down
1 change: 1 addition & 0 deletions build/bazelutil/distdir_files.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -629,6 +629,7 @@ DISTDIR_FILES = {
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/mitchellh/osext/com_github_mitchellh_osext-v0.0.0-20151018003038-5e2d6d41470f.zip": "d8e6e5f6bd749cfa0c1c17c40f5dc0fd19e4a0a83245f46bde23bea4e65d1a20",
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/mitchellh/reflectwalk/com_github_mitchellh_reflectwalk-v1.0.0.zip": "318ab84e22d4554a7540c7ebc9b4fb607e2608578c3a5bb72434203988048145",
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/mjibson/esc/com_github_mjibson_esc-v0.2.0.zip": "9f090786bd43dddb5c0d798b449d5e8aede4cb7d106f56dcac0aebd8fd1929cc",
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/mkungla/bexp/v3/com_github_mkungla_bexp_v3-v3.0.1.zip": "903a932e83da8be3426e29c1484d79f814a825b2c2743c36d43b054910a9f886",
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/mmatczuk/go_generics/com_github_mmatczuk_go_generics-v0.0.0-20181212143635-0aaa050f9bab.zip": "18c1e95c93f1f82be0184bc13bf49eb4350c7a4ff524b1bf440b3eb9ff14acc9",
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/mmcloughlin/geohash/com_github_mmcloughlin_geohash-v0.9.0.zip": "7162856858d9bb3c411d4b42ad19dfff579341ddf0580122e3f1ac3be05c7441",
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/moby/locker/com_github_moby_locker-v1.0.1.zip": "f07361346d12a24e168db7fb2f21281883bee6060f1aedf7507bccf20c4a793f",
Expand Down
1 change: 1 addition & 0 deletions go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -178,6 +178,7 @@ require (
github.com/mattn/goveralls v0.0.2
github.com/mibk/dupl v1.0.0
github.com/mitchellh/reflectwalk v1.0.0
github.com/mkungla/bexp/v3 v3.0.1
github.com/mmatczuk/go_generics v0.0.0-20181212143635-0aaa050f9bab
github.com/montanaflynn/stats v0.6.6
github.com/mozillazg/go-slugify v0.2.0
Expand Down
2 changes: 2 additions & 0 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -1680,6 +1680,8 @@ github.com/mitchellh/osext v0.0.0-20151018003038-5e2d6d41470f/go.mod h1:OkQIRizQ
github.com/mitchellh/reflectwalk v1.0.0 h1:9D+8oIskB4VJBN5SFlmc27fSlIBZaov1Wpk/IfikLNY=
github.com/mitchellh/reflectwalk v1.0.0/go.mod h1:mSTlrgnPZtwu0c4WaC2kGObEpuNDbx0jmZXqmk4esnw=
github.com/mjibson/esc v0.2.0/go.mod h1:9Hw9gxxfHulMF5OJKCyhYD7PzlSdhzXyaGEBRPH1OPs=
github.com/mkungla/bexp/v3 v3.0.1 h1:UqcWAaxWn+rmJ+3ZgwokMrUerGMUeOFihUPrTLPFZ9Q=
github.com/mkungla/bexp/v3 v3.0.1/go.mod h1:zkzndAaEcYdZcu+cB4WkNYQuG7pwjzMOZLn3vM8KD+8=
github.com/mmatczuk/go_generics v0.0.0-20181212143635-0aaa050f9bab h1:QjLgi/L+MjxysinrA8KkNZLf3cAhTluBoSXUvFeN144=
github.com/mmatczuk/go_generics v0.0.0-20181212143635-0aaa050f9bab/go.mod h1:Fs8p4al9GNa3b42YfZOoUVMdjQ2WlNEYlnoboJKPpJ8=
github.com/mmcloughlin/geohash v0.9.0 h1:FihR004p/aE1Sju6gcVq5OLDqGcMnpBY+8moBqIsVOs=
Expand Down
6 changes: 6 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -205,6 +205,7 @@ ALL_TESTS = [
"//pkg/kv/kvserver/gc:gc_test",
"//pkg/kv/kvserver/idalloc:idalloc_test",
"//pkg/kv/kvserver/intentresolver:intentresolver_test",
"//pkg/kv/kvserver/kvflowcontrol/kvflowcontroller:kvflowcontroller_test",
"//pkg/kv/kvserver/kvstorage:kvstorage_test",
"//pkg/kv/kvserver/liveness:liveness_test",
"//pkg/kv/kvserver/logstore:logstore_test",
Expand Down Expand Up @@ -1219,6 +1220,8 @@ GO_TARGETS = [
"//pkg/kv/kvserver/intentresolver:intentresolver",
"//pkg/kv/kvserver/intentresolver:intentresolver_test",
"//pkg/kv/kvserver/kvadmission:kvadmission",
"//pkg/kv/kvserver/kvflowcontrol/kvflowcontroller:kvflowcontroller",
"//pkg/kv/kvserver/kvflowcontrol/kvflowcontroller:kvflowcontroller_test",
"//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb:kvflowcontrolpb",
"//pkg/kv/kvserver/kvflowcontrol:kvflowcontrol",
"//pkg/kv/kvserver/kvserverbase:kvserverbase",
Expand Down Expand Up @@ -2022,6 +2025,7 @@ GO_TARGETS = [
"//pkg/util/admission:admission",
"//pkg/util/admission:admission_test",
"//pkg/util/arith:arith",
"//pkg/util/asciitsdb:asciitsdb",
"//pkg/util/binfetcher:binfetcher",
"//pkg/util/binfetcher:binfetcher_test",
"//pkg/util/bitarray:bitarray",
Expand Down Expand Up @@ -2619,6 +2623,7 @@ GET_X_DATA_TARGETS = [
"//pkg/kv/kvserver/intentresolver:get_x_data",
"//pkg/kv/kvserver/kvadmission:get_x_data",
"//pkg/kv/kvserver/kvflowcontrol:get_x_data",
"//pkg/kv/kvserver/kvflowcontrol/kvflowcontroller:get_x_data",
"//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb:get_x_data",
"//pkg/kv/kvserver/kvserverbase:get_x_data",
"//pkg/kv/kvserver/kvserverpb:get_x_data",
Expand Down Expand Up @@ -3123,6 +3128,7 @@ GET_X_DATA_TARGETS = [
"//pkg/util/admission:get_x_data",
"//pkg/util/admission/admissionpb:get_x_data",
"//pkg/util/arith:get_x_data",
"//pkg/util/asciitsdb:get_x_data",
"//pkg/util/binfetcher:get_x_data",
"//pkg/util/bitarray:get_x_data",
"//pkg/util/bufalloc:get_x_data",
Expand Down
2 changes: 2 additions & 0 deletions pkg/kv/kvserver/kvflowcontrol/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,8 @@ go_library(
"//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb",
"//pkg/roachpb",
"//pkg/util/admission/admissionpb",
"@com_github_cockroachdb_redact//:redact",
"@com_github_dustin_go_humanize//:go-humanize",
],
)

Expand Down
2 changes: 2 additions & 0 deletions pkg/kv/kvserver/kvflowcontrol/doc.go
Original file line number Diff line number Diff line change
Expand Up @@ -353,3 +353,5 @@ package kvflowcontrol
// the receiver to send messages to return tokens up to some point, and
// (b) the receiver has either not received the message for which we've
// deducted tokens, or forgotten about it.
// - Ensure that flow tokens aren't leaked, by checking that after the tests
// quiesce, flow tokens are back to their original limits.
34 changes: 33 additions & 1 deletion pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb"
"github.com/cockroachdb/redact"
"github.com/dustin/go-humanize"
)

// Stream models the stream over which we replicate data traffic, the
Expand All @@ -34,7 +36,10 @@ type Stream struct {
// 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
//
// NB: We use a signed integer to accommodate data structures that deal with
// token deltas, or buckets that are allowed to go into debt.
type Tokens int64

// Controller provides flow control for replication traffic in KV, held at the
// node-level.
Expand Down Expand Up @@ -136,3 +141,30 @@ type DispatchReader interface {
PendingDispatch() []roachpb.NodeID
PendingDispatchFor(roachpb.NodeID) []kvflowcontrolpb.AdmittedRaftLogEntries
}

func (t Tokens) String() string {
return redact.StringWithoutMarkers(t)
}

// SafeFormat implements the redact.SafeFormatter interface.
func (t Tokens) SafeFormat(p redact.SafePrinter, verb rune) {
sign := "+"
if t < 0 {
sign = "-"
t = -t
}
p.Printf("%s%s", sign, humanize.IBytes(uint64(t)))
}

func (s Stream) String() string {
return redact.StringWithoutMarkers(s)
}

// SafeFormat implements the redact.SafeFormatter interface.
func (s Stream) SafeFormat(p redact.SafePrinter, verb rune) {
tenantSt := s.TenantID.String()
if s.TenantID.IsSystem() {
tenantSt = "1"
}
p.Printf("t%s/s%s", tenantSt, s.StoreID.String())
}
55 changes: 55 additions & 0 deletions pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data")
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")

go_library(
name = "kvflowcontroller",
srcs = [
"kvflowcontroller.go",
"kvflowcontroller_metrics.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller",
visibility = ["//visibility:public"],
deps = [
"//pkg/base",
"//pkg/kv/kvserver/kvflowcontrol",
"//pkg/settings",
"//pkg/settings/cluster",
"//pkg/util/admission/admissionpb",
"//pkg/util/buildutil",
"//pkg/util/hlc",
"//pkg/util/log",
"//pkg/util/metric",
"//pkg/util/syncutil",
],
)

go_test(
name = "kvflowcontroller_test",
srcs = [
"kvflowcontrol_token_adjustment_test.go",
"kvflowcontroller_simulation_test.go",
],
args = ["-test.timeout=295s"],
data = glob(["testdata/**"]),
embed = [":kvflowcontroller"],
deps = [
"//pkg/kv/kvserver/kvflowcontrol",
"//pkg/roachpb",
"//pkg/settings/cluster",
"//pkg/testutils/datapathutils",
"//pkg/util/admission/admissionpb",
"//pkg/util/asciitsdb",
"//pkg/util/hlc",
"//pkg/util/leaktest",
"//pkg/util/log",
"//pkg/util/metric",
"//pkg/util/timeutil",
"@com_github_cockroachdb_datadriven//:datadriven",
"@com_github_dustin_go_humanize//:go-humanize",
"@com_github_guptarohit_asciigraph//:asciigraph",
"@com_github_mkungla_bexp_v3//:bexp",
"@com_github_stretchr_testify//require",
],
)

get_x_data(name = "get_x_data")
Original file line number Diff line number Diff line change
@@ -0,0 +1,159 @@
// 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 kvflowcontroller

import (
"context"
"fmt"
"strings"
"testing"
"time"

"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/metric"
"github.com/cockroachdb/datadriven"
"github.com/dustin/go-humanize"
"github.com/stretchr/testify/require"
)

func TestFlowTokenAdjustment(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

var (
ctx = context.Background()
controller *Controller
adjustments []adjustment
stream = kvflowcontrol.Stream{
TenantID: roachpb.SystemTenantID,
StoreID: roachpb.StoreID(1),
}
)

datadriven.RunTest(t, "testdata/flow_token_adjustment",
func(t *testing.T, d *datadriven.TestData) string {
switch d.Cmd {
case "init":
controller = New(
metric.NewRegistry(),
cluster.MakeTestingClusterSettings(),
hlc.NewClockWithSystemTimeSource(time.Nanosecond),
)
adjustments = nil
return ""

case "adjust":
require.NotNilf(t, controller, "uninitialized flow controller (did you use 'init'?)")

for _, line := range strings.Split(d.Input, "\n") {
parts := strings.Fields(line)
require.Len(t, parts, 2, "expected form 'class={regular,elastic} delta={+,-}<size>")

var delta kvflowcontrol.Tokens
var pri admissionpb.WorkPriority

// Parse class={regular,elastic}.
parts[0] = strings.TrimSpace(parts[0])
require.True(t, strings.HasPrefix(parts[0], "class="))
parts[0] = strings.TrimPrefix(strings.TrimSpace(parts[0]), "class=")
switch parts[0] {
case "regular":
pri = admissionpb.NormalPri
case "elastic":
pri = admissionpb.BulkNormalPri
}

// Parse delta={+,-}<size>
parts[1] = strings.TrimSpace(parts[1])
require.True(t, strings.HasPrefix(parts[1], "delta="))
parts[1] = strings.TrimPrefix(strings.TrimSpace(parts[1]), "delta=")
require.True(t, strings.HasPrefix(parts[1], "+") || strings.HasPrefix(parts[1], "-"))
isPositive := strings.Contains(parts[1], "+")
parts[1] = strings.TrimPrefix(parts[1], "+")
parts[1] = strings.TrimPrefix(parts[1], "-")
bytes, err := humanize.ParseBytes(parts[1])
require.NoError(t, err)
delta = kvflowcontrol.Tokens(int64(bytes))
if !isPositive {
delta = -delta
}

controller.adjustTokens(ctx, pri, delta, stream)
adjustments = append(adjustments, adjustment{
pri: pri,
delta: delta,
post: controller.testingGetTokensForStream(stream),
})
}
return ""

case "history":
limit := controller.testingGetLimit()

var buf strings.Builder
buf.WriteString(" regular | elastic\n")
buf.WriteString(fmt.Sprintf(" %8s | %8s\n",
printTrimmedTokens(limit[admissionpb.RegularWorkClass]),
printTrimmedTokens(limit[admissionpb.ElasticWorkClass]),
))
buf.WriteString("======================================\n")
for _, h := range adjustments {
buf.WriteString(fmt.Sprintf("%s\n", h))
}
return buf.String()

default:
return fmt.Sprintf("unknown command: %s", d.Cmd)
}
})
}

type adjustment struct {
pri admissionpb.WorkPriority
delta kvflowcontrol.Tokens
post tokensPerWorkClass
}

func printTrimmedTokens(t kvflowcontrol.Tokens) string {
return strings.ReplaceAll(t.String(), " ", "")
}

func (h adjustment) String() string {
class := admissionpb.WorkClassFromPri(h.pri)

comment := ""
if h.post[admissionpb.RegularWorkClass] <= 0 {
comment = "regular"
}
if h.post[admissionpb.ElasticWorkClass] <= 0 {
if len(comment) == 0 {
comment = "elastic"
} else {
comment = "regular and elastic"
}
}
if len(comment) != 0 {
comment = fmt.Sprintf(" (%s blocked)", comment)
}
return fmt.Sprintf("%8s %7s %8s | %8s%s",
printTrimmedTokens(h.delta),
class,
printTrimmedTokens(h.post[admissionpb.RegularWorkClass]),
printTrimmedTokens(h.post[admissionpb.ElasticWorkClass]),
comment,
)
}
Loading

0 comments on commit 0ab83da

Please sign in to comment.