-
Notifications
You must be signed in to change notification settings - Fork 3.8k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
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.
- Loading branch information
1 parent
798a3e5
commit 9c2e0c8
Showing
11 changed files
with
234 additions
and
30 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.
Oops, something went wrong.
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
43 changes: 43 additions & 0 deletions
43
pkg/migration/migrations/alter_table_protected_timestamp_records.go
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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 | ||
} |
144 changes: 144 additions & 0 deletions
144
pkg/migration/migrations/alter_table_protected_timestamp_records_test.go
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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, | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters