Skip to content

Commit 3317b2c

Browse files
craig[bot]msbutler
andcommitted
Merge #153663
153663: backup: delete handling of pre 22.1 legacy checkpoint files r=jeffswenson a=msbutler Epic: none Release note: none Co-authored-by: Michael Butler <[email protected]>
2 parents 3cb6e13 + 794533c commit 3317b2c

File tree

3 files changed

+20
-105
lines changed

3 files changed

+20
-105
lines changed

pkg/backup/backup_job.go

Lines changed: 3 additions & 37 deletions
Original file line numberDiff line numberDiff line change
@@ -1809,33 +1809,9 @@ func (b *backupResumer) readManifestOnResume(
18091809
// they could be using either the new or the old foreign key
18101810
// representations. We should just preserve whatever representation the
18111811
// table descriptors were using and leave them alone.
1812-
desc, memSize, err := backupinfo.ReadBackupCheckpointManifest(ctx, mem, defaultStore,
1813-
backupinfo.BackupManifestCheckpointName, details.EncryptionOptions, kmsEnv)
1814-
if err != nil {
1815-
if !errors.Is(err, cloud.ErrFileDoesNotExist) {
1816-
return nil, 0, errors.Wrapf(err, "reading backup checkpoint")
1817-
}
1818-
// Try reading temp checkpoint.
1819-
tmpCheckpoint := backupinfo.TempCheckpointFileNameForJob(b.job.ID())
1820-
desc, memSize, err = backupinfo.ReadBackupCheckpointManifest(ctx, mem, defaultStore,
1821-
tmpCheckpoint, details.EncryptionOptions, kmsEnv)
1822-
if err != nil {
1823-
return nil, 0, err
1824-
}
1825-
// "Rename" temp checkpoint.
1826-
if err := backupinfo.WriteBackupManifestCheckpoint(
1827-
ctx, details.URI, details.EncryptionOptions, kmsEnv, &desc, cfg, user,
1828-
); err != nil {
1829-
mem.Shrink(ctx, memSize)
1830-
return nil, 0, errors.Wrapf(err, "renaming temp checkpoint file")
1831-
}
1832-
// Best effort remove temp checkpoint.
1833-
if err := defaultStore.Delete(ctx, tmpCheckpoint); err != nil {
1834-
log.Dev.Errorf(ctx, "error removing temporary checkpoint %s", tmpCheckpoint)
1835-
}
1836-
if err := defaultStore.Delete(ctx, backupinfo.BackupProgressDirectory+"/"+tmpCheckpoint); err != nil {
1837-
log.Dev.Errorf(ctx, "error removing temporary checkpoint %s", backupinfo.BackupProgressDirectory+"/"+tmpCheckpoint)
1838-
}
1812+
desc, memSize, err := backupinfo.ReadBackupCheckpointManifest(ctx, mem, defaultStore, details.EncryptionOptions, kmsEnv)
1813+
if err != nil && !errors.Is(err, cloud.ErrFileDoesNotExist) {
1814+
return nil, 0, errors.Wrapf(err, "reading backup checkpoint")
18391815
}
18401816

18411817
if !desc.ClusterID.Equal(cfg.NodeInfo.LogicalClusterID()) {
@@ -1973,16 +1949,6 @@ func (b *backupResumer) deleteCheckpoint(
19731949
return err
19741950
}
19751951
defer exportStore.Close()
1976-
// We first attempt to delete from base directory to account for older
1977-
// backups, and then from the progress directory.
1978-
err = exportStore.Delete(ctx, backupinfo.BackupManifestCheckpointName)
1979-
if err != nil {
1980-
log.Dev.Warningf(ctx, "unable to delete checkpointed backup descriptor file in base directory: %+v", err)
1981-
}
1982-
err = exportStore.Delete(ctx, backupinfo.BackupManifestCheckpointName+backupinfo.BackupManifestChecksumSuffix)
1983-
if err != nil {
1984-
log.Dev.Warningf(ctx, "unable to delete checkpoint checksum file in base directory: %+v", err)
1985-
}
19861952
// Delete will not delete a nonempty directory, so we have to go through
19871953
// all files and delete each file one by one.
19881954
return exportStore.List(ctx, backupinfo.BackupProgressDirectory, "", func(p string) error {

pkg/backup/backup_test.go

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1739,7 +1739,6 @@ func TestBackupRestoreResume(t *testing.T) {
17391739
checkpointDirectory string
17401740
}{
17411741
{testName: "backup-progress-directory", checkpointDirectory: "/" + backupinfo.BackupProgressDirectory},
1742-
{testName: "backup-base-directory", checkpointDirectory: ""},
17431742
} {
17441743
item := item
17451744
t.Run(item.testName, func(t *testing.T) {

pkg/backup/backupinfo/manifest_handling.go

Lines changed: 17 additions & 67 deletions
Original file line numberDiff line numberDiff line change
@@ -197,26 +197,27 @@ func DecompressData(ctx context.Context, mem *mon.BoundAccount, descBytes []byte
197197

198198
// ReadBackupCheckpointManifest reads and unmarshals a BACKUP-CHECKPOINT
199199
// manifest from filename in the provided export store.
200+
//
201+
// NB: a checkpoint file should always get written before this is called.
200202
func ReadBackupCheckpointManifest(
201203
ctx context.Context,
202204
mem *mon.BoundAccount,
203205
exportStore cloud.ExternalStorage,
204-
filename string,
205206
encryption *jobspb.BackupEncryptionOptions,
206207
kmsEnv cloud.KMSEnv,
207208
) (backuppb.BackupManifest, int64, error) {
208209
ctx, sp := tracing.ChildSpan(ctx, "backupinfo.ReadBackupCheckpointManifest")
209210
defer sp.Finish()
210211

211-
checkpointFile, err := readLatestCheckpointFile(ctx, exportStore, filename)
212+
checkpointFile, err := readLatestCheckpointFile(ctx, exportStore, BackupManifestCheckpointName)
212213
if err != nil {
213214
return backuppb.BackupManifest{}, 0, err
214215
}
215216
defer checkpointFile.Close(ctx)
216217

217218
// Look for a checksum, if one is not found it could be an older backup,
218219
// but we want to continue anyway.
219-
checksumFile, err := readLatestCheckpointFile(ctx, exportStore, filename+BackupManifestChecksumSuffix)
220+
checksumFile, err := readLatestCheckpointFile(ctx, exportStore, BackupManifestCheckpointName+BackupManifestChecksumSuffix)
220221
if err != nil {
221222
if !errors.Is(err, cloud.ErrFileDoesNotExist) {
222223
return backuppb.BackupManifest{}, 0, err
@@ -1278,17 +1279,6 @@ func CheckForBackupLock(
12781279
// 2) BACKUP-LOCK: Written by the coordinator node to lay claim on a backup
12791280
// location. This file is suffixed with the ID of the backup job to prevent a
12801281
// node from reading its own lock file on job resumption.
1281-
//
1282-
// 3) BACKUP-CHECKPOINT: Prior to 22.1.1, nodes would use the BACKUP-CHECKPOINT
1283-
// to lay claim on a backup location. To account for a mixed-version cluster
1284-
// where an older coordinator node may be running a concurrent backup to the
1285-
// same location, we must continue to check for a BACKUP-CHECKPOINT file.
1286-
//
1287-
// NB: The node will continue to write a BACKUP-CHECKPOINT file later in its
1288-
// execution, but we do not have to worry about reading our own
1289-
// BACKUP-CHECKPOINT file (and locking ourselves out) since
1290-
// `checkForPreviousBackup` is invoked as the first step on job resumption, and
1291-
// is not called again.
12921282
func CheckForPreviousBackup(
12931283
ctx context.Context,
12941284
execCfg *sql.ExecutorConfig,
@@ -1343,35 +1333,9 @@ func CheckForPreviousBackup(
13431333
}
13441334
log.Dev.Warningf(ctx, "external storage %s does not support listing: skip checking for BACKUP_LOCK", redactedURI)
13451335
}
1346-
1347-
// Check for a BACKUP-CHECKPOINT that might have been written by a node
1348-
// running a pre-22.1.1 binary.
1349-
//
1350-
// TODO(adityamaru): Delete in 23.1 since we will no longer need to check for
1351-
// BACKUP-CHECKPOINT files as all backups will rely on BACKUP-LOCK to lock a
1352-
// location.
1353-
r, err = readLatestCheckpointFile(ctx, defaultStore, BackupManifestCheckpointName)
1354-
if err == nil {
1355-
r.Close(ctx)
1356-
return pgerror.Newf(pgcode.FileAlreadyExists,
1357-
"%s already contains a %s file (is another operation already in progress?)",
1358-
redactedURI, BackupManifestCheckpointName)
1359-
}
1360-
1361-
if !errors.Is(err, cloud.ErrFileDoesNotExist) {
1362-
return errors.Wrapf(err,
1363-
"%s returned an unexpected error when checking for the existence of %s file",
1364-
redactedURI, BackupManifestCheckpointName)
1365-
}
1366-
13671336
return nil
13681337
}
13691338

1370-
// TempCheckpointFileNameForJob returns temporary filename for backup manifest checkpoint.
1371-
func TempCheckpointFileNameForJob(jobID jobspb.JobID) string {
1372-
return fmt.Sprintf("%s-%d", BackupManifestCheckpointName, jobID)
1373-
}
1374-
13751339
// BackupManifestDescriptors returns the descriptors encoded in the manifest as
13761340
// a slice of mutable descriptors.
13771341
func BackupManifestDescriptors(
@@ -1440,11 +1404,8 @@ func newDescriptorBuilder(
14401404
return nil
14411405
}
14421406

1443-
// WriteBackupManifestCheckpoint writes a new BACKUP-CHECKPOINT MANIFEST and
1444-
// CHECKSUM file. If it is a pure v22.1 cluster or later, it will write a
1445-
// timestamped BACKUP-CHECKPOINT to the /progress directory. If it is a mixed
1446-
// cluster version, it will write a non timestamped BACKUP-CHECKPOINT to the
1447-
// base directory in order to not break backup jobs that resume on a v21.2 node.
1407+
// WriteBackupManifestCheckpoint writes a BACKUP-CHECKPOINT file and CHECKSUM
1408+
// file.
14481409
func WriteBackupManifestCheckpoint(
14491410
ctx context.Context,
14501411
storageURI string,
@@ -1545,12 +1506,9 @@ func readLatestCheckpointFile(
15451506
ctx context.Context, exportStore cloud.ExternalStorage, filename string,
15461507
) (ioctx.ReadCloserCtx, error) {
15471508
filename = strings.TrimPrefix(filename, "/")
1548-
// First try reading from the progress directory. If the backup is from
1549-
// an older version, it may not exist there yet so try reading
1550-
// in the base directory if the first attempt fails.
1509+
15511510
var checkpoint string
15521511
var checkpointFound bool
1553-
var r ioctx.ReadCloserCtx
15541512
var err error
15551513

15561514
// We name files such that the most recent checkpoint will always
@@ -1571,7 +1529,7 @@ func readLatestCheckpointFile(
15711529
// directly. This can still fail if it is a mixed cluster and the
15721530
// checkpoint was written in the base directory.
15731531
if errors.Is(err, cloud.ErrListingUnsupported) {
1574-
r, _, err = exportStore.ReadFile(ctx, BackupProgressDirectory+"/"+filename, cloud.ReadOptions{NoFileSize: true})
1532+
r, _, err := exportStore.ReadFile(ctx, BackupProgressDirectory+"/"+filename, cloud.ReadOptions{NoFileSize: true})
15751533
// If we found the checkpoint in progress, then don't bother reading
15761534
// from base, just return the reader.
15771535
if err == nil {
@@ -1581,26 +1539,18 @@ func readLatestCheckpointFile(
15811539
return nil, err
15821540
}
15831541

1584-
if checkpointFound {
1585-
var name string
1586-
if strings.HasSuffix(filename, BackupManifestChecksumSuffix) {
1587-
name = BackupProgressDirectory + "/" + checkpoint + BackupManifestChecksumSuffix
1588-
} else {
1589-
name = BackupProgressDirectory + "/" + checkpoint
1590-
}
1591-
r, _, err = exportStore.ReadFile(ctx, name, cloud.ReadOptions{NoFileSize: true})
1592-
return r, err
1542+
if !checkpointFound {
1543+
return nil, errors.Newf("latest file %s not found", filename)
15931544
}
15941545

1595-
// If the checkpoint wasn't found in the progress directory, then try
1596-
// reading from the base directory instead.
1597-
r, _, err = exportStore.ReadFile(ctx, filename, cloud.ReadOptions{NoFileSize: true})
1598-
1599-
if err != nil {
1600-
return nil, errors.Wrapf(err, "%s could not be read in the base or progress directory", filename)
1546+
var name string
1547+
if strings.HasSuffix(filename, BackupManifestChecksumSuffix) {
1548+
name = BackupProgressDirectory + "/" + checkpoint + BackupManifestChecksumSuffix
1549+
} else {
1550+
name = BackupProgressDirectory + "/" + checkpoint
16011551
}
1602-
return r, nil
1603-
1552+
r, _, err := exportStore.ReadFile(ctx, name, cloud.ReadOptions{NoFileSize: true})
1553+
return r, err
16041554
}
16051555

16061556
// NewTimestampedCheckpointFileName returns a string of a new checkpoint filename

0 commit comments

Comments
 (0)