Skip to content

Commit

Permalink
Merge #95069
Browse files Browse the repository at this point in the history
95069: catalog: remove errors from Descriptor methods r=postamar a=postamar

Each of these is typically replaced by a couple of functions in the catalog package, one which returns an error and another which doesn't. Error creation is expensive and should be avoided in the hot path.

Fixes #91924.

Release note: None

Co-authored-by: Marius Posta <[email protected]>
  • Loading branch information
craig[bot] and Marius Posta committed Jan 19, 2023
2 parents faabd83 + fa6333e commit f3195cf
Show file tree
Hide file tree
Showing 120 changed files with 760 additions and 916 deletions.
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/restore_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -2116,7 +2116,7 @@ func (r *restoreResumer) publishDescriptors(

badIndexes := devalidateIndexes[mutTable.ID]
for _, badIdx := range badIndexes {
found, err := mutTable.FindIndexWithID(badIdx)
found, err := catalog.MustFindIndexByID(mutTable, badIdx)
if err != nil {
return err
}
Expand Down
3 changes: 1 addition & 2 deletions pkg/ccl/changefeedccl/cdceval/cdc_prev.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,8 +32,7 @@ var _ catalog.Column = (*prevCol)(nil)

func newPrevColumnForDesc(desc *cdcevent.EventDescriptor) (catalog.Column, error) {
colExists := func(n tree.Name) bool {
_, err := desc.TableDescriptor().FindColumnWithName(n)
return err == nil
return catalog.FindColumnByTreeName(desc.TableDescriptor(), n) != nil
}

prevColName := tree.Name("cdc_prev")
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/cdceval/expr_eval.go
Original file line number Diff line number Diff line change
Expand Up @@ -336,7 +336,7 @@ func inputSpecForEventDescriptor(
inputTypes := make([]*types.T, 0, numCols)
var inputCols catalog.TableColMap
for i, c := range ed.ResultColumns() {
col, err := ed.TableDescriptor().FindColumnWithName(tree.Name(c.Name))
col, err := catalog.MustFindColumnByName(ed.TableDescriptor(), c.Name)
if err != nil {
return inputTypes, inputCols, err
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/changefeedccl/cdceval/expr_eval_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -727,10 +727,10 @@ func randEncDatumPrimaryFamily(
t.Helper()
rng, _ := randutil.NewTestRand()

family, err := desc.FindFamilyByID(0)
family, err := catalog.MustFindFamilyByID(desc, 0 /* id */)
require.NoError(t, err)
for _, colID := range family.ColumnIDs {
col, err := desc.FindColumnWithID(colID)
col, err := catalog.MustFindColumnByID(desc, colID)
require.NoError(t, err)
row = append(row, rowenc.EncDatum{Datum: randgen.RandDatum(rng, col.GetType(), col.IsNullable())})
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/changefeedccl/cdceval/validation.go
Original file line number Diff line number Diff line change
Expand Up @@ -228,7 +228,7 @@ func getTargetFamilyDescriptor(
) (*descpb.ColumnFamilyDescriptor, error) {
switch target.Type {
case jobspb.ChangefeedTargetSpecification_PRIMARY_FAMILY_ONLY:
return desc.FindFamilyByID(0)
return catalog.MustFindFamilyByID(desc, 0 /* id */)
case jobspb.ChangefeedTargetSpecification_COLUMN_FAMILY:
var fd *descpb.ColumnFamilyDescriptor
for _, family := range desc.GetFamilies() {
Expand Down Expand Up @@ -366,7 +366,7 @@ func (c *checkColumnsVisitor) VisitCols(expr tree.Expr) (bool, tree.Expr) {
return c.VisitCols(vn)

case *tree.ColumnItem:
col, err := c.desc.FindColumnWithName(e.ColumnName)
col, err := catalog.MustFindColumnByTreeName(c.desc, e.ColumnName)
if err != nil {
c.err = err
return false, expr
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/cdcevent/event.go
Original file line number Diff line number Diff line change
Expand Up @@ -611,7 +611,7 @@ func (it iter) Col(fn ColumnFn) error {
func TestingMakeEventRow(
desc catalog.TableDescriptor, familyID descpb.FamilyID, encRow rowenc.EncDatumRow, deleted bool,
) Row {
family, err := desc.FindFamilyByID(familyID)
family, err := catalog.MustFindFamilyByID(desc, familyID)
if err != nil {
panic(err) // primary column family always exists.
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/changefeedccl/cdcevent/event_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -615,7 +615,7 @@ func mustGetFamily(
t *testing.T, desc catalog.TableDescriptor, familyID descpb.FamilyID,
) *descpb.ColumnFamilyDescriptor {
t.Helper()
f, err := desc.FindFamilyByID(familyID)
f, err := catalog.MustFindFamilyByID(desc, familyID)
require.NoError(t, err)
return f
}
Expand Down Expand Up @@ -660,7 +660,7 @@ func expectResultColumns(
}

for _, colName := range colNames {
col, err := desc.FindColumnWithName(tree.Name(colName))
col, err := catalog.MustFindColumnByName(desc, colName)
require.NoError(t, err)
res = append(res, ResultColumn{
ResultColumn: colinfo.ResultColumn{
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/cdcevent/rowfetcher_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -200,7 +200,7 @@ func (c *rowFetcherCache) RowFetcherForColumnFamily(
}
}

familyDesc, err := tableDesc.FindFamilyByID(family)
familyDesc, err := catalog.MustFindFamilyByID(tableDesc, family)
if err != nil {
return nil, nil, err
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/schemafeed/table_event_filter.go
Original file line number Diff line number Diff line change
Expand Up @@ -375,7 +375,7 @@ func hasNewPrimaryIndexWithNoVisibleColumnChanges(

for i, n := 0, idx.NumPrimaryStoredColumns(); i < n; i++ {
colID := idx.GetStoredColumnID(i)
col, _ := tab.FindColumnWithID(colID)
col := catalog.FindColumnByID(tab, colID)

// If specific columns are targeted, then only consider the column if it is targeted.
if col.Public() && (!hasSpecificColumnTargets || targetedCols.Contains(int(col.GetID()))) {
Expand Down
5 changes: 3 additions & 2 deletions pkg/ccl/partitionccl/drop_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils"
"github.com/cockroachdb/cockroach/pkg/sql/gcjob"
"github.com/cockroachdb/cockroach/pkg/sql/tests"
Expand Down Expand Up @@ -74,7 +75,7 @@ func TestDropIndexWithZoneConfigCCL(t *testing.T) {
)`)
codec := tenantOrSystemCodec(s)
tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, codec, "t", "kv")
index, err := tableDesc.FindIndexWithName("i")
index, err := catalog.MustFindIndexByName(tableDesc, "i")
if err != nil {
t.Fatal(err)
}
Expand Down Expand Up @@ -121,7 +122,7 @@ func TestDropIndexWithZoneConfigCCL(t *testing.T) {
}
}
tableDesc = desctestutils.TestingGetPublicTableDescriptor(kvDB, codec, "t", "kv")
if _, err := tableDesc.FindIndexWithName("i"); err == nil {
if catalog.FindIndexByName(tableDesc, "i") != nil {
t.Fatalf("table descriptor still contains index after index is dropped")
}
close(asyncNotification)
Expand Down
5 changes: 3 additions & 2 deletions pkg/ccl/partitionccl/partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/server"
"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/bootstrap"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils"
Expand Down Expand Up @@ -204,7 +205,7 @@ func (pt *partitioningTest) parse() error {
if !strings.HasPrefix(indexName, "@") {
panic(errors.Errorf("unsupported config: %s", c))
}
idx, err := pt.parsed.tableDesc.FindIndexWithName(indexName[1:])
idx, err := catalog.MustFindIndexByName(pt.parsed.tableDesc, indexName[1:])
if err != nil {
return errors.Wrapf(err, "could not find index %s", indexName)
}
Expand Down Expand Up @@ -1280,7 +1281,7 @@ func TestRepartitioning(t *testing.T) {
}
sqlDB.Exec(t, fmt.Sprintf("ALTER TABLE %s RENAME TO %s", test.old.parsed.tableName, test.new.parsed.tableName))

testIndex, err := test.new.parsed.tableDesc.FindIndexWithName(test.index)
testIndex, err := catalog.MustFindIndexByName(test.new.parsed.tableDesc, test.index)
if err != nil {
t.Fatalf("%+v", err)
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/partitionccl/zone_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -312,7 +312,7 @@ func TestGenerateSubzoneSpans(t *testing.T) {
var actual []string
for _, span := range spans {
subzone := test.parsed.subzones[span.SubzoneIndex]
idx, err := test.parsed.tableDesc.FindIndexWithID(descpb.IndexID(subzone.IndexID))
idx, err := catalog.MustFindIndexByID(test.parsed.tableDesc, descpb.IndexID(subzone.IndexID))
if err != nil {
t.Fatalf("could not find index with ID %d: %+v", subzone.IndexID, err)
}
Expand Down Expand Up @@ -432,7 +432,7 @@ PARTITION p1 VALUES IN (DEFAULT));`)

// Find the temporary index corresponding to the new index.
tbl := desctestutils.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "public", "test")
newIndex, err := tbl.FindIndexWithName("idx")
newIndex, err := catalog.MustFindIndexByName(tbl, "idx")
if err != nil {
t.Fatal(err)
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/streamingccl/replicationtestutils/encoding.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ func EncodeKV(
var colMap catalog.TableColMap
for i, val := range pkeyVals {
datums = append(datums, nativeToDatum(t, val))
col, err := descr.FindColumnWithID(descpb.ColumnID(i + 1))
col, err := catalog.MustFindColumnByID(descr, descpb.ColumnID(i+1))
require.NoError(t, err)
colMap.Set(col.GetID(), col.Ordinal())
}
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/testccl/sqlccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ go_test(
"//pkg/settings/cluster",
"//pkg/spanconfig",
"//pkg/sql",
"//pkg/sql/catalog",
"//pkg/sql/catalog/descpb",
"//pkg/sql/catalog/desctestutils",
"//pkg/sql/gcjob",
Expand Down
3 changes: 2 additions & 1 deletion pkg/ccl/testccl/sqlccl/tenant_gc_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/spanconfig"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils"
"github.com/cockroachdb/cockroach/pkg/sql/gcjob"
Expand Down Expand Up @@ -325,7 +326,7 @@ func TestGCTableOrIndexWaitsForProtectedTimestamps(t *testing.T) {

tableDesc := desctestutils.TestingGetTableDescriptor(execCfg.DB, execCfg.Codec, "db", "public", "t")
tableID := tableDesc.GetID()
idx, err := tableDesc.FindIndexWithName("t_idx")
idx, err := catalog.MustFindIndexByName(tableDesc, "t_idx")
require.NoError(t, err)
indexID := idx.GetID()

Expand Down
1 change: 1 addition & 0 deletions pkg/jobs/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -117,6 +117,7 @@ go_test(
"//pkg/settings/cluster",
"//pkg/spanconfig",
"//pkg/sql",
"//pkg/sql/catalog",
"//pkg/sql/catalog/catalogkeys",
"//pkg/sql/catalog/colinfo",
"//pkg/sql/catalog/descpb",
Expand Down
4 changes: 2 additions & 2 deletions pkg/jobs/registry_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,12 +28,12 @@ import (
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/spanconfig"
"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/desctestutils"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/sql/isql"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
Expand All @@ -59,7 +59,7 @@ func writeColumnMutation(
column string,
m descpb.DescriptorMutation,
) {
col, err := tableDesc.FindColumnWithName(tree.Name(column))
col, err := catalog.MustFindColumnByName(tableDesc, column)
if err != nil {
t.Fatal(err)
}
Expand Down
1 change: 1 addition & 0 deletions pkg/server/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -164,6 +164,7 @@ go_library(
"//pkg/spanconfig/spanconfigsqltranslator",
"//pkg/spanconfig/spanconfigsqlwatcher",
"//pkg/sql",
"//pkg/sql/catalog",
"//pkg/sql/catalog/bootstrap",
"//pkg/sql/catalog/catalogkeys",
"//pkg/sql/catalog/catsessiondata",
Expand Down
5 changes: 3 additions & 2 deletions pkg/server/status.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/spanconfig"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
"github.com/cockroachdb/cockroach/pkg/sql/clusterunique"
Expand Down Expand Up @@ -2538,8 +2539,8 @@ func (s *systemStatusServer) HotRangesV2(
if _, _, idxID, err := s.sqlServer.execCfg.Codec.DecodeIndexPrefix(r.Desc.StartKey.AsRawKey()); err != nil {
log.Warningf(ctx, "cannot decode index prefix for range descriptor: %s: %v", r.Desc, err)
} else {
if index, err := desc.FindIndexWithID(descpb.IndexID(idxID)); err != nil {
log.Warningf(ctx, "cannot get index name for range descriptor: %s: %v", r.Desc, err)
if index := catalog.FindIndexByID(desc, descpb.IndexID(idxID)); index == nil {
log.Warningf(ctx, "cannot get index name for range descriptor: %s: index with ID %d not found", r.Desc, idxID)
} else {
indexName = index.GetName()
}
Expand Down
7 changes: 3 additions & 4 deletions pkg/sql/add_column.go
Original file line number Diff line number Diff line change
Expand Up @@ -207,10 +207,9 @@ func (p *planner) addColumnImpl(
}

if col.Virtual && !col.Nullable {
colName := tree.Name(col.Name)
newCol, err := n.tableDesc.FindColumnWithName(colName)
newCol, err := catalog.MustFindColumnByName(n.tableDesc, col.Name)
if err != nil {
return errors.NewAssertionErrorWithWrappedErrf(err, "failed to find newly added column %v", colName)
return errors.NewAssertionErrorWithWrappedErrf(err, "failed to find newly added column %v", col.Name)
}
if err := addNotNullConstraintMutationForCol(n.tableDesc, newCol); err != nil {
return err
Expand All @@ -223,7 +222,7 @@ func (p *planner) addColumnImpl(
func checkColumnDoesNotExist(
tableDesc catalog.TableDescriptor, name tree.Name,
) (isPublic bool, err error) {
col, _ := tableDesc.FindColumnWithName(name)
col := catalog.FindColumnByTreeName(tableDesc, name)
if col == nil {
return false, nil
}
Expand Down
11 changes: 3 additions & 8 deletions pkg/sql/alter_column_type.go
Original file line number Diff line number Diff line change
Expand Up @@ -203,12 +203,8 @@ func alterColumnTypeGeneral(

// Disallow ALTER COLUMN TYPE general for columns that have a check
// constraint.
for i := range tableDesc.Checks {
uses, err := tableDesc.CheckConstraintUsesColumn(tableDesc.Checks[i], col.GetID())
if err != nil {
return err
}
if uses {
for _, ck := range tableDesc.EnforcedCheckConstraints() {
if ck.CollectReferencedColumnIDs().Contains(col.GetID()) {
return colWithConstraintNotSupportedErr
}
}
Expand Down Expand Up @@ -264,8 +260,7 @@ func alterColumnTypeGeneral(
}

nameExists := func(name string) bool {
_, err := tableDesc.FindColumnWithName(tree.Name(name))
return err == nil
return catalog.FindColumnByName(tableDesc, name) != nil
}

shadowColName := tabledesc.GenerateUniqueName(col.GetName(), nameExists)
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/alter_index_visible.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,9 +53,9 @@ func (p *planner) AlterIndexVisible(
return newZeroNode(nil /* columns */), nil
}

// Check if the index actually exists. FindIndexWithName returns the first
// Check if the index actually exists. MustFindIndexByName returns the first
// catalog.Index in tableDesc.AllIndexes().
idx, err := tableDesc.FindIndexWithName(string(n.Index.Index))
idx, err := catalog.MustFindIndexByName(tableDesc, string(n.Index.Index))
if err != nil {
if n.IfExists {
// Nothing needed if no index exists and IfExists is true.
Expand Down
15 changes: 7 additions & 8 deletions pkg/sql/alter_primary_key.go
Original file line number Diff line number Diff line change
Expand Up @@ -133,7 +133,7 @@ func (p *planner) AlterPrimaryKey(
"use columns instead",
)
}
col, err := tableDesc.FindColumnWithName(elem.Column)
col, err := catalog.MustFindColumnByTreeName(tableDesc, elem.Column)
if err != nil {
return err
}
Expand Down Expand Up @@ -163,8 +163,7 @@ func (p *planner) AlterPrimaryKey(
}

nameExists := func(name string) bool {
_, err := tableDesc.FindIndexWithName(name)
return err == nil
return catalog.FindIndexByName(tableDesc, name) != nil
}

// Make a new index that is suitable to be a primary index.
Expand Down Expand Up @@ -362,7 +361,7 @@ func (p *planner) AlterPrimaryKey(
// as most of this code relies upon for correctness. This code will
// all be replaced by code in the declarative schema changer before
// too long where we'll model this all correctly.
newTempPrimaryIndex, err := tableDesc.FindIndexWithID(newPrimaryIndexDesc.ID + 1)
newTempPrimaryIndex, err := catalog.MustFindIndexByID(tableDesc, newPrimaryIndexDesc.ID+1)
if err != nil {
return errors.NewAssertionErrorWithWrappedErrf(err,
"failed to find newly created temporary index for backfill")
Expand Down Expand Up @@ -433,7 +432,7 @@ func (p *planner) AlterPrimaryKey(
if idx.IsUnique() {
for i := 0; i < idx.NumKeyColumns(); i++ {
colID := idx.GetKeyColumnID(i)
col, err := tableDesc.FindColumnWithID(colID)
col, err := catalog.MustFindColumnByID(tableDesc, colID)
if err != nil {
return false, err
}
Expand All @@ -451,7 +450,7 @@ func (p *planner) AlterPrimaryKey(
newPrimaryKeyColIDs := catalog.MakeTableColSet(newPrimaryIndexDesc.KeyColumnIDs...)
for i := 0; i < idx.NumKeySuffixColumns(); i++ {
colID := idx.GetKeySuffixColumnID(i)
col, err := tableDesc.FindColumnWithID(colID)
col, err := catalog.MustFindColumnByID(tableDesc, colID)
if err != nil {
return false, err
}
Expand Down Expand Up @@ -629,7 +628,7 @@ func (p *planner) shouldCreateIndexes(

// Validate the columns on the indexes
for idx, elem := range alterPKNode.Columns {
col, err := desc.FindColumnWithName(elem.Column)
col, err := catalog.MustFindColumnByTreeName(desc, elem.Column)
if err != nil {
return true, err
}
Expand Down Expand Up @@ -793,7 +792,7 @@ func setKeySuffixColumnIDsFromPrimary(
// toAdd.KeySuffixColumnIDs = append(toAdd.KeySuffixColumnIDs, colID)
// However, this functionality is not supported by the execution engine,
// so prevent it by returning an error.
col, err := table.FindColumnWithID(colID)
col, err := catalog.MustFindColumnByID(table, colID)
if err != nil {
return err
}
Expand Down
Loading

0 comments on commit f3195cf

Please sign in to comment.