From 52705f226fadfcc26b8a534ed0852a9d28fc9935 Mon Sep 17 00:00:00 2001 From: Marius Posta Date: Wed, 3 Feb 2021 15:17:04 -0500 Subject: [PATCH] sql: remove deprecated GetPublicColumns method from TableDescriptor Previously, this method would be called on a table descriptor interface to return a []descpb.ColumnDescriptor value containing all public columns. This patch removes these calls, along with the method definition, in favour of new methods which use the catalog.Column interface type instead. Fixes #59805. Release note: None --- pkg/ccl/backupccl/backup_test.go | 38 ++++----- pkg/ccl/changefeedccl/encoder.go | 18 ++--- pkg/ccl/changefeedccl/rowfetcher_cache.go | 28 ++++--- .../schemafeed/table_event_filter.go | 4 +- pkg/server/BUILD.bazel | 2 +- pkg/server/settingsworker.go | 10 +-- pkg/sql/backfill/backfill.go | 81 ++++++++++--------- pkg/sql/catalog/catalogkeys/keys.go | 4 +- pkg/sql/catalog/descriptor.go | 1 - pkg/sql/catalog/descs/collection_test.go | 4 +- .../hydratedtables/hydratedcache_test.go | 2 +- pkg/sql/catalog/tabledesc/structured.go | 4 +- pkg/sql/catalog/tabledesc/table_desc.go | 5 -- pkg/sql/check.go | 2 +- pkg/sql/crdb_internal.go | 19 +++-- pkg/sql/create_stats.go | 10 +-- pkg/sql/distsql_physical_planner.go | 6 +- pkg/sql/information_schema.go | 6 +- pkg/sql/opt_catalog.go | 27 +++---- pkg/sql/opt_exec_factory.go | 27 +++++-- pkg/sql/pg_catalog.go | 2 +- pkg/sql/physicalplan/aggregator_funcs_test.go | 15 ++-- pkg/sql/rename_database.go | 14 ++-- pkg/sql/row/errors.go | 7 +- pkg/sql/row/fetcher_mvcc_test.go | 11 +-- pkg/sql/row/fetcher_test.go | 5 +- pkg/sql/row/row_converter.go | 21 +++-- pkg/sql/rowenc/index_encoding_test.go | 6 +- pkg/sql/rowexec/inverted_joiner.go | 2 +- pkg/sql/rowexec/scrub_tablereader.go | 6 +- pkg/sql/rowexec/zigzagjoiner.go | 6 +- pkg/sql/scan.go | 7 +- pkg/sql/schema_changer_test.go | 12 +-- pkg/sql/scrub_constraint.go | 6 +- pkg/sql/scrub_fk.go | 6 +- pkg/sql/scrub_index.go | 13 ++- pkg/sql/scrub_physical.go | 6 +- pkg/sql/scrub_test.go | 52 ++++++------ pkg/sql/sequence.go | 6 +- pkg/sql/sequence_test.go | 13 +-- pkg/sql/show_create_clauses.go | 4 +- pkg/sql/show_fingerprints.go | 11 ++- pkg/sql/span/span_builder.go | 8 +- pkg/sql/table_ref_test.go | 18 ++--- pkg/sql/table_test.go | 4 +- pkg/sql/temporary_schema.go | 6 +- pkg/sql/virtual_schema.go | 9 +-- pkg/sqlmigrations/migrations_test.go | 12 +-- 48 files changed, 304 insertions(+), 282 deletions(-) diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index 26138e6cd8dc..3e7a63b32678 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -5216,13 +5216,13 @@ func TestBackupRestoreSequenceOwnership(t *testing.T) { require.Equal(t, tableDesc.GetID(), seqDesc.GetSequenceOpts().SequenceOwner.OwnerTableID, "unexpected table is sequence owner after restore", ) - require.Equal(t, tableDesc.GetPublicColumns()[0].ID, seqDesc.GetSequenceOpts().SequenceOwner.OwnerColumnID, + require.Equal(t, tableDesc.PublicColumnsNew()[0].GetID(), seqDesc.GetSequenceOpts().SequenceOwner.OwnerColumnID, "unexpected column is sequence owner after restore", ) - require.Equal(t, 1, len(tableDesc.GetPublicColumns()[0].OwnsSequenceIds), + require.Equal(t, 1, tableDesc.PublicColumnsNew()[0].NumOwnsSequences(), "unexpected number of sequences owned by d.t after restore", ) - require.Equal(t, seqDesc.GetID(), tableDesc.GetPublicColumns()[0].OwnsSequenceIds[0], + require.Equal(t, seqDesc.GetID(), tableDesc.PublicColumnsNew()[0].GetOwnsSequenceID(0), "unexpected ID of sequence owned by table d.t after restore", ) }) @@ -5269,7 +5269,7 @@ func TestBackupRestoreSequenceOwnership(t *testing.T) { tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "d", "t") - require.Equal(t, 0, len(tableDesc.GetPublicColumns()[0].OwnsSequenceIds), + require.Equal(t, 0, tableDesc.PublicColumnsNew()[0].NumOwnsSequences(), "expected restored table to own 0 sequences", ) @@ -5300,13 +5300,13 @@ func TestBackupRestoreSequenceOwnership(t *testing.T) { require.Equal(t, tableDesc.GetID(), seqDesc.GetSequenceOpts().SequenceOwner.OwnerTableID, "unexpected table is sequence owner after restore", ) - require.Equal(t, tableDesc.GetPublicColumns()[0].ID, seqDesc.GetSequenceOpts().SequenceOwner.OwnerColumnID, + require.Equal(t, tableDesc.PublicColumnsNew()[0].GetID(), seqDesc.GetSequenceOpts().SequenceOwner.OwnerColumnID, "unexpected column is sequence owner after restore", ) - require.Equal(t, 1, len(tableDesc.GetPublicColumns()[0].OwnsSequenceIds), + require.Equal(t, 1, tableDesc.PublicColumnsNew()[0].NumOwnsSequences(), "unexpected number of sequences owned by d.t after restore", ) - require.Equal(t, seqDesc.GetID(), tableDesc.GetPublicColumns()[0].OwnsSequenceIds[0], + require.Equal(t, seqDesc.GetID(), tableDesc.PublicColumnsNew()[0].GetOwnsSequenceID(0), "unexpected ID of sequence owned by table d.t after restore", ) }) @@ -5345,7 +5345,7 @@ func TestBackupRestoreSequenceOwnership(t *testing.T) { newDB.Exec(t, `RESTORE DATABASE d2 FROM $1 WITH skip_missing_sequence_owners`, backupLocD2D3) tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "d2", "t") - require.Equal(t, 0, len(tableDesc.GetPublicColumns()[0].OwnsSequenceIds), + require.Equal(t, 0, tableDesc.PublicColumnsNew()[0].NumOwnsSequences(), "expected restored table to own no sequences.", ) @@ -5365,12 +5365,12 @@ func TestBackupRestoreSequenceOwnership(t *testing.T) { require.Equal(t, td.GetID(), sd.GetSequenceOpts().SequenceOwner.OwnerTableID, "unexpected table owner for sequence seq2 after restore", ) - require.Equal(t, td.GetPublicColumns()[0].ID, sd.GetSequenceOpts().SequenceOwner.OwnerColumnID, + require.Equal(t, td.PublicColumnsNew()[0].GetID(), sd.GetSequenceOpts().SequenceOwner.OwnerColumnID, "unexpected column owner for sequence seq2 after restore") - require.Equal(t, 1, len(td.GetPublicColumns()[0].OwnsSequenceIds), + require.Equal(t, 1, td.PublicColumnsNew()[0].NumOwnsSequences(), "unexpected number of sequences owned by d3.t after restore", ) - require.Equal(t, sd.GetID(), td.GetPublicColumns()[0].OwnsSequenceIds[0], + require.Equal(t, sd.GetID(), td.PublicColumnsNew()[0].GetOwnsSequenceID(0), "unexpected ID of sequences owned by d3.t", ) }) @@ -5394,13 +5394,13 @@ func TestBackupRestoreSequenceOwnership(t *testing.T) { require.Equal(t, tableDesc.GetID(), seqDesc.GetSequenceOpts().SequenceOwner.OwnerTableID, "unexpected table is sequence owner after restore", ) - require.Equal(t, tableDesc.GetPublicColumns()[0].ID, seqDesc.GetSequenceOpts().SequenceOwner.OwnerColumnID, + require.Equal(t, tableDesc.PublicColumnsNew()[0].GetID(), seqDesc.GetSequenceOpts().SequenceOwner.OwnerColumnID, "unexpected column is sequence owner after restore", ) - require.Equal(t, 1, len(tableDesc.GetPublicColumns()[0].OwnsSequenceIds), + require.Equal(t, 1, tableDesc.PublicColumnsNew()[0].NumOwnsSequences(), "unexpected number of sequences owned by d.t after restore", ) - require.Equal(t, seqDesc.GetID(), tableDesc.GetPublicColumns()[0].OwnsSequenceIds[0], + require.Equal(t, seqDesc.GetID(), tableDesc.PublicColumnsNew()[0].GetOwnsSequenceID(0), "unexpected ID of sequence owned by table d.t after restore", ) @@ -5419,20 +5419,20 @@ func TestBackupRestoreSequenceOwnership(t *testing.T) { "unexpected table is sequence owner of d3.seq2 after restore", ) - require.Equal(t, td.GetPublicColumns()[0].ID, sd.GetSequenceOpts().SequenceOwner.OwnerColumnID, + require.Equal(t, td.PublicColumnsNew()[0].GetID(), sd.GetSequenceOpts().SequenceOwner.OwnerColumnID, "unexpected column is sequence owner of d2.seq after restore", ) - require.Equal(t, td.GetPublicColumns()[0].ID, sdSeq2.GetSequenceOpts().SequenceOwner.OwnerColumnID, + require.Equal(t, td.PublicColumnsNew()[0].GetID(), sdSeq2.GetSequenceOpts().SequenceOwner.OwnerColumnID, "unexpected column is sequence owner of d3.seq2 after restore", ) - require.Equal(t, 2, len(td.GetPublicColumns()[0].OwnsSequenceIds), + require.Equal(t, 2, td.PublicColumnsNew()[0].NumOwnsSequences(), "unexpected number of sequences owned by d3.t after restore", ) - require.Equal(t, sd.GetID(), td.GetPublicColumns()[0].OwnsSequenceIds[0], + require.Equal(t, sd.GetID(), td.PublicColumnsNew()[0].GetOwnsSequenceID(0), "unexpected ID of sequence owned by table d3.t after restore", ) - require.Equal(t, sdSeq2.GetID(), td.GetPublicColumns()[0].OwnsSequenceIds[1], + require.Equal(t, sdSeq2.GetID(), td.PublicColumnsNew()[0].GetOwnsSequenceID(1), "unexpected ID of sequence owned by table d3.t after restore", ) }) diff --git a/pkg/ccl/changefeedccl/encoder.go b/pkg/ccl/changefeedccl/encoder.go index dbd3d058a3f4..f74fb4ee29be 100644 --- a/pkg/ccl/changefeedccl/encoder.go +++ b/pkg/ccl/changefeedccl/encoder.go @@ -173,16 +173,15 @@ func (e *jsonEncoder) EncodeValue(_ context.Context, row encodeRow) ([]byte, err var after map[string]interface{} if !row.deleted { - columns := row.tableDesc.GetPublicColumns() + columns := row.tableDesc.PublicColumnsNew() after = make(map[string]interface{}, len(columns)) - for i := range columns { - col := &columns[i] + for i, col := range columns { datum := row.datums[i] - if err := datum.EnsureDecoded(col.Type, &e.alloc); err != nil { + if err := datum.EnsureDecoded(col.GetType(), &e.alloc); err != nil { return nil, err } var err error - after[col.Name], err = tree.AsJSON(datum.Datum, time.UTC) + after[col.GetName()], err = tree.AsJSON(datum.Datum, time.UTC) if err != nil { return nil, err } @@ -191,16 +190,15 @@ func (e *jsonEncoder) EncodeValue(_ context.Context, row encodeRow) ([]byte, err var before map[string]interface{} if row.prevDatums != nil && !row.prevDeleted { - columns := row.prevTableDesc.GetPublicColumns() + columns := row.prevTableDesc.PublicColumnsNew() before = make(map[string]interface{}, len(columns)) - for i := range columns { - col := &columns[i] + for i, col := range columns { datum := row.prevDatums[i] - if err := datum.EnsureDecoded(col.Type, &e.alloc); err != nil { + if err := datum.EnsureDecoded(col.GetType(), &e.alloc); err != nil { return nil, err } var err error - before[col.Name], err = tree.AsJSON(datum.Datum, time.UTC) + before[col.GetName()], err = tree.AsJSON(datum.Datum, time.UTC) if err != nil { return nil, err } diff --git a/pkg/ccl/changefeedccl/rowfetcher_cache.go b/pkg/ccl/changefeedccl/rowfetcher_cache.go index 4c74647c0ba7..34912b532359 100644 --- a/pkg/ccl/changefeedccl/rowfetcher_cache.go +++ b/pkg/ccl/changefeedccl/rowfetcher_cache.go @@ -154,12 +154,24 @@ func (c *rowFetcherCache) RowFetcherForTableDesc( // TODO(dan): Allow for decoding a subset of the columns. var colIdxMap catalog.TableColMap var valNeededForCol util.FastIntSet - for colIdx := range tableDesc.GetPublicColumns() { - colIdxMap.Set(tableDesc.GetPublicColumns()[colIdx].ID, colIdx) - valNeededForCol.Add(colIdx) + for _, col := range tableDesc.PublicColumnsNew() { + colIdxMap.Set(col.GetID(), col.Ordinal()) + valNeededForCol.Add(col.Ordinal()) } var rf row.Fetcher + rfArgs := row.FetcherTableArgs{ + Spans: tableDesc.AllIndexSpans(c.codec), + Desc: tableDesc, + Index: tableDesc.GetPrimaryIndex().IndexDesc(), + ColIdxMap: colIdxMap, + IsSecondaryIndex: false, + Cols: make([]descpb.ColumnDescriptor, len(tableDesc.PublicColumnsNew())), + ValNeededForCol: valNeededForCol, + } + for i, col := range tableDesc.PublicColumnsNew() { + rfArgs.Cols[i] = *col.ColumnDesc() + } if err := rf.Init( context.TODO(), c.codec, @@ -169,15 +181,7 @@ func (c *rowFetcherCache) RowFetcherForTableDesc( false, /* isCheck */ &c.a, nil, /* memMonitor */ - row.FetcherTableArgs{ - Spans: tableDesc.AllIndexSpans(c.codec), - Desc: tableDesc, - Index: tableDesc.GetPrimaryIndex().IndexDesc(), - ColIdxMap: colIdxMap, - IsSecondaryIndex: false, - Cols: tableDesc.GetPublicColumns(), - ValNeededForCol: valNeededForCol, - }, + rfArgs, ); err != nil { return nil, err } diff --git a/pkg/ccl/changefeedccl/schemafeed/table_event_filter.go b/pkg/ccl/changefeedccl/schemafeed/table_event_filter.go index b75b31c0af45..547a2178bc51 100644 --- a/pkg/ccl/changefeedccl/schemafeed/table_event_filter.go +++ b/pkg/ccl/changefeedccl/schemafeed/table_event_filter.go @@ -108,12 +108,12 @@ func dropColumnMutationExists(desc catalog.TableDescriptor) bool { func newColumnBackfillComplete(e TableEvent) (res bool) { // TODO(ajwerner): What is the case where the before has a backfill mutation // and the After doesn't? What about other queued mutations? - return len(e.Before.GetPublicColumns()) < len(e.After.GetPublicColumns()) && + return len(e.Before.PublicColumnsNew()) < len(e.After.PublicColumnsNew()) && e.Before.HasColumnBackfillMutation() && !e.After.HasColumnBackfillMutation() } func newColumnNoBackfill(e TableEvent) (res bool) { - return len(e.Before.GetPublicColumns()) < len(e.After.GetPublicColumns()) && + return len(e.Before.PublicColumnsNew()) < len(e.After.PublicColumnsNew()) && !e.Before.HasColumnBackfillMutation() } diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index 7e706fb35d7c..a777a9f4f667 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -92,6 +92,7 @@ go_library( "//pkg/settings", "//pkg/settings/cluster", "//pkg/sql", + "//pkg/sql/catalog", "//pkg/sql/catalog/bootstrap", "//pkg/sql/catalog/catconstants", "//pkg/sql/catalog/colinfo", @@ -111,7 +112,6 @@ go_library( "//pkg/sql/physicalplan", "//pkg/sql/querycache", "//pkg/sql/roleoption", - "//pkg/sql/row", "//pkg/sql/rowenc", "//pkg/sql/sem/tree", "//pkg/sql/sessiondata", diff --git a/pkg/server/settingsworker.go b/pkg/server/settingsworker.go index 81e8e6f8cfae..a86f1d9eb9c8 100644 --- a/pkg/server/settingsworker.go +++ b/pkg/server/settingsworker.go @@ -48,7 +48,7 @@ func processSystemConfigKVs( var k, v, t string // First we need to decode the setting name field from the index key. { - types := []*types.T{tbl.GetPublicColumns()[0].Type} + types := []*types.T{tbl.PublicColumnsNew()[0].GetType()} nameRow := make([]rowenc.EncDatum, 1) _, matches, _, err := rowenc.DecodeIndexKey(codec, tbl, tbl.GetPrimaryIndex().IndexDesc(), types, nameRow, nil, kv.Key) if err != nil { @@ -83,16 +83,16 @@ func processSystemConfigKVs( colID := lastColID + descpb.ColumnID(colIDDiff) lastColID = colID if idx, ok := colIdxMap.Get(colID); ok { - res, bytes, err = rowenc.DecodeTableValue(a, tbl.GetPublicColumns()[idx].Type, bytes) + res, bytes, err = rowenc.DecodeTableValue(a, tbl.PublicColumnsNew()[idx].GetType(), bytes) if err != nil { return err } switch colID { - case tbl.GetPublicColumns()[1].ID: // value + case tbl.PublicColumnsNew()[1].GetID(): // value v = string(tree.MustBeDString(res)) - case tbl.GetPublicColumns()[3].ID: // valueType + case tbl.PublicColumnsNew()[3].GetID(): // valueType t = string(tree.MustBeDString(res)) - case tbl.GetPublicColumns()[2].ID: // lastUpdated + case tbl.PublicColumnsNew()[2].GetID(): // lastUpdated // TODO(dt): we could decode just the len and then seek `bytes` past // it, without allocating/decoding the unused timestamp. default: diff --git a/pkg/sql/backfill/backfill.go b/pkg/sql/backfill/backfill.go index 922cb41a8f5d..3c39f9c6bd2f 100644 --- a/pkg/sql/backfill/backfill.go +++ b/pkg/sql/backfill/backfill.go @@ -119,15 +119,18 @@ func (cb *ColumnBackfiller) init( // We need all the columns. var valNeededForCol util.FastIntSet - valNeededForCol.AddRange(0, len(desc.GetPublicColumns())-1) + valNeededForCol.AddRange(0, len(desc.PublicColumnsNew())-1) tableArgs := row.FetcherTableArgs{ Desc: desc, Index: desc.GetPrimaryIndex().IndexDesc(), ColIdxMap: catalog.ColumnIDToOrdinalMap(desc.PublicColumnsNew()), - Cols: desc.GetPublicColumns(), + Cols: make([]descpb.ColumnDescriptor, len(desc.PublicColumnsNew())), ValNeededForCol: valNeededForCol, } + for i, col := range desc.PublicColumnsNew() { + tableArgs.Cols[i] = *col.ColumnDesc() + } // Create a bound account associated with the column backfiller. if mon == nil { @@ -165,10 +168,14 @@ func (cb *ColumnBackfiller) InitForLocalUse( if err != nil { return err } + colDescs := make([]descpb.ColumnDescriptor, len(desc.PublicColumnsNew())) + for i, col := range desc.PublicColumnsNew() { + colDescs[i] = *col.ColumnDesc() + } computedExprs, _, err := schemaexpr.MakeComputedExprs( ctx, cb.added, - desc.GetPublicColumns(), + colDescs, desc, tree.NewUnqualifiedTableName(tree.Name(desc.GetName())), evalCtx, @@ -193,6 +200,10 @@ func (cb *ColumnBackfiller) InitForDistributedUse( ) error { cb.initCols(desc) evalCtx := flowCtx.NewEvalCtx() + cols := make([]descpb.ColumnDescriptor, len(desc.PublicColumnsNew())) + for i, col := range desc.PublicColumnsNew() { + cols[i] = *col.ColumnDesc() + } var defaultExprs, computedExprs []tree.TypedExpr // Install type metadata in the target descriptors, as well as resolve any // user defined types in the column expressions. @@ -215,7 +226,7 @@ func (cb *ColumnBackfiller) InitForDistributedUse( computedExprs, _, err = schemaexpr.MakeComputedExprs( ctx, cb.added, - desc.GetPublicColumns(), + cols, desc, tree.NewUnqualifiedTableName(tree.Name(desc.GetName())), evalCtx, @@ -257,8 +268,10 @@ func (cb *ColumnBackfiller) RunColumnBackfillChunk( ) (roachpb.Key, error) { // TODO(dan): Tighten up the bound on the requestedCols parameter to // makeRowUpdater. - requestedCols := make([]descpb.ColumnDescriptor, 0, len(tableDesc.GetPublicColumns())+len(cb.added)+len(cb.dropped)) - requestedCols = append(requestedCols, tableDesc.GetPublicColumns()...) + requestedCols := make([]descpb.ColumnDescriptor, 0, len(tableDesc.PublicColumnsNew())+len(cb.added)+len(cb.dropped)) + for _, col := range tableDesc.PublicColumnsNew() { + requestedCols = append(requestedCols, *col.ColumnDesc()) + } requestedCols = append(requestedCols, cb.added...) requestedCols = append(requestedCols, cb.dropped...) ru, err := row.MakeUpdater( @@ -303,9 +316,13 @@ func (cb *ColumnBackfiller) RunColumnBackfillChunk( b := txn.NewBatch() rowLength := 0 iv := &schemaexpr.RowIndexedVarContainer{ - Cols: append(tableDesc.GetPublicColumns(), cb.added...), + Cols: make([]descpb.ColumnDescriptor, 0, len(tableDesc.PublicColumnsNew())+len(cb.added)), Mapping: ru.FetchColIDtoRowIndex, } + for _, col := range tableDesc.PublicColumnsNew() { + iv.Cols = append(iv.Cols, *col.ColumnDesc()) + } + iv.Cols = append(iv.Cols, cb.added...) cb.evalCtx.IVarContainer = iv for i := int64(0); i < chunkSize; i++ { datums, _, _, err := cb.fetcher.NextRowDecoded(ctx) @@ -650,37 +667,27 @@ 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 catalog.TableDescriptor) { - for i := range desc.GetPublicColumns() { - col := &desc.GetPublicColumns()[i] - ib.cols = append(ib.cols, *col) - if col.IsComputed() && col.Virtual { - ib.computedCols = append(ib.computedCols, *col) - } - } - ib.cols = append([]descpb.ColumnDescriptor(nil), desc.GetPublicColumns()...) - - // If there are ongoing mutations, add columns that are being added and in - // the DELETE_AND_WRITE_ONLY state. - if len(desc.GetMutations()) > 0 { - ib.cols = make([]descpb.ColumnDescriptor, 0, len(desc.GetPublicColumns())+len(desc.GetMutations())) - ib.cols = append(ib.cols, desc.GetPublicColumns()...) - for _, m := range desc.GetMutations() { - if column := m.GetColumn(); column != nil && - m.Direction == descpb.DescriptorMutation_ADD && - m.State == descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY { - ib.cols = append(ib.cols, *column) - if column.IsComputed() { - ib.computedCols = append(ib.computedCols, *column) - } else { - ib.addedCols = append(ib.addedCols, *column) - } + ib.cols = make([]descpb.ColumnDescriptor, 0, len(desc.AllColumnsNew())) + for _, column := range desc.AllColumnsNew() { + columnDesc := *column.ColumnDesc() + if column.Public() { + if column.IsComputed() && column.IsVirtual() { + ib.computedCols = append(ib.computedCols, columnDesc) + } + } else if column.Adding() && column.WriteAndDeleteOnly() { + // If there are ongoing mutations, add columns that are being added and in + // the DELETE_AND_WRITE_ONLY state. + if column.IsComputed() { + ib.computedCols = append(ib.computedCols, columnDesc) + } else { + ib.addedCols = append(ib.addedCols, columnDesc) } + } else { + continue } - } - - // Create a map of each column's ID to its ordinal. - for i := range ib.cols { - ib.colIdxMap.Set(ib.cols[i].ID, i) + // Create a map of each column's ID to its ordinal. + ib.colIdxMap.Set(column.GetID(), len(ib.cols)) + ib.cols = append(ib.cols, columnDesc) } } @@ -706,7 +713,7 @@ func (ib *IndexBackfiller) initIndexes(desc catalog.TableDescriptor) util.FastIn isPrimaryIndex := idx.GetEncodingType(desc.GetPrimaryIndexID()) == descpb.PrimaryIndexEncoding if (idxContainsColumn || isPrimaryIndex) && !ib.cols[i].Virtual && - i < len(desc.GetPublicColumns()) { + i < len(desc.PublicColumnsNew()) { valNeededForCol.Add(i) } } diff --git a/pkg/sql/catalog/catalogkeys/keys.go b/pkg/sql/catalog/catalogkeys/keys.go index b998a0984c90..4ce0a1a110cb 100644 --- a/pkg/sql/catalog/catalogkeys/keys.go +++ b/pkg/sql/catalog/catalogkeys/keys.go @@ -281,7 +281,7 @@ func MakeNameMetadataKey( k = encoding.EncodeUvarintAscending(k, uint64(parentSchemaID)) if name != "" { k = encoding.EncodeBytesAscending(k, []byte(name)) - k = keys.MakeFamilyKey(k, uint32(systemschema.NamespaceTable.GetPublicColumns()[3].ID)) + k = keys.MakeFamilyKey(k, uint32(systemschema.NamespaceTable.PublicColumnsNew()[3].GetID())) } return k } @@ -338,7 +338,7 @@ func MakeDeprecatedNameMetadataKey( k = encoding.EncodeUvarintAscending(k, uint64(parentID)) if name != "" { k = encoding.EncodeBytesAscending(k, []byte(name)) - k = keys.MakeFamilyKey(k, uint32(systemschema.DeprecatedNamespaceTable.GetPublicColumns()[2].ID)) + k = keys.MakeFamilyKey(k, uint32(systemschema.DeprecatedNamespaceTable.PublicColumnsNew()[2].GetID())) } return k } diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index cdbf9336809b..c59f3f546d8b 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -211,7 +211,6 @@ type TableDescriptor interface { FindColumnWithID(id descpb.ColumnID) (Column, error) FindColumnWithName(name tree.Name) (Column, error) - GetPublicColumns() []descpb.ColumnDescriptor // deprecated NamesForColumnIDs(ids descpb.ColumnIDs) ([]string, error) ContainsUserDefinedTypes() bool GetNextColumnID() descpb.ColumnID diff --git a/pkg/sql/catalog/descs/collection_test.go b/pkg/sql/catalog/descs/collection_test.go index 45e0539a2f6e..76cba6594d5a 100644 --- a/pkg/sql/catalog/descs/collection_test.go +++ b/pkg/sql/catalog/descs/collection_test.go @@ -359,7 +359,7 @@ func TestSyntheticDescriptorResolution(t *testing.T) { found, desc, err = descriptors.GetImmutableTableByName(ctx, txn, &tn, tree.ObjectLookupFlags{}) require.True(t, found) require.NoError(t, err) - require.Equal(t, "bar", desc.GetPublicColumns()[0].Name) + require.Equal(t, "bar", desc.PublicColumnsNew()[0].GetName()) // Attempting to resolve the table mutably is not allowed. _, _, err = descriptors.GetMutableTableByName(ctx, txn, &tn, tree.ObjectLookupFlags{}) @@ -369,7 +369,7 @@ func TestSyntheticDescriptorResolution(t *testing.T) { desc, err = descriptors.GetImmutableTableByID(ctx, txn, tableID, tree.ObjectLookupFlags{}) require.NoError(t, err) - require.Equal(t, "bar", desc.GetPublicColumns()[0].Name) + require.Equal(t, "bar", desc.PublicColumnsNew()[0].GetName()) // Attempting to resolve the table mutably is not allowed. _, err = descriptors.GetMutableTableByID(ctx, txn, tableID, tree.ObjectLookupFlags{}) diff --git a/pkg/sql/catalog/hydratedtables/hydratedcache_test.go b/pkg/sql/catalog/hydratedtables/hydratedcache_test.go index 77cec3a188a0..fbfb3e092fae 100644 --- a/pkg/sql/catalog/hydratedtables/hydratedcache_test.go +++ b/pkg/sql/catalog/hydratedtables/hydratedcache_test.go @@ -50,7 +50,7 @@ func TestHydratedCache(t *testing.T) { // Show that the cache returned a new pointer and hydrated the UDT // (user-defined type). require.NotEqual(t, tableDescUDT, hydrated) - require.EqualValues(t, hydrated.GetPublicColumns()[0].Type, typ1T) + require.EqualValues(t, hydrated.PublicColumnsNew()[0].GetType(), typ1T) // Try again and ensure we get pointer-for-pointer the same descriptor. res.calls = 0 diff --git a/pkg/sql/catalog/tabledesc/structured.go b/pkg/sql/catalog/tabledesc/structured.go index 5ac73594564e..5040bbeada6f 100644 --- a/pkg/sql/catalog/tabledesc/structured.go +++ b/pkg/sql/catalog/tabledesc/structured.go @@ -3604,11 +3604,11 @@ func (desc *Mutable) IsNew() bool { } // ColumnsSelectors generates Select expressions for cols. -func ColumnsSelectors(cols []descpb.ColumnDescriptor) tree.SelectExprs { +func ColumnsSelectors(cols []catalog.Column) tree.SelectExprs { exprs := make(tree.SelectExprs, len(cols)) colItems := make([]tree.ColumnItem, len(cols)) for i, col := range cols { - colItems[i].ColumnName = tree.Name(col.Name) + colItems[i].ColumnName = col.ColName() exprs[i].Expr = &colItems[i] } return exprs diff --git a/pkg/sql/catalog/tabledesc/table_desc.go b/pkg/sql/catalog/tabledesc/table_desc.go index 73487e09def8..2ad851e7bec8 100644 --- a/pkg/sql/catalog/tabledesc/table_desc.go +++ b/pkg/sql/catalog/tabledesc/table_desc.go @@ -105,11 +105,6 @@ func (desc *wrapper) IsTemporary() bool { return desc.GetTemporary() } -// GetPublicColumns return the public columns in the descriptor. -func (desc *wrapper) GetPublicColumns() []descpb.ColumnDescriptor { - return desc.Columns -} - // ImmutableCopy implements the MutableDescriptor interface. func (desc *Mutable) ImmutableCopy() catalog.Descriptor { // TODO (lucy): Should the immutable descriptor constructors always make a diff --git a/pkg/sql/check.go b/pkg/sql/check.go index 6b18e903a182..18611479e236 100644 --- a/pkg/sql/check.go +++ b/pkg/sql/check.go @@ -48,7 +48,7 @@ func validateCheckExpr( if err != nil { return err } - colSelectors := tabledesc.ColumnsSelectors(tableDesc.Columns) + colSelectors := tabledesc.ColumnsSelectors(tableDesc.PublicColumnsNew()) columns := tree.AsStringWithFlags(&colSelectors, tree.FmtSerializable) queryStr := fmt.Sprintf(`SELECT %s FROM [%d AS t] WHERE NOT (%s) LIMIT 1`, columns, tableDesc.GetID(), exprStr) log.Infof(ctx, "validating check constraint %q with query %q", expr, queryStr) diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index af7fdeffcc42..8e57324d2b49 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -2038,12 +2038,11 @@ CREATE TABLE crdb_internal.table_columns ( func(db *dbdesc.Immutable, _ string, table catalog.TableDescriptor) error { tableID := tree.NewDInt(tree.DInt(table.GetID())) tableName := tree.NewDString(table.GetName()) - columns := table.GetPublicColumns() - for i := range columns { - col := &columns[i] + columns := table.PublicColumnsNew() + for _, col := range columns { defStr := tree.DNull - if col.DefaultExpr != nil { - defExpr, err := schemaexpr.FormatExprForDisplay(ctx, table, *col.DefaultExpr, &p.semaCtx, tree.FmtParsable) + if col.HasDefault() { + defExpr, err := schemaexpr.FormatExprForDisplay(ctx, table, col.GetDefaultExpr(), &p.semaCtx, tree.FmtParsable) if err != nil { return err } @@ -2053,12 +2052,12 @@ CREATE TABLE crdb_internal.table_columns ( row = append(row, tableID, tableName, - tree.NewDInt(tree.DInt(col.ID)), - tree.NewDString(col.Name), - tree.NewDString(col.Type.DebugString()), - tree.MakeDBool(tree.DBool(col.Nullable)), + tree.NewDInt(tree.DInt(col.GetID())), + tree.NewDString(col.GetName()), + tree.NewDString(col.GetType().DebugString()), + tree.MakeDBool(tree.DBool(col.IsNullable())), defStr, - tree.MakeDBool(tree.DBool(col.Hidden)), + tree.MakeDBool(tree.DBool(col.IsHidden())), ) if err := pusher.pushRow(row...); err != nil { return err diff --git a/pkg/sql/create_stats.go b/pkg/sql/create_stats.go index b10b4e7f0254..b23ddb292851 100644 --- a/pkg/sql/create_stats.go +++ b/pkg/sql/create_stats.go @@ -428,9 +428,9 @@ func createStatsDefaultColumns( // Add all remaining columns in the table, up to maxNonIndexCols. nonIdxCols := 0 - for i := 0; i < len(desc.GetPublicColumns()) && nonIdxCols < maxNonIndexCols; i++ { - col := &desc.GetPublicColumns()[i] - colList := []descpb.ColumnID{col.ID} + for i := 0; i < len(desc.PublicColumnsNew()) && nonIdxCols < maxNonIndexCols; i++ { + col := desc.PublicColumnsNew()[i] + colList := []descpb.ColumnID{col.GetID()} if !trackStatsIfNotExists(colList) { continue @@ -441,12 +441,12 @@ func createStatsDefaultColumns( // enum types only have a few values anyway, include all possible values // for those types, up to defaultHistogramBuckets. maxHistBuckets := uint32(nonIndexColHistogramBuckets) - if col.Type.Family() == types.BoolFamily || col.Type.Family() == types.EnumFamily { + if col.GetType().Family() == types.BoolFamily || col.GetType().Family() == types.EnumFamily { maxHistBuckets = defaultHistogramBuckets } colStats = append(colStats, jobspb.CreateStatsDetails_ColStat{ ColumnIDs: colList, - HasHistogram: !colinfo.ColumnTypeIsInvertedIndexable(col.Type), + HasHistogram: !colinfo.ColumnTypeIsInvertedIndexable(col.GetType()), HistogramMaxBuckets: maxHistBuckets, }) nonIdxCols++ diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index 0411ddf3ff4e..fd9f1563999f 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -1061,7 +1061,7 @@ func tableOrdinal( } func highestTableOrdinal(desc catalog.TableDescriptor, visibility execinfrapb.ScanVisibility) int { - highest := len(desc.GetPublicColumns()) - 1 + highest := len(desc.PublicColumnsNew()) - 1 if visibility == execinfra.ScanVisibilityPublicAndNotPublic { highest = len(desc.AllColumnsNew()) - 1 } @@ -2261,7 +2261,7 @@ func (dsp *DistSQLPlanner) createPlanForZigzagJoin( cols[i].Columns[j] = uint32(col) } - numStreamCols += len(side.scan.desc.GetPublicColumns()) + numStreamCols += len(side.scan.desc.PublicColumnsNew()) } // The zigzag join node only represents inner joins, so hardcode Type to @@ -2318,7 +2318,7 @@ func (dsp *DistSQLPlanner) createPlanForZigzagJoin( i++ } - colOffset += len(side.scan.desc.GetPublicColumns()) + colOffset += len(side.scan.desc.PublicColumnsNew()) } // Set the ON condition. diff --git a/pkg/sql/information_schema.go b/pkg/sql/information_schema.go index 6ce5f7741b1f..7cd8c7986c5d 100755 --- a/pkg/sql/information_schema.go +++ b/pkg/sql/information_schema.go @@ -374,16 +374,14 @@ https://www.postgresql.org/docs/9.5/infoschema-column-privileges.html`, for _, u := range table.GetPrivileges().Users { for _, priv := range columndata { if priv.Mask()&u.Privileges != 0 { - columns := table.GetPublicColumns() - for i := range columns { - cd := &columns[i] + for _, cd := range table.PublicColumnsNew() { if err := addRow( tree.DNull, // grantor tree.NewDString(u.User().Normalized()), // grantee dbNameStr, // table_catalog scNameStr, // table_schema tree.NewDString(table.GetName()), // table_name - tree.NewDString(cd.Name), // column_name + tree.NewDString(cd.GetName()), // column_name tree.NewDString(priv.String()), // privilege_type tree.DNull, // is_grantable ); err != nil { diff --git a/pkg/sql/opt_catalog.go b/pkg/sql/opt_catalog.go index 0e9c841a7df4..1faba9b3a37e 100644 --- a/pkg/sql/opt_catalog.go +++ b/pkg/sql/opt_catalog.go @@ -507,12 +507,12 @@ func (ov *optView) Query() string { // ColumnNameCount is part of the cat.View interface. func (ov *optView) ColumnNameCount() int { - return len(ov.desc.GetPublicColumns()) + return len(ov.desc.PublicColumnsNew()) } // ColumnName is part of the cat.View interface. func (ov *optView) ColumnName(i int) tree.Name { - return tree.Name(ov.desc.GetPublicColumns()[i].Name) + return ov.desc.PublicColumnsNew()[i].ColName() } // optSequence is a wrapper around catalog.TableDescriptor that @@ -1698,7 +1698,7 @@ func newOptVirtualTable( name: *name, } - ot.columns = make([]cat.Column, len(desc.GetPublicColumns())+1) + ot.columns = make([]cat.Column, len(desc.PublicColumnsNew())+1) // Init dummy PK column. ot.columns[0].InitNonVirtual( 0, @@ -1711,18 +1711,17 @@ func newOptVirtualTable( nil, /* defaultExpr */ nil, /* computedExpr */ ) - for i := range desc.GetPublicColumns() { - d := desc.GetPublicColumns()[i] + for i, d := range desc.PublicColumnsNew() { ot.columns[i+1].InitNonVirtual( i+1, - cat.StableID(d.ID), - tree.Name(d.Name), + cat.StableID(d.GetID()), + tree.Name(d.GetName()), cat.Ordinary, - d.Type, - d.Nullable, - cat.MaybeHidden(d.Hidden), - d.DefaultExpr, - d.ComputeExpr, + d.GetType(), + d.IsNullable(), + cat.MaybeHidden(d.IsHidden()), + d.ColumnDesc().DefaultExpr, + d.ColumnDesc().ComputeExpr, ) } @@ -1823,8 +1822,8 @@ func (ot *optVirtualTable) Column(i int) *cat.Column { // getColDesc is part of optCatalogTableInterface. func (ot *optVirtualTable) getColDesc(i int) *descpb.ColumnDescriptor { - if i > 0 && i <= len(ot.desc.GetPublicColumns()) { - return &ot.desc.GetPublicColumns()[i-1] + if i > 0 && i <= len(ot.desc.PublicColumnsNew()) { + return ot.desc.PublicColumnsNew()[i-1].ColumnDesc() } return nil } diff --git a/pkg/sql/opt_exec_factory.go b/pkg/sql/opt_exec_factory.go index c7055b7e4c90..d84e168ff8a1 100644 --- a/pkg/sql/opt_exec_factory.go +++ b/pkg/sql/opt_exec_factory.go @@ -657,7 +657,11 @@ func (ef *execFactory) constructVirtualTableLookupJoin( return nil, err } tableScan.index = indexDesc - vtableCols := colinfo.ResultColumnsFromColDescs(tableDesc.GetID(), tableDesc.GetPublicColumns()) + publicColDescs := make([]descpb.ColumnDescriptor, len(tableDesc.PublicColumnsNew())) + for i, col := range tableDesc.PublicColumnsNew() { + publicColDescs[i] = *col.ColumnDesc() + } + vtableCols := colinfo.ResultColumnsFromColDescs(tableDesc.GetID(), publicColDescs) projectedVtableCols := planColumns(&tableScan) outputCols := make(colinfo.ResultColumns, 0, len(inputCols)+len(projectedVtableCols)) outputCols = append(outputCols, inputCols...) @@ -758,7 +762,7 @@ func (ef *execFactory) constructScanForZigzag( } for c, ok := cols.Next(0); ok; c, ok = cols.Next(c + 1) { - colCfg.wantedColumns = append(colCfg.wantedColumns, tree.ColumnID(tableDesc.GetPublicColumns()[c].ID)) + colCfg.wantedColumns = append(colCfg.wantedColumns, tree.ColumnID(tableDesc.PublicColumnsNew()[c].GetID())) } scan := ef.planner.Scan() @@ -1193,7 +1197,7 @@ func (ef *execFactory) ConstructInsert( // Set the tabColIdxToRetIdx for the mutation. Insert always returns // non-mutation columns in the same order they are defined in the table. - ins.run.tabColIdxToRetIdx = row.ColMapping(tabDesc.GetPublicColumns(), returnColDescs) + ins.run.tabColIdxToRetIdx = makePublicColIdxToRetIdx(tabDesc, returnColDescs) ins.run.rowsNeeded = true } @@ -1268,7 +1272,7 @@ func (ef *execFactory) ConstructInsertFastPath( // Set the tabColIdxToRetIdx for the mutation. Insert always returns // non-mutation columns in the same order they are defined in the table. - ins.run.tabColIdxToRetIdx = row.ColMapping(tabDesc.GetPublicColumns(), returnColDescs) + ins.run.tabColIdxToRetIdx = makePublicColIdxToRetIdx(tabDesc, returnColDescs) ins.run.rowsNeeded = true } @@ -1485,7 +1489,7 @@ func (ef *execFactory) ConstructUpsert( // Update the tabColIdxToRetIdx for the mutation. Upsert returns // non-mutation columns specified, in the same order they are defined // in the table. - ups.run.tw.tabColIdxToRetIdx = row.ColMapping(tabDesc.GetPublicColumns(), returnColDescs) + ups.run.tw.tabColIdxToRetIdx = makePublicColIdxToRetIdx(tabDesc, returnColDescs) ups.run.tw.returnCols = returnColDescs ups.run.tw.rowsNeeded = true } @@ -1675,7 +1679,7 @@ func (ef *execFactory) ConstructCreateView( if !d.ColumnOrdinals.Empty() { ref.ColumnIDs = make([]descpb.ColumnID, 0, d.ColumnOrdinals.Len()) d.ColumnOrdinals.ForEach(func(ord int) { - ref.ColumnIDs = append(ref.ColumnIDs, desc.GetPublicColumns()[ord].ID) + ref.ColumnIDs = append(ref.ColumnIDs, desc.PublicColumnsNew()[ord].GetID()) }) } entry := planDeps[desc.GetID()] @@ -1948,3 +1952,14 @@ func makeColDescList(table cat.Table, cols exec.TableColumnOrdinalSet) []descpb. } return colDescs } + +func makePublicColIdxToRetIdx( + tableDesc catalog.TableDescriptor, returnColDescs []descpb.ColumnDescriptor, +) []int { + publicCols := tableDesc.PublicColumnsNew() + publicColDescs := make([]descpb.ColumnDescriptor, len(publicCols)) + for i, col := range publicCols { + publicColDescs[i] = *col.ColumnDesc() + } + return row.ColMapping(publicColDescs, returnColDescs) +} diff --git a/pkg/sql/pg_catalog.go b/pkg/sql/pg_catalog.go index e06277dea838..6307458fe82a 100644 --- a/pkg/sql/pg_catalog.go +++ b/pkg/sql/pg_catalog.go @@ -604,7 +604,7 @@ https://www.postgresql.org/docs/9.5/catalog-pg-class.html`, relPersistence, // relPersistence tree.DBoolFalse, // relistemp relKind, // relkind - tree.NewDInt(tree.DInt(len(table.GetPublicColumns()))), // relnatts + tree.NewDInt(tree.DInt(len(table.PublicColumnsNew()))), // relnatts tree.NewDInt(tree.DInt(len(table.GetChecks()))), // relchecks tree.DBoolFalse, // relhasoids tree.MakeDBool(tree.DBool(table.IsPhysicalTable())), // relhaspkey diff --git a/pkg/sql/physicalplan/aggregator_funcs_test.go b/pkg/sql/physicalplan/aggregator_funcs_test.go index 2681984cbb09..70b10eee4b91 100644 --- a/pkg/sql/physicalplan/aggregator_funcs_test.go +++ b/pkg/sql/physicalplan/aggregator_funcs_test.go @@ -123,7 +123,7 @@ func checkDistAggregationInfo( fn execinfrapb.AggregatorSpec_Func, info DistAggregationInfo, ) { - colType := tableDesc.GetPublicColumns()[colIdx].Type + colType := tableDesc.PublicColumnsNew()[colIdx].GetType() makeTableReader := func(startPK, endPK int, streamID int) execinfrapb.ProcessorSpec { tr := execinfrapb.TableReaderSpec{ @@ -463,13 +463,16 @@ func TestDistAggregationTable(t *testing.T) { // We're going to test each aggregation function on every column that can be // used as input for it. foundCol := false - for colIdx := 1; colIdx < len(desc.GetPublicColumns()); colIdx++ { + for _, col := range desc.PublicColumnsNew() { + if col.Ordinal() == 0 { + continue + } // See if this column works with this function. - _, _, err := execinfrapb.GetAggregateInfo(fn, desc.GetPublicColumns()[colIdx].Type) + _, _, err := execinfrapb.GetAggregateInfo(fn, col.GetType()) if err != nil { continue } - if fn == execinfrapb.AggregatorSpec_SUM_INT && colIdx == 2 { + if fn == execinfrapb.AggregatorSpec_SUM_INT && col.Ordinal() == 2 { // When using sum_int over int2 column we're likely to hit an // integer out of range error since we insert random DInts into // that column, so we'll skip such config. @@ -477,10 +480,10 @@ func TestDistAggregationTable(t *testing.T) { } foundCol = true for _, numRows := range []int{5, numRows / 10, numRows / 2, numRows} { - name := fmt.Sprintf("%s/%s/%d", fn, desc.GetPublicColumns()[colIdx].Name, numRows) + name := fmt.Sprintf("%s/%s/%d", fn, col.GetName(), numRows) t.Run(name, func(t *testing.T) { checkDistAggregationInfo( - context.Background(), t, tc.Server(0), desc, colIdx, numRows, fn, info) + context.Background(), t, tc.Server(0), desc, col.Ordinal(), numRows, fn, info) }) } } diff --git a/pkg/sql/rename_database.go b/pkg/sql/rename_database.go index a95b7dbfb68e..dac5219a6e61 100644 --- a/pkg/sql/rename_database.go +++ b/pkg/sql/rename_database.go @@ -274,13 +274,13 @@ func isAllowedDependentDescInRenameDatabase( colIDs.Add(int(colID)) } - for _, column := range dependentDesc.GetPublicColumns() { - if !colIDs.Contains(int(column.ID)) { + for _, column := range dependentDesc.PublicColumnsNew() { + if !colIDs.Contains(int(column.GetID())) { continue } - colIDs.Remove(int(column.ID)) + colIDs.Remove(int(column.GetID())) - if column.DefaultExpr == nil { + if !column.HasDefault() { return false, "", errors.AssertionFailedf( "rename_database: expected column id %d in table id %d to have a default expr", dependedOn.ID, @@ -288,11 +288,11 @@ func isAllowedDependentDescInRenameDatabase( ) } // Try parse the default expression and find the table name direct reference. - parsedExpr, err := parser.ParseExpr(*column.DefaultExpr) + parsedExpr, err := parser.ParseExpr(column.GetDefaultExpr()) if err != nil { return false, "", err } - typedExpr, err := tree.TypeCheck(ctx, parsedExpr, nil, column.Type) + typedExpr, err := tree.TypeCheck(ctx, parsedExpr, nil, column.GetType()) if err != nil { return false, "", err } @@ -310,7 +310,7 @@ func isAllowedDependentDescInRenameDatabase( // We only don't allow this if the database name is in there. // This is always the last argument. if tree.Name(parsedSeqName.Parts[parsedSeqName.NumParts-1]).Normalize() == tree.Name(dbName).Normalize() { - return false, column.Name, nil + return false, column.GetName(), nil } } } diff --git a/pkg/sql/row/errors.go b/pkg/sql/row/errors.go index 682faa1ab783..80394685d795 100644 --- a/pkg/sql/row/errors.go +++ b/pkg/sql/row/errors.go @@ -167,10 +167,9 @@ func DecodeRowInfo( colIDs[i] = index.GetColumnID(i) } } else if index.Primary() { - publicColumns := tableDesc.GetPublicColumns() - colIDs = make([]descpb.ColumnID, len(publicColumns)) - for i := range publicColumns { - colIDs[i] = publicColumns[i].ID + colIDs = make([]descpb.ColumnID, len(tableDesc.PublicColumnsNew())) + for i, col := range tableDesc.PublicColumnsNew() { + colIDs[i] = col.GetID() } } else { colIDs = make([]descpb.ColumnID, 0, index.NumColumns()+index.NumExtraColumns()+index.NumStoredColumns()) diff --git a/pkg/sql/row/fetcher_mvcc_test.go b/pkg/sql/row/fetcher_mvcc_test.go index 5e01fec46763..9f5be7815457 100644 --- a/pkg/sql/row/fetcher_mvcc_test.go +++ b/pkg/sql/row/fetcher_mvcc_test.go @@ -92,10 +92,11 @@ func TestRowFetcherMVCCMetadata(t *testing.T) { for _, desc := range []catalog.TableDescriptor{parentDesc, childDesc} { var colIdxMap catalog.TableColMap var valNeededForCol util.FastIntSet - for colIdx := range desc.GetPublicColumns() { - id := desc.GetPublicColumns()[colIdx].ID - colIdxMap.Set(id, colIdx) - valNeededForCol.Add(colIdx) + colDescs := make([]descpb.ColumnDescriptor, len(desc.PublicColumnsNew())) + for i, col := range desc.PublicColumnsNew() { + colIdxMap.Set(col.GetID(), i) + valNeededForCol.Add(i) + colDescs[i] = *col.ColumnDesc() } args = append(args, row.FetcherTableArgs{ Spans: desc.AllIndexSpans(keys.SystemSQLCodec), @@ -103,7 +104,7 @@ func TestRowFetcherMVCCMetadata(t *testing.T) { Index: desc.GetPrimaryIndex().IndexDesc(), ColIdxMap: colIdxMap, IsSecondaryIndex: false, - Cols: desc.GetPublicColumns(), + Cols: colDescs, ValNeededForCol: valNeededForCol, }) } diff --git a/pkg/sql/row/fetcher_test.go b/pkg/sql/row/fetcher_test.go index 3a658bbe1cc9..8ce4c9040cd3 100644 --- a/pkg/sql/row/fetcher_test.go +++ b/pkg/sql/row/fetcher_test.go @@ -56,9 +56,12 @@ func makeFetcherArgs(entries []initFetcherArgs) []FetcherTableArgs { Index: index.IndexDesc(), ColIdxMap: catalog.ColumnIDToOrdinalMap(entry.tableDesc.PublicColumnsNew()), IsSecondaryIndex: !index.Primary(), - Cols: entry.tableDesc.GetPublicColumns(), + Cols: make([]descpb.ColumnDescriptor, len(entry.tableDesc.PublicColumnsNew())), ValNeededForCol: entry.valNeededForCol, } + for j, col := range entry.tableDesc.PublicColumnsNew() { + fetcherArgs[i].Cols[j] = *col.ColumnDesc() + } } return fetcherArgs } diff --git a/pkg/sql/row/row_converter.go b/pkg/sql/row/row_converter.go index 388852f7c406..a909e7708abc 100644 --- a/pkg/sql/row/row_converter.go +++ b/pkg/sql/row/row_converter.go @@ -420,11 +420,13 @@ func NewDatumRowConverter( c.BatchCap = kvDatumRowConverterBatchSize + padding c.KvBatch.KVs = make([]roachpb.KeyValue, 0, c.BatchCap) - colsOrdered := make([]descpb.ColumnDescriptor, len(c.tableDesc.GetPublicColumns())) - for _, col := range c.tableDesc.GetPublicColumns() { + colDescs := make([]descpb.ColumnDescriptor, len(c.tableDesc.PublicColumnsNew())) + colsOrdered := make([]descpb.ColumnDescriptor, len(cols)) + for i, col := range c.tableDesc.PublicColumnsNew() { + colDescs[i] = *col.ColumnDesc() // We prefer to have the order of columns that will be sent into // MakeComputedExprs to map that of Datums. - colsOrdered[ri.InsertColIDtoRowIndex.GetDefault(col.ID)] = col + colsOrdered[ri.InsertColIDtoRowIndex.GetDefault(col.GetID())] = colDescs[i] } // Here, computeExprs will be nil if there's no computed column, or // the list of computed expressions (including nil, for those columns @@ -432,7 +434,7 @@ func NewDatumRowConverter( c.computedExprs, _, err = schemaexpr.MakeComputedExprs( ctx, colsOrdered, - c.tableDesc.GetPublicColumns(), + colDescs, c.tableDesc, tree.NewUnqualifiedTableName(tree.Name(c.tableDesc.GetName())), c.EvalCtx, @@ -443,7 +445,10 @@ func NewDatumRowConverter( c.computedIVarContainer = schemaexpr.RowIndexedVarContainer{ Mapping: ri.InsertColIDtoRowIndex, - Cols: tableDesc.GetPublicColumns(), + Cols: make([]descpb.ColumnDescriptor, len(tableDesc.PublicColumnsNew())), + } + for i, col := range tableDesc.PublicColumnsNew() { + c.computedIVarContainer.Cols[i] = *col.ColumnDesc() } return c, nil } @@ -476,7 +481,11 @@ func (c *DatumRowConverter) Row(ctx context.Context, sourceID int32, rowIndex in var computedColsLookup []descpb.ColumnDescriptor if len(c.computedExprs) > 0 { - computedColsLookup = c.tableDesc.GetPublicColumns() + cols := c.tableDesc.PublicColumnsNew() + computedColsLookup = make([]descpb.ColumnDescriptor, len(cols)) + for i, col := range cols { + computedColsLookup[i] = *col.ColumnDesc() + } } insertRow, err := GenerateInsertRow( diff --git a/pkg/sql/rowenc/index_encoding_test.go b/pkg/sql/rowenc/index_encoding_test.go index 636d16f0171b..8658121490b5 100644 --- a/pkg/sql/rowenc/index_encoding_test.go +++ b/pkg/sql/rowenc/index_encoding_test.go @@ -210,9 +210,9 @@ func TestIndexKey(t *testing.T) { colNames []string colIDs descpb.ColumnIDs ) - for _, c := range tableDesc.GetPublicColumns() { - colNames = append(colNames, c.Name) - colIDs = append(colIDs, c.ID) + for _, c := range tableDesc.PublicColumnsNew() { + colNames = append(colNames, c.GetName()) + colIDs = append(colIDs, c.GetID()) } tableDesc.TableDesc().Families = []descpb.ColumnFamilyDescriptor{{ Name: "defaultFamily", diff --git a/pkg/sql/rowexec/inverted_joiner.go b/pkg/sql/rowexec/inverted_joiner.go index 7bf1b00dd1bf..1d241471ba11 100644 --- a/pkg/sql/rowexec/inverted_joiner.go +++ b/pkg/sql/rowexec/inverted_joiner.go @@ -219,7 +219,7 @@ func newInvertedJoiner( } tableRowIdx := ij.colIdxMap.GetDefault(colID) ij.indexRowToTableRowMap[indexRowIdx] = tableRowIdx - ij.indexRowTypes[indexRowIdx] = ij.desc.GetPublicColumns()[tableRowIdx].Type + ij.indexRowTypes[indexRowIdx] = ij.desc.PublicColumnsNew()[tableRowIdx].GetType() indexRowIdx++ } diff --git a/pkg/sql/rowexec/scrub_tablereader.go b/pkg/sql/rowexec/scrub_tablereader.go index 43a0a8da069f..0ed78bcde846 100644 --- a/pkg/sql/rowexec/scrub_tablereader.go +++ b/pkg/sql/rowexec/scrub_tablereader.go @@ -153,9 +153,9 @@ func (tr *scrubTableReader) generateScrubErrorRow( // Collect all the row values into JSON rowDetails := make(map[string]interface{}) for i, colIdx := range tr.fetcherResultToColIdx { - col := tr.tableDesc.GetPublicColumns()[colIdx] + col := tr.tableDesc.PublicColumnsNew()[colIdx] // TODO(joey): We should maybe try to get the underlying type. - rowDetails[col.Name] = row[i].String(col.Type) + rowDetails[col.GetName()] = row[i].String(col.GetType()) } details["row_data"] = rowDetails details["index_name"] = index.GetName() @@ -193,7 +193,7 @@ func (tr *scrubTableReader) prettyPrimaryKeyValues( } var colIDToRowIdxMap catalog.TableColMap for rowIdx, colIdx := range tr.fetcherResultToColIdx { - colIDToRowIdxMap.Set(tr.tableDesc.GetPublicColumns()[colIdx].ID, rowIdx) + colIDToRowIdxMap.Set(tr.tableDesc.PublicColumnsNew()[colIdx].GetID(), rowIdx) } var primaryKeyValues bytes.Buffer primaryKeyValues.WriteByte('(') diff --git a/pkg/sql/rowexec/zigzagjoiner.go b/pkg/sql/rowexec/zigzagjoiner.go index 4f249289c3ed..8a581434a6fd 100644 --- a/pkg/sql/rowexec/zigzagjoiner.go +++ b/pkg/sql/rowexec/zigzagjoiner.go @@ -338,7 +338,7 @@ func newZigzagJoiner( if err := z.setupInfo(flowCtx, spec, i, colOffset, tables); err != nil { return nil, err } - colOffset += len(z.infos[i].table.GetPublicColumns()) + colOffset += len(z.infos[i].table.PublicColumnsNew()) } z.side = 0 return z, nil @@ -440,7 +440,7 @@ func (z *zigzagJoiner) setupInfo( // Add the outputted columns. neededCols := util.MakeFastIntSet() outCols := z.Out.NeededColumns() - maxCol := colOffset + len(info.table.GetPublicColumns()) + maxCol := colOffset + len(info.table.PublicColumnsNew()) for i, ok := outCols.Next(colOffset); ok && i < maxCol; i, ok = outCols.Next(i + 1) { neededCols.Add(i - colOffset) } @@ -653,7 +653,7 @@ func (zi *zigzagJoinerInfo) eqColTypes() []*types.T { func (zi *zigzagJoinerInfo) eqOrdering() (colinfo.ColumnOrdering, error) { ordering := make(colinfo.ColumnOrdering, len(zi.eqColumns)) for i := range zi.eqColumns { - colID := zi.table.GetPublicColumns()[zi.eqColumns[i]].ID + colID := zi.table.PublicColumnsNew()[zi.eqColumns[i]].GetID() // Search the index columns, then the primary keys to find an ordering for // the current column, 'colID'. var direction encoding.Direction diff --git a/pkg/sql/scan.go b/pkg/sql/scan.go index e70eb05667af..876727bc3bc0 100644 --- a/pkg/sql/scan.go +++ b/pkg/sql/scan.go @@ -308,11 +308,10 @@ func initColsForScan( } if colCfg.addUnwantedAsHidden { - for i := range desc.GetPublicColumns() { - c := &desc.GetPublicColumns()[i] + for _, c := range desc.PublicColumnsNew() { found := false for _, wc := range colCfg.wantedColumns { - if descpb.ColumnID(wc) == c.ID { + if descpb.ColumnID(wc) == c.GetID() { found = true break } @@ -321,7 +320,7 @@ func initColsForScan( // NB: we could amortize this allocation using a second slice, // but addUnwantedAsHidden is only used by scrub, so doing so // doesn't seem worth it. - col := *c + col := *c.ColumnDesc() col.Hidden = true cols = append(cols, &col) } diff --git a/pkg/sql/schema_changer_test.go b/pkg/sql/schema_changer_test.go index 405b8997d6f8..ad291b864c56 100644 --- a/pkg/sql/schema_changer_test.go +++ b/pkg/sql/schema_changer_test.go @@ -2243,10 +2243,10 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT UNIQUE DEFAULT 23 CREATE FAMILY F3 // TODO(erik): Ignore errors or individually drop indexes in // DELETE_AND_WRITE_ONLY which failed during the creation backfill // as a rollback from a drop. - if e := 1; e != len(tableDesc.GetPublicColumns()) { - t.Fatalf("e = %d, v = %d, columns = %+v", e, len(tableDesc.GetPublicColumns()), tableDesc.GetPublicColumns()) - } else if tableDesc.GetPublicColumns()[0].Name != "k" { - t.Fatalf("columns %+v", tableDesc.GetPublicColumns()) + if e := 1; e != len(tableDesc.PublicColumnsNew()) { + t.Fatalf("e = %d, v = %d, columns = %+v", e, len(tableDesc.PublicColumnsNew()), tableDesc.PublicColumnsNew()) + } else if tableDesc.PublicColumnsNew()[0].GetName() != "k" { + t.Fatalf("columns %+v", tableDesc.PublicColumnsNew()) } else if len(tableDesc.GetMutations()) != 2 { t.Fatalf("mutations %+v", tableDesc.GetMutations()) } @@ -4409,10 +4409,10 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT); if num := len(tableDesc.GetMutations()); num > 0 { t.Fatalf("%d outstanding mutation", num) } - if lenCols := len(tableDesc.GetPublicColumns()); lenCols != 2 { + if lenCols := len(tableDesc.PublicColumnsNew()); lenCols != 2 { t.Fatalf("%d columns", lenCols) } - if k, x := tableDesc.GetPublicColumns()[0].Name, tableDesc.GetPublicColumns()[1].Name; k != "k" && x != "x" { + if k, x := tableDesc.PublicColumnsNew()[0].GetName(), tableDesc.PublicColumnsNew()[1].GetName(); k != "k" && x != "x" { t.Fatalf("columns %q, %q in descriptor", k, x) } if checks := tableDesc.AllActiveAndInactiveChecks(); len(checks) != 1 { diff --git a/pkg/sql/scrub_constraint.go b/pkg/sql/scrub_constraint.go index d605d6a21ff7..6b5a0176d947 100644 --- a/pkg/sql/scrub_constraint.go +++ b/pkg/sql/scrub_constraint.go @@ -80,7 +80,7 @@ func (o *sqlCheckConstraintCheckOperation) Start(params runParams) error { tn.ExplicitCatalog = true tn.ExplicitSchema = true sel := &tree.SelectClause{ - Exprs: tabledesc.ColumnsSelectors(o.tableDesc.GetPublicColumns()), + Exprs: tabledesc.ColumnsSelectors(o.tableDesc.PublicColumnsNew()), From: tree.From{ Tables: tree.TableExprs{&tn}, }, @@ -109,8 +109,8 @@ func (o *sqlCheckConstraintCheckOperation) Start(params runParams) error { o.run.rows = rows // Collect all the columns. - for i := range o.tableDesc.GetPublicColumns() { - o.columns = append(o.columns, &o.tableDesc.GetPublicColumns()[i]) + for _, c := range o.tableDesc.PublicColumnsNew() { + o.columns = append(o.columns, c.ColumnDesc()) } // Find the row indexes for all of the primary index columns. o.primaryColIdxs, err = getPrimaryColIdxs(o.tableDesc, o.columns) diff --git a/pkg/sql/scrub_fk.go b/pkg/sql/scrub_fk.go index 760380f1cd4f..040dd37efde6 100644 --- a/pkg/sql/scrub_fk.go +++ b/pkg/sql/scrub_fk.go @@ -106,9 +106,9 @@ func (o *sqlForeignKeyCheckOperation) Start(params runParams) error { // columns and extra columns in the secondary index used for foreign // key referencing. This also implicitly includes all primary index // columns. - columnsByID := make(map[descpb.ColumnID]*descpb.ColumnDescriptor, len(o.tableDesc.GetPublicColumns())) - for i := range o.tableDesc.GetPublicColumns() { - columnsByID[o.tableDesc.GetPublicColumns()[i].ID] = &o.tableDesc.GetPublicColumns()[i] + columnsByID := make(map[descpb.ColumnID]*descpb.ColumnDescriptor, len(o.tableDesc.PublicColumnsNew())) + for _, c := range o.tableDesc.PublicColumnsNew() { + columnsByID[c.GetID()] = c.ColumnDesc() } // Get primary key columns not included in the FK. diff --git a/pkg/sql/scrub_index.go b/pkg/sql/scrub_index.go index 8063288d65b3..18616ea1f6b9 100644 --- a/pkg/sql/scrub_index.go +++ b/pkg/sql/scrub_index.go @@ -75,17 +75,16 @@ func (o *indexCheckOperation) Start(params runParams) error { ctx := params.ctx var colToIdx catalog.TableColMap - for i := range o.tableDesc.GetPublicColumns() { - id := o.tableDesc.GetPublicColumns()[i].ID - colToIdx.Set(id, i) + for _, c := range o.tableDesc.PublicColumnsNew() { + colToIdx.Set(c.GetID(), c.Ordinal()) } var pkColumns, otherColumns []*descpb.ColumnDescriptor for i := 0; i < o.tableDesc.GetPrimaryIndex().NumColumns(); i++ { colID := o.tableDesc.GetPrimaryIndex().GetColumnID(i) - col := &o.tableDesc.GetPublicColumns()[colToIdx.GetDefault(colID)] - pkColumns = append(pkColumns, col) + col := o.tableDesc.PublicColumnsNew()[colToIdx.GetDefault(colID)] + pkColumns = append(pkColumns, col.ColumnDesc()) colToIdx.Set(colID, -1) } @@ -95,8 +94,8 @@ func (o *indexCheckOperation) Start(params runParams) error { // Skip PK column. return } - col := &o.tableDesc.GetPublicColumns()[pos] - otherColumns = append(otherColumns, col) + col := o.tableDesc.PublicColumnsNew()[pos] + otherColumns = append(otherColumns, col.ColumnDesc()) } // Collect all of the columns we are fetching from the index. This diff --git a/pkg/sql/scrub_physical.go b/pkg/sql/scrub_physical.go index 719d41ac6f3a..0bcdb675ff1e 100644 --- a/pkg/sql/scrub_physical.go +++ b/pkg/sql/scrub_physical.go @@ -71,8 +71,8 @@ func (o *physicalCheckOperation) Start(params runParams) error { // Collect all of the columns being scanned. if o.indexDesc.ID == o.tableDesc.GetPrimaryIndexID() { - for i := range o.tableDesc.GetPublicColumns() { - columnIDs = append(columnIDs, tree.ColumnID(o.tableDesc.GetPublicColumns()[i].ID)) + for _, c := range o.tableDesc.PublicColumnsNew() { + columnIDs = append(columnIDs, tree.ColumnID(c.GetID())) } } else { for _, id := range o.indexDesc.ColumnIDs { @@ -88,7 +88,7 @@ func (o *physicalCheckOperation) Start(params runParams) error { for i := range columnIDs { idx := colIDToIdx.GetDefault(descpb.ColumnID(columnIDs[i])) - columns = append(columns, &o.tableDesc.GetPublicColumns()[idx]) + columns = append(columns, o.tableDesc.PublicColumnsNew()[idx].ColumnDesc()) } // Find the row indexes for all of the primary index columns. diff --git a/pkg/sql/scrub_test.go b/pkg/sql/scrub_test.go index 268f23b686d3..53a4dca6bd57 100644 --- a/pkg/sql/scrub_test.go +++ b/pkg/sql/scrub_test.go @@ -63,8 +63,8 @@ INSERT INTO t."tEst" VALUES (10, 20); secondaryIndex := tableDesc.PublicNonPrimaryIndexes()[0] var colIDtoRowIndex catalog.TableColMap - colIDtoRowIndex.Set(tableDesc.GetPublicColumns()[0].ID, 0) - colIDtoRowIndex.Set(tableDesc.GetPublicColumns()[1].ID, 1) + colIDtoRowIndex.Set(tableDesc.PublicColumnsNew()[0].GetID(), 0) + colIDtoRowIndex.Set(tableDesc.PublicColumnsNew()[1].GetID(), 1) // Construct the secondary index key that is currently in the // database. @@ -132,8 +132,8 @@ CREATE INDEX secondary ON t.test (v); secondaryIndex := tableDesc.PublicNonPrimaryIndexes()[0] var colIDtoRowIndex catalog.TableColMap - colIDtoRowIndex.Set(tableDesc.GetPublicColumns()[0].ID, 0) - colIDtoRowIndex.Set(tableDesc.GetPublicColumns()[1].ID, 1) + colIDtoRowIndex.Set(tableDesc.PublicColumnsNew()[0].GetID(), 0) + colIDtoRowIndex.Set(tableDesc.PublicColumnsNew()[1].GetID(), 1) // Construct datums and secondary k/v for our row values (k, v). values := []tree.Datum{tree.NewDInt(10), tree.NewDInt(314)} @@ -226,9 +226,9 @@ INSERT INTO t.test VALUES (10, 20, 1337); secondaryIndex := tableDesc.PublicNonPrimaryIndexes()[0] var colIDtoRowIndex catalog.TableColMap - colIDtoRowIndex.Set(tableDesc.GetPublicColumns()[0].ID, 0) - colIDtoRowIndex.Set(tableDesc.GetPublicColumns()[1].ID, 1) - colIDtoRowIndex.Set(tableDesc.GetPublicColumns()[2].ID, 2) + colIDtoRowIndex.Set(tableDesc.PublicColumnsNew()[0].GetID(), 0) + colIDtoRowIndex.Set(tableDesc.PublicColumnsNew()[1].GetID(), 1) + colIDtoRowIndex.Set(tableDesc.PublicColumnsNew()[2].GetID(), 2) // Generate the existing secondary index key. values := []tree.Datum{tree.NewDInt(10), tree.NewDInt(20), tree.NewDInt(1337)} @@ -345,8 +345,8 @@ INSERT INTO t.test VALUES (10, 2); tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test") var colIDtoRowIndex catalog.TableColMap - colIDtoRowIndex.Set(tableDesc.GetPublicColumns()[0].ID, 0) - colIDtoRowIndex.Set(tableDesc.GetPublicColumns()[1].ID, 1) + colIDtoRowIndex.Set(tableDesc.PublicColumnsNew()[0].GetID(), 0) + colIDtoRowIndex.Set(tableDesc.PublicColumnsNew()[1].GetID(), 1) // Create the primary index key. values := []tree.Datum{tree.NewDInt(10), tree.NewDInt(2)} @@ -367,7 +367,7 @@ INSERT INTO t.test VALUES (10, 2); values = []tree.Datum{tree.NewDInt(10), tree.NewDInt(0)} // Encode the column value. valueBuf, err := rowenc.EncodeTableValue( - []byte(nil), tableDesc.GetPublicColumns()[1].ID, values[1], []byte(nil)) + []byte(nil), tableDesc.PublicColumnsNew()[1].GetID(), values[1], []byte(nil)) if err != nil { t.Fatalf("unexpected error: %s", err) } @@ -448,8 +448,8 @@ func TestScrubFKConstraintFKMissing(t *testing.T) { secondaryIndex := tableDesc.PublicNonPrimaryIndexes()[0] var colIDtoRowIndex catalog.TableColMap - colIDtoRowIndex.Set(tableDesc.GetPublicColumns()[0].ID, 0) - colIDtoRowIndex.Set(tableDesc.GetPublicColumns()[1].ID, 1) + colIDtoRowIndex.Set(tableDesc.PublicColumnsNew()[0].GetID(), 0) + colIDtoRowIndex.Set(tableDesc.PublicColumnsNew()[1].GetID(), 1) // Construct the secondary index key entry as it exists in the // database. @@ -586,8 +586,8 @@ INSERT INTO t.test VALUES (217, 314); values := []tree.Datum{tree.NewDInt(217), tree.NewDInt(314)} var colIDtoRowIndex catalog.TableColMap - colIDtoRowIndex.Set(tableDesc.GetPublicColumns()[0].ID, 0) - colIDtoRowIndex.Set(tableDesc.GetPublicColumns()[1].ID, 1) + colIDtoRowIndex.Set(tableDesc.PublicColumnsNew()[0].GetID(), 0) + colIDtoRowIndex.Set(tableDesc.PublicColumnsNew()[1].GetID(), 1) // Create the primary index key primaryIndexKeyPrefix := rowenc.MakeIndexKeyPrefix( @@ -668,9 +668,9 @@ INSERT INTO t.test VALUES (217, 314, 1337); values := []tree.Datum{tree.NewDInt(217), tree.NewDInt(314), tree.NewDInt(1337)} var colIDtoRowIndex catalog.TableColMap - colIDtoRowIndex.Set(tableDesc.GetPublicColumns()[0].ID, 0) - colIDtoRowIndex.Set(tableDesc.GetPublicColumns()[1].ID, 1) - colIDtoRowIndex.Set(tableDesc.GetPublicColumns()[2].ID, 2) + colIDtoRowIndex.Set(tableDesc.PublicColumnsNew()[0].GetID(), 0) + colIDtoRowIndex.Set(tableDesc.PublicColumnsNew()[1].GetID(), 1) + colIDtoRowIndex.Set(tableDesc.PublicColumnsNew()[2].GetID(), 2) // Create the primary index key primaryIndexKeyPrefix := rowenc.MakeIndexKeyPrefix( @@ -688,7 +688,7 @@ INSERT INTO t.test VALUES (217, 314, 1337); // Encode the second column value. valueBuf, err := rowenc.EncodeTableValue( - []byte(nil), tableDesc.GetPublicColumns()[1].ID, values[1], []byte(nil)) + []byte(nil), tableDesc.PublicColumnsNew()[1].GetID(), values[1], []byte(nil)) if err != nil { t.Fatalf("unexpected error: %s", err) } @@ -774,8 +774,8 @@ CREATE TABLE t.test ( values := []tree.Datum{tree.NewDInt(217), tree.NewDInt(314)} var colIDtoRowIndex catalog.TableColMap - colIDtoRowIndex.Set(tableDesc.GetPublicColumns()[0].ID, 0) - colIDtoRowIndex.Set(tableDesc.GetPublicColumns()[1].ID, 1) + colIDtoRowIndex.Set(tableDesc.PublicColumnsNew()[0].GetID(), 0) + colIDtoRowIndex.Set(tableDesc.PublicColumnsNew()[1].GetID(), 1) // Create the primary index key primaryIndexKeyPrefix := rowenc.MakeIndexKeyPrefix( @@ -791,7 +791,7 @@ CREATE TABLE t.test ( // Encode the second column value. valueBuf, err := rowenc.EncodeTableValue( - []byte(nil), tableDesc.GetPublicColumns()[1].ID, values[1], []byte(nil)) + []byte(nil), tableDesc.PublicColumnsNew()[1].GetID(), values[1], []byte(nil)) if err != nil { t.Fatalf("unexpected error: %s", err) } @@ -809,7 +809,7 @@ CREATE TABLE t.test ( // Encode the second column value. valueBuf, err = rowenc.EncodeTableValue( - []byte(nil), tableDesc.GetPublicColumns()[1].ID, values[1], []byte(nil)) + []byte(nil), tableDesc.PublicColumnsNew()[1].GetID(), values[1], []byte(nil)) if err != nil { t.Fatalf("unexpected error: %s", err) } @@ -878,9 +878,9 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v1 INT, v2 INT); values := []tree.Datum{tree.NewDInt(217), tree.NewDInt(314), tree.NewDInt(1337)} var colIDtoRowIndex catalog.TableColMap - colIDtoRowIndex.Set(tableDesc.GetPublicColumns()[0].ID, 0) - colIDtoRowIndex.Set(tableDesc.GetPublicColumns()[1].ID, 1) - colIDtoRowIndex.Set(tableDesc.GetPublicColumns()[2].ID, 2) + colIDtoRowIndex.Set(tableDesc.PublicColumnsNew()[0].GetID(), 0) + colIDtoRowIndex.Set(tableDesc.PublicColumnsNew()[1].GetID(), 1) + colIDtoRowIndex.Set(tableDesc.PublicColumnsNew()[2].GetID(), 2) // Create the primary index key primaryIndexKeyPrefix := rowenc.MakeIndexKeyPrefix( @@ -896,7 +896,7 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v1 INT, v2 INT); // Encode the second column values. The second column is encoded with // a garbage colIDDiff. valueBuf, err := rowenc.EncodeTableValue( - []byte(nil), tableDesc.GetPublicColumns()[1].ID, values[1], []byte(nil)) + []byte(nil), tableDesc.PublicColumnsNew()[1].GetID(), values[1], []byte(nil)) if err != nil { t.Fatalf("unexpected error: %s", err) } diff --git a/pkg/sql/sequence.go b/pkg/sql/sequence.go index 804c70103d47..1ccf36af9167 100644 --- a/pkg/sql/sequence.go +++ b/pkg/sql/sequence.go @@ -44,7 +44,7 @@ func (p *planner) GetSerialSequenceNameFromColumn( if err != nil { return nil, err } - for _, col := range tableDesc.GetPublicColumns() { + for _, col := range tableDesc.PublicColumnsNew() { if col.ColName() == columnName { // Seems like we have no way of detecting whether this was done using "SERIAL". // Guess by assuming it is SERIAL it it uses only one sequence. @@ -53,11 +53,11 @@ func (p *planner) GetSerialSequenceNameFromColumn( // which have not been thought about (e.g. implication for backup and restore, // as well as backward compatibility) so we're using this heuristic for now. // TODO(#52487): fix this up. - if len(col.UsesSequenceIds) == 1 { + if col.NumUsesSequences() == 1 { seq, err := p.Descriptors().GetImmutableTableByID( ctx, p.txn, - col.UsesSequenceIds[0], + col.GetUsesSequenceID(0), tree.ObjectLookupFlagsWithRequiredTableKind(tree.ResolveRequireSequenceDesc), ) if err != nil { diff --git a/pkg/sql/sequence_test.go b/pkg/sql/sequence_test.go index 25f84ebe329e..54ac943fbc0b 100644 --- a/pkg/sql/sequence_test.go +++ b/pkg/sql/sequence_test.go @@ -163,7 +163,7 @@ func assertColumnOwnsSequences( t *testing.T, kvDB *kv.DB, dbName string, tbName string, colIdx int, seqNames []string, ) { tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, dbName, tbName) - col := tableDesc.GetPublicColumns()[colIdx] + col := tableDesc.PublicColumnsNew()[colIdx] var seqDescs []catalog.TableDescriptor for _, seqName := range seqNames { seqDescs = append( @@ -172,24 +172,25 @@ func assertColumnOwnsSequences( ) } - if len(col.OwnsSequenceIds) != len(seqDescs) { + if col.NumOwnsSequences() != len(seqDescs) { t.Fatalf( "unexpected number of sequence ownership dependencies. expected: %d, got:%d", - len(seqDescs), len(col.OwnsSequenceIds), + len(seqDescs), col.NumOwnsSequences(), ) } - for i, seqID := range col.OwnsSequenceIds { + for i := 0; i < col.NumOwnsSequences(); i++ { + seqID := col.GetOwnsSequenceID(i) if seqID != seqDescs[i].GetID() { t.Fatalf("unexpected sequence id. expected %d got %d", seqDescs[i].GetID(), seqID) } ownerTableID := seqDescs[i].GetSequenceOpts().SequenceOwner.OwnerTableID ownerColID := seqDescs[i].GetSequenceOpts().SequenceOwner.OwnerColumnID - if ownerTableID != tableDesc.GetID() || ownerColID != col.ID { + if ownerTableID != tableDesc.GetID() || ownerColID != col.GetID() { t.Fatalf( "unexpected sequence owner. expected table id %d, got: %d; expected column id %d, got :%d", - tableDesc.GetID(), ownerTableID, col.ID, ownerColID, + tableDesc.GetID(), ownerTableID, col.GetID(), ownerColID, ) } } diff --git a/pkg/sql/show_create_clauses.go b/pkg/sql/show_create_clauses.go index cac95da08027..2c6024a3df15 100644 --- a/pkg/sql/show_create_clauses.go +++ b/pkg/sql/show_create_clauses.go @@ -90,11 +90,11 @@ func ShowCreateView( f.WriteString("VIEW ") f.FormatNode(tn) f.WriteString(" (") - for i := range desc.GetPublicColumns() { + for i, col := range desc.PublicColumnsNew() { if i > 0 { f.WriteString(", ") } - name := desc.PublicColumnsNew()[i].GetName() + name := col.GetName() f.FormatNameP(&name) } f.WriteString(") AS ") diff --git a/pkg/sql/show_fingerprints.go b/pkg/sql/show_fingerprints.go index e69110af0de7..1aafb18b2149 100644 --- a/pkg/sql/show_fingerprints.go +++ b/pkg/sql/show_fingerprints.go @@ -96,7 +96,7 @@ func (n *showFingerprintsNode) Next(params runParams) (bool, error) { } index := n.indexes[n.run.rowIdx] - cols := make([]string, 0, len(n.tableDesc.GetPublicColumns())) + cols := make([]string, 0, len(n.tableDesc.PublicColumnsNew())) addColumn := func(col *descpb.ColumnDescriptor) { // TODO(dan): This is known to be a flawed way to fingerprint. Any datum // with the same string representation is fingerprinted the same, even @@ -110,14 +110,13 @@ func (n *showFingerprintsNode) Next(params runParams) (bool, error) { } if index.ID == n.tableDesc.GetPrimaryIndexID() { - for i := range n.tableDesc.GetPublicColumns() { - addColumn(&n.tableDesc.GetPublicColumns()[i]) + for _, col := range n.tableDesc.PublicColumnsNew() { + addColumn(col.ColumnDesc()) } } else { colsByID := make(map[descpb.ColumnID]*descpb.ColumnDescriptor) - for i := range n.tableDesc.GetPublicColumns() { - col := &n.tableDesc.GetPublicColumns()[i] - colsByID[col.ID] = col + for _, col := range n.tableDesc.PublicColumnsNew() { + colsByID[col.GetID()] = col.ColumnDesc() } colIDs := append(append(index.ColumnIDs, index.ExtraColumnIDs...), index.StoreColumnIDs...) for _, colID := range colIDs { diff --git a/pkg/sql/span/span_builder.go b/pkg/sql/span/span_builder.go index ad9649cf514c..c21c707eca3c 100644 --- a/pkg/sql/span/span_builder.go +++ b/pkg/sql/span/span_builder.go @@ -76,12 +76,10 @@ func MakeBuilder( columnIDs, s.indexColDirs = index.FullColumnIDs() s.indexColTypes = make([]*types.T, len(columnIDs)) for i, colID := range columnIDs { + col, _ := table.FindColumnWithID(colID) // TODO (rohany): do I need to look at table columns with mutations here as well? - for _, col := range table.GetPublicColumns() { - if col.ID == colID { - s.indexColTypes[i] = col.Type - break - } + if col != nil && col.Public() { + s.indexColTypes[i] = col.GetType() } } diff --git a/pkg/sql/table_ref_test.go b/pkg/sql/table_ref_test.go index a8df080a3080..64d051240eed 100644 --- a/pkg/sql/table_ref_test.go +++ b/pkg/sql/table_ref_test.go @@ -49,15 +49,14 @@ CREATE INDEX bc ON test.t(b, c); tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "test", "t") tID := tableDesc.GetID() var aID, bID, cID descpb.ColumnID - for i := range tableDesc.GetPublicColumns() { - c := &tableDesc.GetPublicColumns()[i] - switch c.Name { + for _, c := range tableDesc.PublicColumnsNew() { + switch c.GetName() { case "a": - aID = c.ID + aID = c.GetID() case "b": - bID = c.ID + bID = c.GetID() case "c": - cID = c.ID + cID = c.GetID() } } pkID := tableDesc.GetPrimaryIndexID() @@ -67,11 +66,10 @@ CREATE INDEX bc ON test.t(b, c); tableDesc = catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "test", "hidden") tIDHidden := tableDesc.GetID() var rowIDHidden descpb.ColumnID - for i := range tableDesc.GetPublicColumns() { - c := &tableDesc.GetPublicColumns()[i] - switch c.Name { + for _, c := range tableDesc.PublicColumnsNew() { + switch c.GetName() { case "rowid": - rowIDHidden = c.ID + rowIDHidden = c.GetID() } } diff --git a/pkg/sql/table_test.go b/pkg/sql/table_test.go index 0f6c726c8717..679fa7dacb54 100644 --- a/pkg/sql/table_test.go +++ b/pkg/sql/table_test.go @@ -448,10 +448,10 @@ func TestSerializedUDTsInTableDescriptor(t *testing.T) { ctx := context.Background() getDefault := func(desc catalog.TableDescriptor) string { - return *desc.GetPublicColumns()[0].DefaultExpr + return desc.PublicColumnsNew()[0].GetDefaultExpr() } getComputed := func(desc catalog.TableDescriptor) string { - return *desc.GetPublicColumns()[0].ComputeExpr + return desc.PublicColumnsNew()[0].GetComputeExpr() } getCheck := func(desc catalog.TableDescriptor) string { return desc.GetChecks()[0].Expr diff --git a/pkg/sql/temporary_schema.go b/pkg/sql/temporary_schema.go index 8643b5f477c1..27e5c8da0815 100644 --- a/pkg/sql/temporary_schema.go +++ b/pkg/sql/temporary_schema.go @@ -320,8 +320,8 @@ func cleanupSchemaObjects( for _, colID := range d.ColumnIDs { dependentColIDs.Add(int(colID)) } - for _, col := range dTableDesc.GetPublicColumns() { - if dependentColIDs.Contains(int(col.ID)) { + for _, col := range dTableDesc.PublicColumnsNew() { + if dependentColIDs.Contains(int(col.GetID())) { tbName := tree.MakeTableNameWithSchema( tree.Name(db.GetName()), tree.Name(schema), @@ -335,7 +335,7 @@ func cleanupSchemaObjects( fmt.Sprintf( "ALTER TABLE %s ALTER COLUMN %s DROP DEFAULT", tbName.FQString(), - tree.NameString(col.Name), + tree.NameString(col.GetName()), ), ) if err != nil { diff --git a/pkg/sql/virtual_schema.go b/pkg/sql/virtual_schema.go index ffec2688fe85..9739c7e35e10 100644 --- a/pkg/sql/virtual_schema.go +++ b/pkg/sql/virtual_schema.go @@ -427,7 +427,7 @@ func (e *virtualDefEntry) validateRow(datums tree.Datums, columns colinfo.Result col := &columns[i] datum := datums[i] if datum == tree.DNull { - if !e.desc.GetPublicColumns()[i].Nullable { + if !e.desc.PublicColumnsNew()[i].IsNullable() { return errors.AssertionFailedf("column %s.%s not nullable, but found NULL value", e.desc.GetName(), col.Name) } @@ -449,11 +449,10 @@ func (e *virtualDefEntry) getPlanInfo( idxConstraint *constraint.Constraint, ) (colinfo.ResultColumns, virtualTableConstructor) { var columns colinfo.ResultColumns - for i := range e.desc.GetPublicColumns() { - col := &e.desc.GetPublicColumns()[i] + for _, col := range e.desc.PublicColumnsNew() { columns = append(columns, colinfo.ResultColumn{ - Name: col.Name, - Typ: col.Type, + Name: col.GetName(), + Typ: col.GetType(), TableID: table.GetID(), PGAttributeNum: col.GetPGAttributeNum(), }) diff --git a/pkg/sqlmigrations/migrations_test.go b/pkg/sqlmigrations/migrations_test.go index 683e1f23a83d..ded6f55e99fb 100644 --- a/pkg/sqlmigrations/migrations_test.go +++ b/pkg/sqlmigrations/migrations_test.go @@ -880,9 +880,9 @@ CREATE TABLE system.jobs ( newJobsTable := catalogkv.TestingGetTableDescriptor( mt.kvDB, keys.SystemSQLCodec, "system", "jobs") - require.Equal(t, 7, len(newJobsTable.GetPublicColumns())) - require.Equal(t, "created_by_type", newJobsTable.GetPublicColumns()[5].Name) - require.Equal(t, "created_by_id", newJobsTable.GetPublicColumns()[6].Name) + require.Equal(t, 7, len(newJobsTable.PublicColumnsNew())) + require.Equal(t, "created_by_type", newJobsTable.PublicColumnsNew()[5].GetName()) + require.Equal(t, "created_by_id", newJobsTable.PublicColumnsNew()[6].GetName()) require.Equal(t, 2, len(newJobsTable.GetFamilies())) // Ensure we keep old family name. require.Equal(t, primaryFamilyName, newJobsTable.GetFamilies()[0].Name) @@ -962,9 +962,9 @@ func TestVersionAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTable newJobsTable := catalogkv.TestingGetTableDescriptor( mt.kvDB, keys.SystemSQLCodec, "system", "jobs") - require.Equal(t, 9, len(newJobsTable.GetPublicColumns())) - require.Equal(t, "claim_session_id", newJobsTable.GetPublicColumns()[7].Name) - require.Equal(t, "claim_instance_id", newJobsTable.GetPublicColumns()[8].Name) + require.Equal(t, 9, len(newJobsTable.PublicColumnsNew())) + require.Equal(t, "claim_session_id", newJobsTable.PublicColumnsNew()[7].GetName()) + require.Equal(t, "claim_instance_id", newJobsTable.PublicColumnsNew()[8].GetName()) require.Equal(t, 3, len(newJobsTable.GetFamilies())) // Ensure we keep old family names. require.Equal(t, "fam_0_id_status_created_payload", newJobsTable.GetFamilies()[0].Name)