Skip to content

Commit

Permalink
Merge #123233
Browse files Browse the repository at this point in the history
123233: stats: add a testing knob to disable initial table collection r=yuzefovich a=yuzefovich

This is needed to stabilize `TestScheduledJobsConsumption` on 24.1 and earlier branches. This test became flaky due to the partial backport of #121861 to all branches.

Fixes: #122336.

Release note: None

Co-authored-by: Yahor Yuzefovich <yahor@cockroachlabs.com>
  • Loading branch information
craig[bot] and yuzefovich committed May 3, 2024
2 parents d15210e + 0a91274 commit 23452f7
Show file tree
Hide file tree
Showing 6 changed files with 38 additions and 11 deletions.
1 change: 1 addition & 0 deletions pkg/base/testing_knobs.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,4 +58,5 @@ type TestingKnobs struct {
LOQRecovery ModuleTestingKnobs
KeyVisualizer ModuleTestingKnobs
TenantCapabilitiesTestingKnobs ModuleTestingKnobs
TableStatsKnobs ModuleTestingKnobs
}
10 changes: 6 additions & 4 deletions pkg/ccl/multitenantccl/tenantcostclient/tenant_side_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1123,6 +1123,9 @@ func TestScheduledJobsConsumption(t *testing.T) {
RetryMaxDelay: &zeroDuration,
},
},
TableStatsKnobs: &stats.TableStatsTestingKnobs{
DisableInitialTableCollection: true,
},
},
})

Expand All @@ -1142,10 +1145,9 @@ func TestScheduledJobsConsumption(t *testing.T) {
after.Sub(&before)
require.Zero(t, after.WriteBatches)
require.Zero(t, after.WriteBytes)
// Expect up to 3 batches for initial auto-stats query, schema catalog fill,
// and anything else that happens once during server startup but might not be
// done by this point.
require.LessOrEqual(t, after.ReadBatches, uint64(3))
// Expect up to 2 batches for schema catalog fill and anything else that
// happens once during server startup but might not be done by this point.
require.LessOrEqual(t, after.ReadBatches, uint64(2))

// Make sure that at least 100 writes (deletes) are reported. The TTL job
// should not be exempt from cost control.
Expand Down
5 changes: 5 additions & 0 deletions pkg/server/server_sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -1107,6 +1107,10 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) {
if externalConnKnobs := cfg.TestingKnobs.ExternalConnection; externalConnKnobs != nil {
execCfg.ExternalConnectionTestingKnobs = externalConnKnobs.(*externalconn.TestingKnobs)
}
var tableStatsTestingKnobs *stats.TableStatsTestingKnobs
if tableStatsKnobs := cfg.TestingKnobs.TableStatsKnobs; tableStatsKnobs != nil {
tableStatsTestingKnobs = tableStatsKnobs.(*stats.TableStatsTestingKnobs)
}

// Set up internal memory metrics for use by internal SQL executors.
// Don't add them to the registry now because it will be added as part of pgServer metrics.
Expand Down Expand Up @@ -1160,6 +1164,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) {
internalDB,
execCfg.TableStatsCache,
stats.DefaultAsOfTime,
tableStatsTestingKnobs,
)
execCfg.StatsRefresher = statsRefresher
distSQLServer.ServerConfig.StatsRefresher = statsRefresher
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/stats/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ go_library(
importpath = "github.com/cockroachdb/cockroach/pkg/sql/stats",
visibility = ["//visibility:public"],
deps = [
"//pkg/base",
"//pkg/clusterversion",
"//pkg/jobs/jobspb",
"//pkg/keys",
Expand Down
18 changes: 18 additions & 0 deletions pkg/sql/stats/automatic_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"sync"
"time"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
Expand Down Expand Up @@ -219,6 +220,7 @@ type Refresher struct {
internalDB descs.DB
cache *TableStatisticsCache
randGen autoStatsRand
knobs *TableStatsTestingKnobs

// mutations is the buffered channel used to pass messages containing
// metadata about SQL mutations to the background Refresher thread.
Expand Down Expand Up @@ -277,13 +279,25 @@ type settingOverride struct {
settings catpb.AutoStatsSettings
}

// TableStatsTestingKnobs contains testing knobs for table statistics.
type TableStatsTestingKnobs struct {
// DisableInitialTableCollection, if set, indicates that the "initial table
// collection" performed by the Refresher should be skipped.
DisableInitialTableCollection bool
}

var _ base.ModuleTestingKnobs = &TableStatsTestingKnobs{}

func (k *TableStatsTestingKnobs) ModuleTestingKnobs() {}

// MakeRefresher creates a new Refresher.
func MakeRefresher(
ambientCtx log.AmbientContext,
st *cluster.Settings,
internalDB descs.DB,
cache *TableStatisticsCache,
asOfTime time.Duration,
knobs *TableStatsTestingKnobs,
) *Refresher {
randSource := rand.NewSource(rand.Int63())

Expand All @@ -293,6 +307,7 @@ func MakeRefresher(
internalDB: internalDB,
cache: cache,
randGen: makeAutoStatsRand(randSource),
knobs: knobs,
mutations: make(chan mutation, refreshChanBufferLen),
settings: make(chan settingOverride, refreshChanBufferLen),
asOfTime: asOfTime,
Expand Down Expand Up @@ -428,6 +443,9 @@ func (r *Refresher) Start(
for {
select {
case <-initialTableCollection:
if r.knobs != nil && r.knobs.DisableInitialTableCollection {
continue
}
r.ensureAllTables(ctx, initialTableCollectionDelay)
if len(r.mutationCounts) > 0 {
ensuringAllTables = true
Expand Down
14 changes: 7 additions & 7 deletions pkg/sql/stats/automatic_stats_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -73,7 +73,7 @@ func TestMaybeRefreshStats(t *testing.T) {
s.InternalDB().(descs.DB),
)
require.NoError(t, cache.Start(ctx, codec, s.RangeFeedFactory().(*rangefeed.Factory)))
refresher := MakeRefresher(s.AmbientCtx(), st, internalDB, cache, time.Microsecond /* asOfTime */)
refresher := MakeRefresher(s.AmbientCtx(), st, internalDB, cache, time.Microsecond /* asOfTime */, nil /* knobs */)

// There should not be any stats yet.
if err := checkStatsCount(ctx, cache, descA, 0 /* expected */); err != nil {
Expand Down Expand Up @@ -197,7 +197,7 @@ func TestEnsureAllTablesQueries(t *testing.T) {
s.InternalDB().(descs.DB),
)
require.NoError(t, cache.Start(ctx, codec, s.RangeFeedFactory().(*rangefeed.Factory)))
r := MakeRefresher(s.AmbientCtx(), st, internalDB, cache, time.Microsecond /* asOfTime */)
r := MakeRefresher(s.AmbientCtx(), st, internalDB, cache, time.Microsecond /* asOfTime */, nil /* knobs */)

// Exclude the 3 system tables which don't use autostats.
systemTablesWithStats := bootstrap.NumSystemTablesForSystemTenant - 3
Expand Down Expand Up @@ -299,7 +299,7 @@ func BenchmarkEnsureAllTables(b *testing.B) {
s.InternalDB().(descs.DB),
)
require.NoError(b, cache.Start(ctx, codec, s.RangeFeedFactory().(*rangefeed.Factory)))
r := MakeRefresher(s.AmbientCtx(), st, internalDB, cache, time.Microsecond /* asOfTime */)
r := MakeRefresher(s.AmbientCtx(), st, internalDB, cache, time.Microsecond /* asOfTime */, nil /* knobs */)

b.ResetTimer()
for i := 0; i < b.N; i++ {
Expand Down Expand Up @@ -372,7 +372,7 @@ func TestAverageRefreshTime(t *testing.T) {
s.InternalDB().(descs.DB),
)
require.NoError(t, cache.Start(ctx, codec, s.RangeFeedFactory().(*rangefeed.Factory)))
refresher := MakeRefresher(s.AmbientCtx(), st, internalDB, cache, time.Microsecond /* asOfTime */)
refresher := MakeRefresher(s.AmbientCtx(), st, internalDB, cache, time.Microsecond /* asOfTime */, nil /* knobs */)

// curTime is used as the current time throughout the test to ensure that the
// calculated average refresh time is consistent even if there are delays due
Expand Down Expand Up @@ -619,7 +619,7 @@ func TestAutoStatsReadOnlyTables(t *testing.T) {
s.InternalDB().(descs.DB),
)
require.NoError(t, cache.Start(ctx, codec, s.RangeFeedFactory().(*rangefeed.Factory)))
refresher := MakeRefresher(s.AmbientCtx(), st, internalDB, cache, time.Microsecond /* asOfTime */)
refresher := MakeRefresher(s.AmbientCtx(), st, internalDB, cache, time.Microsecond /* asOfTime */, nil /* knobs */)

AutomaticStatisticsClusterMode.Override(ctx, &st.SV, true)

Expand Down Expand Up @@ -674,7 +674,7 @@ func TestAutoStatsOnStartupClusterSettingOff(t *testing.T) {
s.InternalDB().(descs.DB),
)
require.NoError(t, cache.Start(ctx, codec, s.RangeFeedFactory().(*rangefeed.Factory)))
refresher := MakeRefresher(s.AmbientCtx(), st, internalDB, cache, time.Microsecond /* asOfTime */)
refresher := MakeRefresher(s.AmbientCtx(), st, internalDB, cache, time.Microsecond /* asOfTime */, nil /* knobs */)

// Refresher start should trigger stats collection on t.a.
if err := refresher.Start(
Expand Down Expand Up @@ -721,7 +721,7 @@ func TestNoRetryOnFailure(t *testing.T) {
s.InternalDB().(descs.DB),
)
require.NoError(t, cache.Start(ctx, codec, s.RangeFeedFactory().(*rangefeed.Factory)))
r := MakeRefresher(s.AmbientCtx(), st, internalDB, cache, time.Microsecond /* asOfTime */)
r := MakeRefresher(s.AmbientCtx(), st, internalDB, cache, time.Microsecond /* asOfTime */, nil /* knobs */)

// Try to refresh stats on a table that doesn't exist.
r.maybeRefreshStats(
Expand Down

0 comments on commit 23452f7

Please sign in to comment.