From 9c2e0c843ad4dcfaa214a7fdffc6a7be8eed53a7 Mon Sep 17 00:00:00 2001 From: Aditya Maru Date: Fri, 24 Dec 2021 11:57:22 -0500 Subject: [PATCH] migrations: add target column to system.pts_records This change adds a `BYTES` column to the system.pts_records table, that will store the `Target` on a protected timestamp record. We do not populate the column in this PR, that will be done in a followup. Additionally, we add a tenant migration to `ALTER TABLE ... ADD COLUMN` to run on the upgrade path for older clusters. Informs: #73727 Release note (sql change): `system.protected_timestamp_records` table now has an additional `target` column that will store the target a record protects. This target can either be the entire cluster, a list of tenant keyspaces, or a list of database/table keyspaces. --- .../settings/settings-for-tenants.txt | 2 +- docs/generated/settings/settings.html | 2 +- pkg/clusterversion/cockroach_versions.go | 8 + pkg/clusterversion/key_string.go | 5 +- pkg/migration/migrations/BUILD.bazel | 2 + ...ter_statement_diagnostics_requests_test.go | 40 +++-- ...alter_table_protected_timestamp_records.go | 43 ++++++ ..._table_protected_timestamp_records_test.go | 144 ++++++++++++++++++ pkg/migration/migrations/migrations.go | 6 + pkg/sql/catalog/systemschema/system.go | 11 +- .../testdata/logic_test/information_schema | 1 + 11 files changed, 234 insertions(+), 30 deletions(-) create mode 100644 pkg/migration/migrations/alter_table_protected_timestamp_records.go create mode 100644 pkg/migration/migrations/alter_table_protected_timestamp_records_test.go diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 451fb743b83f..d234018b5715 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -168,4 +168,4 @@ trace.debug.enable boolean false if set, traces for recent requests can be seen trace.jaeger.agent string the address of a Jaeger agent to receive traces using the Jaeger UDP Thrift protocol, as :. If no port is specified, 6381 will be used. 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.zipkin.collector string the address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -version version 21.2-34 set the active cluster version in the format '.' +version version 21.2-36 set the active cluster version in the format '.' diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 0265b2428046..24bd4ec3fa4d 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -174,6 +174,6 @@ trace.jaeger.agentstringthe address of a Jaeger agent to receive traces using the Jaeger UDP Thrift protocol, as :. If no port is specified, 6381 will be used. 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.zipkin.collectorstringthe address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -versionversion21.2-34set the active cluster version in the format '.' +versionversion21.2-36set the active cluster version in the format '.' diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index b04f79aa8f2b..a8914f732563 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -225,6 +225,10 @@ const ( SeedTenantSpanConfigs // Public schema is backed by a descriptor. PublicSchemasWithDescriptors + // AlterSystemProtectedTimestampAddColumn adds a target column to the + // system.protected_ts_records table that describes what is protected by the + // record. + AlterSystemProtectedTimestampAddColumn // ************************************************* // Step (1): Add new versions here. @@ -329,6 +333,10 @@ var versionsSingleton = keyedVersions{ Key: PublicSchemasWithDescriptors, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 34}, }, + { + Key: AlterSystemProtectedTimestampAddColumn, + Version: roachpb.Version{Major: 21, Minor: 2, Internal: 36}, + }, // ************************************************* // Step (2): Add new versions here. diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index be3b9bd385eb..abd04ad7fb6d 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -26,11 +26,12 @@ func _() { _ = x[PreSeedTenantSpanConfigs-15] _ = x[SeedTenantSpanConfigs-16] _ = x[PublicSchemasWithDescriptors-17] + _ = x[AlterSystemProtectedTimestampAddColumn-18] } -const _Key_name = "V21_2Start22_1TargetBytesAvoidExcessAvoidDrainingNamesDrainingNamesMigrationTraceIDDoesntImplyStructuredRecordingAlterSystemTableStatisticsAddAvgSizeColAlterSystemStmtDiagReqsMVCCAddSSTableInsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptors" +const _Key_name = "V21_2Start22_1TargetBytesAvoidExcessAvoidDrainingNamesDrainingNamesMigrationTraceIDDoesntImplyStructuredRecordingAlterSystemTableStatisticsAddAvgSizeColAlterSystemStmtDiagReqsMVCCAddSSTableInsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsAlterSystemProtectedTimestampAddColumn" -var _Key_index = [...]uint16{0, 5, 14, 36, 54, 76, 113, 152, 175, 189, 230, 256, 275, 309, 321, 352, 376, 397, 425} +var _Key_index = [...]uint16{0, 5, 14, 36, 54, 76, 113, 152, 175, 189, 230, 256, 275, 309, 321, 352, 376, 397, 425, 463} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/migration/migrations/BUILD.bazel b/pkg/migration/migrations/BUILD.bazel index c102c43e7632..a1294e6c303a 100644 --- a/pkg/migration/migrations/BUILD.bazel +++ b/pkg/migration/migrations/BUILD.bazel @@ -4,6 +4,7 @@ go_library( name = "migrations", srcs = [ "alter_statement_diagnostics_requests.go", + "alter_table_protected_timestamp_records.go", "alter_table_statistics_avg_size.go", "ensure_no_draining_names.go", "insert_missing_public_schema_namespace_entry.go", @@ -42,6 +43,7 @@ go_test( name = "migrations_test", srcs = [ "alter_statement_diagnostics_requests_test.go", + "alter_table_protected_timestamp_records_test.go", "alter_table_statistics_avg_size_test.go", "builtins_test.go", "ensure_no_draining_names_external_test.go", diff --git a/pkg/migration/migrations/alter_statement_diagnostics_requests_test.go b/pkg/migration/migrations/alter_statement_diagnostics_requests_test.go index a3aaca71c67c..e6cee939c4bc 100644 --- a/pkg/migration/migrations/alter_statement_diagnostics_requests_test.go +++ b/pkg/migration/migrations/alter_statement_diagnostics_requests_test.go @@ -65,20 +65,24 @@ func TestAlterSystemStmtDiagReqs(t *testing.T) { ) // Inject the old copy of the descriptor. - migrations.InjectLegacyTable(ctx, t, s, systemschema.StatementDiagnosticsRequestsTable, getDeprecatedStmtDiagReqsDescriptor) + migrations.InjectLegacyTable(ctx, t, s, systemschema.StatementDiagnosticsRequestsTable, + getDeprecatedStmtDiagReqsDescriptor) + validateSchemaExists := func(expectExists bool) { + migrations.ValidateSchemaExists( + ctx, + t, + s, + sqlDB, + keys.StatementDiagnosticsRequestsTableID, + systemschema.StatementDiagnosticsRequestsTable, + validationStmts, + validationSchemas, + expectExists, + ) + } // Validate that the statement_diagnostics_requests table has the old // schema. - migrations.ValidateSchemaExists( - ctx, - t, - s, - sqlDB, - keys.StatementDiagnosticsRequestsTableID, - systemschema.StatementDiagnosticsRequestsTable, - validationStmts, - validationSchemas, - false, /* expectExists */ - ) + validateSchemaExists(false) // Run the migration. migrations.Migrate( t, @@ -88,17 +92,7 @@ func TestAlterSystemStmtDiagReqs(t *testing.T) { false, /* expectError */ ) // Validate that the table has new schema. - migrations.ValidateSchemaExists( - ctx, - t, - s, - sqlDB, - keys.StatementDiagnosticsRequestsTableID, - systemschema.StatementDiagnosticsRequestsTable, - validationStmts, - validationSchemas, - true, /* expectExists */ - ) + validateSchemaExists(true) } // getDeprecatedStmtDiagReqsDescriptor returns the diff --git a/pkg/migration/migrations/alter_table_protected_timestamp_records.go b/pkg/migration/migrations/alter_table_protected_timestamp_records.go new file mode 100644 index 000000000000..1c51b6758905 --- /dev/null +++ b/pkg/migration/migrations/alter_table_protected_timestamp_records.go @@ -0,0 +1,43 @@ +// Copyright 2021 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 migrations + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/migration" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" +) + +const addTargetCol = ` +ALTER TABLE system.protected_ts_records +ADD COLUMN IF NOT EXISTS target BYTES FAMILY "primary" +` + +func alterTableProtectedTimestampRecords( + ctx context.Context, cs clusterversion.ClusterVersion, d migration.TenantDeps, _ *jobs.Job, +) error { + op := operation{ + name: "add-table-pts-records-target-col", + schemaList: []string{"target"}, + query: addTargetCol, + schemaExistsFn: hasColumn, + } + if err := migrateTable(ctx, cs, d, op, + keys.ProtectedTimestampsRecordsTableID, + systemschema.ProtectedTimestampsRecordsTable); err != nil { + return err + } + return nil +} diff --git a/pkg/migration/migrations/alter_table_protected_timestamp_records_test.go b/pkg/migration/migrations/alter_table_protected_timestamp_records_test.go new file mode 100644 index 000000000000..01293f38da01 --- /dev/null +++ b/pkg/migration/migrations/alter_table_protected_timestamp_records_test.go @@ -0,0 +1,144 @@ +// Copyright 2021 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 migrations_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/migration/migrations" + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" + "github.com/cockroachdb/cockroach/pkg/sql/privilege" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" +) + +func TestAlterSystemProtectedTimestampRecordsTable(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + clusterArgs := base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + DisableAutomaticVersionUpgrade: 1, + BinaryVersionOverride: clusterversion.ByKey( + clusterversion.AlterSystemProtectedTimestampAddColumn - 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 = []migrations.Schema{ + {Name: "target", ValidationFn: migrations.HasColumn}, + } + ) + + // Inject the old copy of the descriptor. + migrations.InjectLegacyTable(ctx, t, s, systemschema.ProtectedTimestampsRecordsTable, getDeprecatedProtectedTimestampRecordsDescriptor) + // Validate that the protected timestamp records table has the old schema. + migrations.ValidateSchemaExists( + ctx, + t, + s, + sqlDB, + keys.ProtectedTimestampsRecordsTableID, + systemschema.ProtectedTimestampsRecordsTable, + []string{}, + validationSchemas, + false, /* expectExists */ + ) + // Run the migration. + migrations.Migrate( + t, + sqlDB, + clusterversion.AlterSystemProtectedTimestampAddColumn, + nil, /* done */ + false, /* expectError */ + ) + // Validate that the table has new schema. + migrations.ValidateSchemaExists( + ctx, + t, + s, + sqlDB, + keys.ProtectedTimestampsRecordsTableID, + systemschema.ProtectedTimestampsRecordsTable, + []string{}, + validationSchemas, + true, /* expectExists */ + ) +} + +// getDeprecatedProtectedTimestampRecordsDescriptor returns the +// system.pts_records table descriptor that was being used before adding a new +// column in the current version. +func getDeprecatedProtectedTimestampRecordsDescriptor() *descpb.TableDescriptor { + falseBoolString := "false" + + return &descpb.TableDescriptor{ + Name: "protected_ts_records", + ID: keys.ProtectedTimestampsRecordsTableID, + ParentID: keys.SystemDatabaseID, + UnexposedParentSchemaID: keys.PublicSchemaID, + Version: 1, + Columns: []descpb.ColumnDescriptor{ + {Name: "id", ID: 1, Type: types.Uuid}, + {Name: "ts", ID: 2, Type: types.Decimal}, + {Name: "meta_type", ID: 3, Type: types.String}, + {Name: "meta", ID: 4, Type: types.Bytes, Nullable: true}, + {Name: "num_spans", ID: 5, Type: types.Int}, + {Name: "spans", ID: 6, Type: types.Bytes}, + {Name: "verified", ID: 7, Type: types.Bool, DefaultExpr: &falseBoolString}, + }, + NextColumnID: 8, + Families: []descpb.ColumnFamilyDescriptor{ + { + Name: "primary", + ColumnNames: []string{"id", "ts", "meta_type", "meta", "num_spans", "spans", "verified"}, + ColumnIDs: []descpb.ColumnID{1, 2, 3, 4, 5, 6, 7}, + }, + }, + NextFamilyID: 1, + PrimaryIndex: descpb.IndexDescriptor{ + Name: "primary", + ID: 1, + Unique: true, + KeyColumnNames: []string{"id"}, + KeyColumnIDs: []descpb.ColumnID{1}, + KeyColumnDirections: []descpb.IndexDescriptor_Direction{ + descpb.IndexDescriptor_ASC, + }, + }, + NextIndexID: 2, + Privileges: descpb.NewCustomSuperuserPrivilegeDescriptor(privilege.ReadWriteData, security.NodeUserName()), + NextMutationID: 1, + FormatVersion: 3, + } +} diff --git a/pkg/migration/migrations/migrations.go b/pkg/migration/migrations/migrations.go index 36c66d510569..1b784c7a9810 100644 --- a/pkg/migration/migrations/migrations.go +++ b/pkg/migration/migrations/migrations.go @@ -69,6 +69,12 @@ var migrations = []migration.Migration{ NoPrecondition, insertMissingPublicSchemaNamespaceEntry, ), + migration.NewTenantMigration( + "add column target to system.protected_ts_records", + toCV(clusterversion.AlterSystemProtectedTimestampAddColumn), + NoPrecondition, + alterTableProtectedTimestampRecords, + ), } func init() { diff --git a/pkg/sql/catalog/systemschema/system.go b/pkg/sql/catalog/systemschema/system.go index cf06a69c9a45..5f4481c07c7c 100644 --- a/pkg/sql/catalog/systemschema/system.go +++ b/pkg/sql/catalog/systemschema/system.go @@ -387,8 +387,9 @@ CREATE TABLE system.protected_ts_records ( num_spans INT8 NOT NULL, -- num spans is important to know how to decode spans spans BYTES NOT NULL, verified BOOL NOT NULL DEFAULT (false), + target BYTES, -- target is an encoded protobuf that specifies what the pts record will protect CONSTRAINT "primary" PRIMARY KEY (id), - FAMILY "primary" (id, ts, meta_type, meta, num_spans, spans, verified) + FAMILY "primary" (id, ts, meta_type, meta, num_spans, spans, verified, target) );` StatementBundleChunksTableSchema = ` @@ -1661,12 +1662,16 @@ var ( {Name: "num_spans", ID: 5, Type: types.Int}, {Name: "spans", ID: 6, Type: types.Bytes}, {Name: "verified", ID: 7, Type: types.Bool, DefaultExpr: &falseBoolString}, + // target will store an encoded protobuf indicating what the protected + // timestamp record will protect. A record can protect either a cluster, + // tenants or a schema objects (databases/tables). + {Name: "target", ID: 8, Type: types.Bytes, Nullable: true}, }, []descpb.ColumnFamilyDescriptor{ { Name: "primary", - ColumnNames: []string{"id", "ts", "meta_type", "meta", "num_spans", "spans", "verified"}, - ColumnIDs: []descpb.ColumnID{1, 2, 3, 4, 5, 6, 7}, + ColumnNames: []string{"id", "ts", "meta_type", "meta", "num_spans", "spans", "verified", "target"}, + ColumnIDs: []descpb.ColumnID{1, 2, 3, 4, 5, 6, 7, 8}, }, }, descpb.IndexDescriptor{ diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index 47892b1961c1..c45290311aeb 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -2031,6 +2031,7 @@ system public protected_ts_records meta system public protected_ts_records meta_type 3 system public protected_ts_records num_spans 5 system public protected_ts_records spans 6 +system public protected_ts_records target 8 system public protected_ts_records ts 2 system public protected_ts_records verified 7 system public rangelog eventType 4