Skip to content

Commit

Permalink
Merge pull request #91926 from cockroachdb/blathers/backport-release-…
Browse files Browse the repository at this point in the history
…22.2.0-91911

release-22.2.0: backupccl: fix bug in resolving encrypted backup manifests
  • Loading branch information
adityamaru authored Nov 15, 2022
2 parents 14c3712 + 4783270 commit 77667a1
Show file tree
Hide file tree
Showing 7 changed files with 239 additions and 72 deletions.
26 changes: 18 additions & 8 deletions pkg/ccl/backupccl/backup_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -808,15 +808,25 @@ func getBackupDetailAndManifest(
mem := execCfg.RootMemoryMonitor.MakeBoundAccount()
defer mem.Close(ctx)

prevBackups, encryptionOptions, memSize, err := backupinfo.FetchPreviousBackups(ctx, &mem, user,
makeCloudStorage, backupDestination.PrevBackupURIs, *initialDetails.EncryptionOptions, &kmsEnv)
var prevBackups []backuppb.BackupManifest
var baseEncryptionOptions *jobspb.BackupEncryptionOptions
if len(backupDestination.PrevBackupURIs) != 0 {
var err error
baseEncryptionOptions, err = backupencryption.GetEncryptionFromBase(ctx, user, makeCloudStorage,
backupDestination.PrevBackupURIs[0], *initialDetails.EncryptionOptions, &kmsEnv)
if err != nil {
return jobspb.BackupDetails{}, backuppb.BackupManifest{}, err
}

if err != nil {
return jobspb.BackupDetails{}, backuppb.BackupManifest{}, err
var memSize int64
prevBackups, memSize, err = backupinfo.GetBackupManifests(ctx, &mem, user,
makeCloudStorage, backupDestination.PrevBackupURIs, baseEncryptionOptions, &kmsEnv)

if err != nil {
return jobspb.BackupDetails{}, backuppb.BackupManifest{}, err
}
defer mem.Shrink(ctx, memSize)
}
defer func() {
mem.Shrink(ctx, memSize)
}()

if len(prevBackups) > 0 {
baseManifest := prevBackups[0]
Expand Down Expand Up @@ -889,7 +899,7 @@ func getBackupDetailAndManifest(
backupDestination.ChosenSubdir,
backupDestination.URIsByLocalityKV,
prevBackups,
encryptionOptions,
baseEncryptionOptions,
&kmsEnv)
if err != nil {
return jobspb.BackupDetails{}, backuppb.BackupManifest{}, err
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/backup_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -659,7 +659,7 @@ func backupPlanHook(
if err := requireEnterprise(p.ExecCfg(), "encryption"); err != nil {
return err
}
encryptionParams.RawPassphrae = pw
encryptionParams.RawPassphrase = pw
case jobspb.EncryptionMode_KMS:
encryptionParams.RawKmsUris = kms
if err := requireEnterprise(p.ExecCfg(), "encryption"); err != nil {
Expand Down
40 changes: 18 additions & 22 deletions pkg/ccl/backupccl/backupdest/backup_destination.go
Original file line number Diff line number Diff line change
Expand Up @@ -97,6 +97,8 @@ type ResolvedDestination struct {
URIsByLocalityKV map[string]string

// PrevBackupURIs is the list of full paths for previous backups in the chain.
// This includes the base backup at index 0, and any subsequent incremental
// backups. This field will not be populated when running a full backup.
PrevBackupURIs []string
}

Expand Down Expand Up @@ -535,14 +537,14 @@ func ResolveBackupManifests(
}
ownedMemSize += memSize

var prev []string
var incrementalBackups []string
if len(incStores) > 0 {
prev, err = FindPriorBackups(ctx, incStores[0], includeManifest)
incrementalBackups, err = FindPriorBackups(ctx, incStores[0], includeManifest)
if err != nil {
return nil, nil, nil, 0, err
}
}
numLayers := len(prev) + 1
numLayers := len(incrementalBackups) + 1

defaultURIs = make([]string, numLayers)
mainBackupManifests = make([]backuppb.BackupManifest, numLayers)
Expand Down Expand Up @@ -571,43 +573,37 @@ func ResolveBackupManifests(
}
}

// For each backup layer we construct the default URI. We don't load the
// manifests in this loop since we want to do that concurrently.
for i := range prev {
// prev[i] is the path to the manifest file itself for layer i -- the
// For each incremental backup layer we construct the default URI. We don't
// load the manifests in this loop since we want to do that concurrently.
for i := range incrementalBackups {
// incrementalBackups[i] is the path to the manifest file itself for layer i -- the
// dirname piece of that path is the subdirectory in each of the
// partitions in which we'll also expect to find a partition manifest.
// Recall full inc URI is <prefix>/<subdir>/<incSubDir>
incSubDir := path.Dir(prev[i])
incSubDir := path.Dir(incrementalBackups[i])
u := *baseURIs[0] // NB: makes a copy to avoid mutating the baseURI.
u.Path = backuputils.JoinURLPath(u.Path, incSubDir)
defaultURIs[i+1] = u.String()
}

// Load the default backup manifests for each backup layer, this is done
// concurrently.
enc := jobspb.BackupEncryptionOptions{
Mode: jobspb.EncryptionMode_None,
}
if encryption != nil {
enc = *encryption
}
defaultManifestsForEachLayer, _, memSize, err := backupinfo.FetchPreviousBackups(ctx, mem, user,
mkStore, defaultURIs[1:], enc, kmsEnv)
defaultManifestsForEachLayer, memSize, err := backupinfo.GetBackupManifests(ctx, mem, user,
mkStore, defaultURIs, encryption, kmsEnv)
if err != nil {
return nil, nil, nil, 0, err
}
ownedMemSize += memSize

// Iterate over the layers one last time to memoize the loaded manifests and
// read the locality info.
// Iterate over the incremental backups one last time to memoize the loaded
// manifests and read the locality info.
//
// TODO(adityamaru): Parallelize the loading of the locality descriptors.
for i := range prev {
for i := range incrementalBackups {
// The manifest for incremental layer i slots in at i+1 since the full
// backup manifest occupies index 0 in `mainBackupManifests`.
mainBackupManifests[i+1] = defaultManifestsForEachLayer[i]
incSubDir := path.Dir(prev[i])
mainBackupManifests[i+1] = defaultManifestsForEachLayer[i+1]
incSubDir := path.Dir(incrementalBackups[i])
partitionURIs := make([]string, numPartitions)
for j := range baseURIs {
u := *baseURIs[j] // NB: makes a copy to avoid mutating the baseURI.
Expand All @@ -616,7 +612,7 @@ func ResolveBackupManifests(
}

localityInfo[i+1], err = backupinfo.GetLocalityInfo(ctx, incStores, partitionURIs,
defaultManifestsForEachLayer[i], encryption, kmsEnv, incSubDir)
defaultManifestsForEachLayer[i+1], encryption, kmsEnv, incSubDir)
if err != nil {
return nil, nil, nil, 0, err
}
Expand Down
6 changes: 3 additions & 3 deletions pkg/ccl/backupccl/backupencryption/encryption.go
Original file line number Diff line number Diff line change
Expand Up @@ -237,7 +237,7 @@ func MakeNewEncryptionOptions(
encryptionInfo = &jobspb.EncryptionInfo{Salt: salt}
encryptionOptions = &jobspb.BackupEncryptionOptions{
Mode: jobspb.EncryptionMode_Passphrase,
Key: storageccl.GenerateKey([]byte(encryptionParams.RawPassphrae), salt),
Key: storageccl.GenerateKey([]byte(encryptionParams.RawPassphrase), salt),
}
case jobspb.EncryptionMode_KMS:
// Generate a 32 byte/256-bit crypto-random number which will serve as
Expand Down Expand Up @@ -354,7 +354,7 @@ func WriteNewEncryptionInfoToBackup(
return cloud.WriteFile(ctx, dest, newEncryptionInfoFile, bytes.NewReader(buf))
}

// GetEncryptionFromBase retrieves the encryption options of a base backup. It
// GetEncryptionFromBase retrieves the encryption options of the base backup. It
// is expected that incremental backups use the same encryption options as the
// base backups.
func GetEncryptionFromBase(
Expand All @@ -381,7 +381,7 @@ func GetEncryptionFromBase(
case jobspb.EncryptionMode_Passphrase:
encryptionOptions = &jobspb.BackupEncryptionOptions{
Mode: jobspb.EncryptionMode_Passphrase,
Key: storageccl.GenerateKey([]byte(encryptionParams.RawPassphrae), opts[0].Salt),
Key: storageccl.GenerateKey([]byte(encryptionParams.RawPassphrase), opts[0].Salt),
}
case jobspb.EncryptionMode_KMS:
var defaultKMSInfo *jobspb.BackupEncryptionOptions_KMSInfo
Expand Down
45 changes: 8 additions & 37 deletions pkg/ccl/backupccl/backupinfo/manifest_handling.go
Original file line number Diff line number Diff line change
Expand Up @@ -1291,43 +1291,11 @@ func NewTimestampedCheckpointFileName() string {
return fmt.Sprintf("%s-%s", BackupManifestCheckpointName, hex.EncodeToString(buffer))
}

// FetchPreviousBackups takes a list of URIs of previous backups and returns
// their manifest as well as the encryption options of the first backup in the
// chain.
func FetchPreviousBackups(
ctx context.Context,
mem *mon.BoundAccount,
user username.SQLUsername,
makeCloudStorage cloud.ExternalStorageFromURIFactory,
prevBackupURIs []string,
encryptionParams jobspb.BackupEncryptionOptions,
kmsEnv cloud.KMSEnv,
) ([]backuppb.BackupManifest, *jobspb.BackupEncryptionOptions, int64, error) {
ctx, sp := tracing.ChildSpan(ctx, "backupinfo.FetchPreviousBackups")
defer sp.Finish()

if len(prevBackupURIs) == 0 {
return nil, nil, 0, nil
}

baseBackup := prevBackupURIs[0]
encryptionOptions, err := backupencryption.GetEncryptionFromBase(ctx, user, makeCloudStorage, baseBackup,
encryptionParams, kmsEnv)
if err != nil {
return nil, nil, 0, err
}
prevBackups, size, err := getBackupManifests(ctx, mem, user, makeCloudStorage, prevBackupURIs,
encryptionOptions, kmsEnv)
if err != nil {
return nil, nil, 0, err
}

return prevBackups, encryptionOptions, size, nil
}

// getBackupManifests fetches the backup manifest from a list of backup URIs.
// The manifests are loaded from External Storage in parallel.
func getBackupManifests(
// GetBackupManifests fetches the backup manifest from a list of backup URIs.
// The caller is expected to pass in the fully hydrated encryptionParams
// required to read the manifests. The manifests are loaded from External
// Storage in parallel.
func GetBackupManifests(
ctx context.Context,
mem *mon.BoundAccount,
user username.SQLUsername,
Expand All @@ -1336,6 +1304,9 @@ func getBackupManifests(
encryption *jobspb.BackupEncryptionOptions,
kmsEnv cloud.KMSEnv,
) ([]backuppb.BackupManifest, int64, error) {
ctx, sp := tracing.ChildSpan(ctx, "backupinfo.GetBackupManifests")
defer sp.Finish()

manifests := make([]backuppb.BackupManifest, len(backupURIs))
if len(backupURIs) == 0 {
return manifests, 0, nil
Expand Down
Loading

0 comments on commit 77667a1

Please sign in to comment.