Skip to content

Commit

Permalink
sql: use catalog.TableDescriptor instead of tabledesc.Immutable
Browse files Browse the repository at this point in the history
Previously we tended to use the concrete tabledesc.Immutable type
rather than the catalog.TableDescriptor type. This was mainly for
historical reasons because the former predates the latter. This patch
builds on my previous commits by switching to the interface type
everywhere outside of the tabledesc package.

This required making the MakeImmutable factory function in tabledesc
private, and changing the signatures of the NewImmutable factory
functions and using them instead of MakeImmutable.

Fixes #59486.

Release note: None
  • Loading branch information
Marius Posta committed Feb 1, 2021
1 parent eeaace8 commit 6c5acc2
Show file tree
Hide file tree
Showing 119 changed files with 386 additions and 384 deletions.
8 changes: 4 additions & 4 deletions pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
Expand Down Expand Up @@ -5885,14 +5886,13 @@ func getMockIndexDesc(indexID descpb.IndexID) descpb.IndexDescriptor {

func getMockTableDesc(
tableID descpb.ID, pkIndex descpb.IndexDescriptor, indexes []descpb.IndexDescriptor,
) tabledesc.Immutable {
) catalog.TableDescriptor {
mockTableDescriptor := descpb.TableDescriptor{
ID: tableID,
PrimaryIndex: pkIndex,
Indexes: indexes,
}
mockImmutableTableDesc := tabledesc.MakeImmutable(mockTableDescriptor)
return mockImmutableTableDesc
return tabledesc.NewImmutable(mockTableDescriptor)
}

// Unit tests for the getLogicallyMergedTableSpans() method.
Expand Down Expand Up @@ -5986,7 +5986,7 @@ func TestLogicallyMergedTableSpans(t *testing.T) {
for _, test := range testCases {
t.Run(test.name, func(t *testing.T) {
tableDesc := getMockTableDesc(test.tableID, test.pkIndex, test.indexes)
spans, err := getLogicallyMergedTableSpans(&tableDesc, unusedMap, codec,
spans, err := getLogicallyMergedTableSpans(tableDesc, unusedMap, codec,
hlc.Timestamp{}, test.checkForKVInBoundsOverride)
var mergedSpans []string
for _, span := range spans {
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/targets_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ func TestDescriptorsMatchingTargets(t *testing.T) {
type tbDesc = descpb.TableDescriptor
type typDesc = descpb.TypeDescriptor
ts1 := hlc.Timestamp{WallTime: 1}
mkTable := func(descriptor tbDesc) *tabledesc.Immutable {
mkTable := func(descriptor tbDesc) catalog.TableDescriptor {
desc := tabledesc.NewImmutable(descriptor)
desc.TableDesc().ModificationTime = ts1
return desc
Expand Down
1 change: 0 additions & 1 deletion pkg/ccl/changefeedccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,6 @@ go_library(
"//pkg/sql/catalog/hydratedtables",
"//pkg/sql/catalog/lease",
"//pkg/sql/catalog/resolver",
"//pkg/sql/catalog/tabledesc",
"//pkg/sql/execinfra",
"//pkg/sql/execinfrapb",
"//pkg/sql/flowinfra",
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/kvfeed/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -54,8 +54,8 @@ go_test(
"//pkg/keys",
"//pkg/roachpb",
"//pkg/settings/cluster",
"//pkg/sql/catalog",
"//pkg/sql/catalog/descpb",
"//pkg/sql/catalog/tabledesc",
"//pkg/sql/rowenc",
"//pkg/sql/sem/tree",
"//pkg/util/ctxgroup",
Expand Down
12 changes: 6 additions & 6 deletions pkg/ccl/changefeedccl/kvfeed/kv_feed_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,8 +21,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
Expand Down Expand Up @@ -83,7 +83,7 @@ func TestKVFeed(t *testing.T) {
spans []roachpb.Span
events []roachpb.RangeFeedEvent

descs []*tabledesc.Immutable
descs []catalog.TableDescriptor

expScans []hlc.Timestamp
expEvents int
Expand Down Expand Up @@ -193,7 +193,7 @@ func TestKVFeed(t *testing.T) {
ts(2),
ts(3),
},
descs: []*tabledesc.Immutable{
descs: []catalog.TableDescriptor{
makeTableDesc(42, 1, ts(1), 2),
addColumnDropBackfillMutation(makeTableDesc(42, 2, ts(3), 1)),
},
Expand All @@ -217,7 +217,7 @@ func TestKVFeed(t *testing.T) {
expScans: []hlc.Timestamp{
ts(2),
},
descs: []*tabledesc.Immutable{
descs: []catalog.TableDescriptor{
makeTableDesc(42, 1, ts(1), 2),
addColumnDropBackfillMutation(makeTableDesc(42, 2, ts(3), 1)),
},
Expand All @@ -242,7 +242,7 @@ func TestKVFeed(t *testing.T) {
expScans: []hlc.Timestamp{
ts(2),
},
descs: []*tabledesc.Immutable{
descs: []catalog.TableDescriptor{
makeTableDesc(42, 1, ts(1), 2),
addColumnDropBackfillMutation(makeTableDesc(42, 2, ts(4), 1)),
},
Expand All @@ -268,7 +268,7 @@ type rawTableFeed struct {
events []schemafeed.TableEvent
}

func newRawTableFeed(descs []*tabledesc.Immutable, initialHighWater hlc.Timestamp) rawTableFeed {
func newRawTableFeed(descs []catalog.TableDescriptor, initialHighWater hlc.Timestamp) rawTableFeed {
sort.Slice(descs, func(i, j int) bool {
if descs[i].GetID() != descs[j].GetID() {
return descs[i].GetID() < descs[j].GetID()
Expand Down
11 changes: 5 additions & 6 deletions pkg/ccl/changefeedccl/rowfetcher_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/hydratedtables"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/lease"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/sql/row"
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
Expand Down Expand Up @@ -69,8 +68,8 @@ func newRowFetcherCache(

func (c *rowFetcherCache) TableDescForKey(
ctx context.Context, key roachpb.Key, ts hlc.Timestamp,
) (*tabledesc.Immutable, error) {
var tableDesc *tabledesc.Immutable
) (catalog.TableDescriptor, error) {
var tableDesc catalog.TableDescriptor
key, err := c.codec.StripTenantPrefix(key)
if err != nil {
return nil, err
Expand All @@ -94,7 +93,7 @@ func (c *rowFetcherCache) TableDescForKey(
if err := c.leaseMgr.Release(desc); err != nil {
return nil, err
}
tableDesc = desc.(*tabledesc.Immutable)
tableDesc = desc.(catalog.TableDescriptor)
if tableDesc.ContainsUserDefinedTypes() {
// If the table contains user defined types, then use the descs.Collection
// to retrieve a TableDescriptor with type metadata hydrated. We open a
Expand Down Expand Up @@ -140,7 +139,7 @@ func (c *rowFetcherCache) TableDescForKey(
}

func (c *rowFetcherCache) RowFetcherForTableDesc(
tableDesc *tabledesc.Immutable,
tableDesc catalog.TableDescriptor,
) (*row.Fetcher, error) {
idVer := idVersion{id: tableDesc.GetID(), version: tableDesc.GetVersion()}
// Ensure that all user defined types are up to date with the cached
Expand All @@ -149,7 +148,7 @@ func (c *rowFetcherCache) RowFetcherForTableDesc(
// guaranteed that the tables have the same version. Additionally, these
// fetchers are always initialized with a single tabledesc.Immutable.
if rf, ok := c.fetchers[idVer]; ok &&
tableDesc.UserDefinedTypeColsHaveSameVersion(rf.GetTables()[0].(*tabledesc.Immutable)) {
tableDesc.UserDefinedTypeColsHaveSameVersion(rf.GetTables()[0].(catalog.TableDescriptor)) {
return rf, nil
}
// TODO(dan): Allow for decoding a subset of the columns.
Expand Down
16 changes: 8 additions & 8 deletions pkg/ccl/changefeedccl/schemafeed/schema_feed.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ import (

// TableEvent represents a change to a table descriptor.
type TableEvent struct {
Before, After *tabledesc.Immutable
Before, After catalog.TableDescriptor
}

// Timestamp refers to the ModificationTime of the After table descriptor.
Expand Down Expand Up @@ -121,7 +121,7 @@ type SchemaFeed struct {
// of the table descriptor seen by the poller. This is needed to determine
// when a backilling mutation has successfully completed - this can only
// be determining by comparing a version to the previous version.
previousTableVersion map[descpb.ID]*tabledesc.Immutable
previousTableVersion map[descpb.ID]catalog.TableDescriptor

// typeDeps tracks dependencies from target tables to user defined types
// that they use.
Expand Down Expand Up @@ -166,7 +166,7 @@ func (t *typeDependencyTracker) removeDependency(typeID, tableID descpb.ID) {
}
}

func (t *typeDependencyTracker) purgeTable(tbl *tabledesc.Immutable) {
func (t *typeDependencyTracker) purgeTable(tbl catalog.TableDescriptor) {
if !tbl.ContainsUserDefinedTypes() {
return
}
Expand All @@ -176,7 +176,7 @@ func (t *typeDependencyTracker) purgeTable(tbl *tabledesc.Immutable) {
}
}

func (t *typeDependencyTracker) ingestTable(tbl *tabledesc.Immutable) {
func (t *typeDependencyTracker) ingestTable(tbl catalog.TableDescriptor) {
if !tbl.ContainsUserDefinedTypes() {
return
}
Expand Down Expand Up @@ -207,7 +207,7 @@ func New(cfg Config) *SchemaFeed {
targets: cfg.Targets,
leaseMgr: cfg.LeaseManager,
}
m.mu.previousTableVersion = make(map[descpb.ID]*tabledesc.Immutable)
m.mu.previousTableVersion = make(map[descpb.ID]catalog.TableDescriptor)
m.mu.highWater = cfg.InitialHighWater
m.mu.typeDeps = typeDependencyTracker{deps: make(map[descpb.ID][]descpb.ID)}
return m
Expand Down Expand Up @@ -276,7 +276,7 @@ func (tf *SchemaFeed) primeInitialTableDescs(ctx context.Context) error {
tf.mu.Lock()
// Register all types used by the initial set of tables.
for _, desc := range initialDescs {
tbl := desc.(*tabledesc.Immutable)
tbl := desc.(catalog.TableDescriptor)
tf.mu.typeDeps.ingestTable(tbl)
}
tf.mu.Unlock()
Expand Down Expand Up @@ -474,7 +474,7 @@ func (e TableEvent) String() string {
return formatEvent(e)
}

func formatDesc(desc *tabledesc.Immutable) string {
func formatDesc(desc catalog.TableDescriptor) string {
return fmt.Sprintf("%d:%d@%v", desc.GetID(), desc.GetVersion(), desc.GetModificationTime())
}

Expand All @@ -495,7 +495,7 @@ func (tf *SchemaFeed) validateDescriptor(
// If a interesting type changed, then we just want to force the lease
// manager to acquire the freshest version of the type.
return tf.leaseMgr.AcquireFreshestFromStore(ctx, desc.ID)
case *tabledesc.Immutable:
case catalog.TableDescriptor:
if err := changefeedbase.ValidateTable(tf.targets, desc); err != nil {
return err
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@ go_library(
importpath = "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/schemafeed/schematestutils",
visibility = ["//visibility:public"],
deps = [
"//pkg/sql/catalog",
"//pkg/sql/catalog/descpb",
"//pkg/sql/catalog/tabledesc",
"//pkg/sql/types",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ package schematestutils
import (
"strconv"

"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/sql/types"
Expand All @@ -23,7 +24,7 @@ import (
// MakeTableDesc makes a generic table descriptor with the provided properties.
func MakeTableDesc(
tableID descpb.ID, version descpb.DescriptorVersion, modTime hlc.Timestamp, cols int,
) *tabledesc.Immutable {
) catalog.TableDescriptor {
td := descpb.TableDescriptor{
Name: "foo",
ID: tableID,
Expand All @@ -50,7 +51,7 @@ func MakeColumnDesc(id descpb.ColumnID) *descpb.ColumnDescriptor {

// AddColumnDropBackfillMutation adds a mutation to desc to drop a column.
// Yes, this does modify an Immutable.
func AddColumnDropBackfillMutation(desc *tabledesc.Immutable) *tabledesc.Immutable {
func AddColumnDropBackfillMutation(desc catalog.TableDescriptor) catalog.TableDescriptor {
desc.TableDesc().Mutations = append(desc.TableDesc().Mutations, descpb.DescriptorMutation{
State: descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY,
Direction: descpb.DescriptorMutation_DROP,
Expand All @@ -61,7 +62,7 @@ func AddColumnDropBackfillMutation(desc *tabledesc.Immutable) *tabledesc.Immutab

// AddNewColumnBackfillMutation adds a mutation to desc to add a column.
// Yes, this does modify an Immutable.
func AddNewColumnBackfillMutation(desc *tabledesc.Immutable) *tabledesc.Immutable {
func AddNewColumnBackfillMutation(desc catalog.TableDescriptor) catalog.TableDescriptor {
desc.TableDesc().Mutations = append(desc.TableDesc().Mutations, descpb.DescriptorMutation{
Descriptor_: &descpb.DescriptorMutation_Column{Column: MakeColumnDesc(desc.GetNextColumnID())},
State: descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY,
Expand Down
6 changes: 3 additions & 3 deletions pkg/ccl/changefeedccl/schemafeed/table_event_filter.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,8 +12,8 @@ import (
"context"

"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/errors"
)

Expand Down Expand Up @@ -92,7 +92,7 @@ func hasNewColumnDropBackfillMutation(e TableEvent) (res bool) {
return !dropColumnMutationExists(e.Before) && dropColumnMutationExists(e.After)
}

func dropColumnMutationExists(desc *tabledesc.Immutable) bool {
func dropColumnMutationExists(desc catalog.TableDescriptor) bool {
for _, m := range desc.GetMutations() {
if m.GetColumn() == nil {
continue
Expand All @@ -117,7 +117,7 @@ func newColumnNoBackfill(e TableEvent) (res bool) {
!e.Before.HasColumnBackfillMutation()
}

func pkChangeMutationExists(desc *tabledesc.Immutable) bool {
func pkChangeMutationExists(desc catalog.TableDescriptor) bool {
for _, m := range desc.GetMutations() {
if m.Direction == descpb.DescriptorMutation_ADD && m.GetPrimaryKeySwap() != nil {
return true
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/changefeedccl/schemafeed/table_event_filter_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@ import (
"time"

"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/schemafeed/schematestutils"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/gogo/protobuf/proto"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -73,7 +73,7 @@ func TestTableEventFilter(t *testing.T) {
name: "don't filter end of add NULL-able computed column",
p: defaultTableEventFilter,
e: TableEvent{
Before: func() *tabledesc.Immutable {
Before: func() catalog.TableDescriptor {
td := addColBackfill(mkTableDesc(42, 4, ts(4), 1))
col := td.GetMutations()[0].GetColumn()
col.Nullable = true
Expand Down
7 changes: 4 additions & 3 deletions pkg/ccl/changefeedccl/sink_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/testutils"
Expand Down Expand Up @@ -54,7 +55,7 @@ func TestKafkaSink(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

table := func(name string) *tabledesc.Immutable {
table := func(name string) catalog.TableDescriptor {
return tabledesc.NewImmutable(descpb.TableDescriptor{Name: name})
}

Expand Down Expand Up @@ -146,7 +147,7 @@ func TestKafkaSinkEscaping(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

table := func(name string) *tabledesc.Immutable {
table := func(name string) catalog.TableDescriptor {
return tabledesc.NewImmutable(descpb.TableDescriptor{Name: name})
}

Expand Down Expand Up @@ -187,7 +188,7 @@ func TestSQLSink(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

table := func(name string) *tabledesc.Immutable {
table := func(name string) catalog.TableDescriptor {
id, _ := strconv.ParseUint(name, 36, 64)
return tabledesc.NewImmutable(descpb.TableDescriptor{Name: name, ID: descpb.ID(id)})
}
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/importccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -133,6 +133,7 @@ go_test(
"//pkg/server",
"//pkg/settings/cluster",
"//pkg/sql",
"//pkg/sql/catalog",
"//pkg/sql/catalog/catalogkv",
"//pkg/sql/catalog/catformat",
"//pkg/sql/catalog/descpb",
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/importccl/import_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -173,7 +173,7 @@ func makeInputConverter(
seqChunkProvider *row.SeqChunkProvider,
) (inputConverter, error) {
injectTimeIntoEvalCtx(evalCtx, spec.WalltimeNanos)
var singleTable *tabledesc.Immutable
var singleTable catalog.TableDescriptor
var singleTableTargetCols tree.NameList
if len(spec.Tables) == 1 {
for _, table := range spec.Tables {
Expand Down
6 changes: 3 additions & 3 deletions pkg/ccl/importccl/import_stmt.go
Original file line number Diff line number Diff line change
Expand Up @@ -1641,15 +1641,15 @@ func (r *importResumer) dropTables(
return nil
}

var revert []*tabledesc.Immutable
var empty []*tabledesc.Immutable
var revert []catalog.TableDescriptor
var empty []catalog.TableDescriptor
for _, tbl := range details.Tables {
if !tbl.IsNew {
desc, err := descsCol.GetMutableTableVersionByID(ctx, tbl.Desc.ID, txn)
if err != nil {
return err
}
imm := desc.ImmutableCopy().(*tabledesc.Immutable)
imm := desc.ImmutableCopy().(catalog.TableDescriptor)
if tbl.WasEmpty {
empty = append(empty, imm)
} else {
Expand Down
Loading

0 comments on commit 6c5acc2

Please sign in to comment.