diff --git a/docs/generated/sql/bnf/show_backup.bnf b/docs/generated/sql/bnf/show_backup.bnf index 5f1570a0b75c..446da5e23b4d 100644 --- a/docs/generated/sql/bnf/show_backup.bnf +++ b/docs/generated/sql/bnf/show_backup.bnf @@ -1,11 +1,11 @@ show_backup_stmt ::= - 'SHOW' 'BACKUPS' 'IN' location - | 'SHOW' 'BACKUP' show_backup_details 'FROM' string_or_placeholder 'IN' string_or_placeholder 'WITH' kv_option_list - | 'SHOW' 'BACKUP' show_backup_details 'FROM' string_or_placeholder 'IN' string_or_placeholder 'WITH' 'OPTIONS' '(' kv_option_list ')' - | 'SHOW' 'BACKUP' show_backup_details 'FROM' string_or_placeholder 'IN' string_or_placeholder - | 'SHOW' 'BACKUP' subdirectory 'IN' location 'WITH' kv_option_list - | 'SHOW' 'BACKUP' subdirectory 'IN' location 'WITH' 'OPTIONS' '(' kv_option_list ')' - | 'SHOW' 'BACKUP' subdirectory 'IN' location + 'SHOW' 'BACKUPS' 'IN' location_opt_list + | 'SHOW' 'BACKUP' show_backup_details 'FROM' string_or_placeholder 'IN' string_or_placeholder_opt_list 'WITH' kv_option_list + | 'SHOW' 'BACKUP' show_backup_details 'FROM' string_or_placeholder 'IN' string_or_placeholder_opt_list 'WITH' 'OPTIONS' '(' kv_option_list ')' + | 'SHOW' 'BACKUP' show_backup_details 'FROM' string_or_placeholder 'IN' string_or_placeholder_opt_list + | 'SHOW' 'BACKUP' subdirectory 'IN' location_opt_list 'WITH' kv_option_list + | 'SHOW' 'BACKUP' subdirectory 'IN' location_opt_list 'WITH' 'OPTIONS' '(' kv_option_list ')' + | 'SHOW' 'BACKUP' subdirectory 'IN' location_opt_list | 'SHOW' 'BACKUP' string_or_placeholder 'WITH' kv_option_list | 'SHOW' 'BACKUP' string_or_placeholder 'WITH' 'OPTIONS' '(' kv_option_list ')' | 'SHOW' 'BACKUP' string_or_placeholder diff --git a/docs/generated/sql/bnf/stmt_block.bnf b/docs/generated/sql/bnf/stmt_block.bnf index d29256947ebc..1f2d34618a6e 100644 --- a/docs/generated/sql/bnf/stmt_block.bnf +++ b/docs/generated/sql/bnf/stmt_block.bnf @@ -699,9 +699,9 @@ use_stmt ::= 'USE' var_value show_backup_stmt ::= - 'SHOW' 'BACKUPS' 'IN' string_or_placeholder - | 'SHOW' 'BACKUP' show_backup_details 'FROM' string_or_placeholder 'IN' string_or_placeholder opt_with_options - | 'SHOW' 'BACKUP' string_or_placeholder 'IN' string_or_placeholder opt_with_options + 'SHOW' 'BACKUPS' 'IN' string_or_placeholder_opt_list + | 'SHOW' 'BACKUP' show_backup_details 'FROM' string_or_placeholder 'IN' string_or_placeholder_opt_list opt_with_options + | 'SHOW' 'BACKUP' string_or_placeholder 'IN' string_or_placeholder_opt_list opt_with_options | 'SHOW' 'BACKUP' string_or_placeholder opt_with_options | 'SHOW' 'BACKUP' 'SCHEMAS' string_or_placeholder opt_with_options | 'SHOW' 'BACKUP' 'FILES' string_or_placeholder opt_with_options diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index b9f5b67fdc7c..5c96263bdec7 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -650,6 +650,7 @@ func TestBackupAndRestoreJobDescription(t *testing.T) { } sqlDB.Exec(t, "BACKUP TO ($1, $2, $3)", backups...) + sqlDB.Exec(t, "BACKUP TO ($1,$2,$3) INCREMENTAL FROM $4", append(incrementals, backups[0])...) sqlDB.Exec(t, "BACKUP INTO ($1, $2, $3)", collections...) sqlDB.Exec(t, "BACKUP INTO LATEST IN ($1, $2, $3)", collections...) sqlDB.Exec(t, "BACKUP INTO LATEST IN ($1, $2, $3) WITH incremental_location = ($4, $5, $6)", @@ -665,6 +666,12 @@ func TestBackupAndRestoreJobDescription(t *testing.T) { time.Sleep(time.Second + 2) sqlDB.Exec(t, "BACKUP INTO ($1, $2, $3) AS OF SYSTEM TIME '-1s'", collections...) + { + // Ensure old style show backup runs properly with locality aware uri + sqlDB.Exec(t, "SHOW BACKUP $1", backups[0]) + sqlDB.Exec(t, "SHOW BACKUP $1", incrementals[0]) + } + // Find the subdirectory created by the full BACKUP INTO statement. matches, err := filepath.Glob(path.Join(tmpDir, "full/*/*/*/"+backupManifestName)) require.NoError(t, err) @@ -680,6 +687,8 @@ func TestBackupAndRestoreJobDescription(t *testing.T) { [][]string{ {fmt.Sprintf("BACKUP TO ('%s', '%s', '%s')", backups[0].(string), backups[1].(string), backups[2].(string))}, + {fmt.Sprintf("BACKUP TO ('%s', '%s', '%s') INCREMENTAL FROM '%s'", incrementals[0], + incrementals[1], incrementals[2], backups[0])}, {fmt.Sprintf("BACKUP INTO '%s' IN ('%s', '%s', '%s')", full1, collections[0], collections[1], collections[2])}, {fmt.Sprintf("BACKUP INTO '%s' IN ('%s', '%s', '%s')", full1, @@ -9581,6 +9590,7 @@ func TestBackupRestoreOldIncrementalDefault(t *testing.T) { sib := fmt.Sprintf("BACKUP DATABASE fkdb INTO LATEST IN %s WITH incremental_location = %s", dest, inc) sqlDB.Exec(t, sib) + sir := fmt.Sprintf("RESTORE DATABASE fkdb FROM LATEST IN %s WITH new_db_name = 'inc_fkdb'", dest) sqlDB.Exec(t, sir) @@ -9690,6 +9700,39 @@ func TestBackupRestoreSeparateExplicitIsDefault(t *testing.T) { sib := fmt.Sprintf("BACKUP DATABASE fkdb INTO LATEST IN %s WITH incremental_location = %s", dest, inc) sqlDB.Exec(t, sib) + { + // Locality Aware Show Backup validation + // TODO (msbutler): move to data driven test after 22.1 backport + + // Assert the localities field populates correctly (not null if backup is locality aware). + localities := sqlDB.QueryStr(t, + fmt.Sprintf("SELECT locality FROM [SHOW BACKUP FILES FROM LATEST IN %s]", dest)) + expectedLocalities := map[string]bool{"default": true, "dc=dc1": true, "dc=dc2": true} + for _, locality := range localities { + if len(br.dest) > 1 { + _, ok := expectedLocalities[locality[0]] + require.Equal(t, true, ok) + } else { + require.Equal(t, "NULL", locality[0]) + } + } + // Assert show backup still works. + sqlDB.Exec(t, fmt.Sprintf("SHOW BACKUPS IN %s", dest)) + sqlDB.Exec(t, fmt.Sprintf("SHOW BACKUP FROM LATEST IN %s", dest)) + + if len(br.dest) > 1 { + // Locality aware show backups will eventually fail if not all localities are provided, + // but for now, they're ok. + sqlDB.Exec(t, fmt.Sprintf("SHOW BACKUP FROM LATEST IN %s", br.dest[1])) + + errorMsg := "SHOW BACKUP on locality aware backups using incremental_location is not" + + " supported yet" + sqlDB.ExpectErr(t, errorMsg, fmt.Sprintf("SHOW BACKUP FROM LATEST IN %s WITH incremental_location= %s", dest, br.inc[0])) + } else { + // non locality aware show backup with incremental_location should work! + sqlDB.Exec(t, fmt.Sprintf("SHOW BACKUP FROM LATEST IN %s WITH incremental_location= %s", dest, inc)) + } + } sir := fmt.Sprintf("RESTORE DATABASE fkdb FROM LATEST IN %s WITH new_db_name = 'inc_fkdb', incremental_location = %s", dest, inc) sqlDB.Exec(t, sir) diff --git a/pkg/ccl/backupccl/manifest_handling.go b/pkg/ccl/backupccl/manifest_handling.go index 2beecf46d34f..016e85ea0cf9 100644 --- a/pkg/ccl/backupccl/manifest_handling.go +++ b/pkg/ccl/backupccl/manifest_handling.go @@ -629,6 +629,8 @@ func loadBackupManifests( return backupManifests, memSize, nil } +var errLocalityDescriptor = errors.New(`Locality Descriptor not found`) + // getLocalityInfo takes a list of stores and their URIs, along with the main // backup manifest searches each for the locality pieces listed in the the // main manifest, returning the mapping. @@ -672,7 +674,7 @@ func getLocalityInfo( } } if !found { - return info, errors.Errorf("expected manifest %s not found in backup locations", filename) + return info, errors.Mark(errors.Newf("expected manifest %s not found in backup locations", filename), errLocalityDescriptor) } } info.URIsByOriginalLocalityKV = urisByOrigLocality diff --git a/pkg/ccl/backupccl/show.go b/pkg/ccl/backupccl/show.go index 170cec2af6d8..a763d1129adc 100644 --- a/pkg/ccl/backupccl/show.go +++ b/pkg/ccl/backupccl/show.go @@ -10,7 +10,6 @@ package backupccl import ( "context" - "net/url" "path" "strings" "time" @@ -20,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants" @@ -34,6 +34,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/protoreflect" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/json" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/mon" @@ -42,26 +43,28 @@ import ( "github.com/cockroachdb/errors" ) -func checkShowBackupURIPrivileges(ctx context.Context, p sql.PlanHookState, uri string) error { - conf, err := cloud.ExternalStorageConfFromURI(uri, p.User()) - if err != nil { - return err - } - if conf.AccessIsWithExplicitAuth() { - return nil - } - if p.ExecCfg().ExternalIODirConfig.EnableNonAdminImplicitAndArbitraryOutbound { - return nil - } - hasAdmin, err := p.HasAdminRole(ctx) - if err != nil { - return err - } - if !hasAdmin { - return pgerror.Newf( - pgcode.InsufficientPrivilege, - "only users with the admin role are allowed to SHOW BACKUP from the specified %s URI", - conf.Provider.String()) +func checkShowBackupURIPrivileges(ctx context.Context, p sql.PlanHookState, uris []string) error { + for _, uri := range uris { + conf, err := cloud.ExternalStorageConfFromURI(uri, p.User()) + if err != nil { + return err + } + if conf.AccessIsWithExplicitAuth() { + continue + } + if p.ExecCfg().ExternalIODirConfig.EnableNonAdminImplicitAndArbitraryOutbound { + continue + } + hasAdmin, err := p.HasAdminRole(ctx) + if err != nil { + return err + } + if !hasAdmin { + return pgerror.Newf( + pgcode.InsufficientPrivilege, + "only users with the admin role are allowed to SHOW BACKUP from the specified %s URI", + conf.Provider.String()) + } } return nil } @@ -70,11 +73,9 @@ type backupInfoReader interface { showBackup( context.Context, *mon.BoundAccount, - cloud.ExternalStorage, - cloud.ExternalStorage, - *jobspb.BackupEncryptionOptions, - []string, - []string, + cloud.ExternalStorageFromURIFactory, + backupInfo, + security.SQLUsername, chan<- tree.Datums, ) error header() colinfo.ResultColumns @@ -93,54 +94,19 @@ func (m manifestInfoReader) header() colinfo.ResultColumns { // showBackup reads backup info from the manifest, populates the manifestInfoReader, // calls the backupShower to process the manifest info into datums, // and pipes the information to the user's sql console via the results channel. - -// TODO(msbutler): during the old backup syntax purge, remove store, incStore, incPaths, -// and pass only `stores []cloud.ExternalStorage` object in signature func (m manifestInfoReader) showBackup( ctx context.Context, mem *mon.BoundAccount, - store cloud.ExternalStorage, - incStore cloud.ExternalStorage, - enc *jobspb.BackupEncryptionOptions, - incPaths []string, - manifestDirs []string, + mkStore cloud.ExternalStorageFromURIFactory, + info backupInfo, + user security.SQLUsername, resultsCh chan<- tree.Datums, ) error { - var memSize int64 + var memReserved int64 + defer func() { - mem.Shrink(ctx, memSize) + mem.Shrink(ctx, memReserved) }() - - var err error - manifests := make([]BackupManifest, len(incPaths)+1) - manifests[0], memSize, err = ReadBackupManifestFromStore(ctx, mem, store, enc) - - if err != nil { - if errors.Is(err, cloud.ErrFileDoesNotExist) { - latestFileExists, errLatestFile := checkForLatestFileInCollection(ctx, store) - - if errLatestFile == nil && latestFileExists { - return errors.WithHintf(err, "The specified path is the root of a backup collection. "+ - "Use SHOW BACKUPS IN with this path to list all the backup subdirectories in the"+ - " collection. SHOW BACKUP can be used with any of these subdirectories to inspect a"+ - " backup.") - } - return errors.CombineErrors(err, errLatestFile) - } - return err - } - - for i := range incPaths { - m, sz, err := readBackupManifest(ctx, mem, incStore, incPaths[i], enc) - if err != nil { - return err - } - memSize += sz - // Blank the stats to prevent memory blowup. - m.DeprecatedStatistics = nil - manifests[i+1] = m - } - // Ensure that the descriptors in the backup manifests are up to date. // // This is necessary in particular for upgrading descriptors with old-style @@ -149,12 +115,13 @@ func (m manifestInfoReader) showBackup( // FKs for which we can't resolve the cross-table references. We can't // display them anyway, because we don't have the referenced table names, // etc. - err = maybeUpgradeDescriptorsInBackupManifests(manifests, true /* skipFKsWithNoMatchingTable */) + err := maybeUpgradeDescriptorsInBackupManifests(info.manifests, + true /* skipFKsWithNoMatchingTable */) if err != nil { return err } - datums, err := m.shower.fn(manifests, manifestDirs) + datums, err := m.shower.fn(info) if err != nil { return err } @@ -184,11 +151,9 @@ func (m metadataSSTInfoReader) header() colinfo.ResultColumns { func (m metadataSSTInfoReader) showBackup( ctx context.Context, mem *mon.BoundAccount, - store cloud.ExternalStorage, - incStore cloud.ExternalStorage, - enc *jobspb.BackupEncryptionOptions, - incPaths []string, - manifestDirs []string, + mkStore cloud.ExternalStorageFromURIFactory, + info backupInfo, + user security.SQLUsername, resultsCh chan<- tree.Datums, ) error { filename := metadataSSTName @@ -204,14 +169,13 @@ func (m metadataSSTInfoReader) showBackup( return nil } } - - if err := DebugDumpMetadataSST(ctx, store, filename, enc, push); err != nil { - return err - } - - for _, i := range incPaths { - filename = strings.TrimSuffix(i, backupManifestName) + metadataSSTName - if err := DebugDumpMetadataSST(ctx, incStore, filename, enc, push); err != nil { + for _, uri := range info.defaultURIs { + store, err := mkStore(ctx, uri, user) + if err != nil { + return errors.Wrapf(err, "creating external store") + } + defer store.Close() + if err := DebugDumpMetadataSST(ctx, store, filename, info.enc, push); err != nil { return err } } @@ -236,9 +200,9 @@ func showBackupPlanHook( return nil, nil, nil, false, err } - var inColFn func() (string, error) + var inColFn func() ([]string, error) if backup.InCollection != nil { - inColFn, err = p.TypeAsString(ctx, backup.InCollection, "SHOW BACKUP") + inColFn, err = p.TypeAsStringArray(ctx, tree.Exprs(backup.InCollection), "SHOW BACKUP") if err != nil { return nil, nil, nil, false, err } @@ -288,20 +252,25 @@ func showBackupPlanHook( ctx, span := tracing.ChildSpan(ctx, stmt.StatementTag()) defer span.Finish() - dest, err := toFn() + var ( + dest []string + subdir string + ) + // For old style show backup, 'to' is the resolved path to the full backup; + // for new SHOW BACKUP, 'to' is the subdirectory. + to, err := toFn() if err != nil { return err } - var subdir string - if inColFn != nil { - subdir = dest + subdir = to dest, err = inColFn() if err != nil { return err } } else { + dest = append(dest, to) // Deprecation notice for old `SHOW BACKUP` syntax. Remove this once the syntax is // deleted in 22.2. p.BufferClientNotice(ctx, @@ -312,6 +281,7 @@ func showBackupPlanHook( " restoreable in the next major version release. Use `BACKUP INTO` instead.", "https://www.cockroachlabs.com/docs/stable/show-backup.html")) } + if err := checkShowBackupURIPrivileges(ctx, p, dest); err != nil { return err } @@ -319,31 +289,32 @@ func showBackupPlanHook( fullyResolvedDest := dest if subdir != "" { if strings.EqualFold(subdir, latestFileName) { - subdir, err = readLatestFile(ctx, dest, p.ExecCfg().DistSQLSrv.ExternalStorageFromURI, p.User()) + subdir, err = readLatestFile(ctx, dest[0], p.ExecCfg().DistSQLSrv.ExternalStorageFromURI, + p.User()) if err != nil { return errors.Wrap(err, "read LATEST path") } } - parsed, err := url.Parse(dest) + fullyResolvedDest, err = appendPaths(dest, subdir) if err != nil { return err } - initialPath := parsed.Path - parsed.Path = JoinURLPath(initialPath, subdir) - fullyResolvedDest = parsed.String() } - - store, err := p.ExecCfg().DistSQLSrv.ExternalStorageFromURI(ctx, fullyResolvedDest, p.User()) - if err != nil { - return errors.Wrapf(err, "make storage") + baseStores := make([]cloud.ExternalStorage, len(fullyResolvedDest)) + for j := range fullyResolvedDest { + baseStores[j], err = p.ExecCfg().DistSQLSrv.ExternalStorageFromURI(ctx, fullyResolvedDest[j], p.User()) + if err != nil { + return errors.Wrapf(err, "make storage") + } + defer baseStores[j].Close() } - defer store.Close() + // TODO(msbutler): put encryption resolution in helper function, hopefully shared with RESTORE // A user that calls SHOW BACKUP on an encrypted incremental // backup will need to pass their full backup's directory to the // encryption_info_dir parameter because the `ENCRYPTION-INFO` file // necessary to decode the incremental backup lives in the full backup dir. - encStore := store + encStore := baseStores[0] if encDir, ok := opts[backupOptEncDir]; ok { encStore, err = p.ExecCfg().DistSQLSrv.ExternalStorageFromURI(ctx, encDir, p.User()) if err != nil { @@ -397,10 +368,14 @@ you must pass the 'encryption_info_dir' parameter that points to the directory o explicitIncPath := opts[backupOptIncStorage] if len(explicitIncPath) > 0 { explicitIncPaths = append(explicitIncPaths, explicitIncPath) + if len(dest) > 1 { + return errors.New("SHOW BACKUP on locality aware backups using incremental_location is" + + " not supported yet") + } } - collection, computedSubdir := CollectionAndSubdir(dest, subdir) - incLocations, err := resolveIncrementalsBackupLocation( + collection, computedSubdir := CollectionAndSubdir(dest[0], subdir) + fullyResolvedIncrementalsDirectory, err := resolveIncrementalsBackupLocation( ctx, p.User(), p.ExecCfg(), @@ -408,9 +383,6 @@ you must pass the 'encryption_info_dir' parameter that points to the directory o []string{collection}, computedSubdir, ) - var incPaths []string - var manifestDirs []string - var incStore cloud.ExternalStorage if err != nil { if errors.Is(err, cloud.ErrListingUnsupported) { // We can proceed with base backups here just fine, so log a warning and move on. @@ -420,62 +392,74 @@ you must pass the 'encryption_info_dir' parameter that points to the directory o } else { return err } - } else if len(incLocations) > 0 { - // There was no error, so check for incrementals. - // - // If there are incrementals, find the backup paths and return them with the store. - // Otherwise, use the vacuous placeholders above. + } + mem := p.ExecCfg().RootMemoryMonitor.MakeBoundAccount() + defer mem.Close(ctx) - incStore, err = p.ExecCfg().DistSQLSrv.ExternalStorageFromURI(ctx, incLocations[0], p.User()) - if err != nil { + var ( + info backupInfo + memReserved int64 + ) + info.subdir = computedSubdir + + mkStore := p.ExecCfg().DistSQLSrv.ExternalStorageFromURI + + info.defaultURIs, info.manifests, info.localityInfo, memReserved, + err = resolveBackupManifests( + ctx, &mem, baseStores, mkStore, fullyResolvedDest, + fullyResolvedIncrementalsDirectory, hlc.Timestamp{}, encryption, p.User()) + defer func() { + mem.Shrink(ctx, memReserved) + }() + if err != nil { + if errors.Is(err, errLocalityDescriptor) && subdir == "" { + p.BufferClientNotice(ctx, + pgnotice.Newf("`SHOW BACKUP` using the old syntax ("+ + "without the `IN` keyword) on a locality aware backup does not display or validate"+ + " data specific to locality aware backups. "+ + "Consider using the new `BACKUP INTO` syntax and `SHOW BACKUP"+ + " FROM IN `")) + } else if errors.Is(err, cloud.ErrFileDoesNotExist) { + latestFileExists, errLatestFile := checkForLatestFileInCollection(ctx, baseStores[0]) + + if errLatestFile == nil && latestFileExists { + return errors.WithHintf(err, "The specified path is the root of a backup collection. "+ + "Use SHOW BACKUPS IN with this path to list all the backup subdirectories in the"+ + " collection. SHOW BACKUP can be used with any of these subdirectories to inspect a"+ + " backup.") + } + return errors.CombineErrors(err, errLatestFile) + } else { return err } - incPaths, err = FindPriorBackups(ctx, incStore, IncludeManifest) - if err != nil { - return errors.Wrapf(err, "make incremental storage") + } + // If backup is locality aware, check that user passed at least some localities. + + // TODO (msbutler): this is an extremely crude check that the user is + // passing at least as many URIS as there are localities in the backup. This + // check is only meant for the 22.1 backport. Ben is working on a much more + // robust check. + for _, locMap := range info.localityInfo { + if len(locMap.URIsByOriginalLocalityKV) > len(dest) && subdir != "" { + p.BufferClientNotice(ctx, + pgnotice.Newf("The backup contains %d localities; however, "+ + "the SHOW BACKUP command contains only %d URIs. To capture all locality aware data, "+ + "pass every locality aware URI from the backup", len(locMap.URIsByOriginalLocalityKV), + len(dest))) } - manifestDirs = getManifestDirs(computedSubdir, incLocations[0], incPaths, explicitIncPaths) } - mem := p.ExecCfg().RootMemoryMonitor.MakeBoundAccount() - defer mem.Close(ctx) - - return infoReader.showBackup(ctx, &mem, store, incStore, encryption, incPaths, - manifestDirs, resultsCh) + return infoReader.showBackup(ctx, &mem, mkStore, info, p.User(), resultsCh) } return fn, infoReader.header(), nil, false, nil } -// getManifestDirs gathers the path to the directory for each backup manifest, -// relative to the collection root. In other words, path.Join(dest, -// manifestDirs[i]) is the resolved manifest path. If the user passed -// incremental_location, the path.Join(explicitIncPath,manifestDirs[i]) is the -// resolved incremental manifest path. -func getManifestDirs( - fullSubdir string, incLocation string, incPaths []string, explicitIncPaths []string, -) []string { - manifestDirs := make([]string, len(incPaths)+1) - - // The full backup manifest path is always in the fullSubdir - manifestDirs[0] = fullSubdir - - // The incLocation reveals if incremental backups were stored in the full - // backup's subdirectory, the default incremental directory, or in a different - // incremental_location. To figure this out, remove the fullSubdir from the - // incremental location, then check if the defaultIncrementalSubdir is in the - // path AND that the user did not pass explicit incremental paths: - defaultIncSubdir := "" - incLocNoSubdir := strings.Replace(incLocation, fullSubdir, "", 1) - splitIncLocation := strings.Split(incLocNoSubdir, "/") - if splitIncLocation[len(splitIncLocation)-1] == DefaultIncrementalsSubdir && len( - explicitIncPaths) == 0 { - defaultIncSubdir = "/" + DefaultIncrementalsSubdir - } - for i, incPath := range incPaths { - incPathNoManifest := strings.Replace(incPath, backupManifestName, "", 1) - manifestDirs[i+1] = path.Join(defaultIncSubdir, fullSubdir, incPathNoManifest) - } - return manifestDirs +type backupInfo struct { + defaultURIs []string + manifests []BackupManifest + subdir string + localityInfo []jobspb.RestoreDetails_BackupLocalityInfo + enc *jobspb.BackupEncryptionOptions } type backupShower struct { @@ -484,7 +468,7 @@ type backupShower struct { // fn is the specific implementation of the shower that can either be a default, ranges, files, // or JSON shower. - fn func(manifests []BackupManifest, manifestDirs []string) ([]tree.Datums, error) + fn func(info backupInfo) ([]tree.Datums, error) } // backupShowerHeaders defines the schema for the table presented to the user. @@ -530,9 +514,9 @@ func backupShowerDefault( ) backupShower { return backupShower{ header: backupShowerHeaders(showSchemas, opts), - fn: func(manifests []BackupManifest, manifestDirs []string) ([]tree.Datums, error) { + fn: func(info backupInfo) ([]tree.Datums, error) { var rows []tree.Datums - for _, manifest := range manifests { + for _, manifest := range info.manifests { // Map database ID to descriptor name. dbIDToName := make(map[descpb.ID]string) schemaIDToName := make(map[descpb.ID]string) @@ -804,8 +788,8 @@ var backupShowerRanges = backupShower{ {Name: "end_key", Typ: types.Bytes}, }, - fn: func(manifests []BackupManifest, manifestDirs []string) (rows []tree.Datums, err error) { - for _, manifest := range manifests { + fn: func(info backupInfo) (rows []tree.Datums, err error) { + for _, manifest := range info.manifests { for _, span := range manifest.Spans { rows = append(rows, tree.Datums{ tree.NewDString(span.Key.String()), @@ -819,7 +803,7 @@ var backupShowerRanges = backupShower{ }, } -func backupShowerFileSetup(inCol tree.Expr) backupShower { +func backupShowerFileSetup(inCol tree.StringOrPlaceholderOptList) backupShower { return backupShower{header: colinfo.ResultColumns{ {Name: "path", Typ: types.String}, {Name: "backup_type", Typ: types.String}, @@ -829,14 +813,24 @@ func backupShowerFileSetup(inCol tree.Expr) backupShower { {Name: "end_key", Typ: types.Bytes}, {Name: "size_bytes", Typ: types.Int}, {Name: "rows", Typ: types.Int}, + {Name: "locality", Typ: types.String}, }, - fn: func(manifests []BackupManifest, manifestDirs []string) (rows []tree.Datums, err error) { - if (inCol != nil) && len(manifestDirs) == 0 { - return nil, errors.AssertionFailedf( - "manifestDirs empty even though backup is in collection") + fn: func(info backupInfo) (rows []tree.Datums, err error) { + + var manifestDirs []string + var localityAware bool + if len(inCol) > 0 { + manifestDirs, err = getManifestDirs(info.subdir, info.defaultURIs) + if err != nil { + return nil, err + } + + if len(info.localityInfo[0].URIsByOriginalLocalityKV) > 0 { + localityAware = true + } } - for i, manifest := range manifests { + for i, manifest := range info.manifests { backupType := "full" if manifest.isIncremental() { backupType = "incremental" @@ -846,6 +840,16 @@ func backupShowerFileSetup(inCol tree.Expr) backupShower { if inCol != nil { filePath = path.Join(manifestDirs[i], filePath) } + locality := "NULL" + if localityAware { + locality = file.LocalityKV + + // A file stored in the default locality has file.LocalityKV == NULL. + // For the SHOW FILES display, set the locality to default. + if file.LocalityKV == "NULL" { + locality = "default" + } + } rows = append(rows, tree.Datums{ tree.NewDString(filePath), tree.NewDString(backupType), @@ -855,6 +859,7 @@ func backupShowerFileSetup(inCol tree.Expr) backupShower { tree.NewDBytes(tree.DBytes(file.Span.EndKey)), tree.NewDInt(tree.DInt(file.EntryCounts.DataSize)), tree.NewDInt(tree.DInt(file.EntryCounts.Rows)), + tree.NewDString(locality), }) } } @@ -863,14 +868,82 @@ func backupShowerFileSetup(inCol tree.Expr) backupShower { } } +// getRootURI splits a fully resolved backup URI at the backup's subdirectory +// and returns the path to that subdirectory. e.g. for a full backup URI, +// getRootURI returns the collectionURI. +func getRootURI(defaultURI string, subdir string) (string, error) { + splitFullBackupPath := strings.Split(defaultURI, subdir) + if len(splitFullBackupPath) != 2 { + return "", errors.AssertionFailedf( + "the full backup URI %s does not contain 1 instance of the subdir %s"+ + "", defaultURI, subdir) + } + return splitFullBackupPath[0], nil +} + +// getManifestDirs gathers the path to the directory of each backup manifest, +// relative to the collection root. Consider the following example: Suppose a +// backup chain contains a full backup with a defaultURI of +// 'userfile:///foo/fullSubdir' and an incremental backup with a defaultURI of +// 'userfile:///foo/incrementals/fullSubdir/incrementalSubdir'. getManifestDirs +// would return a relative path to the full backup manifest's directory, '/fullSubdir', and +// to the incremental backup manifest's directory +// '/incrementals/fullSubdir/incrementalSubdir'. +func getManifestDirs(fullSubdir string, defaultUris []string) ([]string, error) { + fullCollectionRoot, err := getRootURI(defaultUris[0], fullSubdir) + if err != nil { + return nil, err + } + + manifestDirs := make([]string, len(defaultUris)) + + // The full backup manifest path is always in the fullSubdir. + manifestDirs[0] = fullSubdir + + if len(defaultUris) == 1 { + return manifestDirs, nil + } + incRoot, err := getRootURI(defaultUris[1], fullSubdir) + if err != nil { + return nil, err + } + + var incSubdir string + if path.Join(fullCollectionRoot, DefaultIncrementalsSubdir) == incRoot { + // The incremental backup is stored in the default incremental + // directory (i.e. collectionURI/incrementals/fullSubdir) + incSubdir = path.Join(DefaultIncrementalsSubdir, fullSubdir) + } else { + // Implies one of two scenarios: + // 1) the incremental chain is stored in the pre 22.1 + // default location: collectionURI/fullSubdir. + // 2) incremental backups were created with `incremental_location`, + // so while the path to the subdirectory will be different + // than the full backup's, the incremental backups will have the + // same subdirectory, i.e. the full path is incrementalURI/fullSubdir. + incSubdir = fullSubdir + } + + for i, incURI := range defaultUris { + // The first URI corresponds to the defaultURI of the full backup-- we have already dealt with + // this. + if i == 0 { + continue + } + incDir := strings.Split(incURI, incSubdir)[1] + manifestDirs[i] = path.Join(incSubdir, incDir) + } + return manifestDirs, nil +} + var jsonShower = backupShower{ header: colinfo.ResultColumns{ {Name: "manifest", Typ: types.Jsonb}, }, - fn: func(manifests []BackupManifest, manifestDirs []string) ([]tree.Datums, error) { - rows := make([]tree.Datums, len(manifests)) - for i, manifest := range manifests { + fn: func(info backupInfo) ([]tree.Datums, error) { + rows := make([]tree.Datums, len(info.manifests)) + for i, manifest := range info.manifests { j, err := protoreflect.MessageToJSON( &manifest, protoreflect.FmtFlags{EmitDefaults: true, EmitRedacted: true}) if err != nil { @@ -887,7 +960,7 @@ func showBackupsInCollectionPlanHook( ctx context.Context, backup *tree.ShowBackup, p sql.PlanHookState, ) (sql.PlanHookRowFn, colinfo.ResultColumns, []sql.PlanNode, bool, error) { - collectionFn, err := p.TypeAsString(ctx, backup.InCollection, "SHOW BACKUPS") + collectionFn, err := p.TypeAsStringArray(ctx, tree.Exprs(backup.InCollection), "SHOW BACKUPS") if err != nil { return nil, nil, nil, false, err } @@ -905,7 +978,7 @@ func showBackupsInCollectionPlanHook( return err } - store, err := p.ExecCfg().DistSQLSrv.ExternalStorageFromURI(ctx, collection, p.User()) + store, err := p.ExecCfg().DistSQLSrv.ExternalStorageFromURI(ctx, collection[0], p.User()) if err != nil { return errors.Wrapf(err, "connect to external storage") } diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index cc46f0514ba6..0bce783f3d80 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -5702,28 +5702,28 @@ show_histogram_stmt: // %Text: SHOW BACKUP [SCHEMAS|FILES|RANGES] // %SeeAlso: WEBDOCS/show-backup.html show_backup_stmt: - SHOW BACKUPS IN string_or_placeholder + SHOW BACKUPS IN string_or_placeholder_opt_list { $$.val = &tree.ShowBackup{ - InCollection: $4.expr(), + InCollection: $4.stringOrPlaceholderOptList(), } } -| SHOW BACKUP show_backup_details FROM string_or_placeholder IN string_or_placeholder opt_with_options +| SHOW BACKUP show_backup_details FROM string_or_placeholder IN string_or_placeholder_opt_list opt_with_options { $$.val = &tree.ShowBackup{ From: true, Details: $3.showBackupDetails(), Path: $5.expr(), - InCollection: $7.expr(), + InCollection: $7.stringOrPlaceholderOptList(), Options: $8.kvOptions(), } } -| SHOW BACKUP string_or_placeholder IN string_or_placeholder opt_with_options +| SHOW BACKUP string_or_placeholder IN string_or_placeholder_opt_list opt_with_options { $$.val = &tree.ShowBackup{ Details: tree.BackupDefaultDetails, Path: $3.expr(), - InCollection: $5.expr(), + InCollection: $5.stringOrPlaceholderOptList(), Options: $6.kvOptions(), } } diff --git a/pkg/sql/sem/tree/show.go b/pkg/sql/sem/tree/show.go index 9e64ee716959..6cc1fb16abf1 100644 --- a/pkg/sql/sem/tree/show.go +++ b/pkg/sql/sem/tree/show.go @@ -87,10 +87,13 @@ const ( BackupSchemaDetails ) +// TODO (msbutler): 22.2 after removing old style show backup syntax, rename +// Path to Subdir and InCollection to Dest. + // ShowBackup represents a SHOW BACKUP statement. type ShowBackup struct { Path Expr - InCollection Expr + InCollection StringOrPlaceholderOptList From bool Details ShowBackupDetails Options KVOptions @@ -100,7 +103,7 @@ type ShowBackup struct { func (node *ShowBackup) Format(ctx *FmtCtx) { if node.InCollection != nil && node.Path == nil { ctx.WriteString("SHOW BACKUPS IN ") - ctx.FormatNode(node.InCollection) + ctx.FormatNode(&node.InCollection) return } ctx.WriteString("SHOW BACKUP ") @@ -121,7 +124,7 @@ func (node *ShowBackup) Format(ctx *FmtCtx) { ctx.FormatNode(node.Path) if node.InCollection != nil { ctx.WriteString(" IN ") - ctx.FormatNode(node.InCollection) + ctx.FormatNode(&node.InCollection) } if len(node.Options) > 0 { ctx.WriteString(" WITH ")