Skip to content

Commit

Permalink
backupccl: elide expensive ShowCreate call in SHOW BACKUP
Browse files Browse the repository at this point in the history
In #88376 we see this call taking ~all the time on a cluster with
2.5K empty tables. In all cases except `SHOW BACKUP SCHEMAS` we do not
need to construct the SQL representation of the table's schema. This
results in a marked improvement in the performance of `SHOW BACKUP`
as can be seen in #88376 (comment).

Fixes: #88376

Release note (performance improvement): `SHOW BACKUP` on a backup containing
several table descriptors is now more performant
  • Loading branch information
adityamaru committed Sep 23, 2022
1 parent 4f3bb99 commit cc68174
Show file tree
Hide file tree
Showing 5 changed files with 54 additions and 18 deletions.
56 changes: 38 additions & 18 deletions pkg/ccl/backupccl/show.go
Original file line number Diff line number Diff line change
Expand Up @@ -670,6 +670,8 @@ func backupShowerDefault(

var rows []tree.Datums
for layer, manifest := range info.manifests {
ctx, sp := tracing.ChildSpan(ctx, "backupccl.backupShowerDefault.fn.layer")

// Map database ID to descriptor name.
dbIDToName := make(map[descpb.ID]string)
schemaIDToName := make(map[descpb.ID]string)
Expand All @@ -692,11 +694,12 @@ func backupShowerDefault(
}
}
}

var fileSizes []int64
if len(info.fileSizes) > 0 {
fileSizes = info.fileSizes[layer]
}
tableSizes, err := getTableSizes(manifest.Files, fileSizes)
tableSizes, err := getTableSizes(ctx, manifest.Files, fileSizes)
if err != nil {
return nil, err
}
Expand Down Expand Up @@ -772,18 +775,23 @@ func backupShowerDefault(
rowCountDatum = tree.NewDInt(tree.DInt(tableSize.rowCount.Rows))
fileSizeDatum = tree.NewDInt(tree.DInt(tableSize.fileSize))

displayOptions := sql.ShowCreateDisplayOptions{
FKDisplayMode: sql.OmitMissingFKClausesFromCreate,
IgnoreComments: true,
}
createStmt, err := p.ShowCreate(ctx, dbName, manifest.Descriptors,
tabledesc.NewBuilder(desc.TableDesc()).BuildImmutableTable(), displayOptions)
if err != nil {
// We expect that we might get an error here due to X-DB
// references, which were possible on 20.2 betas and rcs.
log.Errorf(ctx, "error while generating create statement: %+v", err)
// Only resolve the table schemas if running `SHOW BACKUP SCHEMAS`.
// In all other cases we discard these results and so it is wasteful
// to construct the SQL representation of the table's schema.
if showSchemas {
displayOptions := sql.ShowCreateDisplayOptions{
FKDisplayMode: sql.OmitMissingFKClausesFromCreate,
IgnoreComments: true,
}
createStmt, err := p.ShowCreate(ctx, dbName, manifest.Descriptors,
tabledesc.NewBuilder(desc.TableDesc()).BuildImmutableTable(), displayOptions)
if err != nil {
// We expect that we might get an error here due to X-DB
// references, which were possible on 20.2 betas and rcs.
log.Errorf(ctx, "error while generating create statement: %+v", err)
}
createStmtDatum = nullIfEmpty(createStmt)
}
createStmtDatum = nullIfEmpty(createStmt)
default:
descriptorType = "unknown"
}
Expand All @@ -805,7 +813,7 @@ func backupShowerDefault(
row = append(row, createStmtDatum)
}
if _, shouldShowPrivileges := opts[backupOptWithPrivileges]; shouldShowPrivileges {
row = append(row, tree.NewDString(showPrivileges(descriptor)))
row = append(row, tree.NewDString(showPrivileges(ctx, descriptor)))
owner := desc.GetPrivileges().Owner().SQLIdentifier()
row = append(row, tree.NewDString(owner))
}
Expand Down Expand Up @@ -867,6 +875,7 @@ func backupShowerDefault(
}
rows = append(rows, row)
}
sp.Finish()
}
return rows, nil
},
Expand All @@ -881,7 +890,11 @@ type descriptorSize struct {
// getLogicalSSTSize gets the total logical bytes stored in each SST. Note that a
// BackupManifest_File identifies a span in an SST and there can be multiple
// spans stored in an SST.
func getLogicalSSTSize(files []backuppb.BackupManifest_File) map[string]int64 {
func getLogicalSSTSize(ctx context.Context, files []backuppb.BackupManifest_File) map[string]int64 {
ctx, span := tracing.ChildSpan(ctx, "backupccl.getLogicalSSTSize")
defer span.Finish()
_ = ctx

sstDataSize := make(map[string]int64)
for _, file := range files {
sstDataSize[file.Path] += file.EntryCounts.DataSize
Expand All @@ -898,8 +911,11 @@ func approximateSpanPhysicalSize(

// getTableSizes gathers row and size count for each table in the manifest
func getTableSizes(
files []backuppb.BackupManifest_File, fileSizes []int64,
ctx context.Context, files []backuppb.BackupManifest_File, fileSizes []int64,
) (map[descpb.ID]descriptorSize, error) {
ctx, span := tracing.ChildSpan(ctx, "backupccl.getTableSizes")
defer span.Finish()

tableSizes := make(map[descpb.ID]descriptorSize)
if len(files) == 0 {
return tableSizes, nil
Expand All @@ -910,7 +926,7 @@ func getTableSizes(
}
showCodec := keys.MakeSQLCodec(tenantID)

logicalSSTSize := getLogicalSSTSize(files)
logicalSSTSize := getLogicalSSTSize(ctx, files)

for i, file := range files {
// TODO(dan): This assumes each file in the backup only
Expand Down Expand Up @@ -985,7 +1001,11 @@ func showRegions(typeDesc catalog.TypeDescriptor, dbname string) (string, error)
return regionsStringBuilder.String(), nil
}

func showPrivileges(descriptor *descpb.Descriptor) string {
func showPrivileges(ctx context.Context, descriptor *descpb.Descriptor) string {
ctx, span := tracing.ChildSpan(ctx, "backupccl.showPrivileges")
defer span.Finish()
_ = ctx // ctx is currently unused, but this new ctx should be used below in the future.

var privStringBuilder strings.Builder

b := descbuilder.NewBuilder(descriptor)
Expand Down Expand Up @@ -1173,7 +1193,7 @@ func backupShowerFileSetup(inCol tree.StringOrPlaceholderOptList) backupShower {
backupType = "incremental"
}

logicalSSTSize := getLogicalSSTSize(manifest.Files)
logicalSSTSize := getLogicalSSTSize(ctx, manifest.Files)
for j, file := range manifest.Files {
filePath := file.Path
if inCol != nil {
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/catalog/descs/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,7 @@ go_library(
"//pkg/util/log",
"//pkg/util/mon",
"//pkg/util/retry",
"//pkg/util/tracing",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_redact//:redact",
"@com_github_lib_pq//oid",
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/catalog/descs/hydrate.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
)

// hydrateDescriptors installs user defined type metadata in all types.T present
Expand Down Expand Up @@ -151,6 +152,9 @@ func makeImmutableTypeLookupFunc(
// 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 {
ctx, sp := tracing.ChildSpan(ctx, "descs.HydrateCatalog")
defer sp.Finish()

fakeLookupFunc := func(_ context.Context, id descpb.ID) (catalog.Descriptor, error) {
return nil, catalog.WrapDescRefErr(id, catalog.ErrDescriptorNotFound)
}
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/resolver.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sqlerrors"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/errors"
"github.com/lib/pq/oid"
)
Expand Down Expand Up @@ -989,6 +990,9 @@ type tableLookupFn = *internalLookupCtx
func newInternalLookupCtxFromDescriptorProtos(
ctx context.Context, rawDescs []descpb.Descriptor,
) (*internalLookupCtx, error) {
ctx, sp := tracing.ChildSpan(ctx, "sql.newInternalLookupCtxFromDescriptorProtos")
defer sp.Finish()

var c nstree.MutableCatalog
for i := range rawDescs {
desc := descbuilder.NewBuilder(&rawDescs[i]).BuildImmutable()
Expand Down
7 changes: 7 additions & 0 deletions pkg/sql/show_create.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
)

type shouldOmitFKClausesFromCreate int
Expand Down Expand Up @@ -72,6 +73,9 @@ func ShowCreateTable(
lCtx simpleSchemaResolver,
displayOptions ShowCreateDisplayOptions,
) (string, error) {
ctx, sp := tracing.ChildSpan(ctx, "sql.ShowCreateTable")
defer sp.Finish()

a := &tree.DatumAlloc{}

f := p.ExtendedEvalContext().FmtCtx(tree.FmtSimple)
Expand Down Expand Up @@ -221,6 +225,9 @@ func (p *planner) ShowCreate(
desc catalog.TableDescriptor,
displayOptions ShowCreateDisplayOptions,
) (string, error) {
ctx, sp := tracing.ChildSpan(ctx, "sql.ShowCreate")
defer sp.Finish()

var stmt string
var err error
tn := tree.MakeUnqualifiedTableName(tree.Name(desc.GetName()))
Expand Down

0 comments on commit cc68174

Please sign in to comment.