diff --git a/pkg/ccl/changefeedccl/changefeed_stmt.go b/pkg/ccl/changefeedccl/changefeed_stmt.go index e537c79eba58..886091fb0039 100644 --- a/pkg/ccl/changefeedccl/changefeed_stmt.go +++ b/pkg/ccl/changefeedccl/changefeed_stmt.go @@ -328,6 +328,11 @@ func createChangefeedJobRecord( } var initialHighWater hlc.Timestamp evalTimestamp := func(s string) (hlc.Timestamp, error) { + if knobs, ok := p.ExecCfg().DistSQLSrv.TestingKnobs.Changefeed.(*TestingKnobs); ok { + if knobs != nil && knobs.OverrideCursor != nil { + s = knobs.OverrideCursor(&statementTime) + } + } asOfClause := tree.AsOfClause{Expr: tree.NewStrVal(s)} asOf, err := p.EvalAsOfTimestamp(ctx, asOfClause) if err != nil { diff --git a/pkg/ccl/changefeedccl/changefeed_test.go b/pkg/ccl/changefeedccl/changefeed_test.go index 2fdae40c9e16..873c7931b07c 100644 --- a/pkg/ccl/changefeedccl/changefeed_test.go +++ b/pkg/ccl/changefeedccl/changefeed_test.go @@ -666,6 +666,7 @@ func TestChangefeedCursor(t *testing.T) { // 'after', throw a couple sleeps around them. We round timestamps to // Microsecond granularity for Postgres compatibility, so make the // sleeps 10x that. + beforeInsert := s.Server.Clock().Now() sqlDB.Exec(t, `INSERT INTO foo VALUES (1, 'before')`) time.Sleep(10 * time.Microsecond) @@ -677,6 +678,32 @@ func TestChangefeedCursor(t *testing.T) { time.Sleep(10 * time.Microsecond) sqlDB.Exec(t, `INSERT INTO foo VALUES (2, 'after')`) + // The below function is currently used to test negative timestamp in cursor i.e of the form + // "-3us". + // Using this function we can calculate the difference with the time that was before + // the insert statement, which is set as the new cursor value inside createChangefeedJobRecord + calculateCursor := func(currentTime *hlc.Timestamp) string { + // Should convert to microseconds as that is the maximum precision we support + diff := (beforeInsert.WallTime - currentTime.WallTime) / 1000 + diffStr := strconv.FormatInt(diff, 10) + "us" + return diffStr + } + + knobs := s.TestingKnobs.DistSQL.(*execinfra.TestingKnobs).Changefeed.(*TestingKnobs) + knobs.OverrideCursor = calculateCursor + + // The "-3 days" is a placeholder here - it will be replaced with actual difference + // in createChangefeedJobRecord + fooInterval := feed(t, f, `CREATE CHANGEFEED FOR foo WITH cursor=$1`, "-3 days") + defer closeFeed(t, fooInterval) + assertPayloads(t, fooInterval, []string{ + `foo: [1]->{"after": {"a": 1, "b": "before"}}`, + `foo: [2]->{"after": {"a": 2, "b": "after"}}`, + }) + + // We do not need to override for the remaining cases + knobs.OverrideCursor = nil + fooLogical := feed(t, f, `CREATE CHANGEFEED FOR foo WITH cursor=$1`, tsLogical) defer closeFeed(t, fooLogical) assertPayloads(t, fooLogical, []string{ diff --git a/pkg/ccl/changefeedccl/testing_knobs.go b/pkg/ccl/changefeedccl/testing_knobs.go index f7a993da2340..5735189ed9fb 100644 --- a/pkg/ccl/changefeedccl/testing_knobs.go +++ b/pkg/ccl/changefeedccl/testing_knobs.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/mon" ) @@ -47,6 +48,12 @@ type TestingKnobs struct { ShouldReplan func(ctx context.Context, oldPlan, newPlan *sql.PhysicalPlan) bool // RaiseRetryableError is a knob used to possibly return an error. RaiseRetryableError func() error + + // This is currently used to test negative timestamp in cursor i.e of the form + // "-3us". Check TestChangefeedCursor for more info. This function needs to be in the + // knobs as current statement time will only be available once the create changefeed statement + // starts executing. + OverrideCursor func(currentTime *hlc.Timestamp) string } // ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface. diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go index 3cd8ff8cf2fd..b26732f73acb 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go @@ -947,7 +947,10 @@ func splitTrigger( // TODO(nvanbenschoten): this is a simple heuristic. If we had a cheap way to // determine the relative sizes of the LHS and RHS, we could be more // sophisticated here and always choose to scan the cheaper side. - emptyRHS, err := isGlobalKeyspaceEmpty(batch, &split.RightDesc) + emptyRHS, err := storage.MVCCIsSpanEmpty(ctx, batch, storage.MVCCIsSpanEmptyOptions{ + StartKey: split.RightDesc.StartKey.AsRawKey(), + EndKey: split.RightDesc.EndKey.AsRawKey(), + }) if err != nil { return enginepb.MVCCStats{}, result.Result{}, errors.Wrapf(err, "unable to determine whether right hand side of split is empty") @@ -978,21 +981,6 @@ func splitTrigger( var splitScansRightForStatsFirst = util.ConstantWithMetamorphicTestBool( "split-scans-right-for-stats-first", false) -// isGlobalKeyspaceEmpty returns whether the global keyspace of the provided -// range is entirely empty. The function returns false if the global keyspace -// contains at least one key. -func isGlobalKeyspaceEmpty(reader storage.Reader, d *roachpb.RangeDescriptor) (bool, error) { - span := d.KeySpan().AsRawSpanWithNoLocals() - iter := reader.NewMVCCIterator(storage.MVCCKeyIterKind, storage.IterOptions{UpperBound: span.EndKey}) - defer iter.Close() - iter.SeekGE(storage.MakeMVCCMetadataKey(span.Key)) - ok, err := iter.Valid() - if err != nil { - return false, err - } - return !ok /* empty */, nil -} - // makeScanStatsFn constructs a splitStatsScanFn for the provided post-split // range descriptor which computes the range's statistics. func makeScanStatsFn( diff --git a/pkg/kv/kvserver/batcheval/cmd_is_span_empty.go b/pkg/kv/kvserver/batcheval/cmd_is_span_empty.go index c2bbfa19ae1c..85a73a560bde 100644 --- a/pkg/kv/kvserver/batcheval/cmd_is_span_empty.go +++ b/pkg/kv/kvserver/batcheval/cmd_is_span_empty.go @@ -16,7 +16,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/errors" ) @@ -35,8 +34,6 @@ func IsSpanEmpty( isEmpty, err := storage.MVCCIsSpanEmpty(ctx, reader, storage.MVCCIsSpanEmptyOptions{ StartKey: args.Key, EndKey: args.EndKey, - StartTS: hlc.MinTimestamp, // beginning of time - EndTS: hlc.MaxTimestamp, // end of time }) if err != nil { return result.Result{}, errors.Wrap(err, "IsSpanEmpty") diff --git a/pkg/kv/kvserver/replica_evaluate.go b/pkg/kv/kvserver/replica_evaluate.go index 68507e3dd765..d3b555bc80ba 100644 --- a/pkg/kv/kvserver/replica_evaluate.go +++ b/pkg/kv/kvserver/replica_evaluate.go @@ -92,6 +92,7 @@ func optimizePuts( // don't need to see intents for this purpose since intents also have // provisional values that we will see. iter := reader.NewMVCCIterator(storage.MVCCKeyIterKind, storage.IterOptions{ + KeyTypes: storage.IterKeyTypePointsAndRanges, // We want to include maxKey in our scan. Since UpperBound is exclusive, we // need to set it to the key after maxKey. UpperBound: maxKey.Next(), diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 6802942f2b08..563576f3a576 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -1771,12 +1771,13 @@ func TestOptimizePuts(t *testing.T) { testCases := []struct { exKey roachpb.Key + exEndKey roachpb.Key // MVCC range key reqs []roachpb.Request expBlind []bool }{ // No existing keys, single put. { - nil, + nil, nil, []roachpb.Request{ &pArgs[0], }, @@ -1786,7 +1787,7 @@ func TestOptimizePuts(t *testing.T) { }, // No existing keys, nine puts. { - nil, + nil, nil, []roachpb.Request{ &pArgs[0], &pArgs[1], &pArgs[2], &pArgs[3], &pArgs[4], &pArgs[5], &pArgs[6], &pArgs[7], &pArgs[8], }, @@ -1796,7 +1797,7 @@ func TestOptimizePuts(t *testing.T) { }, // No existing keys, ten puts. { - nil, + nil, nil, []roachpb.Request{ &pArgs[0], &pArgs[1], &pArgs[2], &pArgs[3], &pArgs[4], &pArgs[5], &pArgs[6], &pArgs[7], &pArgs[8], &pArgs[9], }, @@ -1806,7 +1807,7 @@ func TestOptimizePuts(t *testing.T) { }, // Existing key at "0", ten conditional puts. { - roachpb.Key("0"), + roachpb.Key("0"), nil, []roachpb.Request{ &cpArgs[0], &cpArgs[1], &cpArgs[2], &cpArgs[3], &cpArgs[4], &cpArgs[5], &cpArgs[6], &cpArgs[7], &cpArgs[8], &cpArgs[9], }, @@ -1816,7 +1817,7 @@ func TestOptimizePuts(t *testing.T) { }, // Existing key at "0", ten init puts. { - roachpb.Key("0"), + roachpb.Key("0"), nil, []roachpb.Request{ &ipArgs[0], &ipArgs[1], &ipArgs[2], &ipArgs[3], &ipArgs[4], &ipArgs[5], &ipArgs[6], &ipArgs[7], &ipArgs[8], &ipArgs[9], }, @@ -1826,7 +1827,7 @@ func TestOptimizePuts(t *testing.T) { }, // Existing key at 11, mixed put types. { - roachpb.Key("11"), + roachpb.Key("11"), nil, []roachpb.Request{ &pArgs[0], &cpArgs[1], &pArgs[2], &cpArgs[3], &ipArgs[4], &ipArgs[5], &pArgs[6], &cpArgs[7], &pArgs[8], &ipArgs[9], }, @@ -1836,7 +1837,7 @@ func TestOptimizePuts(t *testing.T) { }, // Existing key at 00, ten puts, expect nothing blind. { - roachpb.Key("00"), + roachpb.Key("00"), nil, []roachpb.Request{ &pArgs[0], &pArgs[1], &pArgs[2], &pArgs[3], &pArgs[4], &pArgs[5], &pArgs[6], &pArgs[7], &pArgs[8], &pArgs[9], }, @@ -1846,7 +1847,7 @@ func TestOptimizePuts(t *testing.T) { }, // Existing key at 00, ten puts in reverse order, expect nothing blind. { - roachpb.Key("00"), + roachpb.Key("00"), nil, []roachpb.Request{ &pArgs[9], &pArgs[8], &pArgs[7], &pArgs[6], &pArgs[5], &pArgs[4], &pArgs[3], &pArgs[2], &pArgs[1], &pArgs[0], }, @@ -1856,7 +1857,7 @@ func TestOptimizePuts(t *testing.T) { }, // Existing key at 05, ten puts, expect first five puts are blind. { - roachpb.Key("05"), + roachpb.Key("05"), nil, []roachpb.Request{ &pArgs[0], &pArgs[1], &pArgs[2], &pArgs[3], &pArgs[4], &pArgs[5], &pArgs[6], &pArgs[7], &pArgs[8], &pArgs[9], }, @@ -1866,7 +1867,7 @@ func TestOptimizePuts(t *testing.T) { }, // Existing key at 09, ten puts, expect first nine puts are blind. { - roachpb.Key("09"), + roachpb.Key("09"), nil, []roachpb.Request{ &pArgs[0], &pArgs[1], &pArgs[2], &pArgs[3], &pArgs[4], &pArgs[5], &pArgs[6], &pArgs[7], &pArgs[8], &pArgs[9], }, @@ -1876,7 +1877,7 @@ func TestOptimizePuts(t *testing.T) { }, // No existing key, ten puts + inc + ten cputs. { - nil, + nil, nil, []roachpb.Request{ &pArgs[0], &pArgs[1], &pArgs[2], &pArgs[3], &pArgs[4], &pArgs[5], &pArgs[6], &pArgs[7], &pArgs[8], &pArgs[9], incArgs, &cpArgs[0], &cpArgs[1], &cpArgs[2], &cpArgs[3], &cpArgs[4], &cpArgs[5], &cpArgs[6], &cpArgs[7], &cpArgs[8], &cpArgs[9], @@ -1888,7 +1889,7 @@ func TestOptimizePuts(t *testing.T) { }, // Duplicate put at 11th key; should see ten puts. { - nil, + nil, nil, []roachpb.Request{ &pArgs[0], &pArgs[1], &pArgs[2], &pArgs[3], &pArgs[4], &pArgs[5], &pArgs[6], &pArgs[7], &pArgs[8], &pArgs[9], &pArgs[9], }, @@ -1898,7 +1899,7 @@ func TestOptimizePuts(t *testing.T) { }, // Duplicate cput at 11th key; should see ten puts. { - nil, + nil, nil, []roachpb.Request{ &pArgs[0], &pArgs[1], &pArgs[2], &pArgs[3], &pArgs[4], &pArgs[5], &pArgs[6], &pArgs[7], &pArgs[8], &pArgs[9], &cpArgs[9], }, @@ -1908,7 +1909,7 @@ func TestOptimizePuts(t *testing.T) { }, // Duplicate iput at 11th key; should see ten puts. { - nil, + nil, nil, []roachpb.Request{ &pArgs[0], &pArgs[1], &pArgs[2], &pArgs[3], &pArgs[4], &pArgs[5], &pArgs[6], &pArgs[7], &pArgs[8], &pArgs[9], &ipArgs[9], }, @@ -1918,7 +1919,7 @@ func TestOptimizePuts(t *testing.T) { }, // Duplicate cput at 10th key; should see ten cputs. { - nil, + nil, nil, []roachpb.Request{ &cpArgs[0], &cpArgs[1], &cpArgs[2], &cpArgs[3], &cpArgs[4], &cpArgs[5], &cpArgs[6], &cpArgs[7], &cpArgs[8], &cpArgs[9], &cpArgs[9], }, @@ -1926,14 +1927,45 @@ func TestOptimizePuts(t *testing.T) { true, true, true, true, true, true, true, true, true, true, false, }, }, + // Existing range key at 00-20, ten puts, expect no blind. + { + roachpb.Key("00"), roachpb.Key("20"), + []roachpb.Request{ + &pArgs[0], &pArgs[1], &pArgs[2], &pArgs[3], &pArgs[4], &pArgs[5], &pArgs[6], &pArgs[7], &pArgs[8], &pArgs[9], + }, + []bool{ + false, false, false, false, false, false, false, false, false, false, + }, + }, + // Existing range key at 05-08, ten puts, expect first five puts are blind. + { + roachpb.Key("05"), roachpb.Key("08"), + []roachpb.Request{ + &pArgs[0], &pArgs[1], &pArgs[2], &pArgs[3], &pArgs[4], &pArgs[5], &pArgs[6], &pArgs[7], &pArgs[8], &pArgs[9], + }, + []bool{ + true, true, true, true, true, false, false, false, false, false, + }, + }, + // Existing range key at 20-21, ten puts, expect all blind. + { + roachpb.Key("20"), roachpb.Key("21"), + []roachpb.Request{ + &pArgs[0], &pArgs[1], &pArgs[2], &pArgs[3], &pArgs[4], &pArgs[5], &pArgs[6], &pArgs[7], &pArgs[8], &pArgs[9], + }, + []bool{ + true, true, true, true, true, true, true, true, true, true, + }, + }, } for i, c := range testCases { - if c.exKey != nil { - if err := storage.MVCCPut(context.Background(), tc.engine, nil, c.exKey, - hlc.Timestamp{}, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("foo"), nil); err != nil { - t.Fatal(err) - } + if c.exEndKey != nil { + require.NoError(t, storage.MVCCDeleteRangeUsingTombstone(ctx, tc.engine, nil, + c.exKey, c.exEndKey, hlc.MinTimestamp, hlc.ClockTimestamp{}, nil, nil, false, 0, nil)) + } else if c.exKey != nil { + require.NoError(t, storage.MVCCPut(ctx, tc.engine, nil, c.exKey, + hlc.Timestamp{}, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("foo"), nil)) } batch := roachpb.BatchRequest{} for _, r := range c.reqs { @@ -1976,10 +2008,11 @@ func TestOptimizePuts(t *testing.T) { if !reflect.DeepEqual(blind, c.expBlind) { t.Errorf("%d: expected %+v; got %+v", i, c.expBlind, blind) } - if c.exKey != nil { - if err := tc.engine.ClearUnversioned(c.exKey); err != nil { - t.Fatal(err) - } + if c.exEndKey != nil { + require.NoError(t, tc.engine.ClearMVCCRangeKey(storage.MVCCRangeKey{ + StartKey: c.exKey, EndKey: c.exEndKey, Timestamp: hlc.MinTimestamp})) + } else if c.exKey != nil { + require.NoError(t, tc.engine.ClearUnversioned(c.exKey)) } } } diff --git a/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go b/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go index 52fe4abe2824..8b6fe8719dec 100644 --- a/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go +++ b/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go @@ -503,10 +503,10 @@ func (s *SQLTranslator) findDescendantLeafIDs( // findDescendantLeafIDsForDescriptor finds all leaf object IDs below the given // descriptor ID in the zone configuration hierarchy. Based on the descriptor // type, these are: -// - Database: IDs of all tables inside the database. -// - Table: ID of the table itself. -// - Schema/Type: Nothing, as schemas/types do not carry zone configurations and -// are not part of the zone configuration hierarchy. +// - Database: IDs of all tables inside the database. +// - Table: ID of the table itself. +// - Other: Nothing, as these do not carry zone configurations and +// are not part of the zone configuration hierarchy. func (s *SQLTranslator) findDescendantLeafIDsForDescriptor( ctx context.Context, id descpb.ID, txn *kv.Txn, descsCol *descs.Collection, ) (descpb.IDs, error) { @@ -521,30 +521,29 @@ func (s *SQLTranslator) findDescendantLeafIDsForDescriptor( return nil, nil // we're excluding this descriptor; nothing to do here } - switch desc.DescriptorType() { - case catalog.Type, catalog.Schema: - // There is nothing to do for {Type, Schema} descriptors as they are not - // part of the zone configuration hierarchy. - return nil, nil - case catalog.Table: + var db catalog.DatabaseDescriptor + switch t := desc.(type) { + case catalog.TableDescriptor: // Tables are leaf objects in the zone configuration hierarchy, so simply // return the ID. return descpb.IDs{id}, nil - case catalog.Database: - // Fallthrough. + case catalog.DatabaseDescriptor: + db = t default: - return nil, errors.AssertionFailedf("unknown descriptor type: %s", desc.DescriptorType()) + // There is nothing to do for non-table-or-database descriptors as they are + // not part of the zone configuration hierarchy. + return nil, nil } // There's nothing for us to do if the descriptor is offline or has been // dropped. - if desc.Offline() || desc.Dropped() { + if db.Offline() || db.Dropped() { return nil, nil } // Expand the database descriptor to all the tables inside it and return their // IDs. - tables, err := descsCol.GetAllTableDescriptorsInDatabase(ctx, txn, desc.GetID()) + tables, err := descsCol.GetAllTableDescriptorsInDatabase(ctx, txn, db) if err != nil { return nil, err } diff --git a/pkg/sql/alter_default_privileges.go b/pkg/sql/alter_default_privileges.go index 2d90ed10166c..eb3cbc5c8bbd 100644 --- a/pkg/sql/alter_default_privileges.go +++ b/pkg/sql/alter_default_privileges.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catprivilege" "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" @@ -76,13 +77,18 @@ func (p *planner) alterDefaultPrivileges( var schemaDescs []*schemadesc.Mutable for _, sc := range n.Schemas { - schemaDesc, err := p.Descriptors().GetMutableSchemaByName(ctx, p.txn, dbDesc, sc.Schema(), tree.SchemaLookupFlags{Required: true}) + immFlags := tree.SchemaLookupFlags{Required: true, AvoidLeased: true} + immSchema, err := p.Descriptors().GetImmutableSchemaByName(ctx, p.txn, dbDesc, sc.Schema(), immFlags) if err != nil { return nil, err } - mutableSchemaDesc, ok := schemaDesc.(*schemadesc.Mutable) - if !ok { - return nil, pgerror.Newf(pgcode.InvalidParameterValue, "%s is not a physical schema", schemaDesc.GetName()) + if immSchema.SchemaKind() != catalog.SchemaUserDefined { + return nil, pgerror.Newf(pgcode.InvalidParameterValue, "%s is not a physical schema", immSchema.GetName()) + } + mutFlags := tree.SchemaLookupFlags{Required: true} + mutableSchemaDesc, err := p.Descriptors().GetMutableSchemaByID(ctx, p.txn, immSchema.GetID(), mutFlags) + if err != nil { + return nil, err } schemaDescs = append(schemaDescs, mutableSchemaDesc) } diff --git a/pkg/sql/alter_function.go b/pkg/sql/alter_function.go index 776258c52814..21be2ea71c9e 100644 --- a/pkg/sql/alter_function.go +++ b/pkg/sql/alter_function.go @@ -17,7 +17,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/funcdesc" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemadesc" "github.com/cockroachdb/cockroach/pkg/sql/decodeusername" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" @@ -233,8 +232,9 @@ func (n *alterFunctionSetSchemaNode) startExec(params runParams) error { return err } - sc, err := params.p.Descriptors().GetMutableSchemaByName( - params.ctx, params.p.txn, db, string(n.n.NewSchemaName), tree.SchemaLookupFlags{Required: true}, + scFlags := tree.SchemaLookupFlags{Required: true, AvoidLeased: true} + sc, err := params.p.Descriptors().GetImmutableSchemaByName( + params.ctx, params.p.txn, db, string(n.n.NewSchemaName), scFlags, ) if err != nil { return err @@ -256,11 +256,16 @@ func (n *alterFunctionSetSchemaNode) startExec(params runParams) error { } } - targetSc := sc.(*schemadesc.Mutable) - if targetSc.GetID() == fnDesc.GetParentSchemaID() { + if sc.GetID() == fnDesc.GetParentSchemaID() { // No-op if moving to the same schema. return nil } + targetSc, err := params.p.Descriptors().GetMutableSchemaByID( + params.ctx, params.p.txn, sc.GetID(), params.p.CommonLookupFlagsRequired(), + ) + if err != nil { + return err + } // Check if there is a conflicting function exists. maybeExistingFuncObj := fnDesc.ToFuncObj() diff --git a/pkg/sql/alter_schema.go b/pkg/sql/alter_schema.go index 75e9d5734ba2..1945dfd88f04 100644 --- a/pkg/sql/alter_schema.go +++ b/pkg/sql/alter_schema.go @@ -57,10 +57,10 @@ func (p *planner) AlterSchema(ctx context.Context, n *tree.AlterSchema) (planNod if err != nil { return nil, err } - schema, err := p.Descriptors().GetSchemaByName(ctx, p.txn, db, + schema, err := p.Descriptors().GetImmutableSchemaByName(ctx, p.txn, db, string(n.Schema.SchemaName), tree.SchemaLookupFlags{ - Required: true, - RequireMutable: true, + Required: true, + AvoidLeased: true, }) if err != nil { return nil, err @@ -73,7 +73,11 @@ func (p *planner) AlterSchema(ctx context.Context, n *tree.AlterSchema) (planNod case catalog.SchemaPublic, catalog.SchemaVirtual, catalog.SchemaTemporary: return nil, pgerror.Newf(pgcode.InvalidSchemaName, "cannot modify schema %q", n.Schema.String()) case catalog.SchemaUserDefined: - desc := schema.(*schemadesc.Mutable) + flags := p.CommonLookupFlagsRequired() + desc, err := p.Descriptors().GetMutableSchemaByID(ctx, p.txn, schema.GetID(), flags) + if err != nil { + return nil, err + } // The user must be a superuser or the owner of the schema to modify it. hasAdmin, err := p.HasAdminRole(ctx) if err != nil { diff --git a/pkg/sql/catalog/accessor.go b/pkg/sql/catalog/accessor.go index b50e6ef9fcfd..2289495e0488 100644 --- a/pkg/sql/catalog/accessor.go +++ b/pkg/sql/catalog/accessor.go @@ -28,7 +28,7 @@ import ( // dependency injection for descriptor retrieval. type Accessor interface { - // GetDatabaseDesc looks up a database by name and returns its + // GetImmutableDatabaseByName looks up a database by name and returns its // descriptor. If the database is not found and required is true, // an error is returned; otherwise a nil reference is returned. // @@ -36,16 +36,10 @@ type Accessor interface { // accommodate the existing resolver.SchemaResolver interface (see #58228). // Use GetMutableDatabaseByName() and GetImmutableDatabaseByName() on // descs.Collection instead when possible. - GetDatabaseDesc( + GetImmutableDatabaseByName( ctx context.Context, txn *kv.Txn, dbName string, flags tree.DatabaseLookupFlags, ) (DatabaseDescriptor, error) - // GetSchemaByName returns true and a SchemaDescriptor object if the target schema - // exists under the target database. - GetSchemaByName( - ctx context.Context, txn *kv.Txn, db DatabaseDescriptor, scName string, flags tree.SchemaLookupFlags, - ) (SchemaDescriptor, error) - // GetObjectNamesAndIDs returns the list of all objects in the given // database and schema. // TODO(solon): when separate schemas are supported, this @@ -58,18 +52,4 @@ type Accessor interface { GetObjectNamesAndIDs( ctx context.Context, txn *kv.Txn, db DatabaseDescriptor, scName string, flags tree.DatabaseListFlags, ) (tree.TableNames, descpb.IDs, error) - - // GetObjectDesc looks up an object by name and returns both its - // descriptor and that of its parent database. If the object is not - // found and flags.required is true, an error is returned, otherwise - // a nil reference is returned. - // - // TODO(ajwerner): clarify the purpose of the transaction here. It's used in - // some cases for some lookups but not in others. For example, if a mutable - // descriptor is requested, it will be utilized however if an immutable - // descriptor is requested then it will only be used for its timestamp and to - // set the deadline. - GetObjectDesc( - ctx context.Context, txn *kv.Txn, db, schema, object string, flags tree.ObjectLookupFlags, - ) (ResolvedObjectPrefix, Descriptor, error) } diff --git a/pkg/sql/catalog/dbdesc/database_test.go b/pkg/sql/catalog/dbdesc/database_test.go index 48c230c37be7..faa67a6dfea1 100644 --- a/pkg/sql/catalog/dbdesc/database_test.go +++ b/pkg/sql/catalog/dbdesc/database_test.go @@ -275,7 +275,7 @@ func TestValidateCrossDatabaseReferences(t *testing.T) { cb.UpsertDescriptorEntry(schemadesc.NewBuilder(&schemaDesc).BuildImmutable()) } _ = cb.ForEachDescriptorEntry(func(desc catalog.Descriptor) error { - cb.UpsertNamespaceEntry(desc, desc.GetID()) + cb.UpsertNamespaceEntry(desc, desc.GetID(), desc.GetModificationTime()) return nil }) expectedErr := fmt.Sprintf("%s %q (%d): %s", desc.DescriptorType(), desc.GetName(), desc.GetID(), test.err) diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index de4c982b84f6..943c519b57a9 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -907,13 +907,26 @@ func FilterDescriptorState(desc Descriptor, flags tree.CommonLookupFlags) error err = errors.Errorf("%s %q is offline: %s", desc.DescriptorType(), desc.GetName(), desc.GetOfflineReason()) } return NewInactiveDescriptorError(err) - case desc.Adding(): - // Only table descriptors can be in the adding state. + case desc.Adding() && + // The ADD state is special. + // We don't want adding descriptors to be visible to DML queries, but we + // want them to be visible to schema changes: + // - when uncommitted we want them to be accessible by name for other + // schema changes, e.g. + // BEGIN; CREATE TABLE t ... ; ALTER TABLE t RENAME TO ...; + // should be possible. + // - when committed we want them to be accessible to their own schema + // change job, where they're referenced by ID. + // + // The AvoidCommittedAdding is set if and only if the lookup is by-name + // and prevents them from seeing committed adding descriptors. + !(flags.AvoidCommittedAdding && desc.IsUncommittedVersion() && (flags.AvoidLeased || flags.RequireMutable)) && + !(!flags.AvoidCommittedAdding && (desc.IsUncommittedVersion() || flags.AvoidLeased || flags.RequireMutable)): + // For the time being, only table descriptors can be in the adding state. return pgerror.WithCandidateCode(newAddingTableError(desc.(TableDescriptor)), pgcode.ObjectNotInPrerequisiteState) - default: - return nil } + return nil } // TableLookupFn is used to resolve a table from an ID, particularly when diff --git a/pkg/sql/catalog/descs/BUILD.bazel b/pkg/sql/catalog/descs/BUILD.bazel index 33efbdf5a31b..90e467143770 100644 --- a/pkg/sql/catalog/descs/BUILD.bazel +++ b/pkg/sql/catalog/descs/BUILD.bazel @@ -60,7 +60,7 @@ go_library( "//pkg/sql/sqlliveness", "//pkg/sql/sqlutil", "//pkg/sql/types", - "//pkg/util/encoding", + "//pkg/util", "//pkg/util/hlc", "//pkg/util/iterutil", "//pkg/util/log", @@ -78,7 +78,6 @@ go_test( srcs = [ "collection_test.go", "errors_test.go", - "helpers_test.go", "main_test.go", "txn_external_test.go", "txn_with_executor_datadriven_test.go", diff --git a/pkg/sql/catalog/descs/collection.go b/pkg/sql/catalog/descs/collection.go index 99756499fd34..f37a2b5c7cd0 100644 --- a/pkg/sql/catalog/descs/collection.go +++ b/pkg/sql/catalog/descs/collection.go @@ -13,9 +13,7 @@ package descs import ( - "bytes" "context" - "fmt" "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" @@ -33,7 +31,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/sqlliveness" - "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/iterutil" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -54,6 +51,7 @@ func newCollection( monitor *mon.BytesMonitor, ) *Collection { v := settings.Version.ActiveVersion(ctx) + cr := catkv.NewCatalogReader(codec, v, systemDatabase) return &Collection{ settings: settings, version: v, @@ -61,7 +59,7 @@ func newCollection( virtual: makeVirtualDescriptors(virtualSchemas), leased: makeLeasedDescriptors(leaseMgr), uncommitted: makeUncommittedDescriptors(monitor), - stored: catkv.MakeStoredCatalog(codec, v, systemDatabase, monitor), + stored: catkv.MakeStoredCatalog(cr, monitor), temporary: makeTemporaryDescriptors(settings, codec, temporarySchemaProvider), } } @@ -347,39 +345,32 @@ func newMutableSyntheticDescriptorAssertionError(id descpb.ID) error { } // GetAllDescriptors returns all descriptors visible by the transaction, -// first checking the Collection's cached descriptors for validity if validate -// is set to true before defaulting to a key-value scan, if necessary. func (tc *Collection) GetAllDescriptors(ctx context.Context, txn *kv.Txn) (nstree.Catalog, error) { if err := tc.stored.EnsureAllDescriptors(ctx, txn); err != nil { return nstree.Catalog{}, err } - var c nstree.MutableCatalog - _ = tc.stored.IterateCachedByID(func(desc catalog.Descriptor) error { - c.UpsertDescriptorEntry(desc) - return nil - }) - _ = tc.uncommitted.iterateUncommittedByID(func(desc catalog.Descriptor) error { - c.UpsertDescriptorEntry(desc) - return nil - }) - _ = tc.synthetic.iterateSyntheticByID(func(desc catalog.Descriptor) error { - c.UpsertDescriptorEntry(desc) - return nil - }) - descs := c.OrderedDescriptors() + var ids catalog.DescriptorIDSet + for _, iterator := range []func(func(desc catalog.Descriptor) error) error{ + tc.stored.IterateCachedByID, + tc.uncommitted.iterateUncommittedByID, + tc.synthetic.iterateSyntheticByID, + // TODO(postamar): include temporary descriptors? + } { + _ = iterator(func(desc catalog.Descriptor) error { + ids.Add(desc.GetID()) + return nil + }) + } flags := tree.CommonLookupFlags{ AvoidLeased: true, IncludeOffline: true, + IncludeDropped: true, } - if err := tc.finalizeDescriptors(ctx, txn, flags, descs, nil /* validationLevels */); err != nil { + // getDescriptorsByID must be used to ensure proper validation hydration etc. + descs, err := tc.getDescriptorsByID(ctx, txn, flags, ids.Ordered()...) + if err != nil { return nstree.Catalog{}, err } - if err := tc.hydrateDescriptors(ctx, txn, flags, descs); err != nil { - // If we ran into an error hydrating the types, that means that we - // have some sort of corrupted descriptor state. Rather than disable - // uses of GetAllDescriptors, just log the error. - log.Errorf(ctx, "%s", err.Error()) - } var ret nstree.MutableCatalog for _, desc := range descs { ret.UpsertDescriptorEntry(desc) @@ -388,7 +379,7 @@ func (tc *Collection) GetAllDescriptors(ctx context.Context, txn *kv.Txn) (nstre } // GetAllDatabaseDescriptors returns all database descriptors visible by the -// transaction. +// transaction, ordered by name. func (tc *Collection) GetAllDatabaseDescriptors( ctx context.Context, txn *kv.Txn, ) ([]catalog.DatabaseDescriptor, error) { @@ -414,48 +405,50 @@ func (tc *Collection) GetAllDatabaseDescriptors( } return nil }) - descs := make([]catalog.Descriptor, 0, m.Len()) + var ids catalog.DescriptorIDSet _ = m.IterateDatabasesByName(func(entry catalog.NameEntry) error { - descs = append(descs, entry.(catalog.Descriptor)) + ids.Add(entry.GetID()) return nil }) flags := tree.CommonLookupFlags{ AvoidLeased: true, IncludeOffline: true, + IncludeDropped: true, } - if err := tc.finalizeDescriptors(ctx, txn, flags, descs, nil /* validationLevels */); err != nil { - return nil, err - } - if err := tc.hydrateDescriptors(ctx, txn, flags, descs); err != nil { + // getDescriptorsByID must be used to ensure proper validation hydration etc. + descs, err := tc.getDescriptorsByID(ctx, txn, flags, ids.Ordered()...) + if err != nil { return nil, err } - dbDescs := make([]catalog.DatabaseDescriptor, len(descs)) - for i, desc := range descs { - dbDescs[i] = desc.(catalog.DatabaseDescriptor) + // Returned slice must be ordered by name. + m.Clear() + dbDescs := make([]catalog.DatabaseDescriptor, 0, len(descs)) + for _, desc := range descs { + m.Upsert(desc, desc.SkipNamespace()) } + _ = m.IterateDatabasesByName(func(entry catalog.NameEntry) error { + dbDescs = append(dbDescs, entry.(catalog.DatabaseDescriptor)) + return nil + }) return dbDescs, nil } // GetAllTableDescriptorsInDatabase returns all the table descriptors visible to // the transaction under the database with the given ID. func (tc *Collection) GetAllTableDescriptorsInDatabase( - ctx context.Context, txn *kv.Txn, dbID descpb.ID, + ctx context.Context, txn *kv.Txn, db catalog.DatabaseDescriptor, ) ([]catalog.TableDescriptor, error) { all, err := tc.GetAllDescriptors(ctx, txn) if err != nil { return nil, err } // Ensure the given ID does indeed belong to a database. - if found, _, err := tc.getDatabaseByID(ctx, txn, dbID, tree.DatabaseLookupFlags{ - AvoidLeased: false, - }); err != nil { - return nil, err - } else if !found { - return nil, sqlerrors.NewUndefinedDatabaseError(fmt.Sprintf("[%d]", dbID)) + if desc := all.LookupDescriptorEntry(db.GetID()); desc == nil || desc.DescriptorType() != catalog.Database { + return nil, sqlerrors.NewUndefinedDatabaseError(db.GetName()) } var ret []catalog.TableDescriptor for _, desc := range all.OrderedDescriptors() { - if desc.GetParentID() == dbID { + if desc.GetParentID() == db.GetID() { if table, ok := desc.(catalog.TableDescriptor); ok { ret = append(ret, table) } @@ -516,29 +509,20 @@ func (tc *Collection) GetObjectNamesAndIDs( return nil, nil, nil } - log.Eventf(ctx, "fetching list of objects for %q", dbDesc.GetName()) - prefix := catalogkeys.MakeObjectNameKey(tc.codec(), dbDesc.GetID(), schema.GetID(), "") - sr, err := txn.Scan(ctx, prefix, prefix.PrefixEnd(), 0) + c, err := tc.stored.ScanNamespaceForSchemaObjects(ctx, txn, dbDesc, schema) if err != nil { return nil, nil, err } - var tableNames tree.TableNames var tableIDs descpb.IDs - - for _, row := range sr { - _, tableName, err := encoding.DecodeUnsafeStringAscending(bytes.TrimPrefix( - row.Key, prefix), nil) - if err != nil { - return nil, nil, err - } - tn := tree.MakeTableNameWithSchema(tree.Name(dbDesc.GetName()), tree.Name(scName), tree.Name(tableName)) + _ = c.ForEachNamespaceEntry(func(e nstree.NamespaceEntry) error { + tn := tree.MakeTableNameWithSchema(tree.Name(dbDesc.GetName()), tree.Name(scName), tree.Name(e.GetName())) tn.ExplicitCatalog = flags.ExplicitPrefix tn.ExplicitSchema = flags.ExplicitPrefix tableNames = append(tableNames, tn) - tableIDs = append(tableIDs, descpb.ID(row.ValueInt())) - } - + tableIDs = append(tableIDs, e.GetID()) + return nil + }) return tableNames, tableIDs, nil } @@ -567,7 +551,7 @@ func (tc *Collection) AddSyntheticDescriptor(desc catalog.Descriptor) { } func (tc *Collection) codec() keys.SQLCodec { - return tc.stored.Codec + return tc.stored.Codec() } // NotifyOfDeletedDescriptor notifies the collection of the ID of a descriptor diff --git a/pkg/sql/catalog/descs/collection_test.go b/pkg/sql/catalog/descs/collection_test.go index 75615e2e9724..e75b9afd560d 100644 --- a/pkg/sql/catalog/descs/collection_test.go +++ b/pkg/sql/catalog/descs/collection_test.go @@ -522,7 +522,7 @@ CREATE TABLE test.schema.t(x INT); } descsCol.SkipValidationOnWrite() - return descsCol.WriteDesc(ctx, false, schemaDesc.(catalog.MutableDescriptor), txn) + return descsCol.WriteDesc(ctx, false, schemaDesc, txn) }), ) @@ -791,12 +791,12 @@ func TestDescriptorCache(t *testing.T) { if err != nil { return err } - schemaDesc.SchemaDesc().Name = "new_name" - schemaDesc.SchemaDesc().Version++ + schemaDesc.Name = "new_name" + schemaDesc.Version++ delete(dbDesc.Schemas, "schema") - dbDesc.Schemas["new_name"] = descpb.DatabaseDescriptor_SchemaInfo{ID: schemaDesc.GetID()} + dbDesc.Schemas["new_name"] = descpb.DatabaseDescriptor_SchemaInfo{ID: schemaDesc.ID} dbDesc.Version++ - err = descriptors.AddUncommittedDescriptor(ctx, schemaDesc.(catalog.MutableDescriptor)) + err = descriptors.AddUncommittedDescriptor(ctx, schemaDesc) if err != nil { return err } @@ -810,7 +810,7 @@ func TestDescriptorCache(t *testing.T) { return err } require.Len(t, schemas, 2) - require.Equal(t, schemaDesc.GetName(), schemas[schemaDesc.GetID()]) + require.Equal(t, schemaDesc.Name, schemas[schemaDesc.ID]) return nil })) }) @@ -935,10 +935,10 @@ func TestHydrateCatalog(t *testing.T) { } return nil }) - // Make a dummy table descriptor to replace the type descriptor. - tableDesc := tabledesc.NewBuilder(&descpb.TableDescriptor{ID: typeDescID}).BuildImmutable() + // Make a dummy database descriptor to replace the type descriptor. + dbDesc := dbdesc.NewBuilder(&descpb.DatabaseDescriptor{ID: typeDescID}).BuildImmutable() mutCat := nstree.MutableCatalog{Catalog: cat} - mutCat.UpsertDescriptorEntry(tableDesc) + mutCat.UpsertDescriptorEntry(dbDesc) return mutCat.Catalog } type testCase struct { @@ -949,7 +949,7 @@ func TestHydrateCatalog(t *testing.T) { {deleteDescriptor("typ"), "type \"[107]\" does not exist"}, {deleteDescriptor("db"), "database \"[104]\" does not exist"}, {deleteDescriptor("schema"), "unknown schema \"[106]\""}, - {replaceTypeDescWithNonTypeDesc, "found relation while looking for type [107]"}, + {replaceTypeDescWithNonTypeDesc, "referenced type ID 107: descriptor is a *dbdesc.immutable: unexpected descriptor type"}, } { require.NoError(t, sql.DescsTxn(ctx, &execCfg, func( ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, @@ -960,7 +960,8 @@ func TestHydrateCatalog(t *testing.T) { } // Hydration should fail when the given catalog is invalid. cat = tc.tamper(cat) - require.EqualError(t, descs.HydrateCatalog(ctx, cat), tc.expectedError) + mc := nstree.MutableCatalog{Catalog: cat} + require.EqualError(t, descs.HydrateCatalog(ctx, mc), tc.expectedError) return nil })) } @@ -973,7 +974,8 @@ func TestHydrateCatalog(t *testing.T) { if err != nil { return err } - require.NoError(t, descs.HydrateCatalog(ctx, cat)) + mc := nstree.MutableCatalog{Catalog: cat} + require.NoError(t, descs.HydrateCatalog(ctx, mc)) tbl := desctestutils.TestingGetTableDescriptor(txn.DB(), keys.SystemSQLCodec, "db", "schema", "table") tblDesc := cat.LookupDescriptorEntry(tbl.GetID()).(catalog.TableDescriptor) expected := types.UserDefinedTypeMetadata{ diff --git a/pkg/sql/catalog/descs/database.go b/pkg/sql/catalog/descs/database.go index 15520becca1c..cb5602239ff1 100644 --- a/pkg/sql/catalog/descs/database.go +++ b/pkg/sql/catalog/descs/database.go @@ -45,29 +45,16 @@ func (tc *Collection) GetImmutableDatabaseByName( return tc.getDatabaseByName(ctx, txn, name, flags) } -// GetDatabaseDesc implements the Accessor interface. -// -// TODO(ajwerner): This exists to support the SchemaResolver interface and -// should be removed or adjusted. -func (tc *Collection) GetDatabaseDesc( - ctx context.Context, txn *kv.Txn, name string, flags tree.DatabaseLookupFlags, -) (desc catalog.DatabaseDescriptor, err error) { - return tc.getDatabaseByName(ctx, txn, name, flags) -} - // getDatabaseByName returns a database descriptor with properties according to // the provided lookup flags. func (tc *Collection) getDatabaseByName( ctx context.Context, txn *kv.Txn, name string, flags tree.DatabaseLookupFlags, ) (catalog.DatabaseDescriptor, error) { - const alwaysLookupLeasedPublicSchema = false - found, desc, err := tc.getByName( - ctx, txn, nil, nil, name, flags.AvoidLeased, flags.RequireMutable, flags.AvoidSynthetic, - alwaysLookupLeasedPublicSchema, - ) + desc, err := tc.getDescriptorByName(ctx, txn, nil /* db */, nil /* sc */, name, flags, catalog.Database) if err != nil { return nil, err - } else if !found { + } + if desc == nil { if flags.Required { return nil, sqlerrors.NewUndefinedDatabaseError(name) } @@ -80,9 +67,6 @@ func (tc *Collection) getDatabaseByName( } return nil, nil } - if dropped, err := filterDescriptorState(db, flags.Required, flags); err != nil || dropped { - return nil, err - } return db, nil } diff --git a/pkg/sql/catalog/descs/descriptor.go b/pkg/sql/catalog/descs/descriptor.go index ada1681117f9..c0b73a00cf46 100644 --- a/pkg/sql/catalog/descs/descriptor.go +++ b/pkg/sql/catalog/descs/descriptor.go @@ -33,7 +33,6 @@ func (tc *Collection) GetMutableDescriptorsByID( ctx context.Context, txn *kv.Txn, ids ...descpb.ID, ) ([]catalog.MutableDescriptor, error) { flags := tree.CommonLookupFlags{ - Required: true, RequireMutable: true, IncludeOffline: true, IncludeDropped: true, @@ -82,20 +81,14 @@ func (tc *Collection) GetImmutableDescriptorByID( } // getDescriptorsByID returns a slice of descriptors by ID according to the -// provided lookup flags. Note that flags.Required is ignored, and an error is -// always returned if no descriptor with the ID exists. +// provided lookup flags. +// +// The Required flag is ignored and always overridden. func (tc *Collection) getDescriptorsByID( ctx context.Context, txn *kv.Txn, flags tree.CommonLookupFlags, ids ...descpb.ID, ) (descs []catalog.Descriptor, err error) { - defer func() { - if err == nil { - err = filterDescriptorsStates(descs, flags) - } - if err != nil { - descs = nil - } - }() - + // Override flags. + flags.Required = true log.VEventf(ctx, 2, "looking up descriptors for ids %v", ids) descs = make([]catalog.Descriptor, len(ids)) vls := make([]catalog.ValidationLevel, len(ids)) @@ -110,6 +103,7 @@ func (tc *Collection) getDescriptorsByID( } for _, fn := range []func(id descpb.ID) (catalog.Descriptor, catalog.ValidationLevel, error){ q.lookupVirtual, + q.lookupTemporary, q.lookupSynthetic, q.lookupUncommitted, q.lookupCached, @@ -158,6 +152,11 @@ func (tc *Collection) getDescriptorsByID( if err := tc.hydrateDescriptors(ctx, txn, flags, descs); err != nil { return nil, err } + for _, desc := range descs { + if err := catalog.FilterDescriptorState(desc, flags); err != nil { + return nil, err + } + } return descs, nil } @@ -174,8 +173,33 @@ type byIDLookupContext struct { func (q *byIDLookupContext) lookupVirtual( id descpb.ID, ) (catalog.Descriptor, catalog.ValidationLevel, error) { + // TODO(postamar): get rid of descriptorless public schemas + if id == keys.PublicSchemaID { + if q.flags.RequireMutable { + err := catalog.NewMutableAccessToVirtualSchemaError(schemadesc.GetPublicSchema()) + return nil, catalog.NoValidation, err + } + return schemadesc.GetPublicSchema(), validate.Write, nil + } desc, err := q.tc.virtual.getByID(q.ctx, id, q.flags.RequireMutable) - return desc, validate.Write, err + if err != nil || desc == nil { + return nil, catalog.NoValidation, err + } + return desc, validate.Write, nil +} + +func (q *byIDLookupContext) lookupTemporary( + id descpb.ID, +) (catalog.Descriptor, catalog.ValidationLevel, error) { + td := q.tc.temporary.getSchemaByID(id) + if td == nil { + return nil, catalog.NoValidation, nil + } + if q.flags.RequireMutable { + err := catalog.NewMutableAccessToVirtualSchemaError(schemadesc.GetPublicSchema()) + return nil, catalog.NoValidation, err + } + return td, validate.Write, nil } func (q *byIDLookupContext) lookupSynthetic( @@ -230,114 +254,231 @@ func (q *byIDLookupContext) lookupLeased( return desc, validate.ImmutableRead, nil } -// filterDescriptorsStates is a helper function for getDescriptorsByID. -func filterDescriptorsStates(descs []catalog.Descriptor, flags tree.CommonLookupFlags) error { - for _, desc := range descs { - // The first return value can safely be ignored, it will always be false - // because the required flag is set. - _, err := filterDescriptorState(desc, true /* required */, flags) - if err == nil { - continue - } - if desc.Adding() && (desc.IsUncommittedVersion() || flags.AvoidLeased || flags.RequireMutable) { - // This is a special case for tables in the adding state: Roughly speaking, - // we always need to resolve tables in the adding state by ID when they were - // newly created in the transaction for DDL statements and for some - // information queries (but not for ordinary name resolution for queries/ - // DML), but we also need to make these tables public in the schema change - // job in a separate transaction. - // TODO (lucy): We need something like an IncludeAdding flag so that callers - // can specify this behavior, instead of having the collection infer the - // desired behavior based on the flags (and likely producing unintended - // behavior). See the similar comment on etDescriptorByName, which covers - // the ordinary name resolution path as well as DDL statements. - continue - } - return err +// getDescriptorByName looks up a descriptor by name. +// +// The Required and AvoidCommittedAdding flags are ignored and overridden. +func (tc *Collection) getDescriptorByName( + ctx context.Context, + txn *kv.Txn, + db catalog.DatabaseDescriptor, + sc catalog.SchemaDescriptor, + name string, + flags tree.CommonLookupFlags, + requestedType catalog.DescriptorType, +) (catalog.Descriptor, error) { + mustBeVirtual, vd, err := tc.getVirtualDescriptorByName(sc, name, flags.RequireMutable, requestedType) + if mustBeVirtual || vd != nil || err != nil || (db == nil && sc != nil) { + return vd, err } - return nil + id, err := tc.getNonVirtualDescriptorID(ctx, txn, db, sc, name, flags) + if err != nil || id == descpb.InvalidID { + return nil, err + } + // When looking up descriptors by name, then descriptors in the adding state + // must be uncommitted to be visible (among other things). + flags.AvoidCommittedAdding = true + descs, err := tc.getDescriptorsByID(ctx, txn, flags, id) + if err != nil { + // Swallow error if the descriptor is dropped. + if errors.Is(err, catalog.ErrDescriptorDropped) { + return nil, nil + } + if errors.Is(err, catalog.ErrDescriptorNotFound) { + // Special case for temporary schemas, which can't always be resolved by + // ID alone. + if db != nil && sc == nil && isTemporarySchema(name) { + return schemadesc.NewTemporarySchema(name, id, db.GetID()), nil + } + // In all other cases, having an ID should imply having a descriptor. + return nil, errors.WithAssertionFailure(err) + } + return nil, err + } + desc := descs[0] + if desc.GetName() != name && !(desc.DescriptorType() == catalog.Schema && isTemporarySchema(name)) { + // TODO(postamar): make Collection aware of name ops + // + // We're prevented from removing this check until the Collection mediates + // name changes in the system.namespace table similarly to how it mediates + // descriptor changes in system.descriptor via the uncommitted descriptors + // layer and the WriteDescsToBatch method. + return nil, nil + } + return desc, nil +} + +type continueOrHalt bool + +const ( + continueLookups continueOrHalt = false + haltLookups continueOrHalt = true +) + +// getVirtualDescriptorByName looks up a virtual descriptor by name. +// +// Virtual descriptors do not always have an ID set, so they need to be treated +// separately from getNonVirtualDescriptorID. Also, validation, type hydration +// and state filtering are irrelevant here. +func (tc *Collection) getVirtualDescriptorByName( + sc catalog.SchemaDescriptor, + name string, + isMutableRequired bool, + requestedType catalog.DescriptorType, +) (continueOrHalt, catalog.Descriptor, error) { + objFlags := tree.ObjectLookupFlags{ + CommonLookupFlags: tree.CommonLookupFlags{ + RequireMutable: isMutableRequired, + }, + } + switch requestedType { + case catalog.Schema: + if vs := tc.virtual.getSchemaByName(name); vs != nil { + return haltLookups, vs, nil + } + case catalog.Type: + objFlags.DesiredObjectKind = tree.TypeObject + fallthrough + case catalog.Table: + isVirtual, vd, err := tc.virtual.getObjectByName(sc.GetName(), name, objFlags) + if isVirtual || vd != nil || err != nil { + return haltLookups, vd, err + } + } + return continueLookups, nil, nil } -func (tc *Collection) getByName( +// getNonVirtualDescriptorID looks up a non-virtual descriptor ID by name by +// going through layers in sequence. +// +// All flags except AvoidLeased, RequireMutable and AvoidSynthetic are ignored. +func (tc *Collection) getNonVirtualDescriptorID( ctx context.Context, txn *kv.Txn, db catalog.DatabaseDescriptor, sc catalog.SchemaDescriptor, name string, - avoidLeased, mutable, avoidSynthetic bool, - alwaysLookupLeasedPublicSchema bool, // passed through to getSchemaByName -) (found bool, desc catalog.Descriptor, err error) { + flags tree.CommonLookupFlags, +) (descpb.ID, error) { + flags = tree.CommonLookupFlags{ + AvoidLeased: flags.AvoidLeased, + RequireMutable: flags.RequireMutable, + AvoidSynthetic: flags.AvoidSynthetic, + } var parentID, parentSchemaID descpb.ID + var isSchema bool if db != nil { - if sc == nil { - // Schema descriptors are handled in a special way, see getSchemaByName - // function declaration for details. - return getSchemaByName( - ctx, tc, txn, db, name, avoidLeased, mutable, avoidSynthetic, - alwaysLookupLeasedPublicSchema, - ) + parentID = db.GetID() + if sc != nil { + parentSchemaID = sc.GetID() + } else { + isSchema = true } - parentID, parentSchemaID = db.GetID(), sc.GetID() } - if sd := tc.synthetic.getSyntheticByName(parentID, parentSchemaID, name); sd != nil && !avoidSynthetic { - if mutable { - return false, nil, newMutableSyntheticDescriptorAssertionError(sd.GetID()) + // Define the lookup functions for each layer. + lookupTemporarySchemaID := func() (continueOrHalt, descpb.ID, error) { + if !isSchema || !isTemporarySchema(name) { + return continueLookups, descpb.InvalidID, nil + } + avoidFurtherLookups, td := tc.temporary.getSchemaByName(ctx, parentID, name) + if td != nil { + return haltLookups, td.GetID(), nil + } + if avoidFurtherLookups { + return haltLookups, descpb.InvalidID, nil } - return true, sd, nil + return continueLookups, descpb.InvalidID, nil } - - desc = tc.uncommitted.getUncommittedByName(parentID, parentSchemaID, name) - - // Look up descriptor in store cache. - if desc == nil { - if cd := tc.stored.GetCachedByName(parentID, parentSchemaID, name); cd != nil { - desc = cd - log.VEventf(ctx, 2, "found cached descriptor %d", desc.GetID()) + lookupSchemaID := func() (continueOrHalt, descpb.ID, error) { + if !isSchema { + return continueLookups, descpb.InvalidID, nil } + // Getting a schema by name uses a special resolution path which can avoid + // a namespace lookup because the mapping of database to schema is stored on + // the database itself. This is an important optimization in the case when + // the schema does not exist. + // + if !db.HasPublicSchemaWithDescriptor() && name == catconstants.PublicSchemaName { + return haltLookups, keys.PublicSchemaID, nil + } + if id := db.GetSchemaID(name); id != descpb.InvalidID { + return haltLookups, id, nil + } + if isTemporarySchema(name) { + // Look for temporary schema IDs in other layers. + return continueLookups, descpb.InvalidID, nil + } + return haltLookups, descpb.InvalidID, nil } - - // Look up leased descriptor. - if desc == nil && !avoidLeased && !mutable && !lease.TestingTableLeasesAreDisabled() { - leasedDesc, shouldReadFromStore, err := tc.leased.getByName(ctx, tc.deadlineHolder(txn), parentID, parentSchemaID, name) - if err != nil { - return false, nil, err + lookupSyntheticID := func() (continueOrHalt, descpb.ID, error) { + if flags.AvoidSynthetic { + return continueLookups, descpb.InvalidID, nil } - if !shouldReadFromStore { - return leasedDesc != nil, leasedDesc, nil + if sd := tc.synthetic.getSyntheticByName(parentID, parentSchemaID, name); sd != nil { + return haltLookups, sd.GetID(), nil } + return continueLookups, descpb.InvalidID, nil } - - // Look up descriptor in storage. - if desc == nil { - desc, err = tc.stored.GetByName(ctx, txn, parentID, parentSchemaID, name) - if err != nil || desc == nil { - return false, nil, err + lookupUncommittedID := func() (continueOrHalt, descpb.ID, error) { + if ud := tc.uncommitted.getUncommittedByName(parentID, parentSchemaID, name); ud != nil { + return haltLookups, ud.GetID(), nil } + return continueLookups, descpb.InvalidID, nil } - - // At this point the descriptor exists. - // Finalize it and return it. - { - ret := []catalog.Descriptor{desc} - flags := tree.CommonLookupFlags{RequireMutable: mutable} - if err = tc.finalizeDescriptors(ctx, txn, flags, ret, nil /* validationLevels */); err != nil { - return false, nil, err + lookupStoreCacheID := func() (continueOrHalt, descpb.ID, error) { + if cd := tc.stored.GetCachedByName(parentID, parentSchemaID, name); cd != nil { + return haltLookups, cd.GetID(), nil } - desc = ret[0] - return desc != nil, desc, err + return continueLookups, descpb.InvalidID, nil } + lookupLeasedID := func() (continueOrHalt, descpb.ID, error) { + if flags.AvoidLeased || flags.RequireMutable || lease.TestingTableLeasesAreDisabled() { + return continueLookups, descpb.InvalidID, nil + } + if isSchema && isTemporarySchema(name) { + return continueLookups, descpb.InvalidID, nil + } + ld, shouldReadFromStore, err := tc.leased.getByName( + ctx, tc.deadlineHolder(txn), parentID, parentSchemaID, name, + ) + if err != nil { + return haltLookups, descpb.InvalidID, err + } + if shouldReadFromStore { + return continueLookups, descpb.InvalidID, nil + } + return haltLookups, ld.GetID(), nil + } + lookupStoredID := func() (continueOrHalt, descpb.ID, error) { + id, err := tc.stored.LookupDescriptorID(ctx, txn, parentID, parentSchemaID, name) + return haltLookups, id, err + } + + // Iterate through each layer until an ID is conclusively found or not, or an + // error is thrown. + for _, fn := range []func() (continueOrHalt, descpb.ID, error){ + lookupTemporarySchemaID, + lookupSchemaID, + lookupSyntheticID, + lookupUncommittedID, + lookupStoreCacheID, + lookupLeasedID, + lookupStoredID, + } { + isDone, id, err := fn() + if err != nil { + return descpb.InvalidID, err + } + if isDone { + return id, nil + } + } + return descpb.InvalidID, nil } // finalizeDescriptors ensures that all descriptors are (1) properly validated // and (2) if mutable descriptors are requested, these are present in the // uncommitted descriptors layer. -// Known validation levels can optionally be provided via validationLevels. -// If none are provided, finalizeDescriptors seeks them out in the appropriate -// layer (stored or uncommitted). - -// nil safe defaults are used instead. In any case, after validation is -// performed the known levels are raised accordingly. func (tc *Collection) finalizeDescriptors( ctx context.Context, txn *kv.Txn, @@ -345,22 +486,6 @@ func (tc *Collection) finalizeDescriptors( descs []catalog.Descriptor, validationLevels []catalog.ValidationLevel, ) error { - if validationLevels == nil { - validationLevels = make([]catalog.ValidationLevel, len(descs)) - for i, desc := range descs { - if tc.uncommitted.getUncommittedByID(desc.GetID()) != nil { - // Uncommitted descriptors should, by definition, already have been - // validated at least at the MutableRead level. This effectively - // excludes them from being validated again right now. - // - // In any case, they will be fully validated when the transaction - // commits. - validationLevels[i] = validate.MutableRead - } else { - validationLevels[i] = tc.stored.GetValidationLevelByID(desc.GetID()) - } - } - } if len(validationLevels) != len(descs) { return errors.AssertionFailedf( "len(validationLevels) = %d should be equal to len(descs) = %d", @@ -398,38 +523,6 @@ func (tc *Collection) finalizeDescriptors( return nil } -// hydrateDescriptors ensures that the descriptors in the slice are hydrated. -// -// Callers expect the descriptors to come back hydrated. -// In practice, array types here are not hydrated, and that's a bummer. -// Nobody presently is upset about it, but it's not a good thing. -// Ideally we'd have a clearer contract regarding hydration and the values -// stored in the various maps inside the collection. One might want to -// store only hydrated values in the various maps. This turns out to be -// somewhat tricky because we'd need to make sure to properly re-hydrate -// all the relevant descriptors when a type descriptor change. Leased -// descriptors are at least as tricky, plus, there we have a cache that -// works relatively well. -// -// TODO(ajwerner): Sort out the hydration mess; define clearly what is -// hydrated where and test the API boundary accordingly. -func (tc *Collection) hydrateDescriptors( - ctx context.Context, txn *kv.Txn, flags tree.CommonLookupFlags, descs []catalog.Descriptor, -) error { - for i, desc := range descs { - hd, isHydratable := desc.(catalog.HydratableDescriptor) - if !isHydratable { - continue - } - var err error - descs[i], err = tc.hydrateTypesInDescWithOptions(ctx, txn, hd, flags.IncludeOffline, flags.AvoidLeased) - if err != nil { - return err - } - } - return nil -} - func (tc *Collection) deadlineHolder(txn *kv.Txn) deadlineHolder { if tc.maxTimestampBoundDeadlineHolder.maxTimestampBound.IsEmpty() { return txn @@ -437,100 +530,6 @@ func (tc *Collection) deadlineHolder(txn *kv.Txn) deadlineHolder { return &tc.maxTimestampBoundDeadlineHolder } -// Getting a schema by name uses a special resolution path which can avoid -// a namespace lookup because the mapping of database to schema is stored on -// the database itself. This is an important optimization in the case when -// the schema does not exist. -// -// TODO(ajwerner): Understand and rationalize the namespace lookup given the -// schema lookup by ID path only returns descriptors owned by this session. -// -// The alwaysLookupLeasedPublicSchema parameter indicates that a missing public -// schema entry in the database descriptor should not be interpreted to -// mean that the public schema is the synthetic public schema, and, instead -// the public schema should be looked up via the lease manager by name. -// This is a workaround activated during the public schema migration to -// avoid a situation where the database does not know about the new public -// schema but the table in the lease manager does. -// -// TODO(ajwerner): Remove alwaysLookupLeasedPublicSchema in 22.2. -func getSchemaByName( - ctx context.Context, - tc *Collection, - txn *kv.Txn, - db catalog.DatabaseDescriptor, - name string, - avoidLeased, mutable, avoidSynthetic bool, - alwaysLookupLeasedPublicSchema bool, -) (bool, catalog.Descriptor, error) { - if !db.HasPublicSchemaWithDescriptor() && name == tree.PublicSchema { - // TODO(ajwerner): Remove alwaysLookupLeasedPublicSchema in 22.2. - if alwaysLookupLeasedPublicSchema { - desc, _, err := tc.leased.getByName(ctx, txn, db.GetID(), 0, catconstants.PublicSchemaName) - if err != nil { - return false, desc, err - } - return true, desc, nil - } - return true, schemadesc.GetPublicSchema(), nil - } - if sc := tc.virtual.getSchemaByName(name); sc != nil { - return true, sc, nil - } - if isTemporarySchema(name) { - if isDone, sc := tc.temporary.getSchemaByName(ctx, db.GetID(), name); sc != nil || isDone { - return sc != nil, sc, nil - } - scID, err := tc.stored.LookupDescriptorID(ctx, txn, db.GetID(), keys.RootNamespaceID, name) - if err != nil || scID == descpb.InvalidID { - return false, nil, err - } - return true, schemadesc.NewTemporarySchema(name, scID, db.GetID()), nil - } - if id := db.GetSchemaID(name); id != descpb.InvalidID { - // TODO(ajwerner): Fill in flags here or, more likely, get rid of - // it on this path. - sc, err := tc.getSchemaByID(ctx, txn, id, tree.SchemaLookupFlags{ - RequireMutable: mutable, - AvoidLeased: avoidLeased, - AvoidSynthetic: avoidSynthetic, - }) - if errors.Is(err, catalog.ErrDescriptorDropped) { - err = nil - } - return sc != nil, sc, err - } - return false, nil, nil -} - func isTemporarySchema(name string) bool { return strings.HasPrefix(name, catconstants.PgTempSchemaName) } - -// filterDescriptorState wraps the more general catalog function to swallow -// the error if the descriptor is being dropped and the descriptor is not -// required. In that case, dropped will be true. A return value of false, nil -// means this descriptor is okay given the flags. -// TODO (lucy): We would like the ByID methods to ignore the Required flag and -// unconditionally return an error for dropped descriptors if IncludeDropped is -// not set, so we can't just pass the flags passed into the methods into this -// function, hence the boolean argument. This is the only user of -// catalog.FilterDescriptorState which needs to pass in nontrivial flags, at -// time of writing, so we should clean up the interface around this bit of -// functionality. -func filterDescriptorState( - desc catalog.Descriptor, required bool, flags tree.CommonLookupFlags, -) (dropped bool, _ error) { - flags = tree.CommonLookupFlags{ - Required: required, - IncludeOffline: flags.IncludeOffline, - IncludeDropped: flags.IncludeDropped, - } - if err := catalog.FilterDescriptorState(desc, flags); err != nil { - if required || !errors.Is(err, catalog.ErrDescriptorDropped) { - return false, err - } - return true, nil - } - return false, nil -} diff --git a/pkg/sql/catalog/descs/dist_sql_type_resolver.go b/pkg/sql/catalog/descs/dist_sql_type_resolver.go index 2b3a6fe74d54..c4b53b995d6d 100644 --- a/pkg/sql/catalog/descs/dist_sql_type_resolver.go +++ b/pkg/sql/catalog/descs/dist_sql_type_resolver.go @@ -66,10 +66,7 @@ func (dt *DistSQLTypeResolver) ResolveTypeByOID( func (dt *DistSQLTypeResolver) GetTypeDescriptor( ctx context.Context, id descpb.ID, ) (tree.TypeName, catalog.TypeDescriptor, error) { - flags := tree.CommonLookupFlags{ - Required: true, - } - descs, err := dt.descriptors.getDescriptorsByID(ctx, dt.txn, flags, id) + descs, err := dt.descriptors.getDescriptorsByID(ctx, dt.txn, tree.CommonLookupFlags{}, id) if err != nil { return tree.TypeName{}, nil, err } @@ -79,14 +76,6 @@ func (dt *DistSQLTypeResolver) GetTypeDescriptor( // User-defined type. typeDesc = t case catalog.TableDescriptor: - // If we find a table descriptor when we were expecting a type descriptor, - // we return the implicitly-created type descriptor that is created for each - // table. Make sure that we hydrate the table ahead of time, since we expect - // that the table's types are fully hydrated below. - t, err = dt.descriptors.hydrateTypesInTableDesc(ctx, dt.txn, t) - if err != nil { - return tree.TypeName{}, nil, err - } typeDesc, err = typedesc.CreateImplicitRecordTypeFromTableDesc(t) if err != nil { return tree.TypeName{}, nil, err diff --git a/pkg/sql/catalog/descs/function.go b/pkg/sql/catalog/descs/function.go index d87e3c5c6136..b960e93a19ed 100644 --- a/pkg/sql/catalog/descs/function.go +++ b/pkg/sql/catalog/descs/function.go @@ -55,15 +55,9 @@ func (tc *Collection) getFunctionByID( } return nil, err } - fn, ok := descs[0].(catalog.FunctionDescriptor) if !ok { return nil, errors.Wrapf(tree.ErrFunctionUndefined, "function %d does not exist", fnID) } - - hydrated, err := tc.hydrateTypesInDescWithOptions(ctx, txn, fn, flags.IncludeOffline, flags.AvoidLeased) - if err != nil { - return nil, err - } - return hydrated.(catalog.FunctionDescriptor), nil + return fn, nil } diff --git a/pkg/sql/catalog/descs/helpers_test.go b/pkg/sql/catalog/descs/helpers_test.go deleted file mode 100644 index faa449615d71..000000000000 --- a/pkg/sql/catalog/descs/helpers_test.go +++ /dev/null @@ -1,13 +0,0 @@ -// Copyright 2022 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package descs - -var HydrateCatalog = hydrateCatalog diff --git a/pkg/sql/catalog/descs/hydrate.go b/pkg/sql/catalog/descs/hydrate.go index 29cfe37db2f2..fbb43b8df048 100644 --- a/pkg/sql/catalog/descs/hydrate.go +++ b/pkg/sql/catalog/descs/hydrate.go @@ -12,369 +12,166 @@ package descs import ( "context" - "fmt" - "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/funcdesc" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/hydrateddesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/nstree" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemadesc" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" - "github.com/cockroachdb/errors" + "github.com/cockroachdb/cockroach/pkg/util" ) -// hydrateTypesInTableDesc installs user defined type metadata in all types.T -// present in the input TableDescriptor. See hydrateTypesInDescWithOptions. -func (tc *Collection) hydrateTypesInTableDesc( - ctx context.Context, txn *kv.Txn, desc catalog.TableDescriptor, -) (catalog.TableDescriptor, error) { - ret, err := tc.hydrateTypesInDescWithOptions(ctx, - txn, - desc, - false, /* includeOffline */ - false /*avoidLeased*/) - if err != nil { - return nil, err - } - return ret.(catalog.TableDescriptor), nil -} - -// hydrateTypesInDescWithOptions installs user defined type metadata in all -// types.T present in the input Descriptor. It always returns the same type of -// Descriptor that was passed in. It ensures that immutable Descriptors are not -// modified during the process of metadata installation. Dropped descriptors do -// not get hydrated. Optionally, when hydrating types we can include offline -// descriptors and avoid leasing depending on the context. -func (tc *Collection) hydrateTypesInDescWithOptions( - ctx context.Context, - txn *kv.Txn, - desc catalog.HydratableDescriptor, - includeOffline bool, - avoidLeased bool, -) (catalog.HydratableDescriptor, error) { - if desc.Dropped() { - return desc, nil - } - // If there aren't any user defined types in the descriptor, then return - // early. - if !desc.ContainsUserDefinedTypes() { - return desc, nil - } - - // It is safe to hydrate directly into Mutable since it is not shared. When - // hydrating mutable descriptors, use the mutable access method to access - // types. - sc, _ := desc.(catalog.SchemaDescriptor) - switch t := desc.(type) { - case *tabledesc.Mutable: - return desc, typedesc.HydrateTypesInTableDescriptor(ctx, t.TableDesc(), getMutableTypeLookupFunc(tc, txn, sc)) - case *schemadesc.Mutable: - return desc, typedesc.HydrateTypesInSchemaDescriptor(ctx, t.SchemaDesc(), getMutableTypeLookupFunc(tc, txn, sc)) - case *funcdesc.Mutable: - return desc, typedesc.HydrateTypesInFunctionDescriptor(ctx, t.FuncDesc(), getMutableTypeLookupFunc(tc, txn, sc)) - default: - // Utilize the cache of hydrated tables if we have one and this descriptor - // was leased. - // TODO(ajwerner): Consider surfacing the mechanism used to retrieve the - // descriptor up to this layer. - if tc.canUseHydratedDescriptorCache(desc.GetID()) { - hydrated, err := tc.hydrated.GetHydratedDescriptor( - ctx, t, getImmutableTypeLookupFunc(tc, txn, includeOffline, avoidLeased, sc), - ) - if err != nil { - return nil, err - } - if hydrated != nil { - return hydrated, nil - } +// hydrateDescriptors installs user defined type metadata in all types.T present +// in the descriptors slice. +// +// Hydration is not thread-safe while immutable descriptors are, therefore this +// method will replace un-hydrated immutable descriptors in the slice with +// hydrated copies. Mutable descriptors are not thread-safe and so are hydrated +// in-place. Dropped descriptors do not get hydrated. +// +// Optionally, when hydrating we can include offline descriptors and avoid +// leasing depending on the context. This is set via the flags. +// +// Collection method callers expect the descriptors to come back hydrated. +// In practice, array types here are not hydrated, and that's a bummer. +// Nobody presently is upset about it, but it's not a good thing. +// Ideally we'd have a clearer contract regarding hydration and the values +// stored in the various maps inside the collection. One might want to +// store only hydrated values in the various maps. This turns out to be +// somewhat tricky because we'd need to make sure to properly re-hydrate +// all the relevant descriptors when a type descriptor change. Leased +// descriptors are at least as tricky, plus, there we have a cache that +// works relatively well. +// +// TODO(ajwerner): Sort out the hydration mess; define clearly what is +// hydrated where and test the API boundary accordingly. +func (tc *Collection) hydrateDescriptors( + ctx context.Context, txn *kv.Txn, flags tree.CommonLookupFlags, descs []catalog.Descriptor, +) error { + + var hydratableMutableIndexes, hydratableImmutableIndexes util.FastIntSet + for i, desc := range descs { + if desc == nil || !hydrateddesc.IsHydratable(desc) { + continue + } + if _, ok := desc.(catalog.MutableDescriptor); ok { + hydratableMutableIndexes.Add(i) + } else { + hydratableImmutableIndexes.Add(i) } - // The cache decided not to give back a hydrated descriptor, likely - // because either we've modified the table or one of the types or because - // this transaction has a stale view of one of the relevant descriptors. - // Proceed to hydrating a fresh copy. } - // ImmutableTableDescriptors need to be copied before hydration, because - // they are potentially read by multiple threads. - getType := getImmutableTypeLookupFunc(tc, txn, includeOffline, avoidLeased, sc) - switch t := desc.(type) { - case catalog.TableDescriptor: - mut := t.NewBuilder().BuildExistingMutable().(*tabledesc.Mutable) - if err := typedesc.HydrateTypesInTableDescriptor(ctx, mut.TableDesc(), getType); err != nil { - return nil, err - } - return mut.ImmutableCopy().(catalog.TableDescriptor), nil - case catalog.SchemaDescriptor: - mut := t.NewBuilder().BuildExistingMutable().(*schemadesc.Mutable) - if err := typedesc.HydrateTypesInSchemaDescriptor(ctx, mut.SchemaDesc(), getType); err != nil { - return nil, err - } - return mut.ImmutableCopy().(catalog.SchemaDescriptor), nil - case catalog.FunctionDescriptor: - mut := t.NewBuilder().BuildExistingMutable().(*funcdesc.Mutable) - if err := typedesc.HydrateTypesInFunctionDescriptor(ctx, mut.FuncDesc(), getType); err != nil { - return nil, err + // Hydrate mutable hydratable descriptors of the slice in-place. + if !hydratableMutableIndexes.Empty() { + typeFn := makeMutableTypeLookupFunc(tc, txn, descs) + for _, i := range hydratableMutableIndexes.Ordered() { + if err := hydrateddesc.Hydrate(ctx, descs[i], typeFn); err != nil { + return err + } } - return mut.ImmutableCopy().(catalog.FunctionDescriptor), nil } - return desc, nil -} - -// HydrateGivenDescriptors installs type metadata in the types present for all -// table descriptors in the slice of descriptors. It is exported so resolution -// on sets of descriptors can hydrate a set of descriptors (i.e. on BACKUPs). -func HydrateGivenDescriptors(ctx context.Context, descs []catalog.Descriptor) error { - // Collect the needed information to set up metadata in those types. - dbDescs := make(map[descpb.ID]catalog.DatabaseDescriptor) - typDescs := make(map[descpb.ID]catalog.TypeDescriptor) - schemaDescs := make(map[descpb.ID]catalog.SchemaDescriptor) - for _, desc := range descs { - switch desc := desc.(type) { - case catalog.DatabaseDescriptor: - dbDescs[desc.GetID()] = desc - case catalog.TypeDescriptor: - typDescs[desc.GetID()] = desc - case catalog.SchemaDescriptor: - schemaDescs[desc.GetID()] = desc - } - } - // If we found any type descriptors, that means that some of the tables we - // scanned might have types that need hydrating. - if len(typDescs) > 0 { - // Since we just scanned all the descriptors, we already have everything - // we need to hydrate our types. Set up an accessor for the type hydration - // method to look into the scanned set of descriptors. - typeLookup := func(ctx context.Context, id descpb.ID) (tree.TypeName, catalog.TypeDescriptor, error) { - typDesc, ok := typDescs[id] - if !ok { - n := tree.MakeUnresolvedName(fmt.Sprintf("[%d]", id)) - return tree.TypeName{}, nil, sqlerrors.NewUndefinedObjectError(&n, - tree.TypeObject) - } - dbDesc, ok := dbDescs[typDesc.GetParentID()] - if !ok { - n := fmt.Sprintf("[%d]", typDesc.GetParentID()) - return tree.TypeName{}, nil, sqlerrors.NewUndefinedDatabaseError(n) - } - // We don't use the collection's ResolveSchemaByID method here because - // we already have all of the descriptors. User defined types are only - // members of the public schema or a user defined schema, so those are - // the only cases we have to consider here. - var scName string - switch typDesc.GetParentSchemaID() { - // TODO(richardjcai): Remove case for keys.PublicSchemaID in 22.2. - case keys.PublicSchemaID: - scName = tree.PublicSchema - default: - scName = schemaDescs[typDesc.GetParentSchemaID()].GetName() - } - name := tree.MakeQualifiedTypeName(dbDesc.GetName(), scName, typDesc.GetName()) - return name, typDesc, nil - } - // Now hydrate all table descriptors. - for i := range descs { + // Replace immutable hydratable descriptors in the slice with hydrated copies + // from the cache, or otherwise by creating a copy and hydrating it. + if !hydratableImmutableIndexes.Empty() { + typeFn := makeImmutableTypeLookupFunc(tc, txn, flags, descs) + for _, i := range hydratableImmutableIndexes.Ordered() { desc := descs[i] - // Never hydrate dropped descriptors. - if desc.Dropped() { - continue + // Utilize the cache of hydrated tables if we have one and this descriptor + // was leased. + // TODO(ajwerner): Consider surfacing the mechanism used to retrieve the + // descriptor up to this layer. + if hd := desc.(catalog.HydratableDescriptor); tc.canUseHydratedDescriptorCache(hd.GetID()) { + if cached, err := tc.hydrated.GetHydratedDescriptor(ctx, hd, typeFn); err != nil { + return err + } else if cached != nil { + descs[i] = cached + continue + } } - - var err error - switch t := desc.(type) { - case catalog.TableDescriptor: - err = typedesc.HydrateTypesInTableDescriptor(ctx, t.TableDesc(), typedesc.TypeLookupFunc(typeLookup)) - case catalog.FunctionDescriptor: - err = typedesc.HydrateTypesInFunctionDescriptor(ctx, t.FuncDesc(), typedesc.TypeLookupFunc(typeLookup)) - case catalog.SchemaDescriptor: - err = typedesc.HydrateTypesInSchemaDescriptor(ctx, t.SchemaDesc(), typedesc.TypeLookupFunc(typeLookup)) - } - if err != nil { + // The cache decided not to give back a hydrated descriptor, likely + // because either we've modified the table or one of the types or because + // this transaction has a stale view of one of the relevant descriptors. + // Proceed to hydrating a fresh copy. + desc = desc.NewBuilder().BuildImmutable() + if err := hydrateddesc.Hydrate(ctx, desc, typeFn); err != nil { return err } + descs[i] = desc } } return nil } -// hydrateCatalog installs type metadata in the types present for all -// table descriptors in a catalog. -func hydrateCatalog(ctx context.Context, cat nstree.Catalog) error { - // Since we have a catalog, we already have everything we need to hydrate our - // types. Set up an accessor for the type hydration method to look into the - // catalog. - typeLookup := func(ctx context.Context, id descpb.ID) (tree.TypeName, catalog.TypeDescriptor, error) { - desc := cat.LookupDescriptorEntry(id) +func makeMutableTypeLookupFunc( + tc *Collection, txn *kv.Txn, descs []catalog.Descriptor, +) typedesc.TypeLookupFunc { + var mut nstree.MutableCatalog + for _, desc := range descs { if desc == nil { - n := tree.MakeUnresolvedName(fmt.Sprintf("[%d]", id)) - return tree.TypeName{}, nil, sqlerrors.NewUndefinedObjectError(&n, - tree.TypeObject) + continue } - typDesc, ok := desc.(catalog.TypeDescriptor) - if !ok { - return tree.TypeName{}, nil, errors.Newf( - "found %s while looking for type [%d]", desc.DescriptorType(), id, - ) + if _, ok := desc.(catalog.MutableDescriptor); !ok { + continue } - dbDesc := cat.LookupDescriptorEntry(typDesc.GetParentID()) - if dbDesc == nil { - n := fmt.Sprintf("[%d]", typDesc.GetParentID()) - return tree.TypeName{}, nil, sqlerrors.NewUndefinedDatabaseError(n) + mut.UpsertDescriptorEntry(desc) + } + mutableLookupFunc := func(ctx context.Context, id descpb.ID) (catalog.Descriptor, error) { + return tc.GetMutableDescriptorByID(ctx, txn, id) + } + return hydrateddesc.MakeTypeLookupFuncForHydration(mut.Catalog, mutableLookupFunc) +} + +func makeImmutableTypeLookupFunc( + tc *Collection, txn *kv.Txn, flags tree.CommonLookupFlags, descs []catalog.Descriptor, +) typedesc.TypeLookupFunc { + var imm nstree.MutableCatalog + for _, desc := range descs { + if desc == nil { + continue } - scDesc := cat.LookupDescriptorEntry(typDesc.GetParentSchemaID()) - if scDesc == nil { - n := fmt.Sprintf("[%d]", typDesc.GetParentSchemaID()) - return tree.TypeName{}, nil, sqlerrors.NewUndefinedSchemaError(n) + if _, ok := desc.(catalog.MutableDescriptor); ok { + continue } - name := tree.MakeQualifiedTypeName(dbDesc.GetName(), scDesc.GetName(), typDesc.GetName()) - return name, typDesc, nil + imm.UpsertDescriptorEntry(desc) } - // Now hydrate all table descriptors. - return cat.ForEachDescriptorEntry(func(desc catalog.Descriptor) error { - // Never hydrate dropped descriptors. - if desc.Dropped() { - return nil - } + immutableLookupFunc := func(ctx context.Context, id descpb.ID) (catalog.Descriptor, error) { + return tc.GetImmutableDescriptorByID(ctx, txn, id, tree.CommonLookupFlags{ + Required: true, + AvoidLeased: flags.AvoidLeased, + IncludeOffline: flags.IncludeOffline, + AvoidSynthetic: true, + }) + } + return hydrateddesc.MakeTypeLookupFuncForHydration(imm.Catalog, immutableLookupFunc) +} - var err error - switch t := desc.(type) { - case catalog.TableDescriptor: - err = typedesc.HydrateTypesInTableDescriptor(ctx, t.TableDesc(), typedesc.TypeLookupFunc(typeLookup)) - case catalog.FunctionDescriptor: - err = typedesc.HydrateTypesInFunctionDescriptor(ctx, t.FuncDesc(), typedesc.TypeLookupFunc(typeLookup)) - case catalog.SchemaDescriptor: - err = typedesc.HydrateTypesInSchemaDescriptor(ctx, t.SchemaDesc(), typedesc.TypeLookupFunc(typeLookup)) +// HydrateCatalog installs type metadata in the type.T objects present for all +// objects referencing them in the catalog. +func HydrateCatalog(ctx context.Context, c nstree.MutableCatalog) error { + fakeLookupFunc := func(_ context.Context, id descpb.ID) (catalog.Descriptor, error) { + return nil, catalog.WrapDescRefErr(id, catalog.ErrDescriptorNotFound) + } + typeLookupFunc := hydrateddesc.MakeTypeLookupFuncForHydration(c.Catalog, fakeLookupFunc) + return c.ForEachDescriptorEntry(func(desc catalog.Descriptor) error { + if !hydrateddesc.IsHydratable(desc) { + return nil } - if err != nil { - return err + if _, isMutable := desc.(catalog.MutableDescriptor); isMutable { + return hydrateddesc.Hydrate(ctx, desc, typeLookupFunc) } - return nil + // Deep-copy the immutable descriptor and overwrite the catalog entry. + desc = desc.NewBuilder().BuildImmutable() + defer c.UpsertDescriptorEntry(desc) + return hydrateddesc.Hydrate(ctx, desc, typeLookupFunc) }) } func (tc *Collection) canUseHydratedDescriptorCache(id descpb.ID) bool { return tc.hydrated != nil && tc.stored.GetCachedByID(id) == nil && + tc.uncommitted.getUncommittedByID(id) == nil && tc.synthetic.getSyntheticByID(id) == nil } - -func getMutableTypeLookupFunc( - tc *Collection, txn *kv.Txn, schema catalog.SchemaDescriptor, -) typedesc.TypeLookupFunc { - return func(ctx context.Context, id descpb.ID) (tree.TypeName, catalog.TypeDescriptor, error) { - // Note that getting mutable table implicit type is not allowed. To - // hydrate table implicit types, we don't really need a mutable type - // descriptor since we are not going to mutate the table because we simply - // need the tuple type and some metadata. So it's adequate here to get a - // fresh immutable. - flags := tree.ObjectLookupFlags{ - CommonLookupFlags: tree.CommonLookupFlags{ - Required: true, - IncludeDropped: true, - IncludeOffline: true, - AvoidLeased: true, - }, - } - typDesc, err := tc.GetImmutableTypeByID(ctx, txn, id, flags) - if err != nil { - return tree.TypeName{}, nil, err - } - - _, dbDesc, err := tc.GetImmutableDatabaseByID( - ctx, txn, typDesc.GetParentID(), - tree.DatabaseLookupFlags{ - Required: true, - IncludeDropped: true, - IncludeOffline: true, - AvoidLeased: true, - }, - ) - if err != nil { - return tree.TypeName{}, nil, err - } - - var scName string - if schema != nil { - scName = schema.GetName() - } else { - sc, err := tc.getSchemaByID( - ctx, txn, typDesc.GetParentSchemaID(), - tree.SchemaLookupFlags{ - Required: true, - IncludeDropped: true, - IncludeOffline: true, - AvoidLeased: true, - }, - ) - if err != nil { - return tree.TypeName{}, nil, err - } - scName = sc.GetName() - } - name := tree.MakeQualifiedTypeName(dbDesc.GetName(), scName, typDesc.GetName()) - return name, typDesc, nil - } -} - -func getImmutableTypeLookupFunc( - tc *Collection, - txn *kv.Txn, - includeOffline bool, - avoidLeased bool, - schema catalog.SchemaDescriptor, -) typedesc.TypeLookupFunc { - // "schema" is optional, it's needed only when a schema is being hydrated. - // This is a hack to prevent a dead loop in which a schema need to be type - // hydrated and, to qualify the type name within the same schema, we need the - // same schema itself (to get the schema itself, type hydration is required as - // well, and here is the dead loop). Since we already know the schema, we just - // use the name from the descriptor. - - return func(ctx context.Context, id descpb.ID) (tree.TypeName, catalog.TypeDescriptor, error) { - desc, err := tc.GetImmutableTypeByID(ctx, txn, id, tree.ObjectLookupFlags{ - CommonLookupFlags: tree.CommonLookupFlags{ - Required: true, - AvoidSynthetic: true, - IncludeOffline: includeOffline, - AvoidLeased: avoidLeased, - }, - }) - if err != nil { - return tree.TypeName{}, nil, err - } - _, dbDesc, err := tc.GetImmutableDatabaseByID(ctx, txn, desc.GetParentID(), - tree.DatabaseLookupFlags{ - Required: true, - AvoidSynthetic: true, - IncludeOffline: includeOffline, - AvoidLeased: avoidLeased, - }) - if err != nil { - return tree.TypeName{}, nil, err - } - - var scName string - if schema != nil && schema.GetID() == desc.GetParentSchemaID() { - scName = schema.GetName() - } else { - sc, err := tc.GetImmutableSchemaByID( - ctx, txn, desc.GetParentSchemaID(), tree.SchemaLookupFlags{ - Required: true, - AvoidSynthetic: true, - IncludeOffline: includeOffline, - AvoidLeased: avoidLeased, - }) - if err != nil { - return tree.TypeName{}, nil, err - } - scName = sc.GetName() - } - name := tree.MakeQualifiedTypeName(dbDesc.GetName(), scName, desc.GetName()) - return name, desc, nil - } -} diff --git a/pkg/sql/catalog/descs/object.go b/pkg/sql/catalog/descs/object.go index 517e3b5a66df..28efb171ade3 100644 --- a/pkg/sql/catalog/descs/object.go +++ b/pkg/sql/catalog/descs/object.go @@ -23,72 +23,51 @@ import ( "github.com/cockroachdb/errors" ) -// GetObjectDesc looks up an object by name and returns both its -// descriptor and that of its parent database. If the object is not -// found and flags.required is true, an error is returned, otherwise -// a nil reference is returned. +// GetObjectByName looks up an object by name and returns both its +// descriptor and that of its parent database and schema. // -// TODO(ajwerner): clarify the purpose of the transaction here. It's used in -// some cases for some lookups but not in others. For example, if a mutable -// descriptor is requested, it will be utilized however if an immutable -// descriptor is requested then it will only be used for its timestamp and to -// set the deadline. -func (tc *Collection) GetObjectDesc( - ctx context.Context, txn *kv.Txn, db, schema, object string, flags tree.ObjectLookupFlags, -) (prefix catalog.ResolvedObjectPrefix, desc catalog.Descriptor, err error) { - return tc.getObjectByName(ctx, txn, db, schema, object, flags) -} - -func (tc *Collection) getObjectByName( +// If the object is not found and flags.required is true, an error is returned, +// otherwise a nil reference is returned. +func (tc *Collection) GetObjectByName( ctx context.Context, txn *kv.Txn, catalogName, schemaName, objectName string, flags tree.ObjectLookupFlags, ) (prefix catalog.ResolvedObjectPrefix, desc catalog.Descriptor, err error) { - defer func() { - if err != nil || desc != nil || !flags.Required { - return - } - if catalogName != "" && prefix.Database == nil { - err = sqlerrors.NewUndefinedDatabaseError(catalogName) - } else if prefix.Schema == nil { - err = sqlerrors.NewUndefinedSchemaError(schemaName) - } else { - tn := tree.MakeTableNameWithSchema( - tree.Name(catalogName), - tree.Name(schemaName), - tree.Name(objectName)) - err = sqlerrors.NewUndefinedRelationError(&tn) - } - }() - const alwaysLookupLeasedPublicSchema = false - prefix, desc, err = tc.getObjectByNameIgnoringRequiredAndType( - ctx, txn, catalogName, schemaName, objectName, flags, - alwaysLookupLeasedPublicSchema, - ) - if err != nil || desc == nil { + prefix, err = tc.getObjectPrefixByName(ctx, txn, catalogName, schemaName, flags) + if err != nil || prefix.Schema == nil { return prefix, nil, err } - if desc.Adding() && desc.IsUncommittedVersion() && - (flags.RequireMutable || flags.CommonLookupFlags.AvoidLeased) { - // Special case: We always return tables in the adding state if they were - // created in the same transaction and a descriptor (effectively) read in - // the same transaction is requested. What this basically amounts to is - // resolving adding descriptors only for DDLs (etc.). - // TODO (lucy): I'm not sure where this logic should live. We could add an - // IncludeAdding flag and pull the special case handling up into the - // callers. Figure that out after we clean up the name resolution layers - // and it becomes more Clear what the callers should be. - // TODO(ajwerner): What's weird about returning here is that we have - // not hydrated the descriptor. I guess the assumption is that it is - // already hydrated. - return prefix, desc, nil - } - if dropped, err := filterDescriptorState( - desc, flags.Required, flags.CommonLookupFlags, - ); err != nil || dropped { - return prefix, nil, err + // Read object descriptor and handle errors and absence. + { + var requestedType catalog.DescriptorType + switch flags.DesiredObjectKind { + case tree.TableObject: + requestedType = catalog.Table + case tree.TypeObject: + requestedType = catalog.Type + default: + return prefix, nil, errors.AssertionFailedf( + "unknown DesiredObjectKind value %v", flags.DesiredObjectKind) + } + desc, err = tc.getDescriptorByName( + ctx, txn, prefix.Database, prefix.Schema, objectName, flags.CommonLookupFlags, requestedType, + ) + if err != nil { + return prefix, nil, err + } + if desc == nil { + if flags.Required { + tn := tree.MakeTableNameWithSchema( + tree.Name(catalogName), + tree.Name(schemaName), + tree.Name(objectName)) + return prefix, nil, sqlerrors.NewUndefinedRelationError(&tn) + } + return prefix, nil, nil + } } + // At this point the descriptor is not nil. switch t := desc.(type) { case catalog.TableDescriptor: // A given table name can resolve to either a type descriptor or a table @@ -101,11 +80,6 @@ func (tc *Collection) getObjectByName( default: return prefix, nil, nil } - tableDesc, err := tc.hydrateTypesInTableDesc(ctx, txn, t) - if err != nil { - return prefix, nil, err - } - desc = tableDesc if flags.DesiredObjectKind == tree.TypeObject { // Since a type descriptor was requested, we need to return the implicitly // created record type for the table that we found. @@ -116,7 +90,7 @@ func (tc *Collection) getObjectByName( return prefix, nil, pgerror.Newf(pgcode.InsufficientPrivilege, "cannot modify table record type %q", objectName) } - desc, err = typedesc.CreateImplicitRecordTypeFromTableDesc(tableDesc) + desc, err = typedesc.CreateImplicitRecordTypeFromTableDesc(t) if err != nil { return prefix, nil, err } @@ -133,74 +107,40 @@ func (tc *Collection) getObjectByName( return prefix, desc, nil } -func (tc *Collection) getObjectByNameIgnoringRequiredAndType( - ctx context.Context, - txn *kv.Txn, - catalogName, schemaName, objectName string, - flags tree.ObjectLookupFlags, - alwaysLookupLeasedPublicSchema bool, -) (prefix catalog.ResolvedObjectPrefix, _ catalog.Descriptor, err error) { - - flags.Required = false +func (tc *Collection) getObjectPrefixByName( + ctx context.Context, txn *kv.Txn, catalogName, schemaName string, objFlags tree.ObjectLookupFlags, +) (prefix catalog.ResolvedObjectPrefix, err error) { // If we're reading the object descriptor from the store, // we should read its parents from the store too to ensure // that subsequent name resolution finds the latest name // in the face of a concurrent rename. - avoidLeasedForParent := flags.AvoidLeased || flags.RequireMutable - // Resolve the database. - parentFlags := tree.DatabaseLookupFlags{ - Required: flags.Required, - AvoidLeased: avoidLeasedForParent, - IncludeDropped: flags.IncludeDropped, - IncludeOffline: flags.IncludeOffline, + flags := tree.CommonLookupFlags{ + Required: objFlags.Required, + AvoidLeased: objFlags.AvoidLeased || objFlags.RequireMutable, + IncludeDropped: objFlags.IncludeDropped, + IncludeOffline: objFlags.IncludeOffline, } - - var db catalog.DatabaseDescriptor if catalogName != "" { - db, err = tc.GetImmutableDatabaseByName(ctx, txn, catalogName, parentFlags) - if err != nil || db == nil { - return catalog.ResolvedObjectPrefix{}, nil, err - } - } - - prefix.Database = db - - { - isVirtual, virtualObject, err := tc.virtual.getObjectByName( - schemaName, objectName, flags, catalogName, - ) + prefix.Database, err = tc.GetImmutableDatabaseByName(ctx, txn, catalogName, flags) if err != nil { - return prefix, nil, err + return prefix, err } - if isVirtual { - sc := tc.virtual.getSchemaByName(schemaName) - return catalog.ResolvedObjectPrefix{ - Database: db, - Schema: sc, - }, virtualObject, nil + if prefix.Database == nil { + if flags.Required { + return prefix, sqlerrors.NewUndefinedDatabaseError(catalogName) + } + return prefix, nil } } - - if catalogName == "" { - return catalog.ResolvedObjectPrefix{}, nil, nil - } - - // Read the ID of the schema out of the database descriptor - // to avoid the need to go look up the schema. - sc, err := tc.getSchemaByNameMaybeLookingUpPublicSchema( - ctx, txn, db, schemaName, parentFlags, alwaysLookupLeasedPublicSchema, - ) - if err != nil || sc == nil { - return prefix, nil, err + prefix.Schema, err = tc.GetImmutableSchemaByName(ctx, txn, prefix.Database, schemaName, flags) + if err != nil { + return prefix, err } - - prefix.Schema = sc - found, obj, err := tc.getByName( - ctx, txn, db, sc, objectName, flags.AvoidLeased, flags.RequireMutable, flags.AvoidSynthetic, - false, // alwaysLookupLeasedPublicSchema - ) - if !found || err != nil { - return prefix, nil, err + if prefix.Schema == nil { + if flags.Required { + return prefix, sqlerrors.NewUndefinedSchemaError(schemaName) + } + return prefix, nil } - return prefix, obj, nil + return prefix, nil } diff --git a/pkg/sql/catalog/descs/schema.go b/pkg/sql/catalog/descs/schema.go index 9a408f2e809b..044b755509a2 100644 --- a/pkg/sql/catalog/descs/schema.go +++ b/pkg/sql/catalog/descs/schema.go @@ -14,7 +14,6 @@ import ( "context" "fmt" - "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -24,8 +23,8 @@ import ( "github.com/cockroachdb/errors" ) -// GetMutableSchemaByName resolves the schema and, if applicable, returns a -// mutable descriptor usable by the transaction. RequireMutable is ignored. +// GetMutableSchemaByName resolves the schema and returns a mutable descriptor +// usable by the transaction. RequireMutable is ignored. // // TODO(ajwerner): Change this to take database by name to avoid any weirdness // due to the descriptor being passed in having been cached and causing @@ -36,24 +35,29 @@ func (tc *Collection) GetMutableSchemaByName( db catalog.DatabaseDescriptor, schemaName string, flags tree.SchemaLookupFlags, -) (catalog.SchemaDescriptor, error) { +) (*schemadesc.Mutable, error) { flags.RequireMutable = true - return tc.getSchemaByName(ctx, txn, db, schemaName, flags) + sc, err := tc.getSchemaByName(ctx, txn, db, schemaName, flags) + if err != nil || sc == nil { + return nil, err + } + return sc.(*schemadesc.Mutable), nil } -// GetSchemaByName returns true and a ResolvedSchema object if the target schema -// exists under the target database. +// GetImmutableSchemaByName returns a catalog.SchemaDescriptor object if the +// target schema exists under the target database. RequireMutable is ignored. // // TODO(ajwerner): Change this to take database by name to avoid any weirdness // due to the descriptor being passed in having been cached and causing // problems. -func (tc *Collection) GetSchemaByName( +func (tc *Collection) GetImmutableSchemaByName( ctx context.Context, txn *kv.Txn, db catalog.DatabaseDescriptor, scName string, flags tree.SchemaLookupFlags, ) (catalog.SchemaDescriptor, error) { + flags.RequireMutable = false return tc.getSchemaByName(ctx, txn, db, scName, flags) } @@ -66,31 +70,11 @@ func (tc *Collection) getSchemaByName( schemaName string, flags tree.SchemaLookupFlags, ) (catalog.SchemaDescriptor, error) { - const alwaysLookupLeasedPublicSchema = false - return tc.getSchemaByNameMaybeLookingUpPublicSchema( - ctx, txn, db, schemaName, flags, alwaysLookupLeasedPublicSchema, - ) -} - -// Like getSchemaByName but with the optional flag to avoid trusting a -// cache miss in the database descriptor for the ID of the public schema. -// -// TODO(ajwerner): Remove this split in 22.2. -func (tc *Collection) getSchemaByNameMaybeLookingUpPublicSchema( - ctx context.Context, - txn *kv.Txn, - db catalog.DatabaseDescriptor, - schemaName string, - flags tree.SchemaLookupFlags, - alwaysLookupLeasedPublicSchema bool, -) (catalog.SchemaDescriptor, error) { - found, desc, err := tc.getByName( - ctx, txn, db, nil, schemaName, flags.AvoidLeased, flags.RequireMutable, - flags.AvoidSynthetic, alwaysLookupLeasedPublicSchema, - ) + desc, err := tc.getDescriptorByName(ctx, txn, db, nil /* sc */, schemaName, flags, catalog.Schema) if err != nil { return nil, err - } else if !found { + } + if desc == nil { if flags.Required { return nil, sqlerrors.NewUndefinedSchemaError(schemaName) } @@ -103,9 +87,6 @@ func (tc *Collection) getSchemaByNameMaybeLookingUpPublicSchema( } return nil, nil } - if dropped, err := filterDescriptorState(schema, flags.Required, flags); dropped || err != nil { - return nil, err - } return schema, nil } @@ -121,7 +102,7 @@ func (tc *Collection) GetImmutableSchemaByID( } // GetMutableSchemaByID returns a mutable schema descriptor with the given -// schema ID. +// schema ID. An error is always returned if the descriptor is not physical. func (tc *Collection) GetMutableSchemaByID( ctx context.Context, txn *kv.Txn, schemaID descpb.ID, flags tree.SchemaLookupFlags, ) (*schemadesc.Mutable, error) { @@ -133,52 +114,9 @@ func (tc *Collection) GetMutableSchemaByID( return desc.(*schemadesc.Mutable), nil } -// GetImmutableSchemaByName returns a ResolvedSchema wrapping an immutable -// descriptor, if applicable. RequireMutable is ignored. -// Required is ignored, and an error is always returned if no descriptor with -// the ID exists. -func (tc *Collection) GetImmutableSchemaByName( - ctx context.Context, - txn *kv.Txn, - db catalog.DatabaseDescriptor, - schemaName string, - flags tree.SchemaLookupFlags, -) (catalog.SchemaDescriptor, error) { - flags.RequireMutable = false - return tc.getSchemaByName(ctx, txn, db, schemaName, flags) -} - func (tc *Collection) getSchemaByID( ctx context.Context, txn *kv.Txn, schemaID descpb.ID, flags tree.SchemaLookupFlags, ) (catalog.SchemaDescriptor, error) { - // TODO(richardjcai): Remove this in 22.2, new schemas created in 22.1 - // are regular UDS and do not use keys.PublicSchemaID. - // We can remove this after 22.1 when we no longer have to consider - // mixed version clusters between 21.2 and 22.1. - if schemaID == keys.PublicSchemaID { - return schemadesc.GetPublicSchema(), nil - } - if sc, err := tc.virtual.getSchemaByID( - ctx, schemaID, flags.RequireMutable, - ); err != nil { - if errors.Is(err, catalog.ErrDescriptorNotFound) { - if flags.Required { - return nil, sqlerrors.NewUndefinedSchemaError(fmt.Sprintf("[%d]", schemaID)) - } - return nil, nil - } - return nil, err - } else if sc != nil { - return sc, err - } - - // If this collection is attached to a session and the session has created - // a temporary schema, then check if the schema ID matches. - if sc := tc.temporary.getSchemaByID(ctx, schemaID); sc != nil { - return sc, nil - } - - // Otherwise, fall back to looking up the descriptor with the desired ID. descs, err := tc.getDescriptorsByID(ctx, txn, flags, schemaID) if err != nil { if errors.Is(err, catalog.ErrDescriptorNotFound) { @@ -193,11 +131,5 @@ func (tc *Collection) getSchemaByID( if !ok { return nil, sqlerrors.NewUndefinedSchemaError(fmt.Sprintf("[%d]", schemaID)) } - - hydrated, err := tc.hydrateTypesInDescWithOptions(ctx, txn, schemaDesc, flags.IncludeOffline, flags.AvoidLeased) - if err != nil { - return nil, err - } - - return hydrated.(catalog.SchemaDescriptor), nil + return schemaDesc, nil } diff --git a/pkg/sql/catalog/descs/table.go b/pkg/sql/catalog/descs/table.go index d4f31d9a76b6..8efc50248fdf 100644 --- a/pkg/sql/catalog/descs/table.go +++ b/pkg/sql/catalog/descs/table.go @@ -50,7 +50,7 @@ func (tc *Collection) getTableByName( ctx context.Context, txn *kv.Txn, name tree.ObjectName, flags tree.ObjectLookupFlags, ) (found bool, _ catalog.TableDescriptor, err error) { flags.DesiredObjectKind = tree.TableObject - _, desc, err := tc.getObjectByName( + _, desc, err := tc.GetObjectByName( ctx, txn, name.Catalog(), name.Schema(), name.Object(), flags) if err != nil || desc == nil { return false, nil, err @@ -67,15 +67,12 @@ func (tc *Collection) GetLeasedImmutableTableByID( if err != nil || desc == nil { return nil, err } - table, err := catalog.AsTableDescriptor(desc) + descs := []catalog.Descriptor{desc} + err = tc.hydrateDescriptors(ctx, txn, tree.CommonLookupFlags{}, descs) if err != nil { return nil, err } - hydrated, err := tc.hydrateTypesInTableDesc(ctx, txn, table) - if err != nil { - return nil, err - } - return hydrated, nil + return catalog.AsTableDescriptor(descs[0]) } // GetUncommittedMutableTableByID returns an uncommitted mutable table by its @@ -162,9 +159,5 @@ func (tc *Collection) getTableByID( return nil, sqlerrors.NewUndefinedRelationError( &tree.TableRef{TableID: int64(tableID)}) } - hydrated, err := tc.hydrateTypesInTableDesc(ctx, txn, table) - if err != nil { - return nil, err - } - return hydrated, nil + return table, nil } diff --git a/pkg/sql/catalog/descs/temporary_descriptors.go b/pkg/sql/catalog/descs/temporary_descriptors.go index 47ec0bd8a283..8e91b6851922 100644 --- a/pkg/sql/catalog/descs/temporary_descriptors.go +++ b/pkg/sql/catalog/descs/temporary_descriptors.go @@ -103,9 +103,7 @@ func (td *temporaryDescriptors) getSchemaByName( // getSchemaByID returns the schema descriptor if it is temporary and belongs // to the current session. -func (td *temporaryDescriptors) getSchemaByID( - ctx context.Context, schemaID descpb.ID, -) catalog.SchemaDescriptor { +func (td *temporaryDescriptors) getSchemaByID(schemaID descpb.ID) catalog.SchemaDescriptor { tsp := td.tsp if tsp == nil { return nil diff --git a/pkg/sql/catalog/descs/type.go b/pkg/sql/catalog/descs/type.go index 99da5b6a0d30..bd9eb106a5e6 100644 --- a/pkg/sql/catalog/descs/type.go +++ b/pkg/sql/catalog/descs/type.go @@ -55,7 +55,7 @@ func (tc *Collection) getTypeByName( ctx context.Context, txn *kv.Txn, name tree.ObjectName, flags tree.ObjectLookupFlags, ) (found bool, _ catalog.TypeDescriptor, err error) { flags.DesiredObjectKind = tree.TypeObject - _, desc, err := tc.getObjectByName( + _, desc, err := tc.GetObjectByName( ctx, txn, name.Catalog(), name.Schema(), name.Object(), flags) if err != nil || desc == nil { return false, nil, err @@ -128,11 +128,6 @@ func (tc *Collection) getTypeByID( // User-defined type. return t, nil case catalog.TableDescriptor: - // Table record type. - t, err = tc.hydrateTypesInTableDesc(ctx, txn, t) - if err != nil { - return nil, err - } return typedesc.CreateImplicitRecordTypeFromTableDesc(t) } return nil, pgerror.Newf( diff --git a/pkg/sql/catalog/descs/virtual_descriptors.go b/pkg/sql/catalog/descs/virtual_descriptors.go index dfad07b40adb..d344209ae2fb 100644 --- a/pkg/sql/catalog/descs/virtual_descriptors.go +++ b/pkg/sql/catalog/descs/virtual_descriptors.go @@ -16,7 +16,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/errors" ) @@ -39,7 +38,7 @@ func (tc *virtualDescriptors) getSchemaByName(schemaName string) catalog.SchemaD } func (tc *virtualDescriptors) getObjectByName( - schema string, object string, flags tree.ObjectLookupFlags, db string, + schema string, object string, flags tree.ObjectLookupFlags, ) (isVirtual bool, _ catalog.Descriptor, _ error) { if tc.vs == nil { return false, nil, nil @@ -48,21 +47,17 @@ func (tc *virtualDescriptors) getObjectByName( if !ok { return false, nil, nil } - desc, err := scEntry.GetObjectByName(object, flags) + obj, err := scEntry.GetObjectByName(object, flags) if err != nil { return true, nil, err } - if desc == nil { - if flags.Required { - obj := tree.NewQualifiedObjectName(db, schema, object, flags.DesiredObjectKind) - return true, nil, sqlerrors.NewUndefinedObjectError(obj, flags.DesiredObjectKind) - } + if obj == nil { return true, nil, nil } if flags.RequireMutable { - return true, nil, catalog.NewMutableAccessToVirtualSchemaError(scEntry, object) + return true, nil, catalog.NewMutableAccessToVirtualObjectError(scEntry, obj) } - return true, desc.Desc(), nil + return true, obj.Desc(), nil } func (tc virtualDescriptors) getByID( @@ -80,7 +75,7 @@ func (tc virtualDescriptors) getByID( id, vd.Desc().GetParentSchemaID(), ) } - return nil, catalog.NewMutableAccessToVirtualSchemaError(vs, vd.Desc().GetName()) + return nil, catalog.NewMutableAccessToVirtualObjectError(vs, vd) } return vd.Desc(), nil } @@ -98,7 +93,7 @@ func (tc virtualDescriptors) getSchemaByID( case !found: return nil, nil case mutable: - return nil, catalog.NewMutableAccessToVirtualSchemaError(vs, vs.Desc().GetName()) + return nil, catalog.NewMutableAccessToVirtualSchemaError(vs.Desc()) default: return vs.Desc(), nil } diff --git a/pkg/sql/catalog/errors.go b/pkg/sql/catalog/errors.go index 15515cadc384..0b8d8ef7d60b 100644 --- a/pkg/sql/catalog/errors.go +++ b/pkg/sql/catalog/errors.go @@ -194,13 +194,34 @@ func WrapFunctionDescRefErr(id descpb.ID, err error) error { // NewMutableAccessToVirtualSchemaError is returned when trying to mutably // access a virtual schema object. -func NewMutableAccessToVirtualSchemaError(entry VirtualSchema, object string) error { - switch entry.Desc().GetName() { +func NewMutableAccessToVirtualSchemaError(schema SchemaDescriptor) error { + switch schema.SchemaKind() { + case SchemaPublic: + return pgerror.New(pgcode.InsufficientPrivilege, + "descriptorless public schema cannot be modified") + case SchemaTemporary: + return pgerror.Newf(pgcode.InsufficientPrivilege, + "%s is a temporary schema and cannot be modified", tree.ErrNameString(schema.GetName())) + case SchemaVirtual: + if schema.GetName() == "pg_catalog" { + return pgerror.New(pgcode.InsufficientPrivilege, "pg_catalog is a system catalog") + } + return pgerror.Newf(pgcode.InsufficientPrivilege, + "%s is a virtual schema and cannot be modified", tree.ErrNameString(schema.GetName())) + } + return errors.AssertionFailedf("schema %q (%d) of kind %d is not virtual", + schema.GetName(), schema.GetID(), schema.SchemaKind()) +} + +// NewMutableAccessToVirtualObjectError is returned when trying to mutably +// access a virtual schema object. +func NewMutableAccessToVirtualObjectError(schema VirtualSchema, object VirtualObject) error { + switch schema.Desc().GetName() { case "pg_catalog": return pgerror.Newf(pgcode.InsufficientPrivilege, - "%s is a system catalog", tree.ErrNameString(object)) + "%s is a system catalog", tree.ErrNameString(object.Desc().GetName())) default: return pgerror.Newf(pgcode.WrongObjectType, - "%s is a virtual object and cannot be modified", tree.ErrNameString(object)) + "%s is a virtual object and cannot be modified", tree.ErrNameString(object.Desc().GetName())) } } diff --git a/pkg/sql/catalog/hydrateddesc/BUILD.bazel b/pkg/sql/catalog/hydrateddesc/BUILD.bazel index 2a102e3a084c..288a3b41b4e5 100644 --- a/pkg/sql/catalog/hydrateddesc/BUILD.bazel +++ b/pkg/sql/catalog/hydrateddesc/BUILD.bazel @@ -3,7 +3,10 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "hydrateddesc", - srcs = ["hydratedcache.go"], + srcs = [ + "hydrate.go", + "hydratedcache.go", + ], importpath = "github.com/cockroachdb/cockroach/pkg/sql/catalog/hydrateddesc", visibility = ["//visibility:public"], deps = [ @@ -13,16 +16,19 @@ go_library( "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/funcdesc", "//pkg/sql/catalog/lease", + "//pkg/sql/catalog/nstree", "//pkg/sql/catalog/schemadesc", "//pkg/sql/catalog/tabledesc", "//pkg/sql/catalog/typedesc", "//pkg/sql/sem/tree", + "//pkg/sql/sqlerrors", "//pkg/util/cache", "//pkg/util/metric", "//pkg/util/protoutil", "//pkg/util/syncutil", "//pkg/util/syncutil/singleflight", "@com_github_biogo_store//llrb", + "@com_github_cockroachdb_errors//:errors", "@com_github_prometheus_client_model//go", ], ) diff --git a/pkg/sql/catalog/hydrateddesc/hydrate.go b/pkg/sql/catalog/hydrateddesc/hydrate.go new file mode 100644 index 000000000000..64be5bcc323e --- /dev/null +++ b/pkg/sql/catalog/hydrateddesc/hydrate.go @@ -0,0 +1,123 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package hydrateddesc + +import ( + "context" + "fmt" + + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/nstree" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" + "github.com/cockroachdb/errors" +) + +// HydrationLookupFunc is the type of function required to look up type +// descriptors and their parent schemas and databases when hydrating an object. +type HydrationLookupFunc func(ctx context.Context, id descpb.ID) (catalog.Descriptor, error) + +// IsHydratable returns false iff the descriptor definitely does not require +// hydration +func IsHydratable(desc catalog.Descriptor) bool { + if desc.Dropped() { + // Don't hydrate dropped descriptors. + return false + } + hd, ok := desc.(catalog.HydratableDescriptor) + return ok && hd.ContainsUserDefinedTypes() +} + +// Hydrate ensures that type metadata is present in any type.T objects +// referenced by the descriptor. Beware when calling on immutable descriptors: +// this is not thread-safe. +func Hydrate( + ctx context.Context, desc catalog.Descriptor, typeLookupFunc typedesc.TypeLookupFunc, +) error { + if !IsHydratable(desc) { + return nil + } + + switch t := desc.(type) { + case catalog.TableDescriptor: + return typedesc.HydrateTypesInTableDescriptor(ctx, t.TableDesc(), typeLookupFunc) + case catalog.SchemaDescriptor: + return typedesc.HydrateTypesInSchemaDescriptor(ctx, t.SchemaDesc(), typeLookupFunc) + case catalog.FunctionDescriptor: + return typedesc.HydrateTypesInFunctionDescriptor(ctx, t.FuncDesc(), typeLookupFunc) + } + return errors.AssertionFailedf("unknown hydratable type %T", desc) +} + +// MakeTypeLookupFuncForHydration builds a typedesc.TypeLookupFunc for the +// use with Hydrate. Type descriptors and their parent schema and database are +// looked up in the nstree.Catalog object before being looked up via the +// HydrationLookupFunc. +func MakeTypeLookupFuncForHydration( + c nstree.Catalog, lookupFn HydrationLookupFunc, +) typedesc.TypeLookupFunc { + return func(ctx context.Context, id descpb.ID) (tn tree.TypeName, typ catalog.TypeDescriptor, err error) { + typDesc := c.LookupDescriptorEntry(id) + if typDesc == nil { + typDesc, err = lookupFn(ctx, id) + if err != nil { + if errors.Is(err, catalog.ErrDescriptorNotFound) { + n := tree.Name(fmt.Sprintf("[%d]", id)) + return tree.TypeName{}, nil, sqlerrors.NewUndefinedTypeError(&n) + } + return tree.TypeName{}, nil, err + } + } + switch t := typDesc.(type) { + case catalog.TypeDescriptor: + typ = t + case catalog.TableDescriptor: + typ, err = typedesc.CreateImplicitRecordTypeFromTableDesc(t) + default: + typ, err = catalog.AsTypeDescriptor(typDesc) + } + if err != nil { + return tree.TypeName{}, nil, err + } + dbDesc := c.LookupDescriptorEntry(typ.GetParentID()) + if dbDesc == nil { + dbDesc, err = lookupFn(ctx, typ.GetParentID()) + if err != nil { + if errors.Is(err, catalog.ErrDescriptorNotFound) { + n := fmt.Sprintf("[%d]", typ.GetParentID()) + return tree.TypeName{}, nil, sqlerrors.NewUndefinedDatabaseError(n) + } + return tree.TypeName{}, nil, err + } + } + if _, err = catalog.AsDatabaseDescriptor(dbDesc); err != nil { + return tree.TypeName{}, nil, err + } + scDesc := c.LookupDescriptorEntry(typ.GetParentSchemaID()) + if scDesc == nil { + scDesc, err = lookupFn(ctx, typ.GetParentSchemaID()) + if err != nil { + if errors.Is(err, catalog.ErrDescriptorNotFound) { + n := fmt.Sprintf("[%d]", typ.GetParentSchemaID()) + return tree.TypeName{}, nil, sqlerrors.NewUndefinedSchemaError(n) + } + return tree.TypeName{}, nil, err + } + } + if _, err = catalog.AsSchemaDescriptor(scDesc); err != nil { + return tree.TypeName{}, nil, err + } + tn = tree.MakeQualifiedTypeName(dbDesc.GetName(), scDesc.GetName(), typ.GetName()) + return tn, typ, nil + } +} diff --git a/pkg/sql/catalog/internal/catkv/BUILD.bazel b/pkg/sql/catalog/internal/catkv/BUILD.bazel index bbbf585d7944..fde4832355a9 100644 --- a/pkg/sql/catalog/internal/catkv/BUILD.bazel +++ b/pkg/sql/catalog/internal/catkv/BUILD.bazel @@ -26,13 +26,14 @@ go_library( "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/internal/validate", "//pkg/sql/catalog/nstree", - "//pkg/sql/catalog/resolver", + "//pkg/sql/catalog/schemadesc", "//pkg/sql/catalog/systemschema", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", "//pkg/sql/sem/catconstants", "//pkg/sql/sem/tree", "//pkg/sql/sqlerrors", + "//pkg/util/hlc", "//pkg/util/log", "//pkg/util/mon", "//pkg/util/syncutil", diff --git a/pkg/sql/catalog/internal/catkv/catalog_query.go b/pkg/sql/catalog/internal/catkv/catalog_query.go index 2c36984789fc..995aca02cb80 100644 --- a/pkg/sql/catalog/internal/catkv/catalog_query.go +++ b/pkg/sql/catalog/internal/catkv/catalog_query.go @@ -53,7 +53,7 @@ func (cq catalogQuery) query( return errors.AssertionFailedf("nil txn for catalog query") } b := txn.NewBatch() - in(cq.Codec, b) + in(cq.codec, b) if err := txn.Run(ctx, b); err != nil { return err } @@ -62,7 +62,7 @@ func (cq catalogQuery) query( return result.Err } for _, row := range result.Rows { - _, catTableID, err := cq.Codec.DecodeTablePrefix(row.Key) + _, catTableID, err := cq.codec.DecodeTablePrefix(row.Key) if err != nil { return err } @@ -79,17 +79,17 @@ func (cq catalogQuery) query( } } } - cq.systemDatabaseCache.update(cq.Version, out.Catalog) + cq.systemDatabaseCache.update(cq.version, out.Catalog) return nil } func (cq catalogQuery) processNamespaceResultRow(row kv.KeyValue, cb *nstree.MutableCatalog) error { - nameInfo, err := catalogkeys.DecodeNameMetadataKey(cq.Codec, row.Key) + nameInfo, err := catalogkeys.DecodeNameMetadataKey(cq.codec, row.Key) if err != nil { return err } if row.Exists() { - cb.UpsertNamespaceEntry(nameInfo, descpb.ID(row.ValueInt())) + cb.UpsertNamespaceEntry(nameInfo, descpb.ID(row.ValueInt()), row.Value.Timestamp) } return nil } @@ -97,7 +97,7 @@ func (cq catalogQuery) processNamespaceResultRow(row kv.KeyValue, cb *nstree.Mut func (cq catalogQuery) processDescriptorResultRow( row kv.KeyValue, cb *nstree.MutableCatalog, ) error { - u32ID, err := cq.Codec.DecodeDescMetadataID(row.Key) + u32ID, err := cq.codec.DecodeDescMetadataID(row.Key) if err != nil { return err } diff --git a/pkg/sql/catalog/internal/catkv/catalog_reader.go b/pkg/sql/catalog/internal/catkv/catalog_reader.go index 2582b54c04d5..bc6abc5514da 100644 --- a/pkg/sql/catalog/internal/catkv/catalog_reader.go +++ b/pkg/sql/catalog/internal/catkv/catalog_reader.go @@ -24,20 +24,97 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" ) -// catalogReader builds catalogQuery objects and runs them, leveraging the -// SystemDatabaseCache if it is present. +// CatalogReader queries the system.namespace and system.descriptor tables, +// leveraging the SystemDatabaseCache if it is present in the implementation. +// +// The main use case for CatalogReader should be StoredCatalog. +type CatalogReader interface { + + // Codec returns the codec used by this CatalogReader. + Codec() keys.SQLCodec + + // ScanAll scans the entirety of the descriptor and namespace tables. + ScanAll(ctx context.Context, txn *kv.Txn) (nstree.Catalog, error) + + // ScanNamespaceForDatabases scans the portion of the namespace table which + // contains all database name entries. + ScanNamespaceForDatabases(ctx context.Context, txn *kv.Txn) (nstree.Catalog, error) + + // ScanNamespaceForDatabaseSchemas scans the portion of the namespace table + // which contains all schema name entries for a given database. + ScanNamespaceForDatabaseSchemas( + ctx context.Context, + txn *kv.Txn, + db catalog.DatabaseDescriptor, + ) (nstree.Catalog, error) + + // ScanNamespaceForSchemaObjects scans the portion of the namespace table + // which contains all object name entries for a given schema. + ScanNamespaceForSchemaObjects( + ctx context.Context, txn *kv.Txn, db catalog.DatabaseDescriptor, sc catalog.SchemaDescriptor, + ) (nstree.Catalog, error) + + // GetDescriptorEntries gets the descriptors for the desired IDs, but looks in + // the system database cache first if there is one. + GetDescriptorEntries( + ctx context.Context, + txn *kv.Txn, + ids []descpb.ID, + isRequired bool, + expectedType catalog.DescriptorType, + ) (nstree.Catalog, error) + + // GetNamespaceEntries gets the descriptor IDs for the desired names, but + // looks in the system database cache first if there is one. + GetNamespaceEntries( + ctx context.Context, txn *kv.Txn, nameInfos []descpb.NameInfo, + ) (nstree.Catalog, error) +} + +// NewCatalogReader is the constructor for the default CatalogReader +// implementation. +func NewCatalogReader( + codec keys.SQLCodec, + version clusterversion.ClusterVersion, + systemDatabaseCache *SystemDatabaseCache, +) CatalogReader { + return &catalogReader{ + codec: codec, + version: version, + systemDatabaseCache: systemDatabaseCache, + } +} + +// NewUncachedCatalogReader is the constructor for the default CatalogReader +// implementation without a SystemDatabaseCache. +func NewUncachedCatalogReader(codec keys.SQLCodec) CatalogReader { + return &catalogReader{ + codec: codec, + } +} + +// catalogReader implements the CatalogReader interface by building catalogQuery +// objects and running them, leveraging the SystemDatabaseCache if present. type catalogReader struct { - Codec keys.SQLCodec - Version clusterversion.ClusterVersion + codec keys.SQLCodec // systemDatabaseCache is a cache of system database catalog information. // Its presence is entirely optional and only serves to eliminate superfluous // round trips to KV. systemDatabaseCache *SystemDatabaseCache + // version only needs to be set when systemDatabaseCache is set. + version clusterversion.ClusterVersion } -// scanAll scans the entirety of the descriptor and namespace tables. -func (cr catalogReader) scanAll(ctx context.Context, txn *kv.Txn) (nstree.Catalog, error) { +var _ CatalogReader = (*catalogReader)(nil) + +// Codec is part of the CatalogReader interface. +func (cr catalogReader) Codec() keys.SQLCodec { + return cr.codec +} + +// ScanAll is part of the CatalogReader interface. +func (cr catalogReader) ScanAll(ctx context.Context, txn *kv.Txn) (nstree.Catalog, error) { var mc nstree.MutableCatalog log.Eventf(ctx, "fetching all descriptors and namespace entries") cq := catalogQuery{catalogReader: cr} @@ -54,9 +131,8 @@ func (cr catalogReader) scanAll(ctx context.Context, txn *kv.Txn) (nstree.Catalo return mc.Catalog, nil } -// scanNamespaceForDatabases scans the portion of the namespace table which -// contains all database name entries. -func (cr catalogReader) scanNamespaceForDatabases( +// ScanNamespaceForDatabases is part of the CatalogReader interface. +func (cr catalogReader) ScanNamespaceForDatabases( ctx context.Context, txn *kv.Txn, ) (nstree.Catalog, error) { var mc nstree.MutableCatalog @@ -72,9 +148,42 @@ func (cr catalogReader) scanNamespaceForDatabases( return mc.Catalog, nil } -// getDescriptorEntries gets the descriptors for the desired IDs, but looks in -// the system database cache first. -func (cr catalogReader) getDescriptorEntries( +// ScanNamespaceForDatabaseSchemas is part of the CatalogReader interface. +func (cr catalogReader) ScanNamespaceForDatabaseSchemas( + ctx context.Context, txn *kv.Txn, db catalog.DatabaseDescriptor, +) (nstree.Catalog, error) { + var mc nstree.MutableCatalog + cq := catalogQuery{catalogReader: cr} + err := cq.query(ctx, txn, &mc, func(codec keys.SQLCodec, b *kv.Batch) { + b.Header.MaxSpanRequestKeys = 0 + prefix := catalogkeys.MakeSchemaNameKey(cr.codec, db.GetID(), "" /* name */) + b.Scan(prefix, prefix.PrefixEnd()) + }) + if err != nil { + return nstree.Catalog{}, err + } + return mc.Catalog, nil +} + +// ScanNamespaceForSchemaObjects is part of the CatalogReader interface. +func (cr catalogReader) ScanNamespaceForSchemaObjects( + ctx context.Context, txn *kv.Txn, db catalog.DatabaseDescriptor, sc catalog.SchemaDescriptor, +) (nstree.Catalog, error) { + var mc nstree.MutableCatalog + cq := catalogQuery{catalogReader: cr} + err := cq.query(ctx, txn, &mc, func(codec keys.SQLCodec, b *kv.Batch) { + b.Header.MaxSpanRequestKeys = 0 + prefix := catalogkeys.MakeObjectNameKey(cr.codec, db.GetID(), sc.GetID(), "" /* name */) + b.Scan(prefix, prefix.PrefixEnd()) + }) + if err != nil { + return nstree.Catalog{}, err + } + return mc.Catalog, nil +} + +// GetDescriptorEntries is part of the CatalogReader interface. +func (cr catalogReader) GetDescriptorEntries( ctx context.Context, txn *kv.Txn, ids []descpb.ID, @@ -90,7 +199,7 @@ func (cr catalogReader) getDescriptorEntries( } var needsQuery bool for _, id := range ids { - if desc := cr.systemDatabaseCache.lookupDescriptor(cr.Version, id); desc != nil { + if desc := cr.systemDatabaseCache.lookupDescriptor(cr.version, id); desc != nil { mc.UpsertDescriptorEntry(desc) } else if id != descpb.InvalidID { needsQuery = true @@ -124,9 +233,8 @@ func (cr catalogReader) getDescriptorEntries( return mc.Catalog, nil } -// getNamespaceEntries gets the descriptor IDs for the desired names, but looks -// in the system database cache first. -func (cr catalogReader) getNamespaceEntries( +// GetNamespaceEntries is part of the CatalogReader interface. +func (cr catalogReader) GetNamespaceEntries( ctx context.Context, txn *kv.Txn, nameInfos []descpb.NameInfo, ) (nstree.Catalog, error) { if len(nameInfos) == 0 { @@ -135,8 +243,8 @@ func (cr catalogReader) getNamespaceEntries( var mc nstree.MutableCatalog var needsQuery bool for _, nameInfo := range nameInfos { - if id := cr.systemDatabaseCache.lookupDescriptorID(cr.Version, nameInfo); id != descpb.InvalidID { - mc.UpsertNamespaceEntry(nameInfo, id) + if id, ts := cr.systemDatabaseCache.lookupDescriptorID(cr.version, nameInfo); id != descpb.InvalidID { + mc.UpsertNamespaceEntry(nameInfo, id, ts) } else if nameInfo.Name != "" { needsQuery = true } diff --git a/pkg/sql/catalog/internal/catkv/direct.go b/pkg/sql/catalog/internal/catkv/direct.go index 5d3d5e44e0c8..69c9b98a9b97 100644 --- a/pkg/sql/catalog/internal/catkv/direct.go +++ b/pkg/sql/catalog/internal/catkv/direct.go @@ -141,15 +141,20 @@ type Direct interface { // direct wraps a StoredCatalog to implement the Direct interface. type direct struct { - *StoredCatalog + StoredCatalog + version clusterversion.ClusterVersion } var _ Direct = &direct{} // MakeDirect returns an implementation of Direct. func MakeDirect(codec keys.SQLCodec, version clusterversion.ClusterVersion) Direct { - cr := catalogReader{Codec: codec, Version: version} - return &direct{StoredCatalog: &StoredCatalog{catalogReader: cr}} + return &direct{ + StoredCatalog: StoredCatalog{ + CatalogReader: NewUncachedCatalogReader(codec), + }, + version: version, + } } // MaybeGetDescriptorByIDUnvalidated is part of the Direct interface. @@ -174,7 +179,7 @@ func (d *direct) MustGetDescriptorsByID( return nil, err } vd := d.NewValidationDereferencer(txn) - ve := validate.Validate(ctx, d.Version, vd, catalog.ValidationReadTelemetry, validate.ImmutableRead, descs...) + ve := validate.Validate(ctx, d.version, vd, catalog.ValidationReadTelemetry, validate.ImmutableRead, descs...) if err := ve.CombinedError(); err != nil { return nil, err } @@ -202,7 +207,7 @@ func (d *direct) readDescriptorsForDirectAccess( if len(ids) == 0 { return nil, nil } - c, err := d.getDescriptorEntries(ctx, txn, ids, isRequired, expectedType) + c, err := d.GetDescriptorEntries(ctx, txn, ids, isRequired, expectedType) if err != nil { return nil, err } @@ -222,7 +227,7 @@ func (d *direct) readDescriptorsForDirectAccess( // GetCatalogUnvalidated is part of the Direct interface. func (d *direct) GetCatalogUnvalidated(ctx context.Context, txn *kv.Txn) (nstree.Catalog, error) { - return d.scanAll(ctx, txn) + return d.ScanAll(ctx, txn) } // MustGetDatabaseDescByID is part of the Direct interface. @@ -362,7 +367,7 @@ func (d *direct) LookupDatabaseID( func (d *direct) WriteNewDescToBatch( ctx context.Context, kvTrace bool, b *kv.Batch, desc catalog.Descriptor, ) error { - descKey := catalogkeys.MakeDescMetadataKey(d.Codec, desc.GetID()) + descKey := catalogkeys.MakeDescMetadataKey(d.Codec(), desc.GetID()) proto := desc.DescriptorProto() if kvTrace { log.VEventf(ctx, 2, "CPut %s -> %s", descKey, proto) diff --git a/pkg/sql/catalog/internal/catkv/stored_catalog.go b/pkg/sql/catalog/internal/catkv/stored_catalog.go index af5886aa46f8..45af2c41076d 100644 --- a/pkg/sql/catalog/internal/catkv/stored_catalog.go +++ b/pkg/sql/catalog/internal/catkv/stored_catalog.go @@ -20,7 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/internal/validate" "github.com/cockroachdb/cockroach/pkg/sql/catalog/nstree" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/errors" ) @@ -32,7 +32,7 @@ import ( // A StoredCatalog can also be initialized in a bare-bones fashion with just // a catalogReader and used for direct catalog access, see MakeDirect. type StoredCatalog struct { - catalogReader + CatalogReader // cache mirrors the descriptors in storage. // This map does not store descriptors by name. @@ -64,19 +64,8 @@ type StoredCatalog struct { } // MakeStoredCatalog returns a new instance of StoredCatalog. -func MakeStoredCatalog( - codec keys.SQLCodec, - version clusterversion.ClusterVersion, - systemDatabaseCache *SystemDatabaseCache, - monitor *mon.BytesMonitor, -) StoredCatalog { - sd := StoredCatalog{ - catalogReader: catalogReader{ - Codec: codec, - Version: version, - systemDatabaseCache: systemDatabaseCache, - }, - } +func MakeStoredCatalog(cr CatalogReader, monitor *mon.BytesMonitor) StoredCatalog { + sd := StoredCatalog{CatalogReader: cr} if monitor != nil { memAcc := monitor.MakeBoundAccount() sd.memAcc = &memAcc @@ -93,7 +82,7 @@ func (sc *StoredCatalog) Reset(ctx context.Context) { } old := *sc *sc = StoredCatalog{ - catalogReader: old.catalogReader, + CatalogReader: old.CatalogReader, cache: old.cache, nameIndex: old.nameIndex, memAcc: old.memAcc, @@ -163,7 +152,7 @@ func (sc *StoredCatalog) EnsureAllDescriptors(ctx context.Context, txn *kv.Txn) if sc.hasAllDescriptors { return nil } - c, err := sc.scanAll(ctx, txn) + c, err := sc.ScanAll(ctx, txn) if err != nil { return err } @@ -182,12 +171,12 @@ func (sc *StoredCatalog) EnsureAllDatabaseDescriptors(ctx context.Context, txn * if sc.hasAllDescriptors || sc.hasAllDatabaseDescriptors { return nil } - c, err := sc.scanNamespaceForDatabases(ctx, txn) + c, err := sc.ScanNamespaceForDatabases(ctx, txn) if err != nil { return err } var readIDs catalog.DescriptorIDSet - _ = c.ForEachNamespaceEntry(func(e catalog.NameEntry) error { + _ = c.ForEachNamespaceEntry(func(e nstree.NamespaceEntry) error { if id := e.GetID(); sc.GetCachedByID(id) == nil { readIDs.Add(id) } @@ -209,14 +198,19 @@ func (sc *StoredCatalog) ensureAllSchemaIDsAndNamesForDatabase( if _, ok := sc.allSchemasForDatabase[db.GetID()]; ok { return nil } - schemasNamespaceEntries, err := resolver.GetForDatabase(ctx, txn, sc.Codec, db) + c, err := sc.ScanNamespaceForDatabaseSchemas(ctx, txn, db) if err != nil { return err } - m := make(map[descpb.ID]string, len(schemasNamespaceEntries)) - for id, entry := range schemasNamespaceEntries { - m[id] = entry.Name + m := make(map[descpb.ID]string) + // This is needed at least for the temp system db during restores. + if !db.HasPublicSchemaWithDescriptor() { + m[keys.PublicSchemaIDForBackup] = catconstants.PublicSchemaName } + _ = c.ForEachNamespaceEntry(func(e nstree.NamespaceEntry) error { + m[e.GetID()] = e.GetName() + return nil + }) sc.allSchemasForDatabase[db.GetID()] = m return nil } @@ -264,7 +258,7 @@ func (sc *StoredCatalog) LookupDescriptorID( return id, nil } // Fall back to querying the namespace table. - c, err := sc.getNamespaceEntries(ctx, txn, []descpb.NameInfo{key}) + c, err := sc.GetNamespaceEntries(ctx, txn, []descpb.NameInfo{key}) if err != nil { return descpb.InvalidID, err } @@ -274,39 +268,6 @@ func (sc *StoredCatalog) LookupDescriptorID( return descpb.InvalidID, nil } -// GetByName reads a descriptor from the storage layer by name. -// -// This is a three-step process: -// 1. resolve the descriptor's ID using the name information, -// 2. actually read the descriptor from storage, -// 3. check that the name in the descriptor is the one we expect; meaning that -// there is no RENAME underway for instance. -func (sc *StoredCatalog) GetByName( - ctx context.Context, txn *kv.Txn, parentID descpb.ID, parentSchemaID descpb.ID, name string, -) (catalog.Descriptor, error) { - id, err := sc.LookupDescriptorID(ctx, txn, parentID, parentSchemaID, name) - if err != nil || id == descpb.InvalidID { - return nil, err - } - desc := sc.GetCachedByID(id) - if desc == nil { - err = sc.EnsureFromStorageByIDs(ctx, txn, catalog.MakeDescriptorIDSet(id), catalog.Any) - if err != nil { - if errors.Is(err, catalog.ErrDescriptorNotFound) { - // Having done the namespace lookupObjectID, the descriptor must exist. - return nil, errors.WithAssertionFailure(err) - } - return nil, err - } - desc = sc.GetCachedByID(id) - } - if desc.GetName() != name { - // TODO(postamar): make StoredCatalog aware of name ops - return nil, nil - } - return desc, nil -} - // EnsureFromStorageByIDs actually reads a batch of descriptors from storage // and adds them to the cache. It assumes (without checking) that they are not // already present in the cache. @@ -319,7 +280,7 @@ func (sc *StoredCatalog) EnsureFromStorageByIDs( if ids.Empty() { return nil } - c, err := sc.getDescriptorEntries(ctx, txn, ids.Ordered(), true /* isRequired */, descriptorType) + c, err := sc.GetDescriptorEntries(ctx, txn, ids.Ordered(), true /* isRequired */, descriptorType) if err != nil { return err } @@ -419,7 +380,7 @@ func (c storedCatalogBackedDereferencer) DereferenceDescriptors( } } if len(fallbackReqs) > 0 { - read, err := c.sc.getDescriptorEntries(ctx, c.txn, fallbackReqs, false /* isRequired */, catalog.Any) + read, err := c.sc.GetDescriptorEntries(ctx, c.txn, fallbackReqs, false /* isRequired */, catalog.Any) if err != nil { return nil, err } @@ -451,7 +412,7 @@ func (c storedCatalogBackedDereferencer) DereferenceDescriptorIDs( ctx context.Context, reqs []descpb.NameInfo, ) ([]descpb.ID, error) { // TODO(postamar): cache namespace entries in StoredCatalog - read, err := c.sc.getNamespaceEntries(ctx, c.txn, reqs) + read, err := c.sc.GetNamespaceEntries(ctx, c.txn, reqs) if err != nil { return nil, err } diff --git a/pkg/sql/catalog/internal/catkv/system_database_cache.go b/pkg/sql/catalog/internal/catkv/system_database_cache.go index a54d422858b0..c371980963fd 100644 --- a/pkg/sql/catalog/internal/catkv/system_database_cache.go +++ b/pkg/sql/catalog/internal/catkv/system_database_cache.go @@ -20,8 +20,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/bootstrap" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/nstree" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemadesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/syncutil" ) @@ -63,7 +65,7 @@ func NewSystemDatabaseCache(codec keys.SQLCodec, settings *cluster.Settings) *Sy ParentSchemaID: desc.GetParentSchemaID(), Name: desc.GetName(), } - warm.UpsertNamespaceEntry(key, desc.GetID()) + warm.UpsertNamespaceEntry(key, desc.GetID(), desc.GetModificationTime()) } return nil }) @@ -75,11 +77,15 @@ func NewSystemDatabaseCache(codec keys.SQLCodec, settings *cluster.Settings) *Sy func (c *SystemDatabaseCache) lookupDescriptor( _ clusterversion.ClusterVersion, id descpb.ID, ) catalog.Descriptor { - if id == keys.SystemDatabaseID { + switch id { + case keys.SystemDatabaseID: return systemschema.SystemDB + case keys.SystemPublicSchemaID: + return schemadesc.GetPublicSchema() } // There are not many descriptors which are known to never change. - // There is the system database descriptor, but this case is handled above. + // There is the system database descriptor and its public schema, but these + // cases are handled above. // As of today, we can't assume that there are any others. return nil } @@ -88,23 +94,28 @@ func (c *SystemDatabaseCache) lookupDescriptor( // the cache. func (c *SystemDatabaseCache) lookupDescriptorID( version clusterversion.ClusterVersion, key catalog.NameKey, -) descpb.ID { +) (descpb.ID, hlc.Timestamp) { if key.GetParentID() == descpb.InvalidID && - key.GetParentSchemaID() == 0 && + key.GetParentSchemaID() == descpb.InvalidID && key.GetName() == catconstants.SystemDatabaseName { - return keys.SystemDatabaseID + return keys.SystemDatabaseID, hlc.Timestamp{} + } + if key.GetParentID() == keys.SystemDatabaseID && + key.GetParentSchemaID() == descpb.InvalidID && + key.GetName() == catconstants.PublicSchemaName { + return keys.SystemPublicSchemaID, hlc.Timestamp{} } if c == nil { - return descpb.InvalidID + return descpb.InvalidID, hlc.Timestamp{} } c.mu.RLock() defer c.mu.RUnlock() if cached := c.mu.m[version.Version]; cached != nil { if e := cached.LookupNamespaceEntry(key); e != nil { - return e.GetID() + return e.GetID(), e.GetMVCCTimestamp() } } - return descpb.InvalidID + return descpb.InvalidID, hlc.Timestamp{} } // update the cache for the specified version with a collection of descriptors @@ -130,7 +141,7 @@ func (c *SystemDatabaseCache) update(version clusterversion.ClusterVersion, in n c.mu.m[version.Version] = cached } for _, e := range nameCandidates { - cached.UpsertNamespaceEntry(e, e.GetID()) + cached.UpsertNamespaceEntry(e, e.GetID(), e.GetMVCCTimestamp()) } } @@ -142,14 +153,14 @@ func (c *SystemDatabaseCache) update(version clusterversion.ClusterVersion, in n // to be no updates. func (c *SystemDatabaseCache) nameCandidatesForUpdate( version clusterversion.ClusterVersion, in nstree.Catalog, -) []catalog.NameEntry { +) []nstree.NamespaceEntry { if c == nil { // This should never happen, when c is nil this function should never // even be called. return nil } - var systemNames []catalog.NameEntry - _ = in.ForEachNamespaceEntry(func(e catalog.NameEntry) error { + var systemNames []nstree.NamespaceEntry + _ = in.ForEachNamespaceEntry(func(e nstree.NamespaceEntry) error { if e.GetParentID() == keys.SystemDatabaseID { systemNames = append(systemNames, e) } @@ -167,7 +178,7 @@ func (c *SystemDatabaseCache) nameCandidatesForUpdate( if cached == nil { return systemNames } - diff := make([]catalog.NameEntry, 0, len(systemNames)) + diff := make([]nstree.NamespaceEntry, 0, len(systemNames)) for _, e := range systemNames { if cached.LookupNamespaceEntry(e) == nil { diff = append(diff, e) diff --git a/pkg/sql/catalog/nstree/BUILD.bazel b/pkg/sql/catalog/nstree/BUILD.bazel index 4319f6723a06..0f981aa78cd2 100644 --- a/pkg/sql/catalog/nstree/BUILD.bazel +++ b/pkg/sql/catalog/nstree/BUILD.bazel @@ -22,6 +22,7 @@ go_library( "//pkg/sql/catalog", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/internal/validate", + "//pkg/util/hlc", "//pkg/util/iterutil", "@com_github_cockroachdb_errors//:errors", "@com_github_google_btree//:btree", diff --git a/pkg/sql/catalog/nstree/catalog.go b/pkg/sql/catalog/nstree/catalog.go index 1a155dba15eb..711c77e19cc2 100644 --- a/pkg/sql/catalog/nstree/catalog.go +++ b/pkg/sql/catalog/nstree/catalog.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/internal/validate" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/errors" ) @@ -41,13 +42,21 @@ func (c Catalog) ForEachDescriptorEntry(fn func(desc catalog.Descriptor) error) }) } +// NamespaceEntry is a catalog.NameEntry augmented with an MVCC timestamp. +type NamespaceEntry interface { + catalog.NameEntry + GetMVCCTimestamp() hlc.Timestamp +} + // ForEachNamespaceEntry iterates over all namespace table entries in an ordered // fashion. -func (c Catalog) ForEachNamespaceEntry(fn func(e catalog.NameEntry) error) error { +func (c Catalog) ForEachNamespaceEntry(fn func(e NamespaceEntry) error) error { if !c.IsInitialized() { return nil } - return c.underlying.byName.ascend(fn) + return c.underlying.byName.ascend(func(entry catalog.NameEntry) error { + return fn(entry.(NamespaceEntry)) + }) } // LookupDescriptorEntry looks up a descriptor by ID. @@ -63,11 +72,15 @@ func (c Catalog) LookupDescriptorEntry(id descpb.ID) catalog.Descriptor { } // LookupNamespaceEntry looks up a descriptor ID by name. -func (c Catalog) LookupNamespaceEntry(key catalog.NameKey) catalog.NameEntry { +func (c Catalog) LookupNamespaceEntry(key catalog.NameKey) NamespaceEntry { if !c.IsInitialized() || key == nil { return nil } - return c.underlying.byName.getByName(key.GetParentID(), key.GetParentSchemaID(), key.GetName()) + e := c.underlying.byName.getByName(key.GetParentID(), key.GetParentSchemaID(), key.GetName()) + if e == nil { + return nil + } + return e.(NamespaceEntry) } // OrderedDescriptors returns the descriptors in an ordered fashion. @@ -89,7 +102,7 @@ func (c Catalog) OrderedDescriptorIDs() []descpb.ID { return nil } ret := make([]descpb.ID, 0, c.underlying.byName.t.Len()) - _ = c.ForEachNamespaceEntry(func(e catalog.NameEntry) error { + _ = c.ForEachNamespaceEntry(func(e NamespaceEntry) error { ret = append(ret, e.GetID()) return nil }) @@ -236,7 +249,9 @@ func (mc *MutableCatalog) DeleteDescriptorEntry(id descpb.ID) { } // UpsertNamespaceEntry adds a name -> id mapping to the MutableCatalog. -func (mc *MutableCatalog) UpsertNamespaceEntry(key catalog.NameKey, id descpb.ID) { +func (mc *MutableCatalog) UpsertNamespaceEntry( + key catalog.NameKey, id descpb.ID, mvccTimestamp hlc.Timestamp, +) { if key == nil || id == descpb.InvalidID { return } @@ -247,7 +262,8 @@ func (mc *MutableCatalog) UpsertNamespaceEntry(key catalog.NameKey, id descpb.ID ParentSchemaID: key.GetParentSchemaID(), Name: key.GetName(), }, - ID: id, + ID: id, + Timestamp: mvccTimestamp, } if replaced := mc.underlying.byName.upsert(nsEntry); replaced != nil { mc.byteSize -= replaced.(*namespaceEntry).ByteSize() @@ -274,9 +290,10 @@ func (mc *MutableCatalog) Clear() { type namespaceEntry struct { descpb.NameInfo descpb.ID + hlc.Timestamp } -var _ catalog.NameEntry = namespaceEntry{} +var _ NamespaceEntry = namespaceEntry{} // GetID implements the catalog.NameEntry interface. func (e namespaceEntry) GetID() descpb.ID { @@ -287,3 +304,7 @@ func (e namespaceEntry) GetID() descpb.ID { func (e namespaceEntry) ByteSize() int64 { return int64(e.NameInfo.Size()) + int64(unsafe.Sizeof(e.ID)) } + +func (e namespaceEntry) GetMVCCTimestamp() hlc.Timestamp { + return e.Timestamp +} diff --git a/pkg/sql/catalog/resolver/BUILD.bazel b/pkg/sql/catalog/resolver/BUILD.bazel index caa863283c0b..fbef0ca11ed2 100644 --- a/pkg/sql/catalog/resolver/BUILD.bazel +++ b/pkg/sql/catalog/resolver/BUILD.bazel @@ -10,8 +10,9 @@ go_library( "//pkg/keys", "//pkg/kv", "//pkg/sql/catalog", - "//pkg/sql/catalog/catalogkeys", "//pkg/sql/catalog/descpb", + "//pkg/sql/catalog/internal/catkv", + "//pkg/sql/catalog/nstree", "//pkg/sql/catalog/tabledesc", "//pkg/sql/catalog/typedesc", "//pkg/sql/pgwire/pgcode", diff --git a/pkg/sql/catalog/resolver/resolver.go b/pkg/sql/catalog/resolver/resolver.go index de706f188635..271501aa0399 100644 --- a/pkg/sql/catalog/resolver/resolver.go +++ b/pkg/sql/catalog/resolver/resolver.go @@ -16,8 +16,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/sql/catalog" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/internal/catkv" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/nstree" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" @@ -48,7 +49,7 @@ type SchemaResolver interface { // Accessor is a crufty name and interface that wraps the *descs.Collection. Accessor() catalog.Accessor CurrentSearchPath() sessiondata.SearchPath - CommonLookupFlags(required bool) tree.CommonLookupFlags + CommonLookupFlagsRequired() tree.CommonLookupFlags } // ObjectNameExistingResolver is the helper interface to resolve table @@ -90,16 +91,17 @@ var ErrNoPrimaryKey = pgerror.Newf(pgcode.NoPrimaryKey, func GetObjectNamesAndIDs( ctx context.Context, txn *kv.Txn, - sc SchemaResolver, + sr SchemaResolver, codec keys.SQLCodec, dbDesc catalog.DatabaseDescriptor, scName string, explicitPrefix bool, ) (tree.TableNames, descpb.IDs, error) { - return sc.Accessor().GetObjectNamesAndIDs(ctx, txn, dbDesc, scName, tree.DatabaseListFlags{ - CommonLookupFlags: sc.CommonLookupFlags(true /* required */), + flags := tree.DatabaseListFlags{ + CommonLookupFlags: sr.CommonLookupFlagsRequired(), ExplicitPrefix: explicitPrefix, - }) + } + return sr.Accessor().GetObjectNamesAndIDs(ctx, txn, dbDesc, scName, flags) } // ResolveExistingTableObject looks up an existing object. @@ -333,37 +335,26 @@ func GetForDatabase( ctx context.Context, txn *kv.Txn, codec keys.SQLCodec, db catalog.DatabaseDescriptor, ) (map[descpb.ID]SchemaEntryForDB, error) { log.Eventf(ctx, "fetching all schema descriptor IDs for database %q (%d)", db.GetName(), db.GetID()) - - nameKey := catalogkeys.MakeSchemaNameKey(codec, db.GetID(), "" /* name */) - kvs, err := txn.Scan(ctx, nameKey, nameKey.PrefixEnd(), 0 /* maxRows */) + cr := catkv.NewUncachedCatalogReader(codec) + c, err := cr.ScanNamespaceForDatabaseSchemas(ctx, txn, db) if err != nil { return nil, err } - - ret := make(map[descpb.ID]SchemaEntryForDB, len(kvs)+1) - + ret := make(map[descpb.ID]SchemaEntryForDB) // This is needed at least for the temp system db during restores. if !db.HasPublicSchemaWithDescriptor() { - ret[descpb.ID(keys.PublicSchemaID)] = SchemaEntryForDB{ - Name: tree.PublicSchema, + ret[keys.PublicSchemaIDForBackup] = SchemaEntryForDB{ + Name: catconstants.PublicSchemaName, Timestamp: txn.ReadTimestamp(), } } - - for _, kv := range kvs { - id := descpb.ID(kv.ValueInt()) - if _, ok := ret[id]; ok { - continue - } - k, err := catalogkeys.DecodeNameMetadataKey(codec, kv.Key) - if err != nil { - return nil, err + _ = c.ForEachNamespaceEntry(func(e nstree.NamespaceEntry) error { + ret[e.GetID()] = SchemaEntryForDB{ + Name: e.GetName(), + Timestamp: e.GetMVCCTimestamp(), } - ret[id] = SchemaEntryForDB{ - Name: k.GetName(), - Timestamp: kv.Value.Timestamp, - } - } + return nil + }) return ret, nil } diff --git a/pkg/sql/catalog/schematelemetry/schema_telemetry_event.go b/pkg/sql/catalog/schematelemetry/schema_telemetry_event.go index eb47088a6061..d8e9ad1510b1 100644 --- a/pkg/sql/catalog/schematelemetry/schema_telemetry_event.go +++ b/pkg/sql/catalog/schematelemetry/schema_telemetry_event.go @@ -118,7 +118,7 @@ func CollectClusterSchemaForTelemetry( events = append(events, redacted[id]) }) // Add the log events for each of the selected namespace entries. - _ = raw.ForEachNamespaceEntry(func(e catalog.NameEntry) error { + _ = raw.ForEachNamespaceEntry(func(e nstree.NamespaceEntry) error { if _, found := nsKeys[descpb.NameInfo{ ParentID: e.GetParentID(), ParentSchemaID: e.GetParentSchemaID(), @@ -154,7 +154,7 @@ func truncatedCatalogKeys( keys := make([]joinedRecordKey, 0, len(descIDs)) { var idsInNamespace catalog.DescriptorIDSet - _ = raw.ForEachNamespaceEntry(func(e catalog.NameEntry) error { + _ = raw.ForEachNamespaceEntry(func(e nstree.NamespaceEntry) error { idsInNamespace.Add(e.GetID()) keys = append(keys, joinedRecordKey{ nsKey: e, diff --git a/pkg/sql/catalog/typedesc/table_implicit_record_type.go b/pkg/sql/catalog/typedesc/table_implicit_record_type.go index 55fe563ef83f..a4b04c9308c2 100644 --- a/pkg/sql/catalog/typedesc/table_implicit_record_type.go +++ b/pkg/sql/catalog/typedesc/table_implicit_record_type.go @@ -232,7 +232,7 @@ func (v TableImplicitRecordType) TypeDesc() *descpb.TypeDescriptor { func (v TableImplicitRecordType) HydrateTypeInfoWithName( ctx context.Context, typ *types.T, name *tree.TypeName, res catalog.TypeDescriptorResolver, ) error { - if typ.IsHydrated() { + if typ.IsHydrated() && typ.TypeMeta.Version == uint32(v.desc.GetVersion()) { return nil } if typ.Family() != types.TupleFamily { diff --git a/pkg/sql/catalog/typedesc/type_desc.go b/pkg/sql/catalog/typedesc/type_desc.go index 791424903b8b..b09267ee5508 100644 --- a/pkg/sql/catalog/typedesc/type_desc.go +++ b/pkg/sql/catalog/typedesc/type_desc.go @@ -813,7 +813,7 @@ func EnsureTypeIsHydrated( } } } - if !t.UserDefined() || t.IsHydrated() { + if !t.UserDefined() { return nil } id, err := GetUserDefinedTypeDescID(t) @@ -889,7 +889,7 @@ func HydrateTypesInSchemaDescriptor( func (desc *immutable) HydrateTypeInfoWithName( ctx context.Context, typ *types.T, name *tree.TypeName, res catalog.TypeDescriptorResolver, ) error { - if typ.IsHydrated() { + if typ.IsHydrated() && typ.TypeMeta.Version == uint32(desc.GetVersion()) { return nil } var enumData *types.EnumMetadata diff --git a/pkg/sql/check.go b/pkg/sql/check.go index 200c2214260f..2fb9a192db6d 100644 --- a/pkg/sql/check.go +++ b/pkg/sql/check.go @@ -387,7 +387,7 @@ func (p *planner) RevalidateUniqueConstraintsInCurrentDB(ctx context.Context) er if err != nil { return err } - tableDescs, err := p.Descriptors().GetAllTableDescriptorsInDatabase(ctx, p.Txn(), db.GetID()) + tableDescs, err := p.Descriptors().GetAllTableDescriptorsInDatabase(ctx, p.Txn(), db) if err != nil { return err } @@ -674,7 +674,7 @@ func (p *planner) ValidateTTLScheduledJobsInCurrentDB(ctx context.Context) error if err != nil { return err } - tableDescs, err := p.Descriptors().GetAllTableDescriptorsInDatabase(ctx, p.Txn(), db.GetID()) + tableDescs, err := p.Descriptors().GetAllTableDescriptorsInDatabase(ctx, p.Txn(), db) if err != nil { return err } diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index da97d27fb104..f4418130bb0a 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -47,6 +47,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/multiregion" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/nstree" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/clusterunique" @@ -4990,7 +4991,7 @@ CREATE TABLE crdb_internal.invalid_objects ( return err } - return c.ForEachNamespaceEntry(func(ne catalog.NameEntry) error { + return c.ForEachNamespaceEntry(func(ne nstree.NamespaceEntry) error { if dbContext != nil { if ne.GetParentID() == descpb.InvalidID { if ne.GetID() != dbContext.GetID() { diff --git a/pkg/sql/create_function.go b/pkg/sql/create_function.go index 03ddc336bc2a..21b06df1e1c0 100644 --- a/pkg/sql/create_function.go +++ b/pkg/sql/create_function.go @@ -65,14 +65,13 @@ func (n *createFunctionNode) startExec(params runParams) error { } } - scDesc, err := params.p.descCollection.GetMutableSchemaByName( - params.ctx, params.p.Txn(), n.dbDesc, n.scDesc.GetName(), - tree.SchemaLookupFlags{Required: true, RequireMutable: true}, + mutFlags := tree.SchemaLookupFlags{Required: true, RequireMutable: true} + mutScDesc, err := params.p.descCollection.GetMutableSchemaByName( + params.ctx, params.p.Txn(), n.dbDesc, n.scDesc.GetName(), mutFlags, ) if err != nil { return err } - mutScDesc := scDesc.(*schemadesc.Mutable) var retErr error params.p.runWithOptions(resolveFlags{contextDatabaseID: n.dbDesc.GetID()}, func() { diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index b3bba8b879e1..ee2cfe203d83 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -82,22 +82,21 @@ func (n *createTableNode) ReadingOwnWrites() {} func (p *planner) getNonTemporarySchemaForCreate( ctx context.Context, db catalog.DatabaseDescriptor, scName string, ) (catalog.SchemaDescriptor, error) { - res, err := p.Descriptors().GetMutableSchemaByName( - ctx, p.txn, db, scName, tree.SchemaLookupFlags{ - Required: true, - RequireMutable: true, - }) + flags := tree.SchemaLookupFlags{Required: true, AvoidLeased: true} + sc, err := p.Descriptors().GetImmutableSchemaByName(ctx, p.txn, db, scName, flags) if err != nil { return nil, err } - switch res.SchemaKind() { - case catalog.SchemaPublic, catalog.SchemaUserDefined: - return res, nil + switch sc.SchemaKind() { + case catalog.SchemaPublic: + return sc, nil + case catalog.SchemaUserDefined: + return p.Descriptors().GetMutableSchemaByID(ctx, p.txn, sc.GetID(), flags) case catalog.SchemaVirtual: return nil, pgerror.Newf(pgcode.InsufficientPrivilege, "schema cannot be modified: %q", scName) default: return nil, errors.AssertionFailedf( - "invalid schema kind for getNonTemporarySchemaForCreate: %d", res.SchemaKind()) + "invalid schema kind for getNonTemporarySchemaForCreate: %d", sc.SchemaKind()) } } diff --git a/pkg/sql/doctor/doctor.go b/pkg/sql/doctor/doctor.go index 5f9e4b6c1c81..61ecc946467e 100644 --- a/pkg/sql/doctor/doctor.go +++ b/pkg/sql/doctor/doctor.go @@ -171,7 +171,7 @@ func ExamineDescriptors( var problemsFound bool var cb nstree.MutableCatalog for _, row := range namespaceTable { - cb.UpsertNamespaceEntry(row.NameInfo, descpb.ID(row.ID)) + cb.UpsertNamespaceEntry(row.NameInfo, descpb.ID(row.ID), hlc.Timestamp{}) } for _, row := range descTable { id := descpb.ID(row.ID) diff --git a/pkg/sql/drop_database.go b/pkg/sql/drop_database.go index 49097835f92e..babf3a549494 100644 --- a/pkg/sql/drop_database.go +++ b/pkg/sql/drop_database.go @@ -18,7 +18,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemadesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/descmetadata" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" @@ -82,10 +81,10 @@ func (p *planner) DropDatabase(ctx context.Context, n *tree.DropDatabase) (planN d := newDropCascadeState() for _, schema := range schemas { - res, err := p.Descriptors().GetSchemaByName( + res, err := p.Descriptors().GetImmutableSchemaByName( ctx, p.txn, dbDesc, schema, tree.SchemaLookupFlags{ - Required: true, - RequireMutable: true, + Required: true, + AvoidLeased: true, }, ) if err != nil { @@ -147,9 +146,11 @@ func (n *dropDatabaseNode) startExec(params runParams) error { } case catalog.SchemaUserDefined: // For user defined schemas, we have to do a bit more work. - mutDesc, ok := schemaToDelete.(*schemadesc.Mutable) - if !ok { - return errors.AssertionFailedf("expected Mutable, found %T", schemaToDelete) + mutDesc, err := p.Descriptors().GetMutableSchemaByID( + ctx, p.txn, schemaToDelete.GetID(), p.CommonLookupFlagsRequired(), + ) + if err != nil { + return err } if err := params.p.dropSchemaImpl(ctx, n.dbDesc, mutDesc); err != nil { return err diff --git a/pkg/sql/drop_schema.go b/pkg/sql/drop_schema.go index 48bad898e4c5..835d164f318a 100644 --- a/pkg/sql/drop_schema.go +++ b/pkg/sql/drop_schema.go @@ -71,10 +71,10 @@ func (p *planner) DropSchema(ctx context.Context, n *tree.DropSchema) (planNode, return nil, err } - sc, err := p.Descriptors().GetSchemaByName( + sc, err := p.Descriptors().GetImmutableSchemaByName( ctx, p.txn, db, scName, tree.SchemaLookupFlags{ - Required: false, - RequireMutable: true, + Required: false, + AvoidLeased: true, }, ) if err != nil { @@ -147,8 +147,12 @@ func (n *dropSchemaNode) startExec(params runParams) error { sc := n.d.schemasToDelete[i].schema schemaIDs[i] = sc.GetID() db := n.d.schemasToDelete[i].dbDesc - - mutDesc := sc.(*schemadesc.Mutable) + mutDesc, err := p.Descriptors().GetMutableSchemaByID( + ctx, p.txn, sc.GetID(), p.CommonLookupFlagsRequired(), + ) + if err != nil { + return err + } if err := p.dropSchemaImpl(ctx, db, mutDesc); err != nil { return err } diff --git a/pkg/sql/importer/import_planning.go b/pkg/sql/importer/import_planning.go index 91c1be508186..189199646ff2 100644 --- a/pkg/sql/importer/import_planning.go +++ b/pkg/sql/importer/import_planning.go @@ -493,7 +493,7 @@ func importPlanHook( // No target table means we're importing whatever we find into the session // database, so it must exist. txn := p.Txn() - db, err = p.Accessor().GetDatabaseDesc(ctx, txn, p.SessionData().Database, tree.DatabaseLookupFlags{ + db, err = p.Accessor().GetImmutableDatabaseByName(ctx, txn, p.SessionData().Database, tree.DatabaseLookupFlags{ AvoidLeased: true, Required: true, }) diff --git a/pkg/sql/importer/import_table_creation.go b/pkg/sql/importer/import_table_creation.go index 7b78c53f4256..8197baf57981 100644 --- a/pkg/sql/importer/import_table_creation.go +++ b/pkg/sql/importer/import_table_creation.go @@ -376,9 +376,9 @@ func (r *fkResolver) CurrentSearchPath() sessiondata.SearchPath { return sessiondata.SearchPath{} } -// CommonLookupFlags implements the resolver.SchemaResolver interface. -func (r *fkResolver) CommonLookupFlags(required bool) tree.CommonLookupFlags { - return tree.CommonLookupFlags{} +// CommonLookupFlagsRequired implements the resolver.SchemaResolver interface. +func (r *fkResolver) CommonLookupFlagsRequired() tree.CommonLookupFlags { + return tree.CommonLookupFlags{Required: true} } // LookupObject implements the tree.ObjectNameExistingResolver interface. diff --git a/pkg/sql/logictest/testdata/logic_test/enums b/pkg/sql/logictest/testdata/logic_test/enums index c49b9099aae4..ef8ea0150027 100644 --- a/pkg/sql/logictest/testdata/logic_test/enums +++ b/pkg/sql/logictest/testdata/logic_test/enums @@ -1697,3 +1697,27 @@ SELECT 'foo'::myenum::bytea; statement error invalid cast: myenum -> bytes SELECT 'foo'::myenum::blob; + +subtest correct_type_hydration_in_table + +statement ok +CREATE TABLE tab2 (k greeting) + +statement ok +INSERT INTO tab2 VALUES ('hello') + +statement ok +BEGIN; + +statement ok +ALTER TABLE tab2 ADD COLUMN j INT + +statement ok +ALTER TYPE greeting ADD VALUE 'salud' AFTER 'hello' + +# The insert should fail but with awareness that 'salud' is an enum type. +statement error pq: enum value "salud" is not yet public +INSERT INTO tab2 VALUES ('salud') + +statement ok +ROLLBACK diff --git a/pkg/sql/opt/norm/general_funcs.go b/pkg/sql/opt/norm/general_funcs.go index 4ed4e4bcf22c..9566bff90363 100644 --- a/pkg/sql/opt/norm/general_funcs.go +++ b/pkg/sql/opt/norm/general_funcs.go @@ -64,6 +64,12 @@ func (c *CustomFuncs) IsTimestampTZ(scalar opt.ScalarExpr) bool { return scalar.DataType().Family() == types.TimestampTZFamily } +// IsJSON returns true if the given scalar expression is of type +// JSON. +func (c *CustomFuncs) IsJSON(scalar opt.ScalarExpr) bool { + return scalar.DataType().Family() == types.JsonFamily +} + // BoolType returns the boolean SQL type. func (c *CustomFuncs) BoolType() *types.T { return types.Bool diff --git a/pkg/sql/opt/norm/rules/scalar.opt b/pkg/sql/opt/norm/rules/scalar.opt index 5d829093f294..92bf212f300f 100644 --- a/pkg/sql/opt/norm/rules/scalar.opt +++ b/pkg/sql/opt/norm/rules/scalar.opt @@ -390,3 +390,10 @@ $input (Not (Function $args:* $private:(FunctionPrivate "st_disjoint"))) => (MakeIntersectionFunction $args) + +# ConvertJSONSubscriptToFetchValue converts json subscripting +# `[...]` into a fetch value operator `->`. +[ConvertJSONSubscriptToFetchValue, Normalize] +(Indirection $input:* $index:* & (IsJSON $input)) +=> +(FetchVal $input $index) diff --git a/pkg/sql/opt/norm/testdata/rules/fold_constants b/pkg/sql/opt/norm/testdata/rules/fold_constants index be1b49840066..bbe5ac8a0f79 100644 --- a/pkg/sql/opt/norm/testdata/rules/fold_constants +++ b/pkg/sql/opt/norm/testdata/rules/fold_constants @@ -1214,10 +1214,11 @@ SELECT j['field'] FROM a ---- project ├── columns: j:9 + ├── immutable ├── scan a │ └── columns: a.j:5 └── projections - └── a.j:5['field'] [as=j:9, outer=(5)] + └── a.j:5->'field' [as=j:9, outer=(5), immutable] # Regression test for #40404. norm expect=FoldIndirection diff --git a/pkg/sql/opt/norm/testdata/rules/scalar b/pkg/sql/opt/norm/testdata/rules/scalar index ca8d5dd7f9f4..6230d71f0e0e 100644 --- a/pkg/sql/opt/norm/testdata/rules/scalar +++ b/pkg/sql/opt/norm/testdata/rules/scalar @@ -18,6 +18,17 @@ exec-ddl CREATE TABLE c (c CHAR PRIMARY KEY) ---- +exec-ddl +CREATE TABLE b +( + k INT PRIMARY KEY, + j JSONB, + i INT, + s STRING, + arr STRING[] +) +---- + # -------------------------------------------------- # CommuteVar # -------------------------------------------------- @@ -2130,3 +2141,92 @@ project │ └── columns: geom1:2 geom2:3 └── projections └── NOT st_intersects(geom1:2, geom2:3) [as="?column?":8, outer=(2,3), immutable] + +# -------------------------------------------------- +# ConvertJSONSubscriptToFetchValue +# -------------------------------------------------- + +norm expect=ConvertJSONSubscriptToFetchValue +SELECT j['c'] FROM b WHERE j['a'] = '"b"' +---- +project + ├── columns: j:8 + ├── immutable + ├── select + │ ├── columns: b.j:2 + │ ├── immutable + │ ├── scan b + │ │ └── columns: b.j:2 + │ └── filters + │ └── (b.j:2->'a') = '"b"' [outer=(2), immutable] + └── projections + └── b.j:2->'c' [as=j:8, outer=(2), immutable] + +norm expect=ConvertJSONSubscriptToFetchValue +SELECT j['a']['b'] FROM b WHERE j['c'] = '1' +---- +project + ├── columns: j:8 + ├── immutable + ├── select + │ ├── columns: b.j:2 + │ ├── immutable + │ ├── scan b + │ │ └── columns: b.j:2 + │ └── filters + │ └── (b.j:2->'c') = '1' [outer=(2), immutable] + └── projections + └── (b.j:2->'a')->'b' [as=j:8, outer=(2), immutable] + +norm expect=ConvertJSONSubscriptToFetchValue +SELECT j[0] FROM b WHERE j[0][1] = '1' +---- +project + ├── columns: j:8 + ├── immutable + ├── select + │ ├── columns: b.j:2 + │ ├── immutable + │ ├── scan b + │ │ └── columns: b.j:2 + │ └── filters + │ └── ((b.j:2->0)->1) = '1' [outer=(2), immutable] + └── projections + └── b.j:2->0 [as=j:8, outer=(2), immutable] + +norm expect=ConvertJSONSubscriptToFetchValue +SELECT j[i], j[s] FROM b +---- +project + ├── columns: j:8 j:9 + ├── immutable + ├── scan b + │ └── columns: b.j:2 i:3 s:4 + └── projections + ├── b.j:2->i:3 [as=j:8, outer=(2,3), immutable] + └── b.j:2->s:4 [as=j:9, outer=(2,4), immutable] + +norm expect-not=ConvertJSONSubscriptToFetchValue +SELECT arr[1] FROM b WHERE arr[2] = 'a' +---- +project + ├── columns: arr:8 + ├── select + │ ├── columns: b.arr:5 + │ ├── scan b + │ │ └── columns: b.arr:5 + │ └── filters + │ └── b.arr:5[2] = 'a' [outer=(5)] + └── projections + └── b.arr:5[1] [as=arr:8, outer=(5)] + +norm expect-not=ConvertJSONSubscriptToFetchValue +SELECT arr[1], arr[2] FROM b; +---- +project + ├── columns: arr:8 arr:9 + ├── scan b + │ └── columns: b.arr:5 + └── projections + ├── b.arr:5[1] [as=arr:8, outer=(5)] + └── b.arr:5[2] [as=arr:9, outer=(5)] diff --git a/pkg/sql/opt/xform/testdata/rules/select b/pkg/sql/opt/xform/testdata/rules/select index c40c116ee199..a18746f7cd90 100644 --- a/pkg/sql/opt/xform/testdata/rules/select +++ b/pkg/sql/opt/xform/testdata/rules/select @@ -2791,6 +2791,19 @@ project │ └── spans: ["7a\x00\x01\x12b\x00\x01", "7a\x00\x01\x12b\x00\x01"] └── key: (1) +opt expect=ConvertJSONSubscriptToFetchValue +SELECT k FROM b WHERE j['a'] = '"b"' +---- +project + ├── columns: k:1!null + ├── immutable + ├── key: (1) + └── scan b@j_inv_idx + ├── columns: k:1!null + ├── inverted constraint: /7/1 + │ └── spans: ["7a\x00\x01\x12b\x00\x01", "7a\x00\x01\x12b\x00\x01"] + └── key: (1) + # Chained fetch val operators. opt expect=GenerateInvertedIndexScans SELECT k FROM b WHERE j->'a'->'b' = '"c"' @@ -3066,6 +3079,30 @@ project ├── key: (1) └── fd: (1)-->(7) +opt expect=ConvertJSONSubscriptToFetchValue +SELECT k FROM b WHERE j['a']['b'] @> '"c"' +---- +project + ├── columns: k:1!null + ├── immutable + ├── key: (1) + └── inverted-filter + ├── columns: k:1!null + ├── inverted expression: /7 + │ ├── tight: true, unique: false + │ └── union spans + │ ├── ["7a\x00\x02b\x00\x01\x12c\x00\x01", "7a\x00\x02b\x00\x01\x12c\x00\x01"] + │ └── ["7a\x00\x02b\x00\x02\x00\x03\x00\x01\x12c\x00\x01", "7a\x00\x02b\x00\x02\x00\x03\x00\x01\x12c\x00\x01"] + ├── key: (1) + └── scan b@j_inv_idx + ├── columns: k:1!null j_inverted_key:7!null + ├── inverted constraint: /7/1 + │ └── spans + │ ├── ["7a\x00\x02b\x00\x01\x12c\x00\x01", "7a\x00\x02b\x00\x01\x12c\x00\x01"] + │ └── ["7a\x00\x02b\x00\x02\x00\x03\x00\x01\x12c\x00\x01", "7a\x00\x02b\x00\x02\x00\x03\x00\x01\x12c\x00\x01"] + ├── key: (1) + └── fd: (1)-->(7) + opt expect=GenerateInvertedIndexScans SELECT k FROM b WHERE j->'a'->'b' <@ '"c"' ---- diff --git a/pkg/sql/rename_database.go b/pkg/sql/rename_database.go index bc6cb53fbb34..8947356fb23d 100644 --- a/pkg/sql/rename_database.go +++ b/pkg/sql/rename_database.go @@ -119,7 +119,7 @@ func (n *renameDatabaseNode) startExec(params runParams) error { // Rather than trying to rewrite them with the changed DB name, we // simply disallow such renames for now. // See #34416. - lookupFlags := p.CommonLookupFlags(true /*required*/) + lookupFlags := p.CommonLookupFlagsRequired() // DDL statements bypass the cache. lookupFlags.AvoidLeased = true schemas, err := p.Descriptors().GetSchemasForDatabase(ctx, p.txn, dbDesc) diff --git a/pkg/sql/rename_table.go b/pkg/sql/rename_table.go index 4e966c3e69a5..f18b2e1edcf4 100644 --- a/pkg/sql/rename_table.go +++ b/pkg/sql/rename_table.go @@ -122,10 +122,10 @@ func (n *renameTableNode) startExec(params runParams) error { return err } - targetSchemaDesc, err = p.Descriptors().GetMutableSchemaByName( + targetSchemaDesc, err = p.Descriptors().GetImmutableSchemaByName( ctx, p.txn, targetDbDesc, oldTn.Schema(), tree.SchemaLookupFlags{ - Required: true, - RequireMutable: true, + Required: true, + AvoidLeased: true, }) if err != nil { return err diff --git a/pkg/sql/repair.go b/pkg/sql/repair.go index c46cee0a0e41..a4616bd1ecb5 100644 --- a/pkg/sql/repair.go +++ b/pkg/sql/repair.go @@ -486,7 +486,7 @@ func (p *planner) UnsafeUpsertNamespaceEntry( if val.Value != nil { existingID = descpb.ID(val.ValueInt()) } - flags := p.CommonLookupFlags(true /* required */) + flags := p.CommonLookupFlagsRequired() flags.IncludeDropped = true flags.IncludeOffline = true validateDescriptor := func() error { diff --git a/pkg/sql/resolver.go b/pkg/sql/resolver.go index d68ac117222d..1da116738289 100644 --- a/pkg/sql/resolver.go +++ b/pkg/sql/resolver.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/catalog/funcdesc" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/nstree" "github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc" @@ -379,7 +380,8 @@ func validateColumnForHasPrivilegeSpecifier( // ObjectLookupFlags is part of the resolver.SchemaResolver interface. func (p *planner) ObjectLookupFlags(required, requireMutable bool) tree.ObjectLookupFlags { - flags := p.CommonLookupFlags(required) + flags := p.CommonLookupFlagsRequired() + flags.Required = required flags.RequireMutable = requireMutable return tree.ObjectLookupFlags{CommonLookupFlags: flags} } @@ -399,9 +401,8 @@ func (p *planner) getDescriptorsFromTargetListForPrivilegeChange( ) ([]DescriptorWithObjectType, error) { const required = true flags := tree.CommonLookupFlags{ - Required: required, - AvoidLeased: p.skipDescriptorCache, - RequireMutable: true, + Required: required, + AvoidLeased: p.skipDescriptorCache, } if targets.Databases != nil { if len(targets.Databases) == 0 { @@ -549,8 +550,8 @@ func (p *planner) getDescriptorsFromTargetListForPrivilegeChange( if err != nil { return nil, err } - sc, err := p.Descriptors().GetMutableSchemaByName( - ctx, p.txn, db, scName.Schema(), tree.SchemaLookupFlags{Required: true}, + sc, err := p.Descriptors().GetImmutableSchemaByName( + ctx, p.txn, db, scName.Schema(), tree.SchemaLookupFlags{Required: true, AvoidLeased: true}, ) if err != nil { return nil, err @@ -597,19 +598,24 @@ func (p *planner) getDescriptorsFromTargetListForPrivilegeChange( } for _, sc := range targetSchemas { - resSchema, err := p.Descriptors().GetSchemaByName( - ctx, p.txn, sc.dbDesc, sc.schema, flags) + resSchema, err := p.Descriptors().GetImmutableSchemaByName( + ctx, p.txn, sc.dbDesc, sc.schema, flags, + ) if err != nil { return nil, err } switch resSchema.SchemaKind() { case catalog.SchemaUserDefined: - descs = append( - descs, - DescriptorWithObjectType{ - descriptor: resSchema, - objectType: privilege.Schema, - }) + mutSchema, err := p.Descriptors().GetMutableSchemaByID( + ctx, p.txn, resSchema.GetID(), flags, + ) + if err != nil { + return nil, err + } + descs = append(descs, DescriptorWithObjectType{ + descriptor: mutSchema, + objectType: privilege.Schema, + }) default: return nil, pgerror.Newf(pgcode.InvalidSchemaName, "cannot change privileges on schema %q", resSchema.GetName()) @@ -981,16 +987,17 @@ type tableLookupFn = *internalLookupCtx // internalLookupCtx. It also hydrates any table descriptors with enum // information. It is intended only for use when dealing with backups. func newInternalLookupCtxFromDescriptorProtos( - ctx context.Context, rawDescs []descpb.Descriptor, prefix catalog.DatabaseDescriptor, + ctx context.Context, rawDescs []descpb.Descriptor, ) (*internalLookupCtx, error) { - descriptors := make([]catalog.Descriptor, len(rawDescs)) + var c nstree.MutableCatalog for i := range rawDescs { - descriptors[i] = descbuilder.NewBuilder(&rawDescs[i]).BuildImmutable() + desc := descbuilder.NewBuilder(&rawDescs[i]).BuildImmutable() + c.UpsertDescriptorEntry(desc) } - lCtx := newInternalLookupCtx(descriptors, prefix) - if err := descs.HydrateGivenDescriptors(ctx, descriptors); err != nil { + if err := descs.HydrateCatalog(ctx, c); err != nil { return nil, err } + lCtx := newInternalLookupCtx(c.OrderedDescriptors(), nil /* prefix */) return lCtx, nil } @@ -1310,10 +1317,11 @@ func (p *planner) ResolveExistingObjectEx( requiredType tree.RequiredTableKind, ) (res catalog.TableDescriptor, err error) { lookupFlags := tree.ObjectLookupFlags{ - CommonLookupFlags: p.CommonLookupFlags(required), + CommonLookupFlags: p.CommonLookupFlagsRequired(), DesiredObjectKind: tree.TableObject, DesiredTableDescKind: requiredType, } + lookupFlags.Required = required desc, prefix, err := resolver.ResolveExistingObject(ctx, p, name, lookupFlags) if err != nil || desc == nil { return nil, err diff --git a/pkg/sql/schema_resolver.go b/pkg/sql/schema_resolver.go index 7964ecb1f2e2..5aa60eef1e00 100644 --- a/pkg/sql/schema_resolver.go +++ b/pkg/sql/schema_resolver.go @@ -75,10 +75,10 @@ func (sr *schemaResolver) CurrentSearchPath() sessiondata.SearchPath { return sr.sessionDataStack.Top().SearchPath } -// CommonLookupFlags implements the resolver.SchemaResolver interface. -func (sr *schemaResolver) CommonLookupFlags(required bool) tree.CommonLookupFlags { +// CommonLookupFlagsRequired implements the resolver.SchemaResolver interface. +func (sr *schemaResolver) CommonLookupFlagsRequired() tree.CommonLookupFlags { return tree.CommonLookupFlags{ - Required: required, + Required: true, AvoidLeased: sr.skipDescriptorCache, } } @@ -87,7 +87,6 @@ func (sr *schemaResolver) CommonLookupFlags(required bool) tree.CommonLookupFlag func (sr *schemaResolver) LookupObject( ctx context.Context, flags tree.ObjectLookupFlags, dbName, scName, obName string, ) (found bool, prefix catalog.ResolvedObjectPrefix, objMeta catalog.Descriptor, err error) { - sc := sr.Accessor() flags.CommonLookupFlags.Required = false flags.CommonLookupFlags.AvoidLeased = sr.skipDescriptorCache @@ -116,7 +115,7 @@ func (sr *schemaResolver) LookupObject( } } - prefix, objMeta, err = sc.GetObjectDesc(ctx, sr.txn, dbName, scName, obName, flags) + prefix, objMeta, err = sr.descCollection.GetObjectByName(ctx, sr.txn, dbName, scName, obName, flags) return objMeta != nil, prefix, objMeta, err } @@ -124,23 +123,17 @@ func (sr *schemaResolver) LookupObject( func (sr *schemaResolver) LookupSchema( ctx context.Context, dbName, scName string, ) (found bool, scMeta catalog.ResolvedObjectPrefix, err error) { - dbDesc, err := sr.descCollection.GetImmutableDatabaseByName(ctx, sr.txn, dbName, - tree.DatabaseLookupFlags{AvoidLeased: sr.skipDescriptorCache}) - if err != nil || dbDesc == nil { + flags := sr.CommonLookupFlagsRequired() + flags.Required = false + db, err := sr.descCollection.GetImmutableDatabaseByName(ctx, sr.txn, dbName, flags) + if err != nil || db == nil { return false, catalog.ResolvedObjectPrefix{}, err } - sc := sr.Accessor() - var resolvedSchema catalog.SchemaDescriptor - resolvedSchema, err = sc.GetSchemaByName( - ctx, sr.txn, dbDesc, scName, sr.CommonLookupFlags(false /* required */), - ) - if err != nil || resolvedSchema == nil { + sc, err := sr.descCollection.GetImmutableSchemaByName(ctx, sr.txn, db, scName, flags) + if err != nil || sc == nil { return false, catalog.ResolvedObjectPrefix{}, err } - return true, catalog.ResolvedObjectPrefix{ - Database: dbDesc, - Schema: resolvedSchema, - }, nil + return true, catalog.ResolvedObjectPrefix{Database: db, Schema: sc}, nil } // CurrentDatabase implements the tree.QualifiedNameResolver interface. @@ -305,21 +298,22 @@ func (sr *schemaResolver) ResolveTypeByOID(ctx context.Context, oid oid.Oid) (*t func (sr *schemaResolver) GetTypeDescriptor( ctx context.Context, id descpb.ID, ) (tree.TypeName, catalog.TypeDescriptor, error) { - desc, err := sr.descCollection.GetImmutableTypeByID(ctx, sr.txn, id, tree.ObjectLookupFlags{}) + tc := sr.descCollection + desc, err := tc.GetImmutableTypeByID(ctx, sr.txn, id, tree.ObjectLookupFlags{}) if err != nil { return tree.TypeName{}, nil, err } // Note that the value of required doesn't matter for lookups by ID. - _, dbDesc, err := sr.descCollection.GetImmutableDatabaseByID(ctx, sr.txn, desc.GetParentID(), sr.CommonLookupFlags(true /* required */)) + flags := sr.CommonLookupFlagsRequired() + _, db, err := tc.GetImmutableDatabaseByID(ctx, sr.txn, desc.GetParentID(), flags) if err != nil { return tree.TypeName{}, nil, err } - sc, err := sr.descCollection.GetImmutableSchemaByID( - ctx, sr.txn, desc.GetParentSchemaID(), tree.SchemaLookupFlags{Required: true}) + sc, err := tc.GetImmutableSchemaByID(ctx, sr.txn, desc.GetParentSchemaID(), flags) if err != nil { return tree.TypeName{}, nil, err } - name := tree.MakeQualifiedTypeName(dbDesc.GetName(), sc.GetName(), desc.GetName()) + name := tree.MakeQualifiedTypeName(db.GetName(), sc.GetName(), desc.GetName()) return name, desc, nil } diff --git a/pkg/sql/schemachanger/scdeps/build_deps.go b/pkg/sql/schemachanger/scdeps/build_deps.go index 3038abef4403..b8ea8f81ab6f 100644 --- a/pkg/sql/schemachanger/scdeps/build_deps.go +++ b/pkg/sql/schemachanger/scdeps/build_deps.go @@ -113,7 +113,7 @@ func (d *buildDeps) MayResolveSchema( name.CatalogName = tree.Name(d.schemaResolver.CurrentDatabase()) } db := d.MayResolveDatabase(ctx, name.CatalogName) - schema, err := d.descsCollection.GetSchemaByName(ctx, d.txn, db, name.Schema(), tree.SchemaLookupFlags{ + schema, err := d.descsCollection.GetImmutableSchemaByName(ctx, d.txn, db, name.Schema(), tree.SchemaLookupFlags{ AvoidLeased: true, }) if err != nil { diff --git a/pkg/sql/schemachanger/scdeps/sctestdeps/config.go b/pkg/sql/schemachanger/scdeps/sctestdeps/config.go index 3accdfae35a8..329c14edab09 100644 --- a/pkg/sql/schemachanger/scdeps/sctestdeps/config.go +++ b/pkg/sql/schemachanger/scdeps/sctestdeps/config.go @@ -39,9 +39,9 @@ var _ Option = (optionFunc)(nil) // WithNamespace sets the TestState namespace to the provided value. func WithNamespace(c nstree.Catalog) Option { return optionFunc(func(state *TestState) { - _ = c.ForEachNamespaceEntry(func(e catalog.NameEntry) error { - state.committed.UpsertNamespaceEntry(e, e.GetID()) - state.uncommitted.UpsertNamespaceEntry(e, e.GetID()) + _ = c.ForEachNamespaceEntry(func(e nstree.NamespaceEntry) error { + state.committed.UpsertNamespaceEntry(e, e.GetID(), e.GetMVCCTimestamp()) + state.uncommitted.UpsertNamespaceEntry(e, e.GetID(), e.GetMVCCTimestamp()) return nil }) }) diff --git a/pkg/sql/schemachanger/scdeps/sctestdeps/database_state.go b/pkg/sql/schemachanger/scdeps/sctestdeps/database_state.go index 21d93fdd03e2..a627f8bf4c2a 100644 --- a/pkg/sql/schemachanger/scdeps/sctestdeps/database_state.go +++ b/pkg/sql/schemachanger/scdeps/sctestdeps/database_state.go @@ -108,7 +108,7 @@ func ReadNamespaceFromDB(t *testing.T, tdb *sqlutils.SQLRunner) nstree.MutableCa // Fetch namespace state. var cb nstree.MutableCatalog nsRows := tdb.QueryStr(t, ` -SELECT "parentID", "parentSchemaID", name, id +SELECT "parentID", "parentSchemaID", name, id FROM system.namespace ORDER BY id`) for _, nsRow := range nsRows { @@ -134,7 +134,7 @@ ORDER BY id`) ParentSchemaID: descpb.ID(parentSchemaID), Name: name, } - cb.UpsertNamespaceEntry(key, descpb.ID(id)) + cb.UpsertNamespaceEntry(key, descpb.ID(id), hlc.Timestamp{}) } return cb } diff --git a/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go b/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go index b0f0b9255d0e..befc2c79cbb2 100644 --- a/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go +++ b/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go @@ -28,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/funcdesc" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/nstree" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemadesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc" @@ -432,7 +433,7 @@ func (s *TestState) ReadObjectNamesAndIDs( ctx context.Context, db catalog.DatabaseDescriptor, schema catalog.SchemaDescriptor, ) (names tree.TableNames, ids descpb.IDs) { m := make(map[string]descpb.ID) - _ = s.uncommitted.ForEachNamespaceEntry(func(e catalog.NameEntry) error { + _ = s.uncommitted.ForEachNamespaceEntry(func(e nstree.NamespaceEntry) error { if e.GetParentID() == db.GetID() && e.GetParentSchemaID() == schema.GetID() { m[e.GetName()] = e.GetID() names = append(names, tree.MakeTableNameWithSchema( diff --git a/pkg/sql/schemachanger/scdeps/sctestdeps/test_state.go b/pkg/sql/schemachanger/scdeps/sctestdeps/test_state.go index 3b1603fe2bd9..6c4da61f646b 100644 --- a/pkg/sql/schemachanger/scdeps/sctestdeps/test_state.go +++ b/pkg/sql/schemachanger/scdeps/sctestdeps/test_state.go @@ -118,9 +118,9 @@ func (s *TestState) WithTxn(fn func(s *TestState)) { defer func() { u := s.uncommitted s.committed, s.uncommitted = nstree.MutableCatalog{}, nstree.MutableCatalog{} - _ = u.ForEachNamespaceEntry(func(e catalog.NameEntry) error { - s.committed.UpsertNamespaceEntry(e, e.GetID()) - s.uncommitted.UpsertNamespaceEntry(e, e.GetID()) + _ = u.ForEachNamespaceEntry(func(e nstree.NamespaceEntry) error { + s.committed.UpsertNamespaceEntry(e, e.GetID(), e.GetMVCCTimestamp()) + s.uncommitted.UpsertNamespaceEntry(e, e.GetID(), e.GetMVCCTimestamp()) return nil }) _ = u.ForEachDescriptorEntry(func(d catalog.Descriptor) error { diff --git a/pkg/sql/scrub.go b/pkg/sql/scrub.go index 7f133c475f5c..b0f6521b2c52 100644 --- a/pkg/sql/scrub.go +++ b/pkg/sql/scrub.go @@ -178,7 +178,7 @@ func (n *scrubNode) startScrubDatabase(ctx context.Context, p *planner, name *tr for i := range tbNames { tableName := &tbNames[i] - _, objDesc, err := p.Accessor().GetObjectDesc( + _, objDesc, err := p.descCollection.GetObjectByName( ctx, p.txn, tableName.Catalog(), tableName.Schema(), tableName.Table(), p.ObjectLookupFlags(true /*required*/, false /*requireMutable*/), ) diff --git a/pkg/sql/sem/tree/name_resolution.go b/pkg/sql/sem/tree/name_resolution.go index e6695b851da3..0f0c352e1676 100644 --- a/pkg/sql/sem/tree/name_resolution.go +++ b/pkg/sql/sem/tree/name_resolution.go @@ -165,6 +165,9 @@ type CommonLookupFlags struct { // AvoidLeased, if set, avoid the leased (possibly stale) version of the // descriptor. It must be set when callers want consistent reads. AvoidLeased bool + // AvoidCommittedAdding specifies if committed descriptors in the adding state + // will be ignored. + AvoidCommittedAdding bool // IncludeOffline specifies if offline descriptors should be visible. IncludeOffline bool // IncludeOffline specifies if dropped descriptors should be visible. @@ -176,7 +179,7 @@ type CommonLookupFlags struct { // SchemaLookupFlags is the flag struct suitable for GetSchemaByName(). type SchemaLookupFlags = CommonLookupFlags -// DatabaseLookupFlags is the flag struct suitable for GetDatabaseDesc(). +// DatabaseLookupFlags is the flag struct suitable for GetImmutableDatabaseByName(). type DatabaseLookupFlags = CommonLookupFlags // DatabaseListFlags is the flag struct suitable for GetObjectNamesAndIDs(). @@ -238,7 +241,7 @@ func (r RequiredTableKind) String() string { return requiredTypeNames[r] } -// ObjectLookupFlags is the flag struct suitable for GetObjectDesc(). +// ObjectLookupFlags is the flag struct suitable for GetObjectByName(). type ObjectLookupFlags struct { CommonLookupFlags AllowWithoutPrimaryKey bool diff --git a/pkg/sql/set_schema.go b/pkg/sql/set_schema.go index 61bab844409e..f307e8ebafe1 100644 --- a/pkg/sql/set_schema.go +++ b/pkg/sql/set_schema.go @@ -42,10 +42,11 @@ func (p *planner) prepareSetSchema( } // Lookup the schema we want to set to. - res, err := p.Descriptors().GetMutableSchemaByName( + res, err := p.Descriptors().GetImmutableSchemaByName( ctx, p.txn, db, schema, tree.SchemaLookupFlags{ Required: true, RequireMutable: true, + AvoidLeased: true, }) if err != nil { return 0, err diff --git a/pkg/sql/show_create.go b/pkg/sql/show_create.go index 2fb9f74a4fbb..bdf44a1b9c19 100644 --- a/pkg/sql/show_create.go +++ b/pkg/sql/show_create.go @@ -229,9 +229,7 @@ func (p *planner) ShowCreate( } else if desc.IsSequence() { stmt, err = ShowCreateSequence(ctx, &tn, desc) } else { - lCtx, lErr := newInternalLookupCtxFromDescriptorProtos( - ctx, allDescs, nil, /* want all tables */ - ) + lCtx, lErr := newInternalLookupCtxFromDescriptorProtos(ctx, allDescs) if lErr != nil { return "", lErr } diff --git a/pkg/sql/temporary_schema.go b/pkg/sql/temporary_schema.go index 6596b373c5bc..29655d151b9d 100644 --- a/pkg/sql/temporary_schema.go +++ b/pkg/sql/temporary_schema.go @@ -96,17 +96,12 @@ var ( } ) -// TemporarySchemaNameForRestorePrefix is the prefix name of the schema we -// synthesize during a full cluster restore. All temporary objects being -// restored are remapped to belong to this schema allowing the reconciliation -// job to gracefully clean up these objects when it runs. -const TemporarySchemaNameForRestorePrefix string = "pg_temp_0_" - func (p *planner) getOrCreateTemporarySchema( ctx context.Context, db catalog.DatabaseDescriptor, ) (catalog.SchemaDescriptor, error) { tempSchemaName := p.TemporarySchemaName() - sc, err := p.Descriptors().GetMutableSchemaByName(ctx, p.txn, db, tempSchemaName, p.CommonLookupFlags(false)) + flags := tree.CommonLookupFlags{AvoidLeased: true} + sc, err := p.Descriptors().GetImmutableSchemaByName(ctx, p.txn, db, tempSchemaName, flags) if sc != nil || err != nil { return sc, err } @@ -124,7 +119,7 @@ func (p *planner) getOrCreateTemporarySchema( m.SetTemporarySchemaName(sKey.GetName()) m.SetTemporarySchemaIDForDatabase(uint32(db.GetID()), uint32(id)) }) - return p.Descriptors().GetImmutableSchemaByID(ctx, p.Txn(), id, p.CommonLookupFlags(true)) + return p.Descriptors().GetImmutableSchemaByID(ctx, p.Txn(), id, p.CommonLookupFlagsRequired()) } // CreateSchemaNamespaceEntry creates an entry for the schema in the diff --git a/pkg/storage/engine.go b/pkg/storage/engine.go index 971b2cddc366..2a7152d2ca42 100644 --- a/pkg/storage/engine.go +++ b/pkg/storage/engine.go @@ -265,10 +265,11 @@ type MVCCIterator interface { // ValueProto unmarshals the value the iterator is currently // pointing to using a protobuf decoder. ValueProto(msg protoutil.Message) error - // FindSplitKey finds a key from the given span such that the left side of - // the split is roughly targetSize bytes. The returned key will never be - // chosen from the key ranges listed in keys.NoSplitSpans and will always - // sort equal to or after minSplitKey. + // FindSplitKey finds a key from the given span such that the left side of the + // split is roughly targetSize bytes. It only considers MVCC point keys, not + // range keys. The returned key will never be chosen from the key ranges + // listed in keys.NoSplitSpans and will always sort equal to or after + // minSplitKey. // // DO NOT CALL directly (except in wrapper MVCCIterator implementations). Use the // package-level MVCCFindSplitKey instead. For correct operation, the caller @@ -1111,11 +1112,13 @@ func GetIntent(reader Reader, key roachpb.Key) (*roachpb.Intent, error) { return &intent, nil } -// Scan returns up to max key/value objects starting from start (inclusive) -// and ending at end (non-inclusive). Specify max=0 for unbounded scans. Since -// this code may use an intentInterleavingIter, the caller should not attempt -// a single scan to span local and global keys. See the comment in the -// declaration of intentInterleavingIter for details. +// Scan returns up to max point key/value objects from start (inclusive) to end +// (non-inclusive). Specify max=0 for unbounded scans. Since this code may use +// an intentInterleavingIter, the caller should not attempt a single scan to +// span local and global keys. See the comment in the declaration of +// intentInterleavingIter for details. +// +// NB: This function ignores MVCC range keys. It should only be used for tests. func Scan(reader Reader, start, end roachpb.Key, max int64) ([]MVCCKeyValue, error) { var kvs []MVCCKeyValue err := reader.MVCCIterate(start, end, MVCCKeyAndIntentsIterKind, IterKeyTypePointsOnly, diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 625f2a63348a..1acb1a3cc680 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -5303,8 +5303,9 @@ func CanGCEntireRange( } // MVCCFindSplitKey finds a key from the given span such that the left side of -// the split is roughly targetSize bytes. The returned key will never be chosen -// from the key ranges listed in keys.NoSplitSpans. +// the split is roughly targetSize bytes. It only considers MVCC point keys, not +// range keys. The returned key will never be chosen from the key ranges listed +// in keys.NoSplitSpans. func MVCCFindSplitKey( _ context.Context, reader Reader, key, endKey roachpb.RKey, targetSize int64, ) (roachpb.Key, error) { @@ -5712,19 +5713,31 @@ func computeStatsForIterWithVisitors( return ms, nil } -// MVCCIsSpanEmpty returns true if there are no MVCC keys whatsoever in the -// key span in the requested time interval. +// MVCCIsSpanEmpty returns true if there are no MVCC keys whatsoever in the key +// span in the requested time interval. If a time interval is given and any +// inline values are encountered, an error may be returned. func MVCCIsSpanEmpty( ctx context.Context, reader Reader, opts MVCCIsSpanEmptyOptions, ) (isEmpty bool, _ error) { - iter := NewMVCCIncrementalIterator(reader, MVCCIncrementalIterOptions{ - KeyTypes: IterKeyTypePointsAndRanges, - StartKey: opts.StartKey, - EndKey: opts.EndKey, - StartTime: opts.StartTS, - EndTime: opts.EndTS, - IntentPolicy: MVCCIncrementalIterIntentPolicyEmit, - }) + // Only use an MVCCIncrementalIterator if time bounds are given, since it will + // error on any inline values, and the caller may want to respect them instead. + var iter SimpleMVCCIterator + if opts.StartTS.IsEmpty() && opts.EndTS.IsEmpty() { + iter = reader.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{ + KeyTypes: IterKeyTypePointsAndRanges, + LowerBound: opts.StartKey, + UpperBound: opts.EndKey, + }) + } else { + iter = NewMVCCIncrementalIterator(reader, MVCCIncrementalIterOptions{ + KeyTypes: IterKeyTypePointsAndRanges, + StartKey: opts.StartKey, + EndKey: opts.EndKey, + StartTime: opts.StartTS, + EndTime: opts.EndTS, + IntentPolicy: MVCCIncrementalIterIntentPolicyEmit, + }) + } defer iter.Close() iter.SeekGE(MVCCKey{Key: opts.StartKey}) valid, err := iter.Valid() diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index f48549401a9b..b398b18cffea 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -1273,7 +1273,7 @@ func cmdIsSpanEmpty(e *evalCtx) error { if err != nil { return err } - e.results.buf.Print(isEmpty) + e.results.buf.Printf("%t\n", isEmpty) return nil } diff --git a/pkg/storage/testdata/mvcc_histories/clear_range b/pkg/storage/testdata/mvcc_histories/clear_range index 42dec063413c..602c70d58bb6 100644 --- a/pkg/storage/testdata/mvcc_histories/clear_range +++ b/pkg/storage/testdata/mvcc_histories/clear_range @@ -1,5 +1,6 @@ -# Populate some values +# Populate some values. The inline value is a special case in +# that it will cause an error if time bounds are specified. run ok with t=A v=abc resolve @@ -9,6 +10,7 @@ with t=A v=abc resolve put k=b put k=b/123 put k=c +put k=i v=inline ---- >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=44.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=44.000000000,0 wto=false gul=0,0 @@ -17,16 +19,26 @@ data: "a/123"/44.000000000,0 -> /BYTES/abc data: "b"/44.000000000,0 -> /BYTES/abc data: "b/123"/44.000000000,0 -> /BYTES/abc data: "c"/44.000000000,0 -> /BYTES/abc +meta: "i"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +# Last test case sees inline value but does not error +# since no time bound specified. run ok is_span_empty k=a end=+a +is_span_empty k=a end=z +is_span_empty k=a end=+a startTs=45 +is_span_empty k=i end=z ---- false +false +true +false -run ok -is_span_empty k=a end=z +# Case in which inline value is encountered under time bounds. +run error +is_span_empty k=i startTs=0 ts=1 ---- -false +error: (*withstack.withStack:) unexpected inline value found: "i" run ok clear_range k=a end=+a @@ -36,6 +48,7 @@ data: "a/123"/44.000000000,0 -> /BYTES/abc data: "b"/44.000000000,0 -> /BYTES/abc data: "b/123"/44.000000000,0 -> /BYTES/abc data: "c"/44.000000000,0 -> /BYTES/abc +meta: "i"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false run ok is_span_empty k=a end=+a @@ -54,6 +67,7 @@ clear_range k=a end=-a data: "b"/44.000000000,0 -> /BYTES/abc data: "b/123"/44.000000000,0 -> /BYTES/abc data: "c"/44.000000000,0 -> /BYTES/abc +meta: "i"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false run ok is_span_empty k=a end=-a @@ -67,6 +81,7 @@ clear_range k=a end==b data: "b"/44.000000000,0 -> /BYTES/abc data: "b/123"/44.000000000,0 -> /BYTES/abc data: "c"/44.000000000,0 -> /BYTES/abc +meta: "i"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false run ok clear_range k=a end=+b @@ -74,15 +89,33 @@ clear_range k=a end=+b >> at end: data: "b/123"/44.000000000,0 -> /BYTES/abc data: "c"/44.000000000,0 -> /BYTES/abc +meta: "i"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false run ok clear_range k=a end=-b ---- >> at end: data: "c"/44.000000000,0 -> /BYTES/abc +meta: "i"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false run ok clear_range k=a end=-c ---- >> at end: +meta: "i"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false + +run ok +is_span_empty k=a end=z +---- +false + +run ok +clear_range k=i end=+i +---- +>> at end: + +run ok +is_span_empty k=a end=z +---- +true