diff --git a/pkg/ccl/backupccl/BUILD.bazel b/pkg/ccl/backupccl/BUILD.bazel index db1839ece26d..29ce7fa51929 100644 --- a/pkg/ccl/backupccl/BUILD.bazel +++ b/pkg/ccl/backupccl/BUILD.bazel @@ -169,6 +169,7 @@ go_test( "create_scheduled_backup_test.go", "datadriven_test.go", "full_cluster_backup_restore_test.go", + "generative_split_and_scatter_processor_test.go", "key_rewriter_test.go", "main_test.go", "partitioned_backup_test.go", diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index b4386a05d0bb..03742c60d0d6 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -273,6 +273,8 @@ func TestBackupRestoreJobTagAndLabel(t *testing.T) { backupAndRestore(ctx, t, tc, []string{localFoo}, []string{localFoo}, numAccounts) + mu.Lock() + defer mu.Unlock() require.True(t, found) } diff --git a/pkg/ccl/backupccl/generative_split_and_scatter_processor.go b/pkg/ccl/backupccl/generative_split_and_scatter_processor.go index c28c40850831..dbb6147de706 100644 --- a/pkg/ccl/backupccl/generative_split_and_scatter_processor.go +++ b/pkg/ccl/backupccl/generative_split_and_scatter_processor.go @@ -321,14 +321,22 @@ func runGenerativeSplitAndScatter( idx++ if len(chunk.entries) == int(spec.ChunkSize) { chunk.splitKey = entry.Span.Key - restoreEntryChunksCh <- chunk + select { + case <-ctx.Done(): + return ctx.Err() + case restoreEntryChunksCh <- chunk: + } chunk = restoreEntryChunk{} } chunk.entries = append(chunk.entries, entry) } if len(chunk.entries) > 0 { - restoreEntryChunksCh <- chunk + select { + case <-ctx.Done(): + return ctx.Err() + case restoreEntryChunksCh <- chunk: + } } return nil }) @@ -421,6 +429,12 @@ func runGenerativeSplitAndScatter( node: chunkDestination, } + if restoreKnobs, ok := flowCtx.TestingKnobs().BackupRestoreTestingKnobs.(*sql.BackupRestoreTestingKnobs); ok { + if restoreKnobs.RunAfterSplitAndScatteringEntry != nil { + restoreKnobs.RunAfterSplitAndScatteringEntry(ctx) + } + } + select { case <-ctx.Done(): return ctx.Err() diff --git a/pkg/ccl/backupccl/generative_split_and_scatter_processor_test.go b/pkg/ccl/backupccl/generative_split_and_scatter_processor_test.go new file mode 100644 index 000000000000..a19d53bce1d6 --- /dev/null +++ b/pkg/ccl/backupccl/generative_split_and_scatter_processor_test.go @@ -0,0 +1,140 @@ +// Copyright 2023 The Cockroach Authors. +// +// Licensed as a CockroachDB Enterprise file under the Cockroach Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt + +package backupccl + +import ( + "context" + "testing" + + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" + "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/stretchr/testify/require" +) + +// TestRunGenerativeSplitAndScatterContextCancel verifies that +// runGenerativeSplitAndScatter can be interrupted by canceling the supplied +// context. This test would time out if the context cancellation does not +// interrupt the function. +func TestRunGenerativeSplitAndScatterContextCancel(t *testing.T) { + defer leaktest.AfterTest(t)() + + const numAccounts = 1000 + const localFoo = "nodelocal://0/foo" + ctx := context.Background() + ctx, cancel := context.WithCancel(ctx) + tc, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, + InitManualReplication) + defer cleanupFn() + + st := cluster.MakeTestingClusterSettings() + evalCtx := eval.MakeTestingEvalContext(st) + + testDiskMonitor := execinfra.NewTestDiskMonitor(ctx, st) + defer testDiskMonitor.Stop(ctx) + + // Set up the test so that the test context is canceled after the first entry + // has been processed by the generative split and scatterer. + s0 := tc.Server(0) + registry := tc.Server(0).JobRegistry().(*jobs.Registry) + execCfg := s0.ExecutorConfig().(sql.ExecutorConfig) + flowCtx := execinfra.FlowCtx{ + Cfg: &execinfra.ServerConfig{ + Settings: st, + DB: s0.InternalDB().(descs.DB), + JobRegistry: registry, + ExecutorConfig: &execCfg, + TestingKnobs: execinfra.TestingKnobs{ + BackupRestoreTestingKnobs: &sql.BackupRestoreTestingKnobs{ + RunAfterSplitAndScatteringEntry: func(ctx context.Context) { + cancel() + }, + }, + }, + }, + EvalCtx: &evalCtx, + Mon: evalCtx.TestingMon, + DiskMonitor: testDiskMonitor, + NodeID: evalCtx.NodeID, + } + + sqlDB.Exec(t, `SET CLUSTER SETTING bulkio.backup.file_size = '1'`) + sqlDB.Exec(t, `BACKUP INTO $1`, localFoo) + + backups := sqlDB.QueryStr(t, `SHOW BACKUPS IN $1`, localFoo) + require.Equal(t, 1, len(backups)) + uri := localFoo + "/" + backups[0][0] + + codec := keys.MakeSQLCodec(s0.RPCContext().TenantID) + backupTableDesc := desctestutils.TestingGetPublicTableDescriptor(tc.Servers[0].DB(), codec, "data", "bank") + backupStartKey := backupTableDesc.PrimaryIndexSpan(codec).Key + + spec := makeTestingGenerativeSplitAndScatterSpec( + []string{uri}, + []roachpb.Span{{ + Key: backupStartKey, + EndKey: backupStartKey.PrefixEnd(), + }}, + ) + + oldID := backupTableDesc.GetID() + newID := backupTableDesc.GetID() + 1 + newDesc := protoutil.Clone(backupTableDesc.TableDesc()).(*descpb.TableDescriptor) + newDesc.ID = newID + tableRekeys := []execinfrapb.TableRekey{ + { + OldID: uint32(oldID), + NewDesc: mustMarshalDesc(t, newDesc), + }, + } + + kr, err := MakeKeyRewriterFromRekeys(keys.SystemSQLCodec, tableRekeys, nil, false) + require.NoError(t, err) + + chunkSplitScatterers := []splitAndScatterer{makeSplitAndScatterer(flowCtx.Cfg.DB.KV(), kr)} + chunkEntrySpliterScatterers := []splitAndScatterer{makeSplitAndScatterer(flowCtx.Cfg.DB.KV(), kr)} + + // Large enough so doneScatterCh never blocks. + doneScatterCh := make(chan entryNode, 1000) + err = runGenerativeSplitAndScatter(ctx, &flowCtx, &spec, chunkSplitScatterers, chunkEntrySpliterScatterers, doneScatterCh) + + require.Error(t, err, "context canceled") +} + +func makeTestingGenerativeSplitAndScatterSpec( + backupURIs []string, requiredSpans []roachpb.Span, +) execinfrapb.GenerativeSplitAndScatterSpec { + return execinfrapb.GenerativeSplitAndScatterSpec{ + ValidateOnly: false, + URIs: backupURIs, + Encryption: nil, + EndTime: hlc.Timestamp{}, + Spans: requiredSpans, + BackupLocalityInfo: nil, + HighWater: nil, + UserProto: "", + ChunkSize: 1, + TargetSize: 1, + NumEntries: 1, + NumNodes: 1, + JobID: 0, + UseSimpleImportSpans: false, + } +} diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index f439b365b15a..770d5f2e0b85 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -228,9 +228,6 @@ const ( TODODelete_V22_2AlterSystemSQLInstancesAddLocality // TODODelete_V22_2SystemExternalConnectionsTable adds system.external_connections table. TODODelete_V22_2SystemExternalConnectionsTable - // TODODelete_V22_2AlterSystemStatementStatisticsAddIndexRecommendations adds an - // index_recommendations column to the system.statement_statistics table. - TODODelete_V22_2AlterSystemStatementStatisticsAddIndexRecommendations // TODODelete_V22_2RoleIDSequence is the version where the system.role_id_sequence exists. TODODelete_V22_2RoleIDSequence // TODODelete_V22_2AddSystemUserIDColumn is the version where the system.users table has @@ -539,10 +536,6 @@ var rawVersionsSingleton = keyedVersions{ Key: TODODelete_V22_2SystemExternalConnectionsTable, Version: roachpb.Version{Major: 22, Minor: 1, Internal: 30}, }, - { - Key: TODODelete_V22_2AlterSystemStatementStatisticsAddIndexRecommendations, - Version: roachpb.Version{Major: 22, Minor: 1, Internal: 32}, - }, { Key: TODODelete_V22_2RoleIDSequence, Version: roachpb.Version{Major: 22, Minor: 1, Internal: 34}, diff --git a/pkg/kv/kvserver/replica_proposal.go b/pkg/kv/kvserver/replica_proposal.go index dfe25104af8d..1c50959fc1de 100644 --- a/pkg/kv/kvserver/replica_proposal.go +++ b/pkg/kv/kvserver/replica_proposal.go @@ -351,10 +351,17 @@ func (r *Replica) leasePostApplyLocked( requiresExpirationBasedLease := r.requiresExpiringLeaseRLocked() hasExpirationBasedLease := newLease.Type() == roachpb.LeaseExpiration + now := r.store.Clock().NowAsClockTimestamp() + + // NB: ProposedTS is non-nil in practice, but we never fully migrated it + // in so we need to assume that it can be nil. + if iAmTheLeaseHolder && leaseChangingHands && newLease.ProposedTS != nil { + maybeLogSlowLeaseApplyWarning(ctx, time.Duration(now.WallTime-newLease.ProposedTS.WallTime), prevLease, newLease) + } + // Gossip the first range whenever its lease is acquired. We check to make // sure the lease is active so that a trailing replica won't process an old // lease request and attempt to gossip the first range. - now := r.store.Clock().NowAsClockTimestamp() if leaseChangingHands && iAmTheLeaseHolder && r.IsFirstRange() && r.ownsValidLeaseRLocked(ctx, now) { r.gossipFirstRangeLocked(ctx) } @@ -444,6 +451,49 @@ func (r *Replica) leasePostApplyLocked( } } +// maybeLogSlowLeaseApplyWarning is called when the lease changes hands on the +// new leaseholder. It logs if either the new lease was proposed well before it +// became visible on the leaseholder (indicating replication lag) or if the +// previous lease looks like we transferred a lease to a behind/offline replica. +func maybeLogSlowLeaseApplyWarning( + ctx context.Context, newLeaseAppDelay time.Duration, prevLease, newLease *roachpb.Lease, +) { + const slowLeaseApplyWarnThreshold = 500 * time.Millisecond + if newLeaseAppDelay > slowLeaseApplyWarnThreshold { + // If we hold the lease now and the lease was proposed "earlier", there + // must have been replication lag, and possibly reads and/or writes were + // delayed. + // + // We see this most commonly with lease transfers targeting a behind replica, + // or, in the worst case, a snapshot. We are constantly improving our + // heuristics for avoiding that[^1] but if it does happen it's good to know + // from the logs. + // + // In the case of a lease transfer, the two timestamps compared below are from + // different clocks, so there could be skew. We just pretend this is not the + // case, which is good enough here. + // + // [^1]: https://github.com/cockroachdb/cockroach/pull/82758 + log.Warningf(ctx, + "lease %v active after replication lag of ~%.2fs; foreground traffic may have been impacted [prev=%v]", + newLease, newLeaseAppDelay.Seconds(), prevLease, + ) + } else if prevLease.Type() == roachpb.LeaseExpiration && + newLease.Type() == roachpb.LeaseEpoch && + newLease.AcquisitionType == roachpb.LeaseAcquisitionType_Request { + // If the previous lease is expiration-based, but the new lease is not and + // the acquisition was non-cooperative, it is likely that a lease transfer + // (which is expiration-based) went to a follower that then couldn't hold + // the lease alive (for example, didn't apply it in time for it to + // actually serve any traffic). The result was likely an outage which + // resolves right now, so log to point this out. + log.Warningf(ctx, + "lease %v expired before being followed by lease %s; foreground traffic may have been impacted", + prevLease, newLease, + ) + } +} + var addSSTPreApplyWarn = struct { threshold time.Duration log.EveryN diff --git a/pkg/server/combined_statement_stats.go b/pkg/server/combined_statement_stats.go index 4d2da360c3ee..01e042fd8b49 100644 --- a/pkg/server/combined_statement_stats.go +++ b/pkg/server/combined_statement_stats.go @@ -17,7 +17,6 @@ import ( "strings" "time" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" @@ -725,23 +724,6 @@ func getStatementDetailsPerPlanHash( query := fmt.Sprintf( `SELECT - plan_hash, - (statistics -> 'statistics' -> 'planGists'->>0) as plan_gist, - crdb_internal.merge_stats_metadata(array_agg(metadata)) AS metadata, - crdb_internal.merge_statement_stats(array_agg(statistics)) AS statistics, - max(sampled_plan) as sampled_plan, - aggregation_interval - FROM crdb_internal.statement_statistics %s - GROUP BY - plan_hash, - plan_gist, - aggregation_interval - LIMIT $%d`, whereClause, len(args)+1) - expectedNumDatums := 6 - - if settings.Version.IsActive(ctx, clusterversion.TODODelete_V22_2AlterSystemStatementStatisticsAddIndexRecommendations) { - query = fmt.Sprintf( - `SELECT plan_hash, (statistics -> 'statistics' -> 'planGists'->>0) as plan_gist, crdb_internal.merge_stats_metadata(array_agg(metadata)) AS metadata, @@ -756,8 +738,7 @@ func getStatementDetailsPerPlanHash( aggregation_interval, index_recommendations LIMIT $%d`, whereClause, len(args)+1) - expectedNumDatums = 7 - } + expectedNumDatums := 7 args = append(args, limit) diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index b5c347bc9765..532d9384d9d5 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -1632,6 +1632,10 @@ type BackupRestoreTestingKnobs struct { // execution. CaptureResolvedTableDescSpans func([]roachpb.Span) + // RunAfterSplitAndScatteringEntry allows blocking the RESTORE job after a + // single RestoreSpanEntry has been split and scattered. + RunAfterSplitAndScatteringEntry func(ctx context.Context) + // RunAfterProcessingRestoreSpanEntry allows blocking the RESTORE job after a // single RestoreSpanEntry has been processed and added to the SSTBatcher. RunAfterProcessingRestoreSpanEntry func(ctx context.Context) diff --git a/pkg/sql/sqlstats/persistedsqlstats/BUILD.bazel b/pkg/sql/sqlstats/persistedsqlstats/BUILD.bazel index 6bfd05a15d72..e2f6742e2f05 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/BUILD.bazel +++ b/pkg/sql/sqlstats/persistedsqlstats/BUILD.bazel @@ -24,7 +24,6 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/base", - "//pkg/clusterversion", "//pkg/jobs", "//pkg/jobs/jobspb", "//pkg/scheduledjobs", diff --git a/pkg/sql/sqlstats/persistedsqlstats/flush.go b/pkg/sql/sqlstats/persistedsqlstats/flush.go index 2d651807936e..f3791e9beca2 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/flush.go +++ b/pkg/sql/sqlstats/persistedsqlstats/flush.go @@ -15,7 +15,6 @@ import ( "fmt" "time" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/isql" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -487,7 +486,14 @@ func (s *PersistedSQLStats) insertStatementStats( plan := tree.NewDJSON(sqlstatsutil.ExplainTreePlanNodeToJSON(&stats.Stats.SensitiveInfo.MostRecentPlanDescription)) nodeID := s.GetEnabledSQLInstanceID() - values := "$1 ,$2, $3, $4, $5, $6, $7, $8, $9, $10" + indexRecommendations := tree.NewDArray(types.String) + for _, recommendation := range stats.Stats.IndexRecommendations { + if err := indexRecommendations.Append(tree.NewDString(recommendation)); err != nil { + return 0, err + } + } + + values := "$1 ,$2, $3, $4, $5, $6, $7, $8, $9, $10, $11" args := append(make([]interface{}, 0, 11), aggregatedTs, // aggregated_ts serializedFingerprintID, // fingerprint_id @@ -499,17 +505,8 @@ func (s *PersistedSQLStats) insertStatementStats( metadata, // metadata statistics, // statistics plan, // plan + indexRecommendations, // index_recommendations ) - if s.cfg.Settings.Version.IsActive(ctx, clusterversion.TODODelete_V22_2AlterSystemStatementStatisticsAddIndexRecommendations) { - values = values + ", $11" - indexRecommendations := tree.NewDArray(types.String) - for _, recommendation := range stats.Stats.IndexRecommendations { - if err := indexRecommendations.Append(tree.NewDString(recommendation)); err != nil { - return 0, err - } - } - args = append(args, indexRecommendations) - } insertStmt := fmt.Sprintf(` INSERT INTO system.statement_statistics diff --git a/pkg/sql/sqlstats/persistedsqlstats/stmt_reader.go b/pkg/sql/sqlstats/persistedsqlstats/stmt_reader.go index 796259e5f3f3..b96914c5c536 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/stmt_reader.go +++ b/pkg/sql/sqlstats/persistedsqlstats/stmt_reader.go @@ -16,7 +16,6 @@ import ( "strings" "time" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/isql" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -112,10 +111,7 @@ func (s *PersistedSQLStats) getFetchQueryForStmtStatsTable( "statistics", "plan", "agg_interval", - } - - if s.cfg.Settings.Version.IsActive(ctx, clusterversion.TODODelete_V22_2AlterSystemStatementStatisticsAddIndexRecommendations) { - selectedColumns = append(selectedColumns, "index_recommendations") + "index_recommendations", } // [1]: selection columns diff --git a/pkg/upgrade/upgrades/BUILD.bazel b/pkg/upgrade/upgrades/BUILD.bazel index 5fd374a39dcf..2b88baa1d5a4 100644 --- a/pkg/upgrade/upgrades/BUILD.bazel +++ b/pkg/upgrade/upgrades/BUILD.bazel @@ -7,7 +7,6 @@ go_library( "alter_jobs_add_job_type.go", "alter_sql_instances_locality.go", "alter_sql_instances_sql_addr.go", - "alter_statement_statistics_index_recommendations.go", "alter_table_statistics_partial_predicate_and_id.go", "create_index_usage_statement_statistics.go", "delete_descriptors_of_dropped_functions.go", @@ -86,7 +85,6 @@ go_test( "alter_jobs_add_job_type_test.go", "alter_sql_instances_locality_test.go", "alter_sql_instances_sql_addr_test.go", - "alter_statement_statistics_index_recommendations_test.go", "alter_table_statistics_partial_predicate_and_id_test.go", "builtins_test.go", "create_index_usage_statement_statistics_test.go", diff --git a/pkg/upgrade/upgrades/alter_statement_statistics_index_recommendations.go b/pkg/upgrade/upgrades/alter_statement_statistics_index_recommendations.go deleted file mode 100644 index dabfeff4d04e..000000000000 --- a/pkg/upgrade/upgrades/alter_statement_statistics_index_recommendations.go +++ /dev/null @@ -1,41 +0,0 @@ -// 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/keys" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" - "github.com/cockroachdb/cockroach/pkg/upgrade" -) - -const addIndexRecommendationsCol = ` -ALTER TABLE system.statement_statistics -ADD COLUMN IF NOT EXISTS "index_recommendations" STRING[] NOT NULL DEFAULT (array[]::STRING[]) -FAMILY "primary" -` - -func alterSystemStatementStatisticsAddIndexRecommendations( - ctx context.Context, cs clusterversion.ClusterVersion, d upgrade.TenantDeps, -) error { - op := operation{ - name: "add-statement-statistics-index-recommendations-col", - schemaList: []string{"index_rec"}, - query: addIndexRecommendationsCol, - schemaExistsFn: hasColumn, - } - if err := migrateTable(ctx, cs, d, op, keys.StatementStatisticsTableID, systemschema.StatementStatisticsTable); err != nil { - return err - } - return nil -} diff --git a/pkg/upgrade/upgrades/alter_statement_statistics_index_recommendations_test.go b/pkg/upgrade/upgrades/alter_statement_statistics_index_recommendations_test.go deleted file mode 100644 index e76a3ef28b48..000000000000 --- a/pkg/upgrade/upgrades/alter_statement_statistics_index_recommendations_test.go +++ /dev/null @@ -1,197 +0,0 @@ -// 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/keys" - "github.com/cockroachdb/cockroach/pkg/security/username" - "github.com/cockroachdb/cockroach/pkg/server" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/catenumpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" - "github.com/cockroachdb/cockroach/pkg/sql/privilege" - "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" - "github.com/cockroachdb/cockroach/pkg/sql/types" - "github.com/cockroachdb/cockroach/pkg/testutils/skip" - "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" - "github.com/cockroachdb/cockroach/pkg/upgrade/upgrades" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/cockroachdb/cockroach/pkg/util/log" -) - -func TestAlterSystemStatementStatisticsTable(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - skip.WithIssue(t, 95530, "bump minBinary to 22.2. Skip 22.2 mixed-version tests for future cleanup") - - clusterArgs := base.TestClusterArgs{ - ServerArgs: base.TestServerArgs{ - Knobs: base.TestingKnobs{ - Server: &server.TestingKnobs{ - DisableAutomaticVersionUpgrade: make(chan struct{}), - BinaryVersionOverride: clusterversion.ByKey( - clusterversion.TODODelete_V22_2AlterSystemStatementStatisticsAddIndexRecommendations - 1), - }, - }, - }, - } - - var ( - ctx = context.Background() - - tc = testcluster.StartTestCluster(t, 1, clusterArgs) - s = tc.Server(0) - sqlDB = tc.ServerConn(0) - ) - defer tc.Stopper().Stop(ctx) - - var ( - validationSchemas = []upgrades.Schema{ - {Name: "index_recommendations", ValidationFn: upgrades.HasColumn}, - {Name: "primary", ValidationFn: upgrades.HasColumnFamily}, - } - ) - - // Inject the old copy of the descriptor. - upgrades.InjectLegacyTable(ctx, t, s, systemschema.StatementStatisticsTable, getDeprecatedStatementStatisticsDescriptor) - // Validate that the table statement_statistics has the old schema. - upgrades.ValidateSchemaExists( - ctx, - t, - s, - sqlDB, - keys.StatementStatisticsTableID, - systemschema.StatementStatisticsTable, - []string{}, - validationSchemas, - false, /* expectExists */ - ) - // Run the upgrade. - upgrades.Upgrade( - t, - sqlDB, - clusterversion.TODODelete_V22_2AlterSystemStatementStatisticsAddIndexRecommendations, - nil, /* done */ - false, /* expectError */ - ) - // Validate that the table has new schema. - upgrades.ValidateSchemaExists( - ctx, - t, - s, - sqlDB, - keys.StatementStatisticsTableID, - systemschema.StatementStatisticsTable, - []string{}, - validationSchemas, - true, /* expectExists */ - ) -} - -// getDeprecatedStatementStatisticsDescriptor returns the system.statement_statistics -// table descriptor that was being used before adding a new column in the -// current version. -func getDeprecatedStatementStatisticsDescriptor() *descpb.TableDescriptor { - sqlStmtHashComputeExpr := `mod(fnv32(crdb_internal.datums_to_bytes(aggregated_ts, app_name, fingerprint_id, node_id, plan_hash, transaction_fingerprint_id)), 8:::INT8)` - - return &descpb.TableDescriptor{ - Name: string(catconstants.StatementStatisticsTableName), - ID: keys.StatementStatisticsTableID, - ParentID: keys.SystemDatabaseID, - UnexposedParentSchemaID: keys.PublicSchemaID, - Version: 1, - Columns: []descpb.ColumnDescriptor{ - {Name: "aggregated_ts", ID: 1, Type: types.TimestampTZ, Nullable: false}, - {Name: "fingerprint_id", ID: 2, Type: types.Bytes, Nullable: false}, - {Name: "transaction_fingerprint_id", ID: 3, Type: types.Bytes, Nullable: false}, - {Name: "plan_hash", ID: 4, Type: types.Bytes, Nullable: false}, - {Name: "app_name", ID: 5, Type: types.String, Nullable: false}, - {Name: "node_id", ID: 6, Type: types.Int, Nullable: false}, - {Name: "agg_interval", ID: 7, Type: types.Interval, Nullable: false}, - {Name: "metadata", ID: 8, Type: types.Jsonb, Nullable: false}, - {Name: "statistics", ID: 9, Type: types.Jsonb, Nullable: false}, - {Name: "plan", ID: 10, Type: types.Jsonb, Nullable: false}, - { - Name: "crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8", - ID: 11, - Type: types.Int4, - Nullable: false, - ComputeExpr: &sqlStmtHashComputeExpr, - Hidden: true, - }, - }, - NextColumnID: 12, - Families: []descpb.ColumnFamilyDescriptor{ - { - Name: "primary", - ID: 0, - ColumnNames: []string{ - "crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8", - "aggregated_ts", "fingerprint_id", "transaction_fingerprint_id", "plan_hash", "app_name", "node_id", - "agg_interval", "metadata", "statistics", "plan", - }, - ColumnIDs: []descpb.ColumnID{11, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10}, - DefaultColumnID: 0, - }, - }, - NextFamilyID: 1, - PrimaryIndex: descpb.IndexDescriptor{ - Name: tabledesc.LegacyPrimaryKeyIndexName, - ID: 1, - Unique: true, - KeyColumnNames: []string{ - "crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8", - "aggregated_ts", - "fingerprint_id", - "transaction_fingerprint_id", - "plan_hash", - "app_name", - "node_id", - }, - KeyColumnDirections: []catenumpb.IndexColumn_Direction{ - catenumpb.IndexColumn_ASC, - catenumpb.IndexColumn_ASC, - catenumpb.IndexColumn_ASC, - catenumpb.IndexColumn_ASC, - catenumpb.IndexColumn_ASC, - catenumpb.IndexColumn_ASC, - catenumpb.IndexColumn_ASC, - }, - KeyColumnIDs: []descpb.ColumnID{11, 1, 2, 3, 4, 5, 6}, - Version: descpb.StrictIndexColumnIDGuaranteesVersion, - Sharded: catpb.ShardedDescriptor{ - IsSharded: true, - Name: "crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8", - ShardBuckets: 8, - ColumnNames: []string{ - "aggregated_ts", - "app_name", - "fingerprint_id", - "node_id", - "plan_hash", - "transaction_fingerprint_id", - }, - }, - }, - NextIndexID: 3, - Privileges: catpb.NewCustomSuperuserPrivilegeDescriptor(privilege.ReadWriteData, username.NodeUserName()), - NextMutationID: 1, - FormatVersion: 3, - } -} diff --git a/pkg/upgrade/upgrades/upgrades.go b/pkg/upgrade/upgrades/upgrades.go index af59aaa64f77..784b3c48f0e5 100644 --- a/pkg/upgrade/upgrades/upgrades.go +++ b/pkg/upgrade/upgrades/upgrades.go @@ -123,12 +123,6 @@ var upgrades = []upgradebase.Upgrade{ upgrade.NoPrecondition, systemExternalConnectionsTableMigration, ), - upgrade.NewTenantUpgrade( - "add column index_recommendations to table system.statement_statistics", - toCV(clusterversion.TODODelete_V22_2AlterSystemStatementStatisticsAddIndexRecommendations), - upgrade.NoPrecondition, - alterSystemStatementStatisticsAddIndexRecommendations, - ), upgrade.NewTenantUpgrade("add system.role_id_sequence", toCV(clusterversion.TODODelete_V22_2RoleIDSequence), upgrade.NoPrecondition,