Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
104800: dev,lint: generate code before linting r=rail a=rickystewart

Generated code needs to be present for `TestGCAssert`. Tell `dev` to generate code before running lints. We skip generating code if run under `--short` since `GCAssert` is skipped under `--short` anyway.

Also a small improvement to `lint` so that it pulls the `go` binary from Bazel `runfiles` instead of it needing to be injected.

Epic: none
Release note: None

104844: util: replace quotapool.TokenBucket with cockroachdb/quotapool r=RaduBerinde a=RaduBerinde

I have pulled out the token bucket implementation in a separate repo (so Pebble can use it too). This commit removes it from Cockroach and refactors the code to use the library.

Epic: none
Release note: None

Co-authored-by: Ricky Stewart <[email protected]>
Co-authored-by: Radu Berinde <[email protected]>
  • Loading branch information
3 people committed Jun 14, 2023
3 parents 244ef1c + c3670d4 + 45b42d3 commit 8e3bf5b
Show file tree
Hide file tree
Showing 18 changed files with 101 additions and 326 deletions.
10 changes: 10 additions & 0 deletions DEPS.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -1611,6 +1611,16 @@ def go_deps():
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/stress/com_github_cockroachdb_stress-v0.0.0-20220803192808-1806698b1b7b.zip",
],
)
go_repository(
name = "com_github_cockroachdb_tokenbucket",
build_file_proto_mode = "disable_global",
importpath = "github.com/cockroachdb/tokenbucket",
sha256 = "7711efac97ce89c704e84b30c6c73887040d8ba50ce5b286e7113f4af7012339",
strip_prefix = "github.com/cockroachdb/[email protected]",
urls = [
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/tokenbucket/com_github_cockroachdb_tokenbucket-v0.0.0-20230613231145-182959a1fad6.zip",
],
)
go_repository(
name = "com_github_cockroachdb_tools",
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 @@ -320,6 +320,7 @@ DISTDIR_FILES = {
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/stress/com_github_cockroachdb_stress-v0.0.0-20220803192808-1806698b1b7b.zip": "3fda531795c600daf25532a4f98be2a1335cd1e5e182c72789bca79f5f69fcc1",
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/tablewriter/com_github_cockroachdb_tablewriter-v0.0.5-0.20200105123400-bd15540e8847.zip": "79daf1c29ec50cdd8dd1ea33f8a814963646a45a2ebe22742d652579340ebde0",
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/teamcity/com_github_cockroachdb_teamcity-v0.0.0-20180905144921-8ca25c33eb11.zip": "9df6b028c9fb5bff7bdad844bda504356945fd6d3cd583c50f68d8b8e85060f6",
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/tokenbucket/com_github_cockroachdb_tokenbucket-v0.0.0-20230613231145-182959a1fad6.zip": "7711efac97ce89c704e84b30c6c73887040d8ba50ce5b286e7113f4af7012339",
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/tools/com_github_cockroachdb_tools-v0.0.0-20211112185054-642e51449b40.zip": "37a3737dd23768b4997b2f0341d625658f5862cdbf808f7fbf3a7f9fd25913a7",
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/ttycolor/com_github_cockroachdb_ttycolor-v0.0.0-20210902133924-c7d7dcdde4e8.zip": "1260533510c89abd6d8af573a40f0246f6865d5091144dea509b2c48e7c61614",
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/vitess/com_github_cockroachdb_vitess-v0.0.0-20210218160543-54524729cc82.zip": "71f14e67f9396930d978d85c47b853f5cc4ce340e53cf88bf7d731b8428b2f77",
Expand Down
1 change: 0 additions & 1 deletion build/teamcity/cockroach/ci/tests/lint_impl.sh
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,6 @@
set -xeuo pipefail

# GCAssert requirements -- start
export PATH="$(dirname $(bazel run @go_sdk//:bin/go --run_under=realpath)):$PATH"
bazel run //pkg/gen:code
bazel run //pkg/cmd/generate-cgo:generate-cgo --run_under="cd $(bazel info workspace) && "
# GCAssert requirements -- end
Expand Down
1 change: 1 addition & 0 deletions go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -120,6 +120,7 @@ require (
github.com/cockroachdb/redact v1.1.5
github.com/cockroachdb/returncheck v0.0.0-20200612231554-92cdbca611dd
github.com/cockroachdb/stress v0.0.0-20220803192808-1806698b1b7b
github.com/cockroachdb/tokenbucket v0.0.0-20230613231145-182959a1fad6
github.com/cockroachdb/tools v0.0.0-20211112185054-642e51449b40
github.com/cockroachdb/ttycolor v0.0.0-20210902133924-c7d7dcdde4e8
github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd
Expand Down
2 changes: 2 additions & 0 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -499,6 +499,8 @@ github.com/cockroachdb/tablewriter v0.0.5-0.20200105123400-bd15540e8847 h1:c7yLg
github.com/cockroachdb/tablewriter v0.0.5-0.20200105123400-bd15540e8847/go.mod h1:zq6QwlOf5SlnkVbMSr5EoBv3636FWnp+qbPhuoO21uA=
github.com/cockroachdb/teamcity v0.0.0-20180905144921-8ca25c33eb11 h1:UAqRo5xPCyTtZznAJ9iPVpDUFxGI0a6QWtQ8E+zwJRg=
github.com/cockroachdb/teamcity v0.0.0-20180905144921-8ca25c33eb11/go.mod h1:3299Mt0Q7PkqGqbsxhvbrTpMqRyIcZ6OMw4IEmiO09g=
github.com/cockroachdb/tokenbucket v0.0.0-20230613231145-182959a1fad6 h1:DJK8W/iB+s/qkTtmXSrHA49lp5O3OsR7E6z4byOLy34=
github.com/cockroachdb/tokenbucket v0.0.0-20230613231145-182959a1fad6/go.mod h1:7nc4anLGjupUW/PeY5qiNYsdNXj7zopG+eqsS7To5IQ=
github.com/cockroachdb/tools v0.0.0-20211112185054-642e51449b40 h1:qVTb3XEv+7VVjPetop8yGbN95HfCeqZx+VMveeSJPZw=
github.com/cockroachdb/tools v0.0.0-20211112185054-642e51449b40/go.mod h1:cllxeV+TYc387/XzQRnDg6YThHoDzFewovWffzAm37Q=
github.com/cockroachdb/ttycolor v0.0.0-20210902133924-c7d7dcdde4e8 h1:Hli+oX84dKq44sLVCcsGKqifm5Lg9J8VoJ2P3h9iPdI=
Expand Down
26 changes: 22 additions & 4 deletions pkg/cmd/dev/lint.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,26 @@ func (d *dev) lint(cmd *cobra.Command, commandLine []string) error {
}
}

lintEnv := os.Environ()

if !short {
// First, generate code to make sure GCAssert and any other
// tests that depend on generated code still work.
if err := d.generateGo(cmd); err != nil {
return err
}
// We also need `CC` and `CXX` set appropriately.
cc, err := d.exec.LookPath("cc")
if err != nil {
return fmt.Errorf("`cc` is not installed; needed for `TestGCAssert` (%w)", err)
}
cc = strings.TrimSpace(cc)
d.log.Printf("export CC=%s", cc)
d.log.Printf("export CXX=%s", cc)
envWithCc := []string{"CC=" + cc, "CXX=" + cc}
lintEnv = append(envWithCc, lintEnv...)
}

var args []string
// NOTE the --config=test here. It's very important we compile the test binary with the
// appropriate stuff (gotags, etc.)
Expand Down Expand Up @@ -80,13 +100,11 @@ func (d *dev) lint(cmd *cobra.Command, commandLine []string) error {
if !strings.HasPrefix(pkg, "./") {
pkg = "./" + pkg
}
env := os.Environ()
envvar := fmt.Sprintf("PKG=%s", pkg)
d.log.Printf("export %s", envvar)
env = append(env, envvar)
return d.exec.CommandContextWithEnv(ctx, env, "bazel", args...)
lintEnv = append(lintEnv, envvar)
}
err := d.exec.CommandContextInheritingStdStreams(ctx, "bazel", args...)
err := d.exec.CommandContextWithEnv(ctx, lintEnv, "bazel", args...)
if err != nil {
return err
}
Expand Down
25 changes: 25 additions & 0 deletions pkg/cmd/dev/testdata/datadriven/lint
Original file line number Diff line number Diff line change
@@ -1,6 +1,12 @@
exec
dev lint
----
bazel run //pkg/gen:code
bazel info workspace --color=no
bazel run //pkg/cmd/generate-cgo:generate-cgo '--run_under=cd crdb-checkout && '
which cc
export CC=
export CXX=
bazel run --config=test //build/bazelutil:lint -- -test.v
bazel build //pkg/cmd/cockroach-short //pkg/cmd/dev //pkg/obsservice/cmd/obsservice //pkg/cmd/roachprod //pkg/cmd/roachtest --//build/toolchains:nogo_flag

Expand All @@ -12,17 +18,36 @@ bazel run --config=test //build/bazelutil:lint -- -test.v -test.short -test.time
exec
dev lint pkg/cmd/dev
----
bazel run //pkg/gen:code
bazel info workspace --color=no
bazel run //pkg/cmd/generate-cgo:generate-cgo '--run_under=cd crdb-checkout && '
which cc
export CC=
export CXX=
export PKG=./pkg/cmd/dev
bazel run --config=test //build/bazelutil:lint -- -test.v
bazel build //pkg/cmd/cockroach-short //pkg/cmd/dev //pkg/obsservice/cmd/obsservice //pkg/cmd/roachprod //pkg/cmd/roachtest --//build/toolchains:nogo_flag

exec
dev lint -f TestLowercaseFunctionNames --cpus 4
----
bazel run //pkg/gen:code
bazel info workspace --color=no
bazel run //pkg/cmd/generate-cgo:generate-cgo '--run_under=cd crdb-checkout && '
which cc
export CC=
export CXX=
bazel run --config=test //build/bazelutil:lint --local_cpu_resources=4 -- -test.v -test.run Lint/TestLowercaseFunctionNames

exec
dev lint --cpus 4
----
bazel run //pkg/gen:code
bazel info workspace --color=no
bazel run //pkg/cmd/generate-cgo:generate-cgo '--run_under=cd crdb-checkout && '
which cc
export CC=
export CXX=
bazel run --config=test //build/bazelutil:lint --local_cpu_resources=4 -- -test.v
bazel build //pkg/cmd/cockroach-short //pkg/cmd/dev //pkg/obsservice/cmd/obsservice //pkg/cmd/roachprod //pkg/cmd/roachtest --//build/toolchains:nogo_flag --local_cpu_resources=4

Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/tenantrate/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ go_library(
"//pkg/util/syncutil",
"//pkg/util/timeutil",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_tokenbucket//:tokenbucket",
],
)

Expand Down
13 changes: 7 additions & 6 deletions pkg/kv/kvserver/tenantrate/limiter.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/quotapool"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/tokenbucket"
)

// Limiter is used to rate-limit KV requests for a given tenant.
Expand Down Expand Up @@ -151,7 +152,7 @@ func (rl *limiter) RecordRead(ctx context.Context, respInfo tenantcostmodel.Resp
amount := tb.config.ReadBatchUnits
amount += float64(respInfo.ReadCount()) * tb.config.ReadRequestUnits
amount += float64(respInfo.ReadBytes()) * tb.config.ReadUnitsPerByte
tb.Adjust(quotapool.Tokens(-amount))
tb.Adjust(tokenbucket.Tokens(-amount))
// Do not notify the head of the queue. In the best case we did not disturb
// the time at which it can be fulfilled and in the worst case, we made it
// further in the future.
Expand All @@ -174,24 +175,24 @@ func (rl *limiter) updateConfig(config Config) {
rl.qp.Update(func(res quotapool.Resource) (shouldNotify bool) {
tb := res.(*tokenBucket)
tb.config = config
tb.UpdateConfig(quotapool.TokensPerSecond(config.Rate), quotapool.Tokens(config.Burst))
tb.UpdateConfig(tokenbucket.TokensPerSecond(config.Rate), tokenbucket.Tokens(config.Burst))
return true
})
}

// tokenBucket represents the token bucket for KV Compute Units and its
// associated configuration. It implements quotapool.Resource.
type tokenBucket struct {
quotapool.TokenBucket
tokenbucket.TokenBucket

config Config
}

var _ quotapool.Resource = (*tokenBucket)(nil)

func (tb *tokenBucket) init(config Config, timeSource timeutil.TimeSource) {
tb.TokenBucket.Init(
quotapool.TokensPerSecond(config.Rate), quotapool.Tokens(config.Burst), timeSource,
tb.TokenBucket.InitWithNowFn(
tokenbucket.TokensPerSecond(config.Rate), tokenbucket.Tokens(config.Burst), timeSource.Now,
)
tb.config = config
}
Expand Down Expand Up @@ -236,7 +237,7 @@ func (req *waitRequest) Acquire(
// value, in case the quota pool is in debt and the read should block.
needed = 0
}
return tb.TryToFulfill(quotapool.Tokens(needed))
return tb.TryToFulfill(tokenbucket.Tokens(needed))
}

// ShouldWait is part of quotapool.Request.
Expand Down
2 changes: 1 addition & 1 deletion pkg/testutils/lint/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ go_test(
"nightly_lint_test.go",
],
args = ["-test.timeout=295s"],
data = glob(["testdata/**"]),
data = glob(["testdata/**"]) + ["@go_sdk//:files"],
embed = [":lint"],
embedsrcs = ["gcassert_paths.txt"],
gotags = ["lint"],
Expand Down
6 changes: 6 additions & 0 deletions pkg/testutils/lint/lint_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,12 @@ const cockroachDB = "github.com/cockroachdb/cockroach"
//go:embed gcassert_paths.txt
var rawGcassertPaths string

func init() {
if bazel.BuiltWithBazel() {
bazel.SetGoEnv()
}
}

func dirCmd(
dir string, name string, args ...string,
) (*exec.Cmd, *bytes.Buffer, stream.Filter, error) {
Expand Down
4 changes: 2 additions & 2 deletions pkg/util/admission/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -36,13 +36,13 @@ go_library(
"//pkg/util/humanizeutil",
"//pkg/util/log",
"//pkg/util/metric",
"//pkg/util/quotapool",
"//pkg/util/syncutil",
"//pkg/util/timeutil",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_logtags//:logtags",
"@com_github_cockroachdb_pebble//:pebble",
"@com_github_cockroachdb_redact//:redact",
"@com_github_cockroachdb_tokenbucket//:tokenbucket",
],
)

Expand Down Expand Up @@ -76,13 +76,13 @@ go_test(
"//pkg/util/leaktest",
"//pkg/util/log",
"//pkg/util/metric",
"//pkg/util/quotapool",
"//pkg/util/syncutil",
"//pkg/util/timeutil",
"//pkg/util/tracing",
"@com_github_cockroachdb_datadriven//:datadriven",
"@com_github_cockroachdb_pebble//:pebble",
"@com_github_cockroachdb_redact//:redact",
"@com_github_cockroachdb_tokenbucket//:tokenbucket",
"@com_github_guptarohit_asciigraph//:asciigraph",
"@com_github_stretchr_testify//require",
],
Expand Down
18 changes: 9 additions & 9 deletions pkg/util/admission/elastic_cpu_granter.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,9 +20,9 @@ import (
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/metric"
"github.com/cockroachdb/cockroach/pkg/util/quotapool"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/tokenbucket"
)

// We don't want the ability for an admitted a request to be able to run
Expand Down Expand Up @@ -108,7 +108,7 @@ type elasticCPUGranter struct {
// GrantCoordinator, which is ordered before the one in WorkQueue, since
// requester.granted() is called while holding GrantCoordinator.mu.
syncutil.Mutex
tb *quotapool.TokenBucket
tb *tokenbucket.TokenBucket
tbLastReset time.Time
tbReset *util.EveryN
utilizationLimit float64
Expand All @@ -122,16 +122,16 @@ var _ granter = &elasticCPUGranter{}
func newElasticCPUGranter(
ambientCtx log.AmbientContext, st *cluster.Settings, metrics *elasticCPUGranterMetrics,
) *elasticCPUGranter {
tokenBucket := &quotapool.TokenBucket{}
tokenBucket.Init(0, 0, timeutil.DefaultTimeSource{})
tokenBucket := &tokenbucket.TokenBucket{}
tokenBucket.InitWithNowFn(0, 0, timeutil.Now)
return newElasticCPUGranterWithTokenBucket(ambientCtx, st, metrics, tokenBucket)
}

func newElasticCPUGranterWithTokenBucket(
ambientCtx log.AmbientContext,
st *cluster.Settings,
metrics *elasticCPUGranterMetrics,
tokenBucket *quotapool.TokenBucket,
tokenBucket *tokenbucket.TokenBucket,
) *elasticCPUGranter {
e := &elasticCPUGranter{
ctx: ambientCtx.AnnotateCtx(context.Background()),
Expand All @@ -156,7 +156,7 @@ func (e *elasticCPUGranter) tryGet(count int64) (granted bool) {
e.mu.Lock()
defer e.mu.Unlock()

granted, _ = e.mu.tb.TryToFulfill(quotapool.Tokens(count))
granted, _ = e.mu.tb.TryToFulfill(tokenbucket.Tokens(count))
return granted
}

Expand All @@ -170,15 +170,15 @@ func (e *elasticCPUGranter) returnGrantWithoutGrantingElsewhere(count int64) {
e.mu.Lock()
defer e.mu.Unlock()

e.mu.tb.Adjust(quotapool.Tokens(count))
e.mu.tb.Adjust(tokenbucket.Tokens(count))
}

// tookWithoutPermission implements granter.
func (e *elasticCPUGranter) tookWithoutPermission(count int64) {
e.mu.Lock()
defer e.mu.Unlock()

e.mu.tb.Adjust(quotapool.Tokens(-count))
e.mu.tb.Adjust(tokenbucket.Tokens(-count))
}

// continueGrantChain implements granter.
Expand Down Expand Up @@ -212,7 +212,7 @@ func (e *elasticCPUGranter) setUtilizationLimit(utilizationLimit float64) {
//
rate := utilizationLimit * float64(int64(runtime.GOMAXPROCS(0))*time.Second.Nanoseconds())
e.mu.utilizationLimit = utilizationLimit
e.mu.tb.UpdateConfig(quotapool.TokensPerSecond(rate), quotapool.Tokens(rate))
e.mu.tb.UpdateConfig(tokenbucket.TokensPerSecond(rate), tokenbucket.Tokens(rate))
if now := timeutil.Now(); now.Sub(e.mu.tbLastReset) > 15*time.Second { // TODO(irfansharif): make this is a cluster setting?
// Periodically reset the token bucket. This is just defense-in-depth
// and at worst, over-admits. We've seen production clusters where the
Expand Down
8 changes: 4 additions & 4 deletions pkg/util/admission/elastic_cpu_granter_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,9 +21,9 @@ import (
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/testutils/datapathutils"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/quotapool"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/datadriven"
"github.com/cockroachdb/tokenbucket"
"github.com/stretchr/testify/require"
)

Expand Down Expand Up @@ -72,7 +72,7 @@ func TestElasticCPUGranter(t *testing.T) {
elasticCPUGranterMetrics *elasticCPUGranterMetrics
elasticCPUGranter *elasticCPUGranter
mt *timeutil.ManualTime
tokenBucket *quotapool.TokenBucket
tokenBucket *tokenbucket.TokenBucket
)

printTokenBucket := func() string {
Expand Down Expand Up @@ -128,8 +128,8 @@ func TestElasticCPUGranter(t *testing.T) {
elasticCPURequester = &testElasticCPURequester{}
elasticCPUGranterMetrics = makeElasticCPUGranterMetrics()
mt = timeutil.NewManualTime(t0)
tokenBucket = &quotapool.TokenBucket{}
tokenBucket.Init(0, 0, mt)
tokenBucket = &tokenbucket.TokenBucket{}
tokenBucket.InitWithNowFn(0, 0, mt.Now)
elasticCPUGranter = newElasticCPUGranterWithTokenBucket(
log.MakeTestingAmbientCtxWithNewTracer(),
cluster.MakeTestingClusterSettings(),
Expand Down
3 changes: 1 addition & 2 deletions pkg/util/quotapool/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,6 @@ go_library(
"intpool.go",
"notify_queue.go",
"quotapool.go",
"token_bucket.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/util/quotapool",
visibility = ["//visibility:public"],
Expand All @@ -19,6 +18,7 @@ go_library(
"//pkg/util/timeutil",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_redact//:redact",
"@com_github_cockroachdb_tokenbucket//:tokenbucket",
],
)

Expand All @@ -32,7 +32,6 @@ go_test(
"intpool_test.go",
"node_size_test.go",
"notify_queue_test.go",
"token_bucket_test.go",
],
args = ["-test.timeout=55s"],
embed = [":quotapool"],
Expand Down
Loading

0 comments on commit 8e3bf5b

Please sign in to comment.