From fb8f99b155bfa573a26938b66a9d3e5e11caecba Mon Sep 17 00:00:00 2001 From: Eric Harmeling Date: Fri, 11 Aug 2023 11:03:19 -0400 Subject: [PATCH 1/8] metrics: assign histogram metric type on histogram construction 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 --- pkg/util/metric/metric.go | 2 ++ pkg/util/schedulerlatency/histogram.go | 1 + 2 files changed, 3 insertions(+) diff --git a/pkg/util/metric/metric.go b/pkg/util/metric/metric.go index 9c62c0cf641f..aa5d7628ab6d 100644 --- a/pkg/util/metric/metric.go +++ b/pkg/util/metric/metric.go @@ -255,6 +255,7 @@ type HistogramOptions struct { } func NewHistogram(opt HistogramOptions) IHistogram { + opt.Metadata.MetricType = prometheusgo.MetricType_HISTOGRAM if hdrEnabled && opt.Mode != HistogramModePrometheus { if opt.Mode == HistogramModePreferHdrLatency { return NewHdrLatency(opt.Metadata, opt.Duration) @@ -492,6 +493,7 @@ func NewManualWindowHistogram( panic(err.Error()) } + meta.MetricType = prometheusgo.MetricType_HISTOGRAM h := &ManualWindowHistogram{ Metadata: meta, } diff --git a/pkg/util/schedulerlatency/histogram.go b/pkg/util/schedulerlatency/histogram.go index 6cb7c35e575f..0db51f01b860 100644 --- a/pkg/util/schedulerlatency/histogram.go +++ b/pkg/util/schedulerlatency/histogram.go @@ -51,6 +51,7 @@ func newRuntimeHistogram(metadata metric.Metadata, buckets []float64) *runtimeHi if buckets[0] == math.Inf(-1) { buckets = buckets[1:] } + metadata.MetricType = prometheusgo.MetricType_HISTOGRAM h := &runtimeHistogram{ Metadata: metadata, // Go runtime histograms as of go1.19 are always in seconds whereas From d4fd4b46243736c622bfe78d8e5164ae16e38f2b Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Wed, 23 Aug 2023 15:34:57 -0700 Subject: [PATCH 2/8] stats: adjust most tests to work with test tenant A couple of tests are skipped with newly filed issues (I applied all adjustments I could think of quickly, and these two tests still didn't work). Release note: None --- pkg/sql/stats/automatic_stats_test.go | 97 ++++++++++++++------------ pkg/sql/stats/create_stats_job_test.go | 34 ++++++--- pkg/sql/stats/delete_stats_test.go | 35 +++++++--- pkg/sql/stats/stats_cache_test.go | 36 +++++----- 4 files changed, 120 insertions(+), 82 deletions(-) diff --git a/pkg/sql/stats/automatic_stats_test.go b/pkg/sql/stats/automatic_stats_test.go index 18aedacea50d..0bb7ddfbdd22 100644 --- a/pkg/sql/stats/automatic_stats_test.go +++ b/pkg/sql/stats/automatic_stats_test.go @@ -48,11 +48,11 @@ func TestMaybeRefreshStats(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) - defer sqlDB.Close() - defer s.Stopper().Stop(ctx) + srv, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() + codec, st := s.Codec(), s.ClusterSettings() - st := cluster.MakeTestingClusterSettings() evalCtx := eval.NewTestingEvalContext(st) defer evalCtx.Stop(ctx) @@ -67,13 +67,13 @@ func TestMaybeRefreshStats(t *testing.T) { CREATE VIEW t.vw AS SELECT k, k+1 FROM t.a;`) executor := s.InternalExecutor().(isql.Executor) - descA := desctestutils.TestingGetPublicTableDescriptor(s.DB(), keys.SystemSQLCodec, "t", "a") + descA := desctestutils.TestingGetPublicTableDescriptor(s.DB(), codec, "t", "a") cache := NewTableStatisticsCache( 10, /* cacheSize */ s.ClusterSettings(), s.InternalDB().(descs.DB), ) - require.NoError(t, cache.Start(ctx, keys.SystemSQLCodec, s.RangeFeedFactory().(*rangefeed.Factory))) + require.NoError(t, cache.Start(ctx, codec, s.RangeFeedFactory().(*rangefeed.Factory))) refresher := MakeRefresher(s.AmbientCtx(), st, executor, cache, time.Microsecond /* asOfTime */) // There should not be any stats yet. @@ -132,7 +132,7 @@ func TestMaybeRefreshStats(t *testing.T) { // Auto stats collection on any system table except system.lease and // system.table_statistics should succeed. descRoleOptions := - desctestutils.TestingGetPublicTableDescriptor(s.DB(), keys.SystemSQLCodec, "system", "role_options") + desctestutils.TestingGetPublicTableDescriptor(s.DB(), codec, "system", "role_options") refresher.maybeRefreshStats( ctx, s.Stopper(), descRoleOptions.GetID(), nil /* explicitSettings */, 10000 /* rowsAffected */, time.Microsecond, /* asOf */ ) @@ -142,7 +142,7 @@ func TestMaybeRefreshStats(t *testing.T) { // Auto stats collection on system.lease should fail (no stats should be collected). descLease := - desctestutils.TestingGetPublicTableDescriptor(s.DB(), keys.SystemSQLCodec, "system", "lease") + desctestutils.TestingGetPublicTableDescriptor(s.DB(), codec, "system", "lease") refresher.maybeRefreshStats( ctx, s.Stopper(), descLease.GetID(), nil /* explicitSettings */, 10000 /* rowsAffected */, time.Microsecond, /* asOf */ ) @@ -152,7 +152,7 @@ func TestMaybeRefreshStats(t *testing.T) { // Auto stats collection on system.table_statistics should fail (no stats should be collected). descTableStats := - desctestutils.TestingGetPublicTableDescriptor(s.DB(), keys.SystemSQLCodec, "system", "table_statistics") + desctestutils.TestingGetPublicTableDescriptor(s.DB(), codec, "system", "table_statistics") refresher.maybeRefreshStats( ctx, s.Stopper(), descTableStats.GetID(), nil /* explicitSettings */, 10000 /* rowsAffected */, time.Microsecond, /* asOf */ ) @@ -163,7 +163,7 @@ func TestMaybeRefreshStats(t *testing.T) { // Ensure that attempt to refresh stats on view does not result in re- // enqueuing the attempt. // TODO(rytaft): Should not enqueue views to begin with. - descVW := desctestutils.TestingGetPublicTableDescriptor(s.DB(), keys.SystemSQLCodec, "t", "vw") + descVW := desctestutils.TestingGetPublicTableDescriptor(s.DB(), codec, "t", "vw") refresher.maybeRefreshStats( ctx, s.Stopper(), descVW.GetID(), nil /* explicitSettings */, 0 /* rowsAffected */, time.Microsecond, /* asOf */ ) @@ -179,11 +179,10 @@ func TestEnsureAllTablesQueries(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) - defer sqlDB.Close() - defer s.Stopper().Stop(ctx) - - st := cluster.MakeTestingClusterSettings() + srv, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() + codec, st := s.Codec(), s.ClusterSettings() sqlRun := sqlutils.MakeSQLRunner(sqlDB) sqlRun.Exec(t, @@ -198,11 +197,15 @@ func TestEnsureAllTablesQueries(t *testing.T) { s.ClusterSettings(), s.InternalDB().(descs.DB), ) - require.NoError(t, cache.Start(ctx, keys.SystemSQLCodec, s.RangeFeedFactory().(*rangefeed.Factory))) + require.NoError(t, cache.Start(ctx, codec, s.RangeFeedFactory().(*rangefeed.Factory))) r := MakeRefresher(s.AmbientCtx(), st, executor, cache, time.Microsecond /* asOfTime */) // Exclude the 3 system tables which don't use autostats. systemTablesWithStats := bootstrap.NumSystemTablesForSystemTenant - 3 + if srv.StartedDefaultTestTenant() { + // Exclude system tables that are not present in the tenant key space. + systemTablesWithStats -= 6 + } numUserTablesWithStats := 2 // This now includes 36 system tables as well as the 2 created above. @@ -314,11 +317,11 @@ func TestAverageRefreshTime(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) - defer sqlDB.Close() - defer s.Stopper().Stop(ctx) + srv, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() + codec, st := s.Codec(), s.ClusterSettings() - st := cluster.MakeTestingClusterSettings() evalCtx := eval.NewTestingEvalContext(st) defer evalCtx.Stop(ctx) @@ -331,13 +334,13 @@ func TestAverageRefreshTime(t *testing.T) { INSERT INTO t.a VALUES (1);`) executor := s.InternalExecutor().(isql.Executor) - table := desctestutils.TestingGetPublicTableDescriptor(s.DB(), keys.SystemSQLCodec, "t", "a") + table := desctestutils.TestingGetPublicTableDescriptor(s.DB(), codec, "t", "a") cache := NewTableStatisticsCache( 10, /* cacheSize */ s.ClusterSettings(), s.InternalDB().(descs.DB), ) - require.NoError(t, cache.Start(ctx, keys.SystemSQLCodec, s.RangeFeedFactory().(*rangefeed.Factory))) + require.NoError(t, cache.Start(ctx, codec, s.RangeFeedFactory().(*rangefeed.Factory))) refresher := MakeRefresher(s.AmbientCtx(), st, executor, cache, time.Microsecond /* asOfTime */) // curTime is used as the current time throughout the test to ensure that the @@ -558,11 +561,11 @@ func TestAutoStatsReadOnlyTables(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) - defer sqlDB.Close() - defer s.Stopper().Stop(ctx) + srv, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() + codec, st := s.Codec(), s.ClusterSettings() - st := cluster.MakeTestingClusterSettings() AutomaticStatisticsClusterMode.Override(ctx, &st.SV, false) AutomaticStatisticsOnSystemTables.Override(ctx, &st.SV, false) evalCtx := eval.NewTestingEvalContext(st) @@ -584,7 +587,7 @@ func TestAutoStatsReadOnlyTables(t *testing.T) { s.ClusterSettings(), s.InternalDB().(descs.DB), ) - require.NoError(t, cache.Start(ctx, keys.SystemSQLCodec, s.RangeFeedFactory().(*rangefeed.Factory))) + require.NoError(t, cache.Start(ctx, codec, s.RangeFeedFactory().(*rangefeed.Factory))) refresher := MakeRefresher(s.AmbientCtx(), st, executor, cache, time.Microsecond /* asOfTime */) AutomaticStatisticsClusterMode.Override(ctx, &st.SV, true) @@ -615,11 +618,11 @@ func TestAutoStatsOnStartupClusterSettingOff(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) - defer sqlDB.Close() - defer s.Stopper().Stop(ctx) + srv, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() + codec, st := s.Codec(), s.ClusterSettings() - st := cluster.MakeTestingClusterSettings() AutomaticStatisticsClusterMode.Override(ctx, &st.SV, false) evalCtx := eval.NewTestingEvalContext(st) defer evalCtx.Stop(ctx) @@ -639,7 +642,7 @@ func TestAutoStatsOnStartupClusterSettingOff(t *testing.T) { s.ClusterSettings(), s.InternalDB().(descs.DB), ) - require.NoError(t, cache.Start(ctx, keys.SystemSQLCodec, s.RangeFeedFactory().(*rangefeed.Factory))) + require.NoError(t, cache.Start(ctx, codec, s.RangeFeedFactory().(*rangefeed.Factory))) refresher := MakeRefresher(s.AmbientCtx(), st, executor, cache, time.Microsecond /* asOfTime */) // Refresher start should trigger stats collection on t.a. @@ -672,10 +675,11 @@ func TestNoRetryOnFailure(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - s := serverutils.StartServerOnly(t, base.TestServerArgs{}) - defer s.Stopper().Stop(ctx) + srv := serverutils.StartServerOnly(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() + codec, st := s.Codec(), s.ClusterSettings() - st := cluster.MakeTestingClusterSettings() evalCtx := eval.NewTestingEvalContext(st) defer evalCtx.Stop(ctx) @@ -685,7 +689,7 @@ func TestNoRetryOnFailure(t *testing.T) { s.ClusterSettings(), s.InternalDB().(descs.DB), ) - require.NoError(t, cache.Start(ctx, keys.SystemSQLCodec, s.RangeFeedFactory().(*rangefeed.Factory))) + require.NoError(t, cache.Start(ctx, codec, s.RangeFeedFactory().(*rangefeed.Factory))) r := MakeRefresher(s.AmbientCtx(), st, executor, cache, time.Microsecond /* asOfTime */) // Try to refresh stats on a table that doesn't exist. @@ -751,11 +755,11 @@ func TestDefaultColumns(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) - defer sqlDB.Close() - defer s.Stopper().Stop(ctx) + srv, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() + st := s.ClusterSettings() - st := cluster.MakeTestingClusterSettings() AutomaticStatisticsClusterMode.Override(ctx, &st.SV, false) evalCtx := eval.NewTestingEvalContext(st) defer evalCtx.Stop(ctx) @@ -788,10 +792,11 @@ func TestAnalyzeSystemTables(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) - defer s.Stopper().Stop(ctx) + srv, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() + codec, st := s.Codec(), s.ClusterSettings() - st := cluster.MakeTestingClusterSettings() AutomaticStatisticsClusterMode.Override(ctx, &st.SV, false) evalCtx := eval.NewTestingEvalContext(st) defer evalCtx.Stop(ctx) @@ -801,7 +806,7 @@ func TestAnalyzeSystemTables(t *testing.T) { s.ClusterSettings(), s.InternalDB().(descs.DB), ) - require.NoError(t, cache.Start(ctx, keys.SystemSQLCodec, s.RangeFeedFactory().(*rangefeed.Factory))) + require.NoError(t, cache.Start(ctx, codec, s.RangeFeedFactory().(*rangefeed.Factory))) var tableNames []string tableNames = make([]string, 0, 40) @@ -870,11 +875,11 @@ func compareStatsCountWithZero( ctx context.Context, cache *TableStatisticsCache, tableName string, - s serverutils.TestServerInterface, + s serverutils.ApplicationLayerInterface, expectZeroRows bool, ) error { desc := - desctestutils.TestingGetPublicTableDescriptor(s.DB(), keys.SystemSQLCodec, "system", tableName) + desctestutils.TestingGetPublicTableDescriptor(s.DB(), s.Codec(), "system", tableName) return testutils.SucceedsSoonError(func() error { stats, err := cache.GetTableStats(ctx, desc) if err != nil { diff --git a/pkg/sql/stats/create_stats_job_test.go b/pkg/sql/stats/create_stats_job_test.go index 0a217b5d8bd4..a10734cceb18 100644 --- a/pkg/sql/stats/create_stats_job_test.go +++ b/pkg/sql/stats/create_stats_job_test.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" @@ -69,7 +70,7 @@ func TestCreateStatsControlJob(t *testing.T) { ctx := context.Background() tc := testcluster.StartTestCluster(t, nodes, params) defer tc.Stopper().Stop(ctx) - sqlDB := sqlutils.MakeSQLRunner(tc.Conns[0]) + sqlDB := sqlutils.MakeSQLRunner(tc.ApplicationLayer(0).SQLConn(t, "")) sqlDB.Exec(t, `CREATE DATABASE d`) sqlDB.Exec(t, `CREATE TABLE d.t (x INT PRIMARY KEY)`) var tID descpb.ID @@ -192,19 +193,19 @@ func TestAtMostOneRunningCreateStats(t *testing.T) { var allowRequest chan struct{} - var serverArgs base.TestServerArgs filter, setTableID := createStatsRequestFilter(&allowRequest) - params := base.TestClusterArgs{ServerArgs: serverArgs} + var params base.TestClusterArgs params.ServerArgs.Knobs.JobsTestingKnobs = jobs.NewTestingKnobsWithShortIntervals() params.ServerArgs.Knobs.Store = &kvserver.StoreTestingKnobs{ TestingRequestFilter: filter, } + params.ServerArgs.DefaultTestTenant = base.TestIsForStuffThatShouldWorkWithSecondaryTenantsButDoesntYet(109379) ctx := context.Background() const nodes = 1 tc := testcluster.StartTestCluster(t, nodes, params) defer tc.Stopper().Stop(ctx) - conn := tc.Conns[0] + conn := tc.ApplicationLayer(0).SQLConn(t, "") sqlDB := sqlutils.MakeSQLRunner(conn) sqlDB.Exec(t, `CREATE DATABASE d`) @@ -237,7 +238,7 @@ func TestAtMostOneRunningCreateStats(t *testing.T) { // Attempt to start an automatic stats run. It should fail. autoStatsRunShouldFail() - // PAUSE JOB does not bloack until the job is paused but only requests it. + // PAUSE JOB does not block until the job is paused but only requests it. // Wait until the job is set to paused. var jobID jobspb.JobID sqlDB.QueryRow(t, `SELECT id FROM system.jobs ORDER BY created DESC LIMIT 1`).Scan(&jobID) @@ -298,7 +299,7 @@ func TestDeleteFailedJob(t *testing.T) { serverArgs := base.TestServerArgs{Knobs: base.TestingKnobs{JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals()}} tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ServerArgs: serverArgs}) defer tc.Stopper().Stop(ctx) - conn := tc.Conns[0] + conn := tc.ApplicationLayer(0).SQLConn(t, "") sqlDB := sqlutils.MakeSQLRunner(conn) sqlDB.Exec(t, `SET CLUSTER SETTING sql.stats.automatic_collection.enabled = false`) @@ -372,7 +373,8 @@ func TestCreateStatsProgress(t *testing.T) { const nodes = 1 tc := testcluster.StartTestCluster(t, nodes, params) defer tc.Stopper().Stop(ctx) - conn := tc.Conns[0] + s := tc.ApplicationLayer(0) + conn := s.SQLConn(t, "") sqlDB := sqlutils.MakeSQLRunner(conn) sqlDB.Exec(t, `SET CLUSTER SETTING sql.stats.automatic_collection.enabled = false`) @@ -450,7 +452,7 @@ func TestCreateStatsProgress(t *testing.T) { // Invalidate the stats cache so that we can be sure to get the latest stats. var tableID descpb.ID sqlDB.QueryRow(t, `SELECT id FROM system.namespace WHERE name = 't'`).Scan(&tableID) - tc.Servers[0].ExecutorConfig().(sql.ExecutorConfig).TableStatsCache.InvalidateTableStats( + s.ExecutorConfig().(sql.ExecutorConfig).TableStatsCache.InvalidateTableStats( ctx, tableID, ) @@ -511,7 +513,7 @@ func TestCreateStatsAsOfTime(t *testing.T) { ctx := context.Background() tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) defer tc.Stopper().Stop(ctx) - sqlDB := sqlutils.MakeSQLRunner(tc.Conns[0]) + sqlDB := sqlutils.MakeSQLRunner(tc.ApplicationLayer(0).SQLConn(t, "")) sqlDB.Exec(t, `CREATE DATABASE d`) sqlDB.Exec(t, `CREATE TABLE d.t (x INT PRIMARY KEY)`) @@ -536,14 +538,26 @@ func TestCreateStatsAsOfTime(t *testing.T) { // Create a blocking request filter for the actions related // to CREATE STATISTICS, i.e. Scanning a user table. See discussion // on jobutils.RunJob for where this might be useful. +// +// Note that it only supports system tenants as well as the secondary tenant +// with serverutils.TestTenantID() tenant ID. func createStatsRequestFilter( allowProgressIota *chan struct{}, ) (kvserverbase.ReplicaRequestFilter, func(descpb.ID)) { var tableToBlock atomic.Value tableToBlock.Store(descpb.InvalidID) + // We must create this request filter before we start the server, so we + // don't know whether we're running against the test tenant or not. Thus, we + // will always try the codec for the first test tenant ID, and if it doesn't + // work, we fallback to the system tenant codec. + possibleCodec := keys.MakeSQLCodec(serverutils.TestTenantID()) return func(ctx context.Context, ba *kvpb.BatchRequest) *kvpb.Error { if req, ok := ba.GetArg(kvpb.Scan); ok { - _, tableID, _ := encoding.DecodeUvarintAscending(req.(*kvpb.ScanRequest).Key) + key := req.(*kvpb.ScanRequest).Key + if strippedKey, err := possibleCodec.StripTenantPrefix(key); err == nil { + key = strippedKey + } + _, tableID, _ := encoding.DecodeUvarintAscending(key) // Ensure that the tableID is what we expect it to be. if tableID > 0 && descpb.ID(tableID) == tableToBlock.Load() { // Read from the channel twice to allow jobutils.RunJob to complete diff --git a/pkg/sql/stats/delete_stats_test.go b/pkg/sql/stats/delete_stats_test.go index cd6b83bebf96..554f99060add 100644 --- a/pkg/sql/stats/delete_stats_test.go +++ b/pkg/sql/stats/delete_stats_test.go @@ -20,7 +20,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" - "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" @@ -40,15 +39,16 @@ func TestDeleteOldStatsForColumns(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - s := serverutils.StartServerOnly(t, base.TestServerArgs{}) - defer s.Stopper().Stop(ctx) + srv := serverutils.StartServerOnly(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() db := s.InternalDB().(descs.DB) cache := NewTableStatisticsCache( 10, /* cacheSize */ s.ClusterSettings(), db, ) - require.NoError(t, cache.Start(ctx, keys.SystemSQLCodec, s.RangeFeedFactory().(*rangefeed.Factory))) + require.NoError(t, cache.Start(ctx, s.Codec(), s.RangeFeedFactory().(*rangefeed.Factory))) // The test data must be ordered by CreatedAt DESC so the calculated set of // expected deleted stats is correct. @@ -335,15 +335,16 @@ func TestDeleteOldStatsForOtherColumns(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - s := serverutils.StartServerOnly(t, base.TestServerArgs{}) - defer s.Stopper().Stop(ctx) + srv := serverutils.StartServerOnly(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() db := s.InternalDB().(isql.DB) cache := NewTableStatisticsCache( 10, /* cacheSize */ s.ClusterSettings(), s.InternalDB().(descs.DB), ) - require.NoError(t, cache.Start(ctx, keys.SystemSQLCodec, s.RangeFeedFactory().(*rangefeed.Factory))) + require.NoError(t, cache.Start(ctx, s.Codec(), s.RangeFeedFactory().(*rangefeed.Factory))) testData := []TableStatisticProto{ { TableID: descpb.ID(100), @@ -672,17 +673,33 @@ func TestStatsAreDeletedForDroppedTables(t *testing.T) { var params base.TestServerArgs params.ScanMaxIdleTime = time.Millisecond // speed up MVCC GC queue scans + params.DefaultTestTenant = base.TestIsForStuffThatShouldWorkWithSecondaryTenantsButDoesntYet(109380) s, sqlDB, _ := serverutils.StartServer(t, params) defer s.Stopper().Stop(context.Background()) runner := sqlutils.MakeSQLRunner(sqlDB) + // Poll for MVCC GC more frequently. + systemDB := sqlutils.MakeSQLRunner(s.SystemLayer().SQLConn(t, "")) + systemDB.Exec(t, "SET CLUSTER SETTING sql.gc_job.wait_for_gc.interval = '1s';") + // Disable auto stats so that it doesn't interfere. runner.Exec(t, "SET CLUSTER SETTING sql.stats.automatic_collection.enabled = false;") - // Poll for MVCC GC more frequently. - runner.Exec(t, "SET CLUSTER SETTING sql.gc_job.wait_for_gc.interval = '1s';") // Cached protected timestamp state delays MVCC GC, update it every second. runner.Exec(t, "SET CLUSTER SETTING kv.protectedts.poll_interval = '1s';") + if s.StartedDefaultTestTenant() { + systemDB.Exec(t, "ALTER TENANT ALL SET CLUSTER SETTING sql.zone_configs.allow_for_secondary_tenant.enabled = true") + // Block until we see that zone configs are enabled. + testutils.SucceedsSoon(t, func() error { + var enabled bool + runner.QueryRow(t, "SHOW CLUSTER SETTING sql.zone_configs.allow_for_secondary_tenant.enabled").Scan(&enabled) + if !enabled { + return errors.New("zone configs are not yet enabled") + } + return nil + }) + } + // This subtest verifies that the statistic for a single dropped table is // deleted promptly. t.Run("basic", func(t *testing.T) { diff --git a/pkg/sql/stats/stats_cache_test.go b/pkg/sql/stats/stats_cache_test.go index 41d85b3d6dba..024baf717ec3 100644 --- a/pkg/sql/stats/stats_cache_test.go +++ b/pkg/sql/stats/stats_cache_test.go @@ -21,7 +21,6 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" @@ -217,8 +216,9 @@ func TestCacheBasic(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - s := serverutils.StartServerOnly(t, base.TestServerArgs{}) - defer s.Stopper().Stop(ctx) + srv := serverutils.StartServerOnly(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() db := s.InternalDB().(descs.DB) expectedStats, err := initTestData(ctx, db.Executor()) if err != nil { @@ -237,7 +237,7 @@ func TestCacheBasic(t *testing.T) { // will result in the cache getting populated. When the stats cache size is // exceeded, entries should be evicted according to the LRU policy. sc := NewTableStatisticsCache(2 /* cacheSize */, s.ClusterSettings(), db) - require.NoError(t, sc.Start(ctx, keys.SystemSQLCodec, s.RangeFeedFactory().(*rangefeed.Factory))) + require.NoError(t, sc.Start(ctx, s.Codec(), s.RangeFeedFactory().(*rangefeed.Factory))) for _, tableID := range tableIDs { checkStatsForTable(ctx, t, sc, expectedStats[tableID], tableID) } @@ -323,8 +323,9 @@ func TestCacheUserDefinedTypes(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) ctx := context.Background() - s, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) - defer s.Stopper().Stop(ctx) + srv, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() sqlRunner := sqlutils.MakeSQLRunner(sqlDB) sqlRunner.Exec(t, `CREATE DATABASE t;`) @@ -338,8 +339,8 @@ func TestCacheUserDefinedTypes(t *testing.T) { // Make a stats cache. sc := NewTableStatisticsCache(1, s.ClusterSettings(), insqlDB) - require.NoError(t, sc.Start(ctx, keys.SystemSQLCodec, s.RangeFeedFactory().(*rangefeed.Factory))) - tbl := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "tt") + require.NoError(t, sc.Start(ctx, s.Codec(), s.RangeFeedFactory().(*rangefeed.Factory))) + tbl := desctestutils.TestingGetPublicTableDescriptor(kvDB, s.Codec(), "t", "tt") // Get stats for our table. We are ensuring here that the access to the stats // for tt properly hydrates the user defined type t before access. stats, err := sc.GetTableStats(ctx, tbl) @@ -373,8 +374,9 @@ func TestCacheWait(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - s := serverutils.StartServerOnly(t, base.TestServerArgs{}) - defer s.Stopper().Stop(ctx) + srv := serverutils.StartServerOnly(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() db := s.InternalDB().(descs.DB) expectedStats, err := initTestData(ctx, db.Executor()) @@ -390,7 +392,7 @@ func TestCacheWait(t *testing.T) { } sort.Sort(tableIDs) sc := NewTableStatisticsCache(len(tableIDs) /* cacheSize */, s.ClusterSettings(), db) - require.NoError(t, sc.Start(ctx, keys.SystemSQLCodec, s.RangeFeedFactory().(*rangefeed.Factory))) + require.NoError(t, sc.Start(ctx, s.Codec(), s.RangeFeedFactory().(*rangefeed.Factory))) for _, tableID := range tableIDs { checkStatsForTable(ctx, t, sc, expectedStats[tableID], tableID) } @@ -436,21 +438,21 @@ func TestCacheAutoRefresh(t *testing.T) { ctx := context.Background() tc := serverutils.StartCluster(t, 3 /* numNodes */, base.TestClusterArgs{}) defer tc.Stopper().Stop(ctx) - s := tc.Server(0) + s := tc.ApplicationLayer(0) sc := NewTableStatisticsCache( 10, /* cacheSize */ s.ClusterSettings(), s.InternalDB().(descs.DB), ) - require.NoError(t, sc.Start(ctx, keys.SystemSQLCodec, s.RangeFeedFactory().(*rangefeed.Factory))) + require.NoError(t, sc.Start(ctx, s.Codec(), s.RangeFeedFactory().(*rangefeed.Factory))) - sr0 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + sr0 := sqlutils.MakeSQLRunner(s.SQLConn(t, "")) sr0.Exec(t, "SET CLUSTER SETTING sql.stats.automatic_collection.enabled = false") sr0.Exec(t, "CREATE DATABASE test") sr0.Exec(t, "CREATE TABLE test.t (k INT PRIMARY KEY, v INT)") sr0.Exec(t, "INSERT INTO test.t VALUES (1, 1), (2, 2), (3, 3)") - tableDesc := desctestutils.TestingGetPublicTableDescriptor(tc.Server(0).DB(), keys.SystemSQLCodec, "test", "t") + tableDesc := desctestutils.TestingGetPublicTableDescriptor(s.DB(), s.Codec(), "test", "t") expectNStats := func(n int) error { stats, err := sc.GetTableStats(ctx, tableDesc) @@ -466,14 +468,14 @@ func TestCacheAutoRefresh(t *testing.T) { if err := expectNStats(0); err != nil { t.Fatal(err) } - sr1 := sqlutils.MakeSQLRunner(tc.ServerConn(1)) + sr1 := sqlutils.MakeSQLRunner(tc.ApplicationLayer(1).SQLConn(t, "")) sr1.Exec(t, "CREATE STATISTICS k ON k FROM test.t") testutils.SucceedsSoon(t, func() error { return expectNStats(1) }) - sr2 := sqlutils.MakeSQLRunner(tc.ServerConn(2)) + sr2 := sqlutils.MakeSQLRunner(tc.ApplicationLayer(2).SQLConn(t, "")) sr2.Exec(t, "CREATE STATISTICS v ON v FROM test.t") testutils.SucceedsSoon(t, func() error { From 52a506c6e8811111eca093c49fd687d1f63cf0d6 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Wed, 23 Aug 2023 19:06:09 -0700 Subject: [PATCH 3/8] sql/catalog: adjust many tests to work with test tenant Three tests are skipped explicitly plus on the order of 10 tests that use `newLeaseTest` helper too. Release note: None --- pkg/sql/catalog/descs/BUILD.bazel | 1 - pkg/sql/catalog/descs/collection_test.go | 140 ++++++------ pkg/sql/catalog/lease/BUILD.bazel | 1 - pkg/sql/catalog/lease/kv_writer_test.go | 15 +- pkg/sql/catalog/lease/lease_internal_test.go | 129 ++++++----- pkg/sql/catalog/lease/lease_test.go | 200 ++++++++++-------- pkg/sql/catalog/redact/BUILD.bazel | 1 - pkg/sql/catalog/redact/redact_test.go | 12 +- pkg/sql/catalog/resolver/resolver_test.go | 10 +- .../schematelemetry/schema_telemetry_test.go | 12 +- pkg/sql/catalog/tabledesc/BUILD.bazel | 1 + pkg/sql/catalog/tabledesc/index_test.go | 34 +-- pkg/sql/catalog/tabledesc/structured_test.go | 29 ++- pkg/sql/exec_util.go | 4 +- pkg/testutils/lint/lint_test.go | 1 + 15 files changed, 337 insertions(+), 253 deletions(-) diff --git a/pkg/sql/catalog/descs/BUILD.bazel b/pkg/sql/catalog/descs/BUILD.bazel index 0a4eae06bf6c..6b0a3cb3075a 100644 --- a/pkg/sql/catalog/descs/BUILD.bazel +++ b/pkg/sql/catalog/descs/BUILD.bazel @@ -91,7 +91,6 @@ go_test( embed = [":descs"], deps = [ "//pkg/base", - "//pkg/keys", "//pkg/kv", "//pkg/kv/kvserver", "//pkg/security/securityassets", diff --git a/pkg/sql/catalog/descs/collection_test.go b/pkg/sql/catalog/descs/collection_test.go index c842db74e426..7a98adaba924 100644 --- a/pkg/sql/catalog/descs/collection_test.go +++ b/pkg/sql/catalog/descs/collection_test.go @@ -21,7 +21,6 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/security/username" @@ -47,7 +46,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" - "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/mon" @@ -60,23 +58,21 @@ func TestCollectionWriteDescToBatch(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) - defer tc.Stopper().Stop(ctx) + srv, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() - s0 := tc.Server(0) - - tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + tdb := sqlutils.MakeSQLRunner(sqlDB) tdb.Exec(t, `CREATE DATABASE db`) tdb.Exec(t, `USE db`) tdb.Exec(t, `CREATE SCHEMA schema`) tdb.Exec(t, `CREATE TABLE db.schema.table()`) - db := s0.DB() - descriptors := s0.ExecutorConfig().(sql.ExecutorConfig).CollectionFactory.NewCollection(ctx) + descriptors := s.ExecutorConfig().(sql.ExecutorConfig).CollectionFactory.NewCollection(ctx) // Note this transaction abuses the mechanisms normally required for updating // tables and is just for testing what this test intends to exercise. - require.NoError(t, db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + require.NoError(t, kvDB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { defer descriptors.ReleaseAll(ctx) tn := tree.MakeTableNameWithSchema("db", "schema", "table") _, mut, err := descs.PrefixAndMutableTable(ctx, descriptors.MutableByName(txn), &tn) @@ -152,17 +148,15 @@ func TestTxnClearsCollectionOnRetry(t *testing.T) { const txnName = "descriptor update" var serverArgs base.TestServerArgs - params := base.TestClusterArgs{ServerArgs: serverArgs} filterFunc, _ := testutils.TestingRequestFilterRetryTxnWithPrefix(t, txnName, 1) - params.ServerArgs.Knobs.Store = &kvserver.StoreTestingKnobs{ + serverArgs.Knobs.Store = &kvserver.StoreTestingKnobs{ TestingRequestFilter: filterFunc, } - tc := testcluster.StartTestCluster(t, 1, params) - defer tc.Stopper().Stop(ctx) - - s := tc.Server(0) + srv, sqlDB, _ := serverutils.StartServer(t, serverArgs) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() - tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + tdb := sqlutils.MakeSQLRunner(sqlDB) tdb.Exec(t, `CREATE DATABASE db`) tdb.Exec(t, `USE db`) tdb.Exec(t, `CREATE SCHEMA schema`) @@ -197,17 +191,17 @@ func TestAddUncommittedDescriptorAndMutableResolution(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) - defer tc.Stopper().Stop(ctx) + srv, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() - s0 := tc.Server(0) - tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + tdb := sqlutils.MakeSQLRunner(sqlDB) tdb.Exec(t, "CREATE DATABASE db") tdb.Exec(t, "USE db") tdb.Exec(t, "CREATE SCHEMA db.sc") tdb.Exec(t, "CREATE TABLE db.sc.tab (i INT PRIMARY KEY)") tdb.Exec(t, "CREATE TYPE db.sc.typ AS ENUM ('foo')") - execCfg := s0.ExecutorConfig().(sql.ExecutorConfig) + execCfg := s.ExecutorConfig().(sql.ExecutorConfig) t.Run("database descriptors", func(t *testing.T) { require.NoError(t, sql.DescsTxn(ctx, &execCfg, func( ctx context.Context, txn isql.Txn, descriptors *descs.Collection, @@ -342,12 +336,11 @@ func TestSyntheticDescriptorResolution(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) - defer tc.Stopper().Stop(ctx) + srv, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() - s0 := tc.Server(0) - - tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + tdb := sqlutils.MakeSQLRunner(sqlDB) tdb.Exec(t, `CREATE DATABASE db`) tdb.Exec(t, `USE db`) tdb.Exec(t, `CREATE TABLE tbl(foo INT)`) @@ -355,7 +348,7 @@ func TestSyntheticDescriptorResolution(t *testing.T) { var tableID descpb.ID row.Scan(&tableID) - execCfg := s0.ExecutorConfig().(sql.ExecutorConfig) + execCfg := s.ExecutorConfig().(sql.ExecutorConfig) require.NoError(t, sql.DescsTxn(ctx, &execCfg, func( ctx context.Context, txn isql.Txn, descriptors *descs.Collection, ) error { @@ -401,12 +394,11 @@ func TestDistSQLTypeResolver_GetTypeDescriptor_FromTable(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) - defer tc.Stopper().Stop(ctx) + srv, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() - s := tc.Server(0) - - tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + tdb := sqlutils.MakeSQLRunner(sqlDB) tdb.Exec(t, `CREATE TABLE t(a INT PRIMARY KEY, b STRING)`) var id descpb.ID tdb.QueryRow(t, "SELECT $1::regclass::int", "t").Scan(&id) @@ -442,13 +434,11 @@ func TestMaybeFixSchemaPrivilegesIntegration(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - s, db, _ := serverutils.StartServer(t, base.TestServerArgs{}) - defer s.Stopper().Stop(ctx) + srv, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() - conn, err := db.Conn(ctx) - require.NoError(t, err) - - _, err = conn.ExecContext(ctx, ` + _, err := sqlDB.ExecContext(ctx, ` CREATE DATABASE test; CREATE SCHEMA test.schema; CREATE USER testuser; @@ -489,7 +479,7 @@ CREATE TABLE test.schema.t(x INT); // Make sure using the schema is fine and we don't encounter a // privilege validation error. - _, err = db.Query("GRANT USAGE ON SCHEMA test.schema TO testuser;") + _, err = sqlDB.Query("GRANT USAGE ON SCHEMA test.schema TO testuser;") require.NoError(t, err) } @@ -502,8 +492,9 @@ func TestCollectionPreservesPostDeserializationChanges(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) - defer s.Stopper().Stop(ctx) + srv, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() tdb := sqlutils.MakeSQLRunner(sqlDB) tdb.Exec(t, "CREATE DATABASE db") @@ -592,12 +583,13 @@ func TestCollectionProperlyUsesMemoryMonitoring(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) - defer tc.Stopper().Stop(ctx) - txn := tc.Server(0).DB().NewTxn(ctx, "test txn") + srv, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() + txn := kvDB.NewTxn(ctx, "test txn") // Create a lot of descriptors. - tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + tdb := sqlutils.MakeSQLRunner(sqlDB) numTblsToInsert := 100 for i := 0; i < numTblsToInsert; i++ { tdb.Exec(t, fmt.Sprintf("CREATE TABLE table_%v()", i)) @@ -611,7 +603,7 @@ func TestCollectionProperlyUsesMemoryMonitoring(t *testing.T) { monitor.Start(ctx, nil, mon.NewStandaloneBudget(math.MaxInt64)) // Create a `Collection` with monitor hooked up. - col := tc.Server(0).ExecutorConfig().(sql.ExecutorConfig).CollectionFactory.NewCollection( + col := s.ExecutorConfig().(sql.ExecutorConfig).CollectionFactory.NewCollection( ctx, descs.WithMonitor(monitor), ) require.Equal(t, int64(0), monitor.AllocBytes()) @@ -631,7 +623,7 @@ func TestCollectionProperlyUsesMemoryMonitoring(t *testing.T) { require.Equal(t, int64(0), monitor.AllocBytes()) // Repeat the process again and assert this time memory allocation will err out. - col = tc.Server(0).ExecutorConfig().(sql.ExecutorConfig).CollectionFactory.NewCollection( + col = s.ExecutorConfig().(sql.ExecutorConfig).CollectionFactory.NewCollection( ctx, descs.WithMonitor(monitor), ) _, err2 := col.GetAllDescriptors(ctx, txn) @@ -650,17 +642,17 @@ func TestDescriptorCache(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) - defer tc.Stopper().Stop(ctx) + srv, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() - tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + tdb := sqlutils.MakeSQLRunner(sqlDB) tdb.Exec(t, `CREATE DATABASE db`) tdb.Exec(t, `USE db`) tdb.Exec(t, `CREATE SCHEMA schema`) tdb.Exec(t, `CREATE TABLE db.schema.table()`) - s0 := tc.Server(0) - execCfg := s0.ExecutorConfig().(sql.ExecutorConfig) + execCfg := s.ExecutorConfig().(sql.ExecutorConfig) t.Run("all descriptors", func(t *testing.T) { require.NoError(t, sql.DescsTxn(ctx, &execCfg, func( ctx context.Context, txn isql.Txn, descriptors *descs.Collection, @@ -780,18 +772,18 @@ func TestGetAllDescriptorsInDatabase(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) - defer tc.Stopper().Stop(ctx) + srv, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() - tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + tdb := sqlutils.MakeSQLRunner(sqlDB) tdb.Exec(t, `CREATE DATABASE db`) tdb.Exec(t, `USE db`) tdb.Exec(t, `CREATE SCHEMA schema`) tdb.Exec(t, `CREATE TABLE db.public.table()`) tdb.Exec(t, `CREATE TABLE db.schema.table()`) - s0 := tc.Server(0) - tm := s0.InternalDB().(descs.DB) + tm := s.InternalDB().(descs.DB) run := func( ctx context.Context, txn descs.Txn, @@ -843,7 +835,7 @@ parent schema name id kind version dropped public `, formatCatalog(allDescs.OrderedDescriptors())) return nil } - sd := sql.NewInternalSessionData(ctx, s0.ClusterSettings(), "TestGetAllDescriptorsInDatabase") + sd := sql.NewInternalSessionData(ctx, s.ClusterSettings(), "TestGetAllDescriptorsInDatabase") sd.Database = "db" require.NoError(t, tm.DescsTxn(ctx, run, isql.WithSessionData(sd))) } @@ -896,17 +888,17 @@ func TestCollectionTimeTravelLookingTooFarBack(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) - defer tc.Stopper().Stop(ctx) + srv, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() - tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + tdb := sqlutils.MakeSQLRunner(sqlDB) tdb.Exec(t, `CREATE DATABASE db`) tdb.Exec(t, `USE db`) tdb.Exec(t, `CREATE SCHEMA schema`) tdb.Exec(t, `CREATE TABLE db.schema.table()`) - s0 := tc.Server(0) - execCfg := s0.ExecutorConfig().(sql.ExecutorConfig) + execCfg := s.ExecutorConfig().(sql.ExecutorConfig) goFarBackInTime := func(fn func(ctx context.Context, txn *kv.Txn, col *descs.Collection) error) error { return sql.DescsTxn(ctx, &execCfg, func(ctx context.Context, txn isql.Txn, col *descs.Collection) error { @@ -947,12 +939,11 @@ func TestHydrateCatalog(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) - defer tc.Stopper().Stop(ctx) - - s := tc.Server(0) + srv, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() - tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + tdb := sqlutils.MakeSQLRunner(sqlDB) tdb.Exec(t, `CREATE DATABASE db`) tdb.Exec(t, `USE db`) tdb.Exec(t, `CREATE SCHEMA schema`) @@ -1030,7 +1021,7 @@ func TestHydrateCatalog(t *testing.T) { } mc := nstree.MutableCatalog{Catalog: cat} require.NoError(t, descs.HydrateCatalog(ctx, mc)) - tbl := desctestutils.TestingGetTableDescriptor(txn.KV().DB(), keys.SystemSQLCodec, "db", "schema", "table") + tbl := desctestutils.TestingGetTableDescriptor(txn.KV().DB(), s.Codec(), "db", "schema", "table") tblDesc := cat.LookupDescriptor(tbl.GetID()).(catalog.TableDescriptor) expectedEnum := types.UserDefinedTypeMetadata{ Name: &types.UserDefinedTypeName{ @@ -1075,12 +1066,11 @@ func TestSyntheticDescriptors(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) - defer tc.Stopper().Stop(ctx) + srv, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() - s := tc.Server(0) - - tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + tdb := sqlutils.MakeSQLRunner(sqlDB) tdb.Exec(t, "CREATE TABLE foo (i INT PRIMARY KEY)") tdb.Exec(t, "CREATE SCHEMA sc") tdb.Exec(t, `CREATE DATABASE "otherDB"`) diff --git a/pkg/sql/catalog/lease/BUILD.bazel b/pkg/sql/catalog/lease/BUILD.bazel index 2ae12b387ba1..f5275bbed102 100644 --- a/pkg/sql/catalog/lease/BUILD.bazel +++ b/pkg/sql/catalog/lease/BUILD.bazel @@ -120,7 +120,6 @@ go_test( "//pkg/util/log", "//pkg/util/randutil", "//pkg/util/retry", - "//pkg/util/stop", "//pkg/util/syncutil", "//pkg/util/timeutil", "//pkg/util/uuid", diff --git a/pkg/sql/catalog/lease/kv_writer_test.go b/pkg/sql/catalog/lease/kv_writer_test.go index 4af18c2be438..5926d8fbae80 100644 --- a/pkg/sql/catalog/lease/kv_writer_test.go +++ b/pkg/sql/catalog/lease/kv_writer_test.go @@ -43,7 +43,7 @@ import ( // lease table from 1 to 2. It is injected from the lease_test package so that // it can use sql primitives. var MoveTablePrimaryIndexIDto2 func( - context.Context, *testing.T, serverutils.TestServerInterface, descpb.ID, + context.Context, *testing.T, serverutils.ApplicationLayerInterface, descpb.ID, ) // TestKVWriterMatchesIEWriter is a rather involved test to exercise the @@ -57,13 +57,16 @@ func TestKVWriterMatchesIEWriter(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - s, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) - defer s.Stopper().Stop(ctx) - tdb := sqlutils.MakeSQLRunner(sqlDB) + srv, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() // Otherwise, we wouldn't get complete SSTs in our export under stress. - tdb.Exec(t, "SET CLUSTER SETTING admission.elastic_cpu.enabled = false") + sqlutils.MakeSQLRunner(srv.SystemLayer().SQLConn(t, "")).Exec( + t, "SET CLUSTER SETTING admission.elastic_cpu.enabled = false", + ) + tdb := sqlutils.MakeSQLRunner(sqlDB) schema := systemschema.LeaseTableSchema makeTable := func(name string) (id descpb.ID) { tdb.Exec(t, strings.Replace(schema, "system.lease", name, 1)) @@ -76,7 +79,7 @@ func TestKVWriterMatchesIEWriter(t *testing.T) { lease2ID := makeTable("lease2") ie := s.InternalExecutor().(isql.Executor) - codec := s.LeaseManager().(*Manager).Codec() + codec := s.Codec() settingsWatcher := s.SettingsWatcher().(*settingswatcher.SettingsWatcher) w := teeWriter{ a: newInternalExecutorWriter(ie, "defaultdb.public.lease1"), diff --git a/pkg/sql/catalog/lease/lease_internal_test.go b/pkg/sql/catalog/lease/lease_internal_test.go index 3fb8c297e4ae..9a2daa7bc748 100644 --- a/pkg/sql/catalog/lease/lease_internal_test.go +++ b/pkg/sql/catalog/lease/lease_internal_test.go @@ -22,7 +22,6 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" @@ -37,7 +36,6 @@ import ( "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/stop" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" "github.com/cockroachdb/logtags" @@ -150,8 +148,9 @@ func TestPurgeOldVersions(t *testing.T) { }, }, } - s, db, kvDB := serverutils.StartServer(t, serverParams) - defer s.Stopper().Stop(context.Background()) + srv, db, kvDB := serverutils.StartServer(t, serverParams) + defer srv.Stopper().Stop(context.Background()) + s := srv.ApplicationLayer() leaseManager := s.LeaseManager().(*Manager) // Block gossip. @@ -168,7 +167,7 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR); t.Fatal(err) } - tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") + tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, s.Codec(), "t", "test") var tables []catalog.TableDescriptor var expiration hlc.Timestamp @@ -253,6 +252,7 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR); // modification time should still be retained. func TestPurgeOldVersionsRetainsDescriptorWithFutureModificationTime(t *testing.T) { defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) // We're going to block gossip so it doesn't come randomly and clear up the // leases we're artificially setting up. gossipSem := make(chan struct{}, 1) @@ -268,8 +268,9 @@ func TestPurgeOldVersionsRetainsDescriptorWithFutureModificationTime(t *testing. }, } ctx := context.Background() - s, db, kvDB := serverutils.StartServer(t, serverParams) - defer s.Stopper().Stop(ctx) + srv, db, kvDB := serverutils.StartServer(t, serverParams) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() leaseManager := s.LeaseManager().(*Manager) // Block gossip. @@ -286,7 +287,7 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR); t.Fatal(err) } - tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") + tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, s.Codec(), "t", "test") futureTime := s.Clock().Now().Add(500*time.Millisecond.Nanoseconds(), 0).WithSynthetic(true) getLatestDesc := func() catalog.TableDescriptor { @@ -317,11 +318,12 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR); if err := txn2.SetFixedTimestamp(ctx, futureTime.Prev()); err != nil { return err } - if _, err := txn2.Get(ctx, "key"); err != nil { + key := append(s.Codec().TenantPrefix(), "key"...) + if _, err := txn2.Get(ctx, key); err != nil { return errors.Wrap(err, "read from other txn in future") } - return txn.Put(ctx, "key", "value") + return txn.Put(ctx, key, "value") } if _, err := leaseManager.Publish(ctx, tableDesc.GetID(), update, logEvent); err != nil { t.Fatal(err) @@ -358,9 +360,11 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR); // do not cause issues. func TestNameCacheDBConflictingTableNames(t *testing.T) { defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) - s, db, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) - defer s.Stopper().Stop(context.Background()) + srv, db, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(context.Background()) + s := srv.ApplicationLayer() leaseManager := s.LeaseManager().(*Manager) if _, err := db.Exec(`SET experimental_enable_temp_tables = true`); err != nil { @@ -391,7 +395,7 @@ CREATE TEMP TABLE t2 (temp int); } for _, tableName := range []string{"t", "t2"} { - tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "defaultdb", tableName) + tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, s.Codec(), "defaultdb", tableName) lease := leaseManager.names.get( context.Background(), tableDesc.GetParentID(), @@ -408,8 +412,11 @@ CREATE TEMP TABLE t2 (temp int); // Test that changing a descriptor's name updates the name cache. func TestNameCacheIsUpdated(t *testing.T) { defer leaktest.AfterTest(t)() - s, db, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) - defer s.Stopper().Stop(context.Background()) + defer log.Scope(t).Close(t) + + srv, db, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(context.Background()) + s := srv.ApplicationLayer() leaseManager := s.LeaseManager().(*Manager) if _, err := db.Exec(` @@ -425,7 +432,7 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR); t.Fatal(err) } - tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") + tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, s.Codec(), "t", "test") // Rename. if _, err := db.Exec("ALTER TABLE t.test RENAME TO t.test2;"); err != nil { @@ -462,8 +469,11 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR); // Tests that a name cache entry with by an expired lease is not returned. func TestNameCacheEntryDoesntReturnExpiredLease(t *testing.T) { defer leaktest.AfterTest(t)() - s, db, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) - defer s.Stopper().Stop(context.Background()) + defer log.Scope(t).Close(t) + + srv, db, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(context.Background()) + s := srv.ApplicationLayer() leaseManager := s.LeaseManager().(*Manager) const tableName = "test" @@ -480,7 +490,7 @@ CREATE TABLE t.%s (k CHAR PRIMARY KEY, v CHAR); t.Fatal(err) } - tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "t", tableName) + tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, s.Codec(), "t", tableName) // Check the assumptions this tests makes: that there is a cache entry // (with a valid lease). @@ -514,6 +524,8 @@ CREATE TABLE t.%s (k CHAR PRIMARY KEY, v CHAR); // the lease expiration time is monotonically increasing. func TestNameCacheContainsLatestLease(t *testing.T) { defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + removalTracker := NewLeaseRemovalTracker() testingKnobs := base.TestingKnobs{ SQLLeaseManager: &ManagerTestingKnobs{ @@ -522,8 +534,9 @@ func TestNameCacheContainsLatestLease(t *testing.T) { }, }, } - s, db, kvDB := serverutils.StartServer(t, base.TestServerArgs{Knobs: testingKnobs}) - defer s.Stopper().Stop(context.Background()) + srv, db, kvDB := serverutils.StartServer(t, base.TestServerArgs{Knobs: testingKnobs}) + defer srv.Stopper().Stop(context.Background()) + s := srv.ApplicationLayer() leaseManager := s.LeaseManager().(*Manager) const tableName = "test" @@ -535,7 +548,7 @@ CREATE TABLE t.%s (k CHAR PRIMARY KEY, v CHAR); t.Fatal(err) } - tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "t", tableName) + tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, s.Codec(), "t", tableName) // Populate the name cache. if _, err := db.Exec("SELECT * FROM t.test;"); err != nil { @@ -586,8 +599,11 @@ CREATE TABLE t.%s (k CHAR PRIMARY KEY, v CHAR); // Test that table names are treated as case sensitive by the name cache. func TestTableNameCaseSensitive(t *testing.T) { defer leaktest.AfterTest(t)() - s, db, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) - defer s.Stopper().Stop(context.Background()) + defer log.Scope(t).Close(t) + + srv, db, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(context.Background()) + s := srv.ApplicationLayer() leaseManager := s.LeaseManager().(*Manager) if _, err := db.Exec(` @@ -602,7 +618,7 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR); t.Fatal(err) } - tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") + tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, s.Codec(), "t", "test") // Check that we cannot get the table by a different name. if leaseManager.names.get( @@ -623,6 +639,8 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR); // AcquireByName()d at the same time. func TestReleaseAcquireByNameDeadlock(t *testing.T) { defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + removalTracker := NewLeaseRemovalTracker() testingKnobs := base.TestingKnobs{ SQLLeaseManager: &ManagerTestingKnobs{ @@ -632,9 +650,10 @@ func TestReleaseAcquireByNameDeadlock(t *testing.T) { }, }, } - s, sqlDB, kvDB := serverutils.StartServer( + srv, sqlDB, kvDB := serverutils.StartServer( t, base.TestServerArgs{Knobs: testingKnobs}) - defer s.Stopper().Stop(context.Background()) + defer srv.Stopper().Stop(context.Background()) + s := srv.ApplicationLayer() leaseManager := s.LeaseManager().(*Manager) if _, err := sqlDB.Exec(` @@ -644,7 +663,7 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR); t.Fatal(err) } - tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") + tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, s.Codec(), "t", "test") // Populate the name cache. ctx := context.Background() @@ -739,8 +758,11 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR); // Manager.acquireFreshestFromStore() in parallel to test for races. func TestAcquireFreshestFromStoreRaces(t *testing.T) { defer leaktest.AfterTest(t)() - s, db, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) - defer s.Stopper().Stop(context.Background()) + defer log.Scope(t).Close(t) + + srv, db, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(context.Background()) + s := srv.ApplicationLayer() leaseManager := s.LeaseManager().(*Manager) if _, err := db.Exec(` @@ -750,7 +772,7 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR); t.Fatal(err) } - tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") + tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, s.Codec(), "t", "test") var wg sync.WaitGroup numRoutines := 10 @@ -778,6 +800,8 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR); // is still valid. They are able to reacquire a lease if needed. func TestParallelLeaseAcquireWithImmediateRelease(t *testing.T) { defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + testingKnobs := base.TestingKnobs{ SQLLeaseManager: &ManagerTestingKnobs{ LeaseStoreTestingKnobs: StorageTestingKnobs{ @@ -788,9 +812,10 @@ func TestParallelLeaseAcquireWithImmediateRelease(t *testing.T) { }, }, } - s, sqlDB, kvDB := serverutils.StartServer( + srv, sqlDB, kvDB := serverutils.StartServer( t, base.TestServerArgs{Knobs: testingKnobs}) - defer s.Stopper().Stop(context.Background()) + defer srv.Stopper().Stop(context.Background()) + s := srv.ApplicationLayer() leaseManager := s.LeaseManager().(*Manager) if _, err := sqlDB.Exec(` @@ -800,7 +825,7 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR); t.Fatal(err) } - tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") + tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, s.Codec(), "t", "test") var wg sync.WaitGroup numRoutines := 10 @@ -841,6 +866,7 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR); // one. func TestLeaseAcquireAndReleaseConcurrently(t *testing.T) { defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) // Result is a struct for moving results to the main result routine. type Result struct { @@ -968,9 +994,9 @@ func TestLeaseAcquireAndReleaseConcurrently(t *testing.T) { // leases are checked for having a different expiration. LeaseJitterFraction.Override(ctx, &serverArgs.SV, 0) - s, sqlDB, _ := serverutils.StartServer( - t, serverArgs) - defer s.Stopper().Stop(context.Background()) + srv, sqlDB, _ := serverutils.StartServer(t, serverArgs) + defer srv.Stopper().Stop(context.Background()) + s := srv.ApplicationLayer() tdb := sqlutils.MakeSQLRunner(sqlDB) tdb.Exec(t, "CREATE TABLE t (i INT PRIMARY KEY)") { @@ -1056,6 +1082,8 @@ func TestLeaseAcquireAndReleaseConcurrently(t *testing.T) { // table descriptor from store through an ExportRequest. func TestReadOlderVersionForTimestamp(t *testing.T) { defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + serverParams := base.TestServerArgs{ Knobs: base.TestingKnobs{ SQLLeaseManager: &ManagerTestingKnobs{ @@ -1065,11 +1093,10 @@ func TestReadOlderVersionForTimestamp(t *testing.T) { }, }, } - var stopper *stop.Stopper - s, sqlDB, _ := serverutils.StartServer(t, serverParams) - stopper = s.Stopper() ctx := context.Background() - defer stopper.Stop(ctx) + srv, sqlDB, _ := serverutils.StartServer(t, serverParams) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() tdb := sqlutils.MakeSQLRunner(sqlDB) // Prevent non-explicit Acquire to leases for testing purposes. @@ -1251,10 +1278,13 @@ func TestReadOlderVersionForTimestamp(t *testing.T) { // correct. func TestDescriptorByteSizeOrder(t *testing.T) { defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + ctx := context.Background() params := base.TestServerArgs{} - s, db, _ := serverutils.StartServer(t, params) - defer s.Stopper().Stop(ctx) + srv, db, _ := serverutils.StartServer(t, params) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() tdb := sqlutils.MakeSQLRunner(db) manager := s.LeaseManager().(*Manager) @@ -1390,10 +1420,12 @@ func TestDescriptorByteSizeOrder(t *testing.T) { // from the binary encoded in the descriptor table. func TestLeasedDescriptorByteSizeBaseline(t *testing.T) { defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + ctx := context.Background() - params := base.TestServerArgs{} - s, db, _ := serverutils.StartServer(t, params) - defer s.Stopper().Stop(ctx) + srv, db, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() tdb := sqlutils.MakeSQLRunner(db) manager := s.LeaseManager().(*Manager) @@ -1500,7 +1532,7 @@ func TestGetDescriptorsFromStoreForIntervalCPULimiterPagination(t *testing.T) { ctx := context.Background() var numRequests int - s, db, kvDB := serverutils.StartServer(t, base.TestServerArgs{ + srv, db, kvDB := serverutils.StartServer(t, base.TestServerArgs{ Knobs: base.TestingKnobs{Store: &kvserver.StoreTestingKnobs{ TestingRequestFilter: func(ctx context.Context, request *kvpb.BatchRequest) *kvpb.Error { for _, ru := range request.Requests { @@ -1519,7 +1551,8 @@ func TestGetDescriptorsFromStoreForIntervalCPULimiterPagination(t *testing.T) { }, }}, }) - defer s.Stopper().Stop(ctx) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() sqlDB := sqlutils.MakeSQLRunner(db) beforeCreate := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} diff --git a/pkg/sql/catalog/lease/lease_test.go b/pkg/sql/catalog/lease/lease_test.go index c8d856d5625e..6e2625c3977f 100644 --- a/pkg/sql/catalog/lease/lease_test.go +++ b/pkg/sql/catalog/lease/lease_test.go @@ -59,7 +59,6 @@ import ( "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/stop" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" @@ -81,7 +80,7 @@ type leaseTest struct { func init() { lease.MoveTablePrimaryIndexIDto2 = func( - ctx context.Context, t *testing.T, s serverutils.TestServerInterface, id descpb.ID, + ctx context.Context, t *testing.T, s serverutils.ApplicationLayerInterface, id descpb.ID, ) { require.NoError(t, sql.TestingDescsTxn(ctx, s, func(ctx context.Context, txn isql.Txn, col *descs.Collection) error { t, err := col.MutableByID(txn.KV()).Table(ctx, id) @@ -97,6 +96,7 @@ func init() { } func newLeaseTest(tb testing.TB, params base.TestServerArgs) *leaseTest { + params.DefaultTestTenant = base.TODOTestTenantDisabled s, db, kvDB := serverutils.StartServer(tb, params) leaseTest := &leaseTest{ TB: tb, @@ -576,6 +576,7 @@ func TestLeaseManagerDrain(testingT *testing.T) { // Test that we fail to lease a table that was marked for deletion. func TestCantLeaseDeletedTable(testingT *testing.T) { defer leaktest.AfterTest(testingT)() + defer log.Scope(testingT).Close(testingT) var mu syncutil.Mutex clearSchemaChangers := false @@ -614,7 +615,7 @@ CREATE TABLE test.t(a INT PRIMARY KEY); t.Fatal(err) } - tableDesc := desctestutils.TestingGetPublicTableDescriptor(t.kvDB, keys.SystemSQLCodec, "test", "t") + tableDesc := desctestutils.TestingGetPublicTableDescriptor(t.kvDB, t.server.Codec(), "test", "t") // Block schema changers so that the table we're about to DROP is not actually // dropped; it will be left in a "deleted" state. @@ -638,7 +639,7 @@ CREATE TABLE test.t(a INT PRIMARY KEY); } func acquire( - ctx context.Context, s serverutils.TestServerInterface, descID descpb.ID, + ctx context.Context, s serverutils.ApplicationLayerInterface, descID descpb.ID, ) (lease.LeasedDescriptor, error) { return s.LeaseManager().(*lease.Manager).Acquire(ctx, s.Clock().Now(), descID) } @@ -685,8 +686,9 @@ func TestLeasesOnDeletedTableAreReleasedImmediately(t *testing.T) { // Disable GC job. GCJob: &sql.GCJobTestingKnobs{RunBeforeResume: func(_ jobspb.JobID) error { select {} }}, } - s, db, kvDB := serverutils.StartServer(t, params) - defer s.Stopper().Stop(context.Background()) + srv, db, kvDB := serverutils.StartServer(t, params) + defer srv.Stopper().Stop(context.Background()) + s := srv.ApplicationLayer() _, err := db.Exec(`SET CLUSTER SETTING sql.defaults.use_declarative_schema_changer = 'off'`) if err != nil { @@ -707,7 +709,7 @@ CREATE TABLE test.t(a INT PRIMARY KEY); t.Fatal(err) } - tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "test", "t") + tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, s.Codec(), "test", "t") ctx := context.Background() lease1, err := acquire(ctx, s, tableDesc.GetID()) @@ -787,8 +789,9 @@ func TestSubqueryLeases(t *testing.T) { }, }, } - s, sqlDB, kvDB := serverutils.StartServer(t, params) - defer s.Stopper().Stop(context.Background()) + srv, sqlDB, kvDB := serverutils.StartServer(t, params) + defer srv.Stopper().Stop(context.Background()) + s := srv.ApplicationLayer() if _, err := sqlDB.Exec(` CREATE DATABASE t; @@ -801,7 +804,7 @@ CREATE TABLE t.foo (v INT); t.Fatalf("CREATE TABLE has acquired a lease: got %d, expected 0", atomic.LoadInt32(&fooAcquiredCount)) } - tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "foo") + tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, s.Codec(), "t", "foo") atomic.StoreInt64(&tableID, int64(tableDesc.GetID())) if _, err := sqlDB.Exec(` @@ -889,6 +892,7 @@ CREATE TABLE t.foo (v INT); // a query are properly released before the query is retried. func TestDescriptorRefreshOnRetry(t *testing.T) { defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) fooAcquiredCount := int32(0) fooReleaseCount := int32(0) @@ -914,8 +918,9 @@ func TestDescriptorRefreshOnRetry(t *testing.T) { }, }, } - s, sqlDB, kvDB := serverutils.StartServer(t, params) - defer s.Stopper().Stop(context.Background()) + srv, sqlDB, kvDB := serverutils.StartServer(t, params) + defer srv.Stopper().Stop(context.Background()) + s := srv.ApplicationLayer() if _, err := sqlDB.Exec(` CREATE DATABASE t; @@ -928,7 +933,7 @@ CREATE TABLE t.foo (v INT); t.Fatalf("CREATE TABLE has acquired a descriptor") } - tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "foo") + tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, s.Codec(), "t", "foo") atomic.StoreInt64(&tableID, int64(tableDesc.GetID())) tx, err := sqlDB.Begin() @@ -982,8 +987,9 @@ func TestTxnObeysTableModificationTime(t *testing.T) { var params base.TestServerArgs params.Knobs.JobsTestingKnobs = jobs.NewTestingKnobsWithShortIntervals() - s, sqlDB, _ := serverutils.StartServer(t, params) - defer s.Stopper().Stop(context.Background()) + params.DefaultTestTenant = base.TestDoesNotWorkWithSecondaryTenantsButWeDontKnowWhyYet(109385) + srv, sqlDB, _ := serverutils.StartServer(t, params) + defer srv.Stopper().Stop(context.Background()) _, err := sqlDB.Exec(` CREATE DATABASE t; @@ -1105,8 +1111,9 @@ func TestLeaseAtLatestVersion(t *testing.T) { }, }, } - s, sqlDB, kvDB := serverutils.StartServer(t, params) - defer s.Stopper().Stop(context.Background()) + srv, sqlDB, kvDB := serverutils.StartServer(t, params) + defer srv.Stopper().Stop(context.Background()) + s := srv.ApplicationLayer() if _, err := sqlDB.Exec(` BEGIN; @@ -1119,7 +1126,7 @@ COMMIT; t.Fatal(err) } - tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "kv") + tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, s.Codec(), "t", "kv") var updated bool if err := crdb.ExecuteTx(context.Background(), sqlDB, nil, func(tx *gosql.Tx) error { // Insert an entry so that the transaction is guaranteed to be @@ -1180,7 +1187,7 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR); t.Fatal(err) } - tableDesc := desctestutils.TestingGetPublicTableDescriptor(t.kvDB, keys.SystemSQLCodec, "t", "test") + tableDesc := desctestutils.TestingGetPublicTableDescriptor(t.kvDB, t.server.Codec(), "t", "test") dbID := tableDesc.GetParentID() tableName := tableDesc.GetName() leaseManager := t.node(1) @@ -1271,8 +1278,8 @@ CREATE TABLE t.test2 (); t.Fatal(err) } - test1Desc := desctestutils.TestingGetPublicTableDescriptor(t.kvDB, keys.SystemSQLCodec, "t", "test1") - test2Desc := desctestutils.TestingGetPublicTableDescriptor(t.kvDB, keys.SystemSQLCodec, "t", "test2") + test1Desc := desctestutils.TestingGetPublicTableDescriptor(t.kvDB, t.server.Codec(), "t", "test1") + test2Desc := desctestutils.TestingGetPublicTableDescriptor(t.kvDB, t.server.Codec(), "t", "test2") dbID := test2Desc.GetParentID() // Acquire a lease on test1 by name. @@ -1401,8 +1408,9 @@ func TestIncrementTableVersion(t *testing.T) { }, }, } - s, sqlDB, kvDB := serverutils.StartServer(t, params) - defer s.Stopper().Stop(context.Background()) + srv, sqlDB, kvDB := serverutils.StartServer(t, params) + defer srv.Stopper().Stop(context.Background()) + codec := srv.ApplicationLayer().Codec() if _, err := sqlDB.Exec(` CREATE DATABASE t; @@ -1411,7 +1419,7 @@ CREATE TABLE t.kv (k CHAR PRIMARY KEY, v CHAR); t.Fatal(err) } - tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "kv") + tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, codec, "t", "kv") if tableDesc.GetVersion() != 1 { t.Fatalf("invalid version %d", tableDesc.GetVersion()) } @@ -1432,7 +1440,7 @@ CREATE TABLE t.kv (k CHAR PRIMARY KEY, v CHAR); } // The first schema change will succeed and increment the version. - tableDesc = desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "kv1") + tableDesc = desctestutils.TestingGetPublicTableDescriptor(kvDB, codec, "t", "kv1") if tableDesc.GetVersion() != 2 { t.Fatalf("invalid version %d", tableDesc.GetVersion()) } @@ -1462,7 +1470,7 @@ CREATE TABLE t.kv (k CHAR PRIMARY KEY, v CHAR); // the table descriptor. If the schema change transaction // doesn't rollback the transaction this descriptor read will // hang. - tableDesc = desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "kv1") + tableDesc = desctestutils.TestingGetPublicTableDescriptor(kvDB, codec, "t", "kv1") if tableDesc.GetVersion() != 2 { t.Fatalf("invalid version %d", tableDesc.GetVersion()) } @@ -1473,7 +1481,7 @@ CREATE TABLE t.kv (k CHAR PRIMARY KEY, v CHAR); } wg.Wait() - tableDesc = desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "kv2") + tableDesc = desctestutils.TestingGetPublicTableDescriptor(kvDB, codec, "t", "kv2") if tableDesc.GetVersion() != 3 { t.Fatalf("invalid version %d", tableDesc.GetVersion()) } @@ -1487,6 +1495,8 @@ CREATE TABLE t.kv (k CHAR PRIMARY KEY, v CHAR); // same with cockroach_restart, which commits on release. func TestTwoVersionInvariantRetryErrorWitSavePoint(t *testing.T) { defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + var violations int64 var params base.TestServerArgs params.Knobs = base.TestingKnobs{ @@ -1504,8 +1514,9 @@ func TestTwoVersionInvariantRetryErrorWitSavePoint(t *testing.T) { }, }, } - s, sqlDB, kvDB := serverutils.StartServer(t, params) - defer s.Stopper().Stop(context.Background()) + srv, sqlDB, kvDB := serverutils.StartServer(t, params) + defer srv.Stopper().Stop(context.Background()) + s := srv.ApplicationLayer() if _, err := sqlDB.Exec(` CREATE DATABASE t; @@ -1515,7 +1526,7 @@ INSERT INTO t.kv VALUES ('a', 'b'); t.Fatal(err) } - tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "kv") + tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, s.Codec(), "t", "kv") if tableDesc.GetVersion() != 1 { t.Fatalf("invalid version %d", tableDesc.GetVersion()) } @@ -1620,8 +1631,9 @@ func TestTwoVersionInvariantRetryError(t *testing.T) { }, }, } - s, sqlDB, kvDB := serverutils.StartServer(t, params) - defer s.Stopper().Stop(context.Background()) + srv, sqlDB, kvDB := serverutils.StartServer(t, params) + defer srv.Stopper().Stop(context.Background()) + s := srv.ApplicationLayer() if _, err := sqlDB.Exec(` CREATE DATABASE t; @@ -1631,7 +1643,7 @@ INSERT INTO t.kv VALUES ('a', 'b'); t.Fatal(err) } - tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "kv") + tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, s.Codec(), "t", "kv") if tableDesc.GetVersion() != 1 { t.Fatalf("invalid version %d", tableDesc.GetVersion()) } @@ -1703,6 +1715,7 @@ INSERT INTO t.kv VALUES ('a', 'b'); func TestModificationTimeTxnOrdering(testingT *testing.T) { defer leaktest.AfterTest(testingT)() + defer log.Scope(testingT).Close(testingT) // Decide how long we should run this. maxTime := time.Duration(20) * time.Second @@ -1768,7 +1781,7 @@ CREATE TABLE t.test0 (k CHAR PRIMARY KEY, v CHAR); require.NoError(t, txn.SetFixedTimestamp(ctx, table.GetModificationTime())) // Look up the descriptor. - descKey := catalogkeys.MakeDescMetadataKey(keys.SystemSQLCodec, descID) + descKey := catalogkeys.MakeDescMetadataKey(t.server.Codec(), descID) res, err := txn.Get(ctx, descKey) if err != nil { t.Fatalf("error while reading proto: %v", err) @@ -1861,8 +1874,8 @@ CREATE TABLE t.test2 (); t.Fatal(err) } - test1Desc := desctestutils.TestingGetPublicTableDescriptor(t.kvDB, keys.SystemSQLCodec, "t", "test1") - test2Desc := desctestutils.TestingGetPublicTableDescriptor(t.kvDB, keys.SystemSQLCodec, "t", "test2") + test1Desc := desctestutils.TestingGetPublicTableDescriptor(t.kvDB, t.server.Codec(), "t", "test1") + test2Desc := desctestutils.TestingGetPublicTableDescriptor(t.kvDB, t.server.Codec(), "t", "test2") dbID := test2Desc.GetParentID() func() { mu.Lock() @@ -1938,7 +1951,9 @@ func TestReadBeforeDrop(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) + s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{ + DefaultTestTenant: base.TestDoesNotWorkWithSecondaryTenantsButWeDontKnowWhyYet(109385), + }) defer s.Stopper().Stop(context.Background()) if _, err := sqlDB.Exec(` @@ -1985,11 +2000,16 @@ INSERT INTO t.kv VALUES ('a', 'b'); // of a TABLE CREATE are pushed to allow them to observe the created table. func TestTableCreationPushesTxnsInRecentPast(t *testing.T) { defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + tc := serverutils.StartCluster(t, 3, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + DefaultTestTenant: base.TestDoesNotWorkWithSecondaryTenantsButWeDontKnowWhyYet(109385), + }, ReplicationMode: base.ReplicationManual, }) defer tc.Stopper().Stop(context.Background()) - sqlDB := tc.ServerConn(0) + sqlDB := tc.ApplicationLayer(0).SQLConn(t, "") if _, err := sqlDB.Exec(` CREATE DATABASE t; @@ -2006,7 +2026,7 @@ func TestTableCreationPushesTxnsInRecentPast(t *testing.T) { // Create a transaction before the table is created. Use a different // node so that clock uncertainty is presumed and it gets pushed. - tx1, err := tc.ServerConn(1).Begin() + tx1, err := tc.ApplicationLayer(1).SQLConn(t, "t").Begin() if err != nil { t.Fatal(err) } @@ -2043,6 +2063,7 @@ INSERT INTO t.kv VALUES ('c', 'd'); // Tests that DeleteOrphanedLeases() deletes only orphaned leases. func TestDeleteOrphanedLeases(testingT *testing.T) { defer leaktest.AfterTest(testingT)() + defer log.Scope(testingT).Close(testingT) var params base.TestServerArgs params.Knobs = base.TestingKnobs{ @@ -2061,8 +2082,8 @@ CREATE TABLE t.after (k CHAR PRIMARY KEY, v CHAR); t.Fatal(err) } - beforeDesc := desctestutils.TestingGetPublicTableDescriptor(t.kvDB, keys.SystemSQLCodec, "t", "before") - afterDesc := desctestutils.TestingGetPublicTableDescriptor(t.kvDB, keys.SystemSQLCodec, "t", "after") + beforeDesc := desctestutils.TestingGetPublicTableDescriptor(t.kvDB, t.server.Codec(), "t", "before") + afterDesc := desctestutils.TestingGetPublicTableDescriptor(t.kvDB, t.server.Codec(), "t", "after") dbID := beforeDesc.GetParentID() // Acquire a lease on "before" by name. @@ -2114,8 +2135,9 @@ func TestLeaseAcquisitionDoesntBlock(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - s, db, _ := serverutils.StartServer(t, base.TestServerArgs{}) - defer s.Stopper().Stop(ctx) + srv, db, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() _, err := db.Exec(`CREATE DATABASE t; CREATE TABLE t.test(k CHAR PRIMARY KEY, v CHAR);`) require.NoError(t, err) @@ -2211,6 +2233,7 @@ func TestLeaseAcquisitionByNameDoesntBlock(t *testing.T) { // are aborted. func TestIntentOnSystemConfigDoesNotPreventSchemaChange(t *testing.T) { defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) ctx := context.Background() tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) @@ -2295,6 +2318,7 @@ func ensureTestTakesLessThan(t *testing.T, allowed time.Duration) func() { // too old. func TestRangefeedUpdatesHandledProperlyInTheFaceOfRaces(t *testing.T) { defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) defer ensureTestTakesLessThan(t, 30*time.Second)() ctx := context.Background() @@ -2410,6 +2434,7 @@ func TestRangefeedUpdatesHandledProperlyInTheFaceOfRaces(t *testing.T) { // See #57834. func TestLeaseWithOfflineTables(t *testing.T) { defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) var descID uint32 testTableID := func() descpb.ID { @@ -2428,8 +2453,9 @@ func TestLeaseWithOfflineTables(t *testing.T) { ctx := context.Background() var params base.TestServerArgs params.Knobs.SQLLeaseManager = &lmKnobs - s, db, kvDB := serverutils.StartServer(t, params) - defer s.Stopper().Stop(ctx) + srv, db, kvDB := serverutils.StartServer(t, params) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() runner := sqlutils.MakeSQLRunner(db) // This statement prevents timer issues due to periodic lease refreshing. @@ -2444,7 +2470,7 @@ func TestLeaseWithOfflineTables(t *testing.T) { `) require.NoError(t, err) - desc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") + desc := desctestutils.TestingGetPublicTableDescriptor(kvDB, s.Codec(), "t", "test") atomic.StoreUint32(&descID, uint32(desc.GetID())) // Sets table descriptor state and waits for that change to propagate to the @@ -2525,17 +2551,21 @@ func TestLeaseWithOfflineTables(t *testing.T) { // to just delete it than deflake it. func TestOutstandingLeasesMetric(t *testing.T) { defer leaktest.AfterTest(t)() - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) + defer log.Scope(t).Close(t) + ctx := context.Background() - defer tc.Stopper().Stop(ctx) - _, err := tc.Conns[0].ExecContext(ctx, "CREATE TABLE a (a INT PRIMARY KEY)") + srv, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() + + _, err := sqlDB.ExecContext(ctx, "CREATE TABLE a (a INT PRIMARY KEY)") assert.NoError(t, err) - _, err = tc.Conns[0].ExecContext(ctx, "CREATE TABLE b (a INT PRIMARY KEY)") + _, err = sqlDB.ExecContext(ctx, "CREATE TABLE b (a INT PRIMARY KEY)") assert.NoError(t, err) - gauge := tc.Servers[0].LeaseManager().(*lease.Manager).TestingOutstandingLeasesGauge() + gauge := s.LeaseManager().(*lease.Manager).TestingOutstandingLeasesGauge() outstandingLeases := gauge.Value() - _, err = tc.Conns[0].ExecContext(ctx, "SELECT * FROM a") + _, err = sqlDB.ExecContext(ctx, "SELECT * FROM a") assert.NoError(t, err) afterQuery := gauge.Value() @@ -2549,7 +2579,7 @@ func TestOutstandingLeasesMetric(t *testing.T) { } // Expect at least 3 leases: one for a, one for the default database, and one for b. - _, err = tc.Conns[0].ExecContext(ctx, "SELECT * FROM b") + _, err = sqlDB.ExecContext(ctx, "SELECT * FROM b") assert.NoError(t, err) afterQuery = gauge.Value() @@ -2563,6 +2593,7 @@ func TestOutstandingLeasesMetric(t *testing.T) { // can read an old descriptor. func TestHistoricalAcquireDroppedDescriptor(t *testing.T) { defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) const typeName = "foo" seenDrop := make(chan error) @@ -2608,14 +2639,13 @@ func TestHistoricalAcquireDroppedDescriptor(t *testing.T) { // store. func TestHistoricalDescriptorAcquire(t *testing.T) { defer leaktest.AfterTest(t)() - var stopper *stop.Stopper - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ - ServerArgs: base.TestServerArgs{}, - }) - stopper = tc.Stopper() + defer log.Scope(t).Close(t) + ctx := context.Background() - defer stopper.Stop(ctx) - tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + srv, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() + tdb := sqlutils.MakeSQLRunner(sqlDB) // Create a schema, create table, alter table a few times to get some history // of tables while keeping timestamp checkpoints for acquire query @@ -2644,10 +2674,9 @@ func TestHistoricalDescriptorAcquire(t *testing.T) { // Acquire descriptor version valid at timestamp ts1. Waits for the most // recent version with the name column before doing so. - _, err = tc.Server(0).LeaseManager().(*lease.Manager).WaitForOneVersion(ctx, tableID.Load().(descpb.ID), base.DefaultRetryOptions()) + _, err = s.LeaseManager().(*lease.Manager).WaitForOneVersion(ctx, tableID.Load().(descpb.ID), base.DefaultRetryOptions()) require.NoError(t, err, "Failed to wait for one version of descriptor: %s", err) - acquiredDescriptor, err := - tc.Server(0).LeaseManager().(*lease.Manager).Acquire(ctx, ts1, tableID.Load().(descpb.ID)) + acquiredDescriptor, err := s.LeaseManager().(*lease.Manager).Acquire(ctx, ts1, tableID.Load().(descpb.ID)) assert.NoError(t, err) // Ensure the modificationTime <= timestamp < expirationTime @@ -2660,6 +2689,7 @@ func TestHistoricalDescriptorAcquire(t *testing.T) { func TestDropDescriptorRacesWithAcquisition(t *testing.T) { defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) // We want to have a transaction to acquire a descriptor on one // node that starts and reads version 1. Then we'll write a new @@ -2784,6 +2814,8 @@ func TestDropDescriptorRacesWithAcquisition(t *testing.T) { // trying to online the table perpetually (as seen in issue #61798). func TestOfflineLeaseRefresh(t *testing.T) { defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + ctx := context.Background() waitForTxn := make(chan chan struct{}) waitForRqstFilter := make(chan chan struct{}) @@ -2805,10 +2837,9 @@ func TestOfflineLeaseRefresh(t *testing.T) { }, } params := base.TestServerArgs{Knobs: base.TestingKnobs{Store: knobs}} - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ServerArgs: params}) - s := tc.Server(0) - defer tc.Stopper().Stop(ctx) - conn := tc.ServerConn(0) + srv, conn, _ := serverutils.StartServer(t, params) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() // Create t1 that will be offline, and t2, // that will serve inserts. @@ -3090,6 +3121,7 @@ SELECT * FROM T1`) // if the deadline is found to be expired. func TestLeaseBulkInsertWithImplicitTxn(t *testing.T) { defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) beforeExecute := syncutil.Mutex{} // Statement that will be paused. @@ -3151,9 +3183,10 @@ func TestLeaseBulkInsertWithImplicitTxn(t *testing.T) { }, } - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ServerArgs: params}) - defer tc.Stopper().Stop(ctx) - conn := tc.ServerConn(0) + srv, conn, _ := serverutils.StartServer(t, params) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() + sql.SecondaryTenantSplitAtEnabled.Override(ctx, &s.ClusterSettings().SV, true) // Setup tables for the test. _, err := conn.Exec(` CREATE TABLE t1(val int); @@ -3162,7 +3195,7 @@ ALTER TABLE t1 SPLIT AT VALUES (1); require.NoError(t, err) // Get the lease manager and table ID for acquiring a lease on. beforeExecute.Lock() - leaseManager = tc.Servers[0].LeaseManager().(*lease.Manager) + leaseManager = s.LeaseManager().(*lease.Manager) beforeExecute.Unlock() tempTableID := uint64(0) err = conn.QueryRow("SELECT table_id FROM crdb_internal.tables WHERE name = $1 AND database_name = current_database()", @@ -3177,10 +3210,8 @@ ALTER TABLE t1 SPLIT AT VALUES (1); t.Run("validate-lease-txn-deadline-ext-update", func(t *testing.T) { updateCompleted := atomic.Value{} updateCompleted.Store(false) - conn, err := tc.ServerConn(0).Conn(ctx) - require.NoError(t, err) - updateConn, err := tc.ServerConn(0).Conn(ctx) - require.NoError(t, err) + conn := s.SQLConn(t, "") + updateConn := s.SQLConn(t, "") resultChan := make(chan error) _, err = conn.ExecContext(ctx, ` INSERT INTO t1 select a from generate_series(1, 100) g(a); @@ -3238,6 +3269,7 @@ INSERT INTO t1 select a from generate_series(1, 100) g(a); // This is important to avoid leaking leases when nodes are being shut down. func TestAmbiguousResultIsRetried(t *testing.T) { defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) type filter = kvserverbase.ReplicaResponseFilter var f atomic.Value @@ -3246,7 +3278,7 @@ func TestAmbiguousResultIsRetried(t *testing.T) { }) f.Store(noop) ctx := context.Background() - s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{ + srv, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{ Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ TestingResponseFilter: func(ctx context.Context, request *kvpb.BatchRequest, response *kvpb.BatchResponse) *kvpb.Error { @@ -3255,14 +3287,16 @@ func TestAmbiguousResultIsRetried(t *testing.T) { }, }, }) - defer s.Stopper().Stop(ctx) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() + codec := s.Codec() sqlutils.MakeSQLRunner(sqlDB).Exec(t, "SET CLUSTER SETTING sql.stats.automatic_collection.enabled = false") sqlutils.MakeSQLRunner(sqlDB).Exec(t, "CREATE TABLE foo ()") tableID := sqlutils.QueryTableID(t, sqlDB, "defaultdb", "public", "foo") - tablePrefix := keys.SystemSQLCodec.TablePrefix(keys.LeaseTableID) + tablePrefix := codec.TablePrefix(keys.LeaseTableID) var txnID atomic.Value txnID.Store(uuid.UUID{}) @@ -3275,7 +3309,7 @@ func TestAmbiguousResultIsRetried(t *testing.T) { if !bytes.HasPrefix(r.Key, tablePrefix) { return nil } - in, _, _, err := keys.DecodeTableIDIndexID(r.Key) + in, _, _, err := codec.DecodeIndexPrefix(r.Key) if err != nil { return kvpb.NewError(errors.WithAssertionFailure(err)) } @@ -3368,8 +3402,9 @@ func TestDescriptorRemovedFromCacheWhenLeaseRenewalForThisDescriptorFails(t *tes // knob will be invoked, and eventually the logic to remove unfound descriptor from cache will be triggered. lease.LeaseDuration.Override(ctx, ¶ms.SV, time.Second) - s, sqlDB, kvDB := serverutils.StartServer(t, params) - defer s.Stopper().Stop(ctx) + srv, sqlDB, kvDB := serverutils.StartServer(t, params) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() tdb = sqlutils.MakeSQLRunner(sqlDB) sql := ` @@ -3380,8 +3415,7 @@ func TestDescriptorRemovedFromCacheWhenLeaseRenewalForThisDescriptorFails(t *tes tdb.Exec(t, sql) // Ensure `typ` is present in the lease manger by acquiring a lease on it. - typeDesc := desctestutils.TestingGetTypeDescriptor(kvDB, keys.SystemSQLCodec, - "test", "public", "typ") + typeDesc := desctestutils.TestingGetPublicTypeDescriptor(kvDB, s.Codec(), "test", "typ") lm := s.LeaseManager().(*lease.Manager) _, err := lm.Acquire(ctx, s.Clock().Now(), typeDesc.GetID()) require.NoError(t, err) diff --git a/pkg/sql/catalog/redact/BUILD.bazel b/pkg/sql/catalog/redact/BUILD.bazel index 6e543c56c4ce..030622552111 100644 --- a/pkg/sql/catalog/redact/BUILD.bazel +++ b/pkg/sql/catalog/redact/BUILD.bazel @@ -25,7 +25,6 @@ go_test( deps = [ ":redact", "//pkg/base", - "//pkg/keys", "//pkg/security/securityassets", "//pkg/security/securitytest", "//pkg/server", diff --git a/pkg/sql/catalog/redact/redact_test.go b/pkg/sql/catalog/redact/redact_test.go index 0c7e28a6f00d..5d73610d25fc 100644 --- a/pkg/sql/catalog/redact/redact_test.go +++ b/pkg/sql/catalog/redact/redact_test.go @@ -15,7 +15,6 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils" "github.com/cockroachdb/cockroach/pkg/sql/catalog/funcdesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/redact" @@ -32,8 +31,9 @@ func TestRedactQueries(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - s, db, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) - defer s.Stopper().Stop(ctx) + srv, db, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + codec := srv.ApplicationLayer().Codec() tdb := sqlutils.MakeSQLRunner(db) tdb.Exec(t, "CREATE TABLE kv (k INT PRIMARY KEY, v STRING)") tdb.Exec(t, "CREATE VIEW view AS SELECT k, v FROM kv WHERE v <> 'constant literal'") @@ -48,7 +48,7 @@ $$`) t.Run("view", func(t *testing.T) { view := desctestutils.TestingGetTableDescriptor( - kvDB, keys.SystemSQLCodec, "defaultdb", "public", "view", + kvDB, codec, "defaultdb", "public", "view", ) mut := tabledesc.NewBuilder(view.TableDesc()).BuildCreatedMutableTable() require.Empty(t, redact.Redact(mut.DescriptorProto())) @@ -57,7 +57,7 @@ $$`) t.Run("create table as", func(t *testing.T) { ctas := desctestutils.TestingGetTableDescriptor( - kvDB, keys.SystemSQLCodec, "defaultdb", "public", "ctas", + kvDB, codec, "defaultdb", "public", "ctas", ) mut := tabledesc.NewBuilder(ctas.TableDesc()).BuildCreatedMutableTable() require.Empty(t, redact.Redact(mut.DescriptorProto())) @@ -65,7 +65,7 @@ $$`) }) t.Run("create function", func(t *testing.T) { - fn := desctestutils.TestingGetFunctionDescriptor(kvDB, keys.SystemSQLCodec, "defaultdb", "public", "f1") + fn := desctestutils.TestingGetFunctionDescriptor(kvDB, codec, "defaultdb", "public", "f1") mut := funcdesc.NewBuilder(fn.FuncDesc()).BuildCreatedMutableFunction() require.Empty(t, redact.Redact(mut.DescriptorProto())) require.Equal(t, `SELECT k FROM defaultdb.public.kv WHERE v != '_'; SELECT k FROM defaultdb.public.kv WHERE v = '_';`, mut.FunctionBody) diff --git a/pkg/sql/catalog/resolver/resolver_test.go b/pkg/sql/catalog/resolver/resolver_test.go index 8d92adbd1b19..cc0e801af2c8 100644 --- a/pkg/sql/catalog/resolver/resolver_test.go +++ b/pkg/sql/catalog/resolver/resolver_test.go @@ -530,8 +530,9 @@ func TestResolveIndex(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) - defer s.Stopper().Stop(ctx) + srv, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() tDB := sqlutils.MakeSQLRunner(sqlDB) tDB.Exec(t, ` @@ -667,8 +668,9 @@ func TestResolveIndexWithOfflineTable(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) - defer s.Stopper().Stop(ctx) + srv, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() tDB := sqlutils.MakeSQLRunner(sqlDB) tDB.Exec(t, ` diff --git a/pkg/sql/catalog/schematelemetry/schema_telemetry_test.go b/pkg/sql/catalog/schematelemetry/schema_telemetry_test.go index 491a1be5c416..267aa6de29ea 100644 --- a/pkg/sql/catalog/schematelemetry/schema_telemetry_test.go +++ b/pkg/sql/catalog/schematelemetry/schema_telemetry_test.go @@ -77,12 +77,16 @@ func TestSchemaTelemetrySchedule(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - s, db, _ := serverutils.StartServer(t, makeTestServerArgs()) - defer s.Stopper().Stop(ctx) + args := makeTestServerArgs() + // 'sql.schema.telemetry.recurrence' setting is settable only by the + // operator. + args.DefaultTestTenant = base.TestIsSpecificToStorageLayerAndNeedsASystemTenant + srv, db, _ := serverutils.StartServer(t, args) + defer srv.Stopper().Stop(ctx) + s := srv.ApplicationLayer() tdb := sqlutils.MakeSQLRunner(db) - clusterID := s.ExecutorConfig().(sql.ExecutorConfig).NodeInfo. - LogicalClusterID() + clusterID := s.ExecutorConfig().(sql.ExecutorConfig).NodeInfo.LogicalClusterID() exp := scheduledjobs.MaybeRewriteCronExpr(clusterID, "@weekly") tdb.CheckQueryResultsRetry(t, qExists, [][]string{{exp, "1"}}) tdb.ExecSucceedsSoon(t, qSet) diff --git a/pkg/sql/catalog/tabledesc/BUILD.bazel b/pkg/sql/catalog/tabledesc/BUILD.bazel index 3477fa00f7f7..8737e92b3f28 100644 --- a/pkg/sql/catalog/tabledesc/BUILD.bazel +++ b/pkg/sql/catalog/tabledesc/BUILD.bazel @@ -119,6 +119,7 @@ go_test( "//pkg/testutils/sqlutils", "//pkg/util/hlc", "//pkg/util/leaktest", + "//pkg/util/log", "//pkg/util/protoutil", "//pkg/util/randutil", "@com_github_cockroachdb_errors//:errors", diff --git a/pkg/sql/catalog/tabledesc/index_test.go b/pkg/sql/catalog/tabledesc/index_test.go index 5bfe69d0949f..d6ae55ad42e6 100644 --- a/pkg/sql/catalog/tabledesc/index_test.go +++ b/pkg/sql/catalog/tabledesc/index_test.go @@ -22,7 +22,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/geo/geoindex" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" - "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" @@ -37,16 +36,19 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/stretchr/testify/require" ) func TestIndexInterface(t *testing.T) { defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) // This server is only used to turn a CREATE TABLE statement into a // catalog.TableDescriptor. - s, conn, db := serverutils.StartServer(t, base.TestServerArgs{}) - defer s.Stopper().Stop(context.Background()) + srv, conn, db := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(context.Background()) + s := srv.ApplicationLayer() if _, err := conn.Exec(`CREATE DATABASE d`); err != nil { t.Fatalf("%+v", err) } @@ -89,7 +91,7 @@ func TestIndexInterface(t *testing.T) { {0, 1, 2}, } - immutable := desctestutils.TestingGetPublicTableDescriptor(db, keys.SystemSQLCodec, "d", "t") + immutable := desctestutils.TestingGetPublicTableDescriptor(db, s.Codec(), "d", "t") require.NotNil(t, immutable) var tableI = immutable require.NotNil(t, tableI) @@ -335,11 +337,14 @@ func TestIndexInterface(t *testing.T) { // redundant column IDs in descpb.IndexDescriptor. func TestIndexStrictColumnIDs(t *testing.T) { defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + ctx := context.Background() // Create a regular table with a secondary index. - s, conn, db := serverutils.StartServer(t, base.TestServerArgs{}) - defer s.Stopper().Stop(context.Background()) + srv, conn, db := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(context.Background()) + codec := srv.ApplicationLayer().Codec() _, err := conn.Exec(` CREATE DATABASE d; CREATE TABLE d.t ( @@ -353,7 +358,7 @@ func TestIndexStrictColumnIDs(t *testing.T) { // Mess with the table descriptor to add redundant columns in the secondary // index while still passing validation. - mut := desctestutils.TestingGetMutableExistingTableDescriptor(db, keys.SystemSQLCodec, "d", "t") + mut := desctestutils.TestingGetMutableExistingTableDescriptor(db, codec, "d", "t") idx := &mut.Indexes[0] id := idx.KeyColumnIDs[0] name := idx.KeyColumnNames[0] @@ -367,7 +372,7 @@ func TestIndexStrictColumnIDs(t *testing.T) { // Store the corrupted table descriptor. err = db.Put( ctx, - catalogkeys.MakeDescMetadataKey(keys.SystemSQLCodec, mut.GetID()), + catalogkeys.MakeDescMetadataKey(codec, mut.GetID()), mut.DescriptorProto(), ) require.NoError(t, err) @@ -388,7 +393,11 @@ func TestIndexStrictColumnIDs(t *testing.T) { var msg string err = rows.Scan(&msg) require.NoError(t, err) - expected := fmt.Sprintf(`InitPut /Table/%d/2/0/0/0/0/0/0 -> /BYTES/0x2300030003000300`, mut.GetID()) + var tenantPrefix string + if srv.StartedDefaultTestTenant() { + tenantPrefix = codec.TenantPrefix().String() + } + expected := fmt.Sprintf(`InitPut %s/Table/%d/2/0/0/0/0/0/0 -> /BYTES/0x2300030003000300`, tenantPrefix, mut.GetID()) require.Equal(t, expected, msg) // Test that with the strict guarantees, this table descriptor would have been @@ -425,8 +434,9 @@ func TestLatestIndexDescriptorVersionValues(t *testing.T) { }, }, } - s, sqlDB, kvDB := serverutils.StartServer(t, args) - defer s.Stopper().Stop(ctx) + srv, sqlDB, kvDB := serverutils.StartServer(t, args) + defer srv.Stopper().Stop(ctx) + codec := srv.ApplicationLayer().Codec() tdb := sqlutils.MakeSQLRunner(sqlDB) // Test relies on legacy schema changer testing knobs. @@ -557,7 +567,7 @@ func TestLatestIndexDescriptorVersionValues(t *testing.T) { // Test again but with RunPostDeserializationChanges. for _, name := range []string{`t`, `s`, `v`} { - desc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "defaultdb", name) + desc := desctestutils.TestingGetPublicTableDescriptor(kvDB, codec, "defaultdb", name) test(desc) } diff --git a/pkg/sql/catalog/tabledesc/structured_test.go b/pkg/sql/catalog/tabledesc/structured_test.go index fbdf0afb883a..433788cd6034 100644 --- a/pkg/sql/catalog/tabledesc/structured_test.go +++ b/pkg/sql/catalog/tabledesc/structured_test.go @@ -19,7 +19,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/clusterversion" - "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/bootstrap" @@ -35,6 +34,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "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/protoutil" "github.com/kr/pretty" "github.com/stretchr/testify/require" @@ -788,8 +788,9 @@ func TestKeysPerRow(t *testing.T) { // TODO(dan): This server is only used to turn a CREATE TABLE statement into // a descpb.TableDescriptor. It should be possible to move MakeTableDesc into // sqlbase. If/when that happens, use it here instead of this server. - s, conn, db := serverutils.StartServer(t, base.TestServerArgs{}) - defer s.Stopper().Stop(context.Background()) + srv, conn, db := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(context.Background()) + codec := srv.ApplicationLayer().Codec() if _, err := conn.Exec(`CREATE DATABASE d`); err != nil { t.Fatalf("%+v", err) } @@ -850,7 +851,7 @@ func TestKeysPerRow(t *testing.T) { tableName := fmt.Sprintf("t%d", i) sqlDB.Exec(t, fmt.Sprintf(`CREATE TABLE d.%s %s`, tableName, test.createTable)) - desc := desctestutils.TestingGetPublicTableDescriptor(db, keys.SystemSQLCodec, "d", tableName) + desc := desctestutils.TestingGetPublicTableDescriptor(db, codec, "d", tableName) require.NotNil(t, desc) idx, err := catalog.MustFindIndexByID(desc, test.indexID) require.NoError(t, err) @@ -968,9 +969,13 @@ func TestDefaultExprNil(t *testing.T) { // TestStrippedDanglingSelfBackReferences checks the proper behavior of the // catalog.StrippedDanglingSelfBackReferences post-deserialization change. func TestStrippedDanglingSelfBackReferences(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + ctx := context.Background() - s, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) - defer s.Stopper().Stop(ctx) + srv, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(ctx) + codec := srv.ApplicationLayer().Codec() tdb := sqlutils.MakeSQLRunner(sqlDB) // Create a table. @@ -978,7 +983,7 @@ func TestStrippedDanglingSelfBackReferences(t *testing.T) { tdb.Exec(t, `CREATE TABLE t.tbl (a INT PRIMARY KEY)`) // Get the descriptor for the table. - tbl := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "tbl") + tbl := desctestutils.TestingGetPublicTableDescriptor(kvDB, codec, "t", "tbl") // Inject some nonsense into the mutation_jobs slice. mut := NewBuilder(tbl.TableDesc()).BuildExistingMutableTable() @@ -999,8 +1004,12 @@ func TestStrippedDanglingSelfBackReferences(t *testing.T) { // correctly removed from descriptors of computed columns as part of the // RunPostDeserializationChanges suite. func TestRemoveDefaultExprFromComputedColumn(t *testing.T) { - s, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) - defer s.Stopper().Stop(context.Background()) + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + srv, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) + defer srv.Stopper().Stop(context.Background()) + codec := srv.ApplicationLayer().Codec() tdb := sqlutils.MakeSQLRunner(sqlDB) const expectedErrRE = `.*: computed column \"b\" cannot also have a DEFAULT expression` @@ -1009,7 +1018,7 @@ func TestRemoveDefaultExprFromComputedColumn(t *testing.T) { tdb.Exec(t, `CREATE TABLE t.tbl (a INT PRIMARY KEY, b INT AS (1) STORED)`) // Get the descriptor for the table. - tbl := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "tbl") + tbl := desctestutils.TestingGetPublicTableDescriptor(kvDB, codec, "t", "tbl") // Setting a default value on the computed column should fail. tdb.ExpectErr(t, expectedErrRE, `ALTER TABLE t.tbl ALTER COLUMN b SET DEFAULT 2`) diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index 4ff3be636f82..249fe6f8c240 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -3753,10 +3753,10 @@ func DescsTxn( } // TestingDescsTxn is a convenience function for running a transaction on -// descriptors when you have a serverutils.TestServerInterface. +// descriptors when you have a serverutils.ApplicationLayerInterface. func TestingDescsTxn( ctx context.Context, - s serverutils.TestServerInterface, + s serverutils.ApplicationLayerInterface, f func(ctx context.Context, txn isql.Txn, col *descs.Collection) error, ) error { execCfg := s.ExecutorConfig().(ExecutorConfig) diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index 6f32a5e30d28..558a1a6252b3 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -2336,6 +2336,7 @@ func TestLint(t *testing.T) { ":!server/storage_api/rangelog_test.go", ":!server/testserver.go", ":!sql/catalog/internal/catkv/catalog_reader_test.go", + ":!sql/catalog/lease/lease_test.go", ":!sql/importer/import_processor_test.go", ":!sql/importer/import_stmt_test.go", ":!sql/importer/read_import_mysql_test.go", From acf63ee49bb05451f1a68bfc9c361d20c52357ca Mon Sep 17 00:00:00 2001 From: Ricky Stewart Date: Thu, 24 Aug 2023 17:21:26 -0500 Subject: [PATCH 4/8] dev: make `dev test --count 1` invalidate cached test results This matches the behavior of `go test`. Epic: none Release note: None --- dev | 2 +- pkg/cmd/dev/test.go | 15 +++++++++------ pkg/cmd/dev/testdata/datadriven/test | 17 +++++++++++------ 3 files changed, 21 insertions(+), 13 deletions(-) diff --git a/dev b/dev index a6aab8d50c0c..9dcac5df35b2 100755 --- a/dev +++ b/dev @@ -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 diff --git a/pkg/cmd/dev/test.go b/pkg/cmd/dev/test.go index 65bb1f3b1702..61578b01688f 100644 --- a/pkg/cmd/dev/test.go +++ b/pkg/cmd/dev/test.go @@ -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") @@ -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 { @@ -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 } @@ -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 != "" { @@ -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, ",")) } diff --git a/pkg/cmd/dev/testdata/datadriven/test b/pkg/cmd/dev/testdata/datadriven/test index d711eb05c3ce..230fd20a018c 100644 --- a/pkg/cmd/dev/testdata/datadriven/test +++ b/pkg/cmd/dev/testdata/datadriven/test @@ -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 @@ -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 @@ -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 @@ -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' @@ -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 From 0ec27751701914f082a38eafe7502d3ac72e7f24 Mon Sep 17 00:00:00 2001 From: Jayant Shrivastava Date: Fri, 25 Aug 2023 11:17:43 -0400 Subject: [PATCH 5/8] changefeedccl: ensure rangefeed setting is enabled in tests 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 https://github.com/cockroachdb/cockroach/issues/109306#issuecomment-1692428425 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: https://github.com/cockroachdb/cockroach/issues/109306 Epic: None Release note: None --- pkg/ccl/changefeedccl/cdceval/expr_eval_test.go | 1 + pkg/ccl/changefeedccl/cdcevent/event_test.go | 3 +++ pkg/ccl/changefeedccl/cdctest/BUILD.bazel | 1 + pkg/ccl/changefeedccl/cdctest/row.go | 8 ++++++++ pkg/ccl/changefeedccl/parquet_test.go | 1 + pkg/kv/kvserver/replica.go | 4 ++-- 6 files changed, 16 insertions(+), 2 deletions(-) diff --git a/pkg/ccl/changefeedccl/cdceval/expr_eval_test.go b/pkg/ccl/changefeedccl/cdceval/expr_eval_test.go index 875b84cea6a9..c291a39d42ab 100644 --- a/pkg/ccl/changefeedccl/cdceval/expr_eval_test.go +++ b/pkg/ccl/changefeedccl/cdceval/expr_eval_test.go @@ -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 ( diff --git a/pkg/ccl/changefeedccl/cdcevent/event_test.go b/pkg/ccl/changefeedccl/cdcevent/event_test.go index 2b5d00da3e73..7c64058218d7 100644 --- a/pkg/ccl/changefeedccl/cdcevent/event_test.go +++ b/pkg/ccl/changefeedccl/cdcevent/event_test.go @@ -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 ( @@ -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`) @@ -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, diff --git a/pkg/ccl/changefeedccl/cdctest/BUILD.bazel b/pkg/ccl/changefeedccl/cdctest/BUILD.bazel index 9fc078027ba0..195dab063c15 100644 --- a/pkg/ccl/changefeedccl/cdctest/BUILD.bazel +++ b/pkg/ccl/changefeedccl/cdctest/BUILD.bazel @@ -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", diff --git a/pkg/ccl/changefeedccl/cdctest/row.go b/pkg/ccl/changefeedccl/cdctest/row.go index 7bbf19c8ee0f..cde58e42aba7 100644 --- a/pkg/ccl/changefeedccl/cdctest/row.go +++ b/pkg/ccl/changefeedccl/cdctest/row.go @@ -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" @@ -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()) diff --git a/pkg/ccl/changefeedccl/parquet_test.go b/pkg/ccl/changefeedccl/parquet_test.go index c545fed3b3fd..822bb7e88f02 100644 --- a/pkg/ccl/changefeedccl/parquet_test.go +++ b/pkg/ccl/changefeedccl/parquet_test.go @@ -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 diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index d4e8b5a204fc..30b468a56f02 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -1844,8 +1844,8 @@ func (r *Replica) checkExecutionCanProceedForRangeFeed( } else if err := r.checkSpanInRangeRLocked(ctx, rSpan); err != nil { return err } else if !r.isRangefeedEnabledRLocked() && !RangefeedEnabled.Get(&r.store.cfg.Settings.SV) { - return errors.Errorf("rangefeeds require the kv.rangefeed.enabled setting. See %s", - docs.URL(`change-data-capture.html#enable-rangefeeds-to-reduce-latency`)) + return errors.Errorf("[r%d] rangefeeds require the kv.rangefeed.enabled setting. See %s", + r.RangeID, docs.URL(`change-data-capture.html#enable-rangefeeds-to-reduce-latency`)) } else if err := r.checkTSAboveGCThresholdRLocked(ts, status, false /* isAdmin */); err != nil { return err } From 98c762d4b53d7fff87636c15ac30bdf222f4ca2b Mon Sep 17 00:00:00 2001 From: Jayant Shrivastava Date: Wed, 23 Aug 2023 11:56:14 -0400 Subject: [PATCH 6/8] 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: https://github.com/cockroachdb/cockroach/issues/96953 Release note: None Epic: None --- .../changefeedccl/changefeed_processors.go | 99 ++++++++++--------- 1 file changed, 53 insertions(+), 46 deletions(-) diff --git a/pkg/ccl/changefeedccl/changefeed_processors.go b/pkg/ccl/changefeedccl/changefeed_processors.go index 6852cf9f8e52..0f391906cafc 100644 --- a/pkg/ccl/changefeedccl/changefeed_processors.go +++ b/pkg/ccl/changefeedccl/changefeed_processors.go @@ -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, @@ -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. @@ -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() @@ -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( @@ -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 { @@ -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, @@ -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() } @@ -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() } @@ -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. From 6e83704d3cdb38d28992db35667fef0819938337 Mon Sep 17 00:00:00 2001 From: Arul Ajmani Date: Fri, 25 Aug 2023 12:11:18 -0400 Subject: [PATCH 7/8] concurrency: use generic lists in the lock table 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 --- pkg/kv/kvserver/concurrency/BUILD.bazel | 1 + pkg/kv/kvserver/concurrency/lock_table.go | 127 +++++++++++----------- 2 files changed, 64 insertions(+), 64 deletions(-) diff --git a/pkg/kv/kvserver/concurrency/BUILD.bazel b/pkg/kv/kvserver/concurrency/BUILD.bazel index 00dc9e83c13a..6e9de2102709 100644 --- a/pkg/kv/kvserver/concurrency/BUILD.bazel +++ b/pkg/kv/kvserver/concurrency/BUILD.bazel @@ -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", diff --git a/pkg/kv/kvserver/concurrency/lock_table.go b/pkg/kv/kvserver/concurrency/lock_table.go index 7186be875646..3eada5528c36 100644 --- a/pkg/kv/kvserver/concurrency/lock_table.go +++ b/pkg/kv/kvserver/concurrency/lock_table.go @@ -11,7 +11,6 @@ package concurrency import ( - "container/list" "fmt" "sort" "sync" @@ -26,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/buildutil" + "github.com/cockroachdb/cockroach/pkg/util/container/list" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" @@ -708,7 +708,7 @@ func (g *lockTableGuardImpl) IsKeyLockedByConflictingTxn( if l.isLocked() { for e := l.holders.Front(); e != nil; e = e.Next() { - tl := e.Value.(*txnLock) + tl := e.Value lockHolderTxn, _ := tl.getLockHolder() if !g.isSameTxn(lockHolderTxn) && lock.Conflicts(tl.getLockMode(), makeLockMode(str, g.txn, g.ts), &g.lt.settings.SV) { @@ -733,7 +733,7 @@ func (g *lockTableGuardImpl) IsKeyLockedByConflictingTxn( } for e := l.queuedLockingRequests.Front(); e != nil; e = e.Next() { - qqg := e.Value.(*queuedGuard) + qqg := e.Value if qqg.guard.seqNum > g.seqNum { // We only need to check for conflicts with requests that came before us // (read: have lower sequence numbers than us). Note that the list of @@ -1163,14 +1163,11 @@ type keyLocks struct { // acquired by a different transaction. // // This list may be empty, in which case the current key is unlocked. - holders list.List + holders list.List[*txnLock] // heldByMap is used to lookup a txnLock in the holders list using transaction // IDs. This obviates the need to iterate over the entire list in various // places where we index by transaction ID. - // - // TODO(arul): This can change to map[uuid.UUID]*list.Element[*txnLock] once - // https://github.com/cockroachdb/cockroach/pull/109084 lands. - heldBy map[uuid.UUID]*list.Element + heldBy map[uuid.UUID]*list.Element[*txnLock] // Information about the requests waiting on the lock. lockWaitQueue @@ -1476,10 +1473,11 @@ type lockWaitQueue struct { // Waiters: An active waiter needs to be notified about changes in who it is // waiting for. - // List of *queueGuard. The list is maintained in increasing order of sequence - // numbers. This helps ensure some degree of fairness as requests are released - // from the head of the queue. Typically, this happens when all locks on the - // associated key are released. + // queuedLockingRequests is of requests queued at a key. They may be waiting + // actively or inactively. The list is maintained in increasing order of + // sequence numbers. This helps ensure some degree of fairness as requests are + // released from the head of the queue. Typically, this happens when all locks + // on the associated key are released. // // When a lock is not held, the head of the list should be comprised of an // inactive, transactional locking request (if the list is non-empty). Keeping @@ -1573,13 +1571,16 @@ type lockWaitQueue struct { // C req2 req3 // This is a deadlock caused by the lock table unless req2 partially // breaks the claim at A. - queuedLockingRequests list.List + queuedLockingRequests list.List[*queuedGuard] - // List of *lockTableGuardImpl. All of these are actively waiting. If - // non-empty, the lock must be held. By definition these cannot be in - // waitSelf state since that requests don't conflict with locks held by their - // transaction. - waitingReaders list.List + // waitingReaders is the list of non-locking reads that are actively waiting. + // If this list is non-empty, the key must be locked, as non-locking reads do + // not wait otherwise. + // + // NB: Non-locking readers can never wait in the waitSelf state, because if + // another request from their transaction already holds a lock on the key, + // they are allowed to proceed. + waitingReaders list.List[*lockTableGuardImpl] // If there is a non-empty set of active waiters that are not waitSelf, then // at least one must be distinguished. @@ -1624,7 +1625,7 @@ func (kl *keyLocks) safeFormat(sb *redact.StringBuilder, txnStatusCache *txnStat first := true for e := kl.holders.Front(); e != nil; e = e.Next() { - tl := e.Value.(*txnLock) + tl := e.Value txn, ts := tl.getLockHolder() var prefix string @@ -1673,7 +1674,7 @@ func (kl *keyLocks) safeFormat(sb *redact.StringBuilder, txnStatusCache *txnStat if kl.waitingReaders.Len() > 0 { sb.SafeString(" waiting readers:\n") for e := kl.waitingReaders.Front(); e != nil; e = e.Next() { - g := e.Value.(*lockTableGuardImpl) + g := e.Value sb.Printf(" req: %d, txn: ", redact.Safe(g.seqNum)) if g.txn == nil { sb.SafeString("none\n") @@ -1685,7 +1686,7 @@ func (kl *keyLocks) safeFormat(sb *redact.StringBuilder, txnStatusCache *txnStat if kl.queuedLockingRequests.Len() > 0 { sb.SafeString(" queued locking requests:\n") for e := kl.queuedLockingRequests.Front(); e != nil; e = e.Next() { - qg := e.Value.(*queuedGuard) + qg := e.Value g := qg.guard sb.Printf(" active: %t req: %d, strength: %s, txn: ", redact.Safe(qg.active), redact.Safe(qg.guard.seqNum), redact.Safe(qg.mode.Strength), @@ -1727,7 +1728,7 @@ func (kl *keyLocks) collectLockStateInfo( // See https://github.com/cockroachdb/cockroach/issues/103894. if !includeUncontended && kl.waitingReaders.Len() == 0 && (kl.queuedLockingRequests.Len() == 0 || - (kl.queuedLockingRequests.Len() == 1 && !kl.queuedLockingRequests.Front().Value.(*queuedGuard).active)) { + (kl.queuedLockingRequests.Len() == 1 && !kl.queuedLockingRequests.Front().Value.active)) { return false, roachpb.LockStateInfo{} } @@ -1743,7 +1744,7 @@ func (kl *keyLocks) lockStateInfo(now time.Time) roachpb.LockStateInfo { if kl.isLocked() { // This doesn't work with multiple lock holders. See // https://github.com/cockroachdb/cockroach/issues/109081. - tl := kl.holders.Front().Value.(*txnLock) + tl := kl.holders.Front().Value txnHolder = tl.txn if tl.isHeldReplicated() { durability = lock.Replicated @@ -1755,7 +1756,7 @@ func (kl *keyLocks) lockStateInfo(now time.Time) roachpb.LockStateInfo { // Add waiting readers before locking requests as they should run first. for e := kl.waitingReaders.Front(); e != nil; e = e.Next() { - readerGuard := e.Value.(*lockTableGuardImpl) + readerGuard := e.Value readerGuard.mu.Lock() lockWaiters = append(lockWaiters, lock.Waiter{ WaitingTxn: readerGuard.txnMeta(), @@ -1768,7 +1769,7 @@ func (kl *keyLocks) lockStateInfo(now time.Time) roachpb.LockStateInfo { // Lastly, add queued locking requests, in order. for e := kl.queuedLockingRequests.Front(); e != nil; e = e.Next() { - qg := e.Value.(*queuedGuard) + qg := e.Value g := qg.guard g.mu.Lock() lockWaiters = append(lockWaiters, lock.Waiter{ @@ -1868,7 +1869,7 @@ func (kl *keyLocks) informActiveWaiters() { // for held locks -- they race with other non-transactional writers and // locking requests. assert(state.held, "waiting readers should be empty if the lock isn't held") - g := e.Value.(*lockTableGuardImpl) + g := e.Value if findDistinguished { kl.distinguishedWaiter = g findDistinguished = false @@ -1885,7 +1886,7 @@ func (kl *keyLocks) informActiveWaiters() { g.mu.Unlock() } for e := kl.queuedLockingRequests.Front(); e != nil; e = e.Next() { - qg := e.Value.(*queuedGuard) + qg := e.Value if !qg.active { continue } @@ -1944,12 +1945,12 @@ func (kl *keyLocks) claimantTxn() (_ *enginepb.TxnMeta, held bool) { // necessitates it to change. So we always return the first lock holder, // ensuring all requests consider the same transaction to have claimed a // key. - return kl.holders.Front().Value.(*txnLock).txn, true + return kl.holders.Front().Value.txn, true } if kl.queuedLockingRequests.Len() == 0 { panic("no queued locking request or lock holder; no one should be waiting on the lock") } - qg := kl.queuedLockingRequests.Front().Value.(*queuedGuard) + qg := kl.queuedLockingRequests.Front().Value return qg.guard.txnMeta(), false } @@ -1959,7 +1960,7 @@ func (kl *keyLocks) claimantTxn() (_ *enginepb.TxnMeta, held bool) { // REQUIRES: kl.mu is locked. func (kl *keyLocks) releaseLockingRequestsFromTxn(txn *enginepb.TxnMeta) { for e := kl.queuedLockingRequests.Front(); e != nil; { - qg := e.Value.(*queuedGuard) + qg := e.Value curr := e e = e.Next() g := qg.guard @@ -1984,10 +1985,10 @@ func (kl *keyLocks) tryMakeNewDistinguished() { var g *lockTableGuardImpl claimantTxn, _ := kl.claimantTxn() if kl.waitingReaders.Len() > 0 { - g = kl.waitingReaders.Front().Value.(*lockTableGuardImpl) + g = kl.waitingReaders.Front().Value } else if kl.queuedLockingRequests.Len() > 0 { for e := kl.queuedLockingRequests.Front(); e != nil; e = e.Next() { - qg := e.Value.(*queuedGuard) + qg := e.Value // Only requests actively waiting at this lock should be considered for // the distinguished distinction. if qg.active && !qg.guard.isSameTxn(claimantTxn) { @@ -2059,7 +2060,7 @@ func (kl *keyLocks) isAnyLockHeldReplicated() (bool, *enginepb.TxnMeta) { return false, nil } for e := kl.holders.Front(); e != nil; e = e.Next() { - tl := e.Value.(*txnLock) + tl := e.Value if tl.isHeldReplicated() { return true, tl.txn } @@ -2077,7 +2078,7 @@ func (kl *keyLocks) lockHeldDuration(now time.Time) time.Duration { var minStartTS time.Time // we'll find the lowest timestamp across all locks held on this key for e := kl.holders.Front(); e != nil; e = e.Next() { - tl := e.Value.(*txnLock) + tl := e.Value if minStartTS.IsZero() || tl.startTime.Before(minStartTS) { minStartTS = tl.startTime } @@ -2094,7 +2095,7 @@ func (kl *keyLocks) totalAndMaxWaitDuration(now time.Time) (time.Duration, time. var totalWaitDuration time.Duration var maxWaitDuration time.Duration for e := kl.waitingReaders.Front(); e != nil; e = e.Next() { - g := e.Value.(*lockTableGuardImpl) + g := e.Value g.mu.Lock() waitDuration := now.Sub(g.mu.curLockWaitStart) totalWaitDuration += waitDuration @@ -2104,7 +2105,7 @@ func (kl *keyLocks) totalAndMaxWaitDuration(now time.Time) (time.Duration, time. g.mu.Unlock() } for e := kl.queuedLockingRequests.Front(); e != nil; e = e.Next() { - qg := e.Value.(*queuedGuard) + qg := e.Value g := qg.guard g.mu.Lock() waitDuration := now.Sub(g.mu.curLockWaitStart) @@ -2298,7 +2299,7 @@ func (kl *keyLocks) alreadyHoldsLockAndIsAllowedToProceed( if !found { return false } - tl := e.Value.(*txnLock) + tl := e.Value heldMode := tl.getLockMode() // Check if the lock is already held by the guard's transaction with an equal // or higher lock strength. If it is, we're good to go. Otherwise, the request @@ -2336,7 +2337,7 @@ func (kl *keyLocks) conflictsWithLockHolders(g *lockTableGuardImpl) bool { return false // the lock isn't held; no conflict to speak of } for e := kl.holders.Front(); e != nil; e = e.Next() { - tl := e.Value.(*txnLock) + tl := e.Value lockHolderTxn, _ := tl.getLockHolder() // We should never get here if the lock is already held by another request // from the same transaction with sufficient strength (read: less than or @@ -2483,7 +2484,7 @@ func (kl *keyLocks) enqueueLockingRequest(g *lockTableGuardImpl) (maxQueueLength if _, inQueue := g.mu.locks[kl]; inQueue { // Find the request; it must already be in the correct position. for e := kl.queuedLockingRequests.Front(); e != nil; e = e.Next() { - qqg := e.Value.(*queuedGuard) + qqg := e.Value if qqg.guard == g { qqg.active = true // set the active status as true, in case it wasn't before // Now that this request is actively waiting in the lock's wait queue, @@ -2513,9 +2514,9 @@ func (kl *keyLocks) enqueueLockingRequest(g *lockTableGuardImpl) (maxQueueLength } // The request isn't in the queue. Add it in the correct position, based on // its sequence number. - var e *list.Element + var e *list.Element[*queuedGuard] for e = kl.queuedLockingRequests.Back(); e != nil; e = e.Prev() { - qqg := e.Value.(*queuedGuard) + qqg := e.Value if qqg.guard.seqNum < qg.guard.seqNum { break } @@ -2583,7 +2584,7 @@ func (kl *keyLocks) shouldRequestActivelyWait(g *lockTableGuardImpl) bool { // issue with what we're doing here, as long as the queue is maintained in // sequence number order. for e := kl.queuedLockingRequests.Front(); e != nil; e = e.Next() { - qqg := e.Value.(*queuedGuard) + qqg := e.Value if qqg.guard == g { // We found our request while scanning from the front without finding any // conflicting waiters; no need to actively wait here. @@ -2635,7 +2636,7 @@ func (kl *keyLocks) claimBeforeProceeding(g *lockTableGuardImpl) { // Find the request; iterate from the front, as requests proceeding are more // likely to be closer to the front than the back. for e := kl.queuedLockingRequests.Front(); e != nil; e = e.Next() { - qqg := e.Value.(*queuedGuard) + qqg := e.Value if qqg.guard == g { // If the request was previously marked as a distinguished waiter, and is // now able to claim the lock and proceed, clear the designation. Note @@ -2700,7 +2701,7 @@ func (kl *keyLocks) isNonConflictingLock(g *lockTableGuardImpl) bool { } for e := kl.holders.Front(); e != nil; e = e.Next() { - tl := e.Value.(*txnLock) + tl := e.Value if g.isSameTxn(tl.txn) { // NB: Unlike the pessimistic (normal) evaluation code path, we do // not need to check the lock's strength if it is already held by @@ -2737,7 +2738,7 @@ func (kl *keyLocks) acquireLock(acq *roachpb.LockAcquisition, clock *hlc.Clock) // Already held. e, found := kl.heldBy[acq.Txn.ID] assert(found, "expected to find lock held by the transaction") - tl := e.Value.(*txnLock) + tl := e.Value _, beforeTs := tl.getLockHolder() err := tl.reacquireLock(acq) if err != nil { @@ -2827,7 +2828,7 @@ func (kl *keyLocks) discoveredLock( var tl *txnLock if kl.isLocked() { e, found := kl.heldBy[txn.ID] - tl = e.Value.(*txnLock) + tl = e.Value if !found { return errors.AssertionFailedf( "discovered lock by different transaction (%s) than existing lock (see issue #63592): %s", @@ -2876,9 +2877,9 @@ func (kl *keyLocks) discoveredLock( active: false, } // g is not necessarily first in the queue in the (rare) case (a) above. - var e *list.Element + var e *list.Element[*queuedGuard] for e = kl.queuedLockingRequests.Front(); e != nil; e = e.Next() { - qqg := e.Value.(*queuedGuard) + qqg := e.Value if qqg.guard.seqNum > g.seqNum { break } @@ -2943,8 +2944,7 @@ func (kl *keyLocks) tryClearLock(force bool) bool { // Clear waitingReaders. for e := kl.waitingReaders.Front(); e != nil; { - g := e.Value.(*lockTableGuardImpl) - + g := e.Value curr := e e = e.Next() kl.waitingReaders.Remove(curr) @@ -2958,8 +2958,7 @@ func (kl *keyLocks) tryClearLock(force bool) bool { // Clear queuedLockingRequests. for e := kl.queuedLockingRequests.Front(); e != nil; { - qg := e.Value.(*queuedGuard) - + qg := e.Value curr := e e = e.Next() kl.queuedLockingRequests.Remove(curr) @@ -3013,7 +3012,7 @@ func (kl *keyLocks) tryUpdateLockLocked(up roachpb.LockUpdate) (heldByTxn, gc bo } e := kl.heldBy[up.Txn.ID] - tl := e.Value.(*txnLock) + tl := e.Value txn := &up.Txn ts := up.Txn.WriteTimestamp _, beforeTs := tl.getLockHolder() @@ -3116,7 +3115,7 @@ func (kl *keyLocks) tryUpdateLockLocked(up roachpb.LockUpdate) (heldByTxn, gc bo func (kl *keyLocks) increasedLockTs(newTs hlc.Timestamp) { distinguishedRemoved := false for e := kl.waitingReaders.Front(); e != nil; { - g := e.Value.(*lockTableGuardImpl) + g := e.Value curr := e e = e.Next() if g.ts.Less(newTs) { @@ -3136,8 +3135,8 @@ func (kl *keyLocks) increasedLockTs(newTs hlc.Timestamp) { // waiter or not. // // REQUIRES: kl.mu to be locked. -func (kl *keyLocks) removeLockingRequest(e *list.Element) bool { - qg := e.Value.(*queuedGuard) +func (kl *keyLocks) removeLockingRequest(e *list.Element[*queuedGuard]) bool { + qg := e.Value g := qg.guard kl.queuedLockingRequests.Remove(e) g.mu.Lock() @@ -3157,8 +3156,8 @@ func (kl *keyLocks) removeLockingRequest(e *list.Element) bool { // removeReader removes the reader, referenced by the supplied list.Element, // from the lock's queuedReaders list. Returns whether the reader was the // distinguished waiter or not. -func (kl *keyLocks) removeReader(e *list.Element) bool { - g := e.Value.(*lockTableGuardImpl) +func (kl *keyLocks) removeReader(e *list.Element[*lockTableGuardImpl]) bool { + g := e.Value kl.waitingReaders.Remove(e) g.mu.Lock() delete(g.mu.locks, kl) @@ -3194,7 +3193,7 @@ func (kl *keyLocks) requestDone(g *lockTableGuardImpl) (gc bool) { distinguishedRemoved := false doneRemoval := false for e := kl.queuedLockingRequests.Front(); e != nil; e = e.Next() { - qg := e.Value.(*queuedGuard) + qg := e.Value if qg.guard == g { kl.queuedLockingRequests.Remove(e) if qg.guard == kl.distinguishedWaiter { @@ -3217,7 +3216,7 @@ func (kl *keyLocks) requestDone(g *lockTableGuardImpl) (gc bool) { if !doneRemoval { for e := kl.waitingReaders.Front(); e != nil; e = e.Next() { - gg := e.Value.(*lockTableGuardImpl) + gg := e.Value if gg == g { kl.waitingReaders.Remove(e) if g == kl.distinguishedWaiter { @@ -3334,7 +3333,7 @@ func (kl *keyLocks) maybeReleaseFirstLockingRequest() { // The prefix of the queue that is non-transactional writers is done // waiting. for e := kl.queuedLockingRequests.Front(); e != nil; { - qg := e.Value.(*queuedGuard) + qg := e.Value g := qg.guard if g.txn != nil { // (transactional) locking request break @@ -3352,7 +3351,7 @@ func (kl *keyLocks) maybeReleaseFirstLockingRequest() { // inactive. The call to doneActivelyWaitingAtLock should nudge it to pick up // its scan from where it left off. e := kl.queuedLockingRequests.Front() - qg := e.Value.(*queuedGuard) + qg := e.Value g := qg.guard if qg.active { qg.active = false // mark as inactive @@ -3587,7 +3586,7 @@ func (t *lockTableImpl) AddDiscoveredLock( l.queuedLockingRequests.Init() l.waitingReaders.Init() l.holders.Init() - l.heldBy = make(map[uuid.UUID]*list.Element) + l.heldBy = make(map[uuid.UUID]*list.Element[*txnLock]) t.locks.Set(l) t.locks.numKeysLocked.Add(1) } else { @@ -3656,7 +3655,7 @@ func (t *lockTableImpl) AcquireLock(acq *roachpb.LockAcquisition) error { l.queuedLockingRequests.Init() l.waitingReaders.Init() l.holders.Init() - l.heldBy = make(map[uuid.UUID]*list.Element) + l.heldBy = make(map[uuid.UUID]*list.Element[*txnLock]) t.locks.Set(l) t.locks.numKeysLocked.Add(1) } else { From a4dcc1893e55470e72c33e77c3b8c1c92e43f1a1 Mon Sep 17 00:00:00 2001 From: Arul Ajmani Date: Fri, 25 Aug 2023 12:15:21 -0400 Subject: [PATCH 8/8] concurrency: use atomic.Uint64 to track sequence numbers Burns down a TODO. Epic: none Release note: None --- pkg/kv/kvserver/concurrency/lock_table.go | 21 ++++++++++----------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/pkg/kv/kvserver/concurrency/lock_table.go b/pkg/kv/kvserver/concurrency/lock_table.go index 3eada5528c36..03987cdbaa9d 100644 --- a/pkg/kv/kvserver/concurrency/lock_table.go +++ b/pkg/kv/kvserver/concurrency/lock_table.go @@ -248,8 +248,7 @@ type lockTableImpl struct { // A. // Now in the queues for A and B req1 is behind req3 and vice versa and // this deadlock has been created entirely due to the lock table's behavior. - // TODO(nvanbenschoten): use an atomic.Uint64. - seqNum uint64 + seqNum atomic.Uint64 // locks contains the btree object (wrapped in the treeMu structure) that // contains the actual keyLocks objects. These keyLocks objects represent the @@ -1473,11 +1472,11 @@ type lockWaitQueue struct { // Waiters: An active waiter needs to be notified about changes in who it is // waiting for. - // queuedLockingRequests is of requests queued at a key. They may be waiting - // actively or inactively. The list is maintained in increasing order of - // sequence numbers. This helps ensure some degree of fairness as requests are - // released from the head of the queue. Typically, this happens when all locks - // on the associated key are released. + // queuedLockingRequests is a list of requests queued at a key. They may be + // waiting actively or inactively. The list is maintained in increasing order + // of sequence numbers. This helps ensure some degree of fairness as requests + // are released from the head of the queue. Typically, this happens when all + // locks on the associated key are released. // // When a lock is not held, the head of the list should be comprised of an // inactive, transactional locking request (if the list is non-empty). Keeping @@ -1573,9 +1572,9 @@ type lockWaitQueue struct { // breaks the claim at A. queuedLockingRequests list.List[*queuedGuard] - // waitingReaders is the list of non-locking reads that are actively waiting. - // If this list is non-empty, the key must be locked, as non-locking reads do - // not wait otherwise. + // waitingReaders is a list of non-locking reads that are actively waiting at + // a key. If this list is non-empty, the key must be locked, as non-locking + // reads do not wait otherwise. // // NB: Non-locking readers can never wait in the waitSelf state, because if // another request from their transaction already holds a lock on the key, @@ -3448,7 +3447,7 @@ func (t *lockTableImpl) ScanAndEnqueue(req Request, guard lockTableGuard) lockTa func (t *lockTableImpl) newGuardForReq(req Request) *lockTableGuardImpl { g := newLockTableGuardImpl() - g.seqNum = atomic.AddUint64(&t.seqNum, 1) + g.seqNum = t.seqNum.Add(1) g.lt = t g.txn = req.Txn g.ts = req.Timestamp