Skip to content

Commit

Permalink
sql/catalog/descs: implement interfaces on *DistSQLTypeResolver
Browse files Browse the repository at this point in the history
This commit updates `DistSQLTypeResolver` to implement all interfaces on a
pointer receiver instead of a value receiver. Implementing interfaces on values
is rarely the right choice, as it forces a heap allocation whenever the object
is boxed into an interface, instead of just forcing the pointer onto the heap
once (on its own or as part of a larger object) and then storing the pointer in
the interface header.

Before this commit, the use of value receivers was causing `HydrateTypeSlice` to
allocate. Outside of #72798 and #72961, this was the single largest source of
heap allocations in TPC-E. With those two PRs applied, `HydrateTypeSlice` was
accounting for **2.30%** of total heap allocations in the workload:

```
----------------------------------------------------------+-------------
      flat  flat%   sum%        cum   cum%   calls calls% + context
----------------------------------------------------------+-------------
                                          27722149 32.66% |   github.com/cockroachdb/cockroach/pkg/sql/execinfra.(*ProcessorBase).InitWithEvalCtx /go/src/github.com/cockroachdb/cockroach/pkg/sql/execinfra/processorsbase.go:790
                                          27460002 32.36% |   github.com/cockroachdb/cockroach/pkg/sql/colflow.(*vectorizedFlowCreator).setupFlow.func1 /go/src/github.com/cockroachdb/cockroach/pkg/sql/colflow/vectorized_flow.go:1097
                                          21266755 25.06% |   github.com/cockroachdb/cockroach/pkg/sql/colflow.(*vectorizedFlowCreator).setupInput /go/src/github.com/cockroachdb/cockroach/pkg/sql/colflow/vectorized_flow.go:818
                                           8421503  9.92% |   github.com/cockroachdb/cockroach/pkg/sql/colfetcher.populateTableArgs /go/src/github.com/cockroachdb/cockroach/pkg/sql/colfetcher/cfetcher_setup.go:174
  84870409  2.30%  2.30%   84870409  2.30%                | github.com/cockroachdb/cockroach/pkg/sql/catalog/descs.DistSQLTypeResolver.HydrateTypeSlice /go/src/github.com/cockroachdb/cockroach/pkg/sql/catalog/descs/dist_sql_type_resolver.go:134
----------------------------------------------------------+-------------
```

With this change, the heap allocation in `HydrateTypeSlice` disappears.

With these three PRs combined, the largest source of heap allocations in the
workload is `context.WithValue`, like the Go gods intended.
```
----------------------------------------------------------+-------------
      flat  flat%   sum%        cum   cum%   calls calls% + context
----------------------------------------------------------+-------------
                                          16723340 38.17% |   github.com/cockroachdb/logtags.WithTags /go/src/github.com/cockroachdb/cockroach/vendor/github.com/cockroachdb/logtags/context.go:34
                                           7405899 16.90% |   google.golang.org/grpc/peer.NewContext /go/src/github.com/cockroachdb/cockroach/vendor/google.golang.org/grpc/peer/peer.go:44
                                           3910493  8.93% |   google.golang.org/grpc.NewContextWithServerTransportStream /go/src/github.com/cockroachdb/cockroach/vendor/google.golang.org/grpc/server.go:1672
                                           3702950  8.45% |   github.com/cockroachdb/cockroach/pkg/util/tracing.maybeWrapCtx /go/src/github.com/cockroachdb/cockroach/pkg/util/tracing/context.go:80
                                           3560952  8.13% |   google.golang.org/grpc/metadata.NewIncomingContext /go/src/github.com/cockroachdb/cockroach/vendor/google.golang.org/grpc/metadata/metadata.go:152
                                           3342479  7.63% |   google.golang.org/grpc.newContextWithRPCInfo /go/src/github.com/cockroachdb/cockroach/vendor/google.golang.org/grpc/rpc_util.go:791
                                           2938326  6.71% |   google.golang.org/grpc/internal/credentials.NewRequestInfoContext /go/src/github.com/cockroachdb/cockroach/vendor/google.golang.org/grpc/internal/credentials/credentials.go:29
                                           1387235  3.17% |   github.com/cockroachdb/cockroach/pkg/util/grpcutil.NewLocalRequestContext /go/src/github.com/cockroachdb/cockroach/pkg/util/grpcutil/grpc_util.go:39
                                            655388  1.50% |   github.com/cockroachdb/cockroach/pkg/sql.withStatement /go/src/github.com/cockroachdb/cockroach/pkg/sql/conn_executor.go:3197
                                            185693  0.42% |   google.golang.org/grpc/metadata.NewOutgoingContext /go/src/github.com/cockroachdb/cockroach/vendor/google.golang.org/grpc/metadata/metadata.go:159
  43812755  2.20%  2.20%   43812755  2.20%                | context.WithValue /usr/local/go/src/context/context.go:533
----------------------------------------------------------+-------------
```
  • Loading branch information
nvanbenschoten committed Nov 19, 2021
1 parent 945da18 commit 1b59a8c
Show file tree
Hide file tree
Showing 4 changed files with 16 additions and 12 deletions.
5 changes: 3 additions & 2 deletions pkg/sql/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -701,9 +701,10 @@ func (sc *SchemaChanger) validateConstraints(
// print the check expression back to the user.
evalCtx.Txn = txn
// Use the DistSQLTypeResolver because we need to resolve types by ID.
semaCtx := tree.MakeSemaContext()
collection := evalCtx.Descs
semaCtx.TypeResolver = descs.NewDistSQLTypeResolver(collection, txn)
resolver := descs.NewDistSQLTypeResolver(collection, txn)
semaCtx := tree.MakeSemaContext()
semaCtx.TypeResolver = &resolver
// TODO (rohany): When to release this? As of now this is only going to get released
// after the check is validated.
defer func() { collection.ReleaseAll(ctx) }()
Expand Down
8 changes: 4 additions & 4 deletions pkg/sql/backfill/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -201,12 +201,12 @@ func (cb *ColumnBackfiller) InitForDistributedUse(
if err := flowCtx.Cfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error {
resolver := flowCtx.TypeResolverFactory.NewTypeResolver(txn)
// Hydrate all the types present in the table.
if err := typedesc.HydrateTypesInTableDescriptor(ctx, desc.TableDesc(), resolver); err != nil {
if err := typedesc.HydrateTypesInTableDescriptor(ctx, desc.TableDesc(), &resolver); err != nil {
return err
}
// Set up a SemaContext to type check the default and computed expressions.
semaCtx := tree.MakeSemaContext()
semaCtx.TypeResolver = resolver
semaCtx.TypeResolver = &resolver
var err error
defaultExprs, err = schemaexpr.MakeDefaultExprs(
ctx, cb.added, &transform.ExprTransformContext{}, evalCtx, &semaCtx,
Expand Down Expand Up @@ -600,13 +600,13 @@ func (ib *IndexBackfiller) InitForDistributedUse(
resolver := flowCtx.TypeResolverFactory.NewTypeResolver(txn)
// Hydrate all the types present in the table.
if err = typedesc.HydrateTypesInTableDescriptor(
ctx, desc.TableDesc(), resolver,
ctx, desc.TableDesc(), &resolver,
); err != nil {
return err
}
// Set up a SemaContext to type check the default and computed expressions.
semaCtx := tree.MakeSemaContext()
semaCtx.TypeResolver = resolver
semaCtx.TypeResolver = &resolver
// Convert any partial index predicate strings into expressions.
predicates, colExprs, referencedColumns, err = constructExprs(
ctx, desc, ib.added, ib.cols, ib.addedCols, ib.computedCols, evalCtx, &semaCtx,
Expand Down
13 changes: 8 additions & 5 deletions pkg/sql/catalog/descs/dist_sql_type_resolver.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,8 +49,9 @@ func (df *DistSQLTypeResolverFactory) NewTypeResolver(txn *kv.Txn) DistSQLTypeRe
// NewSemaContext creates a new SemaContext with a TypeResolver bound to the
// input transaction.
func (df *DistSQLTypeResolverFactory) NewSemaContext(txn *kv.Txn) *tree.SemaContext {
resolver := df.NewTypeResolver(txn)
semaCtx := tree.MakeSemaContext()
semaCtx.TypeResolver = df.NewTypeResolver(txn)
semaCtx.TypeResolver = &resolver
return &semaCtx
}

Expand All @@ -70,14 +71,16 @@ func NewDistSQLTypeResolver(descs *Collection, txn *kv.Txn) DistSQLTypeResolver
}

// ResolveType implements the tree.TypeReferenceResolver interface.
func (dt DistSQLTypeResolver) ResolveType(
func (dt *DistSQLTypeResolver) ResolveType(
context.Context, *tree.UnresolvedObjectName,
) (*types.T, error) {
return nil, errors.AssertionFailedf("cannot resolve types in DistSQL by name")
}

// ResolveTypeByOID implements the tree.TypeReferenceResolver interface.
func (dt DistSQLTypeResolver) ResolveTypeByOID(ctx context.Context, oid oid.Oid) (*types.T, error) {
func (dt *DistSQLTypeResolver) ResolveTypeByOID(
ctx context.Context, oid oid.Oid,
) (*types.T, error) {
id, err := typedesc.UserDefinedTypeOIDToID(oid)
if err != nil {
return nil, err
Expand All @@ -90,7 +93,7 @@ func (dt DistSQLTypeResolver) ResolveTypeByOID(ctx context.Context, oid oid.Oid)
}

// GetTypeDescriptor implements the sqlbase.TypeDescriptorResolver interface.
func (dt DistSQLTypeResolver) GetTypeDescriptor(
func (dt *DistSQLTypeResolver) GetTypeDescriptor(
ctx context.Context, id descpb.ID,
) (tree.TypeName, catalog.TypeDescriptor, error) {
flags := tree.CommonLookupFlags{
Expand Down Expand Up @@ -129,7 +132,7 @@ func (dt DistSQLTypeResolver) GetTypeDescriptor(
}

// HydrateTypeSlice installs metadata into a slice of types.T's.
func (dt DistSQLTypeResolver) HydrateTypeSlice(ctx context.Context, typs []*types.T) error {
func (dt *DistSQLTypeResolver) HydrateTypeSlice(ctx context.Context, typs []*types.T) error {
for _, t := range typs {
if err := typedesc.EnsureTypeIsHydrated(ctx, t, dt); err != nil {
return err
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/execinfra/processorsbase.go
Original file line number Diff line number Diff line change
Expand Up @@ -790,7 +790,7 @@ func (pb *ProcessorBase) InitWithEvalCtx(
return err
}
pb.SemaCtx = tree.MakeSemaContext()
pb.SemaCtx.TypeResolver = resolver
pb.SemaCtx.TypeResolver = &resolver

return pb.OutputHelper.Init(post, coreOutputTypes, &pb.SemaCtx, pb.EvalCtx)
}
Expand Down

0 comments on commit 1b59a8c

Please sign in to comment.