add boilerplate for drive tree processing (#4716)

begins implementation of the drive delta tree support by adding boilerplate funcs for getting the backup data, and adding a framework of unit tests that will be used bring testing parity up to par with current tests.

---

#### Does this PR need a docs update or release note?

- [x]  No

#### Issue(s)

* #4689

#### Test Plan

- [x]  Unit test
This commit is contained in:
Keepers 2023-11-28 14:07:49 -07:00 committed by GitHub
parent 793658c790
commit 6aff258c8b
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
9 changed files with 2054 additions and 1059 deletions

View File

@ -25,6 +25,23 @@ const (
DeletedState CollectionState = 3 DeletedState CollectionState = 3
) )
func (cs CollectionState) String() string {
s := "Unknown State"
switch cs {
case 0:
s = "New"
case 1:
s = "Not Moved"
case 2:
s = "Moved"
case 3:
s = "Deleted"
}
return s
}
type FetchRestoreCollection struct { type FetchRestoreCollection struct {
Collection Collection
FetchItemByNamer FetchItemByNamer

View File

@ -291,6 +291,17 @@ func (c *Collections) Get(
ssmb *prefixmatcher.StringSetMatchBuilder, ssmb *prefixmatcher.StringSetMatchBuilder,
errs *fault.Bus, errs *fault.Bus,
) ([]data.BackupCollection, bool, error) { ) ([]data.BackupCollection, bool, error) {
if c.ctrl.ToggleFeatures.UseDeltaTree {
_, _, err := c.getTree(ctx, prevMetadata, ssmb, errs)
if err != nil {
return nil, false, clues.Wrap(err, "processing backup using tree")
}
return nil,
false,
clues.New("forced error: cannot run tree-based backup: incomplete implementation")
}
deltasByDriveID, prevPathsByDriveID, canUsePrevBackup, err := deserializeAndValidateMetadata( deltasByDriveID, prevPathsByDriveID, canUsePrevBackup, err := deserializeAndValidateMetadata(
ctx, ctx,
prevMetadata, prevMetadata,
@ -750,87 +761,6 @@ func (c *Collections) getCollectionPath(
return collectionPath, nil return collectionPath, nil
} }
type driveEnumerationStats struct {
numPages int
numAddedFiles int
numContainers int
numBytes int64
}
func newPagerLimiter(opts control.Options) *pagerLimiter {
res := &pagerLimiter{limits: opts.PreviewLimits}
if res.limits.MaxContainers == 0 {
res.limits.MaxContainers = defaultPreviewMaxContainers
}
if res.limits.MaxItemsPerContainer == 0 {
res.limits.MaxItemsPerContainer = defaultPreviewMaxItemsPerContainer
}
if res.limits.MaxItems == 0 {
res.limits.MaxItems = defaultPreviewMaxItems
}
if res.limits.MaxBytes == 0 {
res.limits.MaxBytes = defaultPreviewMaxBytes
}
if res.limits.MaxPages == 0 {
res.limits.MaxPages = defaultPreviewMaxPages
}
return res
}
type pagerLimiter struct {
limits control.PreviewItemLimits
}
func (l pagerLimiter) effectiveLimits() control.PreviewItemLimits {
return l.limits
}
func (l pagerLimiter) enabled() bool {
return l.limits.Enabled
}
// sizeLimit returns the total number of bytes this backup should try to
// contain.
func (l pagerLimiter) sizeLimit() int64 {
return l.limits.MaxBytes
}
// atItemLimit returns true if the limiter is enabled and has reached the limit
// for individual items added to collections for this backup.
func (l pagerLimiter) atItemLimit(stats *driveEnumerationStats) bool {
return l.enabled() &&
(stats.numAddedFiles >= l.limits.MaxItems ||
stats.numBytes >= l.limits.MaxBytes)
}
// atContainerItemsLimit returns true if the limiter is enabled and the current
// number of items is above the limit for the number of items for a container
// for this backup.
func (l pagerLimiter) atContainerItemsLimit(numItems int) bool {
return l.enabled() && numItems >= l.limits.MaxItemsPerContainer
}
// atContainerPageLimit returns true if the limiter is enabled and the number of
// pages processed so far is beyond the limit for this backup.
func (l pagerLimiter) atPageLimit(stats *driveEnumerationStats) bool {
return l.enabled() && stats.numPages >= l.limits.MaxPages
}
// atLimit returns true if the limiter is enabled and meets any of the
// conditions for max items, containers, etc for this backup.
func (l pagerLimiter) atLimit(stats *driveEnumerationStats) bool {
return l.enabled() &&
(l.atItemLimit(stats) ||
stats.numContainers >= l.limits.MaxContainers ||
stats.numPages >= l.limits.MaxPages)
}
// PopulateDriveCollections initializes and adds the provided drive items to Collections // PopulateDriveCollections initializes and adds the provided drive items to Collections
// A new collection is created for every drive folder. // A new collection is created for every drive folder.
// Along with populating the collection items and updating the excluded item IDs, this func // Along with populating the collection items and updating the excluded item IDs, this func
@ -926,7 +856,7 @@ func (c *Collections) PopulateDriveCollections(
// Don't check for containers we've already seen. // Don't check for containers we've already seen.
if _, ok := c.CollectionMap[driveID][id]; !ok { if _, ok := c.CollectionMap[driveID][id]; !ok {
if id != lastContainerID { if id != lastContainerID {
if limiter.atLimit(stats) { if limiter.atLimit(stats, ignoreMe) {
break break
} }

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,326 @@
package drive
import (
"context"
"github.com/alcionai/clues"
"github.com/microsoftgraph/msgraph-sdk-go/models"
"github.com/alcionai/corso/src/internal/common/prefixmatcher"
"github.com/alcionai/corso/src/internal/common/ptr"
"github.com/alcionai/corso/src/internal/data"
bupMD "github.com/alcionai/corso/src/pkg/backup/metadata"
"github.com/alcionai/corso/src/pkg/count"
"github.com/alcionai/corso/src/pkg/fault"
"github.com/alcionai/corso/src/pkg/logger"
"github.com/alcionai/corso/src/pkg/services/m365/api"
"github.com/alcionai/corso/src/pkg/services/m365/api/graph"
"github.com/alcionai/corso/src/pkg/services/m365/api/pagers"
)
// this file is used to separate the collections handling between the previous
// (list-based) design, and the in-progress (tree-based) redesign.
// see: https://github.com/alcionai/corso/issues/4688
func (c *Collections) getTree(
ctx context.Context,
prevMetadata []data.RestoreCollection,
ssmb *prefixmatcher.StringSetMatchBuilder,
errs *fault.Bus,
) ([]data.BackupCollection, bool, error) {
ctx = clues.AddTraceName(ctx, "GetTree")
// extract the previous backup's metadata like: deltaToken urls and previousPath maps.
// We'll need these to reconstruct / ensure the correct state of the world, after
// enumerating through all the delta changes.
deltasByDriveID, prevPathsByDriveID, canUsePrevBackup, err := deserializeAndValidateMetadata(
ctx,
prevMetadata,
c.counter,
errs)
if err != nil {
return nil, false, err
}
ctx = clues.Add(ctx, "can_use_previous_backup", canUsePrevBackup)
// in sharepoint, it's possible to delete an entire drive.
// if we don't see a previously-existing drive in the drives enumeration,
// we assume it was deleted and will remove it from storage using a tombstone.
driveTombstones := map[string]struct{}{}
for driveID := range prevPathsByDriveID {
driveTombstones[driveID] = struct{}{}
}
pager := c.handler.NewDrivePager(c.protectedResource.ID(), nil)
drives, err := api.GetAllDrives(ctx, pager)
if err != nil {
return nil, false, err
}
c.counter.Add(count.Drives, int64(len(drives)))
c.counter.Add(count.PrevDeltas, int64(len(deltasByDriveID)))
var (
el = errs.Local()
collections = []data.BackupCollection{}
driveIDToNewDeltaLink = map[string]string{}
driveIDToNewPrevPaths = map[string]map[string]string{}
)
// each drive owns its own delta history. We can't go more granular than that.
// so our first order of business is to enumerate each drive's delta data, and
// to use that as the basis for our backups.
for _, drv := range drives {
if el.Failure() != nil {
break
}
var (
driveID = ptr.Val(drv.GetId())
cl = c.counter.Local()
ictx = clues.Add(
ctx,
"drive_id", driveID,
"drive_name", clues.Hide(ptr.Val(drv.GetName())))
)
ictx = clues.AddLabelCounter(ictx, cl.PlainAdder())
// all the magic happens here. expecations are that this process will:
// - iterate over all data (new or delta, as needed) in the drive
// - condense that data into a set of collections to backup
// - stitch the new and previous path data into a new prevPaths map
// - report the latest delta token details
colls, newPrevPaths, du, err := c.makeDriveCollections(
ictx,
drv,
prevPathsByDriveID[driveID],
cl,
el.Local())
if err != nil {
el.AddRecoverable(ictx, clues.Stack(err))
continue
}
// add all the freshly aggregated data into our results
collections = append(collections, colls...)
driveIDToNewPrevPaths[driveID] = newPrevPaths
driveIDToNewDeltaLink[driveID] = du.URL
// this drive is still in use, so we'd better not delete it.
delete(driveTombstones, driveID)
}
if el.Failure() != nil {
return nil, false, clues.Stack(el.Failure())
}
alertIfPrevPathsHaveCollisions(ctx, driveIDToNewPrevPaths, c.counter, errs)
// clean up any drives that have been deleted since the last backup.
dts, err := c.makeDriveTombstones(ctx, driveTombstones, errs)
if err != nil {
return nil, false, clues.Stack(err)
}
collections = append(collections, dts...)
// persist our updated metadata for use on the next backup
colls := c.makeMetadataCollections(
ctx,
driveIDToNewDeltaLink,
driveIDToNewPrevPaths)
collections = append(collections, colls...)
logger.Ctx(ctx).Infow("produced collections", "count_collections", len(collections))
return collections, canUsePrevBackup, nil
}
func (c *Collections) makeDriveCollections(
ctx context.Context,
d models.Driveable,
prevPaths map[string]string,
counter *count.Bus,
errs *fault.Bus,
) ([]data.BackupCollection, map[string]string, pagers.DeltaUpdate, error) {
cl := c.counter.Local()
cl.Add(count.PrevPaths, int64(len(prevPaths)))
logger.Ctx(ctx).Infow(
"previous metadata for drive",
"count_old_prev_paths", len(prevPaths))
// TODO(keepers): leaving this code around for now as a guide
// while implementation progresses.
// --- pager aggregation
// du, newPrevPaths, err := c.PopulateDriveCollections(
// ctx,
// d,
// tree,
// cl.Local(),
// errs)
// if err != nil {
// return nil, false, clues.Stack(err)
// }
// numDriveItems := c.NumItems - numPrevItems
// numPrevItems = c.NumItems
// cl.Add(count.NewPrevPaths, int64(len(newPrevPaths)))
// --- prev path incorporation
// For both cases we don't need to do set difference on folder map if the
// delta token was valid because we should see all the changes.
// if !du.Reset {
// if len(excludedItemIDs) == 0 {
// continue
// }
// p, err := c.handler.CanonicalPath(odConsts.DriveFolderPrefixBuilder(driveID), c.tenantID)
// if err != nil {
// return nil, false, clues.WrapWC(ictx, err, "making exclude prefix")
// }
// ssmb.Add(p.String(), excludedItemIDs)
// continue
// }
// Set all folders in previous backup but not in the current one with state
// deleted. Need to compare by ID because it's possible to make new folders
// with the same path as deleted old folders. We shouldn't merge items or
// subtrees if that happens though.
// --- post-processing
// Attach an url cache to the drive if the number of discovered items is
// below the threshold. Attaching cache to larger drives can cause
// performance issues since cache delta queries start taking up majority of
// the hour the refreshed URLs are valid for.
// if numDriveItems < urlCacheDriveItemThreshold {
// logger.Ctx(ictx).Infow(
// "adding url cache for drive",
// "num_drive_items", numDriveItems)
// uc, err := newURLCache(
// driveID,
// prevDeltaLink,
// urlCacheRefreshInterval,
// c.handler,
// cl,
// errs)
// if err != nil {
// return nil, false, clues.Stack(err)
// }
// // Set the URL cache instance for all collections in this drive.
// for id := range c.CollectionMap[driveID] {
// c.CollectionMap[driveID][id].urlCache = uc
// }
// }
return nil, nil, pagers.DeltaUpdate{}, clues.New("not yet implemented")
}
// quality-of-life wrapper that transforms each tombstone in the map
// into a backup collection that marks the backup as deleted.
func (c *Collections) makeDriveTombstones(
ctx context.Context,
driveTombstones map[string]struct{},
errs *fault.Bus,
) ([]data.BackupCollection, error) {
c.counter.Add(count.DriveTombstones, int64(len(driveTombstones)))
var (
colls = make([]data.BackupCollection, 0, len(driveTombstones))
el = errs.Local()
)
// generate tombstones for drives that were removed.
for driveID := range driveTombstones {
if el.Failure() != nil {
break
}
prevDrivePath, err := c.handler.PathPrefix(c.tenantID, driveID)
if err != nil {
err = clues.WrapWC(ctx, err, "making drive tombstone for previous path").Label(count.BadPathPrefix)
el.AddRecoverable(ctx, err)
continue
}
// TODO: call NewTombstoneCollection
coll, err := NewCollection(
c.handler,
c.protectedResource,
nil, // delete the drive
prevDrivePath,
driveID,
c.statusUpdater,
c.ctrl,
false,
true,
nil,
c.counter.Local())
if err != nil {
err = clues.WrapWC(ctx, err, "making drive tombstone")
el.AddRecoverable(ctx, err)
continue
}
colls = append(colls, coll)
}
return colls, el.Failure()
}
// quality-of-life wrapper that transforms the delta tokens and previous paths
// into a backup collections for persitence.
func (c *Collections) makeMetadataCollections(
ctx context.Context,
deltaTokens map[string]string,
prevPaths map[string]map[string]string,
) []data.BackupCollection {
colls := []data.BackupCollection{}
pathPrefix, err := c.handler.MetadataPathPrefix(c.tenantID)
if err != nil {
logger.CtxErr(ctx, err).Info("making metadata collection path prefixes")
// It's safe to return here because the logic for starting an
// incremental backup should eventually find that the metadata files are
// empty/missing and default to a full backup.
return colls
}
entries := []graph.MetadataCollectionEntry{
graph.NewMetadataEntry(bupMD.DeltaURLsFileName, deltaTokens),
graph.NewMetadataEntry(bupMD.PreviousPathFileName, prevPaths),
}
md, err := graph.MakeMetadataCollection(
pathPrefix,
entries,
c.statusUpdater,
c.counter.Local())
if err != nil {
logger.CtxErr(ctx, err).Info("making metadata collection for future incremental backups")
// Technically it's safe to continue here because the logic for starting an
// incremental backup should eventually find that the metadata files are
// empty/missing and default to a full backup.
return colls
}
return append(colls, md)
}

View File

@ -0,0 +1,510 @@
package drive
import (
"testing"
"github.com/alcionai/clues"
"github.com/microsoftgraph/msgraph-sdk-go/models"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/stretchr/testify/suite"
"github.com/alcionai/corso/src/internal/common/idname"
"github.com/alcionai/corso/src/internal/common/prefixmatcher"
"github.com/alcionai/corso/src/internal/common/ptr"
"github.com/alcionai/corso/src/internal/data"
dataMock "github.com/alcionai/corso/src/internal/data/mock"
"github.com/alcionai/corso/src/internal/m365/service/onedrive/mock"
"github.com/alcionai/corso/src/internal/m365/support"
"github.com/alcionai/corso/src/internal/tester"
bupMD "github.com/alcionai/corso/src/pkg/backup/metadata"
"github.com/alcionai/corso/src/pkg/control"
"github.com/alcionai/corso/src/pkg/count"
countTD "github.com/alcionai/corso/src/pkg/count/testdata"
"github.com/alcionai/corso/src/pkg/fault"
"github.com/alcionai/corso/src/pkg/path"
"github.com/alcionai/corso/src/pkg/services/m365/api/graph"
apiMock "github.com/alcionai/corso/src/pkg/services/m365/api/mock"
)
// ---------------------------------------------------------------------------
// helpers
// ---------------------------------------------------------------------------
func collWithMBH(mbh BackupHandler) *Collections {
return NewCollections(
mbh,
tenant,
idname.NewProvider(user, user),
func(*support.ControllerOperationStatus) {},
control.Options{ToggleFeatures: control.Toggles{
UseDeltaTree: true,
}},
count.New())
}
func fullOrPrevPath(
t *testing.T,
coll data.BackupCollection,
) path.Path {
var collPath path.Path
if coll.State() != data.DeletedState {
collPath = coll.FullPath()
} else {
collPath = coll.PreviousPath()
}
require.False(
t,
len(collPath.Elements()) < 4,
"malformed or missing collection path")
return collPath
}
func pagerForDrives(drives ...models.Driveable) *apiMock.Pager[models.Driveable] {
return &apiMock.Pager[models.Driveable]{
ToReturn: []apiMock.PagerResult[models.Driveable]{
{Values: drives},
},
}
}
func makePrevMetadataColls(
t *testing.T,
mbh BackupHandler,
previousPaths map[string]map[string]string,
) []data.RestoreCollection {
pathPrefix, err := mbh.MetadataPathPrefix(tenant)
require.NoError(t, err, clues.ToCore(err))
prevDeltas := map[string]string{}
for driveID := range previousPaths {
prevDeltas[driveID] = idx(delta, "prev")
}
mdColl, err := graph.MakeMetadataCollection(
pathPrefix,
[]graph.MetadataCollectionEntry{
graph.NewMetadataEntry(bupMD.DeltaURLsFileName, prevDeltas),
graph.NewMetadataEntry(bupMD.PreviousPathFileName, previousPaths),
},
func(*support.ControllerOperationStatus) {},
count.New())
require.NoError(t, err, "creating metadata collection", clues.ToCore(err))
return []data.RestoreCollection{
dataMock.NewUnversionedRestoreCollection(t, data.NoFetchRestoreCollection{Collection: mdColl}),
}
}
func compareMetadata(
t *testing.T,
mdColl data.Collection,
expectDeltas map[string]string,
expectPrevPaths map[string]map[string]string,
) {
ctx, flush := tester.NewContext(t)
defer flush()
colls := []data.RestoreCollection{
dataMock.NewUnversionedRestoreCollection(t, data.NoFetchRestoreCollection{Collection: mdColl}),
}
deltas, prevs, _, err := deserializeAndValidateMetadata(
ctx,
colls,
count.New(),
fault.New(true))
require.NoError(t, err, "deserializing metadata", clues.ToCore(err))
assert.Equal(t, expectDeltas, deltas, "delta urls")
assert.Equal(t, expectPrevPaths, prevs, "previous paths")
}
// for comparisons done by collection state
type stateAssertion struct {
itemIDs []string
// should never get set by the user.
// this flag gets flipped when calling assertions.compare.
// any unseen collection will error on requireNoUnseenCollections
sawCollection bool
}
// for comparisons done by a given collection path
type collectionAssertion struct {
doNotMerge assert.BoolAssertionFunc
states map[data.CollectionState]*stateAssertion
excludedItems map[string]struct{}
}
type statesToItemIDs map[data.CollectionState][]string
// TODO(keepers): move excludeItems to a more global position.
func newCollAssertion(
doNotMerge bool,
itemsByState statesToItemIDs,
excludeItems ...string,
) collectionAssertion {
states := map[data.CollectionState]*stateAssertion{}
for state, itemIDs := range itemsByState {
states[state] = &stateAssertion{
itemIDs: itemIDs,
}
}
dnm := assert.False
if doNotMerge {
dnm = assert.True
}
return collectionAssertion{
doNotMerge: dnm,
states: states,
excludedItems: makeExcludeMap(excludeItems...),
}
}
// to aggregate all collection-related expectations in the backup
// map collection path -> collection state -> assertion
type collectionAssertions map[string]collectionAssertion
// ensure the provided collection matches expectations as set by the test.
func (cas collectionAssertions) compare(
t *testing.T,
coll data.BackupCollection,
excludes *prefixmatcher.StringSetMatchBuilder,
) {
ctx, flush := tester.NewContext(t)
defer flush()
var (
itemCh = coll.Items(ctx, fault.New(true))
itemIDs = []string{}
)
p := fullOrPrevPath(t, coll)
for itm := range itemCh {
itemIDs = append(itemIDs, itm.ID())
}
expect := cas[p.String()]
expectState := expect.states[coll.State()]
expectState.sawCollection = true
assert.ElementsMatchf(
t,
expectState.itemIDs,
itemIDs,
"expected all items to match in collection with:\nstate %q\npath %q",
coll.State(),
p)
expect.doNotMerge(
t,
coll.DoNotMergeItems(),
"expected collection to have the appropariate doNotMerge flag")
if result, ok := excludes.Get(p.String()); ok {
assert.Equal(
t,
expect.excludedItems,
result,
"excluded items")
}
}
// ensure that no collections in the expected set are still flagged
// as sawCollection == false.
func (cas collectionAssertions) requireNoUnseenCollections(
t *testing.T,
) {
for p, withPath := range cas {
for _, state := range withPath.states {
require.True(
t,
state.sawCollection,
"results should have contained collection:\n\t%q\t\n%q",
state, p)
}
}
}
// ---------------------------------------------------------------------------
// tests
// ---------------------------------------------------------------------------
type CollectionsTreeUnitSuite struct {
tester.Suite
}
func TestCollectionsTreeUnitSuite(t *testing.T) {
suite.Run(t, &CollectionsTreeUnitSuite{Suite: tester.NewUnitSuite(t)})
}
func (suite *CollectionsTreeUnitSuite) TestCollections_MakeDriveTombstones() {
badPfxMBH := mock.DefaultOneDriveBH(user)
badPfxMBH.PathPrefixErr = assert.AnError
twostones := map[string]struct{}{
"t1": {},
"t2": {},
}
table := []struct {
name string
tombstones map[string]struct{}
c *Collections
expectErr assert.ErrorAssertionFunc
expect assert.ValueAssertionFunc
}{
{
name: "nil",
tombstones: nil,
c: collWithMBH(mock.DefaultOneDriveBH(user)),
expectErr: assert.NoError,
expect: assert.Empty,
},
{
name: "none",
tombstones: map[string]struct{}{},
c: collWithMBH(mock.DefaultOneDriveBH(user)),
expectErr: assert.NoError,
expect: assert.Empty,
},
{
name: "some tombstones",
tombstones: twostones,
c: collWithMBH(mock.DefaultOneDriveBH(user)),
expectErr: assert.NoError,
expect: assert.NotEmpty,
},
{
name: "bad prefix path",
tombstones: twostones,
c: collWithMBH(badPfxMBH),
expectErr: assert.Error,
expect: assert.Empty,
},
}
for _, test := range table {
suite.Run(test.name, func() {
t := suite.T()
ctx, flush := tester.NewContext(t)
defer flush()
colls, err := test.c.makeDriveTombstones(ctx, test.tombstones, fault.New(true))
test.expectErr(t, err, clues.ToCore(err))
test.expect(t, colls)
for _, coll := range colls {
assert.Equal(t, data.DeletedState, coll.State(), "tombstones should always delete data")
}
})
}
}
func (suite *CollectionsTreeUnitSuite) TestCollections_MakeMetadataCollections() {
badMetaPfxMBH := mock.DefaultOneDriveBH(user)
badMetaPfxMBH.MetadataPathPrefixErr = assert.AnError
table := []struct {
name string
c *Collections
expect assert.ValueAssertionFunc
}{
{
name: "no errors",
c: collWithMBH(mock.DefaultOneDriveBH(user)),
expect: assert.NotEmpty,
},
{
name: "bad prefix path",
c: collWithMBH(badMetaPfxMBH),
expect: assert.Empty,
},
}
for _, test := range table {
suite.Run(test.name, func() {
var (
t = suite.T()
deltaTokens = map[string]string{}
prevPaths = map[string]map[string]string{}
)
ctx, flush := tester.NewContext(t)
defer flush()
colls := test.c.makeMetadataCollections(ctx, deltaTokens, prevPaths)
test.expect(t, colls)
for _, coll := range colls {
assert.NotEqual(t, data.DeletedState, coll.State(), "metadata is never deleted")
}
})
}
}
// TODO(keepers): implement tree version of populateDriveCollections tests
// TODO(keepers): implement tree version of TestGet single-drive tests
func (suite *CollectionsTreeUnitSuite) TestCollections_MakeDriveCollections() {
drive1 := models.NewDrive()
drive1.SetId(ptr.To(idx(drive, 1)))
drive1.SetName(ptr.To(namex(drive, 1)))
table := []struct {
name string
c *Collections
drive models.Driveable
prevPaths map[string]string
expectErr require.ErrorAssertionFunc
expectCounts countTD.Expected
}{
{
name: "not yet implemented",
c: collWithMBH(mock.DefaultOneDriveBH(user)),
drive: drive1,
expectErr: require.Error,
expectCounts: countTD.Expected{
count.PrevPaths: 0,
},
},
}
for _, test := range table {
suite.Run(test.name, func() {
t := suite.T()
ctx, flush := tester.NewContext(t)
defer flush()
colls, paths, delta, err := test.c.makeDriveCollections(
ctx,
test.drive,
test.prevPaths,
test.c.counter,
fault.New(true))
// TODO(keepers): awaiting implementation
test.expectErr(t, err, clues.ToCore(err))
assert.Empty(t, colls)
assert.Empty(t, paths)
assert.Empty(t, delta.URL)
test.expectCounts.Compare(t, test.c.counter)
})
}
}
// TODO(keepers): implement tree version of TestGet multi-drive tests
func (suite *CollectionsTreeUnitSuite) TestCollections_GetTree() {
metadataPath, err := path.BuildMetadata(
tenant,
user,
path.OneDriveService,
path.FilesCategory,
false)
require.NoError(suite.T(), err, "making metadata path", clues.ToCore(err))
drive1 := models.NewDrive()
drive1.SetId(ptr.To(idx(drive, 1)))
drive1.SetName(ptr.To(namex(drive, 1)))
type expected struct {
canUsePrevBackup assert.BoolAssertionFunc
collAssertions collectionAssertions
counts countTD.Expected
deltas map[string]string
err require.ErrorAssertionFunc
prevPaths map[string]map[string]string
skips int
}
table := []struct {
name string
drivePager *apiMock.Pager[models.Driveable]
enumerator mock.EnumerateItemsDeltaByDrive
previousPaths map[string]map[string]string
metadata []data.RestoreCollection
expect expected
}{
{
name: "not yet implemented",
drivePager: pagerForDrives(drive1),
expect: expected{
canUsePrevBackup: assert.False,
collAssertions: collectionAssertions{
fullPath(1): newCollAssertion(
doNotMergeItems,
statesToItemIDs{data.NotMovedState: {}},
id(file)),
},
counts: countTD.Expected{
count.PrevPaths: 0,
},
deltas: map[string]string{},
err: require.Error,
prevPaths: map[string]map[string]string{},
skips: 0,
},
},
}
for _, test := range table {
suite.Run(test.name, func() {
t := suite.T()
ctx, flush := tester.NewContext(t)
defer flush()
var (
mbh = mock.DefaultDriveBHWith(user, test.drivePager, test.enumerator)
c = collWithMBH(mbh)
prevMetadata = makePrevMetadataColls(t, mbh, test.previousPaths)
globalExcludes = prefixmatcher.NewStringSetBuilder()
errs = fault.New(true)
)
colls, canUsePrevBackup, err := c.getTree(
ctx,
prevMetadata,
globalExcludes,
errs)
test.expect.err(t, err, clues.ToCore(err))
// TODO(keepers): awaiting implementation
assert.Empty(t, colls)
assert.Equal(t, test.expect.skips, len(errs.Skipped()))
test.expect.canUsePrevBackup(t, canUsePrevBackup)
test.expect.counts.Compare(t, c.counter)
if err != nil {
return
}
for _, coll := range colls {
collPath := fullOrPrevPath(t, coll)
if collPath.String() == metadataPath.String() {
compareMetadata(
t,
coll,
test.expect.deltas,
test.expect.prevPaths)
continue
}
test.expect.collAssertions.compare(t, coll, globalExcludes)
}
test.expect.collAssertions.requireNoUnseenCollections(t)
})
}
}

View File

@ -0,0 +1,95 @@
package drive
import "github.com/alcionai/corso/src/pkg/control"
// used to mark an unused variable while we transition handling.
const ignoreMe = -1
type driveEnumerationStats struct {
numPages int
numAddedFiles int
numContainers int
numBytes int64
}
func newPagerLimiter(opts control.Options) *pagerLimiter {
res := &pagerLimiter{limits: opts.PreviewLimits}
if res.limits.MaxContainers == 0 {
res.limits.MaxContainers = defaultPreviewMaxContainers
}
if res.limits.MaxItemsPerContainer == 0 {
res.limits.MaxItemsPerContainer = defaultPreviewMaxItemsPerContainer
}
if res.limits.MaxItems == 0 {
res.limits.MaxItems = defaultPreviewMaxItems
}
if res.limits.MaxBytes == 0 {
res.limits.MaxBytes = defaultPreviewMaxBytes
}
if res.limits.MaxPages == 0 {
res.limits.MaxPages = defaultPreviewMaxPages
}
return res
}
type pagerLimiter struct {
limits control.PreviewItemLimits
}
func (l pagerLimiter) effectiveLimits() control.PreviewItemLimits {
return l.limits
}
func (l pagerLimiter) enabled() bool {
return l.limits.Enabled
}
// sizeLimit returns the total number of bytes this backup should try to
// contain.
func (l pagerLimiter) sizeLimit() int64 {
return l.limits.MaxBytes
}
// atItemLimit returns true if the limiter is enabled and has reached the limit
// for individual items added to collections for this backup.
func (l pagerLimiter) atItemLimit(stats *driveEnumerationStats) bool {
return l.enabled() &&
(stats.numAddedFiles >= l.limits.MaxItems ||
stats.numBytes >= l.limits.MaxBytes)
}
// atContainerItemsLimit returns true if the limiter is enabled and the current
// number of items is above the limit for the number of items for a container
// for this backup.
func (l pagerLimiter) atContainerItemsLimit(numItems int) bool {
return l.enabled() && numItems >= l.limits.MaxItemsPerContainer
}
// atContainerPageLimit returns true if the limiter is enabled and the number of
// pages processed so far is beyond the limit for this backup.
func (l pagerLimiter) atPageLimit(stats *driveEnumerationStats) bool {
return l.enabled() && stats.numPages >= l.limits.MaxPages
}
// atLimit returns true if the limiter is enabled and meets any of the
// conditions for max items, containers, etc for this backup.
func (l pagerLimiter) atLimit(
stats *driveEnumerationStats,
containerCount int,
) bool {
nc := stats.numContainers
if nc == 0 && containerCount > 0 {
nc = containerCount
}
return l.enabled() &&
(l.atItemLimit(stats) ||
nc >= l.limits.MaxContainers ||
stats.numPages >= l.limits.MaxPages)
}

File diff suppressed because it is too large Load Diff

View File

@ -15,6 +15,7 @@ import (
"github.com/alcionai/corso/src/pkg/path" "github.com/alcionai/corso/src/pkg/path"
"github.com/alcionai/corso/src/pkg/selectors" "github.com/alcionai/corso/src/pkg/selectors"
"github.com/alcionai/corso/src/pkg/services/m365/api" "github.com/alcionai/corso/src/pkg/services/m365/api"
apiMock "github.com/alcionai/corso/src/pkg/services/m365/api/mock"
"github.com/alcionai/corso/src/pkg/services/m365/api/pagers" "github.com/alcionai/corso/src/pkg/services/m365/api/pagers"
) )
@ -106,6 +107,18 @@ func DefaultSharePointBH(resourceOwner string) *BackupHandler[models.DriveItemab
} }
} }
func DefaultDriveBHWith(
resource string,
drivePager *apiMock.Pager[models.Driveable],
enumerator EnumerateItemsDeltaByDrive,
) *BackupHandler[models.DriveItemable] {
mbh := DefaultOneDriveBH(resource)
mbh.DrivePagerV = drivePager
mbh.DriveItemEnumeration = enumerator
return mbh
}
func (h BackupHandler[T]) PathPrefix(tID, driveID string) (path.Path, error) { func (h BackupHandler[T]) PathPrefix(tID, driveID string) (path.Path, error) {
pp, err := h.PathPrefixFn(tID, h.ProtectedResource.ID(), driveID) pp, err := h.PathPrefixFn(tID, h.ProtectedResource.ID(), driveID)
if err != nil { if err != nil {
@ -121,7 +134,7 @@ func (h BackupHandler[T]) MetadataPathPrefix(tID string) (path.Path, error) {
return nil, err return nil, err
} }
return pp, h.PathPrefixErr return pp, h.MetadataPathPrefixErr
} }
func (h BackupHandler[T]) CanonicalPath(pb *path.Builder, tID string) (path.Path, error) { func (h BackupHandler[T]) CanonicalPath(pb *path.Builder, tID string) (path.Path, error) {

30
src/pkg/count/testdata/count.go vendored Normal file
View File

@ -0,0 +1,30 @@
package testdata
import (
"testing"
"github.com/stretchr/testify/assert"
"github.com/alcionai/corso/src/pkg/count"
)
type Expected map[count.Key]int64
func (e Expected) Compare(
t *testing.T,
bus *count.Bus,
) {
vs := bus.Values()
results := map[count.Key]int64{}
for k := range e {
results[k] = bus.Get(k)
delete(vs, string(k))
}
for k, v := range vs {
t.Logf("unchecked count %q: %d", k, v)
}
assert.Equal(t, e, Expected(results))
}