From 11a0fad77e045de0ebfac0f8e3f6cccd53626ffd Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Sun, 7 Aug 2022 22:46:58 -0400 Subject: [PATCH 1/8] jobspb,gcjob: rename state enum in anticipation of DelRange We're going to need some new language to capture the states which correspond to ClearRange so we can differentiate them from the DelRange states. Release note: None --- pkg/ccl/testccl/sqlccl/tenant_gc_test.go | 10 +++++----- pkg/jobs/jobspb/jobs.proto | 14 ++++++++++---- pkg/jobs/validate.go | 2 +- pkg/sql/doctor/doctor_test.go | 16 ++++++++-------- pkg/sql/gcjob/gc_job_utils.go | 20 ++++++++++---------- pkg/sql/gcjob/index_garbage_collection.go | 2 +- pkg/sql/gcjob/refresh_statuses.go | 12 ++++++------ pkg/sql/gcjob/table_garbage_collection.go | 2 +- pkg/sql/gcjob/tenant_garbage_collection.go | 12 ++++++------ pkg/sql/gcjob_test/gc_job_test.go | 20 ++++++++++---------- pkg/sql/tenant.go | 2 +- 11 files changed, 59 insertions(+), 53 deletions(-) diff --git a/pkg/ccl/testccl/sqlccl/tenant_gc_test.go b/pkg/ccl/testccl/sqlccl/tenant_gc_test.go index 0b5f123fc91a..a8a41ed87547 100644 --- a/pkg/ccl/testccl/sqlccl/tenant_gc_test.go +++ b/pkg/ccl/testccl/sqlccl/tenant_gc_test.go @@ -120,11 +120,11 @@ func TestGCTenantRemovesSpanConfigs(t *testing.T) { // Run GC on the tenant. progress := &jobspb.SchemaChangeGCProgress{ Tenant: &jobspb.SchemaChangeGCProgress_TenantProgress{ - Status: jobspb.SchemaChangeGCProgress_DELETING, + Status: jobspb.SchemaChangeGCProgress_CLEARING, }, } require.NoError(t, gcClosure(tenantID.ToUint64(), progress)) - require.Equal(t, jobspb.SchemaChangeGCProgress_DELETED, progress.Tenant.Status) + require.Equal(t, jobspb.SchemaChangeGCProgress_CLEARED, progress.Tenant.Status) // Ensure the tenant's span configs and system span configs have been deleted. records, err = scKVAccessor.GetSpanConfigRecords( @@ -287,9 +287,9 @@ func TestGCTableOrIndexWaitsForProtectedTimestamps(t *testing.T) { progress := job.Progress() if gcIndex { - require.Equal(t, jobspb.SchemaChangeGCProgress_DELETED, progress.GetSchemaChangeGC().Indexes[0].Status) + require.Equal(t, jobspb.SchemaChangeGCProgress_CLEARED, progress.GetSchemaChangeGC().Indexes[0].Status) } else { - require.Equal(t, jobspb.SchemaChangeGCProgress_DELETED, progress.GetSchemaChangeGC().Tables[0].Status) + require.Equal(t, jobspb.SchemaChangeGCProgress_CLEARED, progress.GetSchemaChangeGC().Tables[0].Status) } mu.Lock() @@ -513,7 +513,7 @@ func TestGCTenantJobWaitsForProtectedTimestamps(t *testing.T) { _, err = sql.GetTenantRecord(ctx, &execCfg, nil /* txn */, tenID.ToUint64()) require.EqualError(t, err, fmt.Sprintf(`tenant "%d" does not exist`, tenID.ToUint64())) progress := job.Progress() - require.Equal(t, jobspb.SchemaChangeGCProgress_DELETED, progress.GetSchemaChangeGC().Tenant.Status) + require.Equal(t, jobspb.SchemaChangeGCProgress_CLEARED, progress.GetSchemaChangeGC().Tenant.Status) } // PTS record protecting secondary tenant should block tenant GC. diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index 483fe501e632..b9d1f70f9947 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -756,13 +756,19 @@ message SchemaChangeProgress { message SchemaChangeGCProgress { enum Status { - // Waiting for the index/table to expire. - WAITING_FOR_GC = 0; + // Waiting for the index/table to expire before issuing ClearRange + // requests over the table span. + // + // TODO(ajwerner): Remove this in 23.1. + WAITING_FOR_CLEAR = 0; // The GC TTL has expired. This element is marked for imminent deletion - DELETING = 1; + // or is being cleared. + // + // TODO(ajwerner): Remove this in 23.1. + CLEARING = 1; // This element has been deleted. The job is done when all elements are in // this state. - DELETED = 2; + CLEARED = 2; } message IndexProgress { diff --git a/pkg/jobs/validate.go b/pkg/jobs/validate.go index 59ca91ad03c0..79cfa3348603 100644 --- a/pkg/jobs/validate.go +++ b/pkg/jobs/validate.go @@ -110,7 +110,7 @@ func collectDescriptorReferences(j JobMetadata) (ids catalog.DescriptorIDSet) { } case jobspb.TypeSchemaChangeGC: for _, table := range j.Progress.GetSchemaChangeGC().Tables { - if table.Status == jobspb.SchemaChangeGCProgress_DELETED { + if table.Status == jobspb.SchemaChangeGCProgress_CLEARED { continue } ids.Add(table.ID) diff --git a/pkg/sql/doctor/doctor_test.go b/pkg/sql/doctor/doctor_test.go index 60846949e04b..2f8201658796 100644 --- a/pkg/sql/doctor/doctor_test.go +++ b/pkg/sql/doctor/doctor_test.go @@ -570,9 +570,9 @@ func TestExamineJobs(t *testing.T) { Progress: &jobspb.Progress{Details: jobspb.WrapProgressDetails( jobspb.SchemaChangeGCProgress{ Tables: []jobspb.SchemaChangeGCProgress_TableProgress{ - {ID: 1, Status: jobspb.SchemaChangeGCProgress_DELETED}, - {ID: 2, Status: jobspb.SchemaChangeGCProgress_DELETING}, - {ID: 3, Status: jobspb.SchemaChangeGCProgress_WAITING_FOR_GC}, + {ID: 1, Status: jobspb.SchemaChangeGCProgress_CLEARED}, + {ID: 2, Status: jobspb.SchemaChangeGCProgress_CLEARING}, + {ID: 3, Status: jobspb.SchemaChangeGCProgress_WAITING_FOR_CLEAR}, }, })}, Status: jobs.StatusRunning, @@ -583,8 +583,8 @@ func TestExamineJobs(t *testing.T) { Progress: &jobspb.Progress{Details: jobspb.WrapProgressDetails( jobspb.SchemaChangeGCProgress{ Tables: []jobspb.SchemaChangeGCProgress_TableProgress{ - {ID: 1, Status: jobspb.SchemaChangeGCProgress_DELETED}, - {ID: 3, Status: jobspb.SchemaChangeGCProgress_WAITING_FOR_GC}, + {ID: 1, Status: jobspb.SchemaChangeGCProgress_CLEARED}, + {ID: 3, Status: jobspb.SchemaChangeGCProgress_WAITING_FOR_CLEAR}, }, })}, Status: jobs.StatusPauseRequested, @@ -595,11 +595,11 @@ func TestExamineJobs(t *testing.T) { Progress: &jobspb.Progress{Details: jobspb.WrapProgressDetails( jobspb.SchemaChangeGCProgress{ Tables: []jobspb.SchemaChangeGCProgress_TableProgress{ - {ID: 1, Status: jobspb.SchemaChangeGCProgress_DELETED}, - {ID: 3, Status: jobspb.SchemaChangeGCProgress_WAITING_FOR_GC}, + {ID: 1, Status: jobspb.SchemaChangeGCProgress_CLEARED}, + {ID: 3, Status: jobspb.SchemaChangeGCProgress_WAITING_FOR_CLEAR}, }, Indexes: []jobspb.SchemaChangeGCProgress_IndexProgress{ - {IndexID: 10, Status: jobspb.SchemaChangeGCProgress_WAITING_FOR_GC}, + {IndexID: 10, Status: jobspb.SchemaChangeGCProgress_WAITING_FOR_CLEAR}, }, })}, Status: jobs.StatusPaused, diff --git a/pkg/sql/gcjob/gc_job_utils.go b/pkg/sql/gcjob/gc_job_utils.go index 152b91d689e3..7ded5a8b4b32 100644 --- a/pkg/sql/gcjob/gc_job_utils.go +++ b/pkg/sql/gcjob/gc_job_utils.go @@ -32,7 +32,7 @@ func markTableGCed( for i := range progress.Tables { tableProgress := &progress.Tables[i] if tableProgress.ID == tableID { - tableProgress.Status = jobspb.SchemaChangeGCProgress_DELETED + tableProgress.Status = jobspb.SchemaChangeGCProgress_CLEARED if log.V(2) { log.Infof(ctx, "determined table %d is GC'd", tableID) } @@ -50,7 +50,7 @@ func markIndexGCed( for i := range progress.Indexes { indexToUpdate := &progress.Indexes[i] if indexToUpdate.IndexID == garbageCollectedIndexID { - indexToUpdate.Status = jobspb.SchemaChangeGCProgress_DELETED + indexToUpdate.Status = jobspb.SchemaChangeGCProgress_CLEARED log.Infof(ctx, "marked index %d as GC'd", garbageCollectedIndexID) } } @@ -98,7 +98,7 @@ func initializeProgress( var update bool if details.Tenant != nil && progress.Tenant == nil { progress.Tenant = &jobspb.SchemaChangeGCProgress_TenantProgress{ - Status: jobspb.SchemaChangeGCProgress_WAITING_FOR_GC, + Status: jobspb.SchemaChangeGCProgress_WAITING_FOR_CLEAR, } update = true } else if len(progress.Tables) != len(details.Tables) || len(progress.Indexes) != len(details.Indexes) { @@ -128,16 +128,16 @@ func initializeProgress( // Check if we are done GC'ing everything. func isDoneGC(progress *jobspb.SchemaChangeGCProgress) bool { for _, index := range progress.Indexes { - if index.Status != jobspb.SchemaChangeGCProgress_DELETED { + if index.Status != jobspb.SchemaChangeGCProgress_CLEARED { return false } } for _, table := range progress.Tables { - if table.Status != jobspb.SchemaChangeGCProgress_DELETED { + if table.Status != jobspb.SchemaChangeGCProgress_CLEARED { return false } } - if progress.Tenant != nil && progress.Tenant.Status != jobspb.SchemaChangeGCProgress_DELETED { + if progress.Tenant != nil && progress.Tenant.Status != jobspb.SchemaChangeGCProgress_CLEARED { return false } @@ -150,12 +150,12 @@ func runningStatusGC(progress *jobspb.SchemaChangeGCProgress) jobs.RunningStatus tableIDs := make([]string, 0, len(progress.Tables)) indexIDs := make([]string, 0, len(progress.Indexes)) for _, table := range progress.Tables { - if table.Status == jobspb.SchemaChangeGCProgress_DELETING { + if table.Status == jobspb.SchemaChangeGCProgress_CLEARING { tableIDs = append(tableIDs, strconv.Itoa(int(table.ID))) } } for _, index := range progress.Indexes { - if index.Status == jobspb.SchemaChangeGCProgress_DELETING { + if index.Status == jobspb.SchemaChangeGCProgress_CLEARING { indexIDs = append(indexIDs, strconv.Itoa(int(index.IndexID))) } } @@ -163,7 +163,7 @@ func runningStatusGC(progress *jobspb.SchemaChangeGCProgress) jobs.RunningStatus var b strings.Builder b.WriteString("performing garbage collection on") var flag bool - if progress.Tenant != nil && progress.Tenant.Status == jobspb.SchemaChangeGCProgress_DELETING { + if progress.Tenant != nil && progress.Tenant.Status == jobspb.SchemaChangeGCProgress_CLEARING { b.WriteString(" tenant") flag = true } @@ -221,7 +221,7 @@ func getAllTablesWaitingForGC( allRemainingTableIDs = append(allRemainingTableIDs, details.ParentID) } for _, table := range progress.Tables { - if table.Status != jobspb.SchemaChangeGCProgress_DELETED { + if table.Status != jobspb.SchemaChangeGCProgress_CLEARED { allRemainingTableIDs = append(allRemainingTableIDs, table.ID) } } diff --git a/pkg/sql/gcjob/index_garbage_collection.go b/pkg/sql/gcjob/index_garbage_collection.go index f87febbf5e3d..027a3f0ba38d 100644 --- a/pkg/sql/gcjob/index_garbage_collection.go +++ b/pkg/sql/gcjob/index_garbage_collection.go @@ -53,7 +53,7 @@ func gcIndexes( return errors.AssertionFailedf("expected descriptor %d to be a table, not %T", parentID, parentDesc) } for _, index := range droppedIndexes { - if index.Status != jobspb.SchemaChangeGCProgress_DELETING { + if index.Status != jobspb.SchemaChangeGCProgress_CLEARING { continue } diff --git a/pkg/sql/gcjob/refresh_statuses.go b/pkg/sql/gcjob/refresh_statuses.go index fdd028663070..fb7f59d3cc1e 100644 --- a/pkg/sql/gcjob/refresh_statuses.go +++ b/pkg/sql/gcjob/refresh_statuses.go @@ -167,7 +167,7 @@ func updateTableStatus( for i, t := range progress.Tables { droppedTable := &progress.Tables[i] - if droppedTable.ID != table.GetID() || droppedTable.Status == jobspb.SchemaChangeGCProgress_DELETED { + if droppedTable.ID != table.GetID() || droppedTable.Status == jobspb.SchemaChangeGCProgress_CLEARED { continue } @@ -200,7 +200,7 @@ func updateTableStatus( if log.V(2) { log.Infof(ctx, "detected expired table %d", t.ID) } - droppedTable.Status = jobspb.SchemaChangeGCProgress_DELETING + droppedTable.Status = jobspb.SchemaChangeGCProgress_CLEARING } else { if log.V(2) { log.Infof(ctx, "table %d still has %+v until GC", t.ID, lifetime) @@ -231,7 +231,7 @@ func updateIndexesStatus( soonestDeadline = timeutil.Unix(0, int64(math.MaxInt64)) for i := 0; i < len(progress.Indexes); i++ { idxProgress := &progress.Indexes[i] - if idxProgress.Status == jobspb.SchemaChangeGCProgress_DELETED { + if idxProgress.Status == jobspb.SchemaChangeGCProgress_CLEARED { continue } @@ -269,7 +269,7 @@ func updateIndexesStatus( if log.V(2) { log.Infof(ctx, "detected expired index %d from table %d", idxProgress.IndexID, table.GetID()) } - idxProgress.Status = jobspb.SchemaChangeGCProgress_DELETING + idxProgress.Status = jobspb.SchemaChangeGCProgress_CLEARING } else if deadline.Before(soonestDeadline) { soonestDeadline = deadline } @@ -454,7 +454,7 @@ func refreshTenant( details *jobspb.SchemaChangeGCDetails, progress *jobspb.SchemaChangeGCProgress, ) (expired bool, _ time.Time, _ error) { - if progress.Tenant.Status != jobspb.SchemaChangeGCProgress_WAITING_FOR_GC { + if progress.Tenant.Status != jobspb.SchemaChangeGCProgress_WAITING_FOR_CLEAR { return true, time.Time{}, nil } @@ -488,7 +488,7 @@ func refreshTenant( } // At this point, the tenant's keyspace is ready for GC. - progress.Tenant.Status = jobspb.SchemaChangeGCProgress_DELETING + progress.Tenant.Status = jobspb.SchemaChangeGCProgress_CLEARING return true, deadlineUnix, nil } return false, deadlineUnix, nil diff --git a/pkg/sql/gcjob/table_garbage_collection.go b/pkg/sql/gcjob/table_garbage_collection.go index fb5e419de2c2..173557aa2032 100644 --- a/pkg/sql/gcjob/table_garbage_collection.go +++ b/pkg/sql/gcjob/table_garbage_collection.go @@ -40,7 +40,7 @@ func gcTables( log.Infof(ctx, "GC is being considered for tables: %+v", progress.Tables) } for _, droppedTable := range progress.Tables { - if droppedTable.Status != jobspb.SchemaChangeGCProgress_DELETING { + if droppedTable.Status != jobspb.SchemaChangeGCProgress_CLEARING { // Table is not ready to be dropped, or has already been dropped. continue } diff --git a/pkg/sql/gcjob/tenant_garbage_collection.go b/pkg/sql/gcjob/tenant_garbage_collection.go index 7d00a90d2c72..f2054110565c 100644 --- a/pkg/sql/gcjob/tenant_garbage_collection.go +++ b/pkg/sql/gcjob/tenant_garbage_collection.go @@ -34,10 +34,10 @@ func gcTenant( log.Infof(ctx, "GC is being considered for tenant: %d", tenID) } - if progress.Tenant.Status == jobspb.SchemaChangeGCProgress_WAITING_FOR_GC { + if progress.Tenant.Status == jobspb.SchemaChangeGCProgress_WAITING_FOR_CLEAR { return errors.AssertionFailedf( "Tenant id %d is expired and should not be in state %+v", - tenID, jobspb.SchemaChangeGCProgress_WAITING_FOR_GC, + tenID, jobspb.SchemaChangeGCProgress_WAITING_FOR_CLEAR, ) } @@ -46,12 +46,12 @@ func gcTenant( if pgerror.GetPGCode(err) == pgcode.UndefinedObject { // The tenant row is deleted only after its data is cleared so there is // nothing to do in this case but mark the job as done. - if progress.Tenant.Status != jobspb.SchemaChangeGCProgress_DELETED { + if progress.Tenant.Status != jobspb.SchemaChangeGCProgress_CLEARED { // This will happen if the job deletes the tenant row and fails to update // its progress. In this case there's nothing to do but update the job // progress. log.Errorf(ctx, "tenant id %d not found while attempting to GC", tenID) - progress.Tenant.Status = jobspb.SchemaChangeGCProgress_DELETED + progress.Tenant.Status = jobspb.SchemaChangeGCProgress_CLEARED } return nil } @@ -59,7 +59,7 @@ func gcTenant( } // This case should never happen. - if progress.Tenant.Status == jobspb.SchemaChangeGCProgress_DELETED { + if progress.Tenant.Status == jobspb.SchemaChangeGCProgress_CLEARED { return errors.AssertionFailedf("GC state for tenant %+v is DELETED yet the tenant row still exists", info) } @@ -67,6 +67,6 @@ func gcTenant( return errors.Wrapf(err, "gc tenant %d", info.ID) } - progress.Tenant.Status = jobspb.SchemaChangeGCProgress_DELETED + progress.Tenant.Status = jobspb.SchemaChangeGCProgress_CLEARED return nil } diff --git a/pkg/sql/gcjob_test/gc_job_test.go b/pkg/sql/gcjob_test/gc_job_test.go index 81c08812cb5b..b49a9ad77ff4 100644 --- a/pkg/sql/gcjob_test/gc_job_test.go +++ b/pkg/sql/gcjob_test/gc_job_test.go @@ -405,7 +405,7 @@ func TestGCResumer(t *testing.T) { _, err = sql.GetTenantRecord(ctx, &execCfg, nil /* txn */, tenID) require.EqualError(t, err, `tenant "10" does not exist`) progress := job.Progress() - require.Equal(t, jobspb.SchemaChangeGCProgress_DELETED, progress.GetSchemaChangeGC().Tenant.Status) + require.Equal(t, jobspb.SchemaChangeGCProgress_CLEARED, progress.GetSchemaChangeGC().Tenant.Status) }) t.Run("tenant GC job soon", func(t *testing.T) { @@ -433,7 +433,7 @@ func TestGCResumer(t *testing.T) { _, err = sql.GetTenantRecord(ctx, &execCfg, nil /* txn */, tenID) require.EqualError(t, err, `tenant "10" does not exist`) progress := job.Progress() - require.Equal(t, jobspb.SchemaChangeGCProgress_DELETED, progress.GetSchemaChangeGC().Tenant.Status) + require.Equal(t, jobspb.SchemaChangeGCProgress_CLEARED, progress.GetSchemaChangeGC().Tenant.Status) }) t.Run("no tenant and tables in same GC job", func(t *testing.T) { @@ -495,7 +495,7 @@ func TestGCTenant(t *testing.T) { t.Run("unexpected progress state", func(t *testing.T) { progress := &jobspb.SchemaChangeGCProgress{ Tenant: &jobspb.SchemaChangeGCProgress_TenantProgress{ - Status: jobspb.SchemaChangeGCProgress_WAITING_FOR_GC, + Status: jobspb.SchemaChangeGCProgress_WAITING_FOR_CLEAR, }, } require.EqualError( @@ -503,23 +503,23 @@ func TestGCTenant(t *testing.T) { gcClosure(10, progress), "Tenant id 10 is expired and should not be in state WAITING_FOR_GC", ) - require.Equal(t, jobspb.SchemaChangeGCProgress_WAITING_FOR_GC, progress.Tenant.Status) + require.Equal(t, jobspb.SchemaChangeGCProgress_WAITING_FOR_CLEAR, progress.Tenant.Status) }) t.Run("non-existent tenant deleting progress", func(t *testing.T) { progress := &jobspb.SchemaChangeGCProgress{ Tenant: &jobspb.SchemaChangeGCProgress_TenantProgress{ - Status: jobspb.SchemaChangeGCProgress_DELETING, + Status: jobspb.SchemaChangeGCProgress_CLEARING, }, } require.NoError(t, gcClosure(nonexistentTenID, progress)) - require.Equal(t, jobspb.SchemaChangeGCProgress_DELETED, progress.Tenant.Status) + require.Equal(t, jobspb.SchemaChangeGCProgress_CLEARED, progress.Tenant.Status) }) t.Run("existent tenant deleted progress", func(t *testing.T) { progress := &jobspb.SchemaChangeGCProgress{ Tenant: &jobspb.SchemaChangeGCProgress_TenantProgress{ - Status: jobspb.SchemaChangeGCProgress_DELETED, + Status: jobspb.SchemaChangeGCProgress_CLEARED, }, } require.EqualError( @@ -532,7 +532,7 @@ func TestGCTenant(t *testing.T) { t.Run("active tenant GC", func(t *testing.T) { progress := &jobspb.SchemaChangeGCProgress{ Tenant: &jobspb.SchemaChangeGCProgress_TenantProgress{ - Status: jobspb.SchemaChangeGCProgress_DELETING, + Status: jobspb.SchemaChangeGCProgress_CLEARING, }, } require.EqualError( @@ -545,7 +545,7 @@ func TestGCTenant(t *testing.T) { t.Run("drop tenant GC", func(t *testing.T) { progress := &jobspb.SchemaChangeGCProgress{ Tenant: &jobspb.SchemaChangeGCProgress_TenantProgress{ - Status: jobspb.SchemaChangeGCProgress_DELETING, + Status: jobspb.SchemaChangeGCProgress_CLEARING, }, } @@ -561,7 +561,7 @@ func TestGCTenant(t *testing.T) { require.Equal(t, []byte("foo"), val) require.NoError(t, gcClosure(dropTenID, progress)) - require.Equal(t, jobspb.SchemaChangeGCProgress_DELETED, progress.Tenant.Status) + require.Equal(t, jobspb.SchemaChangeGCProgress_CLEARED, progress.Tenant.Status) _, err = sql.GetTenantRecord(ctx, &execCfg, nil /* txn */, dropTenID) require.EqualError(t, err, `tenant "11" does not exist`) require.NoError(t, gcClosure(dropTenID, progress)) diff --git a/pkg/sql/tenant.go b/pkg/sql/tenant.go index 1a25630f6d78..06faa2597346 100644 --- a/pkg/sql/tenant.go +++ b/pkg/sql/tenant.go @@ -522,7 +522,7 @@ func gcTenantJob( progress := jobspb.SchemaChangeGCProgress{} if synchronous { progress.Tenant = &jobspb.SchemaChangeGCProgress_TenantProgress{ - Status: jobspb.SchemaChangeGCProgress_DELETING, + Status: jobspb.SchemaChangeGCProgress_CLEARING, } } gcJobRecord := jobs.Record{ From 8ebdf279aabdd8093822347658ebd52e92c5794b Mon Sep 17 00:00:00 2001 From: Arul Ajmani Date: Tue, 9 Aug 2022 16:51:44 -0400 Subject: [PATCH 2/8] server: add cluster setting to control tenant trace redaction Previously, all operations run on the server on behalf of a secondary tenant would have their logs redacted. This patch introduces a new cluster setting that controls whether logs should be redacted or not. This is only settable by the system tenant and maintains the previous behaviour by default. Release note (ops change): introduce a new cluster setting (`server.secondary_tenants.redact_trace`) which controls if traces should be redacted for ops run on behalf of secondary tenants. --- docs/generated/settings/settings.html | 1 + pkg/server/bench_test.go | 5 +++- pkg/server/node.go | 34 +++++++++++++++++---------- 3 files changed, 27 insertions(+), 13 deletions(-) diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 5eb4b70ec16e..548c78a9aee4 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -86,6 +86,7 @@ server.oidc_authentication.redirect_urlstringhttps://localhost:8080/oidc/v1/callbacksets OIDC redirect URL via a URL string or a JSON string containing a required `redirect_urls` key with an object that maps from region keys to URL strings (URLs should point to your load balancer and must route to the path /oidc/v1/callback) server.oidc_authentication.scopesstringopenidsets OIDC scopes to include with authentication request (space delimited list of strings, required to start with `openid`) server.rangelog.ttlduration720h0m0sif nonzero, range log entries older than this duration are deleted every 10m0s. Should not be lowered below 24 hours. +server.secondary_tenants.redact_trace.enabledbooleantruecontrols if server side traces are redacted for tenant operations server.shutdown.connection_waitduration0sthe maximum amount of time a server waits for all SQL connections to be closed before proceeding with a drain. (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting) server.shutdown.drain_waitduration0sthe amount of time a server waits in an unready state before proceeding with a drain (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting. --drain-wait is to specify the duration of the whole draining process, while server.shutdown.drain_wait is to set the wait time for health probes to notice that the node is not ready.) server.shutdown.lease_transfer_waitduration5sthe timeout for a single iteration of the range lease transfer phase of draining (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting) diff --git a/pkg/server/bench_test.go b/pkg/server/bench_test.go index 5c70170919e3..c653608ab543 100644 --- a/pkg/server/bench_test.go +++ b/pkg/server/bench_test.go @@ -15,6 +15,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/tracing" @@ -57,7 +58,9 @@ func BenchmarkSetupSpanForIncomingRPC(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { - _, sp := setupSpanForIncomingRPC(ctx, roachpb.SystemTenantID, ba, tr) + _, sp := setupSpanForIncomingRPC( + ctx, roachpb.SystemTenantID, ba, tr, cluster.MakeTestingClusterSettings(), + ) sp.finish(ctx, nil /* br */) } }) diff --git a/pkg/server/node.go b/pkg/server/node.go index ea3919de2be4..21c5925b0a27 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -153,6 +153,12 @@ var ( 10*time.Second, settings.NonNegativeDurationWithMaximum(maxGraphiteInterval), ).WithPublic() + redactServerTracesForSecondaryTenants = settings.RegisterBoolSetting( + settings.SystemOnly, + "server.secondary_tenants.redact_trace.enabled", + "controls if server side traces are redacted for tenant operations", + true, + ).WithPublic() ) type nodeMetrics struct { @@ -1097,6 +1103,7 @@ type spanForRequest struct { sp *tracing.Span needRecording bool tenID roachpb.TenantID + settings *cluster.Settings } // finish finishes the span. If the span was recording and br is not nil, the @@ -1114,17 +1121,15 @@ func (sp *spanForRequest) finish(ctx context.Context, br *roachpb.BatchResponse) rec = sp.sp.FinishAndGetConfiguredRecording() if rec != nil { - // Decide if the trace for this RPC, if any, will need to be redacted. It - // needs to be redacted if the response goes to a tenant. In case the request - // is local, then the trace might eventually go to a tenant (and tenID might - // be set), but it will go to the tenant only indirectly, through the response - // of a parent RPC. In that case, that parent RPC is responsible for the - // redaction. + // Decide if the trace for this RPC, if any, will need to be redacted. In + // general, responses sent to a tenant are redacted unless indicated + // otherwise by the cluster setting below. // - // Tenants get a redacted recording, i.e. with anything - // sensitive stripped out of the verbose messages. However, - // structured payloads stay untouched. - needRedaction := sp.tenID != roachpb.SystemTenantID + // Even if the recording sent to a tenant is redacted (anything sensitive + // is stripped out of the verbose messages), structured payloads + // stay untouched. + needRedaction := sp.tenID != roachpb.SystemTenantID && + redactServerTracesForSecondaryTenants.Get(&sp.settings.SV) if needRedaction { if err := redactRecordingForTenant(sp.tenID, rec); err != nil { log.Errorf(ctx, "error redacting trace recording: %s", err) @@ -1151,11 +1156,15 @@ func (sp *spanForRequest) finish(ctx context.Context, br *roachpb.BatchResponse) func (n *Node) setupSpanForIncomingRPC( ctx context.Context, tenID roachpb.TenantID, ba *roachpb.BatchRequest, ) (context.Context, spanForRequest) { - return setupSpanForIncomingRPC(ctx, tenID, ba, n.storeCfg.AmbientCtx.Tracer) + return setupSpanForIncomingRPC(ctx, tenID, ba, n.storeCfg.AmbientCtx.Tracer, n.storeCfg.Settings) } func setupSpanForIncomingRPC( - ctx context.Context, tenID roachpb.TenantID, ba *roachpb.BatchRequest, tr *tracing.Tracer, + ctx context.Context, + tenID roachpb.TenantID, + ba *roachpb.BatchRequest, + tr *tracing.Tracer, + settings *cluster.Settings, ) (context.Context, spanForRequest) { var newSpan *tracing.Span parentSpan := tracing.SpanFromContext(ctx) @@ -1199,6 +1208,7 @@ func setupSpanForIncomingRPC( needRecording: needRecordingCollection, tenID: tenID, sp: newSpan, + settings: settings, } } From a7d5abfc6609d031f7537b3a2727e1d992c21256 Mon Sep 17 00:00:00 2001 From: Arul Ajmani Date: Wed, 10 Aug 2022 11:00:10 -0400 Subject: [PATCH 3/8] kvfollowerreadsccl: skip tests that break when run as secondary tenants We probabilistically run SQL using secondary tenant connections in tests. This breaks TestBoundedStalenessDataDriven and TestFollowerReadsWithStaleDescriptor. Both these tests configure SQL testing knobs to trace statements and then make assertions on the collected traces. As these testing knobs aren't shared by the secondary tenant `TestServer` starts the test simply doesn't work. For now, we simply eschew running these tests with secondary tenants. Release note: None --- pkg/ccl/kvccl/kvfollowerreadsccl/boundedstaleness_test.go | 1 + pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go | 8 ++++++-- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/pkg/ccl/kvccl/kvfollowerreadsccl/boundedstaleness_test.go b/pkg/ccl/kvccl/kvfollowerreadsccl/boundedstaleness_test.go index 2cdfc0a03566..b964abe13128 100644 --- a/pkg/ccl/kvccl/kvfollowerreadsccl/boundedstaleness_test.go +++ b/pkg/ccl/kvccl/kvfollowerreadsccl/boundedstaleness_test.go @@ -273,6 +273,7 @@ func TestBoundedStalenessDataDriven(t *testing.T) { for i := 0; i < numNodes; i++ { i := i clusterArgs.ServerArgsPerNode[i] = base.TestServerArgs{ + DisableDefaultTestTenant: true, Knobs: base.TestingKnobs{ SQLExecutor: &sql.ExecutorTestingKnobs{ WithStatementTrace: func(trace tracingpb.Recording, stmt string) { diff --git a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go index d208aada388d..2c1d8e09b39a 100644 --- a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go +++ b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go @@ -568,13 +568,17 @@ func TestFollowerReadsWithStaleDescriptor(t *testing.T) { tc := testcluster.StartTestCluster(t, 4, base.TestClusterArgs{ ReplicationMode: base.ReplicationManual, - ServerArgs: base.TestServerArgs{UseDatabase: "t"}, + ServerArgs: base.TestServerArgs{ + DisableDefaultTestTenant: true, + UseDatabase: "t", + }, // n4 pretends to have low latency to n2 and n3, so that it tries to use // them for follower reads. // Also, we're going to collect a trace of the test's final query. ServerArgsPerNode: map[int]base.TestServerArgs{ 3: { - UseDatabase: "t", + DisableDefaultTestTenant: true, + UseDatabase: "t", Knobs: base.TestingKnobs{ KVClient: &kvcoord.ClientTestingKnobs{ // Inhibit the checking of connection health done by the From 3d87ddeb7798bfc98302abc549bafcc5a5c429cd Mon Sep 17 00:00:00 2001 From: Arul Ajmani Date: Tue, 9 Aug 2022 17:33:13 -0400 Subject: [PATCH 4/8] kv: ensure secondary tenants route follower reads to the closest replica The dist sender uses node locality information to rank replicas of a range by latency. Previously, this node locality information was read off a node descriptor available in Gossip. Unfortunately, secondary tenants do not have access to Gossip, and as such, would end up randomizing this list of replicas. This manifested itself through unpredictable latencies when running follower reads. We're no longer susceptible to this hazard with this patch. This is done by eschewing the need of a node descriptor from gossip in the DistSender; instead, we now instantiate the DistSender with locality information. However, we do still use Gossip to get the current node's ID when ranking replicas. This is done to ascertain if there is a local replica, and if there is, to always route to it. Unfortunately, because secondary tenants don't have access to Gossip, they can't conform to these semantics. They're susceptible to a hazard where a request may be routed to another replica in the same locality tier as the client even though the client has a local replica as well. This shouldn't be a concern in practice given the diversity heuristic. It also shouldn't be a concern given tenant SQL pods don't run in process with KV nodes today. Resolves #81000 Release note (bug fix): fix an issue where secondary tenants could route follower reads to a random, far away replica instead of one closer. --- pkg/ccl/kvccl/kvfollowerreadsccl/BUILD.bazel | 2 + .../kvfollowerreadsccl/followerreads_test.go | 208 ++++++++++++++++++ pkg/kv/kvclient/kvcoord/dist_sender.go | 21 +- .../kvclient/kvcoord/dist_sender_rangefeed.go | 2 +- pkg/kv/kvclient/kvcoord/dist_sender_test.go | 21 +- pkg/kv/kvclient/kvcoord/replica_slice.go | 24 +- pkg/kv/kvclient/kvcoord/replica_slice_test.go | 22 +- pkg/kv/kvserver/replica_follower_read.go | 3 +- pkg/server/server.go | 1 + pkg/server/tenant.go | 3 +- pkg/server/testserver.go | 5 + pkg/sql/physicalplan/replicaoracle/oracle.go | 4 +- pkg/testutils/serverutils/test_tenant_shim.go | 3 + 13 files changed, 275 insertions(+), 44 deletions(-) diff --git a/pkg/ccl/kvccl/kvfollowerreadsccl/BUILD.bazel b/pkg/ccl/kvccl/kvfollowerreadsccl/BUILD.bazel index fd72ef6ebeab..156bb45eb82b 100644 --- a/pkg/ccl/kvccl/kvfollowerreadsccl/BUILD.bazel +++ b/pkg/ccl/kvccl/kvfollowerreadsccl/BUILD.bazel @@ -45,6 +45,7 @@ go_test( embed = [":kvfollowerreadsccl"], deps = [ "//pkg/base", + "//pkg/ccl/kvccl/kvtenantccl", "//pkg/ccl/utilccl", "//pkg/keys", "//pkg/kv", @@ -57,6 +58,7 @@ go_test( "//pkg/rpc", "//pkg/security/securityassets", "//pkg/security/securitytest", + "//pkg/security/username", "//pkg/server", "//pkg/settings/cluster", "//pkg/sql", diff --git a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go index 2c1d8e09b39a..f3e62ca77407 100644 --- a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go +++ b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go @@ -10,11 +10,16 @@ package kvfollowerreadsccl import ( "context" + gosql "database/sql" + "fmt" "math" + "net/url" "testing" "time" "github.com/cockroachdb/cockroach/pkg/base" + // Blank import kvtenantccl so that we can create a tenant. + _ "github.com/cockroachdb/cockroach/pkg/ccl/kvccl/kvtenantccl" "github.com/cockroachdb/cockroach/pkg/ccl/utilccl" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" @@ -24,10 +29,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" + "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/physicalplan/replicaoracle" "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" @@ -694,3 +701,204 @@ func TestFollowerReadsWithStaleDescriptor(t *testing.T) { require.NoError(t, err) require.Greater(t, followerReadsCountAfter, followerReadsCountBefore) } + +// TestSecondaryTenantFollowerReadsRouting ensures that secondary tenants route +// their requests to the nearest replica. The test runs two versions -- one +// where accurate latency information between nodes is available and another +// where it needs to be estimated using node localities. +func TestSecondaryTenantFollowerReadsRouting(t *testing.T) { + defer leaktest.AfterTest(t)() + defer utilccl.TestingEnableEnterprise()() + + skip.UnderStressRace(t, "times out") + + testutils.RunTrueAndFalse(t, "valid-latency-func", func(t *testing.T, validLatencyFunc bool) { + const numNodes = 4 + + serverArgs := make(map[int]base.TestServerArgs) + localities := make(map[int]roachpb.Locality) + for i := 0; i < numNodes; i++ { + regionName := fmt.Sprintf("region_%d", i) + if i == 3 { + // Make it such that n4 and n2 are in the same region. Below, we'll + // expect a follower read from n4 to be served by n2 because they're + // in the same locality (when validLatencyFunc is false). + regionName = fmt.Sprintf("region_%d", 1) + } + locality := roachpb.Locality{ + Tiers: []roachpb.Tier{{Key: "region", Value: regionName}}, + } + localities[i] = locality + serverArgs[i] = base.TestServerArgs{ + Locality: localities[i], + DisableDefaultTestTenant: true, // we'll create one ourselves below. + } + } + tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgsPerNode: serverArgs, + }) + ctx := context.Background() + defer tc.Stopper().Stop(ctx) + + historicalQuery := `SELECT * FROM t.test AS OF SYSTEM TIME follower_read_timestamp() WHERE k=2` + recCh := make(chan tracingpb.Recording, 1) + + var tenants [numNodes]serverutils.TestTenantInterface + for i := 0; i < numNodes; i++ { + knobs := base.TestingKnobs{} + if i == 3 { // n4 + knobs = base.TestingKnobs{ + KVClient: &kvcoord.ClientTestingKnobs{ + DontConsiderConnHealth: true, + // For the validLatencyFunc=true version of the test, the client + // pretends to have a low latency connection to n2. As a result, we + // expect n2 to be used for follower reads originating from n4. + // + // For the variant where no latency information is available, we + // expect n2 to serve follower reads as well, but because it + // is in the same locality as the client. + LatencyFunc: func(addr string) (time.Duration, bool) { + if !validLatencyFunc { + return 0, false + } + if addr == tc.Server(1).RPCAddr() { + return time.Millisecond, true + } + return 100 * time.Millisecond, true + }, + }, + SQLExecutor: &sql.ExecutorTestingKnobs{ + WithStatementTrace: func(trace tracingpb.Recording, stmt string) { + if stmt == historicalQuery { + recCh <- trace + } + }, + }, + } + } + tt, err := tc.Server(i).StartTenant(ctx, base.TestTenantArgs{ + TenantID: serverutils.TestTenantID(), + Locality: localities[i], + TestingKnobs: knobs, + }) + require.NoError(t, err) + tenants[i] = tt + } + + // Speed up closing of timestamps in order to sleep less below before we can + // use follower_read_timestamp(). Note that we need to override the setting + // for the tenant as well, because the builtin is run in the tenant's sql pod. + systemSQL := sqlutils.MakeSQLRunner(tc.Conns[0]) + systemSQL.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.target_duration = '0.1s'`) + systemSQL.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.side_transport_interval = '0.1s'`) + systemSQL.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.propagation_slack = '0.1s'`) + systemSQL.Exec(t, `ALTER TENANT ALL SET CLUSTER SETTING kv.closed_timestamp.target_duration = '0.1s'`) + systemSQL.Exec(t, `ALTER TENANT ALL SET CLUSTER SETTING kv.closed_timestamp.side_transport_interval = '0.1s'`) + systemSQL.Exec(t, `ALTER TENANT ALL SET CLUSTER SETTING kv.closed_timestamp.propagation_slack = '0.1s'`) + // We're making assertions on traces collected by the tenant using log lines + // in KV so we must ensure they're not redacted. + systemSQL.Exec(t, `SET CLUSTER SETTING server.secondary_tenants.redact_trace.enabled = 'false'`) + + // Wait until all tenant servers are aware of the setting override. + testutils.SucceedsSoon(t, func() error { + settingNames := []string{ + "kv.closed_timestamp.target_duration", "kv.closed_timestamp.side_transport_interval", "kv.closed_timestamp.propagation_slack", + } + for _, settingName := range settingNames { + for i := 0; i < numNodes; i++ { + pgURL, cleanup := sqlutils.PGUrl(t, tenants[i].SQLAddr(), "Tenant", url.User(username.RootUser)) + defer cleanup() + db, err := gosql.Open("postgres", pgURL.String()) + if err != nil { + t.Fatal(err) + } + defer db.Close() + + var val string + err = db.QueryRow( + fmt.Sprintf("SHOW CLUSTER SETTING %s", settingName), + ).Scan(&val) + require.NoError(t, err) + if val != "00:00:00.1" { + return errors.Errorf("tenant server %d is still waiting for %s update: currently %s", + i, + settingName, + val, + ) + } + } + } + return nil + }) + + pgURL, cleanupPGUrl := sqlutils.PGUrl( + t, tenants[3].SQLAddr(), "Tenant", url.User(username.RootUser), + ) + defer cleanupPGUrl() + tenantSQLDB, err := gosql.Open("postgres", pgURL.String()) + require.NoError(t, err) + defer tenantSQLDB.Close() + tenantSQL := sqlutils.MakeSQLRunner(tenantSQLDB) + + tenantSQL.Exec(t, `CREATE DATABASE t`) + tenantSQL.Exec(t, `CREATE TABLE t.test (k INT PRIMARY KEY)`) + + startKey := keys.MakeSQLCodec(serverutils.TestTenantID()).TenantPrefix() + tc.AddVotersOrFatal(t, startKey, tc.Target(1), tc.Target(2)) + desc := tc.LookupRangeOrFatal(t, startKey) + require.Equal(t, []roachpb.ReplicaDescriptor{ + {NodeID: 1, StoreID: 1, ReplicaID: 1}, + {NodeID: 2, StoreID: 2, ReplicaID: 2}, + {NodeID: 3, StoreID: 3, ReplicaID: 3}, + }, desc.Replicas().Descriptors()) + + // Sleep so that we can perform follower reads. The read timestamp needs to be + // above the timestamp when the table was created. + log.Infof(ctx, "test sleeping for the follower read timestamps to pass the table creation timestamp...") + time.Sleep(500 * time.Millisecond) + log.Infof(ctx, "test sleeping... done") + + getFollowerReadCounts := func() [numNodes]int64 { + var counts [numNodes]int64 + for i := range tc.Servers { + err := tc.Servers[i].Stores().VisitStores(func(s *kvserver.Store) error { + counts[i] = s.Metrics().FollowerReadsCount.Count() + return nil + }) + require.NoError(t, err) + } + return counts + } + + // Check that the cache was indeed populated. + tenantSQL.Exec(t, `SELECT * FROM t.test WHERE k = 1`) + tablePrefix := keys.MustAddr(keys.MakeSQLCodec(serverutils.TestTenantID()).TenantPrefix()) + cache := tenants[3].DistSenderI().(*kvcoord.DistSender).RangeDescriptorCache() + entry := cache.GetCached(ctx, tablePrefix, false /* inverted */) + require.NotNil(t, entry) + require.False(t, entry.Lease().Empty()) + require.Equal(t, roachpb.StoreID(1), entry.Lease().Replica.StoreID) + require.Equal(t, []roachpb.ReplicaDescriptor{ + {NodeID: 1, StoreID: 1, ReplicaID: 1}, + {NodeID: 2, StoreID: 2, ReplicaID: 2}, + {NodeID: 3, StoreID: 3, ReplicaID: 3}, + }, entry.Desc().Replicas().Descriptors()) + + followerReadCountsBefore := getFollowerReadCounts() + tenantSQL.Exec(t, historicalQuery) + followerReadsCountsAfter := getFollowerReadCounts() + + rec := <-recCh + // Look at the trace and check that we've served a follower read. + require.True(t, kv.OnlyFollowerReads(rec), "query was served through follower reads: %s", rec) + + for i := 0; i < numNodes; i++ { + if i == 1 { // n2 + require.Greater(t, followerReadsCountsAfter[i], followerReadCountsBefore[i]) + continue + } + require.Equal(t, followerReadsCountsAfter[i], followerReadCountsBefore[i]) + } + }) +} diff --git a/pkg/kv/kvclient/kvcoord/dist_sender.go b/pkg/kv/kvclient/kvcoord/dist_sender.go index 23d873e9eadd..13a09ecdb6f8 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender.go @@ -336,6 +336,11 @@ type DistSender struct { // LatencyFunc is used to estimate the latency to other nodes. latencyFunc LatencyFunc + // locality is the description of the topography of the server on which the + // DistSender is running. It is used to estimate the latency to other nodes + // in the absence of a latency function. + locality roachpb.Locality + // If set, the DistSender will try the replicas in the order they appear in // the descriptor, instead of trying to reorder them by latency. The knob // only applies to requests sent with the LEASEHOLDER routing policy. @@ -386,6 +391,10 @@ type DistSenderConfig struct { FirstRangeProvider FirstRangeProvider RangeDescriptorDB rangecache.RangeDescriptorDB + // Locality is the description of the topography of the server on which the + // DistSender is running. + Locality roachpb.Locality + // KVInterceptor is set for tenants; when set, information about all // BatchRequests and BatchResponses are passed through this interceptor, which // can potentially throttle requests. @@ -405,6 +414,7 @@ func NewDistSender(cfg DistSenderConfig) *DistSender { nodeDescs: cfg.NodeDescs, metrics: makeDistSenderMetrics(), kvInterceptor: cfg.KVInterceptor, + locality: cfg.Locality, } if ds.st == nil { ds.st = cluster.MakeTestingClusterSettings() @@ -545,7 +555,12 @@ func (ds *DistSender) FirstRange() (*roachpb.RangeDescriptor, error) { // getNodeID attempts to return the local node ID. It returns 0 if the DistSender // does not have access to the Gossip network. func (ds *DistSender) getNodeID() roachpb.NodeID { - // TODO(nvanbenschoten): open an issue about the effect of this. + // Today, secondary tenants don't run in process with KV instances, so they + // don't have access to the Gossip network. The DistSender uses the node ID to + // preferentially route requests to a local replica (if one exists). Not + // knowing the node ID, and thus not being able to take advantage of this + // optimization is okay, given tenants not running in-process with KV + // instances have no such optimization to take advantage of to begin with. g, ok := ds.nodeDescs.(*gossip.Gossip) if !ok { return 0 @@ -1968,7 +1983,7 @@ func (ds *DistSender) sendToReplicas( // First order by latency, then move the leaseholder to the front of the // list, if it is known. if !ds.dontReorderReplicas { - replicas.OptimizeReplicaOrder(ds.getNodeDescriptor(), ds.latencyFunc) + replicas.OptimizeReplicaOrder(ds.getNodeID(), ds.latencyFunc, ds.locality) } idx := -1 @@ -1987,7 +2002,7 @@ func (ds *DistSender) sendToReplicas( case roachpb.RoutingPolicy_NEAREST: // Order by latency. log.VEvent(ctx, 2, "routing to nearest replica; leaseholder not required") - replicas.OptimizeReplicaOrder(ds.getNodeDescriptor(), ds.latencyFunc) + replicas.OptimizeReplicaOrder(ds.getNodeID(), ds.latencyFunc, ds.locality) default: log.Fatalf(ctx, "unknown routing policy: %s", ba.RoutingPolicy) diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go b/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go index 5106f5e209aa..3f09a3334424 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go @@ -431,7 +431,7 @@ func (ds *DistSender) singleRangeFeed( if err != nil { return args.Timestamp, err } - replicas.OptimizeReplicaOrder(ds.getNodeDescriptor(), latencyFn) + replicas.OptimizeReplicaOrder(ds.getNodeID(), latencyFn, ds.locality) // The RangeFeed is not used for system critical traffic so use a DefaultClass // connection regardless of the range. opts := SendOptions{class: connectionClass(&ds.st.SV)} diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_test.go b/pkg/kv/kvclient/kvcoord/dist_sender_test.go index f2b072966f22..aeceb8c5cfd3 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_test.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_test.go @@ -389,25 +389,15 @@ func TestSendRPCOrder(t *testing.T) { Settings: cluster.MakeTestingClusterSettings(), } - ds := NewDistSender(cfg) - for _, tc := range testCases { t.Run(tc.name, func(t *testing.T) { verifyCall = makeVerifier(tc.expReplica) - { - // The local node needs to get its attributes during sendRPC. - nd := &roachpb.NodeDescriptor{ - NodeID: 6, - Address: util.MakeUnresolvedAddr("tcp", "invalid.invalid:6"), - Locality: roachpb.Locality{ - Tiers: tc.tiers, - }, - } - g.NodeID.Reset(nd.NodeID) - err := g.SetNodeDescriptor(nd) - require.NoError(t, err) + g.NodeID.Reset(6) + cfg.Locality = roachpb.Locality{ + Tiers: tc.tiers, } + ds := NewDistSender(cfg) ds.rangeCache.Clear() var lease roachpb.Lease @@ -419,9 +409,6 @@ func TestSendRPCOrder(t *testing.T) { Lease: lease, }) - // Kill the cached NodeDescriptor, enforcing a lookup from Gossip. - ds.nodeDescriptor = nil - // Issue the request. header := roachpb.Header{ RangeID: rangeID, // Not used in this test, but why not. diff --git a/pkg/kv/kvclient/kvcoord/replica_slice.go b/pkg/kv/kvclient/kvcoord/replica_slice.go index fc3e010ffdb1..2213babcb132 100644 --- a/pkg/kv/kvclient/kvcoord/replica_slice.go +++ b/pkg/kv/kvclient/kvcoord/replica_slice.go @@ -188,21 +188,23 @@ type LatencyFunc func(string) (time.Duration, bool) // they're to be used for sending RPCs (meaning in the order in which // they'll be probed for the lease). Lower latency and "closer" // (matching in more attributes) replicas are ordered first. If the -// current node is a replica, then it'll be the first one. +// current node has a replica (and the current node's ID is supplied) +// then it'll be the first one. // -// nodeDesc is the descriptor of the current node. It can be nil, in -// which case information about the current descriptor is not used in -// optimizing the order. +// nodeID is the ID of the current node the current node. It can be 0, in which +// case information about the current node is not used in optimizing the order. +// Similarly, latencyFn can be nil, in which case it will not be used. // // Note that this method is not concerned with any information the // node might have about who the lease holder might be. If the // leaseholder is known by the caller, the caller will move it to the // front if appropriate. func (rs ReplicaSlice) OptimizeReplicaOrder( - nodeDesc *roachpb.NodeDescriptor, latencyFn LatencyFunc, + nodeID roachpb.NodeID, latencyFn LatencyFunc, locality roachpb.Locality, ) { - // If we don't know which node we're on, send the RPCs randomly. - if nodeDesc == nil { + // If we don't know which node we're on or its locality, and we don't have + // latency information to other nodes, send the RPCs randomly. + if nodeID == 0 && latencyFn == nil && len(locality.Tiers) == 0 { shuffle.Shuffle(rs) return } @@ -214,10 +216,10 @@ func (rs ReplicaSlice) OptimizeReplicaOrder( return false // i == j } // Replicas on the local node sort first. - if rs[i].NodeID == nodeDesc.NodeID { + if rs[i].NodeID == nodeID { return true // i < j } - if rs[j].NodeID == nodeDesc.NodeID { + if rs[j].NodeID == nodeID { return false // j < i } @@ -228,8 +230,8 @@ func (rs ReplicaSlice) OptimizeReplicaOrder( return latencyI < latencyJ } } - attrMatchI := localityMatch(nodeDesc.Locality.Tiers, rs[i].locality()) - attrMatchJ := localityMatch(nodeDesc.Locality.Tiers, rs[j].locality()) + attrMatchI := localityMatch(locality.Tiers, rs[i].locality()) + attrMatchJ := localityMatch(locality.Tiers, rs[j].locality()) // Longer locality matches sort first (the assumption is that // they'll have better latencies). return attrMatchI > attrMatchJ diff --git a/pkg/kv/kvclient/kvcoord/replica_slice_test.go b/pkg/kv/kvclient/kvcoord/replica_slice_test.go index c234edc69cc5..fc385b245a1d 100644 --- a/pkg/kv/kvclient/kvcoord/replica_slice_test.go +++ b/pkg/kv/kvclient/kvcoord/replica_slice_test.go @@ -175,7 +175,10 @@ func TestReplicaSliceOptimizeReplicaOrder(t *testing.T) { defer log.Scope(t).Close(t) testCases := []struct { name string - node *roachpb.NodeDescriptor + // nodeID of the DistSender. + nodeID roachpb.NodeID + // locality of the DistSender. + locality roachpb.Locality // map from node address (see nodeDesc()) to latency to that node. latencies map[string]time.Duration slice ReplicaSlice @@ -186,8 +189,9 @@ func TestReplicaSliceOptimizeReplicaOrder(t *testing.T) { expOrdered []roachpb.NodeID }{ { - name: "order by locality matching", - node: nodeDesc(t, 1, []string{"country=us", "region=west", "city=la"}), + name: "order by locality matching", + nodeID: 1, + locality: locality(t, []string{"country=us", "region=west", "city=la"}), slice: ReplicaSlice{ info(t, 1, 1, []string{"country=us", "region=west", "city=la"}), info(t, 2, 2, []string{"country=us", "region=west", "city=sf"}), @@ -198,8 +202,9 @@ func TestReplicaSliceOptimizeReplicaOrder(t *testing.T) { expOrdered: []roachpb.NodeID{1, 2, 4, 3}, }, { - name: "order by latency", - node: nodeDesc(t, 1, []string{"country=us", "region=west", "city=la"}), + name: "order by latency", + nodeID: 1, + locality: locality(t, []string{"country=us", "region=west", "city=la"}), latencies: map[string]time.Duration{ "2:26257": time.Hour, "3:26257": time.Minute, @@ -217,8 +222,9 @@ func TestReplicaSliceOptimizeReplicaOrder(t *testing.T) { // Test that replicas on the local node sort first, regardless of factors // like their latency measurement (in production they won't have any // latency measurement). - name: "local node comes first", - node: nodeDesc(t, 1, nil), + name: "local node comes first", + nodeID: 1, + locality: locality(t, nil), latencies: map[string]time.Duration{ "1:26257": 10 * time.Hour, "2:26257": time.Hour, @@ -246,7 +252,7 @@ func TestReplicaSliceOptimizeReplicaOrder(t *testing.T) { } // Randomize the input order, as it's not supposed to matter. shuffle.Shuffle(test.slice) - test.slice.OptimizeReplicaOrder(test.node, latencyFn) + test.slice.OptimizeReplicaOrder(test.nodeID, latencyFn, test.locality) var sortedNodes []roachpb.NodeID sortedNodes = append(sortedNodes, test.slice[0].NodeID) for i := 1; i < len(test.slice); i++ { diff --git a/pkg/kv/kvserver/replica_follower_read.go b/pkg/kv/kvserver/replica_follower_read.go index b5a952c7bd18..582595d429dc 100644 --- a/pkg/kv/kvserver/replica_follower_read.go +++ b/pkg/kv/kvserver/replica_follower_read.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/redact" ) // FollowerReadsEnabled controls whether replicas attempt to serve follower @@ -101,7 +102,7 @@ func (r *Replica) canServeFollowerReadRLocked(ctx context.Context, ba *roachpb.B // // TODO(tschottdorf): once a read for a timestamp T has been served, the replica may // serve reads for that and smaller timestamps forever. - log.Eventf(ctx, "%s; query timestamp below closed timestamp by %s", kvbase.FollowerReadServingMsg, -tsDiff) + log.Eventf(ctx, "%s; query timestamp below closed timestamp by %s", redact.Safe(kvbase.FollowerReadServingMsg), -tsDiff) r.store.metrics.FollowerReadsCount.Inc(1) return true } diff --git a/pkg/server/server.go b/pkg/server/server.go index 08afc206010b..dcf6a0a7ed10 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -365,6 +365,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { RPCRetryOptions: &retryOpts, NodeDialer: nodeDialer, FirstRangeProvider: g, + Locality: cfg.Locality, TestingKnobs: clientTestingKnobs, } distSender := kvcoord.NewDistSender(distSenderCfg) diff --git a/pkg/server/tenant.go b/pkg/server/tenant.go index 81583955c5ff..51860f378430 100644 --- a/pkg/server/tenant.go +++ b/pkg/server/tenant.go @@ -417,7 +417,7 @@ func makeTenantSQLServerArgs( }) var dsKnobs kvcoord.ClientTestingKnobs - if dsKnobsP, ok := baseCfg.TestingKnobs.DistSQL.(*kvcoord.ClientTestingKnobs); ok { + if dsKnobsP, ok := baseCfg.TestingKnobs.KVClient.(*kvcoord.ClientTestingKnobs); ok { dsKnobs = *dsKnobsP } rpcRetryOptions := base.DefaultRetryOptions() @@ -455,6 +455,7 @@ func makeTenantSQLServerArgs( RPCContext: rpcContext, NodeDialer: nodeDialer, RangeDescriptorDB: tenantConnect, + Locality: baseCfg.Locality, KVInterceptor: costController, TestingKnobs: dsKnobs, } diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index 7a61ba1b556e..504d86af6669 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -658,6 +658,11 @@ func (t *TestTenant) DistSQLServer() interface{} { return t.SQLServer.distSQLServer } +// DistSenderI is part of the TestTenantInterface. +func (t *TestTenant) DistSenderI() interface{} { + return t.SQLServer.execCfg.DistSender +} + // RPCContext is part of TestTenantInterface. func (t *TestTenant) RPCContext() *rpc.Context { return t.execCfg.RPCContext diff --git a/pkg/sql/physicalplan/replicaoracle/oracle.go b/pkg/sql/physicalplan/replicaoracle/oracle.go index 0523ce1402ec..b2817c87c778 100644 --- a/pkg/sql/physicalplan/replicaoracle/oracle.go +++ b/pkg/sql/physicalplan/replicaoracle/oracle.go @@ -175,7 +175,7 @@ func (o *closestOracle) ChoosePreferredReplica( if err != nil { return roachpb.ReplicaDescriptor{}, err } - replicas.OptimizeReplicaOrder(&o.nodeDesc, o.latencyFunc) + replicas.OptimizeReplicaOrder(o.nodeDesc.NodeID, o.latencyFunc, o.nodeDesc.Locality) return replicas[0].ReplicaDescriptor, nil } @@ -229,7 +229,7 @@ func (o *binPackingOracle) ChoosePreferredReplica( if err != nil { return roachpb.ReplicaDescriptor{}, err } - replicas.OptimizeReplicaOrder(&o.nodeDesc, o.latencyFunc) + replicas.OptimizeReplicaOrder(o.nodeDesc.NodeID, o.latencyFunc, o.nodeDesc.Locality) // Look for a replica that has been assigned some ranges, but it's not yet full. minLoad := int(math.MaxInt32) diff --git a/pkg/testutils/serverutils/test_tenant_shim.go b/pkg/testutils/serverutils/test_tenant_shim.go index 9691994ec067..dfb88bfb9473 100644 --- a/pkg/testutils/serverutils/test_tenant_shim.go +++ b/pkg/testutils/serverutils/test_tenant_shim.go @@ -64,6 +64,9 @@ type TestTenantInterface interface { // DistSQLServer returns the *distsql.ServerImpl as an interface{}. DistSQLServer() interface{} + // DistSenderI returns the *kvcoord.DistSender as an interface{}. + DistSenderI() interface{} + // JobRegistry returns the *jobs.Registry as an interface{}. JobRegistry() interface{} From befd3405399811d63a019f7fd19c9408b671b70d Mon Sep 17 00:00:00 2001 From: Arul Ajmani Date: Wed, 10 Aug 2022 12:41:52 -0400 Subject: [PATCH 5/8] kvcoord: remove ds.getNodeDescriptor This function is no longer used by the DistSender to optimize replica order. Release note: None --- pkg/kv/kvclient/kvcoord/dist_sender.go | 34 --------------------- pkg/kv/kvclient/kvcoord/dist_sender_test.go | 33 -------------------- 2 files changed, 67 deletions(-) diff --git a/pkg/kv/kvclient/kvcoord/dist_sender.go b/pkg/kv/kvclient/kvcoord/dist_sender.go index 13a09ecdb6f8..9bead2e18321 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender.go @@ -568,40 +568,6 @@ func (ds *DistSender) getNodeID() roachpb.NodeID { return g.NodeID.Get() } -// getNodeDescriptor returns ds.nodeDescriptor, but makes an attempt to load -// it from the Gossip network if a nil value is found. -// We must jump through hoops here to get the node descriptor because it's not available -// until after the node has joined the gossip network and been allowed to initialize -// its stores. -func (ds *DistSender) getNodeDescriptor() *roachpb.NodeDescriptor { - if desc := atomic.LoadPointer(&ds.nodeDescriptor); desc != nil { - return (*roachpb.NodeDescriptor)(desc) - } - // TODO(nvanbenschoten): open an issue about the effect of this. - g, ok := ds.nodeDescs.(*gossip.Gossip) - if !ok { - return nil - } - - ownNodeID := g.NodeID.Get() - if ownNodeID > 0 { - // TODO(tschottdorf): Consider instead adding the NodeID of the - // coordinator to the header, so we can get this from incoming - // requests. Just in case we want to mostly eliminate gossip here. - nodeDesc := &roachpb.NodeDescriptor{} - if err := g.GetInfoProto(gossip.MakeNodeIDKey(ownNodeID), nodeDesc); err == nil { - atomic.StorePointer(&ds.nodeDescriptor, unsafe.Pointer(nodeDesc)) - return nodeDesc - } - } - if log.V(1) { - ctx := ds.AnnotateCtx(context.TODO()) - log.Infof(ctx, "unable to determine this node's attributes for replica "+ - "selection; node is most likely bootstrapping") - } - return nil -} - // CountRanges returns the number of ranges that encompass the given key span. func (ds *DistSender) CountRanges(ctx context.Context, rs roachpb.RSpan) (int64, error) { var count int64 diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_test.go b/pkg/kv/kvclient/kvcoord/dist_sender_test.go index aeceb8c5cfd3..c23533138a12 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_test.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_test.go @@ -2266,39 +2266,6 @@ func TestSendRPCRangeNotFoundError(t *testing.T) { require.Equal(t, leaseholderStoreID, rng.Lease().Replica.StoreID) } -// TestGetNodeDescriptor checks that the Node descriptor automatically gets -// looked up from Gossip. -func TestGetNodeDescriptor(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - ctx := context.Background() - stopper := stop.NewStopper() - defer stopper.Stop(ctx) - - clock := hlc.NewClockWithSystemTimeSource(time.Nanosecond /* maxOffset */) - rpcContext := rpc.NewInsecureTestingContext(ctx, clock, stopper) - g := makeGossip(t, stopper, rpcContext) - ds := NewDistSender(DistSenderConfig{ - AmbientCtx: log.MakeTestingAmbientCtxWithNewTracer(), - Clock: clock, - NodeDescs: g, - RPCContext: rpcContext, - FirstRangeProvider: g, - Settings: cluster.MakeTestingClusterSettings(), - }) - g.NodeID.Reset(5) - if err := g.SetNodeDescriptor(newNodeDesc(5)); err != nil { - t.Fatal(err) - } - testutils.SucceedsSoon(t, func() error { - desc := ds.getNodeDescriptor() - if desc != nil && desc.NodeID == 5 { - return nil - } - return errors.Errorf("wanted NodeID 5, got %v", desc) - }) -} - func TestMultiRangeGapReverse(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) From 0e08303fb15ab6c3efd0711c5daba1b725ecfd57 Mon Sep 17 00:00:00 2001 From: Arul Ajmani Date: Wed, 10 Aug 2022 13:19:11 -0400 Subject: [PATCH 6/8] sql: use locality information when initializing span resolver oracles Previously, we used a node descriptor for locality information about the current node when doing dist SQL planning. This only worked for the system tenant as secondary tenant's used a fake node descriptor. Given the changes to `OptimizeReplicaOrder` in the prior commit, this patch switches to threading in the tenant pod's locality to allow secondary tenant's to make use of this information in dist sql planning. Release note: None --- pkg/server/node.go | 11 +-- pkg/server/server.go | 6 +- pkg/server/server_sql.go | 2 +- pkg/server/tenant.go | 4 +- pkg/sql/distsql_physical_planner.go | 23 +++-- .../physicalplan/replicaoracle/BUILD.bazel | 2 + pkg/sql/physicalplan/replicaoracle/oracle.go | 35 +++++--- .../physicalplan/replicaoracle/oracle_test.go | 85 ++++++++++++------- pkg/sql/physicalplan/span_resolver.go | 10 +-- pkg/sql/physicalplan/span_resolver_test.go | 9 +- 10 files changed, 112 insertions(+), 75 deletions(-) diff --git a/pkg/server/node.go b/pkg/server/node.go index 21c5925b0a27..05542ada7708 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -413,9 +413,7 @@ func (n *Node) AnnotateCtxWithSpan( // start starts the node by registering the storage instance for the RPC // service "Node" and initializing stores for each specified engine. -// Launches periodic store gossiping in a goroutine. A callback can -// be optionally provided that will be invoked once this node's -// NodeDescriptor is available, to help bootstrapping. +// Launches periodic store gossiping in a goroutine. // // addr, sqlAddr, and httpAddr are used to populate the Address, // SQLAddress, and HTTPAddress fields respectively of the @@ -432,7 +430,6 @@ func (n *Node) start( attrs roachpb.Attributes, locality roachpb.Locality, localityAddress []roachpb.LocalityAddress, - nodeDescriptorCallback func(descriptor roachpb.NodeDescriptor), ) error { n.initialStart = initialStart n.startedAt = n.storeCfg.Clock.Now().WallTime @@ -449,12 +446,6 @@ func (n *Node) start( StartedAt: n.startedAt, HTTPAddress: util.MakeUnresolvedAddr(httpAddr.Network(), httpAddr.String()), } - // Invoke any passed in nodeDescriptorCallback as soon as it's available, to - // ensure that other components (currently the DistSQLPlanner) are initialized - // before store startup continues. - if nodeDescriptorCallback != nil { - nodeDescriptorCallback(n.Descriptor) - } // Gossip the node descriptor to make this node addressable by node ID. n.storeCfg.Gossip.NodeID.Set(ctx, n.Descriptor.NodeID) diff --git a/pkg/server/server.go b/pkg/server/server.go index dcf6a0a7ed10..ebdecf98ca4e 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -1302,6 +1302,11 @@ func (s *Server) PreStart(ctx context.Context) error { s.rpcContext.StorageClusterID.Set(ctx, state.clusterID) s.rpcContext.NodeID.Set(ctx, state.nodeID) + // Ensure components in the DistSQLPlanner that rely on the node ID are + // initialized before store startup continues. + s.sqlServer.execCfg.DistSQLPlanner.SetGatewaySQLInstanceID(base.SQLInstanceID(state.nodeID)) + s.sqlServer.execCfg.DistSQLPlanner.ConstructAndSetSpanResolver(ctx, state.nodeID, s.cfg.Locality) + // TODO(irfansharif): Now that we have our node ID, we should run another // check here to make sure we've not been decommissioned away (if we're here // following a server restart). See the discussions in #48843 for how that @@ -1393,7 +1398,6 @@ func (s *Server) PreStart(ctx context.Context) error { s.cfg.NodeAttributes, s.cfg.Locality, s.cfg.LocalityAddresses, - s.sqlServer.execCfg.DistSQLPlanner.SetSQLInstanceInfo, ); err != nil { return err } diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 7db4d1adc5e0..33a28d4205f4 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -1212,8 +1212,8 @@ func (s *SQLServer) setInstanceID(ctx context.Context) error { if err != nil { return err } + s.execCfg.DistSQLPlanner.SetGatewaySQLInstanceID(instanceID) s.sqlLivenessSessionID = sessionID - s.execCfg.DistSQLPlanner.SetSQLInstanceInfo(roachpb.NodeDescriptor{NodeID: roachpb.NodeID(instanceID)}) return nil } diff --git a/pkg/server/tenant.go b/pkg/server/tenant.go index 51860f378430..8d469a86323d 100644 --- a/pkg/server/tenant.go +++ b/pkg/server/tenant.go @@ -244,9 +244,7 @@ func startTenantInternal( tenantAdminServer := newTenantAdminServer(baseCfg.AmbientCtx, s, tenantStatusServer, drainServer) - // TODO(asubiotto): remove this. Right now it is needed to initialize the - // SpanResolver. - s.execCfg.DistSQLPlanner.SetSQLInstanceInfo(roachpb.NodeDescriptor{NodeID: 0}) + s.execCfg.DistSQLPlanner.ConstructAndSetSpanResolver(ctx, 0 /* NodeID */, s.execCfg.Locality) authServer = newAuthenticationServer(baseCfg.Config, s) diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index 12a3cd8787b6..e6fc9287070d 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -223,15 +223,22 @@ func (dsp *DistSQLPlanner) GetSQLInstanceInfo( return dsp.nodeDescs.GetNodeDescriptor(roachpb.NodeID(sqlInstanceID)) } -// SetSQLInstanceInfo sets the planner's node descriptor. -// The first call to SetSQLInstanceInfo leads to the construction of the SpanResolver. -func (dsp *DistSQLPlanner) SetSQLInstanceInfo(desc roachpb.NodeDescriptor) { - dsp.gatewaySQLInstanceID = base.SQLInstanceID(desc.NodeID) - if dsp.spanResolver == nil { - sr := physicalplan.NewSpanResolver(dsp.st, dsp.distSender, dsp.nodeDescs, desc, - dsp.clock, dsp.rpcCtx, ReplicaOraclePolicy) - dsp.SetSpanResolver(sr) +// ConstructAndSetSpanResolver constructs and sets the planner's +// SpanResolver if it is unset. It's a no-op otherwise. +func (dsp *DistSQLPlanner) ConstructAndSetSpanResolver( + ctx context.Context, nodeID roachpb.NodeID, locality roachpb.Locality, +) { + if dsp.spanResolver != nil { + log.Fatal(ctx, "trying to construct and set span resolver when one already exists") } + sr := physicalplan.NewSpanResolver(dsp.st, dsp.distSender, dsp.nodeDescs, nodeID, locality, + dsp.clock, dsp.rpcCtx, ReplicaOraclePolicy) + dsp.SetSpanResolver(sr) +} + +// SetGatewaySQLInstanceID sets the planner's SQL instance ID. +func (dsp *DistSQLPlanner) SetGatewaySQLInstanceID(id base.SQLInstanceID) { + dsp.gatewaySQLInstanceID = id } // GatewayID returns the ID of the gateway. diff --git a/pkg/sql/physicalplan/replicaoracle/BUILD.bazel b/pkg/sql/physicalplan/replicaoracle/BUILD.bazel index bdbcb5cd71d3..5b3a45f4be31 100644 --- a/pkg/sql/physicalplan/replicaoracle/BUILD.bazel +++ b/pkg/sql/physicalplan/replicaoracle/BUILD.bazel @@ -29,11 +29,13 @@ go_test( "//pkg/gossip", "//pkg/roachpb", "//pkg/rpc", + "//pkg/testutils", "//pkg/util", "//pkg/util/hlc", "//pkg/util/leaktest", "//pkg/util/metric", "//pkg/util/stop", + "@com_github_stretchr_testify//require", ], ) diff --git a/pkg/sql/physicalplan/replicaoracle/oracle.go b/pkg/sql/physicalplan/replicaoracle/oracle.go index b2817c87c778..929df9b8e6c4 100644 --- a/pkg/sql/physicalplan/replicaoracle/oracle.go +++ b/pkg/sql/physicalplan/replicaoracle/oracle.go @@ -43,7 +43,8 @@ var ( // Config is used to construct an OracleFactory. type Config struct { NodeDescs kvcoord.NodeDescStore - NodeDesc roachpb.NodeDescriptor // current node + NodeID roachpb.NodeID // current node's ID. 0 for secondary tenants. + Locality roachpb.Locality // current node's locality. Settings *cluster.Settings Clock *hlc.Clock RPCContext *rpc.Context @@ -147,16 +148,22 @@ func (o *randomOracle) ChoosePreferredReplica( type closestOracle struct { nodeDescs kvcoord.NodeDescStore - // nodeDesc is the descriptor of the current node. It will be used to give - // preference to the current node and others "close" to it. - nodeDesc roachpb.NodeDescriptor + // nodeID and locality of the current node. Used to give preference to the + // current node and others "close" to it. + // + // NodeID may be 0 in which case the current node will not be given any + // preference. NodeID being 0 indicates that no KV instance is available + // inside the same process. + nodeID roachpb.NodeID + locality roachpb.Locality latencyFunc kvcoord.LatencyFunc } func newClosestOracle(cfg Config) Oracle { return &closestOracle{ nodeDescs: cfg.NodeDescs, - nodeDesc: cfg.NodeDesc, + nodeID: cfg.NodeID, + locality: cfg.Locality, latencyFunc: latencyFunc(cfg.RPCContext), } } @@ -175,7 +182,7 @@ func (o *closestOracle) ChoosePreferredReplica( if err != nil { return roachpb.ReplicaDescriptor{}, err } - replicas.OptimizeReplicaOrder(o.nodeDesc.NodeID, o.latencyFunc, o.nodeDesc.Locality) + replicas.OptimizeReplicaOrder(o.nodeID, o.latencyFunc, o.locality) return replicas[0].ReplicaDescriptor, nil } @@ -197,9 +204,14 @@ const maxPreferredRangesPerLeaseHolder = 10 type binPackingOracle struct { maxPreferredRangesPerLeaseHolder int nodeDescs kvcoord.NodeDescStore - // nodeDesc is the descriptor of the current node. It will be used to give - // preference to the current node and others "close" to it. - nodeDesc roachpb.NodeDescriptor + // nodeID and locality of the current node. Used to give preference to the + // current node and others "close" to it. + // + // NodeID may be 0 in which case the current node will not be given any + // preference. NodeID being 0 indicates that no KV instance is available + // inside the same process. + nodeID roachpb.NodeID + locality roachpb.Locality latencyFunc kvcoord.LatencyFunc } @@ -207,7 +219,8 @@ func newBinPackingOracle(cfg Config) Oracle { return &binPackingOracle{ maxPreferredRangesPerLeaseHolder: maxPreferredRangesPerLeaseHolder, nodeDescs: cfg.NodeDescs, - nodeDesc: cfg.NodeDesc, + nodeID: cfg.NodeID, + locality: cfg.Locality, latencyFunc: latencyFunc(cfg.RPCContext), } } @@ -229,7 +242,7 @@ func (o *binPackingOracle) ChoosePreferredReplica( if err != nil { return roachpb.ReplicaDescriptor{}, err } - replicas.OptimizeReplicaOrder(o.nodeDesc.NodeID, o.latencyFunc, o.nodeDesc.Locality) + replicas.OptimizeReplicaOrder(o.nodeID, o.latencyFunc, o.locality) // Look for a replica that has been assigned some ranges, but it's not yet full. minLoad := int(math.MaxInt32) diff --git a/pkg/sql/physicalplan/replicaoracle/oracle_test.go b/pkg/sql/physicalplan/replicaoracle/oracle_test.go index 17adb2ebe78b..1cce2c93e815 100644 --- a/pkg/sql/physicalplan/replicaoracle/oracle_test.go +++ b/pkg/sql/physicalplan/replicaoracle/oracle_test.go @@ -13,6 +13,7 @@ package replicaoracle import ( "context" "fmt" + "math/rand" "strings" "testing" "time" @@ -21,11 +22,13 @@ import ( "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/stop" + "github.com/stretchr/testify/require" ) // TestRandomOracle defeats TestUnused for RandomChoice. @@ -35,41 +38,49 @@ func TestRandomOracle(t *testing.T) { func TestClosest(t *testing.T) { defer leaktest.AfterTest(t)() - ctx := context.Background() - stopper := stop.NewStopper() - defer stopper.Stop(ctx) - g, _ := makeGossip(t, stopper) - nd, _ := g.GetNodeDescriptor(1) - o := NewOracle(ClosestChoice, Config{ - NodeDescs: g, - NodeDesc: *nd, - }) - o.(*closestOracle).latencyFunc = func(s string) (time.Duration, bool) { - if strings.HasSuffix(s, "2") { - return time.Nanosecond, true + testutils.RunTrueAndFalse(t, "valid-latency-func", func(t *testing.T, validLatencyFunc bool) { + ctx := context.Background() + stopper := stop.NewStopper() + defer stopper.Stop(ctx) + g, _ := makeGossip(t, stopper) + nd2, err := g.GetNodeDescriptor(2) + require.NoError(t, err) + o := NewOracle(ClosestChoice, Config{ + NodeDescs: g, + NodeID: 1, + Locality: nd2.Locality, // pretend node 2 is closest. + }) + o.(*closestOracle).latencyFunc = func(s string) (time.Duration, bool) { + if strings.HasSuffix(s, "2") { + return time.Nanosecond, validLatencyFunc + } + return time.Millisecond, validLatencyFunc } - return time.Millisecond, true - } - info, err := o.ChoosePreferredReplica( - ctx, - nil, /* txn */ - &roachpb.RangeDescriptor{ - InternalReplicas: []roachpb.ReplicaDescriptor{ - {NodeID: 4, StoreID: 4}, - {NodeID: 2, StoreID: 2}, - {NodeID: 3, StoreID: 3}, + internalReplicas := []roachpb.ReplicaDescriptor{ + {NodeID: 4, StoreID: 4}, + {NodeID: 2, StoreID: 2}, + {NodeID: 3, StoreID: 3}, + } + rand.Shuffle(len(internalReplicas), func(i, j int) { + internalReplicas[i], internalReplicas[j] = internalReplicas[j], internalReplicas[i] + }) + info, err := o.ChoosePreferredReplica( + ctx, + nil, /* txn */ + &roachpb.RangeDescriptor{ + InternalReplicas: internalReplicas, }, - }, - nil, /* leaseHolder */ - roachpb.LAG_BY_CLUSTER_SETTING, - QueryState{}, - ) - if err != nil { - t.Fatalf("Failed to choose closest replica: %v", err) - } - if info.NodeID != 2 { - t.Fatalf("Failed to choose node 2, got %v", info.NodeID) - } + nil, /* leaseHolder */ + roachpb.LAG_BY_CLUSTER_SETTING, + QueryState{}, + ) + if err != nil { + t.Fatalf("Failed to choose closest replica: %v", err) + } + if info.NodeID != 2 { + t.Fatalf("Failed to choose node 2, got %v", info.NodeID) + } + }) } func makeGossip(t *testing.T, stopper *stop.Stopper) (*gossip.Gossip, *hlc.Clock) { @@ -99,5 +110,13 @@ func newNodeDesc(nodeID roachpb.NodeID) *roachpb.NodeDescriptor { return &roachpb.NodeDescriptor{ NodeID: nodeID, Address: util.MakeUnresolvedAddr("tcp", fmt.Sprintf("invalid.invalid:%d", nodeID)), + Locality: roachpb.Locality{ + Tiers: []roachpb.Tier{ + { + Key: "region", + Value: fmt.Sprintf("region_%d", nodeID), + }, + }, + }, } } diff --git a/pkg/sql/physicalplan/span_resolver.go b/pkg/sql/physicalplan/span_resolver.go index 4ea55a5536b6..9c59f0f5b5b1 100644 --- a/pkg/sql/physicalplan/span_resolver.go +++ b/pkg/sql/physicalplan/span_resolver.go @@ -118,7 +118,6 @@ type SpanResolverIterator interface { type spanResolver struct { st *cluster.Settings distSender *kvcoord.DistSender - nodeDesc roachpb.NodeDescriptor oracle replicaoracle.Oracle } @@ -129,17 +128,18 @@ func NewSpanResolver( st *cluster.Settings, distSender *kvcoord.DistSender, nodeDescs kvcoord.NodeDescStore, - nodeDesc roachpb.NodeDescriptor, + nodeID roachpb.NodeID, + locality roachpb.Locality, clock *hlc.Clock, rpcCtx *rpc.Context, policy replicaoracle.Policy, ) SpanResolver { return &spanResolver{ - st: st, - nodeDesc: nodeDesc, + st: st, oracle: replicaoracle.NewOracle(policy, replicaoracle.Config{ NodeDescs: nodeDescs, - NodeDesc: nodeDesc, + NodeID: nodeID, + Locality: locality, Settings: st, Clock: clock, RPCContext: rpcCtx, diff --git a/pkg/sql/physicalplan/span_resolver_test.go b/pkg/sql/physicalplan/span_resolver_test.go index 6083d71658ae..7ab53881ea7b 100644 --- a/pkg/sql/physicalplan/span_resolver_test.go +++ b/pkg/sql/physicalplan/span_resolver_test.go @@ -90,7 +90,8 @@ func TestSpanResolverUsesCaches(t *testing.T) { s3.Cfg.Settings, s3.DistSenderI().(*kvcoord.DistSender), s3.Gossip(), - s3.GetNode().Descriptor, + s3.GetNode().Descriptor.NodeID, + s3.GetNode().Descriptor.Locality, s3.Clock(), nil, // rpcCtx replicaoracle.BinPackingChoice) @@ -201,7 +202,8 @@ func TestSpanResolver(t *testing.T) { s.(*server.TestServer).Cfg.Settings, s.DistSenderI().(*kvcoord.DistSender), s.GossipI().(*gossip.Gossip), - s.(*server.TestServer).GetNode().Descriptor, + s.(*server.TestServer).GetNode().Descriptor.NodeID, + s.(*server.TestServer).GetNode().Descriptor.Locality, s.Clock(), nil, // rpcCtx replicaoracle.BinPackingChoice) @@ -299,7 +301,8 @@ func TestMixedDirections(t *testing.T) { s.(*server.TestServer).Cfg.Settings, s.DistSenderI().(*kvcoord.DistSender), s.GossipI().(*gossip.Gossip), - s.(*server.TestServer).GetNode().Descriptor, + s.(*server.TestServer).GetNode().Descriptor.NodeID, + s.(*server.TestServer).GetNode().Descriptor.Locality, s.Clock(), nil, // rpcCtx replicaoracle.BinPackingChoice) From 85590e2046704128aca016c130131f2cc1bcf8d6 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Tue, 9 Aug 2022 01:13:20 -0400 Subject: [PATCH 7/8] gcjob: issue DeleteRange tombstones and then wait for GC Note that this does not change anything about tenant GC. Fixes #70427 Release note (sql change): The asynchronous garbage collection process has been changed such that very soon after dropping a table, index, or database, or after refreshing a materialized view, the system will issue range deletion tombstones over the dropped data. These tombstones will result in the KV statistics properly counting these bytes as garbage. Before this change, the asynchronous "gc job" would wait out the TTL and then issue a lower-level operation to clear out the data. That meant that while the job was waiting out the TTL, the data would appear in the statistics to still be live. This was confusing. --- .../settings/settings-for-tenants.txt | 2 +- docs/generated/settings/settings.html | 2 +- pkg/ccl/backupccl/backup_test.go | 15 +- .../full_cluster_backup_restore_test.go | 10 +- .../changefeedccl/kvfeed/physical_kv_feed.go | 11 +- .../multiregionccl/regional_by_row_test.go | 13 +- pkg/ccl/partitionccl/drop_test.go | 15 +- .../spanconfiglimiterccl/BUILD.bazel | 1 + .../spanconfiglimiterccl/drop_table_test.go | 9 +- .../datadriven_test.go | 1 + .../testdata/protectedts | 7 + pkg/ccl/testccl/sqlccl/BUILD.bazel | 1 + pkg/ccl/testccl/sqlccl/tenant_gc_test.go | 2 + pkg/clusterversion/cockroach_versions.go | 11 ++ pkg/clusterversion/key_string.go | 5 +- pkg/jobs/jobspb/jobs.proto | 3 + .../spanconfigtestcluster/cluster.go | 3 + pkg/sql/catalog/lease/lease_test.go | 2 + pkg/sql/drop_test.go | 72 ++++--- pkg/sql/gcjob/BUILD.bazel | 1 + pkg/sql/gcjob/gc_job.go | 179 +++++++++++++++++- pkg/sql/gcjob/gc_job_utils.go | 41 +++- pkg/sql/gcjob/index_garbage_collection.go | 116 +++++++++++- pkg/sql/gcjob/refresh_statuses.go | 2 +- pkg/sql/gcjob/table_garbage_collection.go | 88 ++++++++- pkg/sql/gcjob_test/BUILD.bazel | 1 + pkg/sql/gcjob_test/gc_job_test.go | 106 +++-------- pkg/sql/importer/import_stmt_test.go | 8 +- .../logictest/testdata/logic_test/alter_table | 16 +- pkg/sql/mvcc_backfiller_test.go | 6 + pkg/sql/schema_change_plan_node.go | 1 + pkg/sql/schema_changer.go | 41 +++- pkg/sql/schema_changer_test.go | 47 ++++- pkg/sql/schemachanger/scdeps/BUILD.bazel | 1 + pkg/sql/schemachanger/scdeps/exec_deps.go | 9 + pkg/sql/schemachanger/scdeps/run_deps.go | 1 + .../scdeps/sctestdeps/test_deps.go | 5 + pkg/sql/schemachanger/scexec/dependencies.go | 4 + pkg/sql/schemachanger/scexec/exec_mutation.go | 1 + .../scexec/executor_external_test.go | 1 + pkg/sql/schemachanger/scexec/gc_jobs.go | 23 ++- pkg/sql/truncate.go | 8 +- pkg/sql/unsplit_range_test.go | 4 + 43 files changed, 701 insertions(+), 194 deletions(-) diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 6f5cab27f47d..904a2a935cc9 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -286,4 +286,4 @@ trace.jaeger.agent string the address of a Jaeger agent to receive traces using trace.opentelemetry.collector string address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.span_registry.enabled boolean true if set, ongoing traces can be seen at https:///#/debug/tracez trace.zipkin.collector string the address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -version version 22.1-54 set the active cluster version in the format '.' +version version 22.1-56 set the active cluster version in the format '.' diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index c8eec0073e94..c437c7e7c359 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -218,6 +218,6 @@ trace.opentelemetry.collectorstringaddress of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.span_registry.enabledbooleantrueif set, ongoing traces can be seen at https:///#/debug/tracez trace.zipkin.collectorstringthe address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -versionversion22.1-54set the active cluster version in the format '.' +versionversion22.1-56set the active cluster version in the format '.' diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index ab8621ebfff8..31edd044ff92 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -8963,12 +8963,15 @@ func TestGCDropIndexSpanExpansion(t *testing.T) { // the tenant. More investigation is required. Tracked with #76378. DisableDefaultTestTenant: true, Knobs: base.TestingKnobs{ - GCJob: &sql.GCJobTestingKnobs{RunBeforePerformGC: func(id jobspb.JobID) error { - gcJobID = id - aboutToGC <- struct{}{} - <-allowGC - return nil - }}, + GCJob: &sql.GCJobTestingKnobs{ + RunBeforePerformGC: func(id jobspb.JobID) error { + gcJobID = id + aboutToGC <- struct{}{} + <-allowGC + return nil + }, + SkipWaitingForMVCCGC: true, + }, }, }}) defer tc.Stopper().Stop(ctx) diff --git a/pkg/ccl/backupccl/full_cluster_backup_restore_test.go b/pkg/ccl/backupccl/full_cluster_backup_restore_test.go index ec4ef8cd397b..5d209a3cf9ae 100644 --- a/pkg/ccl/backupccl/full_cluster_backup_restore_test.go +++ b/pkg/ccl/backupccl/full_cluster_backup_restore_test.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/security/username" + clustersettings "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" @@ -48,8 +49,10 @@ func TestFullClusterBackup(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + settings := clustersettings.MakeTestingClusterSettings() params := base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ + Settings: settings, Knobs: base.TestingKnobs{ SpanConfig: &spanconfig.TestingKnobs{ // We compare job progress before and after a restore. Disable @@ -57,6 +60,11 @@ func TestFullClusterBackup(t *testing.T) { // the progress data during the backup/restore process. JobDisablePersistingCheckpoints: true, }, + GCJob: &sql.GCJobTestingKnobs{ + // We want to run the GC job to completion without waiting for + // MVCC GC. + SkipWaitingForMVCCGC: true, + }, }, }} const numAccounts = 10 @@ -67,7 +75,7 @@ func TestFullClusterBackup(t *testing.T) { // Closed when the restore is allowed to progress with the rest of the backup. allowProgressAfterPreRestore := make(chan struct{}) - // Closed to signal the the zones have been restored. + // Closed to signal the zones have been restored. restoredZones := make(chan struct{}) for _, server := range tcRestore.Servers { registry := server.JobRegistry().(*jobs.Registry) diff --git a/pkg/ccl/changefeedccl/kvfeed/physical_kv_feed.go b/pkg/ccl/changefeedccl/kvfeed/physical_kv_feed.go index ff52d56be302..0ba41d5b7a5c 100644 --- a/pkg/ccl/changefeedccl/kvfeed/physical_kv_feed.go +++ b/pkg/ccl/changefeedccl/kvfeed/physical_kv_feed.go @@ -124,9 +124,14 @@ func (p *rangefeed) addEventsToBuffer(ctx context.Context) error { return errors.Errorf("unexpected SST ingestion: %v", t) case *roachpb.RangeFeedDeleteRange: - // For now, we just error on MVCC range tombstones. These are currently + // For now, we just ignore on MVCC range tombstones. These are currently // only expected to be used by schema GC and IMPORT INTO, and such spans - // should not have active changefeeds across them. + // should not have active changefeeds across them, at least at the times + // of interest. A case where one will show up in a changefeed is when + // the primary index changes while we're watching it and then the old + // primary index is dropped. In this case, we'll get a schema event to + // restart into the new primary index, but the DeleteRange may come + // through before the schema event. // // TODO(erikgrinaker): Write an end-to-end test which verifies that an // IMPORT INTO which gets rolled back using MVCC range tombstones will @@ -136,7 +141,7 @@ func (p *rangefeed) addEventsToBuffer(ctx context.Context) error { // catchup scans should detect that this happened and prevent reading // anything in that timespan. See: // https://github.com/cockroachdb/cockroach/issues/70433 - return errors.Errorf("unexpected MVCC range deletion: %v", t) + continue default: return errors.Errorf("unexpected RangeFeedEvent variant %v", t) diff --git a/pkg/ccl/multiregionccl/regional_by_row_test.go b/pkg/ccl/multiregionccl/regional_by_row_test.go index 3be16797c9ea..df37fedda245 100644 --- a/pkg/ccl/multiregionccl/regional_by_row_test.go +++ b/pkg/ccl/multiregionccl/regional_by_row_test.go @@ -598,7 +598,10 @@ func TestIndexCleanupAfterAlterFromRegionalByRow(t *testing.T) { }, // Decrease the adopt loop interval so that retries happen quickly. JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), - GCJob: &sql.GCJobTestingKnobs{RunBeforeResume: func(_ jobspb.JobID) error { <-blockGC; return nil }}, + GCJob: &sql.GCJobTestingKnobs{ + RunBeforeResume: func(_ jobspb.JobID) error { <-blockGC; return nil }, + SkipWaitingForMVCCGC: true, + }, } _, sqlDB, cleanup := multiregionccltestutils.TestingCreateMultiRegionCluster( @@ -699,14 +702,6 @@ CREATE TABLE regional_by_row ( // Unblock GC jobs. close(blockGC) - // The GC jobs for the temporary indexes should be cleaned up immediately. - testutils.SucceedsSoon(t, queryAndEnsureThatIndexGCJobsSucceeded(expectedGCJobsForTempIndexes)) - // The GC jobs for the drops should still be waiting out the GC TTL. - err = queryIndexGCJobsAndValidateCount(`running`, expectedGCJobsForDrops) - require.NoError(t, err) - - // Change gc.ttlseconds to speed up the cleanup. - _ = sqlRunner.Exec(t, `ALTER TABLE regional_by_row CONFIGURE ZONE USING gc.ttlseconds = 1`) // Validate that indexes are cleaned up. testutils.SucceedsSoon(t, queryAndEnsureThatIndexGCJobsSucceeded(expectedGCJobsForDrops+expectedGCJobsForTempIndexes)) diff --git a/pkg/ccl/partitionccl/drop_test.go b/pkg/ccl/partitionccl/drop_test.go index 201e5fafe834..87dc3d817332 100644 --- a/pkg/ccl/partitionccl/drop_test.go +++ b/pkg/ccl/partitionccl/drop_test.go @@ -58,6 +58,7 @@ func TestDropIndexWithZoneConfigCCL(t *testing.T) { <-asyncNotification return nil }, + SkipWaitingForMVCCGC: true, }, } s, sqlDBRaw, kvDB := serverutils.StartServer(t, params) @@ -174,6 +175,12 @@ SELECT job_id }) } + knobs := base.TestingKnobs{ + GCJob: &sql.GCJobTestingKnobs{ + SkipWaitingForMVCCGC: true, + }, + } + // This is a regression test for a bug which was caused by not using hydrated // descriptors in the index gc job to re-write zone config subzone spans. // This test ensures that subzone spans can be re-written by creating a @@ -191,7 +198,9 @@ SELECT job_id defer log.Scope(t).Close(t) ctx := context.Background() - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) + tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{Knobs: knobs}, + }) defer tc.Stopper().Stop(ctx) tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0)) @@ -231,7 +240,9 @@ SELECT job_id defer log.Scope(t).Close(t) ctx := context.Background() - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) + tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{Knobs: knobs}, + }) defer tc.Stopper().Stop(ctx) tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0)) diff --git a/pkg/ccl/spanconfigccl/spanconfiglimiterccl/BUILD.bazel b/pkg/ccl/spanconfigccl/spanconfiglimiterccl/BUILD.bazel index 402daed457ff..8a47979570ba 100644 --- a/pkg/ccl/spanconfigccl/spanconfiglimiterccl/BUILD.bazel +++ b/pkg/ccl/spanconfigccl/spanconfiglimiterccl/BUILD.bazel @@ -23,6 +23,7 @@ go_test( "//pkg/server", "//pkg/spanconfig", "//pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster", + "//pkg/sql", "//pkg/sql/gcjob", "//pkg/testutils", "//pkg/testutils/serverutils", diff --git a/pkg/ccl/spanconfigccl/spanconfiglimiterccl/drop_table_test.go b/pkg/ccl/spanconfigccl/spanconfiglimiterccl/drop_table_test.go index 14d0d231cfa9..b217bad66d54 100644 --- a/pkg/ccl/spanconfigccl/spanconfiglimiterccl/drop_table_test.go +++ b/pkg/ccl/spanconfigccl/spanconfiglimiterccl/drop_table_test.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/gcjob" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" @@ -43,8 +44,12 @@ func TestDropTableLowersSpanCount(t *testing.T) { tenantID := roachpb.MakeTenantID(10) tenant, err := ts.StartTenant(ctx, base.TestTenantArgs{ - TenantID: tenantID, - TestingKnobs: base.TestingKnobs{}, + TenantID: tenantID, + TestingKnobs: base.TestingKnobs{ + GCJob: &sql.GCJobTestingKnobs{ + SkipWaitingForMVCCGC: true, + }, + }, }) require.NoError(t, err) diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go index 9565a8e39be8..204e865474a5 100644 --- a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go @@ -94,6 +94,7 @@ func TestDataDriven(t *testing.T) { gcWaiter.L.Unlock() return nil }, + SkipWaitingForMVCCGC: true, } scKnobs := &spanconfig.TestingKnobs{ // Instead of relying on the GC job to wait out TTLs and clear out diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/protectedts b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/protectedts index 68c1b959cdd7..238edb5e95cf 100644 --- a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/protectedts +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/protectedts @@ -106,6 +106,9 @@ release record-id=3 release record-id=4 ---- +block-gc-jobs +---- + # Create an index on t1 to ensure that subzones also see protected timestamps. exec-sql CREATE INDEX idx ON db.t1(id); @@ -128,3 +131,7 @@ translate database=db /Table/106/{3-4} ttl_seconds=1 num_replicas=7 num_voters=5 protection_policies=[{ts: 6}] /Table/10{6/4-7} num_replicas=7 num_voters=5 protection_policies=[{ts: 6}] /Table/10{7-8} num_replicas=7 + + +unblock-gc-jobs +---- diff --git a/pkg/ccl/testccl/sqlccl/BUILD.bazel b/pkg/ccl/testccl/sqlccl/BUILD.bazel index b6f104df9979..59c3112d638d 100644 --- a/pkg/ccl/testccl/sqlccl/BUILD.bazel +++ b/pkg/ccl/testccl/sqlccl/BUILD.bazel @@ -39,6 +39,7 @@ go_test( "//pkg/sql/tests", "//pkg/testutils", "//pkg/testutils/serverutils", + "//pkg/testutils/skip", "//pkg/testutils/sqlutils", "//pkg/testutils/testcluster", "//pkg/util/hlc", diff --git a/pkg/ccl/testccl/sqlccl/tenant_gc_test.go b/pkg/ccl/testccl/sqlccl/tenant_gc_test.go index a8a41ed87547..86922eff04dd 100644 --- a/pkg/ccl/testccl/sqlccl/tenant_gc_test.go +++ b/pkg/ccl/testccl/sqlccl/tenant_gc_test.go @@ -30,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/gcjob" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -152,6 +153,7 @@ func TestGCTenantRemovesSpanConfigs(t *testing.T) { func TestGCTableOrIndexWaitsForProtectedTimestamps(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + skip.WithIssue(t, 85876) defer gcjob.SetSmallMaxGCIntervalForTest()() var mu struct { diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index fcbc611b73d9..89edc04d4a0b 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -310,6 +310,13 @@ const ( // options table id column cannot be null. This is the final step // of the system.role_options table migration. SetRoleOptionsUserIDColumnNotNull + // UseDelRangeInGCJob enables the use of the DelRange operation in the + // GC job. Before it is enabled, the GC job uses ClearRange operations + // after the job waits out the GC TTL. After it has been enabled, the + // job instead issues DelRange operations at the beginning of the job + // and then waits for the data to be removed automatically before removing + // the descriptor and zone configurations. + UseDelRangeInGCJob // ************************************************* // Step (1): Add new versions here. @@ -524,6 +531,10 @@ var versionsSingleton = keyedVersions{ Key: SetRoleOptionsUserIDColumnNotNull, Version: roachpb.Version{Major: 22, Minor: 1, Internal: 54}, }, + { + Key: UseDelRangeInGCJob, + Version: roachpb.Version{Major: 22, Minor: 1, Internal: 56}, + }, // ************************************************* // Step (2): Add new versions here. // Do not add new versions to a patch release. diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index 6672154ea43a..650c02f410a1 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -52,11 +52,12 @@ func _() { _ = x[RoleOptionsTableHasIDColumn-41] _ = x[RoleOptionsIDColumnIsBackfilled-42] _ = x[SetRoleOptionsUserIDColumnNotNull-43] + _ = x[UseDelRangeInGCJob-44] } -const _Key_name = "V21_2Start22_1ProbeRequestPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreEnablePebbleFormatVersionBlockPropertiesEnableLeaseHolderRemovalChangefeedIdlenessRowLevelTTLEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsSuperRegionsEnableNewChangefeedOptionsV22_1Start22_2LocalTimestampsPebbleFormatSplitUserKeysMarkedCompactedEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocalitySystemExternalConnectionsTableAlterSystemStatementStatisticsAddIndexRecommendationsRoleIDSequenceAddSystemUserIDColumnSystemUsersIDColumnIsBackfilledSetSystemUsersUserIDColumnNotNullSQLSchemaTelemetryScheduledJobsSchemaChangeSupportsCreateFunctionDeleteRequestReturnKeyPebbleFormatPrePebblev1MarkedRoleOptionsTableHasIDColumnRoleOptionsIDColumnIsBackfilledSetRoleOptionsUserIDColumnNotNull" +const _Key_name = "V21_2Start22_1ProbeRequestPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreEnablePebbleFormatVersionBlockPropertiesEnableLeaseHolderRemovalChangefeedIdlenessRowLevelTTLEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsSuperRegionsEnableNewChangefeedOptionsV22_1Start22_2LocalTimestampsPebbleFormatSplitUserKeysMarkedCompactedEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocalitySystemExternalConnectionsTableAlterSystemStatementStatisticsAddIndexRecommendationsRoleIDSequenceAddSystemUserIDColumnSystemUsersIDColumnIsBackfilledSetSystemUsersUserIDColumnNotNullSQLSchemaTelemetryScheduledJobsSchemaChangeSupportsCreateFunctionDeleteRequestReturnKeyPebbleFormatPrePebblev1MarkedRoleOptionsTableHasIDColumnRoleOptionsIDColumnIsBackfilledSetRoleOptionsUserIDColumnNotNullUseDelRangeInGCJob" -var _Key_index = [...]uint16{0, 5, 14, 26, 54, 84, 112, 133, 173, 197, 215, 226, 250, 274, 296, 308, 334, 339, 348, 363, 403, 437, 471, 493, 513, 532, 565, 584, 604, 625, 660, 694, 724, 777, 791, 812, 843, 876, 907, 941, 963, 992, 1019, 1050, 1083} +var _Key_index = [...]uint16{0, 5, 14, 26, 54, 84, 112, 133, 173, 197, 215, 226, 250, 274, 296, 308, 334, 339, 348, 363, 403, 437, 471, 493, 513, 532, 565, 584, 604, 625, 660, 694, 724, 777, 791, 812, 843, 876, 907, 941, 963, 992, 1019, 1050, 1083, 1101} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index b9d1f70f9947..484924087a74 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -769,6 +769,9 @@ message SchemaChangeGCProgress { // This element has been deleted. The job is done when all elements are in // this state. CLEARED = 2; + // The index has been deleted, but we need to wait for the data to be + // removed before the relevant descriptors and zone configs can be deleted. + WAITING_FOR_MVCC_GC = 3; } message IndexProgress { diff --git a/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster/cluster.go b/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster/cluster.go index 83cfeb8c6b7a..368b82120f0e 100644 --- a/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster/cluster.go +++ b/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster/cluster.go @@ -67,6 +67,9 @@ func (h *Handle) InitializeTenant(ctx context.Context, tenID roachpb.TenantID) * TenantID: tenID, TestingKnobs: base.TestingKnobs{ SpanConfig: h.scKnobs, + GCJob: &sql.GCJobTestingKnobs{ + SkipWaitingForMVCCGC: true, + }, }, } var err error diff --git a/pkg/sql/catalog/lease/lease_test.go b/pkg/sql/catalog/lease/lease_test.go index c3da270da5f1..5190d0cc6e00 100644 --- a/pkg/sql/catalog/lease/lease_test.go +++ b/pkg/sql/catalog/lease/lease_test.go @@ -944,6 +944,8 @@ CREATE TABLE t.foo (v INT); // table descriptor. func TestTxnObeysTableModificationTime(t *testing.T) { defer leaktest.AfterTest(t)() + + skip.WithIssue(t, 85876) params := createTestServerParams() s, sqlDB, kvDB := serverutils.StartServer(t, params) defer s.Stopper().Stop(context.Background()) diff --git a/pkg/sql/drop_test.go b/pkg/sql/drop_test.go index 72e8ebc44144..9b98bb8b22d8 100644 --- a/pkg/sql/drop_test.go +++ b/pkg/sql/drop_test.go @@ -36,7 +36,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" - "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/gcjob" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -111,9 +110,16 @@ func TestDropDatabase(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) params, _ := tests.CreateTestServerParams() + ctx, cancel := context.WithCancel(context.Background()) + params.Knobs.GCJob = &sql.GCJobTestingKnobs{ + RunBeforeResume: func(jobID jobspb.JobID) error { + <-ctx.Done() + return ctx.Err() + }, + } s, sqlDB, kvDB := serverutils.StartServer(t, params) defer s.Stopper().Stop(context.Background()) - ctx := context.Background() + defer cancel() // Fix the column families so the key counts below don't change if the // family heuristics are updated. @@ -265,6 +271,8 @@ func TestDropDatabaseDeleteData(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + skip.WithIssue(t, 85876) + defer gcjob.SetSmallMaxGCIntervalForTest()() params, _ := tests.CreateTestServerParams() @@ -404,22 +412,14 @@ INSERT INTO t.kv2 VALUES ('c', 'd'), ('a', 'b'), ('e', 'a'); func TestDropIndex(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + + skip.WithIssue(t, 85876) const chunkSize = 200 params, _ := tests.CreateTestServerParams() - emptySpan := true - clearIndexAttempt := false params.Knobs = base.TestingKnobs{ SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ BackfillChunkSize: chunkSize, }, - DistSQL: &execinfra.TestingKnobs{ - RunBeforeBackfillChunk: func(sp roachpb.Span) error { - if clearIndexAttempt && (sp.Key != nil || sp.EndKey != nil) { - emptySpan = false - } - return nil - }, - }, } s, sqlDB, kvDB := serverutils.StartServer(t, params) defer s.Stopper().Stop(context.Background()) @@ -448,10 +448,6 @@ func TestDropIndex(t *testing.T) { if _, err := tableDesc.FindIndexWithName("foo"); err == nil { t.Fatalf("table descriptor still contains index after index is dropped") } - // Index data hasn't been deleted. - tests.CheckKeyCount(t, kvDB, indexSpan, numRows) - tests.CheckKeyCount(t, kvDB, tableDesc.TableSpan(keys.SystemSQLCodec), 3*numRows) - // TODO (lucy): Maybe this test API should use an offset starting // from the most recent job instead. const migrationJobOffset = 0 @@ -476,49 +472,39 @@ func TestDropIndex(t *testing.T) { t.Fatal(err) } newIdxSpan := tableDesc.IndexSpan(keys.SystemSQLCodec, newIdx.GetID()) - tests.CheckKeyCount(t, kvDB, newIdxSpan, numRows) - tests.CheckKeyCount(t, kvDB, tableDesc.TableSpan(keys.SystemSQLCodec), 4*numRows) - - clearIndexAttempt = true - // Add a zone config for the table. - if _, err := sqltestutils.AddImmediateGCZoneConfig(sqlDB, tableDesc.GetID()); err != nil { - t.Fatal(err) - } - testutils.SucceedsSoon(t, func() error { - return jobutils.VerifySystemJob(t, sqlRun, migrationJobOffset+1, jobspb.TypeSchemaChange, jobs.StatusSucceeded, jobs.Record{ + return jobutils.VerifySystemJob(t, sqlRun, migrationJobOffset+1, jobspb.TypeSchemaChange, jobs.StatusRunning, jobs.Record{ Username: username.RootUserName(), Description: `DROP INDEX t.public.kv@foo`, DescriptorIDs: descpb.IDs{ tableDesc.GetID(), }, + RunningStatus: "waiting for MVCC GC", }) }) testutils.SucceedsSoon(t, func() error { - if err := jobutils.VerifySystemJob(t, sqlRun, 0, jobspb.TypeSchemaChangeGC, jobs.StatusSucceeded, jobs.Record{ + if err := jobutils.VerifySystemJob(t, sqlRun, 0, jobspb.TypeSchemaChangeGC, jobs.StatusRunning, jobs.Record{ Username: username.RootUserName(), Description: `GC for temporary index used during index backfill`, DescriptorIDs: descpb.IDs{ tableDesc.GetID(), }, + RunningStatus: "waiting for MVCC GC", }); err != nil { return err } - return jobutils.VerifySystemJob(t, sqlRun, 1, jobspb.TypeSchemaChangeGC, jobs.StatusSucceeded, jobs.Record{ + return jobutils.VerifySystemJob(t, sqlRun, 1, jobspb.TypeSchemaChangeGC, jobs.StatusRunning, jobs.Record{ Username: username.RootUserName(), Description: `GC for DROP INDEX t.public.kv@foo`, DescriptorIDs: descpb.IDs{ tableDesc.GetID(), }, + RunningStatus: "waiting for MVCC GC", }) }) - if !emptySpan { - t.Fatalf("tried to clear index with non-empty resume span") - } - tests.CheckKeyCount(t, kvDB, newIdxSpan, numRows) tests.CheckKeyCount(t, kvDB, indexSpan, 0) tests.CheckKeyCount(t, kvDB, tableDesc.TableSpan(keys.SystemSQLCodec), 3*numRows) @@ -578,7 +564,7 @@ func TestDropIndexWithZoneConfigOSS(t *testing.T) { if sqlutils.ZoneConfigExists(t, sqlDB, "INDEX t.public.kv@foo") { t.Fatal("zone config for index still exists") } - tests.CheckKeyCount(t, kvDB, indexSpan, numRows) + // TODO(benesch): Run scrub here. It can't currently handle the way t.kv // declares column families. @@ -593,10 +579,18 @@ func TestDropIndexWithZoneConfigOSS(t *testing.T) { func TestDropTable(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + params, _ := tests.CreateTestServerParams() + ctx, cancel := context.WithCancel(context.Background()) + params.Knobs.GCJob = &sql.GCJobTestingKnobs{ + RunBeforeResume: func(jobID jobspb.JobID) error { + <-ctx.Done() + return ctx.Err() + }, + } s, sqlDB, kvDB := serverutils.StartServer(t, params) defer s.Stopper().Stop(context.Background()) - ctx := context.Background() + defer cancel() numRows := 2*row.TableTruncateChunkSize + 1 if err := tests.CreateKVTable(sqlDB, "kv", numRows); err != nil { @@ -684,6 +678,7 @@ func TestDropTable(t *testing.T) { func TestDropTableDeleteData(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + skip.WithIssue(t, 85876) params, _ := tests.CreateTestServerParams() defer gcjob.SetSmallMaxGCIntervalForTest()() @@ -836,6 +831,8 @@ func writeTableDesc(ctx context.Context, db *kv.DB, tableDesc *tabledesc.Mutable func TestDropTableWhileUpgradingFormat(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + + skip.WithIssue(t, 85876) ctx := context.Background() defer gcjob.SetSmallMaxGCIntervalForTest()() @@ -1258,6 +1255,7 @@ func TestDropIndexOnHashShardedIndexWithStoredShardColumn(t *testing.T) { // entire database. func TestDropDatabaseWithForeignKeys(t *testing.T) { defer leaktest.AfterTest(t)() + skip.WithIssue(t, 85876) defer log.Scope(t).Close(t) params, _ := tests.CreateTestServerParams() s, sqlDB, kvDB := serverutils.StartServer(t, params) @@ -1408,9 +1406,9 @@ func TestDropLargeDatabaseWithLegacySchemaChanger(t *testing.T) { dropLargeDatabaseGeneric(t, sqltestutils.GenerateViewBasedGraphSchemaParams{ SchemaName: "largedb", - NumTablesPerDepth: 5, - NumColumnsPerTable: 6, - GraphDepth: 4, + NumTablesPerDepth: 4, + NumColumnsPerTable: 3, + GraphDepth: 3, }, false) } diff --git a/pkg/sql/gcjob/BUILD.bazel b/pkg/sql/gcjob/BUILD.bazel index 76c00e401509..8d7f7105f685 100644 --- a/pkg/sql/gcjob/BUILD.bazel +++ b/pkg/sql/gcjob/BUILD.bazel @@ -16,6 +16,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/gcjob", visibility = ["//visibility:public"], deps = [ + "//pkg/clusterversion", "//pkg/config", "//pkg/config/zonepb", "//pkg/jobs", diff --git a/pkg/sql/gcjob/gc_job.go b/pkg/sql/gcjob/gc_job.go index f315c523a75b..a1f08f0c96e4 100644 --- a/pkg/sql/gcjob/gc_job.go +++ b/pkg/sql/gcjob/gc_job.go @@ -15,6 +15,7 @@ import ( "strings" "time" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" @@ -23,7 +24,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" @@ -95,6 +98,70 @@ func performGC( return nil } +// performGC GCs any schema elements that are in the DELETING state and returns +// a bool indicating if it GC'd any elements. +func deleteData( + ctx context.Context, + execCfg *sql.ExecutorConfig, + details *jobspb.SchemaChangeGCDetails, + progress *jobspb.SchemaChangeGCProgress, +) error { + switch { + case details.Indexes != nil: + return errors.Wrap( + deleteIndexData(ctx, execCfg, details.ParentID, progress), + "attempting to delete index data", + ) + case details.Tables != nil: + return errors.Wrap( + deleteTableData(ctx, execCfg, progress), + "attempted to delete table data", + ) + default: + return nil + } +} + +func deleteTableData( + ctx context.Context, cfg *sql.ExecutorConfig, progress *jobspb.SchemaChangeGCProgress, +) error { + if log.ExpensiveLogEnabled(ctx, 2) { + log.Infof(ctx, "GC is being considered for tables: %+v", progress.Tables) + } + for _, droppedTable := range progress.Tables { + var table catalog.TableDescriptor + if err := sql.DescsTxn(ctx, cfg, func(ctx context.Context, txn *kv.Txn, col *descs.Collection) (err error) { + table, err = col.Direct().MustGetTableDescByID(ctx, txn, droppedTable.ID) + return err + }); err != nil { + if errors.Is(err, catalog.ErrDescriptorNotFound) { + // This can happen if another GC job created for the same table got to + // the table first. See #50344. + log.Warningf(ctx, "table descriptor %d not found while attempting to GC, skipping", droppedTable.ID) + // Update the details payload to indicate that the table was dropped. + markTableGCed(ctx, droppedTable.ID, progress, jobspb.SchemaChangeGCProgress_CLEARED) + continue + } + return errors.Wrapf(err, "fetching table %d", droppedTable.ID) + } + + // TODO(ajwerner): How does this happen? + if !table.Dropped() { + // We shouldn't drop this table yet. + continue + } + if err := DeleteAllTableData(ctx, cfg.DB, cfg.DistSender, cfg.Codec, table); err != nil { + return err + } + + // Update the details payload to indicate that the table was dropped. + markTableGCed( + ctx, table.GetID(), progress, jobspb.SchemaChangeGCProgress_WAITING_FOR_MVCC_GC, + ) + } + return nil +} + // unsplitRangesInSpan unsplits any manually splits ranges within a span. func unsplitRangesInSpan(ctx context.Context, kvDB *kv.DB, span roachpb.Span) error { ranges, err := kvclient.ScanMetaKVs(ctx, kvDB.NewTxn(ctx, "unsplit-ranges-in-span"), span) @@ -211,7 +278,6 @@ func (r schemaChangeGCResumer) Resume(ctx context.Context, execCtx interface{}) } }() p := execCtx.(sql.JobExecContext) - // TODO(pbardea): Wait for no versions. // Clone the ExecConfig so that fields can be overwritten for testing knobs. execCfg := *p.ExecCfg() @@ -234,11 +300,117 @@ func (r schemaChangeGCResumer) Resume(ctx context.Context, execCtx interface{}) if err != nil { return err } - if err := maybeUnsplitRanges(ctx, &execCfg, r.job.ID(), details, progress); err != nil { return err } + if details.Tenant != nil || + !p.ExecCfg().Settings.Version.IsActive(ctx, clusterversion.UseDelRangeInGCJob) { + return r.legacyWaitAndClearTableData(ctx, execCfg, details, progress) + } + return r.deleteDataAndWaitForGC(ctx, execCfg, details, progress) +} + +func (r schemaChangeGCResumer) deleteDataAndWaitForGC( + ctx context.Context, + execCfg sql.ExecutorConfig, + details *jobspb.SchemaChangeGCDetails, + progress *jobspb.SchemaChangeGCProgress, +) error { + persistProgress(ctx, &execCfg, r.job.ID(), progress, + sql.RunningStatusDeletingData) + if fn := execCfg.GCJobTestingKnobs.RunBeforePerformGC; fn != nil { + if err := fn(r.job.ID()); err != nil { + return err + } + } + if err := deleteData(ctx, &execCfg, details, progress); err != nil { + return err + } + persistProgress(ctx, &execCfg, r.job.ID(), progress, sql.RunningStatusWaitingForMVCCGC) + r.job.MarkIdle(true) + return waitForGC(ctx, &execCfg, details, progress) +} + +func waitForGC( + ctx context.Context, + execCfg *sql.ExecutorConfig, + details *jobspb.SchemaChangeGCDetails, + progress *jobspb.SchemaChangeGCProgress, +) error { + switch { + case details.Indexes != nil: + return errors.Wrap( + waitForIndexGC(ctx, execCfg, details.ParentID, progress), + "attempting to delete index data", + ) + case details.Tables != nil: + return errors.Wrap( + deleteTableDescriptorsAfterGC(ctx, execCfg, details, progress), + "attempted to delete table data", + ) + default: + return nil + } +} + +// EmptySpanPollInterval is the interval at which the GC job will poll the +// spans to determine whether the data have been garbage collected. +var EmptySpanPollInterval = settings.RegisterDurationSetting( + settings.TenantReadOnly, + "sql.gc_job.wait_for_gc.interval", + "interval at which the GC job should poll to see if the deleted data has been GC'd", + 5*time.Minute, + settings.NonNegativeDuration, +) + +func waitForEmptyPrefix( + ctx context.Context, db *kv.DB, sv *settings.Values, skipWaiting bool, prefix roachpb.Key, +) error { + if skipWaiting { + log.Infof(ctx, "not waiting for MVCC GC in %v due to testing knob", prefix) + return nil + } + var timer timeutil.Timer + defer timer.Stop() + // TODO(ajwerner): Allow for settings watchers to be un-registered (#73830), + // then observe changes to the setting. + for { + timer.Reset(EmptySpanPollInterval.Get(sv)) + select { + case <-timer.C: + timer.Read = true + if empty, err := checkForEmptySpan( + ctx, db, prefix, prefix.PrefixEnd(), + ); empty || err != nil { + return err + } + case <-ctx.Done(): + return ctx.Err() + } + } +} + +func checkForEmptySpan(ctx context.Context, db *kv.DB, from, to roachpb.Key) (empty bool, _ error) { + var ba kv.Batch + ba.Header.MaxSpanRequestKeys = 1 + ba.AddRawRequest(&roachpb.IsSpanEmptyRequest{ + RequestHeader: roachpb.RequestHeader{ + Key: from, EndKey: to, + }, + }) + if err := db.Run(ctx, &ba); err != nil { + return false, err + } + return ba.RawResponse().Responses[0].GetIsSpanEmpty().IsEmpty(), nil +} + +func (r schemaChangeGCResumer) legacyWaitAndClearTableData( + ctx context.Context, + execCfg sql.ExecutorConfig, + details *jobspb.SchemaChangeGCDetails, + progress *jobspb.SchemaChangeGCProgress, +) error { tableDropTimes, indexDropTimes := getDropTimes(details) gossipUpdateC, cleanup := execCfg.GCJobNotifier.AddNotifyee(ctx) @@ -263,6 +435,7 @@ func (r schemaChangeGCResumer) Resume(ctx context.Context, execCtx interface{}) ctx, &execCfg, remainingTables, tableDropTimes, indexDropTimes, r.job.ID(), progress, ) } else { + var err error expired, earliestDeadline, err = refreshTenant(ctx, &execCfg, details.Tenant.DropTime, details, progress) if err != nil { return err @@ -271,7 +444,7 @@ func (r schemaChangeGCResumer) Resume(ctx context.Context, execCtx interface{}) timerDuration = time.Until(earliestDeadline) if expired { - // Some elements have been marked as DELETING so save the progress. + // Some elements have been marked as DELETING to save the progress. persistProgress(ctx, &execCfg, r.job.ID(), progress, runningStatusGC(progress)) if fn := execCfg.GCJobTestingKnobs.RunBeforePerformGC; fn != nil { if err := fn(r.job.ID()); err != nil { diff --git a/pkg/sql/gcjob/gc_job_utils.go b/pkg/sql/gcjob/gc_job_utils.go index 7ded5a8b4b32..5cfdcb5471f7 100644 --- a/pkg/sql/gcjob/gc_job_utils.go +++ b/pkg/sql/gcjob/gc_job_utils.go @@ -27,12 +27,15 @@ import ( // markTableGCed updates the job payload details to indicate that the specified // table was GC'd. func markTableGCed( - ctx context.Context, tableID descpb.ID, progress *jobspb.SchemaChangeGCProgress, + ctx context.Context, + tableID descpb.ID, + progress *jobspb.SchemaChangeGCProgress, + status jobspb.SchemaChangeGCProgress_Status, ) { for i := range progress.Tables { tableProgress := &progress.Tables[i] if tableProgress.ID == tableID { - tableProgress.Status = jobspb.SchemaChangeGCProgress_CLEARED + tableProgress.Status = status if log.V(2) { log.Infof(ctx, "determined table %d is GC'd", tableID) } @@ -45,12 +48,13 @@ func markIndexGCed( ctx context.Context, garbageCollectedIndexID descpb.IndexID, progress *jobspb.SchemaChangeGCProgress, + nextStatus jobspb.SchemaChangeGCProgress_Status, ) { // Update the job details to remove the dropped indexes. for i := range progress.Indexes { indexToUpdate := &progress.Indexes[i] if indexToUpdate.IndexID == garbageCollectedIndexID { - indexToUpdate.Status = jobspb.SchemaChangeGCProgress_CLEARED + indexToUpdate.Status = nextStatus log.Infof(ctx, "marked index %d as GC'd", garbageCollectedIndexID) } } @@ -104,10 +108,16 @@ func initializeProgress( } else if len(progress.Tables) != len(details.Tables) || len(progress.Indexes) != len(details.Indexes) { update = true for _, table := range details.Tables { - progress.Tables = append(progress.Tables, jobspb.SchemaChangeGCProgress_TableProgress{ID: table.ID}) + progress.Tables = append(progress.Tables, jobspb.SchemaChangeGCProgress_TableProgress{ + ID: table.ID, + Status: jobspb.SchemaChangeGCProgress_WAITING_FOR_CLEAR, + }) } for _, index := range details.Indexes { - progress.Indexes = append(progress.Indexes, jobspb.SchemaChangeGCProgress_IndexProgress{IndexID: index.IndexID}) + progress.Indexes = append(progress.Indexes, jobspb.SchemaChangeGCProgress_IndexProgress{ + IndexID: index.IndexID, + Status: jobspb.SchemaChangeGCProgress_WAITING_FOR_CLEAR, + }) } } @@ -147,14 +157,23 @@ func isDoneGC(progress *jobspb.SchemaChangeGCProgress) bool { // runningStatusGC generates a RunningStatus string which always remains under // a certain size, given any progress struct. func runningStatusGC(progress *jobspb.SchemaChangeGCProgress) jobs.RunningStatus { + var anyWaitingForMVCCGC bool + maybeSetAnyDeletedOrWaitingForMVCCGC := func(status jobspb.SchemaChangeGCProgress_Status) { + switch status { + case jobspb.SchemaChangeGCProgress_WAITING_FOR_MVCC_GC: + anyWaitingForMVCCGC = true + } + } tableIDs := make([]string, 0, len(progress.Tables)) indexIDs := make([]string, 0, len(progress.Indexes)) for _, table := range progress.Tables { + maybeSetAnyDeletedOrWaitingForMVCCGC(table.Status) if table.Status == jobspb.SchemaChangeGCProgress_CLEARING { tableIDs = append(tableIDs, strconv.Itoa(int(table.ID))) } } for _, index := range progress.Indexes { + maybeSetAnyDeletedOrWaitingForMVCCGC(index.Status) if index.Status == jobspb.SchemaChangeGCProgress_CLEARING { indexIDs = append(indexIDs, strconv.Itoa(int(index.IndexID))) } @@ -203,11 +222,15 @@ func runningStatusGC(progress *jobspb.SchemaChangeGCProgress) jobs.RunningStatus flag = true } - if !flag { - // `flag` not set implies we're not GCing anything. - return sql.RunningStatusWaitingGC + switch { + // `flag` not set implies we're not GCing anything. + case !flag && anyWaitingForMVCCGC: + return sql.RunningStatusWaitingForMVCCGC + case !flag: + return sql.RunningStatusWaitingGC // legacy status + default: + return jobs.RunningStatus(b.String()) } - return jobs.RunningStatus(b.String()) } // getAllTablesWaitingForGC returns a slice with all of the table IDs which have diff --git a/pkg/sql/gcjob/index_garbage_collection.go b/pkg/sql/gcjob/index_garbage_collection.go index 027a3f0ba38d..351fdd9fbf44 100644 --- a/pkg/sql/gcjob/index_garbage_collection.go +++ b/pkg/sql/gcjob/index_garbage_collection.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog" @@ -26,6 +27,50 @@ import ( "github.com/cockroachdb/errors" ) +// deleteIndexData is used to issue range deletion tombstones over all indexes +// being gc'd. +func deleteIndexData( + ctx context.Context, + execCfg *sql.ExecutorConfig, + parentID descpb.ID, + progress *jobspb.SchemaChangeGCProgress, +) error { + droppedIndexes := progress.Indexes + if log.V(2) { + log.Infof(ctx, "GC is being considered on table %d for indexes indexes: %+v", parentID, droppedIndexes) + } + + // Before deleting any indexes, ensure that old versions of the table descriptor + // are no longer in use. This is necessary in the case of truncate, where we + // schedule a GC Job in the transaction that commits the truncation. + parentDesc, err := sql.WaitToUpdateLeases(ctx, execCfg.LeaseManager, parentID) + if err != nil { + return err + } + + parentTable, isTable := parentDesc.(catalog.TableDescriptor) + if !isTable { + return errors.AssertionFailedf("expected descriptor %d to be a table, not %T", parentID, parentDesc) + } + for _, index := range droppedIndexes { + // TODO(ajwerner): Is there any reason to check on the current status of + // the index? At time of writing, we don't checkpoint between operating on + // individual indexes, and we always delete the data before moving on to + // waiting, so it seems like there's nothing to check for. + + if err := clearIndex( + ctx, execCfg, parentTable, index.IndexID, deleteAllSpanData, + ); err != nil { + return errors.Wrapf(err, "deleting index %d from table %d", index.IndexID, parentTable.GetID()) + } + markIndexGCed( + ctx, index.IndexID, progress, + jobspb.SchemaChangeGCProgress_WAITING_FOR_MVCC_GC, + ) + } + return nil +} + // gcIndexes find the indexes that need to be GC'd, GC's them, and then updates // the cleans up the table descriptor, zone configs and job payload to indicate // the work that it did. @@ -57,7 +102,9 @@ func gcIndexes( continue } - if err := clearIndex(ctx, execCfg, parentTable, index.IndexID); err != nil { + if err := clearIndex( + ctx, execCfg, parentTable, index.IndexID, clearSpanData, + ); err != nil { return errors.Wrapf(err, "clearing index %d from table %d", index.IndexID, parentTable.GetID()) } @@ -86,17 +133,27 @@ func gcIndexes( return errors.Wrapf(err, "removing index %d zone configs", index.IndexID) } - markIndexGCed(ctx, index.IndexID, progress) + markIndexGCed( + ctx, index.IndexID, progress, jobspb.SchemaChangeGCProgress_CLEARED, + ) } return nil } +type clearOrDeleteSpanDataFunc = func( + ctx context.Context, + db *kv.DB, + distSender *kvcoord.DistSender, + span roachpb.RSpan, +) error + // clearIndexes issues Clear Range requests over all specified indexes. func clearIndex( ctx context.Context, execCfg *sql.ExecutorConfig, tableDesc catalog.TableDescriptor, indexID descpb.IndexID, + clearOrDeleteSpanData clearOrDeleteSpanDataFunc, ) error { log.Infof(ctx, "clearing index %d from table %d", indexID, tableDesc.GetID()) @@ -110,5 +167,58 @@ func clearIndex( return errors.Wrap(err, "failed to addr index end") } rSpan := roachpb.RSpan{Key: start, EndKey: end} - return clearSpanData(ctx, execCfg.DB, execCfg.DistSender, rSpan) + return clearOrDeleteSpanData(ctx, execCfg.DB, execCfg.DistSender, rSpan) +} + +func waitForIndexGC( + ctx context.Context, + execCfg *sql.ExecutorConfig, + parentID descpb.ID, + progress *jobspb.SchemaChangeGCProgress, +) error { + + for _, index := range progress.Indexes { + if index.Status != jobspb.SchemaChangeGCProgress_CLEARING { + continue + } + + if err := waitForEmptyPrefix( + ctx, execCfg.DB, execCfg.SV(), + execCfg.GCJobTestingKnobs.SkipWaitingForMVCCGC, + execCfg.Codec.IndexPrefix(uint32(parentID), uint32(index.IndexID)), + ); err != nil { + return errors.Wrapf(err, "waiting for gc of index %d from table %d", + index.IndexID, parentID) + } + + // All the data chunks have been removed. Now also removed the + // zone configs for the dropped indexes, if any. + removeIndexZoneConfigs := func( + ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, + ) error { + freshParentTableDesc, err := descriptors.GetMutableTableByID( + ctx, txn, parentID, tree.ObjectLookupFlags{ + CommonLookupFlags: tree.CommonLookupFlags{ + AvoidLeased: true, + Required: true, + IncludeDropped: true, + IncludeOffline: true, + }, + }) + if err != nil { + return err + } + return sql.RemoveIndexZoneConfigs( + ctx, txn, execCfg, descriptors, freshParentTableDesc, []uint32{uint32(index.IndexID)}, + ) + } + if err := sql.DescsTxn(ctx, execCfg, removeIndexZoneConfigs); err != nil { + return errors.Wrapf(err, "removing index %d zone configs", index.IndexID) + } + + markIndexGCed( + ctx, index.IndexID, progress, jobspb.SchemaChangeGCProgress_CLEARED, + ) + } + return nil } diff --git a/pkg/sql/gcjob/refresh_statuses.go b/pkg/sql/gcjob/refresh_statuses.go index fb7f59d3cc1e..ba555f6f3558 100644 --- a/pkg/sql/gcjob/refresh_statuses.go +++ b/pkg/sql/gcjob/refresh_statuses.go @@ -141,7 +141,7 @@ func updateStatusForGCElements( }); err != nil { if errors.Is(err, catalog.ErrDescriptorNotFound) { log.Warningf(ctx, "table %d not found, marking as GC'd", tableID) - markTableGCed(ctx, tableID, progress) + markTableGCed(ctx, tableID, progress, jobspb.SchemaChangeGCProgress_CLEARED) return false, true, maxDeadline } log.Warningf(ctx, "error while calculating GC time for table %d, err: %+v", tableID, err) diff --git a/pkg/sql/gcjob/table_garbage_collection.go b/pkg/sql/gcjob/table_garbage_collection.go index 173557aa2032..5ba73bcb03e0 100644 --- a/pkg/sql/gcjob/table_garbage_collection.go +++ b/pkg/sql/gcjob/table_garbage_collection.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -55,12 +56,13 @@ func gcTables( // the table first. See #50344. log.Warningf(ctx, "table descriptor %d not found while attempting to GC, skipping", droppedTable.ID) // Update the details payload to indicate that the table was dropped. - markTableGCed(ctx, droppedTable.ID, progress) + markTableGCed(ctx, droppedTable.ID, progress, jobspb.SchemaChangeGCProgress_CLEARED) continue } return errors.Wrapf(err, "fetching table %d", droppedTable.ID) } + // TODO(ajwerner): How does this happen? if !table.Dropped() { // We shouldn't drop this table yet. continue @@ -94,7 +96,7 @@ func gcTables( } // Update the details payload to indicate that the table was dropped. - markTableGCed(ctx, table.GetID(), progress) + markTableGCed(ctx, table.GetID(), progress, jobspb.SchemaChangeGCProgress_CLEARED) } return nil } @@ -268,3 +270,85 @@ func deleteAllSpanData( return nil } + +func deleteTableDescriptorsAfterGC( + ctx context.Context, + execCfg *sql.ExecutorConfig, + details *jobspb.SchemaChangeGCDetails, + progress *jobspb.SchemaChangeGCProgress, +) error { + for _, droppedTable := range progress.Tables { + if droppedTable.Status == jobspb.SchemaChangeGCProgress_CLEARED { + // Table is not ready to be dropped, or has already been dropped. + continue + } + + var table catalog.TableDescriptor + if err := sql.DescsTxn(ctx, execCfg, func(ctx context.Context, txn *kv.Txn, col *descs.Collection) (err error) { + table, err = col.Direct().MustGetTableDescByID(ctx, txn, droppedTable.ID) + return err + }); err != nil { + if errors.Is(err, catalog.ErrDescriptorNotFound) { + // This can happen if another GC job created for the same table got to + // the table first. See #50344. + log.Warningf(ctx, "table descriptor %d not found while attempting to GC, skipping", droppedTable.ID) + // Update the details payload to indicate that the table was dropped. + markTableGCed(ctx, droppedTable.ID, progress, jobspb.SchemaChangeGCProgress_CLEARED) + continue + } + return errors.Wrapf(err, "fetching table %d", droppedTable.ID) + } + + // TODO(ajwerner): How does this happen? + if !table.Dropped() { + // We shouldn't drop this table yet. + continue + } + + // First, delete all the table data. + if err := waitForEmptyPrefix( + ctx, execCfg.DB, &execCfg.Settings.SV, + execCfg.GCJobTestingKnobs.SkipWaitingForMVCCGC, + execCfg.Codec.TablePrefix(uint32(table.GetID())), + ); err != nil { + return errors.Wrapf(err, "waiting for empty table %d", table.GetID()) + } + + delta, err := spanconfig.Delta(ctx, execCfg.SpanConfigSplitter, table, nil /* uncommitted */) + if err != nil { + return err + } + + // Deduct from system.span_count appropriately. + if err := execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + _, err := execCfg.SpanConfigLimiter.ShouldLimit(ctx, txn, delta) + return err + }); err != nil { + return errors.Wrapf(err, "deducting span count for table %d", table.GetID()) + } + + // Finished deleting all the table data, now delete the table meta data. + if err := sql.DeleteTableDescAndZoneConfig( + ctx, execCfg.DB, execCfg.Settings, execCfg.Codec, table, + ); err != nil { + return errors.Wrapf(err, "dropping table descriptor for table %d", table.GetID()) + } + + // Update the details payload to indicate that the table was dropped. + markTableGCed(ctx, table.GetID(), progress, jobspb.SchemaChangeGCProgress_CLEARED) + } + + // Drop database zone config when all the tables have been GCed. + if details.ParentID != descpb.InvalidID && isDoneGC(progress) { + if err := deleteDatabaseZoneConfig( + ctx, + execCfg.DB, + execCfg.Codec, + execCfg.Settings, + details.ParentID, + ); err != nil { + return errors.Wrap(err, "deleting database zone config") + } + } + return nil +} diff --git a/pkg/sql/gcjob_test/BUILD.bazel b/pkg/sql/gcjob_test/BUILD.bazel index f7dfc1f776b3..96f569c7ca8d 100644 --- a/pkg/sql/gcjob_test/BUILD.bazel +++ b/pkg/sql/gcjob_test/BUILD.bazel @@ -10,6 +10,7 @@ go_test( ], deps = [ "//pkg/base", + "//pkg/clusterversion", "//pkg/config", "//pkg/config/zonepb", "//pkg/jobs", diff --git a/pkg/sql/gcjob_test/gc_job_test.go b/pkg/sql/gcjob_test/gc_job_test.go index b49a9ad77ff4..8773c62b284d 100644 --- a/pkg/sql/gcjob_test/gc_job_test.go +++ b/pkg/sql/gcjob_test/gc_job_test.go @@ -19,6 +19,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/jobs" @@ -28,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/bootstrap" @@ -261,83 +263,20 @@ func TestSchemaChangeGCJob(t *testing.T) { } } -func TestSchemaChangeGCJobTableGCdWhileWaitingForExpiration(t *testing.T) { - defer leaktest.AfterTest(t)() - - args := base.TestServerArgs{Knobs: base.TestingKnobs{JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals()}} - - // We're going to drop a table then manually delete it, then update the - // database zone config and ensure the job finishes successfully. - s, db, kvDB := serverutils.StartServer(t, args) - ctx := context.Background() - defer s.Stopper().Stop(ctx) - sqlDB := sqlutils.MakeSQLRunner(db) - - // Note: this is to avoid a common failure during shutdown when a range - // merge runs concurrently with node shutdown leading to a panic due to - // pebble already being closed. See #51544. - sqlDB.Exec(t, "SET CLUSTER SETTING kv.range_merge.queue_enabled = false") - - sqlDB.Exec(t, "CREATE DATABASE db") - sqlDB.Exec(t, "CREATE TABLE db.foo ()") - var dbID, tableID descpb.ID - sqlDB.QueryRow(t, ` -SELECT parent_id, table_id - FROM crdb_internal.tables - WHERE database_name = $1 AND name = $2; -`, "db", "foo").Scan(&dbID, &tableID) - sqlDB.Exec(t, "DROP TABLE db.foo") - - // Now we should be able to find our GC job - var jobID jobspb.JobID - var status jobs.Status - var runningStatus jobs.RunningStatus - sqlDB.QueryRow(t, ` -SELECT job_id, status, running_status - FROM crdb_internal.jobs - WHERE description LIKE 'GC for DROP TABLE db.public.foo'; -`).Scan(&jobID, &status, &runningStatus) - require.Equal(t, jobs.StatusRunning, status) - require.Equal(t, sql.RunningStatusWaitingGC, runningStatus) - - // Manually delete the table. - require.NoError(t, kvDB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - nameKey := catalogkeys.MakePublicObjectNameKey(keys.SystemSQLCodec, dbID, "foo") - if _, err := txn.Del(ctx, nameKey); err != nil { - return err - } - descKey := catalogkeys.MakeDescMetadataKey(keys.SystemSQLCodec, tableID) - _, err := txn.Del(ctx, descKey) - return err - })) - // Update the GC TTL to tickle the job to refresh the status and discover that - // it has been removed. Use a SucceedsSoon to deal with races between setting - // the zone config and when the job subscribes to the zone config. - var i int - testutils.SucceedsSoon(t, func() error { - i++ - sqlDB.Exec(t, "ALTER DATABASE db CONFIGURE ZONE USING gc.ttlseconds = 60 * 60 * 25 + $1", i) - var status jobs.Status - sqlDB.QueryRow(t, "SELECT status FROM [SHOW JOB $1]", jobID).Scan(&status) - if status != jobs.StatusSucceeded { - return errors.Errorf("job status %v != %v", status, jobs.StatusSucceeded) - } - return nil - }) -} - func TestGCJobRetry(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) ctx := context.Background() var failed atomic.Value failed.Store(false) - params := base.TestServerArgs{} + cs := cluster.MakeTestingClusterSettings() + gcjob.EmptySpanPollInterval.Override(ctx, &cs.SV, 100*time.Millisecond) + params := base.TestServerArgs{Settings: cs} params.Knobs.JobsTestingKnobs = jobs.NewTestingKnobsWithShortIntervals() params.Knobs.Store = &kvserver.StoreTestingKnobs{ TestingRequestFilter: func(ctx context.Context, request roachpb.BatchRequest) *roachpb.Error { - _, ok := request.GetArg(roachpb.ClearRange) - if !ok { + r, ok := request.GetArg(roachpb.DeleteRange) + if !ok || !r.(*roachpb.DeleteRangeRequest).UseRangeTombstone { return nil } if failed.Load().(bool) { @@ -356,17 +295,16 @@ func TestGCJobRetry(t *testing.T) { tdb.Exec(t, "CREATE TABLE foo (i INT PRIMARY KEY)") tdb.Exec(t, "ALTER TABLE foo CONFIGURE ZONE USING gc.ttlseconds = 1;") tdb.Exec(t, "DROP TABLE foo CASCADE;") - var jobID int64 + var jobID string tdb.QueryRow(t, ` SELECT job_id FROM [SHOW JOBS] WHERE job_type = 'SCHEMA CHANGE GC' AND description LIKE '%foo%';`, ).Scan(&jobID) - var status jobs.Status - tdb.QueryRow(t, - "SELECT status FROM [SHOW JOB WHEN COMPLETE $1]", jobID, - ).Scan(&status) - require.Equal(t, jobs.StatusSucceeded, status) + tdb.CheckQueryResultsRetry(t, + "SELECT running_status FROM crdb_internal.jobs WHERE job_id = "+jobID, + [][]string{{string(sql.RunningStatusWaitingForMVCCGC)}}, + ) } // TestGCTenant is lightweight test that tests the branching logic in Resume @@ -375,7 +313,7 @@ func TestGCResumer(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) defer jobs.ResetConstructors()() - gcjob.SetSmallMaxGCIntervalForTest() + defer gcjob.SetSmallMaxGCIntervalForTest()() ctx := context.Background() args := base.TestServerArgs{Knobs: base.TestingKnobs{JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals()}} @@ -501,7 +439,7 @@ func TestGCTenant(t *testing.T) { require.EqualError( t, gcClosure(10, progress), - "Tenant id 10 is expired and should not be in state WAITING_FOR_GC", + "Tenant id 10 is expired and should not be in state WAITING_FOR_CLEAR", ) require.Equal(t, jobspb.SchemaChangeGCProgress_WAITING_FOR_CLEAR, progress.Tenant.Status) }) @@ -576,20 +514,30 @@ func TestGCTenant(t *testing.T) { // no system config provided by the SystemConfigProvider. It is a regression // test for a panic which could occur due to a slow systemconfigwatcher // initialization. +// +// TODO(ajwerner): Remove this test in 23.1. func TestGCJobNoSystemConfig(t *testing.T) { defer leaktest.AfterTest(t)() provider := fakeSystemConfigProvider{} - settings := cluster.MakeTestingClusterSettings() - stopper := stop.NewStopper() + var ( + v0 = clusterversion.ByKey(clusterversion.UseDelRangeInGCJob - 1) + v1 = clusterversion.ByKey(clusterversion.UseDelRangeInGCJob) + ) + settings := cluster.MakeTestingClusterSettingsWithVersions(v1, v0, false /* initializeVersion */) ctx := context.Background() - + require.NoError(t, clusterversion.Initialize(ctx, v0, &settings.SV)) + stopper := stop.NewStopper() gcKnobs := &sql.GCJobTestingKnobs{} s, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{ Settings: settings, Stopper: stopper, Knobs: base.TestingKnobs{ GCJob: gcKnobs, + Server: &server.TestingKnobs{ + DisableAutomaticVersionUpgrade: make(chan struct{}), + BinaryVersionOverride: v0, + }, }, }) defer stopper.Stop(ctx) diff --git a/pkg/sql/importer/import_stmt_test.go b/pkg/sql/importer/import_stmt_test.go index cf8cd97f7f96..9018652f203b 100644 --- a/pkg/sql/importer/import_stmt_test.go +++ b/pkg/sql/importer/import_stmt_test.go @@ -2044,8 +2044,8 @@ func TestFailedImportGC(t *testing.T) { close(blockGC) // Ensure that a GC job was created, and wait for it to finish. doneGCQuery := fmt.Sprintf( - "SELECT count(*) FROM [SHOW JOBS] WHERE job_type = '%s' AND status = '%s' AND created > %s", - "SCHEMA CHANGE GC", jobs.StatusSucceeded, beforeImport.String(), + "SELECT count(*) FROM [SHOW JOBS] WHERE job_type = '%s' AND running_status = '%s' AND created > %s", + "SCHEMA CHANGE GC", sql.RunningStatusWaitingForMVCCGC, beforeImport.String(), ) sqlDB.CheckQueryResultsRetry(t, doneGCQuery, [][]string{{"1"}}) // Expect there are no more KVs for this span. @@ -6301,8 +6301,8 @@ func TestImportPgDumpSchemas(t *testing.T) { // Ensure that a GC job was created, and wait for it to finish. doneGCQuery := fmt.Sprintf( - "SELECT count(*) FROM [SHOW JOBS] WHERE job_type = '%s' AND status = '%s' AND created > %s", - "SCHEMA CHANGE GC", jobs.StatusSucceeded, beforeImport.String(), + "SELECT count(*) FROM [SHOW JOBS] WHERE job_type = '%s' AND running_status = '%s' AND created > %s", + "SCHEMA CHANGE GC", sql.RunningStatusWaitingForMVCCGC, beforeImport.String(), ) doneSchemaDropQuery := fmt.Sprintf( diff --git a/pkg/sql/logictest/testdata/logic_test/alter_table b/pkg/sql/logictest/testdata/logic_test/alter_table index f3d11aea170f..5bfe974eb3b0 100644 --- a/pkg/sql/logictest/testdata/logic_test/alter_table +++ b/pkg/sql/logictest/testdata/logic_test/alter_table @@ -216,15 +216,15 @@ ALTER TABLE t DROP CONSTRAINT foo statement ok DROP INDEX foo CASCADE -query TTTTTRT -SELECT job_type, description, user_name, status, running_status, fraction_completed, error +query TTTTRT retry +SELECT job_type, description, user_name, status, fraction_completed, error FROM crdb_internal.jobs WHERE job_type = 'SCHEMA CHANGE' OR job_type = 'SCHEMA CHANGE GC' ORDER BY created DESC LIMIT 2 ---- -SCHEMA CHANGE GC GC for DROP INDEX test.public.t@foo CASCADE root running waiting for GC TTL 0 · -SCHEMA CHANGE DROP INDEX test.public.t@foo CASCADE root succeeded NULL 1 · +SCHEMA CHANGE GC GC for DROP INDEX test.public.t@foo CASCADE root running 0 · +SCHEMA CHANGE DROP INDEX test.public.t@foo CASCADE root succeeded 1 · query TTBITTBBB colnames SHOW INDEXES FROM t @@ -301,15 +301,15 @@ INSERT INTO t (a, d, x, y, z) VALUES (33, 34, DECIMAL '2.0', DECIMAL '2.1', DECI statement ok DROP INDEX t@t_f_idx -query TTTTTRT -SELECT job_type, description, user_name, status, running_status, fraction_completed, error +query TTTTRT retry +SELECT job_type, description, user_name, status, fraction_completed, error FROM crdb_internal.jobs WHERE job_type = 'SCHEMA CHANGE' OR job_type = 'SCHEMA CHANGE GC' ORDER BY created DESC LIMIT 2 ---- -SCHEMA CHANGE GC GC for DROP INDEX test.public.t@t_f_idx root running waiting for GC TTL 0 · -SCHEMA CHANGE DROP INDEX test.public.t@t_f_idx root succeeded NULL 1 · +SCHEMA CHANGE GC GC for DROP INDEX test.public.t@t_f_idx root running 0 · +SCHEMA CHANGE DROP INDEX test.public.t@t_f_idx root succeeded 1 · statement ok ALTER TABLE t DROP COLUMN f diff --git a/pkg/sql/mvcc_backfiller_test.go b/pkg/sql/mvcc_backfiller_test.go index 1187c081f4b6..f589841948ed 100644 --- a/pkg/sql/mvcc_backfiller_test.go +++ b/pkg/sql/mvcc_backfiller_test.go @@ -128,6 +128,9 @@ func TestIndexBackfillMergeRetry(t *testing.T) { }, // Decrease the adopt loop interval so that retries happen quickly. JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), + GCJob: &sql.GCJobTestingKnobs{ + SkipWaitingForMVCCGC: true, + }, } s, sqlDB, kvDB := serverutils.StartServer(t, params) @@ -567,6 +570,9 @@ func TestIndexBackfillMergeTxnRetry(t *testing.T) { }, }, JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), + GCJob: &sql.GCJobTestingKnobs{ + SkipWaitingForMVCCGC: true, + }, } s, sqlDB, kvDB = serverutils.StartServer(t, params) diff --git a/pkg/sql/schema_change_plan_node.go b/pkg/sql/schema_change_plan_node.go index ef772445a37e..2f59aa2b3156 100644 --- a/pkg/sql/schema_change_plan_node.go +++ b/pkg/sql/schema_change_plan_node.go @@ -254,6 +254,7 @@ func newSchemaChangerTxnRunDependencies( sessionData, ) return scdeps.NewExecutorDependencies( + execCfg.Settings, execCfg.Codec, sessionData, txn, diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index 1c142f94af42..415fbda04713 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -18,6 +18,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" @@ -68,6 +69,12 @@ import ( ) const ( + // RunningStatusWaitingForMVCCGC is used for the GC job when it has cleared + // the data but is waiting for MVCC GC to remove the data. + RunningStatusWaitingForMVCCGC jobs.RunningStatus = "waiting for MVCC GC" + // RunningStatusDeletingData is used for the GC job when it is about + // to clear the data. + RunningStatusDeletingData jobs.RunningStatus = "deleting data" // RunningStatusWaitingGC is for jobs that are currently in progress and // are waiting for the GC interval to expire RunningStatusWaitingGC jobs.RunningStatus = "waiting for GC TTL" @@ -510,8 +517,11 @@ func startGCJob( userName username.SQLUsername, schemaChangeDescription string, details jobspb.SchemaChangeGCDetails, + useLegacyGCJob bool, ) error { - jobRecord := CreateGCJobRecord(schemaChangeDescription, userName, details) + jobRecord := CreateGCJobRecord( + schemaChangeDescription, userName, details, useLegacyGCJob, + ) jobID := jobRegistry.MakeJobID() if err := db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { _, err := jobRegistry.CreateJobWithTxn(ctx, jobRecord, jobID, txn) @@ -719,7 +729,11 @@ func (sc *SchemaChanger) exec(ctx context.Context) error { }, } if err := startGCJob( - ctx, sc.db, sc.jobRegistry, sc.job.Payload().UsernameProto.Decode(), sc.job.Payload().Description, gcDetails, + ctx, sc.db, sc.jobRegistry, + sc.job.Payload().UsernameProto.Decode(), + sc.job.Payload().Description, + gcDetails, + !sc.settings.Version.IsActive(ctx, clusterversion.UseDelRangeInGCJob), ); err != nil { return err } @@ -1062,6 +1076,7 @@ func (sc *SchemaChanger) rollbackSchemaChange(ctx context.Context, err error) er }, }, }, + !sc.settings.Version.IsActive(ctx, clusterversion.UseDelRangeInGCJob), ) if _, err := sc.jobRegistry.CreateJobWithTxn(ctx, jobRecord, gcJobID, txn); err != nil { return err @@ -1277,7 +1292,10 @@ func (sc *SchemaChanger) createIndexGCJobWithDropTime( ParentID: sc.descID, } - gcJobRecord := CreateGCJobRecord(jobDesc, sc.job.Payload().UsernameProto.Decode(), indexGCDetails) + gcJobRecord := CreateGCJobRecord( + jobDesc, sc.job.Payload().UsernameProto.Decode(), indexGCDetails, + !sc.settings.Version.IsActive(ctx, clusterversion.UseDelRangeInGCJob), + ) jobID := sc.jobRegistry.MakeJobID() if _, err := sc.jobRegistry.CreateJobWithTxn(ctx, gcJobRecord, jobID, txn); err != nil { return err @@ -2279,7 +2297,10 @@ func (sc *SchemaChanger) reverseMutation( // CreateGCJobRecord creates the job record for a GC job, setting some // properties which are common for all GC jobs. func CreateGCJobRecord( - originalDescription string, userName username.SQLUsername, details jobspb.SchemaChangeGCDetails, + originalDescription string, + userName username.SQLUsername, + details jobspb.SchemaChangeGCDetails, + useLegacyGCJob bool, ) jobs.Record { descriptorIDs := make([]descpb.ID, 0) if len(details.Indexes) > 0 { @@ -2291,13 +2312,17 @@ func CreateGCJobRecord( descriptorIDs = append(descriptorIDs, table.ID) } } + runningStatus := RunningStatusDeletingData + if useLegacyGCJob { + runningStatus = RunningStatusWaitingGC + } return jobs.Record{ Description: fmt.Sprintf("GC for %s", originalDescription), Username: userName, DescriptorIDs: descriptorIDs, Details: details, Progress: jobspb.SchemaChangeGCProgress{}, - RunningStatus: RunningStatusWaitingGC, + RunningStatus: runningStatus, NonCancelable: true, } } @@ -2315,6 +2340,9 @@ type GCJobTestingKnobs struct { // Notifier is used to optionally inject a new gcjobnotifier.Notifier. Notifier *gcjobnotifier.Notifier + + // If true, the GC job will not wait for MVCC GC. + SkipWaitingForMVCCGC bool } // ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface. @@ -2723,6 +2751,9 @@ func (r schemaChangeResumer) Resume(ctx context.Context, execCtx interface{}) er r.job.Payload().UsernameProto.Decode(), r.job.Payload().Description, multiTableGCDetails, + !p.ExecCfg().Settings.Version.IsActive( + ctx, clusterversion.UseDelRangeInGCJob, + ), ); err != nil { return err } diff --git a/pkg/sql/schema_changer_test.go b/pkg/sql/schema_changer_test.go index 9ee110ee43e1..379a7da560e9 100644 --- a/pkg/sql/schema_changer_test.go +++ b/pkg/sql/schema_changer_test.go @@ -401,9 +401,16 @@ func runSchemaChangeWithOperations( // validate schema change operations. We wait for any SCHEMA // CHANGE GC jobs for temp indexes to show that the temp index // has been cleared. - if _, err := sqlDB.Exec(`SHOW JOBS WHEN COMPLETE (SELECT job_id FROM [SHOW JOBS] WHERE job_type = 'SCHEMA CHANGE GC')`); err != nil { - t.Fatal(err) - } + sqlutils.MakeSQLRunner(sqlDB).CheckQueryResultsRetry(t, ` +SELECT count(*) + FROM ( + SELECT job_id + FROM [SHOW JOBS] + WHERE job_type = 'SCHEMA CHANGE GC' + AND status != 'succeeded' + AND running_status != 'waiting for MVCC GC' + )`, + [][]string{{"0"}}) testutils.SucceedsSoon(t, func() error { return sqltestutils.CheckTableKeyCount(ctx, kvDB, keyMultiple, maxValue+numInserts) }) @@ -736,6 +743,7 @@ func TestDropWhileBackfill(t *testing.T) { // We expect this to also reduce the memory footprint of the test. maxValue = 200 } + ctx, cancel := context.WithCancel(context.Background()) params, _ := tests.CreateTestServerParams() notifyBackfill := func() { mu.Lock() @@ -766,6 +774,13 @@ func TestDropWhileBackfill(t *testing.T) { StartupMigrationManager: &startupmigrations.MigrationManagerTestingKnobs{ DisableBackfillMigrations: true, }, + // Prevent the GC job from running. + GCJob: &sql.GCJobTestingKnobs{ + RunBeforeResume: func(_ jobspb.JobID) error { + <-ctx.Done() + return ctx.Err() + }, + }, } tc := serverutils.StartNewTestCluster(t, numNodes, @@ -774,6 +789,7 @@ func TestDropWhileBackfill(t *testing.T) { ServerArgs: params, }) defer tc.Stopper().Stop(context.Background()) + defer cancel() kvDB := tc.Server(0).DB() sqlDB := tc.ServerConn(0) @@ -804,8 +820,6 @@ CREATE UNIQUE INDEX vidx ON t.test (v); } sql.SplitTable(t, tc, tableDesc, sps) - ctx := context.Background() - // number of keys == 2 * number of rows; 1 column family and 1 index entry // for each row. if err := sqltestutils.CheckTableKeyCount(ctx, kvDB, 2, maxValue); err != nil { @@ -821,7 +835,9 @@ CREATE UNIQUE INDEX vidx ON t.test (v); wg.Add(1) go func() { // Start schema change that eventually runs a partial backfill. - if _, err := sqlDB.Exec("CREATE UNIQUE INDEX bar ON t.test (v)"); err != nil && !testutils.IsError(err, "descriptor is being dropped") { + if _, err := sqlDB.Exec( + "CREATE UNIQUE INDEX bar ON t.test (v)", + ); err != nil && !testutils.IsError(err, "descriptor is being dropped") { t.Error(err) } wg.Done() @@ -2531,7 +2547,7 @@ func TestPrimaryKeyChangeWithPrecedingIndexCreation(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - ctx := context.Background() + ctx, cancel := context.WithCancel(context.Background()) var chunkSize int64 = 100 var maxValue = 4000 @@ -2578,9 +2594,16 @@ func TestPrimaryKeyChangeWithPrecedingIndexCreation(t *testing.T) { }, // Decrease the adopt loop interval so that retries happen quickly. JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), + // Prevent the GC job from running so we ensure that all the keys + // which were written remain. + GCJob: &sql.GCJobTestingKnobs{RunBeforeResume: func(jobID jobspb.JobID) error { + <-ctx.Done() + return ctx.Err() + }}, } s, sqlDB, kvDB := serverutils.StartServer(t, params) defer s.Stopper().Stop(ctx) + defer cancel() if _, err := sqlDB.Exec(`CREATE DATABASE t`); err != nil { t.Fatal(err) @@ -3380,7 +3403,7 @@ CREATE TABLE t.test (k INT NOT NULL, v INT); `) require.NoError(t, err) - _, err = sqlDB.Exec(`SET use_declarative_schema_changer = off; + _, err = sqlDB.Exec(`SET use_declarative_schema_changer = off; ALTER TABLE t.test ALTER PRIMARY KEY USING COLUMNS (k)`) require.NoError(t, err) @@ -3810,11 +3833,18 @@ func TestBackfillCompletesOnChunkBoundary(t *testing.T) { // [0...maxValue], so that the backfill processing ends on // a chunk boundary. const maxValue = 3*chunkSize - 1 + ctx, cancel := context.WithCancel(context.Background()) params, _ := tests.CreateTestServerParams() params.Knobs = base.TestingKnobs{ SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ BackfillChunkSize: chunkSize, }, + GCJob: &sql.GCJobTestingKnobs{ + RunBeforeResume: func(jobID jobspb.JobID) error { + <-ctx.Done() + return ctx.Err() + }, + }, } tc := serverutils.StartNewTestCluster(t, numNodes, @@ -3823,6 +3853,7 @@ func TestBackfillCompletesOnChunkBoundary(t *testing.T) { ServerArgs: params, }) defer tc.Stopper().Stop(context.Background()) + defer cancel() kvDB := tc.Server(0).DB() sqlDB := tc.ServerConn(0) // Declarative schema changer does not use then new MVCC backfiller, so diff --git a/pkg/sql/schemachanger/scdeps/BUILD.bazel b/pkg/sql/schemachanger/scdeps/BUILD.bazel index 2d90d3b60008..1c68c264f92f 100644 --- a/pkg/sql/schemachanger/scdeps/BUILD.bazel +++ b/pkg/sql/schemachanger/scdeps/BUILD.bazel @@ -12,6 +12,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scdeps", visibility = ["//visibility:public"], deps = [ + "//pkg/clusterversion", "//pkg/config", "//pkg/jobs", "//pkg/jobs/jobspb", diff --git a/pkg/sql/schemachanger/scdeps/exec_deps.go b/pkg/sql/schemachanger/scdeps/exec_deps.go index 19e3e9a7f118..e2d3356e9c17 100644 --- a/pkg/sql/schemachanger/scdeps/exec_deps.go +++ b/pkg/sql/schemachanger/scdeps/exec_deps.go @@ -15,6 +15,7 @@ import ( "fmt" "time" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" @@ -23,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/server/telemetry" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -50,6 +52,7 @@ type JobRegistry interface { // NewExecutorDependencies returns an scexec.Dependencies implementation built // from the given arguments. func NewExecutorDependencies( + settings *cluster.Settings, codec keys.SQLCodec, sessionData *sessiondata.SessionData, txn *kv.Txn, @@ -81,6 +84,7 @@ func NewExecutorDependencies( statsRefresher: statsRefresher, schemaChangerJobID: schemaChangerJobID, kvTrace: kvTrace, + settings: settings, }, backfiller: backfiller, merger: merger, @@ -108,6 +112,7 @@ type txnDeps struct { deletedDescriptors catalog.DescriptorIDSet schemaChangerJobID jobspb.JobID kvTrace bool + settings *cluster.Settings } func (d *txnDeps) UpdateSchemaChangeJob( @@ -285,6 +290,10 @@ func (d *txnDeps) CheckPausepoint(name string) error { return d.jobRegistry.CheckPausepoint(name) } +func (d *txnDeps) UseLegacyGCJob(ctx context.Context) bool { + return !d.settings.Version.IsActive(ctx, clusterversion.UseDelRangeInGCJob) +} + func (d *txnDeps) SchemaChangerJobID() jobspb.JobID { if d.schemaChangerJobID == 0 { d.schemaChangerJobID = d.jobRegistry.MakeJobID() diff --git a/pkg/sql/schemachanger/scdeps/run_deps.go b/pkg/sql/schemachanger/scdeps/run_deps.go index bfc063eeed2c..c6a5a13c6fd5 100644 --- a/pkg/sql/schemachanger/scdeps/run_deps.go +++ b/pkg/sql/schemachanger/scdeps/run_deps.go @@ -123,6 +123,7 @@ func (d *jobExecutionDeps) WithTxnInJob(ctx context.Context, fn scrun.JobTxnFunc statsRefresher: d.statsRefresher, schemaChangerJobID: d.job.ID(), kvTrace: d.kvTrace, + settings: d.settings, }, backfiller: d.backfiller, merger: d.merger, diff --git a/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go b/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go index 81106b3941fc..0acb0601b696 100644 --- a/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go +++ b/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go @@ -859,6 +859,11 @@ func (s *TestState) CheckPausepoint(name string) error { return nil } +// UseLegacyGCJob is false. +func (s *TestState) UseLegacyGCJob(ctx context.Context) bool { + return false +} + // UpdateSchemaChangeJob implements the scexec.TransactionalJobRegistry interface. func (s *TestState) UpdateSchemaChangeJob( ctx context.Context, id jobspb.JobID, fn scexec.JobUpdateCallback, diff --git a/pkg/sql/schemachanger/scexec/dependencies.go b/pkg/sql/schemachanger/scexec/dependencies.go index 9b57f007f364..33ee6087c8d1 100644 --- a/pkg/sql/schemachanger/scexec/dependencies.go +++ b/pkg/sql/schemachanger/scexec/dependencies.go @@ -139,6 +139,10 @@ type TransactionalJobRegistry interface { // See (*jobs.Registry).CheckPausepoint CheckPausepoint(name string) error + // UseLegacyGCJob indicate whether the legacy GC job should be used. + // This only matters for setting the initial RunningStatus. + UseLegacyGCJob(ctx context.Context) bool + // TODO(ajwerner): Deal with setting the running status to indicate // validating, backfilling, or generally performing metadata changes // and waiting for lease draining. diff --git a/pkg/sql/schemachanger/scexec/exec_mutation.go b/pkg/sql/schemachanger/scexec/exec_mutation.go index 9a9718501ad2..76f27e0cb08f 100644 --- a/pkg/sql/schemachanger/scexec/exec_mutation.go +++ b/pkg/sql/schemachanger/scexec/exec_mutation.go @@ -55,6 +55,7 @@ func executeDescriptorMutationOps(ctx context.Context, deps Dependencies, ops [] // scanned. dbZoneConfigsToDelete, gcJobRecords := mvs.gcJobs.makeRecords( deps.TransactionalJobRegistry().MakeJobID, + !deps.TransactionalJobRegistry().UseLegacyGCJob(ctx), ) if err := performBatchedCatalogWrites( ctx, diff --git a/pkg/sql/schemachanger/scexec/executor_external_test.go b/pkg/sql/schemachanger/scexec/executor_external_test.go index 10d8475d5388..1a7e8fc04d58 100644 --- a/pkg/sql/schemachanger/scexec/executor_external_test.go +++ b/pkg/sql/schemachanger/scexec/executor_external_test.go @@ -65,6 +65,7 @@ func (ti testInfra) newExecDeps( const kvTrace = true const schemaChangerJobID = 1 return scdeps.NewExecutorDependencies( + ti.settings, ti.lm.Codec(), &sessiondata.SessionData{}, txn, diff --git a/pkg/sql/schemachanger/scexec/gc_jobs.go b/pkg/sql/schemachanger/scexec/gc_jobs.go index b239a5f14aff..2c44cd55c2ee 100644 --- a/pkg/sql/schemachanger/scexec/gc_jobs.go +++ b/pkg/sql/schemachanger/scexec/gc_jobs.go @@ -82,7 +82,7 @@ func (gj *gcJobs) AddNewGCJobForIndex( // tables, their IDs will be in dbZoneConfigsToRemove and will not be mentioned // in any of the returned job records. func (gj gcJobs) makeRecords( - mkJobID func() jobspb.JobID, + mkJobID func() jobspb.JobID, useLegacyJob bool, ) (dbZoneConfigsToRemove catalog.DescriptorIDSet, gcJobRecords []jobs.Record) { type stmts struct { s []scop.StatementForDropJob @@ -146,7 +146,9 @@ func (gj gcJobs) makeRecords( addTableToJob(&s, &j, gj.tables[i]) } gcJobRecords = append(gcJobRecords, - createGCJobRecord(mkJobID(), formatStatements(&s), username.NodeUserName(), j)) + createGCJobRecord( + mkJobID(), formatStatements(&s), username.NodeUserName(), j, useLegacyJob, + )) } { var j jobspb.SchemaChangeGCDetails @@ -158,8 +160,9 @@ func (gj gcJobs) makeRecords( addTableToJob(&s, &j, t) } if len(j.Tables) > 0 { - gcJobRecords = append(gcJobRecords, - createGCJobRecord(mkJobID(), formatStatements(&s), username.NodeUserName(), j)) + gcJobRecords = append(gcJobRecords, createGCJobRecord( + mkJobID(), formatStatements(&s), username.NodeUserName(), j, useLegacyJob, + )) } } indexes := gj.indexes @@ -185,8 +188,9 @@ func (gj gcJobs) makeRecords( }) } if len(j.Indexes) > 0 { - gcJobRecords = append(gcJobRecords, - createGCJobRecord(mkJobID(), formatStatements(&s), username.NodeUserName(), j)) + gcJobRecords = append(gcJobRecords, createGCJobRecord( + mkJobID(), formatStatements(&s), username.NodeUserName(), j, useLegacyJob, + )) } } return dbZoneConfigsToRemove, gcJobRecords @@ -219,6 +223,7 @@ func createGCJobRecord( description string, userName username.SQLUsername, details jobspb.SchemaChangeGCDetails, + useLegacyJob bool, ) jobs.Record { descriptorIDs := make([]descpb.ID, 0) if len(details.Indexes) > 0 { @@ -233,6 +238,10 @@ func createGCJobRecord( descriptorIDs = append(descriptorIDs, details.ParentID) } } + runningStatus := jobs.RunningStatus("waiting for MVCC GC") + if useLegacyJob { + runningStatus = "waiting for GC TTL" + } return jobs.Record{ JobID: id, Description: "GC for " + description, @@ -240,7 +249,7 @@ func createGCJobRecord( DescriptorIDs: descriptorIDs, Details: details, Progress: jobspb.SchemaChangeGCProgress{}, - RunningStatus: "waiting for GC TTL", + RunningStatus: runningStatus, NonCancelable: true, } } diff --git a/pkg/sql/truncate.go b/pkg/sql/truncate.go index fc8a5970f119..ad4778caa0c1 100644 --- a/pkg/sql/truncate.go +++ b/pkg/sql/truncate.go @@ -14,6 +14,7 @@ import ( "context" "math/rand" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" @@ -246,7 +247,12 @@ func (p *planner) truncateTable(ctx context.Context, id descpb.ID, jobDesc strin Indexes: droppedIndexes, ParentID: tableDesc.ID, } - record := CreateGCJobRecord(jobDesc, p.User(), details) + record := CreateGCJobRecord( + jobDesc, p.User(), details, + !p.execCfg.Settings.Version.IsActive( + ctx, clusterversion.UseDelRangeInGCJob, + ), + ) if _, err := p.ExecCfg().JobRegistry.CreateAdoptableJobWithTxn( ctx, record, p.ExecCfg().JobRegistry.MakeJobID(), p.txn); err != nil { return err diff --git a/pkg/sql/unsplit_range_test.go b/pkg/sql/unsplit_range_test.go index 8442f2cbee86..3f0483dc3940 100644 --- a/pkg/sql/unsplit_range_test.go +++ b/pkg/sql/unsplit_range_test.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils" @@ -269,6 +270,9 @@ func TestUnsplitRanges(t *testing.T) { run := func(t *testing.T, tc testCase) { params, _ := tests.CreateTestServerParams() params.Knobs.JobsTestingKnobs = jobs.NewTestingKnobsWithShortIntervals() + params.Knobs.GCJob = &sql.GCJobTestingKnobs{ + SkipWaitingForMVCCGC: true, + } defer gcjob.SetSmallMaxGCIntervalForTest()() From de12cee396210cf905a0677b5ecc76773a14990a Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Tue, 9 Aug 2022 23:57:30 -0400 Subject: [PATCH 8/8] upgrades,clusterversion: write a migration to wait for the GC job to upgrade Release note: None --- .../settings/settings-for-tenants.txt | 2 +- docs/generated/settings/settings.html | 2 +- pkg/clusterversion/cockroach_versions.go | 7 + pkg/clusterversion/key_string.go | 5 +- pkg/sql/gcjob/gc_job.go | 20 ++- pkg/sql/gcjob/gcjobnotifier/BUILD.bazel | 1 + pkg/sql/gcjob/gcjobnotifier/notifier.go | 17 +++ pkg/sql/gcjob/index_garbage_collection.go | 12 +- pkg/upgrade/upgrades/BUILD.bazel | 6 + pkg/upgrade/upgrades/upgrades.go | 5 + .../upgrades/wait_for_del_range_in_gc_job.go | 134 ++++++++++++++++++ .../wait_for_del_range_in_gc_job_test.go | 128 +++++++++++++++++ 12 files changed, 330 insertions(+), 9 deletions(-) create mode 100644 pkg/upgrade/upgrades/wait_for_del_range_in_gc_job.go create mode 100644 pkg/upgrade/upgrades/wait_for_del_range_in_gc_job_test.go diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 904a2a935cc9..58d825f01013 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -286,4 +286,4 @@ trace.jaeger.agent string the address of a Jaeger agent to receive traces using trace.opentelemetry.collector string address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.span_registry.enabled boolean true if set, ongoing traces can be seen at https:///#/debug/tracez trace.zipkin.collector string the address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -version version 22.1-56 set the active cluster version in the format '.' +version version 22.1-58 set the active cluster version in the format '.' diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index c437c7e7c359..70e5db376ca8 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -218,6 +218,6 @@ trace.opentelemetry.collectorstringaddress of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.span_registry.enabledbooleantrueif set, ongoing traces can be seen at https:///#/debug/tracez trace.zipkin.collectorstringthe address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -versionversion22.1-56set the active cluster version in the format '.' +versionversion22.1-58set the active cluster version in the format '.' diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 89edc04d4a0b..f2c553097ba9 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -317,6 +317,9 @@ const ( // and then waits for the data to be removed automatically before removing // the descriptor and zone configurations. UseDelRangeInGCJob + // WaitedForDelRangeInGCJob corresponds to the migration which waits for + // the GC jobs to adopt the use of DelRange with tombstones. + WaitedForDelRangeInGCJob // ************************************************* // Step (1): Add new versions here. @@ -535,6 +538,10 @@ var versionsSingleton = keyedVersions{ Key: UseDelRangeInGCJob, Version: roachpb.Version{Major: 22, Minor: 1, Internal: 56}, }, + { + Key: WaitedForDelRangeInGCJob, + Version: roachpb.Version{Major: 22, Minor: 1, Internal: 58}, + }, // ************************************************* // Step (2): Add new versions here. // Do not add new versions to a patch release. diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index 650c02f410a1..e3ad7ef650dc 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -53,11 +53,12 @@ func _() { _ = x[RoleOptionsIDColumnIsBackfilled-42] _ = x[SetRoleOptionsUserIDColumnNotNull-43] _ = x[UseDelRangeInGCJob-44] + _ = x[WaitedForDelRangeInGCJob-45] } -const _Key_name = "V21_2Start22_1ProbeRequestPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreEnablePebbleFormatVersionBlockPropertiesEnableLeaseHolderRemovalChangefeedIdlenessRowLevelTTLEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsSuperRegionsEnableNewChangefeedOptionsV22_1Start22_2LocalTimestampsPebbleFormatSplitUserKeysMarkedCompactedEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocalitySystemExternalConnectionsTableAlterSystemStatementStatisticsAddIndexRecommendationsRoleIDSequenceAddSystemUserIDColumnSystemUsersIDColumnIsBackfilledSetSystemUsersUserIDColumnNotNullSQLSchemaTelemetryScheduledJobsSchemaChangeSupportsCreateFunctionDeleteRequestReturnKeyPebbleFormatPrePebblev1MarkedRoleOptionsTableHasIDColumnRoleOptionsIDColumnIsBackfilledSetRoleOptionsUserIDColumnNotNullUseDelRangeInGCJob" +const _Key_name = "V21_2Start22_1ProbeRequestPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreEnablePebbleFormatVersionBlockPropertiesEnableLeaseHolderRemovalChangefeedIdlenessRowLevelTTLEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsSuperRegionsEnableNewChangefeedOptionsV22_1Start22_2LocalTimestampsPebbleFormatSplitUserKeysMarkedCompactedEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocalitySystemExternalConnectionsTableAlterSystemStatementStatisticsAddIndexRecommendationsRoleIDSequenceAddSystemUserIDColumnSystemUsersIDColumnIsBackfilledSetSystemUsersUserIDColumnNotNullSQLSchemaTelemetryScheduledJobsSchemaChangeSupportsCreateFunctionDeleteRequestReturnKeyPebbleFormatPrePebblev1MarkedRoleOptionsTableHasIDColumnRoleOptionsIDColumnIsBackfilledSetRoleOptionsUserIDColumnNotNullUseDelRangeInGCJobWaitedForDelRangeInGCJob" -var _Key_index = [...]uint16{0, 5, 14, 26, 54, 84, 112, 133, 173, 197, 215, 226, 250, 274, 296, 308, 334, 339, 348, 363, 403, 437, 471, 493, 513, 532, 565, 584, 604, 625, 660, 694, 724, 777, 791, 812, 843, 876, 907, 941, 963, 992, 1019, 1050, 1083, 1101} +var _Key_index = [...]uint16{0, 5, 14, 26, 54, 84, 112, 133, 173, 197, 215, 226, 250, 274, 296, 308, 334, 339, 348, 363, 403, 437, 471, 493, 513, 532, 565, 584, 604, 625, 660, 694, 724, 777, 791, 812, 843, 876, 907, 941, 963, 992, 1019, 1050, 1083, 1101, 1125} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/sql/gcjob/gc_job.go b/pkg/sql/gcjob/gc_job.go index a1f08f0c96e4..8681a19aad44 100644 --- a/pkg/sql/gcjob/gc_job.go +++ b/pkg/sql/gcjob/gc_job.go @@ -341,7 +341,7 @@ func waitForGC( switch { case details.Indexes != nil: return errors.Wrap( - waitForIndexGC(ctx, execCfg, details.ParentID, progress), + deleteIndexZoneConfigsAfterGC(ctx, execCfg, details.ParentID, progress), "attempting to delete index data", ) case details.Tables != nil: @@ -415,6 +415,16 @@ func (r schemaChangeGCResumer) legacyWaitAndClearTableData( gossipUpdateC, cleanup := execCfg.GCJobNotifier.AddNotifyee(ctx) defer cleanup() + + // Now that we've registered to be notified, check to see if we raced + // with the new version becoming active. + // + // TODO(ajwerner): Adopt the DeleteRange protocol for tenant GC. + if details.Tenant == nil && + execCfg.Settings.Version.IsActive(ctx, clusterversion.UseDelRangeInGCJob) { + return r.deleteDataAndWaitForGC(ctx, execCfg, details, progress) + } + var timerDuration time.Duration ts := timeutil.DefaultTimeSource{} @@ -425,6 +435,14 @@ func (r schemaChangeGCResumer) legacyWaitAndClearTableData( ); err != nil { return err } + // We'll be notified if the new version becomes active, so check and + // see if it's now time to change to the new protocol. + // + // TODO(ajwerner): Adopt the DeleteRange protocol for tenant GC. + if details.Tenant == nil && + execCfg.Settings.Version.IsActive(ctx, clusterversion.UseDelRangeInGCJob) { + return r.deleteDataAndWaitForGC(ctx, execCfg, details, progress) + } // Refresh the status of all elements in case any GC TTLs have changed. var expired bool diff --git a/pkg/sql/gcjob/gcjobnotifier/BUILD.bazel b/pkg/sql/gcjob/gcjobnotifier/BUILD.bazel index 677f1b4bf45c..8f00a5527975 100644 --- a/pkg/sql/gcjob/gcjobnotifier/BUILD.bazel +++ b/pkg/sql/gcjob/gcjobnotifier/BUILD.bazel @@ -7,6 +7,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/gcjob/gcjobnotifier", visibility = ["//visibility:public"], deps = [ + "//pkg/clusterversion", "//pkg/config", "//pkg/gossip", "//pkg/keys", diff --git a/pkg/sql/gcjob/gcjobnotifier/notifier.go b/pkg/sql/gcjob/gcjobnotifier/notifier.go index 17d5da8f8a8d..823c35d8c50b 100644 --- a/pkg/sql/gcjob/gcjobnotifier/notifier.go +++ b/pkg/sql/gcjob/gcjobnotifier/notifier.go @@ -17,6 +17,7 @@ package gcjobnotifier import ( "context" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" @@ -69,6 +70,8 @@ func (n *Notifier) SystemConfigProvider() config.SystemConfigProvider { func noopFunc() {} // AddNotifyee should be called prior to the first reading of the system config. +// The returned channel will also receive a notification if the cluster version +// UseDelRangeInGCJob is activated. // // TODO(lucy,ajwerner): Currently we're calling refreshTables on every zone // config update to any table. We should really be only updating a cached @@ -141,10 +144,21 @@ func (n *Notifier) Start(ctx context.Context) { func (n *Notifier) run(_ context.Context) { defer n.markStopped() systemConfigUpdateCh, _ := n.provider.RegisterSystemConfigChannel() + var haveNotified bool + versionSettingChanged := make(chan struct{}, 1) + versionBeingWaited := clusterversion.ByKey(clusterversion.UseDelRangeInGCJob) + n.settings.Version.SetOnChange(func(ctx context.Context, newVersion clusterversion.ClusterVersion) { + if !haveNotified && versionBeingWaited.LessEq(newVersion.Version) { + haveNotified = true + versionSettingChanged <- struct{}{} + } + }) for { select { case <-n.stopper.ShouldQuiesce(): return + case <-versionSettingChanged: + n.notify() case <-systemConfigUpdateCh: n.maybeNotify() } @@ -170,7 +184,10 @@ func (n *Notifier) maybeNotify() { if !zoneConfigUpdated { return } + n.notify() +} +func (n *Notifier) notify() { for c := range n.mu.notifyees { select { case c <- struct{}{}: diff --git a/pkg/sql/gcjob/index_garbage_collection.go b/pkg/sql/gcjob/index_garbage_collection.go index 351fdd9fbf44..8e63f08d11cd 100644 --- a/pkg/sql/gcjob/index_garbage_collection.go +++ b/pkg/sql/gcjob/index_garbage_collection.go @@ -170,7 +170,7 @@ func clearIndex( return clearOrDeleteSpanData(ctx, execCfg.DB, execCfg.DistSender, rSpan) } -func waitForIndexGC( +func deleteIndexZoneConfigsAfterGC( ctx context.Context, execCfg *sql.ExecutorConfig, parentID descpb.ID, @@ -178,7 +178,7 @@ func waitForIndexGC( ) error { for _, index := range progress.Indexes { - if index.Status != jobspb.SchemaChangeGCProgress_CLEARING { + if index.Status == jobspb.SchemaChangeGCProgress_CLEARED { continue } @@ -212,10 +212,14 @@ func waitForIndexGC( ctx, txn, execCfg, descriptors, freshParentTableDesc, []uint32{uint32(index.IndexID)}, ) } - if err := sql.DescsTxn(ctx, execCfg, removeIndexZoneConfigs); err != nil { + err := sql.DescsTxn(ctx, execCfg, removeIndexZoneConfigs) + switch { + case errors.Is(err, catalog.ErrDescriptorNotFound): + log.Infof(ctx, "removing index %d zone config from table %d failed: %v", + index.IndexID, parentID, err) + case err != nil: return errors.Wrapf(err, "removing index %d zone configs", index.IndexID) } - markIndexGCed( ctx, index.IndexID, progress, jobspb.SchemaChangeGCProgress_CLEARED, ) diff --git a/pkg/upgrade/upgrades/BUILD.bazel b/pkg/upgrade/upgrades/BUILD.bazel index e12248f180af..10efa66149bb 100644 --- a/pkg/upgrade/upgrades/BUILD.bazel +++ b/pkg/upgrade/upgrades/BUILD.bazel @@ -19,12 +19,14 @@ go_library( "system_users_role_id_migration.go", "upgrade_sequence_to_be_referenced_by_ID.go", "upgrades.go", + "wait_for_del_range_in_gc_job.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/upgrade/upgrades", visibility = ["//visibility:public"], deps = [ "//pkg/clusterversion", "//pkg/jobs", + "//pkg/jobs/jobspb", "//pkg/keys", "//pkg/kv", "//pkg/security/username", @@ -40,6 +42,8 @@ go_library( "//pkg/sql/catalog/systemschema", "//pkg/sql/catalog/tabledesc", "//pkg/sql/parser", + "//pkg/sql/pgwire/pgcode", + "//pkg/sql/pgwire/pgerror", "//pkg/sql/sem/tree", "//pkg/sql/sessiondata", "//pkg/sql/sqlutil", @@ -47,6 +51,7 @@ go_library( "//pkg/util/hlc", "//pkg/util/log", "//pkg/util/protoutil", + "//pkg/util/retry", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_redact//:redact", "@com_github_kr_pretty//:pretty", @@ -71,6 +76,7 @@ go_test( "sampled_stmt_diagnostics_requests_test.go", "system_privileges_test.go", "upgrade_sequence_to_be_referenced_by_ID_external_test.go", + "wait_for_del_range_in_gc_job_test.go", ], data = glob(["testdata/**"]), embed = [":upgrades"], diff --git a/pkg/upgrade/upgrades/upgrades.go b/pkg/upgrade/upgrades/upgrades.go index 3be36bd3b050..992567be0d7e 100644 --- a/pkg/upgrade/upgrades/upgrades.go +++ b/pkg/upgrade/upgrades/upgrades.go @@ -143,6 +143,11 @@ var upgrades = []upgrade.Upgrade{ NoPrecondition, setSystemRoleOptionsUserIDColumnNotNull, ), + upgrade.NewTenantUpgrade("ensure all GC jobs send DeleteRange requests", + toCV(clusterversion.WaitedForDelRangeInGCJob), + checkForPausedGCJobs, + waitForDelRangeInGCJob, + ), } func init() { diff --git a/pkg/upgrade/upgrades/wait_for_del_range_in_gc_job.go b/pkg/upgrade/upgrades/wait_for_del_range_in_gc_job.go new file mode 100644 index 000000000000..e194cdef5b21 --- /dev/null +++ b/pkg/upgrade/upgrades/wait_for_del_range_in_gc_job.go @@ -0,0 +1,134 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package upgrades + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/upgrade" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/retry" + "github.com/cockroachdb/errors" +) + +// waitForDelRangeInGCJob ensures that any running GC jobs have adopted the new +// DeleteRange protocol. +func waitForDelRangeInGCJob( + ctx context.Context, _ clusterversion.ClusterVersion, deps upgrade.TenantDeps, _ *jobs.Job, +) error { + for r := retry.StartWithCtx(ctx, retry.Options{}); r.Next(); { + jobIDs, err := collectJobIDsFromQuery( + ctx, deps.InternalExecutor, "wait-for-gc-job-upgrades", ` + WITH jobs AS ( + SELECT * + FROM ( + SELECT id, + crdb_internal.pb_to_json('payload', payload) AS pl, + crdb_internal.pb_to_json('progress', progress) AS progress, + `+jobs.NextRunClause+` AS next_run, + args.max_delay as max_delay + FROM system.jobs, + ( + SELECT ( + SELECT value + FROM crdb_internal.cluster_settings + WHERE variable = 'jobs.registry.retry.initial_delay' + )::INTERVAL::FLOAT8 AS initial_delay, + ( + SELECT value + FROM crdb_internal.cluster_settings + WHERE variable = 'jobs.registry.retry.max_delay' + )::INTERVAL::FLOAT8 AS max_delay + ) AS args + WHERE status IN `+jobs.NonTerminalStatusTupleString+` + ) + WHERE (pl->'schemaChangeGC') IS NOT NULL + AND ( + next_run < (now() + '5m'::INTERVAL) + OR max_delay < '5m'::INTERVAL::FLOAT8 + ) + ), + tables AS ( + SELECT id, + json_array_elements(pl->'schemaChangeGC'->'tables') AS pl, + json_array_elements(progress->'schemaChangeGC'->'tables') AS progress + FROM jobs + ), + indexes AS ( + SELECT id, + json_array_elements(pl->'schemaChangeGC'->'indexes'), + json_array_elements(progress->'schemaChangeGC'->'indexes') + FROM jobs + ), + elements AS (SELECT * FROM tables UNION ALL SELECT * FROM indexes) + SELECT id + FROM elements + WHERE COALESCE(progress->>'status' NOT IN ('WAITING_FOR_MVCC_GC', 'CLEARED'), true) +GROUP BY id; +`) + if err != nil || len(jobIDs) == 0 { + return err + } + log.Infof(ctx, "waiting for %d GC jobs to adopt the new protocol: %v", len(jobIDs), jobIDs) + } + return ctx.Err() +} + +// If there are any paused GC jobs, they are going to make this migration +// take forever, so we should detect them and give the user the opportunity +// to unpause them with a clear error. +func checkForPausedGCJobs( + ctx context.Context, version clusterversion.ClusterVersion, deps upgrade.TenantDeps, +) (retErr error) { + jobIDs, err := collectJobIDsFromQuery( + ctx, deps.InternalExecutor, "check-for-paused-gc-jobs", ` +SELECT job_id + FROM crdb_internal.jobs + WHERE job_type = 'SCHEMA CHANGE GC' + AND status IN ('paused', 'pause-requested')`) + if err != nil { + return err + } + if len(jobIDs) > 0 { + return errors.WithHint(pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, + "paused GC jobs prevent upgrading GC job behavior: %v", jobIDs), + "unpause the jobs to allow the upgrade to proceed") + } + return nil +} + +// collectJobIDsFromQuery is a helper to execute a query which returns rows +// where the first column is a jobID and returns the job IDs from those rows. +func collectJobIDsFromQuery( + ctx context.Context, ie sqlutil.InternalExecutor, opName string, query string, +) (jobIDs []jobspb.JobID, retErr error) { + it, err := ie.QueryIteratorEx(ctx, opName, nil, /* txn */ + sessiondata.InternalExecutorOverride{ + User: username.NodeUserName(), + }, query) + if err != nil { + return nil, err + } + defer func() { retErr = errors.CombineErrors(retErr, it.Close()) }() + for ok, err := it.Next(ctx); ok && err == nil; ok, err = it.Next(ctx) { + jobIDs = append(jobIDs, jobspb.JobID(tree.MustBeDInt(it.Cur()[0]))) + } + return jobIDs, err +} diff --git a/pkg/upgrade/upgrades/wait_for_del_range_in_gc_job_test.go b/pkg/upgrade/upgrades/wait_for_del_range_in_gc_job_test.go new file mode 100644 index 000000000000..f8c98867e5cd --- /dev/null +++ b/pkg/upgrade/upgrades/wait_for_del_range_in_gc_job_test.go @@ -0,0 +1,128 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package upgrades_test + +import ( + "context" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/stretchr/testify/require" +) + +func TestWaitForDelRangeInGCJob(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + var ( + v0 = clusterversion.ByKey(clusterversion.UseDelRangeInGCJob - 1) + v1 = clusterversion.ByKey(clusterversion.WaitedForDelRangeInGCJob) + ) + + ctx := context.Background() + settings := cluster.MakeTestingClusterSettingsWithVersions(v1, v0, false /* initializeVersion */) + require.NoError(t, clusterversion.Initialize(ctx, v0, &settings.SV)) + + testServer, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{ + Settings: settings, + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + DisableAutomaticVersionUpgrade: make(chan struct{}), + BinaryVersionOverride: v0, + }, + JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), + }, + }) + defer testServer.Stopper().Stop(ctx) + + tdb := sqlutils.MakeSQLRunner(sqlDB) + tdb.Exec(t, `SET CLUSTER SETTING jobs.debug.pausepoints = $1`, + `gcjob.before_resume`) + tdb.Exec(t, `SET CLUSTER SETTING jobs.registry.retry.max_delay = '10s'`) + execStmts := func(t *testing.T, stmts ...string) { + for _, stmt := range stmts { + tdb.Exec(t, stmt) + } + } + execStmts(t, + "CREATE DATABASE db", + "CREATE TABLE db.foo(i int primary key, j int)", + "INSERT INTO db.foo(i) SELECT * FROM generate_series(1, 1000)", + "CREATE TABLE foo (i int primary key, j int, index idx(j))", + "INSERT INTO foo(i) SELECT * FROM generate_series(1, 1000)", + ) + + var beforeDrop string + tdb.QueryRow(t, "SELECT cluster_logical_timestamp()").Scan(&beforeDrop) + + // Grab the table ID for db.foo and the table id and index ID for foo. + var dbFooID, fooID, idxID uint32 + tdb.QueryRow(t, ` +SELECT 'db.foo'::REGCLASS::INT8, + 'foo'::REGCLASS::INT8, + ( + SELECT index_id + FROM crdb_internal.table_indexes + WHERE descriptor_id = 'foo'::REGCLASS AND index_name = 'idx' + );`).Scan(&dbFooID, &fooID, &idxID) + execStmts(t, + "DROP DATABASE db CASCADE", + "DROP INDEX foo@idx", + ) + + // One for the index, one for the database. + tdb.CheckQueryResultsRetry(t, ` +SELECT count(*) + FROM crdb_internal.jobs + WHERE job_type = 'SCHEMA CHANGE GC' + AND status = 'paused'`, + [][]string{{"2"}}) + tdb.ExpectErr(t, `verifying precondition for version 22.1-\d+: `+ + `paused GC jobs prevent upgrading GC job behavior: \[\d+ \d+]`, + "SET CLUSTER SETTING version = crdb_internal.node_executable_version()") + + tdb.Exec(t, `SET CLUSTER SETTING jobs.debug.pausepoints = DEFAULT`) + + // Now resume the jobs. + tdb.Exec(t, + "RESUME JOBS SELECT job_id FROM crdb_internal.jobs WHERE status = 'paused'") + + // Upgrade the version. Then ensure that the data has been + // removed from the span but the jobs are still running. + tdb.Exec(t, "SET CLUSTER SETTING version = crdb_internal.node_executable_version()") + + codec := testServer.ExecutorConfig().(sql.ExecutorConfig).Codec + { + dbFooStart := codec.TablePrefix(dbFooID) + res, err := kvDB.Scan(ctx, dbFooStart, dbFooStart.PrefixEnd(), 1) + require.NoError(t, err) + require.Len(t, res, 0) + } + { + idxStart := codec.IndexPrefix(fooID, idxID) + res, err := kvDB.Scan(ctx, idxStart, idxStart.PrefixEnd(), 1) + require.NoError(t, err) + require.Len(t, res, 0) + } + + // Make sure that there is still MVCC history. + tdb.CheckQueryResults(t, "SELECT count(*) FROM foo@idx AS OF SYSTEM TIME "+beforeDrop, + [][]string{{"1000"}}) +}