Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
108597: metrics: assign histogram metric type on histogram construction r=ericharmeling a=ericharmeling

This commit assigns prometheusgo.MetricType_HISTOGRAM to the
Metadata.MetricType on histogram construction.

Before this change, GetMetadata() was returning the
Metadata.MetricType zero value (prometheusgo.MetricType_COUNTER)
for all histograms that did not explicitly specify the
prometheusgo.MetricType_HISTOGRAM for Metadata.MetricType in
their Metadata definitions. This prevented checks on histogram
Metadata.MetricType from properly evaluating the metrics as
histograms.

Fixes #106448.
Fixes #107701.

Releaes note: None

109345: changefeedccl: refactor kvfeed startup in changeaggregator processor  r=miretskiy a=jayshrivastava

changefeedccl: refactor kvfeed startup in changeaggregator processor
This change cleans up the code used to start up the kv feed
in change aggregator processors. This change removes uncessessary code,
adds a better API, and makes code easier to reason about.

Informs: #96953
Release note: None
Epic: None

109386: sql: adjust many tests to work with test tenant r=yuzefovich a=yuzefovich

Epic: CRDB-18499
Informs #76378

Release note: None

109476: dev: make `dev test --count 1` invalidate cached test results r=dt a=rickystewart

This matches the behavior of `go test`.

Epic: none
Release note: None

109506: changefeedccl: ensure rangefeed setting is enabled in tests r=miretskiy a=jayshrivastava

Previously, many tests which create rangefeeds would not explicitly set the `kv.rangefeed.enabled` setting to be true. These tests would still work because, by default, rangefeeds are enabled via span configs. However, it was observed that span configs are not immediately applied when range splits occur. This would cause the testing rangefeed reader to encounter errors and/or timeout on very rare occasions. See #109306 (comment) for more info.

This change updates these tests to set the `kv.rangefeed.enabled` cluster setting to be true, which removes the dependency on span configs.

Closes: #109306
Epic: None
Release note: None

109511: concurrency: use generic lists in the lock table r=nvanbenschoten a=arulajmani


Now that cda4fa2 has landed, we can make use of generic lists in a few places in the lock table.

Epic: none
Release note: None

Co-authored-by: Eric Harmeling <[email protected]>
Co-authored-by: Jayant Shrivastava <[email protected]>
Co-authored-by: Yahor Yuzefovich <[email protected]>
Co-authored-by: Ricky Stewart <[email protected]>
Co-authored-by: Arul Ajmani <[email protected]>
  • Loading branch information
6 people committed Aug 25, 2023
7 parents be9d15a + fb8f99b + 98c762d + 52a506c + acf63ee + 0ec2775 + a4dcc18 commit ee7cdcf
Show file tree
Hide file tree
Showing 33 changed files with 616 additions and 463 deletions.
2 changes: 1 addition & 1 deletion dev
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,7 @@ fi
set -euo pipefail

# Bump this counter to force rebuilding `dev` on all machines.
DEV_VERSION=84
DEV_VERSION=85

THIS_DIR=$(cd "$(dirname "$0")" && pwd)
BINARY_DIR=$THIS_DIR/bin/dev-versions
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/changefeedccl/cdceval/expr_eval_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@ func TestEvaluator(t *testing.T) {
defer s.Stopper().Stop(context.Background())

sqlDB := sqlutils.MakeSQLRunner(db)
sqlDB.Exec(t, "SET CLUSTER SETTING kv.rangefeed.enabled = true")
sqlDB.Exec(t, `CREATE TYPE status AS ENUM ('open', 'closed', 'inactive')`)
sqlDB.Exec(t, `
CREATE TABLE foo (
Expand Down
3 changes: 3 additions & 0 deletions pkg/ccl/changefeedccl/cdcevent/event_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -140,6 +140,7 @@ func TestEventDecoder(t *testing.T) {
defer s.Stopper().Stop(context.Background())

sqlDB := sqlutils.MakeSQLRunner(db)
sqlDB.Exec(t, "SET CLUSTER SETTING kv.rangefeed.enabled = true")
sqlDB.Exec(t, `CREATE TYPE status AS ENUM ('open', 'closed', 'inactive')`)
sqlDB.Exec(t, `
CREATE TABLE foo (
Expand Down Expand Up @@ -419,6 +420,7 @@ func TestEventColumnOrderingWithSchemaChanges(t *testing.T) {
defer s.Stopper().Stop(context.Background())

sqlDB := sqlutils.MakeSQLRunner(db)
sqlDB.Exec(t, "SET CLUSTER SETTING kv.rangefeed.enabled = true")
// Use alter column type to force column reordering.
sqlDB.Exec(t, `SET enable_experimental_alter_column_type_general = true`)

Expand Down Expand Up @@ -761,6 +763,7 @@ func BenchmarkEventDecoder(b *testing.B) {
defer s.Stopper().Stop(context.Background())

sqlDB := sqlutils.MakeSQLRunner(db)
sqlDB.Exec(b, "SET CLUSTER SETTING kv.rangefeed.enabled = true")
sqlDB.Exec(b, `
CREATE TABLE foo (
a INT,
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/changefeedccl/cdctest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ go_library(
"//pkg/keys",
"//pkg/kv/kvclient/rangefeed",
"//pkg/kv/kvpb",
"//pkg/kv/kvserver",
"//pkg/roachpb",
"//pkg/sql",
"//pkg/sql/catalog",
Expand Down
8 changes: 8 additions & 0 deletions pkg/ccl/changefeedccl/cdctest/row.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed"
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
Expand All @@ -37,6 +38,13 @@ func MakeRangeFeedValueReader(
) (func(t testing.TB) *kvpb.RangeFeedValue, func()) {
t.Helper()
execCfg := execCfgI.(sql.ExecutorConfig)

// Rangefeeds might still work even when this setting is false because span
// configs may enable them, but relying on span configs can be prone to
// issues as seen in #109507. Therefore, we assert that the cluster setting
// is set.
require.True(t, kvserver.RangefeedEnabled.Get(&execCfg.Settings.SV))

rows := make(chan *kvpb.RangeFeedValue)
ctx, cleanup := context.WithCancel(context.Background())

Expand Down
99 changes: 53 additions & 46 deletions pkg/ccl/changefeedccl/changefeed_processors.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,6 @@ type changeAggregator struct {
errCh chan error
// kvFeedDoneCh is closed when the kvfeed exits.
kvFeedDoneCh chan struct{}
kvFeedMemMon *mon.BytesMonitor

// drainWatchCh is signaled if the job registry on this node is being
// drained, which is a proxy for the node being drained. If a drain occurs,
Expand Down Expand Up @@ -253,17 +252,6 @@ func (ca *changeAggregator) Start(ctx context.Context) {
ca.knobs = *cfKnobs
}

// TODO(yevgeniy): Introduce separate changefeed monitor that's a parent
// for all changefeeds to control memory allocated to all changefeeds.
pool := ca.flowCtx.Cfg.BackfillerMonitor
if ca.knobs.MemMonitor != nil {
pool = ca.knobs.MemMonitor
}
limit := changefeedbase.PerChangefeedMemLimit.Get(&ca.flowCtx.Cfg.Settings.SV)
kvFeedMemMon := mon.NewMonitorInheritWithLimit("kvFeed", limit, pool)
kvFeedMemMon.StartNoReserved(ctx, pool)
ca.kvFeedMemMon = kvFeedMemMon

// The job registry has a set of metrics used to monitor the various jobs it
// runs. They're all stored as the `metric.Struct` interface because of
// dependency cycles.
Expand Down Expand Up @@ -313,7 +301,14 @@ func (ca *changeAggregator) Start(ctx context.Context) {
kvFeedHighWater = ca.spec.Feed.StatementTime
}

ca.eventProducer, err = ca.startKVFeed(ctx, spans, kvFeedHighWater, needsInitialScan, feed)
// TODO(yevgeniy): Introduce separate changefeed monitor that's a parent
// for all changefeeds to control memory allocated to all changefeeds.
pool := ca.flowCtx.Cfg.BackfillerMonitor
if ca.knobs.MemMonitor != nil {
pool = ca.knobs.MemMonitor
}
limit := changefeedbase.PerChangefeedMemLimit.Get(&ca.flowCtx.Cfg.Settings.SV)
ca.eventProducer, ca.kvFeedDoneCh, ca.errCh, err = ca.startKVFeed(ctx, spans, kvFeedHighWater, needsInitialScan, feed, pool, limit)
if err != nil {
ca.MoveToDraining(err)
ca.cancel()
Expand Down Expand Up @@ -362,39 +357,51 @@ func (ca *changeAggregator) startKVFeed(
initialHighWater hlc.Timestamp,
needsInitialScan bool,
config ChangefeedConfig,
) (kvevent.Reader, error) {
parentMemMon *mon.BytesMonitor,
memLimit int64,
) (kvevent.Reader, chan struct{}, chan error, error) {
cfg := ca.flowCtx.Cfg
kvFeedMemMon := mon.NewMonitorInheritWithLimit("kvFeed", memLimit, parentMemMon)
kvFeedMemMon.StartNoReserved(ctx, parentMemMon)
buf := kvevent.NewThrottlingBuffer(
kvevent.NewMemBuffer(ca.kvFeedMemMon.MakeBoundAccount(), &cfg.Settings.SV, &ca.metrics.KVFeedMetrics),
kvevent.NewMemBuffer(kvFeedMemMon.MakeBoundAccount(), &cfg.Settings.SV, &ca.metrics.KVFeedMetrics),
cdcutils.NodeLevelThrottler(&cfg.Settings.SV, &ca.metrics.ThrottleMetrics))

// KVFeed takes ownership of the kvevent.Writer portion of the buffer, while
// we return the kvevent.Reader part to the caller.
kvfeedCfg, err := ca.makeKVFeedCfg(ctx, config, spans, buf, initialHighWater, needsInitialScan)
kvfeedCfg, err := ca.makeKVFeedCfg(ctx, config, spans, buf, initialHighWater, needsInitialScan, kvFeedMemMon)
if err != nil {
return nil, err
return nil, nil, nil, err
}

// Give errCh enough buffer both possible errors from supporting goroutines,
// but only the first one is ever used.
ca.errCh = make(chan error, 2)
ca.kvFeedDoneCh = make(chan struct{})
errCh := make(chan error, 1)
doneCh := make(chan struct{})
// If RunAsyncTask immediately returns an error, the kvfeed was not run and
// will not run.
if err := ca.flowCtx.Stopper().RunAsyncTask(ctx, "changefeed-poller", func(ctx context.Context) {
defer close(ca.kvFeedDoneCh)
// Trying to call MoveToDraining here is racy (`MoveToDraining called in
// state stateTrailingMeta`), so return the error via a channel.
ca.errCh <- kvfeed.Run(ctx, kvfeedCfg)
defer close(doneCh)
defer kvFeedMemMon.Stop(ctx)
errCh <- kvfeed.Run(ctx, kvfeedCfg)
}); err != nil {
// If err != nil then the RunAsyncTask closure never ran, which means we
// need to manually close ca.kvFeedDoneCh so `(*changeAggregator).close`
// doesn't hang.
close(ca.kvFeedDoneCh)
ca.errCh <- err
ca.cancel()
return nil, err
return nil, nil, nil, err
}

return buf, nil
return buf, doneCh, errCh, nil
}

func (ca *changeAggregator) waitForKVFeedDone() {
if ca.kvFeedDoneCh != nil {
<-ca.kvFeedDoneCh
}
}

func (ca *changeAggregator) checkKVFeedErr() error {
select {
case err := <-ca.errCh:
return err
default:
return nil
}
}

func (ca *changeAggregator) makeKVFeedCfg(
Expand All @@ -404,6 +411,7 @@ func (ca *changeAggregator) makeKVFeedCfg(
buf kvevent.Writer,
initialHighWater hlc.Timestamp,
needsInitialScan bool,
memMon *mon.BytesMonitor,
) (kvfeed.Config, error) {
schemaChange, err := config.Opts.GetSchemaChangeHandlingOptions()
if err != nil {
Expand Down Expand Up @@ -435,7 +443,7 @@ func (ca *changeAggregator) makeKVFeedCfg(
Metrics: &ca.metrics.KVFeedMetrics,
OnBackfillCallback: ca.sliMetrics.getBackfillCallback(),
OnBackfillRangeCallback: ca.sliMetrics.getBackfillRangeCallback(),
MM: ca.kvFeedMemMon,
MM: memMon,
InitialHighWater: initialHighWater,
EndTime: config.EndTime,
WithDiff: filters.WithDiff,
Expand Down Expand Up @@ -499,18 +507,20 @@ func (ca *changeAggregator) setupSpansAndFrontier() (spans []roachpb.Span, err e

// close has two purposes: to synchronize on the completion of the helper
// goroutines created by the Start method, and to clean up any resources used by
// the processor. Due to the fact that this method may be called even if the
// processor did not finish completion, there is an excessive amount of nil
// checking.
// the processor.
//
// Due to the fact that this method may be called even if the processor did not
// finish completion, there is an excessive amount of nil checking. For example,
// (*changeAggregator) Start() may encounter an error and move the processor to
// draining before one of the fields below (ex. ca.drainDone) is set.
func (ca *changeAggregator) close() {
if ca.Closed {
return
}
ca.cancel()
// Wait for the poller to finish shutting down.
if ca.kvFeedDoneCh != nil {
<-ca.kvFeedDoneCh
}
ca.waitForKVFeedDone()

if ca.drainDone != nil {
ca.drainDone()
}
Expand All @@ -526,9 +536,7 @@ func (ca *changeAggregator) close() {
_ = ca.sink.Close()
}
ca.memAcc.Close(ca.Ctx())
if ca.kvFeedMemMon != nil {
ca.kvFeedMemMon.Stop(ca.Ctx())
}

ca.MemMonitor.Stop(ca.Ctx())
ca.InternalClose()
}
Expand Down Expand Up @@ -607,11 +615,10 @@ func (ca *changeAggregator) Next() (rowenc.EncDatumRow, *execinfrapb.ProducerMet
err = nil
}
} else {
select {
// If the poller errored first, that's the
// interesting one, so overwrite `err`.
case err = <-ca.errCh:
default:
if kvFeedErr := ca.checkKVFeedErr(); kvFeedErr != nil {
err = kvFeedErr
}
}
// Shut down the poller if it wasn't already.
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/changefeedccl/parquet_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@ func TestParquetRows(t *testing.T) {
maxRowGroupSize := int64(2)

sqlDB := sqlutils.MakeSQLRunner(db)
sqlDB.Exec(t, "SET CLUSTER SETTING kv.rangefeed.enabled = true")

for _, tc := range []struct {
testName string
Expand Down
15 changes: 9 additions & 6 deletions pkg/cmd/dev/test.go
Original file line number Diff line number Diff line change
Expand Up @@ -119,7 +119,7 @@ pkg/kv/kvserver:kvserver_test) instead.`,
// visible.
testCmd.Flags().BoolP(vFlag, "v", false, "show testing process output")
testCmd.Flags().Bool(changedFlag, false, "automatically determine tests to run. This is done on a best-effort basis by asking git which files have changed. Only .go files and files in testdata/ directories are factored into this analysis.")
testCmd.Flags().Int(countFlag, 1, "run test the given number of times")
testCmd.Flags().Int(countFlag, 0, "run test the given number of times")
testCmd.Flags().BoolP(showLogsFlag, "", false, "show crdb logs in-line")
testCmd.Flags().Bool(stressFlag, false, "run tests under stress")
testCmd.Flags().Bool(raceFlag, false, "run tests using race builds")
Expand Down Expand Up @@ -281,9 +281,6 @@ func (d *dev) test(cmd *cobra.Command, commandLine []string) error {
}

args = append(args, testTargets...)
if ignoreCache {
args = append(args, "--nocache_test_results")
}
args = append(args, "--test_env=GOTRACEBACK=all")

if rewrite {
Expand Down Expand Up @@ -336,7 +333,7 @@ func (d *dev) test(cmd *cobra.Command, commandLine []string) error {
}

if stress {
if count == 1 {
if count == 0 {
// Default to 1000 unless a different count was provided.
count = 1000
}
Expand All @@ -362,7 +359,9 @@ func (d *dev) test(cmd *cobra.Command, commandLine []string) error {
if showLogs {
args = append(args, "--test_arg", "-show-logs")
}
if count != 1 {
if count == 1 {
ignoreCache = true
} else if count != 0 {
args = append(args, fmt.Sprintf("--runs_per_test=%d", count))
}
if vModule != "" {
Expand All @@ -379,6 +378,10 @@ func (d *dev) test(cmd *cobra.Command, commandLine []string) error {
args = append(args, "--test_sharding_strategy=disabled")
}

if ignoreCache {
args = append(args, "--nocache_test_results")
}

if len(goTags) > 0 {
args = append(args, "--define", "gotags=bazel,gss,"+strings.Join(goTags, ","))
}
Expand Down
17 changes: 11 additions & 6 deletions pkg/cmd/dev/testdata/datadriven/test
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ bazel test pkg/util/tracing:all --test_env=GOTRACEBACK=all '--test_filter=TestSt
exec
dev test pkg/util/tracing -f TestStartChild* --ignore-cache
----
bazel test pkg/util/tracing:all --nocache_test_results --test_env=GOTRACEBACK=all '--test_filter=TestStartChild*' --test_sharding_strategy=disabled --test_output errors --build_event_binary_file=/tmp/path
bazel test pkg/util/tracing:all --test_env=GOTRACEBACK=all '--test_filter=TestStartChild*' --test_sharding_strategy=disabled --nocache_test_results --test_output errors --build_event_binary_file=/tmp/path

exec
dev test //pkg/testutils --timeout=10s
Expand Down Expand Up @@ -62,7 +62,7 @@ exec
dev test pkg/cmd/dev -f TestDataDriven/test --rewrite -v
----
bazel info workspace --color=no
bazel test pkg/cmd/dev:all --nocache_test_results --test_env=GOTRACEBACK=all --test_env=COCKROACH_WORKSPACE=crdb-checkout --test_arg -rewrite --sandbox_writable_path=crdb-checkout/pkg/cmd/dev --test_filter=TestDataDriven/test --test_arg -test.v --test_sharding_strategy=disabled --test_output all --build_event_binary_file=/tmp/path
bazel test pkg/cmd/dev:all --test_env=GOTRACEBACK=all --test_env=COCKROACH_WORKSPACE=crdb-checkout --test_arg -rewrite --sandbox_writable_path=crdb-checkout/pkg/cmd/dev --test_filter=TestDataDriven/test --test_arg -test.v --test_sharding_strategy=disabled --nocache_test_results --test_output all --build_event_binary_file=/tmp/path

exec
dev test pkg/server -f=TestSpanStatsResponse -v --count=5 --vmodule=raft=1
Expand All @@ -84,7 +84,7 @@ exec
dev test pkg/ccl/logictestccl -f=TestTenantLogic/3node-tenant/system -v --rewrite
----
bazel info workspace --color=no
bazel test pkg/ccl/logictestccl:all --nocache_test_results --test_env=GOTRACEBACK=all --test_env=COCKROACH_WORKSPACE=crdb-checkout --test_arg -rewrite --sandbox_writable_path=crdb-checkout/pkg/ccl/logictestccl --sandbox_writable_path=crdb-checkout/pkg/sql/logictest --sandbox_writable_path=crdb-checkout/pkg/sql/opt/exec/execbuilder --test_filter=TestTenantLogic/3node-tenant/system --test_arg -test.v --test_sharding_strategy=disabled --test_output all --build_event_binary_file=/tmp/path
bazel test pkg/ccl/logictestccl:all --test_env=GOTRACEBACK=all --test_env=COCKROACH_WORKSPACE=crdb-checkout --test_arg -rewrite --sandbox_writable_path=crdb-checkout/pkg/ccl/logictestccl --sandbox_writable_path=crdb-checkout/pkg/sql/logictest --sandbox_writable_path=crdb-checkout/pkg/sql/opt/exec/execbuilder --test_filter=TestTenantLogic/3node-tenant/system --test_arg -test.v --test_sharding_strategy=disabled --nocache_test_results --test_output all --build_event_binary_file=/tmp/path

exec
dev test pkg/spanconfig/spanconfigkvsubscriber -f=TestDecodeSpanTargets -v --stream-output
Expand Down Expand Up @@ -115,19 +115,19 @@ exec
dev test pkg/sql/schemachanger --rewrite -v
----
bazel info workspace --color=no
bazel test pkg/sql/schemachanger:all --nocache_test_results --test_env=GOTRACEBACK=all --test_env=COCKROACH_WORKSPACE=crdb-checkout --test_arg -rewrite --sandbox_writable_path=crdb-checkout/pkg/sql/schemachanger --test_arg -test.v --test_sharding_strategy=disabled --test_output all --build_event_binary_file=/tmp/path
bazel test pkg/sql/schemachanger:all --test_env=GOTRACEBACK=all --test_env=COCKROACH_WORKSPACE=crdb-checkout --test_arg -rewrite --sandbox_writable_path=crdb-checkout/pkg/sql/schemachanger --test_arg -test.v --test_sharding_strategy=disabled --nocache_test_results --test_output all --build_event_binary_file=/tmp/path

exec
dev test pkg/sql/opt/xform --rewrite
----
bazel info workspace --color=no
bazel test pkg/sql/opt/xform:all --nocache_test_results --test_env=GOTRACEBACK=all --test_env=COCKROACH_WORKSPACE=crdb-checkout --test_arg -rewrite --sandbox_writable_path=crdb-checkout/pkg/sql/opt/xform --sandbox_writable_path=crdb-checkout/pkg/sql/opt/testutils/opttester/testfixtures --test_sharding_strategy=disabled --test_output errors --build_event_binary_file=/tmp/path
bazel test pkg/sql/opt/xform:all --test_env=GOTRACEBACK=all --test_env=COCKROACH_WORKSPACE=crdb-checkout --test_arg -rewrite --sandbox_writable_path=crdb-checkout/pkg/sql/opt/xform --sandbox_writable_path=crdb-checkout/pkg/sql/opt/testutils/opttester/testfixtures --test_sharding_strategy=disabled --nocache_test_results --test_output errors --build_event_binary_file=/tmp/path

exec
dev test pkg/sql/... --rewrite
----
bazel info workspace --color=no
bazel test pkg/sql/... --nocache_test_results --test_env=GOTRACEBACK=all --test_env=COCKROACH_WORKSPACE=crdb-checkout --test_arg -rewrite --sandbox_writable_path=crdb-checkout/pkg/sql --test_sharding_strategy=disabled --test_output errors --build_event_binary_file=/tmp/path
bazel test pkg/sql/... --test_env=GOTRACEBACK=all --test_env=COCKROACH_WORKSPACE=crdb-checkout --test_arg -rewrite --sandbox_writable_path=crdb-checkout/pkg/sql --test_sharding_strategy=disabled --nocache_test_results --test_output errors --build_event_binary_file=/tmp/path

exec
dev test pkg/spanconfig/spanconfigstore --test-args '-test.timeout=0.5s'
Expand Down Expand Up @@ -162,3 +162,8 @@ dev test pkg/spanconfig/spanconfigstore --stress --count 250
----
getenv DEV_I_UNDERSTAND_ABOUT_STRESS
bazel test pkg/spanconfig/spanconfigstore:all --test_env=GOTRACEBACK=all --test_env=COCKROACH_STRESS=true --notest_keep_going --runs_per_test=250 --test_output errors --build_event_binary_file=/tmp/path

exec
dev test pkg/spanconfig/spanconfigstore --count 1
----
bazel test pkg/spanconfig/spanconfigstore:all --test_env=GOTRACEBACK=all --nocache_test_results --test_output errors --build_event_binary_file=/tmp/path
1 change: 1 addition & 0 deletions pkg/kv/kvserver/concurrency/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ go_library(
"//pkg/settings/cluster",
"//pkg/storage/enginepb",
"//pkg/util/buildutil",
"//pkg/util/container/list",
"//pkg/util/hlc",
"//pkg/util/humanizeutil",
"//pkg/util/log",
Expand Down
Loading

0 comments on commit ee7cdcf

Please sign in to comment.