Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
71898: schemachanger: inject all dependencies r=postamar a=postamar

    Although the declarative schema changer had been written with
    dependency injection in mind, the level of abstraction was still too
    low for straightforward end-to-end testing. The schema changer doesn't
    care about kv.Txn, jobs.JobRegistry, etc. Fundamentally, here's how it
    interacts with the rest of the system:
    1. The builder in scbuild:
      a. The builder queries the catalog for the current descriptor and
         namespace state.
      b. The builder also needs a valid tree.SemaContext and
         tree.EvalContex, but only in a limited way. DDLs don't require the
         evaluation of built-in functions or the like, meaning that suitable
         context objects can be built with few indirect dependencies like
         the codec, session data, etc.
    2. The executor in scexec:
      a. Naturally the executor also needs access to the catalog for reading
         and writing. In the scmutationexec package these dependencies had
         already been abstract enough, meaning there was little left to do
         except formalize read-only vs. mutating dependencies a bit.
      b. The same holds for the backfill operations, with the
         IndexBackfiller and JobProgressTracker interfaces.
      c. The executor needs to know about testing knobs and other debug
         info.
    
    In addition to making dependencies explicit as described above in
    scbuild.Dependencies and scexec.Dependencies interfaces, this commit
    also makes dependencies explicit for the actual running of the schema
    changer, to allow end-to-end testing. Concretely, we added a layer of
    abstraction around the execution of each phase:
    1. For scop.StatementPhase, the schema changer receives a target
       state, runs the planner to generate ops-stages, and executes these
       ops.
    2. For scop.PreCommitPhase, it does exactly that, but also creates and
       enqueues a job, which will deal with the subsequent
       scop.PostCommitPhase asynchronously. This job creation depends on
       things like the job registry, which we don't care for if we're doing
       end-to-end testing: in that case we'll want to run the post-commit
       phase synchronously after this one.
    3. For scop.PostCommitPhase, it's like the previous phases, except in
       the context of the Resume method of the schema change job, which
       requires a bit of plumbing to update the progress of the job itself,
       and also to update the descriptors affected by the job, because those
       hold a reference to this job which serves as a lock, blocking other
       concurrent schema changes.
    
    This new layer of abstraction is defined in the new scrun package.
    The actual implementations of the dependency interfaces are all moved to
    a new scdeps package.
    
    This commit also introduces test implementations of these dependency
    interfaces, which so far are used in the builder tests alongside
    the regular implementations. Adoption in the executor tests, as well as
    end-to-end testing of the declarative schema changer, will be added in
    a subsequent commit.
    
    Release note: None


71942: util/tracing: remove handling for impossible error r=andreimatei a=andreimatei

InjectMetaInto can no longer return an error (and I don't forsee it ever
returning an error again), so this patch cleans up the unused retval.

Release note: None

72046: dev: override bazel timeout when run under stress r=irfansharif a=irfansharif

When running under stress and no timeout is specified, we want
to respect the timeout passed down to stress[^1]. Not doing so has bazel
default to a timeout based on the test target's size[^2], which is not
what we want.

[^1]: Through --stress-arg=-maxtime or if nothing is specified, a
     -maxtime of 0 that's taken as "run forever")
[^2]: https://docs.bazel.build/versions/main/test-encyclopedia.html#role-of-the-test-runner

Release note: None

72103: deps: bump elastic/gosigar to  to v0.14.1 r=stevendanna a=knz

This adds support for macOS M1 variants and fixes a couple of bugs.

Release note: None

Co-authored-by: Marius Posta <[email protected]>
Co-authored-by: Andrei Matei <[email protected]>
Co-authored-by: irfan sharif <[email protected]>
Co-authored-by: Raphael 'kena' Poss <[email protected]>
  • Loading branch information
5 people committed Oct 28, 2021
5 parents 580c176 + f5c052c + 9d85ef3 + 5542b9b + 0fe8454 commit 90881be
Show file tree
Hide file tree
Showing 68 changed files with 3,216 additions and 1,552 deletions.
8 changes: 4 additions & 4 deletions DEPS.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -1526,8 +1526,8 @@ def go_deps():
name = "com_github_elastic_gosigar",
build_file_proto_mode = "disable_global",
importpath = "github.com/elastic/gosigar",
sum = "h1:bPIzW1Qkut7n9uwvPAXbnLDVEd45TV5ZwxYZAVX/zEQ=",
version = "v0.10.0",
sum = "h1:T0aQ7n/n2ZA9W7DmAnj60v+qzqKERdBgJBO1CG2W6rc=",
version = "v0.14.1",
)

go_repository(
Expand Down Expand Up @@ -6239,8 +6239,8 @@ def go_deps():
name = "org_golang_x_sys",
build_file_proto_mode = "disable_global",
importpath = "golang.org/x/sys",
sum = "h1:KzbpndAYEM+4oHRp9JmB2ewj0NHHxO3Z0g7Gus2O1kk=",
version = "v0.0.0-20211015200801-69063c4bb744",
sum = "h1:2B5p2L5IfGiD7+b9BOoRMC6DgObAVZV+Fsp050NqXik=",
version = "v0.0.0-20211025201205-69cdffdb9359",
)
go_repository(
name = "org_golang_x_term",
Expand Down
4 changes: 2 additions & 2 deletions go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ require (
github.com/docker/go-connections v0.4.0
github.com/dustin/go-humanize v1.0.0
github.com/edsrzf/mmap-go v1.0.0
github.com/elastic/gosigar v0.10.0
github.com/elastic/gosigar v0.14.1
github.com/emicklei/dot v0.15.0
github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a
github.com/ghemawat/stream v0.0.0-20171120220530-696b145b53b9
Expand Down Expand Up @@ -157,7 +157,7 @@ require (
golang.org/x/oauth2 v0.0.0-20210819190943-2bc19b11175f
golang.org/x/perf v0.0.0-20180704124530-6e6d33e29852
golang.org/x/sync v0.0.0-20210220032951-036812b2e83c
golang.org/x/sys v0.0.0-20211015200801-69063c4bb744
golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359
golang.org/x/term v0.0.0-20210220032956-6a3ed077a48d
golang.org/x/text v0.3.7
golang.org/x/time v0.0.0-20210723032227-1f47c861a9ac
Expand Down
8 changes: 5 additions & 3 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -598,8 +598,8 @@ github.com/eclipse/paho.mqtt.golang v1.2.0/go.mod h1:H9keYFcgq3Qr5OUJm/JZI/i6U7j
github.com/edsrzf/mmap-go v1.0.0 h1:CEBF7HpRnUCSJgGUb5h1Gm7e3VkmVDrR8lvWVLtrOFw=
github.com/edsrzf/mmap-go v1.0.0/go.mod h1:YO35OhQPt3KJa3ryjFM5Bs14WD66h8eGKpfaBNrHW5M=
github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385/go.mod h1:0vRUJqYpeSZifjYj7uP3BG/gKcuzL9xWVV/Y+cK33KM=
github.com/elastic/gosigar v0.10.0 h1:bPIzW1Qkut7n9uwvPAXbnLDVEd45TV5ZwxYZAVX/zEQ=
github.com/elastic/gosigar v0.10.0/go.mod h1:cdorVVzy1fhmEqmtgqkoE3bYtCfSCkVyjTyCIo22xvs=
github.com/elastic/gosigar v0.14.1 h1:T0aQ7n/n2ZA9W7DmAnj60v+qzqKERdBgJBO1CG2W6rc=
github.com/elastic/gosigar v0.14.1/go.mod h1:iXRIGg2tLnu7LBdpqzyQfGDEidKCfWcCMS0WKyPWoMs=
github.com/elazarl/goproxy v0.0.0-20170405201442-c4fc26588b6e/go.mod h1:/Zj4wYkgs4iZTTu3o/KG3Itv/qCCa8VVMlb3i9OVuzc=
github.com/elazarl/goproxy v0.0.0-20180725130230-947c36da3153/go.mod h1:/Zj4wYkgs4iZTTu3o/KG3Itv/qCCa8VVMlb3i9OVuzc=
github.com/emicklei/dot v0.15.0 h1:XDBW0Xco1QNyRb33cqLe10cT04yMWL1XpCZfa98Q6Og=
Expand Down Expand Up @@ -2231,6 +2231,7 @@ golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJ
golang.org/x/sync v0.0.0-20210220032951-036812b2e83c h1:5KslGYwFpkhGh+Q16bwMP3cOontH8FOep7tGV86Y7SQ=
golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sys v0.0.0-20170830134202-bb24a47a89ea/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20180810173357-98c5dad5d1a0/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20180823144017-11551d06cbcc/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20180903190138-2b024373dcd9/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
Expand Down Expand Up @@ -2351,8 +2352,9 @@ golang.org/x/sys v0.0.0-20210823070655-63515b42dcdf/go.mod h1:oPkhp1MJrh7nUepCBc
golang.org/x/sys v0.0.0-20210906170528-6f6e22806c34/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20210909193231-528a39cd75f3/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20210927094055-39ccf1dd6fa6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20211015200801-69063c4bb744 h1:KzbpndAYEM+4oHRp9JmB2ewj0NHHxO3Z0g7Gus2O1kk=
golang.org/x/sys v0.0.0-20211015200801-69063c4bb744/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359 h1:2B5p2L5IfGiD7+b9BOoRMC6DgObAVZV+Fsp050NqXik=
golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw=
golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo=
golang.org/x/term v0.0.0-20210220032956-6a3ed077a48d h1:SZxvLBoTP5yHO3Frd4z4vrF+DBX9vMVanchswa69toE=
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/restore_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -959,7 +959,7 @@ func maybeUpgradeDescriptors(
if tableDesc, isTable := desc.(catalog.TableDescriptor); isTable {
b = tabledesc.NewBuilderForFKUpgrade(tableDesc.TableDesc(), skipFKsWithNoMatchingTable)
} else {
b = catalogkv.NewBuilder(desc.DescriptorProto())
b = desc.NewBuilder()
}
err := b.RunPostDeserializationChanges(ctx, descGetter)
if err != nil {
Expand Down
33 changes: 21 additions & 12 deletions pkg/cmd/dev/test.go
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@ func makeTestCmd(runE func(cmd *cobra.Command, args []string) error) *cobra.Comm
addCommonTestFlags(testCmd)
testCmd.Flags().BoolP(vFlag, "v", false, "enable logging during test runs")
testCmd.Flags().Bool(stressFlag, false, "run tests under stress")
testCmd.Flags().String(stressArgsFlag, "", "Additional arguments to pass to stress")
testCmd.Flags().String(stressArgsFlag, "", "additional arguments to pass to stress")
testCmd.Flags().Bool(raceFlag, false, "run tests using race builds")
testCmd.Flags().Bool(ignoreCacheFlag, false, "ignore cached test runs")
testCmd.Flags().String(rewriteFlag, "", "argument to pass to underlying (only applicable for certain tests, e.g. logic and datadriven tests). If unspecified, -rewrite will be passed to the test binary.")
Expand Down Expand Up @@ -83,9 +83,6 @@ func (d *dev) test(cmd *cobra.Command, commandLine []string) error {
rewrite = "-rewrite"
}

d.log.Printf("unit test args: stress=%t race=%t filter=%s timeout=%s ignore-cache=%t pkgs=%s",
stress, race, filter, timeout, ignoreCache, pkgs)

var args []string
args = append(args, "test")
args = append(args, mustGetRemoteCacheArgs(remoteCacheAddr)...)
Expand Down Expand Up @@ -178,17 +175,29 @@ func (d *dev) test(cmd *cobra.Command, commandLine []string) error {
args = append(args, fmt.Sprintf("--sandbox_writable_path=%s", filepath.Join(workspace, dir)))
}
}
if stress && timeout > 0 {
args = append(args, "--run_under", fmt.Sprintf("%s -maxtime=%s %s", stressTarget, timeout, stressArgs))
// The timeout should be a bit higher than the stress duration.
// Bazel will probably think the timeout for this test isn't so
// long.
args = append(args, fmt.Sprintf("--test_timeout=%d", int((timeout+1*time.Second).Seconds())))
} else if stress {
args = append(args, "--run_under", fmt.Sprintf("%s %s", stressTarget, stressArgs))
if stress {
if timeout > 0 {
args = append(args, "--run_under",
fmt.Sprintf("%s -maxtime=%s %s", stressTarget, timeout, stressArgs))

// The bazel timeout needs to be higher than the stress duration to
// pass reliably.
args = append(args, fmt.Sprintf("--test_timeout=%.0f", (timeout+time.Second).Seconds()))
} else {
// We're running under stress and no timeout is specified. We want
// to respect the timeout passed down to stress[1]. Similar to above
// we want the bazel timeout to be longer, so lets just set it to
// 24h.
//
// [1]: Through --stress-arg=-maxtime or if nothing is specified, a
// -maxtime of 0 that's taken as "run forever")
args = append(args, "--run_under", fmt.Sprintf("%s %s", stressTarget, stressArgs))
args = append(args, fmt.Sprintf("--test_timeout=%.0f", 24*time.Hour.Seconds()))
}
} else if timeout > 0 {
args = append(args, fmt.Sprintf("--test_timeout=%d", int(timeout.Seconds())))
}

if filter != "" {
args = append(args, fmt.Sprintf("--test_filter=%s", filter))
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/dev/testdata/recording/test.txt
Original file line number Diff line number Diff line change
Expand Up @@ -84,7 +84,7 @@ bazel query 'kind(go_test, //pkg/util/tracing:all)'
----
//pkg/util/tracing:tracing_test

bazel test --test_sharding_strategy=disabled //pkg/util/tracing:tracing_test --run_under '@com_github_cockroachdb_stress//:stress ' '--test_filter=TestStartChild*' --test_output streamed
bazel test --test_sharding_strategy=disabled //pkg/util/tracing:tracing_test --run_under '@com_github_cockroachdb_stress//:stress ' --test_timeout=86400 '--test_filter=TestStartChild*' --test_output streamed
----
----
//pkg/util/tracing:tracing_test PASSED in 12.3s
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/dev/testdata/test.txt
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ bazel test //pkg/util/tracing:tracing_test --nocache_test_results '--test_filter
dev test --stress pkg/util/tracing --filter TestStartChild*
----
bazel query 'kind(go_test, //pkg/util/tracing:all)'
bazel test --test_sharding_strategy=disabled //pkg/util/tracing:tracing_test --run_under '@com_github_cockroachdb_stress//:stress ' '--test_filter=TestStartChild*' --test_output streamed
bazel test --test_sharding_strategy=disabled //pkg/util/tracing:tracing_test --run_under '@com_github_cockroachdb_stress//:stress ' --test_timeout=86400 '--test_filter=TestStartChild*' --test_output streamed

dev test --stress pkg/util/tracing --filter TestStartChild* --timeout=10s -v
----
Expand Down
5 changes: 1 addition & 4 deletions pkg/kv/kvserver/replica_proposal.go
Original file line number Diff line number Diff line change
Expand Up @@ -913,9 +913,6 @@ func (r *Replica) getTraceData(ctx context.Context) map[string]string {
traceCarrier := tracing.MapCarrier{
Map: make(map[string]string),
}
if err := r.AmbientContext.Tracer.InjectMetaInto(sp.Meta(), traceCarrier); err != nil {
log.Errorf(ctx, "failed to inject sp context (%+v) as trace data: %s", sp.Meta(), err)
return nil
}
r.AmbientContext.Tracer.InjectMetaInto(sp.Meta(), traceCarrier)
return traceCarrier.Map
}
3 changes: 2 additions & 1 deletion pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -346,12 +346,13 @@ go_library(
"//pkg/sql/rowinfra",
"//pkg/sql/schemachange",
"//pkg/sql/schemachanger/scbuild",
"//pkg/sql/schemachanger/scdeps",
"//pkg/sql/schemachanger/scexec",
"//pkg/sql/schemachanger/scgraphviz",
"//pkg/sql/schemachanger/scop",
"//pkg/sql/schemachanger/scpb",
"//pkg/sql/schemachanger/scplan",
"//pkg/sql/schemachanger/screl",
"//pkg/sql/schemachanger/scrun",
"//pkg/sql/scrub",
"//pkg/sql/sem/builtins",
"//pkg/sql/sem/transform",
Expand Down
5 changes: 5 additions & 0 deletions pkg/sql/catalog/dbdesc/database_desc.go
Original file line number Diff line number Diff line change
Expand Up @@ -135,6 +135,11 @@ func (desc *immutable) DescriptorProto() *descpb.Descriptor {
}
}

// NewBuilder implements the catalog.Descriptor interface.
func (desc *immutable) NewBuilder() catalog.DescriptorBuilder {
return NewBuilder(desc.DatabaseDesc())
}

// IsMultiRegion implements the DatabaseDescriptor interface.
func (desc *immutable) IsMultiRegion() bool {
return desc.RegionConfig != nil
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/catalog/descriptor.go
Original file line number Diff line number Diff line change
Expand Up @@ -182,6 +182,9 @@ type Descriptor interface {
// DescriptorProto prepares this descriptor for serialization.
DescriptorProto() *descpb.Descriptor

// NewBuilder initializes a DescriptorBuilder with this descriptor.
NewBuilder() DescriptorBuilder

// GetReferencedDescIDs returns the IDs of all descriptors directly referenced
// by this descriptor, including itself.
GetReferencedDescIDs() (DescriptorIDSet, error)
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/catalog/descs/database.go
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,7 @@ func (tc *Collection) getDatabaseByName(
ctx context.Context, txn *kv.Txn, name string, flags tree.DatabaseLookupFlags,
) (catalog.DatabaseDescriptor, error) {
found, desc, err := tc.getByName(
ctx, txn, nil, nil, name, flags.AvoidCached, flags.RequireMutable,
ctx, txn, nil, nil, name, flags.AvoidCached, flags.RequireMutable, flags.AvoidSynthetic,
)
if err != nil {
return nil, err
Expand Down
10 changes: 5 additions & 5 deletions pkg/sql/catalog/descs/descriptor.go
Original file line number Diff line number Diff line change
Expand Up @@ -158,20 +158,20 @@ func (tc *Collection) getByName(
db catalog.DatabaseDescriptor,
sc catalog.SchemaDescriptor,
name string,
avoidCached, mutable bool,
avoidCached, mutable, avoidSynthetic bool,
) (found bool, desc catalog.Descriptor, err error) {

var parentID, parentSchemaID descpb.ID
if db != nil {
if sc == nil {
// Schema descriptors are handled in a special way, see getSchemaByName
// function declaration for details.
return getSchemaByName(ctx, tc, txn, db, name, avoidCached, mutable)
return getSchemaByName(ctx, tc, txn, db, name, avoidCached, mutable, avoidSynthetic)
}
parentID, parentSchemaID = db.GetID(), sc.GetID()
}

if found, sd := tc.synthetic.getByName(parentID, parentSchemaID, name); found {
if found, sd := tc.synthetic.getByName(parentID, parentSchemaID, name); found && !avoidSynthetic {
if mutable {
return false, nil, newMutableSyntheticDescriptorAssertionError(sd.GetID())
}
Expand Down Expand Up @@ -255,8 +255,7 @@ func getSchemaByName(
txn *kv.Txn,
db catalog.DatabaseDescriptor,
name string,
avoidCached bool,
mutable bool,
avoidCached, mutable, avoidSynthetic bool,
) (bool, catalog.Descriptor, error) {
if name == tree.PublicSchema {
return true, schemadesc.GetPublicSchema(), nil
Expand All @@ -277,6 +276,7 @@ func getSchemaByName(
sc, err := tc.getSchemaByID(ctx, txn, id, tree.SchemaLookupFlags{
RequireMutable: mutable,
AvoidCached: avoidCached,
AvoidSynthetic: avoidSynthetic,
})
// Deal with the fact that ByID retrieval always uses required and the
// logic here never returns an error if the descriptor does not exist.
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/catalog/descs/object.go
Original file line number Diff line number Diff line change
Expand Up @@ -192,7 +192,7 @@ func (tc *Collection) getObjectByNameIgnoringRequiredAndType(

prefix.Schema = sc
found, obj, err := tc.getByName(
ctx, txn, db, sc, objectName, flags.AvoidCached, flags.RequireMutable,
ctx, txn, db, sc, objectName, flags.AvoidCached, flags.RequireMutable, flags.AvoidSynthetic,
)
if !found || err != nil {
return prefix, nil, err
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/catalog/descs/schema.go
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,7 @@ func (tc *Collection) getSchemaByName(
flags tree.SchemaLookupFlags,
) (catalog.SchemaDescriptor, error) {
found, desc, err := tc.getByName(
ctx, txn, db, nil, schemaName, flags.AvoidCached, flags.RequireMutable,
ctx, txn, db, nil, schemaName, flags.AvoidCached, flags.RequireMutable, flags.AvoidSynthetic,
)
if err != nil {
return nil, err
Expand Down
3 changes: 1 addition & 2 deletions pkg/sql/catalog/descs/uncommitted_descriptors.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,6 @@ package descs

import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/lease"
Expand Down Expand Up @@ -63,7 +62,7 @@ func (u *uncommittedDescriptor) getMutable() catalog.MutableDescriptor {
if u.mutable != nil {
return u.mutable
}
return catalogkv.NewBuilder(u.immutable.DescriptorProto()).BuildExistingMutable()
return u.immutable.NewBuilder().BuildExistingMutable()
}

var _ catalog.NameEntry = (*uncommittedDescriptor)(nil)
Expand Down
5 changes: 5 additions & 0 deletions pkg/sql/catalog/schemadesc/schema_desc.go
Original file line number Diff line number Diff line change
Expand Up @@ -141,6 +141,11 @@ func (desc *immutable) DescriptorProto() *descpb.Descriptor {
}
}

// NewBuilder implements the catalog.Descriptor interface.
func (desc *immutable) NewBuilder() catalog.DescriptorBuilder {
return NewBuilder(desc.SchemaDesc())
}

// ValidateSelf implements the catalog.Descriptor interface.
func (desc *immutable) ValidateSelf(vea catalog.ValidationErrorAccumulator) {
// Validate local properties of the descriptor.
Expand Down
5 changes: 5 additions & 0 deletions pkg/sql/catalog/schemadesc/synthetic_schema_desc.go
Original file line number Diff line number Diff line change
Expand Up @@ -78,6 +78,11 @@ func (p synthetic) DescriptorProto() *descpb.Descriptor {
"%s schema cannot be encoded", p.kindName())
return nil // unreachable
}
func (p synthetic) NewBuilder() catalog.DescriptorBuilder {
log.Fatalf(context.TODO(),
"%s schema cannot create a builder", p.kindName())
return nil // unreachable
}
func (p synthetic) GetReferencedDescIDs() (catalog.DescriptorIDSet, error) {
return catalog.DescriptorIDSet{}, nil
}
Expand Down
5 changes: 5 additions & 0 deletions pkg/sql/catalog/tabledesc/table_desc.go
Original file line number Diff line number Diff line change
Expand Up @@ -101,6 +101,11 @@ func (desc *wrapper) DescriptorProto() *descpb.Descriptor {
}
}

// NewBuilder implements the catalog.Descriptor interface.
func (desc *wrapper) NewBuilder() catalog.DescriptorBuilder {
return NewBuilder(desc.TableDesc())
}

// GetPrimaryIndexID implements the TableDescriptor interface.
func (desc *wrapper) GetPrimaryIndexID() descpb.IndexID {
return desc.PrimaryIndex.ID
Expand Down
6 changes: 6 additions & 0 deletions pkg/sql/catalog/typedesc/table_implicit_record_type.go
Original file line number Diff line number Diff line change
Expand Up @@ -176,6 +176,12 @@ func (v TableImplicitRecordType) DescriptorProto() *descpb.Descriptor {
return nil
}

// NewBuilder implements the Descriptor interface.
func (v TableImplicitRecordType) NewBuilder() catalog.DescriptorBuilder {
v.panicNotSupported("NewBuilder")
return nil
}

// GetReferencedDescIDs implements the Descriptor interface.
func (v TableImplicitRecordType) GetReferencedDescIDs() (catalog.DescriptorIDSet, error) {
return catalog.DescriptorIDSet{}, errors.AssertionFailedf(
Expand Down
5 changes: 5 additions & 0 deletions pkg/sql/catalog/typedesc/type_desc.go
Original file line number Diff line number Diff line change
Expand Up @@ -177,6 +177,11 @@ func (desc *immutable) DescriptorProto() *descpb.Descriptor {
}
}

// NewBuilder implements the catalog.Descriptor interface.
func (desc *immutable) NewBuilder() catalog.DescriptorBuilder {
return NewBuilder(desc.TypeDesc())
}

// PrimaryRegionName implements the TypeDescriptor interface.
func (desc *immutable) PrimaryRegionName() (descpb.RegionName, error) {
if desc.Kind != descpb.TypeDescriptor_MULTIREGION_ENUM {
Expand Down
Loading

0 comments on commit 90881be

Please sign in to comment.