Skip to content

Commit

Permalink
sql: support version numbers on descriptor validation cockroachdb#76166
Browse files Browse the repository at this point in the history
Previously, the descriptor validation code did not
take a version number, so it was not possible to version
gate new validation logic. This was inadequate because
when new fields are introduced we don't want their validation
to kick in for certain cases like the debug doctor, such as any
new fields with non-zero defaults. To address this, this patch add supports
for version numbers inside the validation, and updates unit tests
to pass this in as well. It also adds a new option on debug doctor
to run a version of validation.

Release note (cli change): Add new optional version argument
to the doctor examine command. This can be used to enable /
disable validation when examining older zip directories.
  • Loading branch information
fqazi authored and RajivTS committed Mar 6, 2022
1 parent c27a727 commit 24ed8a3
Show file tree
Hide file tree
Showing 84 changed files with 417 additions and 170 deletions.
3 changes: 2 additions & 1 deletion pkg/ccl/backupccl/restore_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -1885,7 +1885,8 @@ func (r *restoreResumer) publishDescriptors(
return err
}
}
if err := mutTable.AllocateIDs(ctx); err != nil {
version := r.settings.Version.ActiveVersion(ctx)
if err := mutTable.AllocateIDs(ctx, version); err != nil {
return err
}
allMutDescs = append(allMutDescs, mutTable)
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/changefeedccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -157,6 +157,7 @@ go_test(
"//pkg/ccl/utilccl",
"//pkg/cloud",
"//pkg/cloud/impl:cloudimpl",
"//pkg/clusterversion",
"//pkg/gossip",
"//pkg/jobs",
"//pkg/jobs/jobspb",
Expand Down
3 changes: 2 additions & 1 deletion pkg/ccl/changefeedccl/avro_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"github.com/cockroachdb/apd/v3"
"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase"
"github.com/cockroachdb/cockroach/pkg/ccl/importccl"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
Expand Down Expand Up @@ -76,7 +77,7 @@ func parseTableDesc(createTableStmt string) (catalog.TableDescriptor, error) {
if err != nil {
return nil, err
}
return mutDesc, descbuilder.ValidateSelf(mutDesc)
return mutDesc, descbuilder.ValidateSelf(mutDesc, clusterversion.TestingClusterVersion)
}

func parseValues(tableDesc catalog.TableDescriptor, values string) ([]rowenc.EncDatumRow, error) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/changefeed_stmt.go
Original file line number Diff line number Diff line change
Expand Up @@ -894,7 +894,7 @@ func (b *changefeedResumer) OnPauseRequest(
func getQualifiedTableName(
ctx context.Context, execCfg *sql.ExecutorConfig, txn *kv.Txn, desc catalog.TableDescriptor,
) (string, error) {
col := execCfg.CollectionFactory.MakeCollection(nil /* temporarySchemaProvider */)
col := execCfg.CollectionFactory.MakeCollection(ctx, nil /* TemporarySchemaProvider */)
dbDesc, err := col.Direct().MustGetDatabaseDescByID(ctx, txn, desc.GetParentID())
if err != nil {
return "", err
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/rowfetcher_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,7 @@ func newRowFetcherCache(
return &rowFetcherCache{
codec: codec,
leaseMgr: leaseMgr,
collection: cf.NewCollection(nil /* TemporarySchemaProvider */),
collection: cf.NewCollection(ctx, nil /* TemporarySchemaProvider */),
db: db,
fetchers: cache.NewUnorderedCache(rfCacheConfig),
}
Expand Down
4 changes: 3 additions & 1 deletion pkg/ccl/importccl/read_import_mysql.go
Original file line number Diff line number Diff line change
Expand Up @@ -431,6 +431,7 @@ func mysqlTableToCockroach(
seqDesc, err = sql.NewSequenceTableDesc(
ctx,
nil, /* planner */
evalCtx.Settings,
seqName,
opts,
parentDB.GetID(),
Expand Down Expand Up @@ -600,7 +601,8 @@ func addDelayedFKs(
if err := fixDescriptorFKState(def.tbl); err != nil {
return err
}
if err := def.tbl.AllocateIDs(ctx); err != nil {
version := evalCtx.Settings.Version.ActiveVersion(ctx)
if err := def.tbl.AllocateIDs(ctx, version); err != nil {
return err
}
}
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/importccl/read_import_pgdump.go
Original file line number Diff line number Diff line change
Expand Up @@ -322,6 +322,7 @@ func createPostgresSequences(
desc, err := sql.NewSequenceTableDesc(
ctx,
nil, /* planner */
execCfg.Settings,
schemaAndTableName.table,
seq.Options,
parentID,
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/importccl/testutils_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@ func descForTable(
desc, err := sql.NewSequenceTableDesc(
ctx,
nil, /* planner */
settings,
name,
tree.SequenceOptions{},
parent,
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/partitionccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@ go_test(
"//pkg/ccl/storageccl",
"//pkg/ccl/testutilsccl",
"//pkg/ccl/utilccl",
"//pkg/clusterversion",
"//pkg/config",
"//pkg/config/zonepb",
"//pkg/jobs",
Expand Down
3 changes: 2 additions & 1 deletion pkg/ccl/partitionccl/partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/ccl/importccl"
"github.com/cockroachdb/cockroach/pkg/ccl/utilccl"
"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"
Expand Down Expand Up @@ -154,7 +155,7 @@ func (pt *partitioningTest) parse() error {
return err
}
pt.parsed.tableDesc = mutDesc
if err := descbuilder.ValidateSelf(pt.parsed.tableDesc); err != nil {
if err := descbuilder.ValidateSelf(pt.parsed.tableDesc, clusterversion.TestingClusterVersion); err != nil {
return err
}
}
Expand Down
36 changes: 30 additions & 6 deletions pkg/cli/doctor.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,8 +28,10 @@ import (
"github.com/cockroachdb/cockroach/pkg/cli/clierrorplus"
"github.com/cockroachdb/cockroach/pkg/cli/clisqlclient"
"github.com/cockroachdb/cockroach/pkg/cli/exit"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/doctor"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
Expand Down Expand Up @@ -74,6 +76,7 @@ tables are queried either from a live cluster or from an unzipped debug.zip.
}

type doctorFn = func(
version *clusterversion.ClusterVersion,
descTable doctor.DescriptorTable,
namespaceTable doctor.NamespaceTable,
jobsTable doctor.JobsTable,
Expand All @@ -82,19 +85,27 @@ type doctorFn = func(

func makeZipDirCommand(fn doctorFn) *cobra.Command {
return &cobra.Command{
Use: "zipdir <debug_zip_dir>",
Use: "zipdir <debug_zip_dir> [version]",
Short: "run doctor tool on data from an unzipped debug.zip",
Long: `
Run the doctor tool on system data from an unzipped debug.zip. This command
requires the path of the unzipped 'debug' directory as its argument.
requires the path of the unzipped 'debug' directory as its argument. A version
can be optionally specified, which will be used enable / disable validation
that may not exist on downlevel versions.
`,
Args: cobra.ExactArgs(1),
Args: cobra.RangeArgs(1, 2),
RunE: func(cmd *cobra.Command, args []string) error {
descs, ns, jobs, err := fromZipDir(args[0])
var version *clusterversion.ClusterVersion
if len(args) == 2 {
version = &clusterversion.ClusterVersion{
Version: roachpb.MustParseVersion(args[1]),
}
}
if err != nil {
return err
}
return fn(descs, ns, jobs, os.Stdout)
return fn(version, descs, ns, jobs, os.Stdout)
},
}
}
Expand All @@ -118,7 +129,7 @@ Run the doctor tool system data from a live cluster specified by --url.
if err != nil {
return err
}
return fn(descs, ns, jobs, os.Stdout)
return fn(nil, descs, ns, jobs, os.Stdout)
}),
}
}
Expand All @@ -137,6 +148,7 @@ var doctorRecreateClusterCmd = makeClusterCommand(runDoctorRecreate)
var doctorRecreateZipDirCmd = makeZipDirCommand(runDoctorRecreate)

func runDoctorRecreate(
_ *clusterversion.ClusterVersion,
descTable doctor.DescriptorTable,
namespaceTable doctor.NamespaceTable,
jobsTable doctor.JobsTable,
Expand All @@ -146,14 +158,26 @@ func runDoctorRecreate(
}

func runDoctorExamine(
version *clusterversion.ClusterVersion,
descTable doctor.DescriptorTable,
namespaceTable doctor.NamespaceTable,
jobsTable doctor.JobsTable,
out io.Writer,
) (err error) {
if version == nil {
version = &clusterversion.ClusterVersion{
Version: clusterversion.DoctorBinaryVersion,
}
}
var valid bool
valid, err = doctor.Examine(
context.Background(), descTable, namespaceTable, jobsTable, debugCtx.verbose, out)
context.Background(),
*version,
descTable,
namespaceTable,
jobsTable,
debugCtx.verbose,
out)
if err != nil {
return err
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/cli/doctor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@ func TestDoctorZipDir(t *testing.T) {
defer c.Cleanup()

t.Run("examine", func(t *testing.T) {
out, err := c.RunWithCapture("debug doctor examine zipdir testdata/doctor/debugzip")
out, err := c.RunWithCapture("debug doctor examine zipdir testdata/doctor/debugzip 21.1")
if err != nil {
t.Fatal(err)
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/cli/testdata/doctor/test_examine_zipdir
Original file line number Diff line number Diff line change
@@ -1,6 +1,6 @@
debug doctor examine zipdir testdata/doctor/debugzip
----
debug doctor examine zipdir testdata/doctor/debugzip
debug doctor examine zipdir testdata/doctor/debugzip 21.1
WARNING: errors occurred during the production of system.jobs.txt, contents may be missing or incomplete.
Examining 37 descriptors and 42 namespace entries...
ParentID 52, ParentSchemaID 29: relation "users" (53): referenced database ID 52: referenced descriptor not found
Expand Down
1 change: 1 addition & 0 deletions pkg/clusterversion/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ go_library(
"keyed_versions.go",
"setting.go",
"testutils.go",
"utilversions.go",
":gen-key-stringer", # keep
],
embed = [":clusterversion_go_proto"],
Expand Down
14 changes: 14 additions & 0 deletions pkg/clusterversion/utilversions.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,14 @@
// 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 clusterversion

// DoctorBinaryVersion level used by the debug doctor when validating any files.
var DoctorBinaryVersion = binaryVersion
3 changes: 2 additions & 1 deletion pkg/jobs/registry_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"time"

"github.com/cockroachdb/cockroach/pkg/base"
"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"
Expand Down Expand Up @@ -81,7 +82,7 @@ func writeMutation(
) {
tableDesc.Mutations = append(tableDesc.Mutations, m)
tableDesc.Version++
if err := descbuilder.ValidateSelf(tableDesc); err != nil {
if err := descbuilder.ValidateSelf(tableDesc, clusterversion.TestingClusterVersion); err != nil {
t.Fatal(err)
}
if err := kvDB.Put(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -88,7 +88,9 @@ func TestEnsureNoDrainingNames(t *testing.T) {
// Check that the draining name persists in the descriptor and in the db's
// schema mapping.
{
db := desctestutils.TestingGetDatabaseDescriptor(s.DB(), c, "t")
version := tc.Server(0).ClusterSettings().Version.ActiveVersion(ctx)
db := desctestutils.TestingGetDatabaseDescriptorWitVersion(
s.DB(), c, version, "t")
_ = db.ForEachSchemaInfo(func(id descpb.ID, name string, isDropped bool) error {
switch name {
case "foo":
Expand All @@ -98,7 +100,8 @@ func TestEnsureNoDrainingNames(t *testing.T) {
}
return nil
})
foo := desctestutils.TestingGetSchemaDescriptor(s.DB(), c, db.GetID(), "foo")
foo := desctestutils.TestingGetSchemaDescriptorWithVersion(
s.DB(), c, version, db.GetID(), "foo")
require.NotEmpty(t, foo.GetDrainingNames())
}

Expand All @@ -120,15 +123,16 @@ func TestEnsureNoDrainingNames(t *testing.T) {
// Check that there are no draining names and that the database schema mapping
// is correct.
{
db := desctestutils.TestingGetDatabaseDescriptor(s.DB(), c, "t")
version := tc.Server(0).ClusterSettings().Version.ActiveVersion(ctx)
db := desctestutils.TestingGetDatabaseDescriptorWitVersion(s.DB(), c, version, "t")
_ = db.ForEachSchemaInfo(func(id descpb.ID, name string, isDropped bool) error {
require.False(t, isDropped)
require.True(t, name == "foo" || name == "bar")
return nil
})
foo := desctestutils.TestingGetSchemaDescriptor(s.DB(), c, db.GetID(), "foo")
foo := desctestutils.TestingGetSchemaDescriptorWithVersion(s.DB(), c, version, db.GetID(), "foo")
require.Empty(t, foo.GetDrainingNames())
bar := desctestutils.TestingGetSchemaDescriptor(s.DB(), c, db.GetID(), "bar")
bar := desctestutils.TestingGetSchemaDescriptorWithVersion(s.DB(), c, version, db.GetID(), "bar")
require.Empty(t, bar.GetDrainingNames())
}
}
3 changes: 2 additions & 1 deletion pkg/sql/add_column.go
Original file line number Diff line number Diff line change
Expand Up @@ -161,7 +161,8 @@ func (p *planner) addColumnImpl(

// We need to allocate new ID for the created column in order to correctly
// assign sequence ownership.
if err := n.tableDesc.AllocateIDs(params.ctx); err != nil {
version := params.ExecCfg().Settings.Version.ActiveVersion(params.ctx)
if err := n.tableDesc.AllocateIDs(params.ctx, version); err != nil {
return err
}

Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/alter_column_type.go
Original file line number Diff line number Diff line change
Expand Up @@ -415,7 +415,8 @@ func alterColumnTypeGeneral(
tableDesc.SetPrimaryIndex(primaryIndex)
}

if err := tableDesc.AllocateIDs(ctx); err != nil {
version := params.ExecCfg().Settings.Version.ActiveVersion(ctx)
if err := tableDesc.AllocateIDs(ctx, version); err != nil {
return err
}

Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/alter_index.go
Original file line number Diff line number Diff line change
Expand Up @@ -136,7 +136,8 @@ func (n *alterIndexNode) startExec(params runParams) error {

}

if err := n.tableDesc.AllocateIDs(params.ctx); err != nil {
version := params.ExecCfg().Settings.Version.ActiveVersion(params.ctx)
if err := n.tableDesc.AllocateIDs(params.ctx, version); err != nil {
return err
}

Expand Down
5 changes: 3 additions & 2 deletions pkg/sql/alter_primary_key.go
Original file line number Diff line number Diff line change
Expand Up @@ -237,7 +237,8 @@ func (p *planner) AlterPrimaryKey(
if err := tableDesc.AddIndexMutation(newPrimaryIndexDesc, descpb.DescriptorMutation_ADD); err != nil {
return err
}
if err := tableDesc.AllocateIDs(ctx); err != nil {
version := p.ExecCfg().Settings.Version.ActiveVersion(ctx)
if err := tableDesc.AllocateIDs(ctx, version); err != nil {
return err
}

Expand Down Expand Up @@ -521,7 +522,7 @@ func (p *planner) AlterPrimaryKey(
}
tableDesc.AddPrimaryKeySwapMutation(swapArgs)

if err := descbuilder.ValidateSelf(tableDesc); err != nil {
if err := descbuilder.ValidateSelf(tableDesc, version); err != nil {
return err
}

Expand Down
6 changes: 4 additions & 2 deletions pkg/sql/alter_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -307,7 +307,8 @@ func (n *alterTableNode) startExec(params runParams) error {

// We need to allocate IDs upfront in the event we need to update the zone config
// in the same transaction.
if err := n.tableDesc.AllocateIDs(params.ctx); err != nil {
version := params.ExecCfg().Settings.Version.ActiveVersion(params.ctx)
if err := n.tableDesc.AllocateIDs(params.ctx, version); err != nil {
return err
}
if err := params.p.configureZoneConfigForNewIndexPartitioning(
Expand Down Expand Up @@ -1055,7 +1056,8 @@ func (n *alterTableNode) startExec(params runParams) error {
}

// Allocate IDs now, so new IDs are available to subsequent commands
if err := n.tableDesc.AllocateIDs(params.ctx); err != nil {
version := params.ExecCfg().Settings.Version.ActiveVersion(params.ctx)
if err := n.tableDesc.AllocateIDs(params.ctx, version); err != nil {
return err
}
}
Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/alter_table_locality.go
Original file line number Diff line number Diff line change
Expand Up @@ -345,7 +345,8 @@ func (n *alterTableSetLocalityNode) alterTableLocalityToRegionalByRow(
mutationIdxAllowedInSameTxn = &mutationIdx
newColumnName = &partColName

if err := n.tableDesc.AllocateIDs(params.ctx); err != nil {
version := params.ExecCfg().Settings.Version.ActiveVersion(params.ctx)
if err := n.tableDesc.AllocateIDs(params.ctx, version); err != nil {
return err
}

Expand Down
1 change: 1 addition & 0 deletions pkg/sql/catalog/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ go_library(
importpath = "github.com/cockroachdb/cockroach/pkg/sql/catalog",
visibility = ["//visibility:public"],
deps = [
"//pkg/clusterversion",
"//pkg/geo/geoindex",
"//pkg/keys",
"//pkg/kv",
Expand Down
Loading

0 comments on commit 24ed8a3

Please sign in to comment.