Skip to content

Commit

Permalink
migrations: add target column to system.pts_records
Browse files Browse the repository at this point in the history
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
adityamaru committed Jan 5, 2022
1 parent 798a3e5 commit 9c2e0c8
Show file tree
Hide file tree
Showing 11 changed files with 234 additions and 30 deletions.
2 changes: 1 addition & 1 deletion docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -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 <host>:<port>. 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 <host>:<port>. If no port is specified, 4317 will be used.
trace.zipkin.collector string the address of a Zipkin instance to receive traces, as <host>:<port>. If no port is specified, 9411 will be used.
version version 21.2-34 set the active cluster version in the format '<major>.<minor>'
version version 21.2-36 set the active cluster version in the format '<major>.<minor>'
2 changes: 1 addition & 1 deletion docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -174,6 +174,6 @@
<tr><td><code>trace.jaeger.agent</code></td><td>string</td><td><code></code></td><td>the address of a Jaeger agent to receive traces using the Jaeger UDP Thrift protocol, as <host>:<port>. If no port is specified, 6381 will be used.</td></tr>
<tr><td><code>trace.opentelemetry.collector</code></td><td>string</td><td><code></code></td><td>address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as <host>:<port>. If no port is specified, 4317 will be used.</td></tr>
<tr><td><code>trace.zipkin.collector</code></td><td>string</td><td><code></code></td><td>the address of a Zipkin instance to receive traces, as <host>:<port>. If no port is specified, 9411 will be used.</td></tr>
<tr><td><code>version</code></td><td>version</td><td><code>21.2-34</code></td><td>set the active cluster version in the format '<major>.<minor>'</td></tr>
<tr><td><code>version</code></td><td>version</td><td><code>21.2-36</code></td><td>set the active cluster version in the format '<major>.<minor>'</td></tr>
</tbody>
</table>
8 changes: 8 additions & 0 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down Expand Up @@ -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.
Expand Down
5 changes: 3 additions & 2 deletions pkg/clusterversion/key_string.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

2 changes: 2 additions & 0 deletions pkg/migration/migrations/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down Expand Up @@ -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",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand All @@ -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
Expand Down
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
}
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,
}
}
6 changes: 6 additions & 0 deletions pkg/migration/migrations/migrations.go
Original file line number Diff line number Diff line change
Expand Up @@ -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() {
Expand Down
11 changes: 8 additions & 3 deletions pkg/sql/catalog/systemschema/system.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 = `
Expand Down Expand Up @@ -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{
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/logictest/testdata/logic_test/information_schema
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down

0 comments on commit 9c2e0c8

Please sign in to comment.