From 6c5acc2af0e3b16dd75c34ce0154154222541c9e Mon Sep 17 00:00:00 2001 From: Marius Posta Date: Wed, 27 Jan 2021 16:22:34 -0500 Subject: [PATCH] sql: use catalog.TableDescriptor instead of tabledesc.Immutable 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 --- pkg/ccl/backupccl/backup_test.go | 8 ++-- pkg/ccl/backupccl/targets_test.go | 2 +- pkg/ccl/changefeedccl/BUILD.bazel | 1 - pkg/ccl/changefeedccl/kvfeed/BUILD.bazel | 2 +- pkg/ccl/changefeedccl/kvfeed/kv_feed_test.go | 12 +++--- pkg/ccl/changefeedccl/rowfetcher_cache.go | 11 +++--- .../changefeedccl/schemafeed/schema_feed.go | 16 ++++---- .../schemafeed/schematestutils/BUILD.bazel | 1 + .../schematestutils/schema_test_utils.go | 7 ++-- .../schemafeed/table_event_filter.go | 6 +-- .../schemafeed/table_event_filter_test.go | 4 +- pkg/ccl/changefeedccl/sink_test.go | 7 ++-- pkg/ccl/importccl/BUILD.bazel | 1 + pkg/ccl/importccl/import_processor.go | 2 +- pkg/ccl/importccl/import_stmt.go | 6 +-- pkg/ccl/importccl/import_stmt_test.go | 10 ++--- pkg/ccl/importccl/import_table_creation.go | 2 +- pkg/ccl/importccl/read_import_avro.go | 4 +- pkg/ccl/importccl/read_import_avro_test.go | 8 ++-- pkg/ccl/importccl/read_import_base.go | 18 ++++----- pkg/ccl/importccl/read_import_csv.go | 4 +- pkg/ccl/importccl/read_import_mysqlout.go | 4 +- pkg/ccl/importccl/read_import_pgcopy.go | 4 +- pkg/ccl/importccl/read_import_workload.go | 10 ++--- pkg/ccl/storageccl/BUILD.bazel | 1 + pkg/ccl/storageccl/key_rewriter.go | 7 ++-- pkg/ccl/workloadccl/format/BUILD.bazel | 2 +- pkg/ccl/workloadccl/format/sstable.go | 6 +-- pkg/sql/backfill.go | 22 +++++------ pkg/sql/backfill/backfill.go | 34 +++++++++------- pkg/sql/catalog/catalogkv/catalogkv.go | 4 +- pkg/sql/catalog/catalogkv/test_utils.go | 10 ++--- pkg/sql/catalog/descs/collection.go | 17 ++++---- .../catalog/hydratedtables/hydratedcache.go | 8 ++-- .../hydratedtables/hydratedcache_test.go | 18 ++++----- pkg/sql/catalog/lease/lease_internal_test.go | 28 ++++++------- pkg/sql/catalog/lease/lease_test.go | 2 +- pkg/sql/catalog/resolver/resolver.go | 8 ++-- pkg/sql/catalog/tabledesc/structured.go | 14 +++---- pkg/sql/catalog/tabledesc/structured_test.go | 4 +- pkg/sql/catalog/tabledesc/table_desc.go | 2 +- pkg/sql/colfetcher/colbatch_scan.go | 2 +- pkg/sql/comment_on_column.go | 4 +- pkg/sql/comment_on_table.go | 4 +- pkg/sql/crdb_internal.go | 8 ++-- pkg/sql/create_stats.go | 6 +-- pkg/sql/create_table.go | 2 +- pkg/sql/delete_range.go | 6 +-- pkg/sql/distsql_physical_planner.go | 14 ++++--- pkg/sql/distsql_physical_planner_test.go | 4 +- pkg/sql/distsql_plan_csv.go | 2 +- pkg/sql/distsql_plan_stats.go | 2 +- pkg/sql/drop_test.go | 3 +- pkg/sql/gcjob/index_garbage_collection.go | 8 ++-- pkg/sql/gcjob/refresh_statuses.go | 5 +-- pkg/sql/gcjob/table_garbage_collection.go | 5 +-- pkg/sql/insert_fast_path.go | 3 +- pkg/sql/old_foreign_key_desc_test.go | 7 ++-- pkg/sql/opt_catalog.go | 39 +++++++++---------- pkg/sql/opt_exec_factory.go | 7 ++-- pkg/sql/physicalplan/BUILD.bazel | 2 +- pkg/sql/physicalplan/aggregator_funcs_test.go | 4 +- pkg/sql/physicalplan/span_resolver_test.go | 8 ++-- pkg/sql/planhook.go | 3 +- pkg/sql/planner.go | 3 +- pkg/sql/reassign_owned_by.go | 3 +- pkg/sql/relocate.go | 4 +- pkg/sql/rename_database.go | 3 +- pkg/sql/resolver.go | 14 +++---- pkg/sql/revert.go | 4 +- pkg/sql/revert_test.go | 10 ++--- pkg/sql/row/BUILD.bazel | 1 - pkg/sql/row/deleter.go | 3 +- pkg/sql/row/expr_walker.go | 8 ++-- pkg/sql/row/expr_walker_test.go | 7 ++-- pkg/sql/row/fetcher.go | 3 +- pkg/sql/row/fetcher_mvcc_test.go | 3 +- pkg/sql/row/fetcher_test.go | 4 +- pkg/sql/row/inserter.go | 3 +- pkg/sql/row/row_converter.go | 7 ++-- pkg/sql/row/updater.go | 3 +- pkg/sql/rowenc/index_encoding_test.go | 4 +- pkg/sql/rowexec/BUILD.bazel | 1 - pkg/sql/rowexec/bulk_row_writer.go | 9 +++-- pkg/sql/rowexec/columnbackfiller.go | 3 +- pkg/sql/rowexec/indexbackfiller.go | 3 +- pkg/sql/rowexec/inverted_joiner.go | 12 +++--- pkg/sql/rowexec/joinreader.go | 8 ++-- pkg/sql/rowexec/joinreader_test.go | 3 +- pkg/sql/rowexec/rowfetcher.go | 3 +- pkg/sql/rowexec/scrub_tablereader.go | 6 +-- pkg/sql/rowexec/zigzagjoiner.go | 10 ++--- pkg/sql/scan.go | 7 ++-- pkg/sql/schema_changer.go | 12 +++--- pkg/sql/schema_changer_test.go | 8 ++-- pkg/sql/scrub.go | 14 +++---- pkg/sql/scrub_constraint.go | 5 ++- pkg/sql/scrub_fk.go | 6 +-- pkg/sql/scrub_index.go | 5 +-- pkg/sql/scrub_physical.go | 5 +-- pkg/sql/sequence.go | 4 +- pkg/sql/sequence_select.go | 6 +-- pkg/sql/sequence_test.go | 4 +- pkg/sql/show_create.go | 3 +- pkg/sql/show_fingerprints.go | 4 +- pkg/sql/show_stats.go | 4 +- pkg/sql/span/BUILD.bazel | 1 - pkg/sql/span/span_builder.go | 5 +-- pkg/sql/split.go | 3 +- pkg/sql/table_test.go | 8 ++-- pkg/sql/tests/BUILD.bazel | 2 +- pkg/sql/tests/hash_sharded_test.go | 6 +-- pkg/sql/tests/system_table_test.go | 4 +- pkg/sql/truncate.go | 6 ++- pkg/sql/unsplit.go | 3 +- pkg/sql/views.go | 3 +- pkg/sql/virtual_schema.go | 6 +-- pkg/sql/virtual_table.go | 4 +- pkg/sql/zone_config.go | 2 +- 119 files changed, 386 insertions(+), 384 deletions(-) diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index 5f97650f1c0d..26138e6cd8dc 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -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" @@ -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. @@ -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 { diff --git a/pkg/ccl/backupccl/targets_test.go b/pkg/ccl/backupccl/targets_test.go index 98e107e9bdf4..e9f8d8f0495d 100644 --- a/pkg/ccl/backupccl/targets_test.go +++ b/pkg/ccl/backupccl/targets_test.go @@ -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 diff --git a/pkg/ccl/changefeedccl/BUILD.bazel b/pkg/ccl/changefeedccl/BUILD.bazel index 365e79a4ac40..c8bdfd4e18d6 100644 --- a/pkg/ccl/changefeedccl/BUILD.bazel +++ b/pkg/ccl/changefeedccl/BUILD.bazel @@ -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", diff --git a/pkg/ccl/changefeedccl/kvfeed/BUILD.bazel b/pkg/ccl/changefeedccl/kvfeed/BUILD.bazel index 1607c66fa9e5..c2b66a8c0abe 100644 --- a/pkg/ccl/changefeedccl/kvfeed/BUILD.bazel +++ b/pkg/ccl/changefeedccl/kvfeed/BUILD.bazel @@ -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", diff --git a/pkg/ccl/changefeedccl/kvfeed/kv_feed_test.go b/pkg/ccl/changefeedccl/kvfeed/kv_feed_test.go index d7f91482f451..9a7212ca1122 100644 --- a/pkg/ccl/changefeedccl/kvfeed/kv_feed_test.go +++ b/pkg/ccl/changefeedccl/kvfeed/kv_feed_test.go @@ -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" @@ -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 @@ -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)), }, @@ -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)), }, @@ -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)), }, @@ -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() diff --git a/pkg/ccl/changefeedccl/rowfetcher_cache.go b/pkg/ccl/changefeedccl/rowfetcher_cache.go index d5d7ba940dc3..cf0072e6dddc 100644 --- a/pkg/ccl/changefeedccl/rowfetcher_cache.go +++ b/pkg/ccl/changefeedccl/rowfetcher_cache.go @@ -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" @@ -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 @@ -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 @@ -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 @@ -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. diff --git a/pkg/ccl/changefeedccl/schemafeed/schema_feed.go b/pkg/ccl/changefeedccl/schemafeed/schema_feed.go index c0050ecfa9ac..40a3b882b69e 100644 --- a/pkg/ccl/changefeedccl/schemafeed/schema_feed.go +++ b/pkg/ccl/changefeedccl/schemafeed/schema_feed.go @@ -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. @@ -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. @@ -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 } @@ -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 } @@ -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 @@ -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() @@ -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()) } @@ -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 } diff --git a/pkg/ccl/changefeedccl/schemafeed/schematestutils/BUILD.bazel b/pkg/ccl/changefeedccl/schemafeed/schematestutils/BUILD.bazel index 409eeb8c53f1..a73520490d16 100644 --- a/pkg/ccl/changefeedccl/schemafeed/schematestutils/BUILD.bazel +++ b/pkg/ccl/changefeedccl/schemafeed/schematestutils/BUILD.bazel @@ -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", diff --git a/pkg/ccl/changefeedccl/schemafeed/schematestutils/schema_test_utils.go b/pkg/ccl/changefeedccl/schemafeed/schematestutils/schema_test_utils.go index c67c51b4408c..3e6cbe0e3100 100644 --- a/pkg/ccl/changefeedccl/schemafeed/schematestutils/schema_test_utils.go +++ b/pkg/ccl/changefeedccl/schemafeed/schematestutils/schema_test_utils.go @@ -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" @@ -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, @@ -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, @@ -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, diff --git a/pkg/ccl/changefeedccl/schemafeed/table_event_filter.go b/pkg/ccl/changefeedccl/schemafeed/table_event_filter.go index eb1bde896fae..b75b31c0af45 100644 --- a/pkg/ccl/changefeedccl/schemafeed/table_event_filter.go +++ b/pkg/ccl/changefeedccl/schemafeed/table_event_filter.go @@ -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" ) @@ -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 @@ -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 diff --git a/pkg/ccl/changefeedccl/schemafeed/table_event_filter_test.go b/pkg/ccl/changefeedccl/schemafeed/table_event_filter_test.go index 86f8fca482da..fde8002e8a72 100644 --- a/pkg/ccl/changefeedccl/schemafeed/table_event_filter_test.go +++ b/pkg/ccl/changefeedccl/schemafeed/table_event_filter_test.go @@ -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" @@ -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 diff --git a/pkg/ccl/changefeedccl/sink_test.go b/pkg/ccl/changefeedccl/sink_test.go index f8ea3de82883..2d5e8d8c79ce 100644 --- a/pkg/ccl/changefeedccl/sink_test.go +++ b/pkg/ccl/changefeedccl/sink_test.go @@ -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" @@ -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}) } @@ -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}) } @@ -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)}) } diff --git a/pkg/ccl/importccl/BUILD.bazel b/pkg/ccl/importccl/BUILD.bazel index e163cffb884e..2350a4f2b8ad 100644 --- a/pkg/ccl/importccl/BUILD.bazel +++ b/pkg/ccl/importccl/BUILD.bazel @@ -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", diff --git a/pkg/ccl/importccl/import_processor.go b/pkg/ccl/importccl/import_processor.go index 9fdb3ed04381..7aae80e09781 100644 --- a/pkg/ccl/importccl/import_processor.go +++ b/pkg/ccl/importccl/import_processor.go @@ -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 { diff --git a/pkg/ccl/importccl/import_stmt.go b/pkg/ccl/importccl/import_stmt.go index 7864cdc565ff..15ff10424af3 100644 --- a/pkg/ccl/importccl/import_stmt.go +++ b/pkg/ccl/importccl/import_stmt.go @@ -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 { diff --git a/pkg/ccl/importccl/import_stmt_test.go b/pkg/ccl/importccl/import_stmt_test.go index eb9f171e0ee1..986d0ab6073b 100644 --- a/pkg/ccl/importccl/import_stmt_test.go +++ b/pkg/ccl/importccl/import_stmt_test.go @@ -39,9 +39,9 @@ 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" "github.com/cockroachdb/cockroach/pkg/sql/distsql" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/gcjob" @@ -2097,7 +2097,7 @@ func TestImportCSVStmt(t *testing.T) { // it was created in. dbID := sqlutils.QueryDatabaseID(t, sqlDB.DB, "failedimport") tableID := descpb.ID(dbID + 1) - var td *tabledesc.Immutable + var td catalog.TableDescriptor if err := kvDB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { td, err = catalogkv.MustGetTableDescByID(ctx, txn, keys.SystemSQLCodec, tableID) return err @@ -3634,7 +3634,7 @@ func BenchmarkCSVConvertRecord(b *testing.B) { importCtx := ¶llelImportContext{ evalCtx: &evalCtx, - tableDesc: tableDesc.ImmutableCopy().(*tabledesc.Immutable), + tableDesc: tableDesc.ImmutableCopy().(catalog.TableDescriptor), kvCh: kvCh, } @@ -4541,7 +4541,7 @@ func BenchmarkDelimitedConvertRecord(b *testing.B) { RowSeparator: '\n', FieldSeparator: '\t', }, kvCh, 0, 0, - tableDesc.ImmutableCopy().(*tabledesc.Immutable), nil /* targetCols */, &evalCtx) + tableDesc.ImmutableCopy().(catalog.TableDescriptor), nil /* targetCols */, &evalCtx) require.NoError(b, err) producer := &csvBenchmarkStream{ @@ -4644,7 +4644,7 @@ func BenchmarkPgCopyConvertRecord(b *testing.B) { Null: `\N`, MaxRowSize: 4096, }, kvCh, 0, 0, - tableDesc.ImmutableCopy().(*tabledesc.Immutable), nil /* targetCols */, &evalCtx) + tableDesc.ImmutableCopy().(catalog.TableDescriptor), nil /* targetCols */, &evalCtx) require.NoError(b, err) producer := &csvBenchmarkStream{ diff --git a/pkg/ccl/importccl/import_table_creation.go b/pkg/ccl/importccl/import_table_creation.go index dcc7f3d84fea..cfac361306df 100644 --- a/pkg/ccl/importccl/import_table_creation.go +++ b/pkg/ccl/importccl/import_table_creation.go @@ -318,7 +318,7 @@ func (r fkResolver) LookupSchema( // Implements the sql.SchemaResolver interface. func (r fkResolver) LookupTableByID( ctx context.Context, id descpb.ID, -) (*tabledesc.Immutable, error) { +) (catalog.TableDescriptor, error) { return nil, errSchemaResolver } diff --git a/pkg/ccl/importccl/read_import_avro.go b/pkg/ccl/importccl/read_import_avro.go index 4f02d3d8f717..f7f67375b51d 100644 --- a/pkg/ccl/importccl/read_import_avro.go +++ b/pkg/ccl/importccl/read_import_avro.go @@ -18,7 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/lexbase" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -454,7 +454,7 @@ var _ inputConverter = &avroInputReader{} func newAvroInputReader( kvCh chan row.KVBatch, - tableDesc *tabledesc.Immutable, + tableDesc catalog.TableDescriptor, avroOpts roachpb.AvroOptions, walltime int64, parallelism int, diff --git a/pkg/ccl/importccl/read_import_avro_test.go b/pkg/ccl/importccl/read_import_avro_test.go index b8e0b09e3743..0e747781d2d9 100644 --- a/pkg/ccl/importccl/read_import_avro_test.go +++ b/pkg/ccl/importccl/read_import_avro_test.go @@ -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/parser" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -142,7 +142,7 @@ func (g *intArrGen) Gen() interface{} { // A testHelper to generate avro data. type testHelper struct { schemaJSON string - schemaTable *tabledesc.Immutable + schemaTable catalog.TableDescriptor codec *goavro.Codec gens []avroGen settings *cluster.Settings @@ -200,7 +200,7 @@ func newTestHelper(ctx context.Context, t *testing.T, gens ...avroGen) *testHelp return &testHelper{ schemaJSON: string(schemaJSON), schemaTable: descForTable(ctx, t, createStmt, 10, 20, NoFKs). - ImmutableCopy().(*tabledesc.Immutable), + ImmutableCopy().(catalog.TableDescriptor), codec: codec, gens: gens, settings: st, @@ -595,7 +595,7 @@ func benchmarkAvroImport(b *testing.B, avroOpts roachpb.AvroOptions, testData st require.NoError(b, err) avro, err := newAvroInputReader(kvCh, - tableDesc.ImmutableCopy().(*tabledesc.Immutable), + tableDesc.ImmutableCopy().(catalog.TableDescriptor), avroOpts, 0, 0, &evalCtx) require.NoError(b, err) diff --git a/pkg/ccl/importccl/read_import_base.go b/pkg/ccl/importccl/read_import_base.go index 7f2dbcee48fa..dc1693232339 100644 --- a/pkg/ccl/importccl/read_import_base.go +++ b/pkg/ccl/importccl/read_import_base.go @@ -26,7 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -412,14 +412,14 @@ func newImportRowError(err error, row string, num int64) error { // parallelImportContext describes state associated with the import. type parallelImportContext struct { - walltime int64 // Import time stamp. - numWorkers int // Parallelism. - batchSize int // Number of records to batch. - evalCtx *tree.EvalContext // Evaluation context. - tableDesc *tabledesc.Immutable // Table descriptor we're importing into. - targetCols tree.NameList // List of columns to import. nil if importing all columns. - kvCh chan row.KVBatch // Channel for sending KV batches. - seqChunkProvider *row.SeqChunkProvider // Used to reserve chunks of sequence values. + walltime int64 // Import time stamp. + numWorkers int // Parallelism. + batchSize int // Number of records to batch. + evalCtx *tree.EvalContext // Evaluation context. + tableDesc catalog.TableDescriptor // Table descriptor we're importing into. + targetCols tree.NameList // List of columns to import. nil if importing all columns. + kvCh chan row.KVBatch // Channel for sending KV batches. + seqChunkProvider *row.SeqChunkProvider // Used to reserve chunks of sequence values. } // importFileContext describes state specific to a file being imported. diff --git a/pkg/ccl/importccl/read_import_csv.go b/pkg/ccl/importccl/read_import_csv.go index d68bc01cad3f..b087ddc39f25 100644 --- a/pkg/ccl/importccl/read_import_csv.go +++ b/pkg/ccl/importccl/read_import_csv.go @@ -15,7 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -39,7 +39,7 @@ func newCSVInputReader( opts roachpb.CSVOptions, walltime int64, parallelism int, - tableDesc *tabledesc.Immutable, + tableDesc catalog.TableDescriptor, targetCols tree.NameList, evalCtx *tree.EvalContext, seqChunkProvider *row.SeqChunkProvider, diff --git a/pkg/ccl/importccl/read_import_mysqlout.go b/pkg/ccl/importccl/read_import_mysqlout.go index 7e847c52422e..43eb38506f2f 100644 --- a/pkg/ccl/importccl/read_import_mysqlout.go +++ b/pkg/ccl/importccl/read_import_mysqlout.go @@ -17,7 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -38,7 +38,7 @@ func newMysqloutfileReader( kvCh chan row.KVBatch, walltime int64, parallelism int, - tableDesc *tabledesc.Immutable, + tableDesc catalog.TableDescriptor, targetCols tree.NameList, evalCtx *tree.EvalContext, ) (*mysqloutfileReader, error) { diff --git a/pkg/ccl/importccl/read_import_pgcopy.go b/pkg/ccl/importccl/read_import_pgcopy.go index 9906fbc77f1e..225de1d14e67 100644 --- a/pkg/ccl/importccl/read_import_pgcopy.go +++ b/pkg/ccl/importccl/read_import_pgcopy.go @@ -19,7 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -44,7 +44,7 @@ func newPgCopyReader( kvCh chan row.KVBatch, walltime int64, parallelism int, - tableDesc *tabledesc.Immutable, + tableDesc catalog.TableDescriptor, targetCols tree.NameList, evalCtx *tree.EvalContext, ) (*pgCopyReader, error) { diff --git a/pkg/ccl/importccl/read_import_workload.go b/pkg/ccl/importccl/read_import_workload.go index 65bac517563e..500997fe3c56 100644 --- a/pkg/ccl/importccl/read_import_workload.go +++ b/pkg/ccl/importccl/read_import_workload.go @@ -20,7 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -36,14 +36,14 @@ import ( type workloadReader struct { evalCtx *tree.EvalContext - table *tabledesc.Immutable + table catalog.TableDescriptor kvCh chan row.KVBatch } var _ inputConverter = &workloadReader{} func newWorkloadReader( - kvCh chan row.KVBatch, table *tabledesc.Immutable, evalCtx *tree.EvalContext, + kvCh chan row.KVBatch, table catalog.TableDescriptor, evalCtx *tree.EvalContext, ) *workloadReader { return &workloadReader{evalCtx: evalCtx, table: table, kvCh: kvCh} } @@ -178,7 +178,7 @@ func (w *workloadReader) readFiles( // WorkloadKVConverter converts workload.BatchedTuples to []roachpb.KeyValues. type WorkloadKVConverter struct { - tableDesc *tabledesc.Immutable + tableDesc catalog.TableDescriptor rows workload.BatchedTuples batchIdxAtomic int64 batchEnd int @@ -194,7 +194,7 @@ type WorkloadKVConverter struct { // range of batches, emitted converted kvs to the given channel. func NewWorkloadKVConverter( fileID int32, - tableDesc *tabledesc.Immutable, + tableDesc catalog.TableDescriptor, rows workload.BatchedTuples, batchStart, batchEnd int, kvCh chan row.KVBatch, diff --git a/pkg/ccl/storageccl/BUILD.bazel b/pkg/ccl/storageccl/BUILD.bazel index a00f59f7ed86..8c8f3e141194 100644 --- a/pkg/ccl/storageccl/BUILD.bazel +++ b/pkg/ccl/storageccl/BUILD.bazel @@ -25,6 +25,7 @@ go_library( "//pkg/roachpb", "//pkg/settings", "//pkg/settings/cluster", + "//pkg/sql/catalog", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/tabledesc", "//pkg/sql/sem/builtins", diff --git a/pkg/ccl/storageccl/key_rewriter.go b/pkg/ccl/storageccl/key_rewriter.go index 44ee0540fe79..d9a1ed2891bc 100644 --- a/pkg/ccl/storageccl/key_rewriter.go +++ b/pkg/ccl/storageccl/key_rewriter.go @@ -14,6 +14,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" + "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/util/encoding" @@ -77,12 +78,12 @@ func (p prefixRewriter) rewriteKey(key []byte) ([]byte, bool) { // and splits. type KeyRewriter struct { prefixes prefixRewriter - descs map[descpb.ID]*tabledesc.Immutable + descs map[descpb.ID]catalog.TableDescriptor } // MakeKeyRewriterFromRekeys makes a KeyRewriter from Rekey protos. func MakeKeyRewriterFromRekeys(rekeys []roachpb.ImportRequest_TableRekey) (*KeyRewriter, error) { - descs := make(map[descpb.ID]*tabledesc.Immutable) + descs := make(map[descpb.ID]catalog.TableDescriptor) for _, rekey := range rekeys { var desc descpb.Descriptor if err := protoutil.Unmarshal(rekey.NewDesc, &desc); err != nil { @@ -98,7 +99,7 @@ func MakeKeyRewriterFromRekeys(rekeys []roachpb.ImportRequest_TableRekey) (*KeyR } // MakeKeyRewriter makes a KeyRewriter from a map of descs keyed by original ID. -func MakeKeyRewriter(descs map[descpb.ID]*tabledesc.Immutable) (*KeyRewriter, error) { +func MakeKeyRewriter(descs map[descpb.ID]catalog.TableDescriptor) (*KeyRewriter, error) { var prefixes prefixRewriter seenPrefixes := make(map[string]bool) for oldID, desc := range descs { diff --git a/pkg/ccl/workloadccl/format/BUILD.bazel b/pkg/ccl/workloadccl/format/BUILD.bazel index 5867272e1c9b..50619aedb5d0 100644 --- a/pkg/ccl/workloadccl/format/BUILD.bazel +++ b/pkg/ccl/workloadccl/format/BUILD.bazel @@ -8,8 +8,8 @@ go_library( deps = [ "//pkg/ccl/importccl", "//pkg/keys", + "//pkg/sql/catalog", "//pkg/sql/catalog/descpb", - "//pkg/sql/catalog/tabledesc", "//pkg/sql/parser", "//pkg/sql/row", "//pkg/sql/sem/tree", diff --git a/pkg/ccl/workloadccl/format/sstable.go b/pkg/ccl/workloadccl/format/sstable.go index c36a753b2442..3ae2ed695b3b 100644 --- a/pkg/ccl/workloadccl/format/sstable.go +++ b/pkg/ccl/workloadccl/format/sstable.go @@ -15,8 +15,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/importccl" "github.com/cockroachdb/cockroach/pkg/keys" + "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/parser" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -32,7 +32,7 @@ import ( // Table. func ToTableDescriptor( t workload.Table, tableID descpb.ID, ts time.Time, -) (*tabledesc.Immutable, error) { +) (catalog.TableDescriptor, error) { ctx := context.Background() semaCtx := tree.MakeSemaContext() stmt, err := parser.ParseOne(fmt.Sprintf(`CREATE TABLE "%s" %s`, t.Name, t.Schema)) @@ -49,7 +49,7 @@ func ToTableDescriptor( if err != nil { return nil, err } - return tableDesc.ImmutableCopy().(*tabledesc.Immutable), nil + return tableDesc.ImmutableCopy().(catalog.TableDescriptor), nil } // ToSSTable constructs a single sstable with the kvs necessary to represent a diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index 5ce421d52f09..81cedf6e2899 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -353,7 +353,7 @@ func (sc *SchemaChanger) runBackfill(ctx context.Context) error { // on the new version of the table descriptor. It returns the new table descs. func (sc *SchemaChanger) dropConstraints( ctx context.Context, constraints []descpb.ConstraintToUpdate, -) (map[descpb.ID]*tabledesc.Immutable, error) { +) (map[descpb.ID]catalog.TableDescriptor, error) { log.Infof(ctx, "dropping %d constraints", len(constraints)) fksByBackrefTable := make(map[descpb.ID][]*descpb.ConstraintToUpdate) @@ -468,7 +468,7 @@ func (sc *SchemaChanger) dropConstraints( } log.Info(ctx, "finished dropping constraints") - tableDescs := make(map[descpb.ID]*tabledesc.Immutable, len(fksByBackrefTable)+1) + tableDescs := make(map[descpb.ID]catalog.TableDescriptor, len(fksByBackrefTable)+1) if err := sc.txn(ctx, func( ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, ) (err error) { @@ -660,7 +660,7 @@ func (sc *SchemaChanger) validateConstraints( } readAsOf := sc.clock.Now() - var tableDesc *tabledesc.Immutable + var tableDesc catalog.TableDescriptor if err := sc.fixedTimestampTxn(ctx, readAsOf, func(ctx context.Context, txn *kv.Txn) error { tableDesc, err = catalogkv.MustGetTableDescByID(ctx, txn, sc.execCfg.Codec, sc.descID) @@ -745,7 +745,7 @@ func (sc *SchemaChanger) validateConstraints( // reuse an existing kv.Txn safely. func (sc *SchemaChanger) getTableVersion( ctx context.Context, txn *kv.Txn, tc *descs.Collection, version descpb.DescriptorVersion, -) (*tabledesc.Immutable, error) { +) (catalog.TableDescriptor, error) { tableDesc, err := tc.GetImmutableTableByID(ctx, txn, sc.descID, tree.ObjectLookupFlags{}) if err != nil { return nil, err @@ -765,7 +765,7 @@ func (sc *SchemaChanger) getTableVersion( func TruncateInterleavedIndexes( ctx context.Context, execCfg *ExecutorConfig, - table *tabledesc.Immutable, + table catalog.TableDescriptor, indexes []descpb.IndexDescriptor, ) error { log.Infof(ctx, "truncating %d interleaved indexes", len(indexes)) @@ -1406,7 +1406,7 @@ func (sc *SchemaChanger) validateIndexes(ctx context.Context) error { } readAsOf := sc.clock.Now() - var tableDesc *tabledesc.Immutable + var tableDesc catalog.TableDescriptor if err := sc.fixedTimestampTxn(ctx, readAsOf, func(ctx context.Context, txn *kv.Txn) (err error) { tableDesc, err = catalogkv.MustGetTableDescByID(ctx, txn, sc.execCfg.Codec, sc.descID) return err @@ -1465,7 +1465,7 @@ func (sc *SchemaChanger) validateIndexes(ctx context.Context) error { // at the historical fixed timestamp for checks. func (sc *SchemaChanger) validateInvertedIndexes( ctx context.Context, - tableDesc *tabledesc.Immutable, + tableDesc catalog.TableDescriptor, indexes []*descpb.IndexDescriptor, runHistoricalTxn historicalTxnRunner, ) error { @@ -1579,7 +1579,7 @@ func (sc *SchemaChanger) validateInvertedIndexes( // at the historical fixed timestamp for checks. func (sc *SchemaChanger) validateForwardIndexes( ctx context.Context, - tableDesc *tabledesc.Immutable, + tableDesc catalog.TableDescriptor, indexes []*descpb.IndexDescriptor, runHistoricalTxn historicalTxnRunner, ) error { @@ -2208,7 +2208,7 @@ func columnBackfillInTxn( txn *kv.Txn, evalCtx *tree.EvalContext, semaCtx *tree.SemaContext, - tableDesc *tabledesc.Immutable, + tableDesc catalog.TableDescriptor, traceKV bool, ) error { // A column backfill in the ADD state is a noop. @@ -2250,7 +2250,7 @@ func indexBackfillInTxn( txn *kv.Txn, evalCtx *tree.EvalContext, semaCtx *tree.SemaContext, - tableDesc *tabledesc.Immutable, + tableDesc catalog.TableDescriptor, traceKV bool, ) error { var indexBackfillerMon *mon.BytesMonitor @@ -2285,7 +2285,7 @@ func indexTruncateInTxn( txn *kv.Txn, execCfg *ExecutorConfig, evalCtx *tree.EvalContext, - tableDesc *tabledesc.Immutable, + tableDesc catalog.TableDescriptor, idx *descpb.IndexDescriptor, traceKV bool, ) error { diff --git a/pkg/sql/backfill/backfill.go b/pkg/sql/backfill/backfill.go index 0f73d21bc837..b466c4271538 100644 --- a/pkg/sql/backfill/backfill.go +++ b/pkg/sql/backfill/backfill.go @@ -76,7 +76,7 @@ type ColumnBackfiller struct { } // initCols is a helper to populate some column metadata on a ColumnBackfiller. -func (cb *ColumnBackfiller) initCols(desc *tabledesc.Immutable) { +func (cb *ColumnBackfiller) initCols(desc catalog.TableDescriptor) { for _, m := range desc.GetMutations() { if ColumnMutationFilter(m) { desc := *m.GetColumn() @@ -96,7 +96,7 @@ func (cb *ColumnBackfiller) init( evalCtx *tree.EvalContext, defaultExprs []tree.TypedExpr, computedExprs []tree.TypedExpr, - desc *tabledesc.Immutable, + desc catalog.TableDescriptor, mon *mon.BytesMonitor, ) error { cb.evalCtx = evalCtx @@ -155,7 +155,7 @@ func (cb *ColumnBackfiller) InitForLocalUse( ctx context.Context, evalCtx *tree.EvalContext, semaCtx *tree.SemaContext, - desc *tabledesc.Immutable, + desc catalog.TableDescriptor, mon *mon.BytesMonitor, ) error { cb.initCols(desc) @@ -186,7 +186,10 @@ func (cb *ColumnBackfiller) InitForLocalUse( // necessary due to the different procedure for accessing user defined type // metadata as part of a distributed flow. func (cb *ColumnBackfiller) InitForDistributedUse( - ctx context.Context, flowCtx *execinfra.FlowCtx, desc *tabledesc.Immutable, mon *mon.BytesMonitor, + ctx context.Context, + flowCtx *execinfra.FlowCtx, + desc catalog.TableDescriptor, + mon *mon.BytesMonitor, ) error { cb.initCols(desc) evalCtx := flowCtx.NewEvalCtx() @@ -246,7 +249,7 @@ func (cb *ColumnBackfiller) Close(ctx context.Context) { func (cb *ColumnBackfiller) RunColumnBackfillChunk( ctx context.Context, txn *kv.Txn, - tableDesc *tabledesc.Immutable, + tableDesc catalog.TableDescriptor, sp roachpb.Span, chunkSize int64, alsoCommit bool, @@ -364,7 +367,9 @@ func (cb *ColumnBackfiller) RunColumnBackfillChunk( } // ConvertBackfillError returns a cleaner SQL error for a failed Batch. -func ConvertBackfillError(ctx context.Context, tableDesc *tabledesc.Immutable, b *kv.Batch) error { +func ConvertBackfillError( + ctx context.Context, tableDesc catalog.TableDescriptor, b *kv.Batch, +) error { // A backfill on a new schema element has failed and the batch contains // information useful in printing a sensible error. However // ConvertBatchError() will only work correctly if the schema elements @@ -444,7 +449,7 @@ func (ib *IndexBackfiller) InitForLocalUse( ctx context.Context, evalCtx *tree.EvalContext, semaCtx *tree.SemaContext, - desc *tabledesc.Immutable, + desc catalog.TableDescriptor, mon *mon.BytesMonitor, ) error { // Initialize ib.cols and ib.colIdxMap. @@ -563,7 +568,10 @@ func constructExprs( // due to the different procedure for accessing user defined type metadata as // part of a distributed flow. func (ib *IndexBackfiller) InitForDistributedUse( - ctx context.Context, flowCtx *execinfra.FlowCtx, desc *tabledesc.Immutable, mon *mon.BytesMonitor, + ctx context.Context, + flowCtx *execinfra.FlowCtx, + desc catalog.TableDescriptor, + mon *mon.BytesMonitor, ) error { // Initialize ib.cols and ib.colIdxMap. ib.initCols(desc) @@ -641,7 +649,7 @@ func (ib *IndexBackfiller) ShrinkBoundAccount(ctx context.Context, shrinkBy int6 // initCols is a helper to populate column metadata of an IndexBackfiller. It // populates the cols and colIdxMap fields. -func (ib *IndexBackfiller) initCols(desc *tabledesc.Immutable) { +func (ib *IndexBackfiller) initCols(desc catalog.TableDescriptor) { for i := range desc.GetPublicColumns() { col := &desc.GetPublicColumns()[i] ib.cols = append(ib.cols, *col) @@ -679,7 +687,7 @@ func (ib *IndexBackfiller) initCols(desc *tabledesc.Immutable) { // initIndexes is a helper to populate index metadata of an IndexBackfiller. It // populates the added field. It returns a set of column ordinals that must be // fetched in order to backfill the added indexes. -func (ib *IndexBackfiller) initIndexes(desc *tabledesc.Immutable) util.FastIntSet { +func (ib *IndexBackfiller) initIndexes(desc catalog.TableDescriptor) util.FastIntSet { var valNeededForCol util.FastIntSet mutationID := desc.GetMutations()[0].MutationID @@ -714,7 +722,7 @@ func (ib *IndexBackfiller) init( predicateExprs map[descpb.IndexID]tree.TypedExpr, colExprs map[descpb.ColumnID]tree.TypedExpr, valNeededForCol util.FastIntSet, - desc *tabledesc.Immutable, + desc catalog.TableDescriptor, mon *mon.BytesMonitor, ) error { ib.evalCtx = evalCtx @@ -773,7 +781,7 @@ func (ib *IndexBackfiller) init( func (ib *IndexBackfiller) BuildIndexEntriesChunk( ctx context.Context, txn *kv.Txn, - tableDesc *tabledesc.Immutable, + tableDesc catalog.TableDescriptor, sp roachpb.Span, chunkSize int64, traceKV bool, @@ -956,7 +964,7 @@ func (ib *IndexBackfiller) BuildIndexEntriesChunk( func (ib *IndexBackfiller) RunIndexBackfillChunk( ctx context.Context, txn *kv.Txn, - tableDesc *tabledesc.Immutable, + tableDesc catalog.TableDescriptor, sp roachpb.Span, chunkSize int64, alsoCommit bool, diff --git a/pkg/sql/catalog/catalogkv/catalogkv.go b/pkg/sql/catalog/catalogkv/catalogkv.go index 301d607c6ff7..e1c6c455bdb2 100644 --- a/pkg/sql/catalog/catalogkv/catalogkv.go +++ b/pkg/sql/catalog/catalogkv/catalogkv.go @@ -553,13 +553,13 @@ func GetDatabaseDescByID( // returning an error if the table is not found. func MustGetTableDescByID( ctx context.Context, txn *kv.Txn, codec keys.SQLCodec, id descpb.ID, -) (*tabledesc.Immutable, error) { +) (catalog.TableDescriptor, error) { desc, err := GetDescriptorByID(ctx, txn, codec, id, Immutable, TableDescriptorKind, true /* required */) if err != nil || desc == nil { return nil, err } - return desc.(*tabledesc.Immutable), nil + return desc.(catalog.TableDescriptor), nil } // MustGetDatabaseDescByID looks up the database descriptor given its ID, diff --git a/pkg/sql/catalog/catalogkv/test_utils.go b/pkg/sql/catalog/catalogkv/test_utils.go index 2793bf462c20..45b935de7a11 100644 --- a/pkg/sql/catalog/catalogkv/test_utils.go +++ b/pkg/sql/catalog/catalogkv/test_utils.go @@ -30,8 +30,8 @@ import ( // trivial change that just touches lots of lines. func TestingGetTableDescriptorFromSchema( kvDB *kv.DB, codec keys.SQLCodec, database string, schema string, table string, -) *tabledesc.Immutable { - return testingGetObjectDescriptor(kvDB, codec, database, schema, table).(*tabledesc.Immutable) +) catalog.TableDescriptor { + return testingGetObjectDescriptor(kvDB, codec, database, schema, table).(catalog.TableDescriptor) } // TestingGetTableDescriptor retrieves a table descriptor directly from the KV @@ -42,7 +42,7 @@ func TestingGetTableDescriptorFromSchema( // removing it altogether. func TestingGetTableDescriptor( kvDB *kv.DB, codec keys.SQLCodec, database string, table string, -) *tabledesc.Immutable { +) catalog.TableDescriptor { return TestingGetImmutableTableDescriptor(kvDB, codec, database, table) } @@ -50,8 +50,8 @@ func TestingGetTableDescriptor( // directly from the KV layer. func TestingGetImmutableTableDescriptor( kvDB *kv.DB, codec keys.SQLCodec, database string, table string, -) *tabledesc.Immutable { - return testingGetObjectDescriptor(kvDB, codec, database, "public", table).(*tabledesc.Immutable) +) catalog.TableDescriptor { + return testingGetObjectDescriptor(kvDB, codec, database, "public", table).(catalog.TableDescriptor) } // TestingGetMutableExistingTableDescriptor retrieves a Mutable diff --git a/pkg/sql/catalog/descs/collection.go b/pkg/sql/catalog/descs/collection.go index 47cbcc4599ac..c01ab3630dbb 100644 --- a/pkg/sql/catalog/descs/collection.go +++ b/pkg/sql/catalog/descs/collection.go @@ -538,12 +538,12 @@ func (tc *Collection) GetMutableTableByName( // according to the provided lookup flags. RequireMutable is ignored. func (tc *Collection) GetImmutableTableByName( ctx context.Context, txn *kv.Txn, name tree.ObjectName, flags tree.ObjectLookupFlags, -) (found bool, _ *tabledesc.Immutable, _ error) { +) (found bool, _ catalog.TableDescriptor, _ error) { found, desc, err := tc.getTableByName(ctx, txn, name, flags, false /* mutable */) if err != nil || !found { return false, nil, err } - return true, desc.(*tabledesc.Immutable), nil + return true, desc, nil } // getTableByName returns a table descriptor with properties according to the @@ -921,12 +921,12 @@ func (tc *Collection) GetMutableTableByID( // the ID exists. func (tc *Collection) GetImmutableTableByID( ctx context.Context, txn *kv.Txn, tableID descpb.ID, flags tree.ObjectLookupFlags, -) (*tabledesc.Immutable, error) { +) (catalog.TableDescriptor, error) { desc, err := tc.getTableByID(ctx, txn, tableID, flags, false /* mutable */) if err != nil { return nil, err } - return desc.(*tabledesc.Immutable), nil + return desc, nil } func (tc *Collection) getTableByID( @@ -1190,7 +1190,7 @@ func (tc *Collection) hydrateTypesInTableDesc( } return desc, typedesc.HydrateTypesInTableDescriptor(ctx, t.TableDesc(), typedesc.TypeLookupFunc(getType)) - case *tabledesc.Immutable: + case catalog.TableDescriptor: // ImmutableTableDescriptors need to be copied before hydration, because // they are potentially read by multiple threads. If there aren't any user // defined types in the descriptor, then return early. @@ -1389,9 +1389,9 @@ func (tc *Collection) GetDescriptorsWithNewVersion() []lease.IDVersion { // GetUncommittedTables returns all the tables updated or created in the // transaction. -func (tc *Collection) GetUncommittedTables() (tables []*tabledesc.Immutable) { +func (tc *Collection) GetUncommittedTables() (tables []catalog.TableDescriptor) { for _, desc := range tc.uncommittedDescriptors { - table, ok := desc.immutable.(*tabledesc.Immutable) + table, ok := desc.immutable.(catalog.TableDescriptor) if ok && desc.immutable.IsUncommittedVersion() { tables = append(tables, table) } @@ -1654,7 +1654,8 @@ func HydrateGivenDescriptors(ctx context.Context, descs []catalog.Descriptor) er if desc.Dropped() { continue } - if tblDesc, ok := desc.(*tabledesc.Immutable); ok { + tblDesc, ok := desc.(catalog.TableDescriptor) + if ok { if err := typedesc.HydrateTypesInTableDescriptor( ctx, tblDesc.TableDesc(), diff --git a/pkg/sql/catalog/hydratedtables/hydratedcache.go b/pkg/sql/catalog/hydratedtables/hydratedcache.go index 40c93ad78fa1..bdea66bf3556 100644 --- a/pkg/sql/catalog/hydratedtables/hydratedcache.go +++ b/pkg/sql/catalog/hydratedtables/hydratedcache.go @@ -123,7 +123,7 @@ func NewCache(settings *cluster.Settings) *Cache { } type hydratedTableDescriptor struct { - tableDesc *tabledesc.Immutable + tableDesc catalog.TableDescriptor typeDescs []*cachedType } @@ -141,8 +141,8 @@ type cachedType struct { // descriptor on their own. If the table descriptor does not contain any // user-defined types, it will be returned unchanged. func (c *Cache) GetHydratedTableDescriptor( - ctx context.Context, table *tabledesc.Immutable, res catalog.TypeDescriptorResolver, -) (hydrated *tabledesc.Immutable, err error) { + ctx context.Context, table catalog.TableDescriptor, res catalog.TypeDescriptorResolver, +) (hydrated catalog.TableDescriptor, err error) { // If the table has an uncommitted version, it cannot be cached. Return nil // forcing the caller to hydrate. @@ -251,7 +251,7 @@ func (c *Cache) GetHydratedTableDescriptor( if err != nil { return nil, err } - return res.(*tabledesc.Immutable), nil + return res.(catalog.TableDescriptor), nil } } diff --git a/pkg/sql/catalog/hydratedtables/hydratedcache_test.go b/pkg/sql/catalog/hydratedtables/hydratedcache_test.go index cdc3b5271776..77cec3a188a0 100644 --- a/pkg/sql/catalog/hydratedtables/hydratedcache_test.go +++ b/pkg/sql/catalog/hydratedtables/hydratedcache_test.go @@ -38,7 +38,7 @@ func TestHydratedCache(t *testing.T) { m := c.Metrics() dg := mkDescGetter(descs...) res := &descGetterTypeDescriptorResolver{dg: &dg} - td := tableDescUDT.ImmutableCopy().(*tabledesc.Immutable) + td := tableDescUDT.ImmutableCopy().(catalog.TableDescriptor) hydrated, err := c.GetHydratedTableDescriptor(ctx, td, res) require.NoError(t, err) assertMetrics(t, m, 0, 1) @@ -68,7 +68,7 @@ func TestHydratedCache(t *testing.T) { m := c.Metrics() dg := mkDescGetter(descs...) res := &descGetterTypeDescriptorResolver{dg: &dg} - td := tableDescNoUDT.ImmutableCopy().(*tabledesc.Immutable) + td := tableDescNoUDT.ImmutableCopy().(catalog.TableDescriptor) _, err := c.GetHydratedTableDescriptor(ctx, td, res) require.NoError(t, err) assertMetrics(t, m, 0, 0) @@ -78,7 +78,7 @@ func TestHydratedCache(t *testing.T) { m := c.Metrics() dg := mkDescGetter(descs...) res := &descGetterTypeDescriptorResolver{dg: &dg} - td := tableDescUDT.ImmutableCopy().(*tabledesc.Immutable) + td := tableDescUDT.ImmutableCopy().(catalog.TableDescriptor) hydrated, err := c.GetHydratedTableDescriptor(ctx, td, res) require.NoError(t, err) assertMetrics(t, m, 0, 1) @@ -102,7 +102,7 @@ func TestHydratedCache(t *testing.T) { m := c.Metrics() dg := mkDescGetter(descs...) res := &descGetterTypeDescriptorResolver{dg: &dg} - td := tableDescUDT.ImmutableCopy().(*tabledesc.Immutable) + td := tableDescUDT.ImmutableCopy().(catalog.TableDescriptor) hydrated, err := c.GetHydratedTableDescriptor(ctx, td, res) require.NoError(t, err) assertMetrics(t, m, 0, 1) @@ -124,7 +124,7 @@ func TestHydratedCache(t *testing.T) { dg := mkDescGetter(descs...) res := &descGetterTypeDescriptorResolver{dg: &dg} res.unqualifiedName = true - td := tableDescUDT.ImmutableCopy().(*tabledesc.Immutable) + td := tableDescUDT.ImmutableCopy().(catalog.TableDescriptor) hydrated, err := c.GetHydratedTableDescriptor(ctx, td, res) require.NoError(t, err) assertMetrics(t, m, 0, 1) @@ -146,7 +146,7 @@ func TestHydratedCache(t *testing.T) { dg := mkDescGetter(descs...) res := &descGetterTypeDescriptorResolver{dg: &dg} res.unqualifiedName = true - td := tableDescUDT.ImmutableCopy().(*tabledesc.Immutable) + td := tableDescUDT.ImmutableCopy().(catalog.TableDescriptor) hydrated, err := c.GetHydratedTableDescriptor(ctx, td, res) require.NoError(t, err) assertMetrics(t, m, 0, 1) @@ -176,7 +176,7 @@ func TestHydratedCache(t *testing.T) { calledCh <- errCh return <-errCh } - td := tableDescUDT.ImmutableCopy().(*tabledesc.Immutable) + td := tableDescUDT.ImmutableCopy().(catalog.TableDescriptor) callOneErrCh := make(chan error, 1) go func() { @@ -202,7 +202,7 @@ func TestHydratedCache(t *testing.T) { res := &descGetterTypeDescriptorResolver{dg: &dg} mut := tabledesc.NewExistingMutable(*dg[tableUDTID].(catalog.TableDescriptor).TableDesc()) mut.MaybeIncrementVersion() - td := mut.ImmutableCopy().(*tabledesc.Immutable) + td := mut.ImmutableCopy().(catalog.TableDescriptor) hydrated, err := c.GetHydratedTableDescriptor(ctx, td, res) require.NoError(t, err) require.Nil(t, hydrated) @@ -226,7 +226,7 @@ func TestHydratedCache(t *testing.T) { // This behavior is a bit bizarre but exists to not waste the work of // hydrating the descriptor if we've already started to do it. // This case should not meaningfully arise in practice. - td := tableDescUDT.ImmutableCopy().(*tabledesc.Immutable) + td := tableDescUDT.ImmutableCopy().(catalog.TableDescriptor) { hydrated, err := c.GetHydratedTableDescriptor(ctx, td, resWithMut) require.NoError(t, err) diff --git a/pkg/sql/catalog/lease/lease_internal_test.go b/pkg/sql/catalog/lease/lease_internal_test.go index c4e4e8951c21..4c6296d2b3cc 100644 --- a/pkg/sql/catalog/lease/lease_internal_test.go +++ b/pkg/sql/catalog/lease/lease_internal_test.go @@ -160,7 +160,7 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR); tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") - var tables []tabledesc.Immutable + var tables []catalog.TableDescriptor var expiration hlc.Timestamp getLeases := func() { for i := 0; i < 3; i++ { @@ -171,7 +171,7 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR); if err != nil { t.Fatal(err) } - tables = append(tables, *table.(*tabledesc.Immutable)) + tables = append(tables, table.(catalog.TableDescriptor)) expiration = exp if err := leaseManager.Release(table); err != nil { t.Fatal(err) @@ -220,7 +220,7 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR); // without a lease. ts.mu.Lock() tableVersion := &descriptorVersionState{ - Descriptor: &tables[0], + Descriptor: tables[0], expiration: tables[5].GetModificationTime(), } ts.mu.active.insert(tableVersion) @@ -326,7 +326,7 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR); if lease.GetID() != tableDesc.GetID() { t.Fatalf("new name has wrong ID: %d (expected: %d)", lease.GetID(), tableDesc.GetID()) } - if err := leaseManager.Release(lease.Descriptor.(*tabledesc.Immutable)); err != nil { + if err := leaseManager.Release(lease.Descriptor); err != nil { t.Fatal(err) } @@ -353,7 +353,7 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR); if lease.GetID() != tableDesc.GetID() { t.Fatalf("new name has wrong ID: %d (expected: %d)", lease.GetID(), tableDesc.GetID()) } - if err := leaseManager.Release(lease.Descriptor.(*tabledesc.Immutable)); err != nil { + if err := leaseManager.Release(lease.Descriptor); err != nil { t.Fatal(err) } } @@ -386,7 +386,7 @@ CREATE TABLE t.%s (k CHAR PRIMARY KEY, v CHAR); if lease := leaseManager.names.get(tableDesc.GetParentID(), tableDesc.GetParentSchemaID(), tableName, s.Clock().Now()); lease == nil { t.Fatalf("name cache has no unexpired entry for (%d, %s)", tableDesc.GetParentID(), tableName) } else { - if err := leaseManager.Release(lease.Descriptor.(*tabledesc.Immutable)); err != nil { + if err := leaseManager.Release(lease.Descriptor); err != nil { t.Fatal(err) } } @@ -437,7 +437,7 @@ CREATE TABLE t.%s (k CHAR PRIMARY KEY, v CHAR); t.Fatalf("name cache has no unexpired entry for (%d, %s)", tableDesc.GetParentID(), tableName) } - tracker := removalTracker.TrackRemoval(lease.Descriptor.(*tabledesc.Immutable)) + tracker := removalTracker.TrackRemoval(lease.Descriptor) // Acquire another lease. if _, err := acquireNodeLease(context.Background(), leaseManager, tableDesc.GetID()); err != nil { @@ -453,7 +453,7 @@ CREATE TABLE t.%s (k CHAR PRIMARY KEY, v CHAR); t.Fatalf("same lease %s", newLease.expiration.GoTime()) } - if err := leaseManager.Release(lease.Descriptor.(*tabledesc.Immutable)); err != nil { + if err := leaseManager.Release(lease.Descriptor); err != nil { t.Fatal(err) } @@ -462,7 +462,7 @@ CREATE TABLE t.%s (k CHAR PRIMARY KEY, v CHAR); t.Fatal(err) } - if err := leaseManager.Release(lease.Descriptor.(*tabledesc.Immutable)); err != nil { + if err := leaseManager.Release(lease.Descriptor); err != nil { t.Fatal(err) } } @@ -544,7 +544,7 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR); // Release. // tableChan acts as a barrier, synchronizing the two routines at every // iteration. - tableChan := make(chan *tabledesc.Immutable) + tableChan := make(chan catalog.TableDescriptor) errChan := make(chan error) go func() { for table := range tableChan { @@ -566,7 +566,7 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR); if err != nil { t.Fatal(err) } - table := desc.(*tabledesc.Immutable) + table := desc.(catalog.TableDescriptor) // This test will need to wait until leases are removed from the store // before creating new leases because the jitter used in the leases' // expiration causes duplicate key errors when trying to create new @@ -735,7 +735,7 @@ func TestLeaseAcquireAndReleaseConcurrently(t *testing.T) { // Result is a struct for moving results to the main result routine. type Result struct { - table *tabledesc.Immutable + table catalog.TableDescriptor exp hlc.Timestamp err error } @@ -749,7 +749,7 @@ func TestLeaseAcquireAndReleaseConcurrently(t *testing.T) { acquireChan chan Result, ) { table, e, err := m.Acquire(ctx, m.storage.clock.Now(), descID) - acquireChan <- Result{err: err, exp: e, table: table.(*tabledesc.Immutable)} + acquireChan <- Result{err: err, exp: e, table: table.(catalog.TableDescriptor)} } testCases := []struct { @@ -852,7 +852,7 @@ func TestLeaseAcquireAndReleaseConcurrently(t *testing.T) { return } table, e, err := m.Acquire(ctx, s.Clock().Now(), descID) - acquireChan <- Result{err: err, exp: e, table: table.(*tabledesc.Immutable)} + acquireChan <- Result{err: err, exp: e, table: table.(catalog.TableDescriptor)} }(ctx, leaseManager, acquireResultChan) } else { diff --git a/pkg/sql/catalog/lease/lease_test.go b/pkg/sql/catalog/lease/lease_test.go index 08b5235104ce..ad6b566db779 100644 --- a/pkg/sql/catalog/lease/lease_test.go +++ b/pkg/sql/catalog/lease/lease_test.go @@ -1671,7 +1671,7 @@ CREATE TABLE t.test0 (k CHAR PRIMARY KEY, v CHAR); if err != nil { t.Fatalf("error while publishing: %v", err) } - table := desc.(*tabledesc.Immutable) + table := desc.(catalog.TableDescriptor) // Wait a little time to give a chance to other goroutines to // race past. diff --git a/pkg/sql/catalog/resolver/resolver.go b/pkg/sql/catalog/resolver/resolver.go index 38e5a5316438..a5243a9ac826 100644 --- a/pkg/sql/catalog/resolver/resolver.go +++ b/pkg/sql/catalog/resolver/resolver.go @@ -47,7 +47,7 @@ type SchemaResolver interface { CurrentSearchPath() sessiondata.SearchPath CommonLookupFlags(required bool) tree.CommonLookupFlags ObjectLookupFlags(required bool, requireMutable bool) tree.ObjectLookupFlags - LookupTableByID(ctx context.Context, id descpb.ID) (*tabledesc.Immutable, error) + LookupTableByID(ctx context.Context, id descpb.ID) (catalog.TableDescriptor, error) tree.TypeReferenceResolver } @@ -85,7 +85,7 @@ func GetObjectNames( // if no object is found. func ResolveExistingTableObject( ctx context.Context, sc SchemaResolver, tn *tree.TableName, lookupFlags tree.ObjectLookupFlags, -) (res *tabledesc.Immutable, err error) { +) (res catalog.TableDescriptor, err error) { // TODO: As part of work for #34240, an UnresolvedObjectName should be // passed as an argument to this function. un := tn.ToUnresolvedObjectName() @@ -94,7 +94,7 @@ func ResolveExistingTableObject( return nil, err } tn.ObjectNamePrefix = prefix - return desc.(*tabledesc.Immutable), nil + return desc.(catalog.TableDescriptor), nil } // ResolveMutableExistingTableObject looks up an existing mutable object. @@ -208,7 +208,7 @@ func ResolveExistingObject( return descI.(*tabledesc.Mutable), prefix, nil } - return descI.(*tabledesc.Immutable), prefix, nil + return descI.(catalog.TableDescriptor), prefix, nil default: return nil, prefix, errors.AssertionFailedf( "unknown desired object kind %d", lookupFlags.DesiredObjectKind) diff --git a/pkg/sql/catalog/tabledesc/structured.go b/pkg/sql/catalog/tabledesc/structured.go index 35ff0b1334ad..6a94856d745d 100644 --- a/pkg/sql/catalog/tabledesc/structured.go +++ b/pkg/sql/catalog/tabledesc/structured.go @@ -107,8 +107,8 @@ func NewFilledInExistingMutable( return &Mutable{wrapper: w, ClusterVersion: *tbl}, nil } -// MakeImmutable returns an Immutable from the given TableDescriptor. -func MakeImmutable(tbl descpb.TableDescriptor) Immutable { +// makeImmutable returns an Immutable from the given TableDescriptor. +func makeImmutable(tbl descpb.TableDescriptor) Immutable { publicAndNonPublicCols := tbl.Columns readableCols := tbl.Columns @@ -174,7 +174,7 @@ func MakeImmutable(tbl descpb.TableDescriptor) Immutable { // NewImmutable returns a Immutable from the given TableDescriptor. // This function assumes that this descriptor has not been modified from the // version stored in the key-value store. -func NewImmutable(tbl descpb.TableDescriptor) *Immutable { +func NewImmutable(tbl descpb.TableDescriptor) catalog.TableDescriptor { return NewImmutableWithIsUncommittedVersion(tbl, false /* isUncommittedVersion */) } @@ -184,8 +184,8 @@ func NewImmutable(tbl descpb.TableDescriptor) *Immutable { // an Immutable from an existing descriptor which may have a new version. func NewImmutableWithIsUncommittedVersion( tbl descpb.TableDescriptor, isUncommittedVersion bool, -) *Immutable { - desc := MakeImmutable(tbl) +) catalog.TableDescriptor { + desc := makeImmutable(tbl) desc.isUncommittedVersion = isUncommittedVersion return &desc } @@ -194,12 +194,12 @@ func NewImmutableWithIsUncommittedVersion( // post-deserialization upgrades. func NewFilledInImmutable( ctx context.Context, dg catalog.DescGetter, tbl *descpb.TableDescriptor, -) (*Immutable, error) { +) (catalog.TableDescriptor, error) { changes, err := maybeFillInDescriptor(ctx, dg, tbl, false /* skipFKsWithNoMatchingTable */) if err != nil { return nil, err } - desc := MakeImmutable(*tbl) + desc := makeImmutable(*tbl) desc.postDeserializationChanges = changes return &desc, nil } diff --git a/pkg/sql/catalog/tabledesc/structured_test.go b/pkg/sql/catalog/tabledesc/structured_test.go index a85538255ce4..3b0de3a118b2 100644 --- a/pkg/sql/catalog/tabledesc/structured_test.go +++ b/pkg/sql/catalog/tabledesc/structured_test.go @@ -1687,7 +1687,7 @@ func TestMaybeUpgradeFormatVersion(t *testing.T) { tests := []struct { desc descpb.TableDescriptor expUpgrade bool - verify func(int, *Immutable) // nil means no extra verification. + verify func(int, catalog.TableDescriptor) // nil means no extra verification. }{ { desc: descpb.TableDescriptor{ @@ -1698,7 +1698,7 @@ func TestMaybeUpgradeFormatVersion(t *testing.T) { Privileges: descpb.NewDefaultPrivilegeDescriptor(security.RootUserName()), }, expUpgrade: true, - verify: func(i int, desc *Immutable) { + verify: func(i int, desc catalog.TableDescriptor) { if len(desc.GetFamilies()) == 0 { t.Errorf("%d: expected families to be set, but it was empty", i) } diff --git a/pkg/sql/catalog/tabledesc/table_desc.go b/pkg/sql/catalog/tabledesc/table_desc.go index 6f98d5ce4a1d..08d2b39316c8 100644 --- a/pkg/sql/catalog/tabledesc/table_desc.go +++ b/pkg/sql/catalog/tabledesc/table_desc.go @@ -223,7 +223,7 @@ func (desc *Mutable) ImmutableCopy() catalog.Descriptor { // TODO (lucy): Should the immutable descriptor constructors always make a // copy, so we don't have to do it here? imm := NewImmutable(*protoutil.Clone(desc.TableDesc()).(*descpb.TableDescriptor)) - imm.isUncommittedVersion = desc.IsUncommittedVersion() + imm.(*Immutable).isUncommittedVersion = desc.IsUncommittedVersion() return imm } diff --git a/pkg/sql/colfetcher/colbatch_scan.go b/pkg/sql/colfetcher/colbatch_scan.go index 5733e08df0ac..72597adeaf73 100644 --- a/pkg/sql/colfetcher/colbatch_scan.go +++ b/pkg/sql/colfetcher/colbatch_scan.go @@ -239,7 +239,7 @@ func initCRowFetcher( codec keys.SQLCodec, allocator *colmem.Allocator, fetcher *cFetcher, - desc *tabledesc.Immutable, + desc catalog.TableDescriptor, colIdxMap catalog.TableColMap, valNeededForCol util.FastIntSet, spec *execinfrapb.TableReaderSpec, diff --git a/pkg/sql/comment_on_column.go b/pkg/sql/comment_on_column.go index 244925e5cb84..91f0d25ebd31 100644 --- a/pkg/sql/comment_on_column.go +++ b/pkg/sql/comment_on_column.go @@ -15,7 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/security" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" @@ -24,7 +24,7 @@ import ( type commentOnColumnNode struct { n *tree.CommentOnColumn - tableDesc *tabledesc.Immutable + tableDesc catalog.TableDescriptor } // CommentOnColumn add comment on a column. diff --git a/pkg/sql/comment_on_table.go b/pkg/sql/comment_on_table.go index 6f1b5c609e53..421272b8dd27 100644 --- a/pkg/sql/comment_on_table.go +++ b/pkg/sql/comment_on_table.go @@ -15,7 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/security" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" @@ -24,7 +24,7 @@ import ( type commentOnTableNode struct { n *tree.CommentOnTable - tableDesc *tabledesc.Immutable + tableDesc catalog.TableDescriptor } // CommentOnTable add comment on a table. diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index fa46411347ae..113efce23d0a 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -365,7 +365,7 @@ CREATE TABLE crdb_internal.tables ( // Note: we do not use forEachTableDesc() here because we want to // include added and dropped descriptors. for _, desc := range descs { - table, ok := desc.(*tabledesc.Immutable) + table, ok := desc.(catalog.TableDescriptor) if !ok || p.CheckAnyPrivilege(ctx, table) != nil { continue } @@ -484,7 +484,7 @@ CREATE TABLE crdb_internal.schema_changes ( // Note: we do not use forEachTableDesc() here because we want to // include added and dropped descriptors. for _, desc := range descs { - table, ok := desc.(*tabledesc.Immutable) + table, ok := desc.(catalog.TableDescriptor) if !ok || p.CheckAnyPrivilege(ctx, table) != nil { continue } @@ -2509,7 +2509,7 @@ CREATE TABLE crdb_internal.ranges_no_leases ( for _, desc := range descs { id := uint32(desc.GetID()) switch desc := desc.(type) { - case *tabledesc.Immutable: + case catalog.TableDescriptor: parents[id] = uint32(desc.GetParentID()) tableNames[id] = desc.GetName() indexNames[id] = make(map[uint32]string) @@ -2776,7 +2776,7 @@ CREATE TABLE crdb_internal.zones ( return err } - var table *tabledesc.Immutable + var table catalog.TableDescriptor if zs.Database != "" { database, err := catalogkv.MustGetDatabaseDescByID(ctx, p.txn, p.ExecCfg().Codec, descpb.ID(id)) if err != nil { diff --git a/pkg/sql/create_stats.go b/pkg/sql/create_stats.go index e971f422167f..1a9e18adb4b1 100644 --- a/pkg/sql/create_stats.go +++ b/pkg/sql/create_stats.go @@ -21,10 +21,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" @@ -158,7 +158,7 @@ func (n *createStatsNode) startJob(ctx context.Context, resultsCh chan<- tree.Da // makeJobRecord creates a CreateStats job record which can be used to plan and // execute statistics creation. func (n *createStatsNode) makeJobRecord(ctx context.Context) (*jobs.Record, error) { - var tableDesc *tabledesc.Immutable + var tableDesc catalog.TableDescriptor var fqTableName string var err error switch t := n.Table.(type) { @@ -302,7 +302,7 @@ const maxNonIndexCols = 100 // other columns from the table. We only collect histograms for index columns, // plus any other boolean or enum columns (where the "histogram" is tiny). func createStatsDefaultColumns( - desc *tabledesc.Immutable, multiColEnabled bool, + desc catalog.TableDescriptor, multiColEnabled bool, ) ([]jobspb.CreateStatsDetails_ColStat, error) { colStats := make([]jobspb.CreateStatsDetails_ColStat, 0, len(desc.ActiveIndexes())) diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index 0cb0159c0c91..3a4aade564a2 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -420,7 +420,7 @@ func (n *createTableNode) startExec(params runParams) error { params.ctx, params.p.txn, params.ExecCfg().Codec, - desc.ImmutableCopy().(*tabledesc.Immutable), + desc.ImmutableCopy().(catalog.TableDescriptor), desc.Columns, params.p.alloc) if err != nil { diff --git a/pkg/sql/delete_range.go b/pkg/sql/delete_range.go index 9266edace663..1ff0a3816620 100644 --- a/pkg/sql/delete_range.go +++ b/pkg/sql/delete_range.go @@ -16,8 +16,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" + "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/row" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -40,10 +40,10 @@ type deleteRangeNode struct { // spans are the spans to delete. spans roachpb.Spans // desc is the table descriptor the delete is operating on. - desc *tabledesc.Immutable + desc catalog.TableDescriptor // interleavedDesc are the table descriptors of any child interleaved tables // the delete is operating on. - interleavedDesc []*tabledesc.Immutable + interleavedDesc []catalog.TableDescriptor // fetcher is around to decode the returned keys from the DeleteRange, so that // we can count the number of rows deleted. fetcher row.Fetcher diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index 1522d3c7aa39..11c87bdae09e 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -25,9 +25,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/distsql" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -967,7 +967,7 @@ func (dsp *DistSQLPlanner) nodeVersionIsCompatible(nodeID roachpb.NodeID) bool { return distsql.FlowVerIsCompatible(dsp.planVersion, v.MinAcceptedVersion, v.Version) } -func getIndexIdx(index *descpb.IndexDescriptor, desc *tabledesc.Immutable) (uint32, error) { +func getIndexIdx(index *descpb.IndexDescriptor, desc catalog.TableDescriptor) (uint32, error) { foundIndex, _ := desc.FindIndexWithID(index.ID) if foundIndex != nil && foundIndex.Public() { return uint32(foundIndex.Ordinal()), nil @@ -1039,7 +1039,7 @@ func getVirtualColumn( // tableOrdinal returns the index of a column with the given ID. func tableOrdinal( - desc *tabledesc.Immutable, colID descpb.ColumnID, visibility execinfrapb.ScanVisibility, + desc catalog.TableDescriptor, colID descpb.ColumnID, visibility execinfrapb.ScanVisibility, ) int { for i := range desc.GetPublicColumns() { if desc.GetPublicColumns()[i].ID == colID { @@ -1067,7 +1067,7 @@ func tableOrdinal( panic(errors.AssertionFailedf("column %d not in desc.Columns", colID)) } -func highestTableOrdinal(desc *tabledesc.Immutable, visibility execinfrapb.ScanVisibility) int { +func highestTableOrdinal(desc catalog.TableDescriptor, visibility execinfrapb.ScanVisibility) int { highest := len(desc.GetPublicColumns()) - 1 if visibility == execinfra.ScanVisibilityPublicAndNotPublic { highest = len(desc.GetPublicColumns()) + len(desc.MutationColumns()) - 1 @@ -1078,7 +1078,9 @@ func highestTableOrdinal(desc *tabledesc.Immutable, visibility execinfrapb.ScanV // toTableOrdinals returns a mapping from column ordinals in cols to table // reader column ordinals. func toTableOrdinals( - cols []*descpb.ColumnDescriptor, desc *tabledesc.Immutable, visibility execinfrapb.ScanVisibility, + cols []*descpb.ColumnDescriptor, + desc catalog.TableDescriptor, + visibility execinfrapb.ScanVisibility, ) []int { res := make([]int, len(cols)) for i := range res { @@ -1234,7 +1236,7 @@ func (dsp *DistSQLPlanner) createTableReaders( type tableReaderPlanningInfo struct { spec *execinfrapb.TableReaderSpec post execinfrapb.PostProcessSpec - desc *tabledesc.Immutable + desc catalog.TableDescriptor spans []roachpb.Span reverse bool scanVisibility execinfrapb.ScanVisibility diff --git a/pkg/sql/distsql_physical_planner_test.go b/pkg/sql/distsql_physical_planner_test.go index a8becdeed824..d41fdb78e255 100644 --- a/pkg/sql/distsql_physical_planner_test.go +++ b/pkg/sql/distsql_physical_planner_test.go @@ -32,8 +32,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/distsql" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -65,7 +65,7 @@ import ( // TODO(radu): we should verify that the queries in tests using SplitTable // are indeed distributed as intended. func SplitTable( - t *testing.T, tc serverutils.TestClusterInterface, desc *tabledesc.Immutable, sps []SplitPoint, + t *testing.T, tc serverutils.TestClusterInterface, desc catalog.TableDescriptor, sps []SplitPoint, ) { if tc.ReplicationMode() != base.ReplicationManual { t.Fatal("SplitTable called on a test cluster that was not in manual replication mode") diff --git a/pkg/sql/distsql_plan_csv.go b/pkg/sql/distsql_plan_csv.go index 25d29ab36174..702e433b2e07 100644 --- a/pkg/sql/distsql_plan_csv.go +++ b/pkg/sql/distsql_plan_csv.go @@ -158,7 +158,7 @@ func presplitTableBoundaries( expirationTime := cfg.DB.Clock().Now().Add(time.Hour.Nanoseconds(), 0) for _, tbl := range tables { // TODO(ajwerner): Consider passing in the wrapped descriptors. - tblDesc := tabledesc.MakeImmutable(*tbl.Desc) + tblDesc := tabledesc.NewImmutable(*tbl.Desc) for _, span := range tblDesc.AllIndexSpans(cfg.Codec) { if err := cfg.DB.AdminSplit(ctx, span.Key, expirationTime); err != nil { return err diff --git a/pkg/sql/distsql_plan_stats.go b/pkg/sql/distsql_plan_stats.go index 6be5535b46c0..5320f86009d1 100644 --- a/pkg/sql/distsql_plan_stats.go +++ b/pkg/sql/distsql_plan_stats.go @@ -53,7 +53,7 @@ var maxTimestampAge = settings.RegisterDurationSetting( ) func (dsp *DistSQLPlanner) createStatsPlan( - planCtx *PlanningCtx, desc *tabledesc.Immutable, reqStats []requestedStat, job *jobs.Job, + planCtx *PlanningCtx, desc catalog.TableDescriptor, reqStats []requestedStat, job *jobs.Job, ) (*PhysicalPlan, error) { if len(reqStats) == 0 { return nil, errors.New("no stats requested") diff --git a/pkg/sql/drop_test.go b/pkg/sql/drop_test.go index ef39c900d9fb..35f5d62f4a33 100644 --- a/pkg/sql/drop_test.go +++ b/pkg/sql/drop_test.go @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" @@ -709,7 +710,7 @@ func TestDropTableDeleteData(t *testing.T) { const numRows = 2*row.TableTruncateChunkSize + 1 const numKeys = 3 * numRows const numTables = 5 - var descs []*tabledesc.Immutable + var descs []catalog.TableDescriptor for i := 0; i < numTables; i++ { tableName := fmt.Sprintf("test%d", i) if err := tests.CreateKVTable(sqlDB, tableName, numRows); err != nil { diff --git a/pkg/sql/gcjob/index_garbage_collection.go b/pkg/sql/gcjob/index_garbage_collection.go index f96a967907d3..e1b75b1616ce 100644 --- a/pkg/sql/gcjob/index_garbage_collection.go +++ b/pkg/sql/gcjob/index_garbage_collection.go @@ -17,9 +17,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "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" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -45,7 +45,7 @@ func gcIndexes( return err } - var parentTable *tabledesc.Immutable + var parentTable catalog.TableDescriptor if err := execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { parentTable, err = catalogkv.MustGetTableDescByID(ctx, txn, execCfg.Codec, parentID) return err @@ -82,7 +82,7 @@ func gcIndexes( func clearIndex( ctx context.Context, execCfg *sql.ExecutorConfig, - tableDesc *tabledesc.Immutable, + tableDesc catalog.TableDescriptor, index descpb.IndexDescriptor, ) error { log.Infof(ctx, "clearing index %d from table %d", index.ID, tableDesc.GetID()) @@ -109,7 +109,7 @@ func clearIndex( func completeDroppedIndex( ctx context.Context, execCfg *sql.ExecutorConfig, - table *tabledesc.Immutable, + table catalog.TableDescriptor, indexID descpb.IndexID, progress *jobspb.SchemaChangeGCProgress, ) error { diff --git a/pkg/sql/gcjob/refresh_statuses.go b/pkg/sql/gcjob/refresh_statuses.go index 8b8e635ff89b..95db21e99ca4 100644 --- a/pkg/sql/gcjob/refresh_statuses.go +++ b/pkg/sql/gcjob/refresh_statuses.go @@ -26,7 +26,6 @@ import ( "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" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" @@ -145,7 +144,7 @@ func updateTableStatus( execCfg *sql.ExecutorConfig, ttlSeconds int64, protectedtsCache protectedts.Cache, - table *tabledesc.Immutable, + table catalog.TableDescriptor, tableDropTimes map[descpb.ID]int64, progress *jobspb.SchemaChangeGCProgress, ) time.Time { @@ -190,7 +189,7 @@ func updateIndexesStatus( ctx context.Context, execCfg *sql.ExecutorConfig, tableTTL int32, - table *tabledesc.Immutable, + table catalog.TableDescriptor, protectedtsCache protectedts.Cache, zoneCfg *zonepb.ZoneConfig, indexDropTimes map[descpb.IndexID]int64, diff --git a/pkg/sql/gcjob/table_garbage_collection.go b/pkg/sql/gcjob/table_garbage_collection.go index 6d3615547b6e..7d8277cc25ab 100644 --- a/pkg/sql/gcjob/table_garbage_collection.go +++ b/pkg/sql/gcjob/table_garbage_collection.go @@ -22,7 +22,6 @@ import ( "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/tabledesc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" @@ -43,7 +42,7 @@ func gcTables( continue } - var table *tabledesc.Immutable + var table catalog.TableDescriptor if err := execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { var err error table, err = catalogkv.MustGetTableDescByID(ctx, txn, execCfg.Codec, droppedTable.ID) @@ -87,7 +86,7 @@ func ClearTableData( db *kv.DB, distSender *kvcoord.DistSender, codec keys.SQLCodec, - table *tabledesc.Immutable, + table catalog.TableDescriptor, ) error { // If DropTime isn't set, assume this drop request is from a version // 1.1 server and invoke legacy code that uses DeleteRange and range GC. diff --git a/pkg/sql/insert_fast_path.go b/pkg/sql/insert_fast_path.go index f1e0015d182c..0e29d1451a28 100644 --- a/pkg/sql/insert_fast_path.go +++ b/pkg/sql/insert_fast_path.go @@ -18,7 +18,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/opt/exec" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" @@ -87,7 +86,7 @@ type insertFastPathFKSpanInfo struct { type insertFastPathFKCheck struct { exec.InsertFastPathFKCheck - tabDesc *tabledesc.Immutable + tabDesc catalog.TableDescriptor idxDesc *descpb.IndexDescriptor keyPrefix []byte colMap catalog.TableColMap diff --git a/pkg/sql/old_foreign_key_desc_test.go b/pkg/sql/old_foreign_key_desc_test.go index db3e9aa008bc..00fd31cae192 100644 --- a/pkg/sql/old_foreign_key_desc_test.go +++ b/pkg/sql/old_foreign_key_desc_test.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "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" @@ -56,7 +57,7 @@ CREATE INDEX ON t.t1 (x); // downgradeForeignKey downgrades a table descriptor's foreign key representation // to the pre-19.2 table descriptor format where foreign key information // is stored on the index. - downgradeForeignKey := func(tbl *tabledesc.Immutable) *tabledesc.Immutable { + downgradeForeignKey := func(tbl catalog.TableDescriptor) catalog.TableDescriptor { // Downgrade the outbound foreign keys. for i := range tbl.GetOutboundFKs() { fk := &tbl.GetOutboundFKs()[i] @@ -64,7 +65,7 @@ CREATE INDEX ON t.t1 (x); if err != nil { t.Fatal(err) } - var referencedTbl *tabledesc.Immutable + var referencedTbl catalog.TableDescriptor err = kvDB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { referencedTbl, err = catalogkv.MustGetTableDescByID(ctx, txn, keys.SystemSQLCodec, fk.ReferencedTableID) return err @@ -95,7 +96,7 @@ CREATE INDEX ON t.t1 (x); if err != nil { t.Fatal(err) } - var originTbl *tabledesc.Immutable + var originTbl catalog.TableDescriptor if err := kvDB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { originTbl, err = catalogkv.MustGetTableDescByID(ctx, txn, keys.SystemSQLCodec, fk.OriginTableID) return err diff --git a/pkg/sql/opt_catalog.go b/pkg/sql/opt_catalog.go index 5a35c7956a13..8715bfaaec45 100644 --- a/pkg/sql/opt_catalog.go +++ b/pkg/sql/opt_catalog.go @@ -27,7 +27,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" @@ -59,7 +58,7 @@ type optCatalog struct { // repeated calls for the same data source. // Note that the data source object might still need to be recreated if // something outside of the descriptor has changed (e.g. table stats). - dataSources map[*tabledesc.Immutable]cat.DataSource + dataSources map[catalog.TableDescriptor]cat.DataSource // tn is a temporary name used during resolution to avoid heap allocation. tn tree.TableName @@ -72,7 +71,7 @@ var _ cat.Catalog = &optCatalog{} // called for each query. func (oc *optCatalog) init(planner *planner) { oc.planner = planner - oc.dataSources = make(map[*tabledesc.Immutable]cat.DataSource) + oc.dataSources = make(map[catalog.TableDescriptor]cat.DataSource) } // reset prepares the optCatalog to be used for a new query. @@ -81,7 +80,7 @@ func (oc *optCatalog) reset() { // This deals with possible edge cases where we do a lot of DDL in a // long-lived session. if len(oc.dataSources) > 100 { - oc.dataSources = make(map[*tabledesc.Immutable]cat.DataSource) + oc.dataSources = make(map[catalog.TableDescriptor]cat.DataSource) } oc.cfg = oc.planner.execCfg.SystemConfig.GetSystemConfig() @@ -276,7 +275,7 @@ func getDescFromCatalogObjectForPermissions(o cat.Object) (catalog.Descriptor, e } } -func getDescForDataSource(o cat.DataSource) (*tabledesc.Immutable, error) { +func getDescForDataSource(o cat.DataSource) (catalog.TableDescriptor, error) { switch t := o.(type) { case *optTable: return t.desc, nil @@ -358,7 +357,7 @@ func (oc *optCatalog) fullyQualifiedNameWithTxn( // dataSourceForDesc returns a data source wrapper for the given descriptor. // The wrapper might come from the cache, or it may be created now. func (oc *optCatalog) dataSourceForDesc( - ctx context.Context, flags cat.Flags, desc *tabledesc.Immutable, name *cat.DataSourceName, + ctx context.Context, flags cat.Flags, desc catalog.TableDescriptor, name *cat.DataSourceName, ) (cat.DataSource, error) { // Because they are backed by physical data, we treat materialized views // as tables for the purposes of planning. @@ -390,7 +389,7 @@ func (oc *optCatalog) dataSourceForDesc( // dataSourceForTable returns a table data source wrapper for the given descriptor. // The wrapper might come from the cache, or it may be created now. func (oc *optCatalog) dataSourceForTable( - ctx context.Context, flags cat.Flags, desc *tabledesc.Immutable, name *cat.DataSourceName, + ctx context.Context, flags cat.Flags, desc catalog.TableDescriptor, name *cat.DataSourceName, ) (cat.DataSource, error) { if desc.IsVirtualTable() { // Virtual tables can have multiple effective instances that utilize the @@ -438,7 +437,7 @@ var emptyZoneConfig = &zonepb.ZoneConfig{} // ZoneConfigs are stored in protobuf binary format in the SystemConfig, which // is gossiped around the cluster. Note that the returned ZoneConfig might be // somewhat stale, since it's taken from the gossiped SystemConfig. -func (oc *optCatalog) getZoneConfig(desc *tabledesc.Immutable) (*zonepb.ZoneConfig, error) { +func (oc *optCatalog) getZoneConfig(desc catalog.TableDescriptor) (*zonepb.ZoneConfig, error) { // Lookup table's zone if system config is available (it may not be as node // is starting up and before it's received the gossiped config). If it is // not available, use an empty config that has no zone constraints. @@ -460,15 +459,15 @@ func (oc *optCatalog) codec() keys.SQLCodec { return oc.planner.ExecCfg().Codec } -// optView is a wrapper around sqlbase.Immutable that implements +// optView is a wrapper around catalog.TableDescriptor that implements // the cat.Object, cat.DataSource, and cat.View interfaces. type optView struct { - desc *tabledesc.Immutable + desc catalog.TableDescriptor } var _ cat.View = &optView{} -func newOptView(desc *tabledesc.Immutable) *optView { +func newOptView(desc catalog.TableDescriptor) *optView { return &optView{desc: desc} } @@ -516,16 +515,16 @@ func (ov *optView) ColumnName(i int) tree.Name { return tree.Name(ov.desc.GetPublicColumns()[i].Name) } -// optSequence is a wrapper around sqlbase.Immutable that +// optSequence is a wrapper around catalog.TableDescriptor that // implements the cat.Object and cat.DataSource interfaces. type optSequence struct { - desc *tabledesc.Immutable + desc catalog.TableDescriptor } var _ cat.DataSource = &optSequence{} var _ cat.Sequence = &optSequence{} -func newOptSequence(desc *tabledesc.Immutable) *optSequence { +func newOptSequence(desc catalog.TableDescriptor) *optSequence { return &optSequence{desc: desc} } @@ -556,10 +555,10 @@ func (os *optSequence) Name() tree.Name { // SequenceMarker is part of the cat.Sequence interface. func (os *optSequence) SequenceMarker() {} -// optTable is a wrapper around sqlbase.Immutable that caches +// optTable is a wrapper around catalog.TableDescriptor that caches // index wrappers and maintains a ColumnID => Column mapping for fast lookup. type optTable struct { - desc *tabledesc.Immutable + desc catalog.TableDescriptor // columns contains all the columns presented to the catalog. This includes: // - ordinary table columns (those in the table descriptor) @@ -613,7 +612,7 @@ type optTable struct { var _ cat.Table = &optTable{} func newOptTable( - desc *tabledesc.Immutable, + desc catalog.TableDescriptor, codec keys.SQLCodec, stats []*stats.TableStatistic, tblZone *zonepb.ZoneConfig, @@ -897,7 +896,7 @@ func (ot *optTable) PostgresDescriptorID() cat.StableID { // isStale checks if the optTable object needs to be refreshed because the stats, // zone config, or used types have changed. False positives are ok. func (ot *optTable) isStale( - rawDesc *tabledesc.Immutable, tableStats []*stats.TableStatistic, zone *zonepb.ZoneConfig, + rawDesc catalog.TableDescriptor, tableStats []*stats.TableStatistic, zone *zonepb.ZoneConfig, ) bool { // Fast check to verify that the statistics haven't changed: we check the // length and the address of the underlying array. This is not a perfect @@ -1625,7 +1624,7 @@ func (fk *optForeignKeyConstraint) UpdateReferenceAction() tree.ReferenceAction // optVirtualTable is similar to optTable but is used with virtual tables. type optVirtualTable struct { - desc *tabledesc.Immutable + desc catalog.TableDescriptor // columns contains all the columns presented to the catalog. This includes // the dummy PK column and the columns in the table descriptor. @@ -1664,7 +1663,7 @@ type optVirtualTable struct { var _ cat.Table = &optVirtualTable{} func newOptVirtualTable( - ctx context.Context, oc *optCatalog, desc *tabledesc.Immutable, name *cat.DataSourceName, + ctx context.Context, oc *optCatalog, desc catalog.TableDescriptor, name *cat.DataSourceName, ) (*optVirtualTable, error) { // Calculate the stable ID (see the comment for optVirtualTable.id). id := cat.StableID(desc.GetID()) diff --git a/pkg/sql/opt_exec_factory.go b/pkg/sql/opt_exec_factory.go index d2a58ee9c260..c7055b7e4c90 100644 --- a/pkg/sql/opt_exec_factory.go +++ b/pkg/sql/opt_exec_factory.go @@ -26,7 +26,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/inverted" "github.com/cockroachdb/cockroach/pkg/sql/opt" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" @@ -135,7 +134,7 @@ func (ef *execFactory) ConstructScan( func generateScanSpans( evalCtx *tree.EvalContext, codec keys.SQLCodec, - tabDesc *tabledesc.Immutable, + tabDesc catalog.TableDescriptor, indexDesc *descpb.IndexDescriptor, params exec.ScanParams, ) (roachpb.Spans, error) { @@ -750,7 +749,7 @@ func (ef *execFactory) ConstructInvertedJoin( // and requested cols. func (ef *execFactory) constructScanForZigzag( indexDesc *descpb.IndexDescriptor, - tableDesc *tabledesc.Immutable, + tableDesc catalog.TableDescriptor, cols exec.TableColumnOrdinalSet, ) (*scanNode, error) { @@ -1597,7 +1596,7 @@ func (ef *execFactory) ConstructDeleteRange( if len(interleavedTables) > 0 { dr.interleavedFastPath = true - dr.interleavedDesc = make([]*tabledesc.Immutable, len(interleavedTables)) + dr.interleavedDesc = make([]catalog.TableDescriptor, len(interleavedTables)) for i := range dr.interleavedDesc { dr.interleavedDesc[i] = interleavedTables[i].(*optTable).desc } diff --git a/pkg/sql/physicalplan/BUILD.bazel b/pkg/sql/physicalplan/BUILD.bazel index 47705f8ca994..abb7b58db693 100644 --- a/pkg/sql/physicalplan/BUILD.bazel +++ b/pkg/sql/physicalplan/BUILD.bazel @@ -54,9 +54,9 @@ go_test( "//pkg/security", "//pkg/security/securitytest", "//pkg/server", + "//pkg/sql/catalog", "//pkg/sql/catalog/catalogkeys", "//pkg/sql/catalog/catalogkv", - "//pkg/sql/catalog/tabledesc", "//pkg/sql/distsql", "//pkg/sql/execinfra", "//pkg/sql/execinfrapb", diff --git a/pkg/sql/physicalplan/aggregator_funcs_test.go b/pkg/sql/physicalplan/aggregator_funcs_test.go index f9f25c527358..2681984cbb09 100644 --- a/pkg/sql/physicalplan/aggregator_funcs_test.go +++ b/pkg/sql/physicalplan/aggregator_funcs_test.go @@ -20,8 +20,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/distsql" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -117,7 +117,7 @@ func checkDistAggregationInfo( ctx context.Context, t *testing.T, srv serverutils.TestServerInterface, - tableDesc *tabledesc.Immutable, + tableDesc catalog.TableDescriptor, colIdx int, numRows int, fn execinfrapb.AggregatorSpec_Func, diff --git a/pkg/sql/physicalplan/span_resolver_test.go b/pkg/sql/physicalplan/span_resolver_test.go index 90ff68eb6204..c9211cff5dd6 100644 --- a/pkg/sql/physicalplan/span_resolver_test.go +++ b/pkg/sql/physicalplan/span_resolver_test.go @@ -23,8 +23,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/physicalplan" "github.com/cockroachdb/cockroach/pkg/sql/physicalplan/replicaoracle" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -167,7 +167,7 @@ func populateCache(db *gosql.DB, expectedNumRows int) error { // `CREATE TABLE test (k INT PRIMARY KEY)` at row with value pk (the row will be // the first on the right of the split). func splitRangeAtVal( - ts *server.TestServer, tableDesc *tabledesc.Immutable, pk int, + ts *server.TestServer, tableDesc catalog.TableDescriptor, pk int, ) (roachpb.RangeDescriptor, roachpb.RangeDescriptor, error) { if len(tableDesc.PublicNonPrimaryIndexes()) != 0 { return roachpb.RangeDescriptor{}, roachpb.RangeDescriptor{}, @@ -321,7 +321,7 @@ func TestMixedDirections(t *testing.T) { func setupRanges( db *gosql.DB, s *server.TestServer, cdb *kv.DB, t *testing.T, -) ([]roachpb.RangeDescriptor, *tabledesc.Immutable) { +) ([]roachpb.RangeDescriptor, catalog.TableDescriptor) { if _, err := db.Exec(`CREATE DATABASE t`); err != nil { t.Fatal(err) } @@ -451,7 +451,7 @@ func expectResolved(actual [][]rngInfo, expected ...[]rngInfo) error { return nil } -func makeSpan(tableDesc *tabledesc.Immutable, i, j int) roachpb.Span { +func makeSpan(tableDesc catalog.TableDescriptor, i, j int) roachpb.Span { makeKey := func(val int) roachpb.Key { key, err := rowenc.TestingMakePrimaryIndexKey(tableDesc, val) if err != nil { diff --git a/pkg/sql/planhook.go b/pkg/sql/planhook.go index 8522bf961b88..f61af5e7bba6 100644 --- a/pkg/sql/planhook.go +++ b/pkg/sql/planhook.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -92,7 +93,7 @@ type PlanHookState interface { ctx context.Context, tn *tree.TableName, required bool, requiredType tree.RequiredTableKind, ) (table *tabledesc.Mutable, err error) ShowCreate( - ctx context.Context, dbPrefix string, allDescs []descpb.Descriptor, desc *tabledesc.Immutable, displayOptions ShowCreateDisplayOptions, + ctx context.Context, dbPrefix string, allDescs []descpb.Descriptor, desc catalog.TableDescriptor, displayOptions ShowCreateDisplayOptions, ) (string, error) CreateSchemaNamespaceEntry(ctx context.Context, schemaNameKey roachpb.Key, schemaID descpb.ID) error diff --git a/pkg/sql/planner.go b/pkg/sql/planner.go index ed85f7398a79..c387992bbbe4 100644 --- a/pkg/sql/planner.go +++ b/pkg/sql/planner.go @@ -28,7 +28,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" "github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/opt/exec" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/querycache" @@ -506,7 +505,7 @@ func (p *planner) ResolveTableName(ctx context.Context, tn *tree.TableName) (tre // of having its own logic for lookups. func (p *planner) LookupTableByID( ctx context.Context, tableID descpb.ID, -) (*tabledesc.Immutable, error) { +) (catalog.TableDescriptor, error) { if entry, err := p.getVirtualTabler().getVirtualTableEntryByID(tableID); err == nil { return entry.desc, nil } diff --git a/pkg/sql/reassign_owned_by.go b/pkg/sql/reassign_owned_by.go index 54a6a0397b04..41e82ac010ff 100644 --- a/pkg/sql/reassign_owned_by.go +++ b/pkg/sql/reassign_owned_by.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/server/telemetry" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemadesc" @@ -150,7 +151,7 @@ func (n *reassignOwnedByNode) reassignSchemaOwner( } func (n *reassignOwnedByNode) reassignTableOwner( - tbDesc *tabledesc.Immutable, params runParams, + tbDesc catalog.TableDescriptor, params runParams, ) error { mutableTbDesc, err := params.p.Descriptors().GetMutableDescriptorByID( params.ctx, tbDesc.GetID(), params.p.txn) diff --git a/pkg/sql/relocate.go b/pkg/sql/relocate.go index 1adabfcc0eac..40fd2bc6cf24 100644 --- a/pkg/sql/relocate.go +++ b/pkg/sql/relocate.go @@ -17,9 +17,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -30,7 +30,7 @@ type relocateNode struct { optColumnsSlot relocateLease bool - tableDesc *tabledesc.Immutable + tableDesc catalog.TableDescriptor index *descpb.IndexDescriptor rows planNode diff --git a/pkg/sql/rename_database.go b/pkg/sql/rename_database.go index 535594cb517d..a95b7dbfb68e 100644 --- a/pkg/sql/rename_database.go +++ b/pkg/sql/rename_database.go @@ -18,7 +18,6 @@ import ( "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/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" @@ -261,7 +260,7 @@ func isAllowedDependentDescInRenameDatabase( ctx context.Context, dependedOn *descpb.TableDescriptor_Reference, tbDesc catalog.TableDescriptor, - dependentDesc *tabledesc.Immutable, + dependentDesc catalog.TableDescriptor, dbName string, ) (bool, string, error) { // If it is a sequence, and it does not contain the database name, then we have diff --git a/pkg/sql/resolver.go b/pkg/sql/resolver.go index d6d72eca95ad..45a972fd8d99 100644 --- a/pkg/sql/resolver.go +++ b/pkg/sql/resolver.go @@ -122,7 +122,7 @@ func (p *planner) ResolveMutableTableDescriptor( func (p *planner) ResolveUncachedTableDescriptor( ctx context.Context, tn *tree.TableName, required bool, requiredType tree.RequiredTableKind, -) (table *tabledesc.Immutable, err error) { +) (table catalog.TableDescriptor, err error) { p.runWithOptions(resolveFlags{skipCache: true}, func() { lookupFlags := tree.ObjectLookupFlags{ CommonLookupFlags: tree.CommonLookupFlags{Required: required}, @@ -723,7 +723,7 @@ type internalLookupCtx struct { schemaDescs map[descpb.ID]*schemadesc.Immutable schemaNames map[descpb.ID]string schemaIDs []descpb.ID - tbDescs map[descpb.ID]*tabledesc.Immutable + tbDescs map[descpb.ID]catalog.TableDescriptor tbIDs []descpb.ID typDescs map[descpb.ID]*typedesc.Immutable typIDs []descpb.ID @@ -811,7 +811,7 @@ func newInternalLookupCtx( schemaNames := map[descpb.ID]string{ keys.PublicSchemaID: tree.PublicSchema, } - tbDescs := make(map[descpb.ID]*tabledesc.Immutable) + tbDescs := make(map[descpb.ID]catalog.TableDescriptor) typDescs := make(map[descpb.ID]*typedesc.Immutable) var tbIDs, typIDs, dbIDs, schemaIDs []descpb.ID // Record descriptors for name lookups. @@ -824,7 +824,7 @@ func newInternalLookupCtx( // Only make the database visible for iteration if the prefix was included. dbIDs = append(dbIDs, desc.GetID()) } - case *tabledesc.Immutable: + case catalog.TableDescriptor: tbDescs[desc.GetID()] = desc if prefix == nil || prefix.GetID() == desc.GetParentID() { // Only make the table visible for iteration if the prefix was included. @@ -1066,7 +1066,7 @@ func (p *planner) ResolveUncachedTableDescriptorEx( name *tree.UnresolvedObjectName, required bool, requiredType tree.RequiredTableKind, -) (table *tabledesc.Immutable, err error) { +) (table catalog.TableDescriptor, err error) { p.runWithOptions(resolveFlags{skipCache: true}, func() { table, err = p.ResolveExistingObjectEx(ctx, name, required, requiredType) }) @@ -1090,7 +1090,7 @@ func (p *planner) ResolveExistingObjectEx( name *tree.UnresolvedObjectName, required bool, requiredType tree.RequiredTableKind, -) (res *tabledesc.Immutable, err error) { +) (res catalog.TableDescriptor, err error) { lookupFlags := tree.ObjectLookupFlags{ CommonLookupFlags: tree.CommonLookupFlags{Required: required}, DesiredObjectKind: tree.TableObject, @@ -1102,7 +1102,7 @@ func (p *planner) ResolveExistingObjectEx( } tn := tree.MakeTableNameFromPrefix(prefix, tree.Name(name.Object())) name.SetAnnotation(&p.semaCtx.Annotations, &tn) - table := desc.(*tabledesc.Immutable) + table := desc.(catalog.TableDescriptor) // Ensure that the user can access the target schema. if err := p.canResolveDescUnderSchema(ctx, table.GetParentSchemaID(), table); err != nil { diff --git a/pkg/sql/revert.go b/pkg/sql/revert.go index f0127df190c2..504d12c16164 100644 --- a/pkg/sql/revert.go +++ b/pkg/sql/revert.go @@ -15,8 +15,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" + "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/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" @@ -33,7 +33,7 @@ func RevertTables( ctx context.Context, db *kv.DB, execCfg *ExecutorConfig, - tables []*tabledesc.Immutable, + tables []catalog.TableDescriptor, targetTime hlc.Timestamp, batchSize int64, ) error { diff --git a/pkg/sql/revert_test.go b/pkg/sql/revert_test.go index a48d8ae5083d..7678c7921b0e 100644 --- a/pkg/sql/revert_test.go +++ b/pkg/sql/revert_test.go @@ -20,9 +20,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "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" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" @@ -80,7 +80,7 @@ func TestRevertTable(t *testing.T) { // Revert the table to ts. desc := catalogkv.TestingGetTableDescriptor(kv, keys.SystemSQLCodec, "test", "test") desc.TableDesc().State = descpb.DescriptorState_OFFLINE // bypass the offline check. - require.NoError(t, sql.RevertTables(context.Background(), kv, &execCfg, []*tabledesc.Immutable{desc}, targetTime, 10)) + require.NoError(t, sql.RevertTables(context.Background(), kv, &execCfg, []catalog.TableDescriptor{desc}, targetTime, 10)) var reverted int db.QueryRow(t, `SELECT xor_agg(k # rev) FROM test`).Scan(&reverted) @@ -109,14 +109,14 @@ func TestRevertTable(t *testing.T) { child := catalogkv.TestingGetTableDescriptor(kv, keys.SystemSQLCodec, "test", "child") child.TableDesc().State = descpb.DescriptorState_OFFLINE t.Run("reject only parent", func(t *testing.T) { - require.Error(t, sql.RevertTables(ctx, kv, &execCfg, []*tabledesc.Immutable{desc}, targetTime, 10)) + require.Error(t, sql.RevertTables(ctx, kv, &execCfg, []catalog.TableDescriptor{desc}, targetTime, 10)) }) t.Run("reject only child", func(t *testing.T) { - require.Error(t, sql.RevertTables(ctx, kv, &execCfg, []*tabledesc.Immutable{child}, targetTime, 10)) + require.Error(t, sql.RevertTables(ctx, kv, &execCfg, []catalog.TableDescriptor{child}, targetTime, 10)) }) t.Run("rollback parent and child", func(t *testing.T) { - require.NoError(t, sql.RevertTables(ctx, kv, &execCfg, []*tabledesc.Immutable{desc, child}, targetTime, sql.RevertTableDefaultBatchSize)) + require.NoError(t, sql.RevertTables(ctx, kv, &execCfg, []catalog.TableDescriptor{desc, child}, targetTime, sql.RevertTableDefaultBatchSize)) var reverted, revertedChild int db.QueryRow(t, `SELECT xor_agg(k # rev) FROM test`).Scan(&reverted) diff --git a/pkg/sql/row/BUILD.bazel b/pkg/sql/row/BUILD.bazel index d6c2bbe15e07..22907d3b2def 100644 --- a/pkg/sql/row/BUILD.bazel +++ b/pkg/sql/row/BUILD.bazel @@ -33,7 +33,6 @@ go_library( "//pkg/sql/catalog/colinfo", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/schemaexpr", - "//pkg/sql/catalog/tabledesc", "//pkg/sql/execinfra", "//pkg/sql/execinfrapb", "//pkg/sql/pgwire/pgcode", diff --git a/pkg/sql/row/deleter.go b/pkg/sql/row/deleter.go index 4ebe2e29232b..29ee48804127 100644 --- a/pkg/sql/row/deleter.go +++ b/pkg/sql/row/deleter.go @@ -18,7 +18,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "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/log" @@ -42,7 +41,7 @@ type Deleter struct { // FetchCols; otherwise, all columns that are part of the key of any index // (either primary or secondary) are included in FetchCols. func MakeDeleter( - codec keys.SQLCodec, tableDesc *tabledesc.Immutable, requestedCols []descpb.ColumnDescriptor, + codec keys.SQLCodec, tableDesc catalog.TableDescriptor, requestedCols []descpb.ColumnDescriptor, ) Deleter { indexes := tableDesc.DeletableNonPrimaryIndexes() indexDescs := make([]descpb.IndexDescriptor, len(indexes)) diff --git a/pkg/sql/row/expr_walker.go b/pkg/sql/row/expr_walker.go index 3ad88d097d8e..11c5f3dc3119 100644 --- a/pkg/sql/row/expr_walker.go +++ b/pkg/sql/row/expr_walker.go @@ -19,9 +19,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" @@ -102,7 +102,7 @@ func makeBuiltinOverride( // default expressions which use sequences. type SequenceMetadata struct { id descpb.ID - seqDesc *tabledesc.Immutable + seqDesc catalog.TableDescriptor instancesPerRow int64 curChunk *jobspb.SequenceValChunk curVal int64 @@ -307,7 +307,7 @@ func (j *SeqChunkProvider) RequestChunk( func incrementSequenceByVal( ctx context.Context, - descriptor *tabledesc.Immutable, + descriptor catalog.TableDescriptor, db *kv.DB, codec keys.SQLCodec, incrementBy int64, @@ -334,7 +334,7 @@ func incrementSequenceByVal( return val, nil } -func boundsExceededError(descriptor *tabledesc.Immutable) error { +func boundsExceededError(descriptor catalog.TableDescriptor) error { seqOpts := descriptor.GetSequenceOpts() isAscending := seqOpts.Increment > 0 diff --git a/pkg/sql/row/expr_walker_test.go b/pkg/sql/row/expr_walker_test.go index dbe6cc2b6e9c..9cc6876161a3 100644 --- a/pkg/sql/row/expr_walker_test.go +++ b/pkg/sql/row/expr_walker_test.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "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/sem/tree" @@ -37,8 +38,8 @@ func createAndIncrementSeqDescriptor( incrementBy int64, seqOpts descpb.TableDescriptor_SequenceOpts, db *kv.DB, -) tabledesc.Immutable { - desc := tabledesc.MakeImmutable(descpb.TableDescriptor{ +) catalog.TableDescriptor { + desc := tabledesc.NewImmutable(descpb.TableDescriptor{ ID: descpb.ID(id), SequenceOpts: &seqOpts, }) @@ -200,7 +201,7 @@ func TestJobBackedSeqChunkProvider(t *testing.T) { test.incrementBy, test.seqIDToOpts[id], db) seqMetadata := &SequenceMetadata{ id: descpb.ID(id), - seqDesc: &seqDesc, + seqDesc: seqDesc, instancesPerRow: test.instancesPerRow, curChunk: nil, curVal: 0, diff --git a/pkg/sql/row/fetcher.go b/pkg/sql/row/fetcher.go index 53ade2816ed7..59b1e0053fec 100644 --- a/pkg/sql/row/fetcher.go +++ b/pkg/sql/row/fetcher.go @@ -23,7 +23,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -162,7 +161,7 @@ type FetcherTableArgs struct { // InitCols initializes the columns in FetcherTableArgs. func (fta *FetcherTableArgs) InitCols( - desc *tabledesc.Immutable, + desc catalog.TableDescriptor, scanVisibility execinfrapb.ScanVisibility, systemColumns []descpb.ColumnDescriptor, virtualColumn *descpb.ColumnDescriptor, diff --git a/pkg/sql/row/fetcher_mvcc_test.go b/pkg/sql/row/fetcher_mvcc_test.go index 84133cdb37aa..5e01fec46763 100644 --- a/pkg/sql/row/fetcher_mvcc_test.go +++ b/pkg/sql/row/fetcher_mvcc_test.go @@ -22,7 +22,6 @@ import ( "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" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -90,7 +89,7 @@ func TestRowFetcherMVCCMetadata(t *testing.T) { parentDesc := catalogkv.TestingGetImmutableTableDescriptor(kvDB, keys.SystemSQLCodec, `d`, `parent`) childDesc := catalogkv.TestingGetImmutableTableDescriptor(kvDB, keys.SystemSQLCodec, `d`, `child`) var args []row.FetcherTableArgs - for _, desc := range []*tabledesc.Immutable{parentDesc, childDesc} { + for _, desc := range []catalog.TableDescriptor{parentDesc, childDesc} { var colIdxMap catalog.TableColMap var valNeededForCol util.FastIntSet for colIdx := range desc.GetPublicColumns() { diff --git a/pkg/sql/row/fetcher_test.go b/pkg/sql/row/fetcher_test.go index e4326d1967cb..0e379806d881 100644 --- a/pkg/sql/row/fetcher_test.go +++ b/pkg/sql/row/fetcher_test.go @@ -22,9 +22,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "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/catalogkv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -39,7 +39,7 @@ import ( ) type initFetcherArgs struct { - tableDesc *tabledesc.Immutable + tableDesc catalog.TableDescriptor indexIdx int valNeededForCol util.FastIntSet spans roachpb.Spans diff --git a/pkg/sql/row/inserter.go b/pkg/sql/row/inserter.go index 220535788d90..9e66fa40a0af 100644 --- a/pkg/sql/row/inserter.go +++ b/pkg/sql/row/inserter.go @@ -19,7 +19,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "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/log" @@ -47,7 +46,7 @@ func MakeInserter( ctx context.Context, txn *kv.Txn, codec keys.SQLCodec, - tableDesc *tabledesc.Immutable, + tableDesc catalog.TableDescriptor, insertCols []descpb.ColumnDescriptor, alloc *rowenc.DatumAlloc, ) (Inserter, error) { diff --git a/pkg/sql/row/row_converter.go b/pkg/sql/row/row_converter.go index bd82ceec498d..d0ac14ed4df6 100644 --- a/pkg/sql/row/row_converter.go +++ b/pkg/sql/row/row_converter.go @@ -20,7 +20,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" "github.com/cockroachdb/cockroach/pkg/sql/sem/transform" @@ -94,7 +93,7 @@ func GenerateInsertRow( insertCols []descpb.ColumnDescriptor, computedColsLookup []descpb.ColumnDescriptor, evalCtx *tree.EvalContext, - tableDesc *tabledesc.Immutable, + tableDesc catalog.TableDescriptor, rowVals tree.Datums, rowContainerForComputedVals *schemaexpr.RowIndexedVarContainer, ) (tree.Datums, error) { @@ -206,7 +205,7 @@ type DatumRowConverter struct { KvBatch KVBatch BatchCap int - tableDesc *tabledesc.Immutable + tableDesc catalog.TableDescriptor // Tracks which column indices in the set of visible columns are part of the // user specified target columns. This can be used before populating Datums @@ -290,7 +289,7 @@ func (c *DatumRowConverter) getSequenceAnnotation( // NewDatumRowConverter returns an instance of a DatumRowConverter. func NewDatumRowConverter( ctx context.Context, - tableDesc *tabledesc.Immutable, + tableDesc catalog.TableDescriptor, targetColNames tree.NameList, evalCtx *tree.EvalContext, kvCh chan<- KVBatch, diff --git a/pkg/sql/row/updater.go b/pkg/sql/row/updater.go index ccedf4ec62ed..b2246e8958d9 100644 --- a/pkg/sql/row/updater.go +++ b/pkg/sql/row/updater.go @@ -20,7 +20,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "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/log" @@ -85,7 +84,7 @@ func MakeUpdater( ctx context.Context, txn *kv.Txn, codec keys.SQLCodec, - tableDesc *tabledesc.Immutable, + tableDesc catalog.TableDescriptor, updateCols []descpb.ColumnDescriptor, requestedCols []descpb.ColumnDescriptor, updateType rowUpdaterType, diff --git a/pkg/sql/rowenc/index_encoding_test.go b/pkg/sql/rowenc/index_encoding_test.go index 29399c4ef3bb..636d16f0171b 100644 --- a/pkg/sql/rowenc/index_encoding_test.go +++ b/pkg/sql/rowenc/index_encoding_test.go @@ -52,7 +52,7 @@ type indexKeyTest struct { secondaryValues []tree.Datum // len must be at least secondaryInterleaveComponents+1 } -func makeTableDescForTest(test indexKeyTest) (*tabledesc.Immutable, catalog.TableColMap) { +func makeTableDescForTest(test indexKeyTest) (catalog.TableDescriptor, catalog.TableColMap) { primaryColumnIDs := make([]descpb.ColumnID, len(test.primaryValues)) secondaryColumnIDs := make([]descpb.ColumnID, len(test.secondaryValues)) columns := make([]descpb.ColumnDescriptor, len(test.primaryValues)+len(test.secondaryValues)) @@ -111,7 +111,7 @@ func makeTableDescForTest(test indexKeyTest) (*tabledesc.Immutable, catalog.Tabl } func decodeIndex( - codec keys.SQLCodec, tableDesc *tabledesc.Immutable, index *descpb.IndexDescriptor, key []byte, + codec keys.SQLCodec, tableDesc catalog.TableDescriptor, index *descpb.IndexDescriptor, key []byte, ) ([]tree.Datum, error) { types, err := colinfo.GetColumnTypes(tableDesc, index.ColumnIDs, nil) if err != nil { diff --git a/pkg/sql/rowexec/BUILD.bazel b/pkg/sql/rowexec/BUILD.bazel index 35ce3195f656..5736789c3bef 100644 --- a/pkg/sql/rowexec/BUILD.bazel +++ b/pkg/sql/rowexec/BUILD.bazel @@ -147,7 +147,6 @@ go_test( "//pkg/sql/catalog/catalogkv", "//pkg/sql/catalog/colinfo", "//pkg/sql/catalog/descpb", - "//pkg/sql/catalog/tabledesc", "//pkg/sql/execinfra", "//pkg/sql/execinfrapb", "//pkg/sql/flowinfra", diff --git a/pkg/sql/rowexec/bulk_row_writer.go b/pkg/sql/rowexec/bulk_row_writer.go index 9b0f1fe4609a..ffcec8c01863 100644 --- a/pkg/sql/rowexec/bulk_row_writer.go +++ b/pkg/sql/rowexec/bulk_row_writer.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -38,7 +39,7 @@ type bulkRowWriter struct { flowCtx *execinfra.FlowCtx processorID int32 batchIdxAtomic int64 - tableDesc tabledesc.Immutable + tableDesc catalog.TableDescriptor spec execinfrapb.BulkRowWriterSpec input execinfra.RowSource output execinfra.RowReceiver @@ -59,7 +60,7 @@ func newBulkRowWriterProcessor( flowCtx: flowCtx, processorID: processorID, batchIdxAtomic: 0, - tableDesc: tabledesc.MakeImmutable(spec.Table), + tableDesc: tabledesc.NewImmutable(spec.Table), spec: spec, input: input, output: output, @@ -103,7 +104,7 @@ func (sp *bulkRowWriter) work(ctx context.Context) error { var g ctxgroup.Group conv, err := row.NewDatumRowConverter(ctx, - &sp.tableDesc, nil /* targetColNames */, sp.EvalCtx, kvCh, nil /* seqChunkProvider */) + sp.tableDesc, nil /* targetColNames */, sp.EvalCtx, kvCh, nil /* seqChunkProvider */) if err != nil { return err } @@ -127,7 +128,7 @@ func (sp *bulkRowWriter) wrapDupError(ctx context.Context, orig error) error { return orig } v := &roachpb.Value{RawBytes: typed.Value} - return row.NewUniquenessConstraintViolationError(ctx, &sp.tableDesc, typed.Key, v) + return row.NewUniquenessConstraintViolationError(ctx, sp.tableDesc, typed.Key, v) } func (sp *bulkRowWriter) ingestLoop(ctx context.Context, kvCh chan row.KVBatch) error { diff --git a/pkg/sql/rowexec/columnbackfiller.go b/pkg/sql/rowexec/columnbackfiller.go index c1817ae9f1f5..c27d7259092f 100644 --- a/pkg/sql/rowexec/columnbackfiller.go +++ b/pkg/sql/rowexec/columnbackfiller.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/backfill" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -28,7 +29,7 @@ type columnBackfiller struct { backfill.ColumnBackfiller - desc *tabledesc.Immutable + desc catalog.TableDescriptor } var _ execinfra.Processor = &columnBackfiller{} diff --git a/pkg/sql/rowexec/indexbackfiller.go b/pkg/sql/rowexec/indexbackfiller.go index 866128e775f4..ed7013209230 100644 --- a/pkg/sql/rowexec/indexbackfiller.go +++ b/pkg/sql/rowexec/indexbackfiller.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql/backfill" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -42,7 +43,7 @@ type indexBackfiller struct { adder kvserverbase.BulkAdder - desc *tabledesc.Immutable + desc catalog.TableDescriptor spec execinfrapb.BackfillerSpec diff --git a/pkg/sql/rowexec/inverted_joiner.go b/pkg/sql/rowexec/inverted_joiner.go index 5c04991b944e..49f634582caf 100644 --- a/pkg/sql/rowexec/inverted_joiner.go +++ b/pkg/sql/rowexec/inverted_joiner.go @@ -64,7 +64,7 @@ type invertedJoiner struct { runningState invertedJoinerState diskMonitor *mon.BytesMonitor - desc tabledesc.Immutable + desc catalog.TableDescriptor // The map from ColumnIDs in the table to the column position. colIdxMap catalog.TableColMap index *descpb.IndexDescriptor @@ -185,7 +185,7 @@ func newInvertedJoiner( return nil, errors.AssertionFailedf("unexpected inverted join type %s", spec.Type) } ij := &invertedJoiner{ - desc: tabledesc.MakeImmutable(spec.Table), + desc: tabledesc.NewImmutable(spec.Table), input: input, inputTypes: input.OutputTypes(), prefixEqualityCols: spec.PrefixEqualityColumns, @@ -301,7 +301,7 @@ func newInvertedJoiner( // We use ScanVisibilityPublic since inverted joins are not used for mutations, // and so do not need to see in-progress schema changes. _, _, err = initRowFetcher( - flowCtx, &fetcher, &ij.desc, int(spec.IndexIdx), ij.colIdxMap, false, /* reverse */ + flowCtx, &fetcher, ij.desc, int(spec.IndexIdx), ij.colIdxMap, false, /* reverse */ allIndexCols, false /* isCheck */, flowCtx.EvalCtx.Mon, &ij.alloc, execinfra.ScanVisibilityPublic, descpb.ScanLockingStrength_FOR_NONE, descpb.ScanLockingWaitPolicy_BLOCK, nil /* systemColumns */, nil, /* virtualColumn */ @@ -322,7 +322,7 @@ func newInvertedJoiner( ij.fetcher = &fetcher } - ij.spanBuilder = span.MakeBuilder(flowCtx.EvalCtx, flowCtx.Codec(), &ij.desc, ij.index) + ij.spanBuilder = span.MakeBuilder(flowCtx.EvalCtx, flowCtx.Codec(), ij.desc, ij.index) ij.spanBuilder.SetNeededColumns(allIndexCols) // Initialize memory monitors and row container for index rows. @@ -448,7 +448,7 @@ func (ij *invertedJoiner) readInput() (invertedJoinerState, *execinfrapb.Produce ij.indexRow[:len(ij.prefixEqualityCols)], ij.indexRowTypes[:len(ij.prefixEqualityCols)], ij.index.ColumnDirections, - &ij.desc, + ij.desc, ij.index, &ij.alloc, nil, /* keyPrefix */ @@ -541,7 +541,7 @@ func (ij *invertedJoiner) performScan() (invertedJoinerState, *execinfrapb.Produ ij.indexRow[:len(ij.prefixEqualityCols)], ij.indexRowTypes[:len(ij.prefixEqualityCols)], ij.index.ColumnDirections, - &ij.desc, + ij.desc, ij.index, &ij.alloc, nil, /* keyPrefix */ diff --git a/pkg/sql/rowexec/joinreader.go b/pkg/sql/rowexec/joinreader.go index 1221a321da00..8c32281fc8ce 100644 --- a/pkg/sql/rowexec/joinreader.go +++ b/pkg/sql/rowexec/joinreader.go @@ -76,7 +76,7 @@ type joinReader struct { diskMonitor *mon.BytesMonitor - desc tabledesc.Immutable + desc catalog.TableDescriptor index *descpb.IndexDescriptor colIdxMap catalog.TableColMap maintainOrdering bool @@ -188,7 +188,7 @@ func newJoinReader( return nil, errors.Errorf("unsupported joinReaderType") } jr := &joinReader{ - desc: tabledesc.MakeImmutable(spec.Table), + desc: tabledesc.NewImmutable(spec.Table), maintainOrdering: spec.MaintainOrdering, input: input, lookupCols: lookupCols, @@ -292,7 +292,7 @@ func newJoinReader( var fetcher row.Fetcher _, _, err = initRowFetcher( - flowCtx, &fetcher, &jr.desc, int(spec.IndexIdx), jr.colIdxMap, false, /* reverse */ + flowCtx, &fetcher, jr.desc, int(spec.IndexIdx), jr.colIdxMap, false, /* reverse */ rightCols, false /* isCheck */, jr.EvalCtx.Mon, &jr.alloc, spec.Visibility, spec.LockingStrength, spec.LockingWaitPolicy, sysColDescs, nil, /* virtualColumn */ ) @@ -322,7 +322,7 @@ func (jr *joinReader) initJoinReaderStrategy( neededRightCols util.FastIntSet, readerType joinReaderType, ) { - spanBuilder := span.MakeBuilder(flowCtx.EvalCtx, flowCtx.Codec(), &jr.desc, jr.index) + spanBuilder := span.MakeBuilder(flowCtx.EvalCtx, flowCtx.Codec(), jr.desc, jr.index) spanBuilder.SetNeededColumns(neededRightCols) var keyToInputRowIndices map[string][]int diff --git a/pkg/sql/rowexec/joinreader_test.go b/pkg/sql/rowexec/joinreader_test.go index 83f738d3730d..63665d3d25e3 100644 --- a/pkg/sql/rowexec/joinreader_test.go +++ b/pkg/sql/rowexec/joinreader_test.go @@ -28,7 +28,6 @@ import ( "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" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -654,7 +653,7 @@ func TestJoinReader(t *testing.T) { ) diskMonitor.Start(ctx, nil /* pool */, mon.MakeStandaloneBudget(math.MaxInt64)) defer diskMonitor.Stop(ctx) - for i, td := range []*tabledesc.Immutable{tdSecondary, tdFamily, tdInterleaved} { + for i, td := range []catalog.TableDescriptor{tdSecondary, tdFamily, tdInterleaved} { for _, c := range testCases { for _, reqOrdering := range []bool{true, false} { // Small and large batches exercise different paths of interest for diff --git a/pkg/sql/rowexec/rowfetcher.go b/pkg/sql/rowexec/rowfetcher.go index cf86d1b30044..9e630b4e5b8f 100644 --- a/pkg/sql/rowexec/rowfetcher.go +++ b/pkg/sql/rowexec/rowfetcher.go @@ -18,7 +18,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "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/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/row" @@ -65,7 +64,7 @@ type rowFetcher interface { func initRowFetcher( flowCtx *execinfra.FlowCtx, fetcher *row.Fetcher, - desc *tabledesc.Immutable, + desc catalog.TableDescriptor, indexIdx int, colIdxMap catalog.TableColMap, reverseScan bool, diff --git a/pkg/sql/rowexec/scrub_tablereader.go b/pkg/sql/rowexec/scrub_tablereader.go index 37a11550c379..a20623831125 100644 --- a/pkg/sql/rowexec/scrub_tablereader.go +++ b/pkg/sql/rowexec/scrub_tablereader.go @@ -48,7 +48,7 @@ var ScrubTypes = []*types.T{ type scrubTableReader struct { tableReader - tableDesc tabledesc.Immutable + tableDesc catalog.TableDescriptor // fetcherResultToColIdx maps Fetcher results to the column index in // the TableDescriptor. This is only initialized and used during scrub // physical checks. @@ -79,7 +79,7 @@ func newScrubTableReader( indexIdx: int(spec.IndexIdx), } - tr.tableDesc = tabledesc.MakeImmutable(spec.Table) + tr.tableDesc = tabledesc.NewImmutable(spec.Table) tr.limitHint = execinfra.LimitHint(spec.LimitHint, post) if err := tr.Init( @@ -125,7 +125,7 @@ func newScrubTableReader( var fetcher row.Fetcher if _, _, err := initRowFetcher( - flowCtx, &fetcher, &tr.tableDesc, int(spec.IndexIdx), tr.tableDesc.ColumnIdxMap(), + flowCtx, &fetcher, tr.tableDesc, int(spec.IndexIdx), tr.tableDesc.ColumnIdxMap(), spec.Reverse, neededColumns, true /* isCheck */, flowCtx.EvalCtx.Mon, &tr.alloc, execinfra.ScanVisibilityPublic, spec.LockingStrength, spec.LockingWaitPolicy, nil /* systemColumns */, nil, /* virtualColumn */ diff --git a/pkg/sql/rowexec/zigzagjoiner.go b/pkg/sql/rowexec/zigzagjoiner.go index 6faad7349386..9655b1f02e72 100644 --- a/pkg/sql/rowexec/zigzagjoiner.go +++ b/pkg/sql/rowexec/zigzagjoiner.go @@ -284,9 +284,9 @@ func newZigzagJoiner( z := &zigzagJoiner{} // TODO(ajwerner): Utilize a cached copy of these tables. - tables := make([]tabledesc.Immutable, len(spec.Tables)) + tables := make([]catalog.TableDescriptor, len(spec.Tables)) for i := range spec.Tables { - tables[i] = tabledesc.MakeImmutable(spec.Tables[i]) + tables[i] = tabledesc.NewImmutable(spec.Tables[i]) } leftColumnTypes := tables[0].ColumnTypes() rightColumnTypes := tables[1].ColumnTypes() @@ -374,7 +374,7 @@ func (z *zigzagJoiner) Start(ctx context.Context) context.Context { type zigzagJoinerInfo struct { fetcher row.Fetcher alloc *rowenc.DatumAlloc - table *tabledesc.Immutable + table catalog.TableDescriptor index *descpb.IndexDescriptor indexTypes []*types.T indexDirs []descpb.IndexDescriptor_Direction @@ -411,13 +411,13 @@ func (z *zigzagJoiner) setupInfo( spec *execinfrapb.ZigzagJoinerSpec, side int, colOffset int, - tables []tabledesc.Immutable, + tables []catalog.TableDescriptor, ) error { z.side = side info := z.infos[side] info.alloc = &rowenc.DatumAlloc{} - info.table = &tables[side] + info.table = tables[side] info.eqColumns = spec.EqColumns[side].Columns indexOrdinal := spec.IndexOrdinals[side] info.index = info.table.ActiveIndexes()[indexOrdinal].IndexDesc() diff --git a/pkg/sql/scan.go b/pkg/sql/scan.go index b5fb6cad52a9..0dd5607d3636 100644 --- a/pkg/sql/scan.go +++ b/pkg/sql/scan.go @@ -19,7 +19,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/opt/exec" @@ -46,7 +45,7 @@ type scanNode struct { // Enforce this using NoCopy. _ util.NoCopy - desc *tabledesc.Immutable + desc catalog.TableDescriptor index *descpb.IndexDescriptor // Set if an index was explicitly specified. @@ -204,7 +203,7 @@ func (n *scanNode) limitHint() int64 { func (n *scanNode) initTable( ctx context.Context, p *planner, - desc *tabledesc.Immutable, + desc catalog.TableDescriptor, indexFlags *tree.IndexFlags, colCfg scanColumnsConfig, ) error { @@ -265,7 +264,7 @@ func findReadableColumnByID( // initColsForScan initializes cols according to desc and colCfg. func initColsForScan( - desc *tabledesc.Immutable, colCfg scanColumnsConfig, + desc catalog.TableDescriptor, colCfg scanColumnsConfig, ) (cols []*descpb.ColumnDescriptor, err error) { if colCfg.wantedColumns == nil { return nil, errors.AssertionFailedf("unexpectedly wantedColumns is nil") diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index 52be744baaf9..d355bbf88c13 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -342,7 +342,7 @@ func (sc *SchemaChanger) backfillQueryIntoTable( // this writing) this code path is only used for standalone CREATE // TABLE AS statements, which cannot be traced. func (sc *SchemaChanger) maybeBackfillCreateTableAs( - ctx context.Context, table *tabledesc.Immutable, + ctx context.Context, table catalog.TableDescriptor, ) error { if !(table.Adding() && table.IsAs()) { return nil @@ -353,7 +353,7 @@ func (sc *SchemaChanger) maybeBackfillCreateTableAs( } func (sc *SchemaChanger) maybeBackfillMaterializedView( - ctx context.Context, table *tabledesc.Immutable, + ctx context.Context, table catalog.TableDescriptor, ) error { if !(table.Adding() && table.MaterializedView()) { return nil @@ -365,7 +365,7 @@ func (sc *SchemaChanger) maybeBackfillMaterializedView( // maybe make a table PUBLIC if it's in the ADD state. func (sc *SchemaChanger) maybeMakeAddTablePublic( - ctx context.Context, table *tabledesc.Immutable, + ctx context.Context, table catalog.TableDescriptor, ) error { if !table.Adding() { return nil @@ -608,7 +608,7 @@ func (sc *SchemaChanger) exec(ctx context.Context) error { return nil } - tableDesc, ok := desc.(*tabledesc.Immutable) + tableDesc, ok := desc.(catalog.TableDescriptor) if !ok { // If our descriptor is not a table, then just drain leases. if err := waitToUpdateLeases(false /* refreshStats */); err != nil { @@ -1609,7 +1609,7 @@ func (sc *SchemaChanger) maybeReverseMutations(ctx context.Context, causingError return err } - tableDesc := scTable.ImmutableCopy().(*tabledesc.Immutable) + tableDesc := scTable.ImmutableCopy().(catalog.TableDescriptor) // Mark the schema change job as failed and create a rollback job. err = sc.updateJobForRollback(ctx, txn, tableDesc) if err != nil { @@ -2408,7 +2408,7 @@ func (sc *SchemaChanger) queueCleanupJobs( // DeleteTableDescAndZoneConfig removes a table's descriptor and zone config from the KV database. func DeleteTableDescAndZoneConfig( - ctx context.Context, db *kv.DB, codec keys.SQLCodec, tableDesc *tabledesc.Immutable, + ctx context.Context, db *kv.DB, codec keys.SQLCodec, tableDesc catalog.TableDescriptor, ) error { log.Infof(ctx, "removing table descriptor and zone config for table %d", tableDesc.GetID()) return db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { diff --git a/pkg/sql/schema_changer_test.go b/pkg/sql/schema_changer_test.go index d68bbf5713d7..405b8997d6f8 100644 --- a/pkg/sql/schema_changer_test.go +++ b/pkg/sql/schema_changer_test.go @@ -34,12 +34,12 @@ 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/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/gcjob" "github.com/cockroachdb/cockroach/pkg/sql/sqltestutils" @@ -163,7 +163,7 @@ INSERT INTO t.test VALUES ('a', 'b'), ('c', 'd'); for _, direction := range []descpb.DescriptorMutation_Direction{ descpb.DescriptorMutation_ADD, descpb.DescriptorMutation_DROP, } { - tableDesc.TableDesc().Mutations[0].Direction = direction + tableDesc.GetMutations()[0].Direction = direction expectedVersion++ if err := kvDB.Put( ctx, @@ -5200,7 +5200,7 @@ func TestIndexBackfillValidation(t *testing.T) { const maxValue = 1000 backfillCount := int64(0) var db *kv.DB - var tableDesc *tabledesc.Immutable + var tableDesc catalog.TableDescriptor params.Knobs = base.TestingKnobs{ SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ BackfillChunkSize: maxValue / 5, @@ -5271,7 +5271,7 @@ func TestInvertedIndexBackfillValidation(t *testing.T) { const maxValue = 1000 backfillCount := int64(0) var db *kv.DB - var tableDesc *tabledesc.Immutable + var tableDesc catalog.TableDescriptor params.Knobs = base.TestingKnobs{ SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ BackfillChunkSize: maxValue / 5, diff --git a/pkg/sql/scrub.go b/pkg/sql/scrub.go index 76f633bc11f7..4e80b9a8060e 100644 --- a/pkg/sql/scrub.go +++ b/pkg/sql/scrub.go @@ -15,11 +15,11 @@ import ( "fmt" "strings" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/rowcontainer" @@ -193,7 +193,7 @@ func (n *scrubNode) startScrubDatabase(ctx context.Context, p *planner, name *tr if err != nil { return err } - tableDesc := objDesc.(*tabledesc.Immutable) + tableDesc := objDesc.(catalog.TableDescriptor) // Skip non-tables and don't throw an error if we encounter one. if !tableDesc.IsTable() { continue @@ -206,7 +206,7 @@ func (n *scrubNode) startScrubDatabase(ctx context.Context, p *planner, name *tr } func (n *scrubNode) startScrubTable( - ctx context.Context, p *planner, tableDesc *tabledesc.Immutable, tableName *tree.TableName, + ctx context.Context, p *planner, tableDesc catalog.TableDescriptor, tableName *tree.TableName, ) error { ts, hasTS, err := p.getTimestamp(ctx, n.n.AsOf) if err != nil { @@ -284,7 +284,7 @@ func (n *scrubNode) startScrubTable( // getPrimaryColIdxs returns a list of the primary index columns and // their corresponding index in the columns list. func getPrimaryColIdxs( - tableDesc *tabledesc.Immutable, columns []*descpb.ColumnDescriptor, + tableDesc catalog.TableDescriptor, columns []*descpb.ColumnDescriptor, ) (primaryColIdxs []int, err error) { for i := 0; i < tableDesc.GetPrimaryIndex().NumColumns(); i++ { colID := tableDesc.GetPrimaryIndex().GetColumnID(i) @@ -344,7 +344,7 @@ func pairwiseOp(left []string, right []string, op string) []string { // createPhysicalCheckOperations will return the physicalCheckOperation // for all indexes on a table. func createPhysicalCheckOperations( - tableDesc *tabledesc.Immutable, tableName *tree.TableName, + tableDesc catalog.TableDescriptor, tableName *tree.TableName, ) (checks []checkOperation) { for _, idx := range tableDesc.ActiveIndexes() { checks = append(checks, newPhysicalCheckOperation(tableName, tableDesc, idx.IndexDesc())) @@ -360,7 +360,7 @@ func createPhysicalCheckOperations( // first invalid index. func createIndexCheckOperations( indexNames tree.NameList, - tableDesc *tabledesc.Immutable, + tableDesc catalog.TableDescriptor, tableName *tree.TableName, asOf hlc.Timestamp, ) (results []checkOperation, err error) { @@ -418,7 +418,7 @@ func createConstraintCheckOperations( ctx context.Context, p *planner, constraintNames tree.NameList, - tableDesc *tabledesc.Immutable, + tableDesc catalog.TableDescriptor, tableName *tree.TableName, asOf hlc.Timestamp, ) (results []checkOperation, err error) { diff --git a/pkg/sql/scrub_constraint.go b/pkg/sql/scrub_constraint.go index 2df7b248f44e..d605d6a21ff7 100644 --- a/pkg/sql/scrub_constraint.go +++ b/pkg/sql/scrub_constraint.go @@ -15,6 +15,7 @@ import ( "go/constant" "time" + "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/parser" @@ -27,7 +28,7 @@ import ( // CHECK constraint on a table. type sqlCheckConstraintCheckOperation struct { tableName *tree.TableName - tableDesc *tabledesc.Immutable + tableDesc catalog.TableDescriptor checkDesc *descpb.TableDescriptor_CheckConstraint asOf hlc.Timestamp @@ -51,7 +52,7 @@ type sqlCheckConstraintCheckRun struct { func newSQLCheckConstraintCheckOperation( tableName *tree.TableName, - tableDesc *tabledesc.Immutable, + tableDesc catalog.TableDescriptor, checkDesc *descpb.TableDescriptor_CheckConstraint, asOf hlc.Timestamp, ) *sqlCheckConstraintCheckOperation { diff --git a/pkg/sql/scrub_fk.go b/pkg/sql/scrub_fk.go index d28636826320..ff4a71e4d62e 100644 --- a/pkg/sql/scrub_fk.go +++ b/pkg/sql/scrub_fk.go @@ -25,8 +25,8 @@ import ( // sqlForeignKeyCheckOperation is a check on an indexes physical data. type sqlForeignKeyCheckOperation struct { tableName *tree.TableName - tableDesc *tabledesc.Immutable - referencedTableDesc *tabledesc.Immutable + tableDesc catalog.TableDescriptor + referencedTableDesc catalog.TableDescriptor constraint *descpb.ConstraintDetail asOf hlc.Timestamp @@ -45,7 +45,7 @@ type sqlForeignKeyConstraintCheckRun struct { func newSQLForeignKeyCheckOperation( tableName *tree.TableName, - tableDesc *tabledesc.Immutable, + tableDesc catalog.TableDescriptor, constraint descpb.ConstraintDetail, asOf hlc.Timestamp, ) *sqlForeignKeyCheckOperation { diff --git a/pkg/sql/scrub_index.go b/pkg/sql/scrub_index.go index 8e0f95817998..8063288d65b3 100644 --- a/pkg/sql/scrub_index.go +++ b/pkg/sql/scrub_index.go @@ -18,7 +18,6 @@ import ( "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/scrub" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -33,7 +32,7 @@ import ( // that refers to a primary index key that cannot be found. type indexCheckOperation struct { tableName *tree.TableName - tableDesc *tabledesc.Immutable + tableDesc catalog.TableDescriptor indexDesc *descpb.IndexDescriptor asOf hlc.Timestamp @@ -58,7 +57,7 @@ type indexCheckRun struct { func newIndexCheckOperation( tableName *tree.TableName, - tableDesc *tabledesc.Immutable, + tableDesc catalog.TableDescriptor, indexDesc *descpb.IndexDescriptor, asOf hlc.Timestamp, ) *indexCheckOperation { diff --git a/pkg/sql/scrub_physical.go b/pkg/sql/scrub_physical.go index c7aa498a0607..6fb263a32244 100644 --- a/pkg/sql/scrub_physical.go +++ b/pkg/sql/scrub_physical.go @@ -16,7 +16,6 @@ import ( "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/rowcontainer" "github.com/cockroachdb/cockroach/pkg/sql/rowexec" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -29,7 +28,7 @@ var _ checkOperation = &physicalCheckOperation{} // physicalCheckOperation is a check on an indexes physical data. type physicalCheckOperation struct { tableName *tree.TableName - tableDesc *tabledesc.Immutable + tableDesc catalog.TableDescriptor indexDesc *descpb.IndexDescriptor // columns is a list of the columns returned in the query result @@ -51,7 +50,7 @@ type physicalCheckRun struct { } func newPhysicalCheckOperation( - tableName *tree.TableName, tableDesc *tabledesc.Immutable, indexDesc *descpb.IndexDescriptor, + tableName *tree.TableName, tableDesc catalog.TableDescriptor, indexDesc *descpb.IndexDescriptor, ) *physicalCheckOperation { return &physicalCheckOperation{ tableName: tableName, diff --git a/pkg/sql/sequence.go b/pkg/sql/sequence.go index f45b34894af0..15c036ef7e51 100644 --- a/pkg/sql/sequence.go +++ b/pkg/sql/sequence.go @@ -111,7 +111,7 @@ func (p *planner) IncrementSequence(ctx context.Context, seqName *tree.TableName return val, nil } -func boundsExceededError(descriptor *tabledesc.Immutable) error { +func boundsExceededError(descriptor catalog.TableDescriptor) error { seqOpts := descriptor.GetSequenceOpts() isAscending := seqOpts.Increment > 0 @@ -212,7 +212,7 @@ func MakeSequenceKeyVal( // GetSequenceValue returns the current value of the sequence. func (p *planner) GetSequenceValue( - ctx context.Context, codec keys.SQLCodec, desc *tabledesc.Immutable, + ctx context.Context, codec keys.SQLCodec, desc catalog.TableDescriptor, ) (int64, error) { if desc.GetSequenceOpts() == nil { return 0, errors.New("descriptor is not a sequence") diff --git a/pkg/sql/sequence_select.go b/pkg/sql/sequence_select.go index 841ecdc7c192..c9187c4086d1 100644 --- a/pkg/sql/sequence_select.go +++ b/pkg/sql/sequence_select.go @@ -13,7 +13,7 @@ package sql import ( "context" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/errors" ) @@ -21,7 +21,7 @@ import ( type sequenceSelectNode struct { optColumnsSlot - desc *tabledesc.Immutable + desc catalog.TableDescriptor val int64 done bool @@ -29,7 +29,7 @@ type sequenceSelectNode struct { var _ planNode = &sequenceSelectNode{} -func (p *planner) SequenceSelectNode(desc *tabledesc.Immutable) (planNode, error) { +func (p *planner) SequenceSelectNode(desc catalog.TableDescriptor) (planNode, error) { if desc.GetSequenceOpts() == nil { return nil, errors.New("descriptor is not a sequence") } diff --git a/pkg/sql/sequence_test.go b/pkg/sql/sequence_test.go index c9a040572d94..25f84ebe329e 100644 --- a/pkg/sql/sequence_test.go +++ b/pkg/sql/sequence_test.go @@ -18,9 +18,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -164,7 +164,7 @@ func assertColumnOwnsSequences( ) { tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, dbName, tbName) col := tableDesc.GetPublicColumns()[colIdx] - var seqDescs []*tabledesc.Immutable + var seqDescs []catalog.TableDescriptor for _, seqName := range seqNames { seqDescs = append( seqDescs, diff --git a/pkg/sql/show_create.go b/pkg/sql/show_create.go index 2badb12be6bf..111cfb7b2b08 100644 --- a/pkg/sql/show_create.go +++ b/pkg/sql/show_create.go @@ -18,7 +18,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/catformat" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" - "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/sql/sessiondata" @@ -235,7 +234,7 @@ func (p *planner) ShowCreate( ctx context.Context, dbPrefix string, allDescs []descpb.Descriptor, - desc *tabledesc.Immutable, + desc catalog.TableDescriptor, displayOptions ShowCreateDisplayOptions, ) (string, error) { var stmt string diff --git a/pkg/sql/show_fingerprints.go b/pkg/sql/show_fingerprints.go index 24803786fa82..e69110af0de7 100644 --- a/pkg/sql/show_fingerprints.go +++ b/pkg/sql/show_fingerprints.go @@ -16,8 +16,8 @@ import ( "strings" "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/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" @@ -28,7 +28,7 @@ import ( type showFingerprintsNode struct { optColumnsSlot - tableDesc *tabledesc.Immutable + tableDesc catalog.TableDescriptor indexes []*descpb.IndexDescriptor run showFingerprintsRun diff --git a/pkg/sql/show_stats.go b/pkg/sql/show_stats.go index a42b32dcc772..6361738c2e54 100644 --- a/pkg/sql/show_stats.go +++ b/pkg/sql/show_stats.go @@ -14,9 +14,9 @@ import ( "context" encjson "encoding/json" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/stats" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -189,7 +189,7 @@ func (p *planner) ShowTableStats(ctx context.Context, n *tree.ShowTableStats) (p }, nil } -func statColumnString(desc *tabledesc.Immutable, colID tree.Datum) string { +func statColumnString(desc catalog.TableDescriptor, colID tree.Datum) string { id := descpb.ColumnID(*colID.(*tree.DInt)) colDesc, err := desc.FindColumnByID(id) if err != nil { diff --git a/pkg/sql/span/BUILD.bazel b/pkg/sql/span/BUILD.bazel index 7ab3ebd4a2ea..7119a2607433 100644 --- a/pkg/sql/span/BUILD.bazel +++ b/pkg/sql/span/BUILD.bazel @@ -10,7 +10,6 @@ go_library( "//pkg/roachpb", "//pkg/sql/catalog", "//pkg/sql/catalog/descpb", - "//pkg/sql/catalog/tabledesc", "//pkg/sql/inverted", "//pkg/sql/opt/constraint", "//pkg/sql/opt/exec", diff --git a/pkg/sql/span/span_builder.go b/pkg/sql/span/span_builder.go index 8700a94918d6..ad9649cf514c 100644 --- a/pkg/sql/span/span_builder.go +++ b/pkg/sql/span/span_builder.go @@ -17,7 +17,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "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/inverted" "github.com/cockroachdb/cockroach/pkg/sql/opt/constraint" "github.com/cockroachdb/cockroach/pkg/sql/opt/exec" @@ -34,7 +33,7 @@ import ( type Builder struct { evalCtx *tree.EvalContext codec keys.SQLCodec - table *tabledesc.Immutable + table catalog.TableDescriptor index *descpb.IndexDescriptor indexColTypes []*types.T indexColDirs []descpb.IndexDescriptor_Direction @@ -60,7 +59,7 @@ var _ = (*Builder).UnsetNeededFamilies func MakeBuilder( evalCtx *tree.EvalContext, codec keys.SQLCodec, - table *tabledesc.Immutable, + table catalog.TableDescriptor, index *descpb.IndexDescriptor, ) *Builder { s := &Builder{ diff --git a/pkg/sql/split.go b/pkg/sql/split.go index 7096d87f0e36..04ec4c4cd660 100644 --- a/pkg/sql/split.go +++ b/pkg/sql/split.go @@ -17,7 +17,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -29,7 +28,7 @@ import ( type splitNode struct { optColumnsSlot - tableDesc *tabledesc.Immutable + tableDesc catalog.TableDescriptor index *descpb.IndexDescriptor rows planNode run splitRun diff --git a/pkg/sql/table_test.go b/pkg/sql/table_test.go index dda4bcc27cfb..0f6c726c8717 100644 --- a/pkg/sql/table_test.go +++ b/pkg/sql/table_test.go @@ -447,19 +447,19 @@ func TestSerializedUDTsInTableDescriptor(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - getDefault := func(desc *tabledesc.Immutable) string { + getDefault := func(desc catalog.TableDescriptor) string { return *desc.GetPublicColumns()[0].DefaultExpr } - getComputed := func(desc *tabledesc.Immutable) string { + getComputed := func(desc catalog.TableDescriptor) string { return *desc.GetPublicColumns()[0].ComputeExpr } - getCheck := func(desc *tabledesc.Immutable) string { + getCheck := func(desc catalog.TableDescriptor) string { return desc.GetChecks()[0].Expr } testdata := []struct { colSQL string expectedExpr string - getExpr func(desc *tabledesc.Immutable) string + getExpr func(desc catalog.TableDescriptor) string }{ // Test a simple UDT as the default value. { diff --git a/pkg/sql/tests/BUILD.bazel b/pkg/sql/tests/BUILD.bazel index cda3c774a423..8819f72886ca 100644 --- a/pkg/sql/tests/BUILD.bazel +++ b/pkg/sql/tests/BUILD.bazel @@ -61,11 +61,11 @@ go_test( "//pkg/security/securitytest", "//pkg/server", "//pkg/sql", + "//pkg/sql/catalog", "//pkg/sql/catalog/bootstrap", "//pkg/sql/catalog/catalogkv", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/systemschema", - "//pkg/sql/catalog/tabledesc", "//pkg/sql/parser", "//pkg/sql/pgwire/pgcode", "//pkg/sql/privilege", diff --git a/pkg/sql/tests/hash_sharded_test.go b/pkg/sql/tests/hash_sharded_test.go index 588edd51347d..6330d7a74f5f 100644 --- a/pkg/sql/tests/hash_sharded_test.go +++ b/pkg/sql/tests/hash_sharded_test.go @@ -16,9 +16,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" + "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" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -28,7 +28,7 @@ import ( // getShardColumnID fetches the id of the shard column associated with the given sharded // index. func getShardColumnID( - t *testing.T, tableDesc *tabledesc.Immutable, shardedIndexName string, + t *testing.T, tableDesc catalog.TableDescriptor, shardedIndexName string, ) descpb.ColumnID { idx, err := tableDesc.FindIndexWithName(shardedIndexName) if err != nil { @@ -47,7 +47,7 @@ func getShardColumnID( // 2. A hidden check constraint was created on the aforementioned shard column. // 3. The first column in the index set is the aforementioned shard column. func verifyTableDescriptorState( - t *testing.T, tableDesc *tabledesc.Immutable, shardedIndexName string, + t *testing.T, tableDesc catalog.TableDescriptor, shardedIndexName string, ) { idx, err := tableDesc.FindIndexWithName(shardedIndexName) if err != nil { diff --git a/pkg/sql/tests/system_table_test.go b/pkg/sql/tests/system_table_test.go index b4bbbdfad048..fe110ada1484 100644 --- a/pkg/sql/tests/system_table_test.go +++ b/pkg/sql/tests/system_table_test.go @@ -22,10 +22,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/bootstrap" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -162,7 +162,7 @@ func TestSystemTableLiterals(t *testing.T) { type testcase struct { id descpb.ID schema string - pkg *tabledesc.Immutable + pkg catalog.TableDescriptor } for _, test := range []testcase{ diff --git a/pkg/sql/truncate.go b/pkg/sql/truncate.go index 41b6ef3d701e..8c7321e982b2 100644 --- a/pkg/sql/truncate.go +++ b/pkg/sql/truncate.go @@ -315,7 +315,11 @@ func (p *planner) truncateTable( // can even eliminate the need to use a transaction for each chunk at a later // stage if it proves inefficient). func ClearTableDataInChunks( - ctx context.Context, db *kv.DB, codec keys.SQLCodec, tableDesc *tabledesc.Immutable, traceKV bool, + ctx context.Context, + db *kv.DB, + codec keys.SQLCodec, + tableDesc catalog.TableDescriptor, + traceKV bool, ) error { const chunkSize = row.TableTruncateChunkSize var resume roachpb.Span diff --git a/pkg/sql/unsplit.go b/pkg/sql/unsplit.go index d4e94aac072e..c8faf3fd54ed 100644 --- a/pkg/sql/unsplit.go +++ b/pkg/sql/unsplit.go @@ -17,7 +17,6 @@ import ( "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" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/errors" @@ -26,7 +25,7 @@ import ( type unsplitNode struct { optColumnsSlot - tableDesc *tabledesc.Immutable + tableDesc catalog.TableDescriptor index *descpb.IndexDescriptor run unsplitRun rows planNode diff --git a/pkg/sql/views.go b/pkg/sql/views.go index 3685a76ef0ed..6d479cb69568 100644 --- a/pkg/sql/views.go +++ b/pkg/sql/views.go @@ -16,7 +16,6 @@ import ( "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/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -28,7 +27,7 @@ import ( type planDependencyInfo struct { // desc is a reference to the descriptor for the table being // depended on. - desc *tabledesc.Immutable + desc catalog.TableDescriptor // deps is the list of ways in which the current plan depends on // that table. There can be more than one entries when the same // table is used in different places. The entries can also be diff --git a/pkg/sql/virtual_schema.go b/pkg/sql/virtual_schema.go index 3d5bbcdf7da9..e2f652199677 100644 --- a/pkg/sql/virtual_schema.go +++ b/pkg/sql/virtual_schema.go @@ -376,7 +376,7 @@ func (v *virtualSchemaEntry) GetObjectByName( type virtualDefEntry struct { virtualDef virtualSchemaDef - desc *tabledesc.Immutable + desc catalog.TableDescriptor comment string validWithNoDatabaseContext bool } @@ -737,7 +737,7 @@ func (vs *VirtualSchemaHolder) getVirtualTableEntryByID(id descpb.ID) (*virtualD // VirtualTabler is used to fetch descriptors for virtual tables and databases. type VirtualTabler interface { - getVirtualTableDesc(tn *tree.TableName) (*tabledesc.Immutable, error) + getVirtualTableDesc(tn *tree.TableName) (catalog.TableDescriptor, error) getVirtualSchemaEntry(name string) (*virtualSchemaEntry, bool) getVirtualTableEntry(tn *tree.TableName) (*virtualDefEntry, error) getVirtualTableEntryByID(id descpb.ID) (*virtualDefEntry, error) @@ -750,7 +750,7 @@ type VirtualTabler interface { // getVirtualTableDesc is part of the VirtualTabler interface. func (vs *VirtualSchemaHolder) getVirtualTableDesc( tn *tree.TableName, -) (*tabledesc.Immutable, error) { +) (catalog.TableDescriptor, error) { t, err := vs.getVirtualTableEntry(tn) if err != nil || t == nil { return nil, err diff --git a/pkg/sql/virtual_table.go b/pkg/sql/virtual_table.go index 52ed3e1c0957..6123d9e70bc4 100644 --- a/pkg/sql/virtual_table.go +++ b/pkg/sql/virtual_table.go @@ -14,10 +14,10 @@ import ( "context" "sync" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/opt/constraint" "github.com/cockroachdb/cockroach/pkg/sql/opt/exec" "github.com/cockroachdb/cockroach/pkg/sql/rowcontainer" @@ -203,7 +203,7 @@ type vTableLookupJoinNode struct { dbName string db *dbdesc.Immutable - table *tabledesc.Immutable + table catalog.TableDescriptor index *descpb.IndexDescriptor // eqCol is the single equality column ordinal into the lookup table. Virtual // indexes only support a single indexed column currently. diff --git a/pkg/sql/zone_config.go b/pkg/sql/zone_config.go index db8aa866cba5..4f040fbf86b7 100644 --- a/pkg/sql/zone_config.go +++ b/pkg/sql/zone_config.go @@ -264,7 +264,7 @@ func (p *planner) resolveTableForZone( res = mutRes } } else if zs.TargetsTable() { - var immutRes *tabledesc.Immutable + var immutRes catalog.TableDescriptor p.runWithOptions(resolveFlags{skipCache: true}, func() { flags := tree.ObjectLookupFlagsWithRequiredTableKind(tree.ResolveAnyTableKind) flags.IncludeOffline = true