retrieve manifests and metadata from kopia (#1744)

## Description

In a backup operation, begins the operation by
retrieving all backup manifests and metadata
from prior operations.

## Type of change

- [x] 🌻 Feature

## Issue(s)

* #1725

## Test Plan

- [x]  Unit test
- [x] 💚 E2E
This commit is contained in:
Keepers 2022-12-12 15:36:50 -07:00 committed by GitHub
parent 565c33af4e
commit 4d46847f6c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 408 additions and 46 deletions

View File

@ -200,9 +200,14 @@ func connectS3Cmd(cmd *cobra.Command, args []string) error {
}
func s3Overrides() map[string]string {
var (
prvM365 = account.ProviderM365.String()
prvS3 = storage.ProviderS3.String()
)
return map[string]string{
config.AccountProviderTypeKey: account.ProviderM365.String(),
config.StorageProviderTypeKey: storage.ProviderS3.String(),
config.AccountProviderTypeKey: prvM365,
config.StorageProviderTypeKey: prvS3,
storage.Bucket: bucket,
storage.Endpoint: endpoint,
storage.Prefix: prefix,

View File

@ -42,6 +42,8 @@ func (gc *GraphConnector) DataCollections(
return nil, err
}
// serialize metadata into maps here
switch sels.Service {
case selectors.ServiceExchange:
return gc.ExchangeDataCollection(ctx, sels)

View File

@ -13,6 +13,12 @@ import (
// given endpoint. The endpoint granularity varies by service.
const DeltaTokenFileName = "delta"
// MetadataFileNames produces the standard set of filenames used to store graph
// metadata such as delta tokens and folderID->path references.
func MetadataFileNames() []string {
return []string{DeltaTokenFileName}
}
type QueryParams struct {
Category path.CategoryType
ResourceOwner string

View File

@ -35,11 +35,26 @@ type snapshotManager interface {
type OwnersCats struct {
ResourceOwners map[string]struct{}
ServiceCats map[string]struct{}
ServiceCats map[string]ServiceCat
}
type ServiceCat struct {
Service path.ServiceType
Category path.CategoryType
}
// MakeServiceCat produces the expected OwnersCats.ServiceCats key from a
// path service and path category, as well as the ServiceCat value.
func MakeServiceCat(s path.ServiceType, c path.CategoryType) (string, ServiceCat) {
return serviceCatString(s, c), ServiceCat{s, c}
}
func serviceCatTag(p path.Path) string {
return p.Service().String() + p.Category().String()
return serviceCatString(p.Service(), p.Category())
}
func serviceCatString(s path.ServiceType, c path.CategoryType) string {
return s.String() + c.String()
}
// MakeTagKV normalizes the provided key to protect it from clobbering
@ -194,29 +209,34 @@ func fetchPrevManifests(
// newest complete manifest for the tuple. Manifests are deduped such that if
// multiple tuples match the same manifest it will only be returned once.
// External callers can access this via wrapper.FetchPrevSnapshotManifests().
// If tags are provided, manifests must include a superset of the k:v pairs
// specified by those tags. Tags should pass their raw values, and will be
// normalized inside the func using MakeTagKV.
func fetchPrevSnapshotManifests(
ctx context.Context,
sm snapshotManager,
oc *OwnersCats,
tags map[string]string,
) []*snapshot.Manifest {
mans := map[manifest.ID]*snapshot.Manifest{}
tags = normalizeTagKVs(tags)
// For each serviceCat/resource owner pair that we will be backing up, see if
// there's a previous incomplete snapshot and/or a previous complete snapshot
// we can pass in. Can be expanded to return more than the most recent
// snapshots, but may require more memory at runtime.
for serviceCat := range oc.ServiceCats {
serviceTagKey, serviceTagValue := MakeTagKV(serviceCat)
for resourceOwner := range oc.ResourceOwners {
resourceOwnerTagKey, resourceOwnerTagValue := MakeTagKV(resourceOwner)
allTags := normalizeTagKVs(map[string]string{
serviceCat: "",
resourceOwner: "",
})
tags := map[string]string{
serviceTagKey: serviceTagValue,
resourceOwnerTagKey: resourceOwnerTagValue,
for k, v := range tags {
allTags[k] = v
}
found, err := fetchPrevManifests(ctx, sm, mans, tags)
found, err := fetchPrevManifests(ctx, sm, mans, allTags)
if err != nil {
logger.Ctx(ctx).Warnw(
"fetching previous snapshot manifests for service/category/resource owner",
@ -244,3 +264,19 @@ func fetchPrevSnapshotManifests(
return res
}
func normalizeTagKVs(tags map[string]string) map[string]string {
t2 := make(map[string]string, len(tags))
for k, v := range tags {
mk, mv := MakeTagKV(k)
if len(v) == 0 {
v = mv
}
t2[mk] = v
}
return t2
}

View File

@ -41,7 +41,7 @@ var (
testUser2: {},
testUser3: {},
},
ServiceCats: map[string]struct{}{
ServiceCats: map[string]ServiceCat{
testMail: {},
testEvents: {},
},
@ -52,7 +52,7 @@ var (
testUser2: {},
testUser3: {},
},
ServiceCats: map[string]struct{}{
ServiceCats: map[string]ServiceCat{
testMail: {},
},
}
@ -442,7 +442,7 @@ func (suite *SnapshotFetchUnitSuite) TestFetchPrevSnapshots() {
}
}
snaps := fetchPrevSnapshotManifests(ctx, msm, test.input)
snaps := fetchPrevSnapshotManifests(ctx, msm, test.input, nil)
expected := make([]*snapshot.Manifest, 0, len(test.expectedIdxs))
for _, i := range test.expectedIdxs {
@ -462,6 +462,99 @@ func (suite *SnapshotFetchUnitSuite) TestFetchPrevSnapshots() {
}
}
func (suite *SnapshotFetchUnitSuite) TestFetchPrevSnapshots_customTags() {
data := []manifestInfo{
newManifestInfo(
testID1,
testT1,
false,
testMail,
testUser1,
"fnords",
"smarf",
),
}
expectLoad1T1 := map[manifest.ID]int{
testID1: 1,
}
table := []struct {
name string
input *OwnersCats
tags map[string]string
// Use this to denote which manifests in data should be expected. Allows
// defining data in a table while not repeating things between data and
// expected.
expectedIdxs []int
// Expected number of times a manifest should try to be loaded from kopia.
// Used to check that caching is functioning properly.
expectedLoadCounts map[manifest.ID]int
}{
{
name: "no tags specified",
tags: nil,
expectedIdxs: []int{0},
expectedLoadCounts: expectLoad1T1,
},
{
name: "all custom tags",
tags: map[string]string{
"fnords": "",
"smarf": "",
},
expectedIdxs: []int{0},
expectedLoadCounts: expectLoad1T1,
},
{
name: "subset of custom tags",
tags: map[string]string{"fnords": ""},
expectedIdxs: []int{0},
expectedLoadCounts: expectLoad1T1,
},
{
name: "custom tag mismatch",
tags: map[string]string{"bojangles": ""},
expectedIdxs: nil,
expectedLoadCounts: nil,
},
}
for _, test := range table {
suite.T().Run(test.name, func(t *testing.T) {
ctx, flush := tester.NewContext()
defer flush()
msm := &mockSnapshotManager{
data: data,
}
loadCounts := map[manifest.ID]int{}
msm.loadCallback = func(ids []manifest.ID) {
for _, id := range ids {
loadCounts[id]++
}
}
snaps := fetchPrevSnapshotManifests(ctx, msm, testAllUsersAllCats, test.tags)
expected := make([]*snapshot.Manifest, 0, len(test.expectedIdxs))
for _, i := range test.expectedIdxs {
expected = append(expected, data[i].man)
}
assert.ElementsMatch(t, expected, snaps)
// Need to manually check because we don't know the order the
// user/service/category labels will be iterated over. For some tests this
// could cause more loads than the ideal case.
assert.Len(t, loadCounts, len(test.expectedLoadCounts))
for id, count := range loadCounts {
assert.GreaterOrEqual(t, test.expectedLoadCounts[id], count)
}
})
}
}
// mockErrorSnapshotManager returns an error the first time LoadSnapshot and
// FindSnapshot are called. After that it passes the calls through to the
// contained snapshotManager.
@ -495,7 +588,7 @@ func (msm *mockErrorSnapshotManager) LoadSnapshots(
return msm.sm.LoadSnapshots(ctx, ids)
}
func (suite *SnapshotFetchUnitSuite) TestFetchPrevSnapshotsWorksWithErrors() {
func (suite *SnapshotFetchUnitSuite) TestFetchPrevSnapshots_withErrors() {
ctx, flush := tester.NewContext()
defer flush()
@ -532,7 +625,7 @@ func (suite *SnapshotFetchUnitSuite) TestFetchPrevSnapshotsWorksWithErrors() {
},
}
snaps := fetchPrevSnapshotManifests(ctx, msm, input)
snaps := fetchPrevSnapshotManifests(ctx, msm, input, nil)
// Only 1 snapshot should be chosen because the other two attempts fail.
// However, which one is returned is non-deterministic because maps are used.

View File

@ -409,7 +409,7 @@ func inflateCollectionTree(
updatedPaths := make(map[string]path.Path)
ownerCats := &OwnersCats{
ResourceOwners: make(map[string]struct{}),
ServiceCats: make(map[string]struct{}),
ServiceCats: make(map[string]ServiceCat),
}
for _, s := range collections {
@ -435,7 +435,7 @@ func inflateCollectionTree(
}
serviceCat := serviceCatTag(s.FullPath())
ownerCats.ServiceCats[serviceCat] = struct{}{}
ownerCats.ServiceCats[serviceCat] = ServiceCat{}
ownerCats.ResourceOwners[s.FullPath().ResourceOwner()] = struct{}{}
node.collection = s

View File

@ -439,7 +439,7 @@ func (suite *HierarchyBuilderUnitSuite) TestBuildDirectoryTree() {
user1Encoded: 5,
user2Encoded: 42,
}
expectedServiceCats := map[string]struct{}{
expectedServiceCats := map[string]ServiceCat{
serviceCatTag(suite.testPath): {},
serviceCatTag(p2): {},
}
@ -518,7 +518,7 @@ func (suite *HierarchyBuilderUnitSuite) TestBuildDirectoryTree_MixedDirectory()
p2, err := suite.testPath.Append(subdir, false)
require.NoError(suite.T(), err)
expectedServiceCats := map[string]struct{}{
expectedServiceCats := map[string]ServiceCat{
serviceCatTag(suite.testPath): {},
serviceCatTag(p2): {},
}

View File

@ -29,6 +29,12 @@ const (
serializationVersion uint32 = 1
)
// common manifest tags
const (
TagBackupID = "backup-id"
TagBackupCategory = "is-canon-backup"
)
var (
errNotConnected = errors.New("not connected to repo")
errNoRestorePath = errors.New("no restore path given")
@ -110,6 +116,7 @@ func (w Wrapper) BackupCollections(
previousSnapshots []*snapshot.Manifest,
collections []data.Collection,
service path.ServiceType,
tags map[string]string,
) (*BackupStats, *details.Details, error) {
if w.c == nil {
return nil, nil, errNotConnected
@ -132,7 +139,7 @@ func (w Wrapper) BackupCollections(
return nil, nil, errors.Wrap(err, "building kopia directories")
}
s, err := w.makeSnapshotWithRoot(ctx, dirTree, oc, progress)
s, err := w.makeSnapshotWithRoot(ctx, dirTree, oc, progress, tags)
if err != nil {
return nil, nil, err
}
@ -145,12 +152,13 @@ func (w Wrapper) makeSnapshotWithRoot(
root fs.Directory,
oc *OwnersCats,
progress *corsoProgress,
addlTags map[string]string,
) (*BackupStats, error) {
var man *snapshot.Manifest
prevSnaps := fetchPrevSnapshotManifests(ctx, w.c, oc)
bc := &stats.ByteCounter{}
var (
man *snapshot.Manifest
prevSnaps = fetchPrevSnapshotManifests(ctx, w.c, oc, nil)
bc = &stats.ByteCounter{}
)
err := repo.WriteSession(
ctx,
@ -197,6 +205,15 @@ func (w Wrapper) makeSnapshotWithRoot(
}
man.Tags = tagsFromStrings(oc)
for k, v := range addlTags {
mk, mv := MakeTagKV(k)
if len(v) == 0 {
v = mv
}
man.Tags[mk] = v
}
if _, err := snapshot.SaveSnapshot(innerCtx, rw, man); err != nil {
err = errors.Wrap(err, "saving snapshot")
@ -389,13 +406,17 @@ func (w Wrapper) DeleteSnapshot(
// incomplete. An incomplete manifest may be returned if it is newer than the
// newest complete manifest for the tuple. Manifests are deduped such that if
// multiple tuples match the same manifest it will only be returned once.
// If tags are provided, manifests must include a superset of the k:v pairs
// specified by those tags. Tags should pass their raw values, and will be
// normalized inside the func using MakeTagKV.
func (w Wrapper) FetchPrevSnapshotManifests(
ctx context.Context,
oc OwnersCats,
tags map[string]string,
) ([]*snapshot.Manifest, error) {
if w.c == nil {
return nil, errors.WithStack(errNotConnected)
}
return fetchPrevSnapshotManifests(ctx, w.c, &oc), nil
return fetchPrevSnapshotManifests(ctx, w.c, &oc, tags), nil
}

View File

@ -210,12 +210,21 @@ func (suite *KopiaIntegrationSuite) TestBackupCollections() {
),
}
// tags that are expected to populate as a side effect
// of the backup process.
baseTagKeys := []string{
serviceCatTag(suite.testPath1),
suite.testPath1.ResourceOwner(),
serviceCatTag(suite.testPath2),
suite.testPath2.ResourceOwner(),
}
// tags that are supplied by the caller.
customTags := map[string]string{
"fnords": "smarf",
"brunhilda": "",
}
expectedTags := map[string]string{}
for _, k := range baseTagKeys {
@ -223,6 +232,10 @@ func (suite *KopiaIntegrationSuite) TestBackupCollections() {
expectedTags[tk] = tv
}
for k, v := range normalizeTagKVs(customTags) {
expectedTags[k] = v
}
table := []struct {
name string
expectedUploadedFiles int
@ -247,6 +260,7 @@ func (suite *KopiaIntegrationSuite) TestBackupCollections() {
nil,
collections,
path.ExchangeService,
customTags,
)
assert.NoError(t, err)
@ -302,6 +316,7 @@ func (suite *KopiaIntegrationSuite) TestRestoreAfterCompressionChange() {
nil,
[]data.Collection{dc1, dc2},
path.ExchangeService,
nil,
)
require.NoError(t, err)
@ -372,6 +387,7 @@ func (suite *KopiaIntegrationSuite) TestBackupCollections_ReaderError() {
nil,
collections,
path.ExchangeService,
nil,
)
require.NoError(t, err)
@ -415,6 +431,7 @@ func (suite *KopiaIntegrationSuite) TestBackupCollectionsHandlesNoCollections()
nil,
test.collections,
path.UnknownService,
nil,
)
require.NoError(t, err)
@ -564,6 +581,7 @@ func (suite *KopiaSimpleRepoIntegrationSuite) SetupTest() {
nil,
collections,
path.ExchangeService,
nil,
)
require.NoError(t, err)
require.Equal(t, stats.ErrorCount, 0)

View File

@ -6,9 +6,11 @@ import (
"github.com/google/uuid"
multierror "github.com/hashicorp/go-multierror"
"github.com/kopia/kopia/snapshot"
"github.com/pkg/errors"
"github.com/alcionai/corso/src/internal/connector"
"github.com/alcionai/corso/src/internal/connector/graph"
"github.com/alcionai/corso/src/internal/connector/support"
"github.com/alcionai/corso/src/internal/data"
D "github.com/alcionai/corso/src/internal/diagnostics"
@ -23,6 +25,7 @@ import (
"github.com/alcionai/corso/src/pkg/backup/details"
"github.com/alcionai/corso/src/pkg/control"
"github.com/alcionai/corso/src/pkg/logger"
"github.com/alcionai/corso/src/pkg/path"
"github.com/alcionai/corso/src/pkg/selectors"
"github.com/alcionai/corso/src/pkg/store"
)
@ -124,19 +127,25 @@ func (op *BackupOperation) Run(ctx context.Context) (err error) {
}
}()
_, mdColls, err := produceManifestsAndMetadata(ctx, op.kopia, op.store, op.Selectors, op.account)
if err != nil {
opStats.readErr = errors.Wrap(err, "connecting to M365")
return opStats.readErr
}
gc, err := connectToM365(ctx, op.Selectors, op.account)
if err != nil {
opStats.readErr = errors.Wrap(err, "connecting to M365")
return opStats.readErr
}
cs, err := produceBackupDataCollections(ctx, gc, op.Selectors)
cs, err := produceBackupDataCollections(ctx, gc, op.Selectors, mdColls)
if err != nil {
opStats.readErr = errors.Wrap(err, "retrieving data to backup")
return opStats.readErr
}
opStats.k, backupDetails, err = consumeBackupDataCollections(ctx, op.kopia, op.Selectors, cs)
opStats.k, backupDetails, err = consumeBackupDataCollections(ctx, op.kopia, op.Selectors, cs, op.Results.BackupID)
if err != nil {
opStats.writeErr = errors.Wrap(err, "backing up service data")
return opStats.writeErr
@ -155,11 +164,152 @@ func (op *BackupOperation) Run(ctx context.Context) (err error) {
return err
}
// calls kopia to retrieve prior backup manifests, metadata collections to supply backup heuristics.
func produceManifestsAndMetadata(
ctx context.Context,
kw *kopia.Wrapper,
sw *store.Wrapper,
sel selectors.Selector,
acct account.Account,
) ([]*snapshot.Manifest, []data.Collection, error) {
complete, closer := observe.MessageWithCompletion("Fetching backup heuristics:")
defer func() {
complete <- struct{}{}
close(complete)
closer()
}()
m365, err := acct.M365Config()
if err != nil {
return nil, nil, err
}
var (
tid = m365.AzureTenantID
oc = selectorToOwnersCats(sel)
collections []data.Collection
)
ms, err := kw.FetchPrevSnapshotManifests(
ctx,
oc,
map[string]string{kopia.TagBackupCategory: ""})
if err != nil {
return nil, nil, err
}
for _, man := range ms {
if len(man.IncompleteReason) > 0 {
continue
}
bup := backup.Backup{}
if err := sw.Get(
ctx,
model.BackupSchema,
model.StableID(man.Tags[kopia.TagBackupID]),
&bup,
); err != nil {
return nil, nil, err
}
colls, err := collectMetadata(ctx, kw, graph.MetadataFileNames(), oc, tid, bup.SnapshotID)
if err != nil {
// prior metadata isn't guaranteed to exist.
// if it doesn't, we'll just have to do a
// full backup for that data.
if errors.Is(err, errNotRestored) {
continue
}
return nil, nil, err
}
collections = append(collections, colls...)
}
return ms, collections, err
}
var errNotRestored = errors.New("unable to restore metadata")
func collectMetadata(
ctx context.Context,
kw *kopia.Wrapper,
fileNames []string,
oc kopia.OwnersCats,
tenantID, snapshotID string,
) ([]data.Collection, error) {
paths := []path.Path{}
for _, fn := range fileNames {
for ro := range oc.ResourceOwners {
for _, sc := range oc.ServiceCats {
p, err := path.Builder{}.
Append(fn).
ToServiceCategoryMetadataPath(
tenantID,
ro,
sc.Service,
sc.Category,
true)
if err != nil {
return nil, errors.Wrapf(err, "building metadata path")
}
paths = append(paths, p)
}
}
}
dcs, err := kw.RestoreMultipleItems(ctx, snapshotID, paths, nil)
if err != nil {
return nil, errors.Wrap(err, "collecting prior metadata")
}
return dcs, nil
}
func selectorToOwnersCats(sel selectors.Selector) kopia.OwnersCats {
service := sel.PathService()
oc := kopia.OwnersCats{
ResourceOwners: map[string]struct{}{},
ServiceCats: map[string]kopia.ServiceCat{},
}
ros, err := sel.ResourceOwners()
if err != nil {
return kopia.OwnersCats{}
}
for _, sl := range [][]string{ros.Includes, ros.Filters} {
for _, ro := range sl {
oc.ResourceOwners[ro] = struct{}{}
}
}
pcs, err := sel.PathCategories()
if err != nil {
return kopia.OwnersCats{}
}
for _, sl := range [][]path.CategoryType{pcs.Includes, pcs.Filters} {
for _, cat := range sl {
k, v := kopia.MakeServiceCat(service, cat)
oc.ServiceCats[k] = v
}
}
return oc
}
// calls the producer to generate collections of data to backup
func produceBackupDataCollections(
ctx context.Context,
gc *connector.GraphConnector,
sel selectors.Selector,
metadata []data.Collection,
) ([]data.Collection, error) {
complete, closer := observe.MessageWithCompletion("Discovering items to backup:")
defer func() {
@ -168,12 +318,7 @@ func produceBackupDataCollections(
closer()
}()
cs, err := gc.DataCollections(ctx, sel, nil)
if err != nil {
return nil, err
}
return cs, nil
return gc.DataCollections(ctx, sel, metadata)
}
// calls kopia to backup the collections of data
@ -182,6 +327,7 @@ func consumeBackupDataCollections(
kw *kopia.Wrapper,
sel selectors.Selector,
cs []data.Collection,
backupID model.StableID,
) (*kopia.BackupStats, *details.Details, error) {
complete, closer := observe.MessageWithCompletion("Backing up data:")
defer func() {
@ -190,12 +336,12 @@ func consumeBackupDataCollections(
closer()
}()
kstats, deets, err := kw.BackupCollections(ctx, nil, cs, sel.PathService())
if err != nil {
return nil, nil, err
tags := map[string]string{
kopia.TagBackupID: string(backupID),
kopia.TagBackupCategory: "",
}
return kstats, deets, nil
return kw.BackupCollections(ctx, nil, cs, sel.PathService(), tags)
}
// writes the results metrics to the operation results.

View File

@ -262,7 +262,7 @@ func TestBackupOpIntegrationSuite(t *testing.T) {
if err := tester.RunOnAny(
tester.CorsoCITests,
tester.CorsoOperationTests,
"flomp",
tester.CorsoOperationBackupTests,
); err != nil {
t.Skip(err)
}
@ -322,16 +322,21 @@ func (suite *BackupOpIntegrationSuite) TestBackup_Run_exchange() {
m365UserID := tester.M365UserID(suite.T())
tests := []struct {
name string
selectFunc func() *selectors.ExchangeBackup
name string
selectFunc func() *selectors.ExchangeBackup
resourceOwner string
category path.CategoryType
}{
{
name: "Integration Exchange.Mail",
selectFunc: func() *selectors.ExchangeBackup {
sel := selectors.NewExchangeBackup()
sel.Include(sel.MailFolders([]string{m365UserID}, []string{exchange.DefaultMailFolder}, selectors.PrefixMatch()))
return sel
},
resourceOwner: m365UserID,
category: path.EmailCategory,
},
{
name: "Integration Exchange.Contacts",
@ -341,8 +346,11 @@ func (suite *BackupOpIntegrationSuite) TestBackup_Run_exchange() {
[]string{m365UserID},
[]string{exchange.DefaultContactFolder},
selectors.PrefixMatch()))
return sel
},
resourceOwner: m365UserID,
category: path.ContactsCategory,
},
{
name: "Integration Exchange.Events",
@ -351,6 +359,8 @@ func (suite *BackupOpIntegrationSuite) TestBackup_Run_exchange() {
sel.Include(sel.EventCalendars([]string{m365UserID}, []string{exchange.DefaultCalendar}, selectors.PrefixMatch()))
return sel
},
resourceOwner: m365UserID,
category: path.EventsCategory,
},
}
for _, test := range tests {
@ -392,13 +402,37 @@ func (suite *BackupOpIntegrationSuite) TestBackup_Run_exchange() {
mb.CalledWith[events.BackupStart][0][events.BackupID],
bo.Results.BackupID, "backupID pre-declaration")
// Check that metadata files with delta tokens were created. Currently
// these files will only be made for contacts and email in Exchange if any
// items were backed up. Events does not support delta queries.
// verify that we can find the new backup id in the manifests
var (
sck, scv = kopia.MakeServiceCat(sel.PathService(), test.category)
oc = kopia.OwnersCats{
ResourceOwners: map[string]struct{}{test.resourceOwner: {}},
ServiceCats: map[string]kopia.ServiceCat{sck: scv},
}
tags = map[string]string{kopia.TagBackupCategory: ""}
found bool
)
mans, err := kw.FetchPrevSnapshotManifests(ctx, oc, tags)
assert.NoError(t, err)
for _, man := range mans {
tk, _ := kopia.MakeTagKV(kopia.TagBackupID)
if man.Tags[tk] == string(bo.Results.BackupID) {
found = true
break
}
}
assert.True(t, found, "backup retrieved by previous snapshot manifest")
if failed {
return
}
// Check that metadata files with delta tokens were created. Currently
// these files will only be made for contacts and email in Exchange if any
// items were backed up. Events does not support delta queries.
m365, err := acct.M365Config()
require.NoError(t, err)

View File

@ -73,7 +73,7 @@ func (ss *streamStore) WriteBackupDetails(
},
}
backupStats, _, err := ss.kw.BackupCollections(ctx, nil, []data.Collection{dc}, ss.service)
backupStats, _, err := ss.kw.BackupCollections(ctx, nil, []data.Collection{dc}, ss.service, nil)
if err != nil {
return "", nil
}

View File

@ -28,6 +28,7 @@ const (
CorsoModelStoreTests = "CORSO_MODEL_STORE_TESTS"
CorsoOneDriveTests = "CORSO_ONE_DRIVE_TESTS"
CorsoOperationTests = "CORSO_OPERATION_TESTS"
CorsoOperationBackupTests = "CORSO_OPERATION_BACKUP_TESTS"
CorsoRepositoryTests = "CORSO_REPOSITORY_TESTS"
)