Compare commits

...

2 Commits

Author SHA1 Message Date
ryanfkeepers
d64b2e8237 adds a subtree migration iface to kopia
Introdudces the user pn to id migration. Since
onedrive doesn't enumerate folders on backup,
we have to force the migration by introducing a
subtree migrator in kopia.
2023-04-13 09:20:57 -06:00
ryanfkeepers
0e3469c23e fix overlapping bases after fallback union
In a case where the current manifests and the
fallback contain both distinct and overlapping
categories, the end result contains multiple
bases for the same category.  Verifydistinctresons
didn't catch this because it includes the resource
owner.
2023-04-13 09:17:54 -06:00
16 changed files with 683 additions and 73 deletions

View File

@ -132,7 +132,7 @@ func StateOf(prev, curr path.Path) CollectionState {
return NewState return NewState
} }
if curr.Folder(false) != prev.Folder(false) { if curr.String() != prev.String() {
return MovedState return MovedState
} }

View File

@ -0,0 +1,40 @@
package kopia
import (
"strings"
"github.com/alcionai/corso/src/pkg/path"
)
type SubtreeMigrator interface {
// GetNewSubtree potentially transforms a given subtree (repo tree prefix
// corresponding to a kopia Reason (eg: resource owner, service, category)
// into a new subtree when merging items from the base tree.
GetNewSubtree(oldSubtree *path.Builder) *path.Builder
}
type subtreeOwnerMigrator struct {
new, old string
}
// migrates any subtree with a matching old owner onto the new owner
func (om subtreeOwnerMigrator) GetNewSubtree(old *path.Builder) *path.Builder {
if old == nil {
return nil
}
elems := old.Elements()
if len(elems) < 4 {
return old
}
if strings.EqualFold(elems[2], om.old) {
elems[2] = om.new
}
return path.Builder{}.Append(elems...)
}
func NewSubtreeOwnerMigration(new, old string) *subtreeOwnerMigrator {
return &subtreeOwnerMigrator{new, old}
}

View File

@ -0,0 +1,62 @@
package kopia
import (
"testing"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/suite"
"github.com/alcionai/corso/src/internal/tester"
"github.com/alcionai/corso/src/pkg/path"
)
type MigrationsUnitSuite struct {
tester.Suite
}
func TestMigrationsUnitSuite(t *testing.T) {
suite.Run(t, &MigrationsUnitSuite{Suite: tester.NewUnitSuite(t)})
}
func (suite *MigrationsUnitSuite) TestSubtreeOwnerMigration_GetNewSubtree() {
table := []struct {
name string
input *path.Builder
expect *path.Builder
}{
{
name: "nil builder",
},
{
name: "builder too small",
input: path.Builder{}.Append("foo"),
expect: path.Builder{}.Append("foo"),
},
{
name: "non-matching owner",
input: path.Builder{}.Append("foo", "bar", "ownerronwo", "baz"),
expect: path.Builder{}.Append("foo", "bar", "ownerronwo", "baz"),
},
{
name: "migrated",
input: path.Builder{}.Append("foo", "bar", "owner", "baz"),
expect: path.Builder{}.Append("foo", "bar", "migrated", "baz"),
},
}
for _, test := range table {
suite.Run(test.name, func() {
var (
t = suite.T()
stm = NewSubtreeOwnerMigration("migrated", "owner")
result = stm.GetNewSubtree(test.input)
)
if result == nil {
assert.Nil(t, test.expect)
return
}
assert.Equal(t, test.expect.String(), result.String())
})
}
}

View File

@ -20,6 +20,22 @@ func encodeElements(elements ...string) []string {
return encoded return encoded
} }
func decodeElements(elements ...string) []string {
decoded := make([]string, 0, len(elements))
for _, e := range elements {
bs, err := encoder.DecodeString(e)
if err != nil {
decoded = append(decoded, "error decoding: "+e)
continue
}
decoded = append(decoded, string(bs))
}
return decoded
}
// encodeAsPath takes a set of elements and returns the concatenated elements as // encodeAsPath takes a set of elements and returns the concatenated elements as
// if they were a path. The elements are joined with the separator in the golang // if they were a path. The elements are joined with the separator in the golang
// path package. // path package.

View File

@ -137,7 +137,10 @@ type corsoProgress struct {
deets *details.Builder deets *details.Builder
// toMerge represents items that we don't have in-memory item info for. The // toMerge represents items that we don't have in-memory item info for. The
// item info for these items should be sourced from a base snapshot later on. // item info for these items should be sourced from a base snapshot later on.
toMerge *mergeDetails toMerge *mergeDetails
// the SubtreeMigrator can be optionally poplated to force a subtree migration
// when merging files from the a prior base.
stm SubtreeMigrator
mu sync.RWMutex mu sync.RWMutex
totalBytes int64 totalBytes int64
errs *fault.Bus errs *fault.Bus
@ -890,8 +893,7 @@ func traverseBaseDir(
oldDirPath, oldDirPath,
currentPath, currentPath,
dEntry, dEntry,
roots, roots)
)
}) })
if err != nil { if err != nil {
return clues.Wrap(err, "traversing base directory") return clues.Wrap(err, "traversing base directory")
@ -948,6 +950,7 @@ func inflateBaseTree(
snap IncrementalBase, snap IncrementalBase,
updatedPaths map[string]path.Path, updatedPaths map[string]path.Path,
roots map[string]*treeMap, roots map[string]*treeMap,
stm SubtreeMigrator,
) error { ) error {
// Only complete snapshots should be used to source base information. // Only complete snapshots should be used to source base information.
// Snapshots for checkpoints will rely on kopia-assisted dedupe to efficiently // Snapshots for checkpoints will rely on kopia-assisted dedupe to efficiently
@ -991,6 +994,11 @@ func inflateBaseTree(
return clues.Wrap(err, "subtree root is not directory").WithClues(ictx) return clues.Wrap(err, "subtree root is not directory").WithClues(ictx)
} }
migratedPath := subtreePath
if stm != nil {
migratedPath = stm.GetNewSubtree(subtreePath)
}
// We're assuming here that the prefix for the path has not changed (i.e. // We're assuming here that the prefix for the path has not changed (i.e.
// all of tenant, service, resource owner, and category are the same in the // all of tenant, service, resource owner, and category are the same in the
// old snapshot (snap) and the snapshot we're currently trying to make. // old snapshot (snap) and the snapshot we're currently trying to make.
@ -999,7 +1007,7 @@ func inflateBaseTree(
0, 0,
updatedPaths, updatedPaths,
subtreePath.Dir(), subtreePath.Dir(),
subtreePath.Dir(), migratedPath.Dir(),
subtreeDir, subtreeDir,
roots, roots,
); err != nil { ); err != nil {
@ -1050,7 +1058,8 @@ func inflateDirTree(
} }
for _, snap := range baseSnaps { for _, snap := range baseSnaps {
if err = inflateBaseTree(ctx, loader, snap, updatedPaths, roots); err != nil { err := inflateBaseTree(ctx, loader, snap, updatedPaths, roots, progress.stm)
if err != nil {
return nil, clues.Wrap(err, "inflating base snapshot tree(s)") return nil, clues.Wrap(err, "inflating base snapshot tree(s)")
} }
} }

View File

@ -183,16 +183,16 @@ func expectDirs(
) { ) {
t.Helper() t.Helper()
if exactly { ents := make([]string, 0, len(entries))
require.Len(t, entries, len(dirs))
}
names := make([]string, 0, len(entries))
for _, e := range entries { for _, e := range entries {
names = append(names, e.Name()) ents = append(ents, e.Name())
} }
assert.Subset(t, names, dirs) if exactly {
require.Lenf(t, entries, len(dirs), "expected %+v\ngot %+v", decodeElements(dirs...), decodeElements(ents...))
}
assert.Subsetf(t, dirs, ents, "expected %+v\ngot %+v", decodeElements(dirs...), decodeElements(ents...))
} }
func getDirEntriesForEntry( func getDirEntriesForEntry(
@ -2727,3 +2727,248 @@ func (suite *HierarchyBuilderUnitSuite) TestBuildDirectoryTreeSelectsCorrectSubt
expectTree(t, ctx, expected, dirTree) expectTree(t, ctx, expected, dirTree)
} }
type baseMigrator struct {
pb *path.Builder
}
func (bm baseMigrator) GetNewSubtree(_ *path.Builder) *path.Builder {
return bm.pb
}
func (suite *HierarchyBuilderUnitSuite) TestBuildDirectoryTree_migratesBaseSubtree() {
tester.LogTimeOfTest(suite.T())
t := suite.T()
ctx, flush := tester.NewContext()
defer flush()
const contactsDir = "contacts"
var (
inboxPath = makePath(
suite.T(),
[]string{testTenant, service, "owny", category, testInboxID},
false)
inboxFileName1 = testFileName
inboxFileName2 = testFileName2
inboxFileData1 = testFileData
inboxFileData1v2 = testFileData5
inboxFileData2 = testFileData2
contactsFileName1 = testFileName3
contactsFileData1 = testFileData3
eventsFileName1 = testFileName5
eventsFileData1 = testFileData
)
// Must be a function that returns a new instance each time as StreamingFile
// can only return its Reader once.
// baseSnapshot with the following layout:
// - a-tenant
// - exchange
// - user1
// - email
// - Inbox
// - file1
// - contacts
// - contacts
// - file2
getBaseSnapshot1 := func() fs.Entry {
return baseWithChildren(
[]string{
testTenant,
service,
testUser,
},
[]fs.Entry{
virtualfs.NewStaticDirectory(
encodeElements(category)[0],
[]fs.Entry{
virtualfs.NewStaticDirectory(
encodeElements(testInboxID)[0],
[]fs.Entry{
virtualfs.StreamingFileWithModTimeFromReader(
encodeElements(inboxFileName1)[0],
time.Time{},
io.NopCloser(bytes.NewReader(inboxFileData1)),
),
},
),
},
),
virtualfs.NewStaticDirectory(
encodeElements(path.ContactsCategory.String())[0],
[]fs.Entry{
virtualfs.NewStaticDirectory(
encodeElements(contactsDir)[0],
[]fs.Entry{
virtualfs.StreamingFileWithModTimeFromReader(
encodeElements(contactsFileName1)[0],
time.Time{},
io.NopCloser(bytes.NewReader(contactsFileData1)),
),
},
),
},
),
},
)
}
// Must be a function that returns a new instance each time as StreamingFile
// can only return its Reader once.
// baseSnapshot with the following layout:
// - a-tenant
// - exchange
// - user1
// - email
// - Inbox
// - file1 <- has different data version
// - events
// - events
// - file3
getBaseSnapshot2 := func() fs.Entry {
return baseWithChildren(
[]string{
testTenant,
service,
testUser,
},
[]fs.Entry{
virtualfs.NewStaticDirectory(
encodeElements(category)[0],
[]fs.Entry{
virtualfs.NewStaticDirectory(
encodeElements(testInboxID)[0],
[]fs.Entry{
virtualfs.StreamingFileWithModTimeFromReader(
encodeElements(inboxFileName1)[0],
time.Time{},
// Wrap with a backup reader so it gets the version injected.
newBackupStreamReader(
serializationVersion,
io.NopCloser(bytes.NewReader(inboxFileData1v2)),
),
),
},
),
},
),
virtualfs.NewStaticDirectory(
encodeElements(path.EventsCategory.String())[0],
[]fs.Entry{
virtualfs.NewStaticDirectory(
encodeElements("events")[0],
[]fs.Entry{
virtualfs.StreamingFileWithModTimeFromReader(
encodeElements(eventsFileName1)[0],
time.Time{},
io.NopCloser(bytes.NewReader(eventsFileData1)),
),
},
),
},
),
},
)
}
// Check the following:
// * contacts pulled from base1 unchanged even if no collections reference
// it
// * email pulled from base2
// * new email added
// * events not pulled from base2 as it's not listed as a Reason
//
// Expected output:
// - a-tenant
// - exchange
// - owny <- migrated
// - email
// - Inbox
// - file1 <- version of data from second base
// - file2
// - contacts
// - contacts
// - file2
expected := expectedTreeWithChildren(
[]string{testTenant, path.ExchangeService.String(), "owny"},
[]*expectedNode{
{
name: category,
children: []*expectedNode{
{
name: testInboxID,
children: []*expectedNode{
{
name: inboxFileName1,
children: []*expectedNode{},
data: inboxFileData1v2,
},
{
name: inboxFileName2,
children: []*expectedNode{},
data: inboxFileData2,
},
},
},
},
},
{
name: path.ContactsCategory.String(),
children: []*expectedNode{
{
name: contactsDir,
children: []*expectedNode{
{
name: contactsFileName1,
children: []*expectedNode{},
},
},
},
},
},
},
)
progress := &corsoProgress{
pending: map[string]*itemDetails{},
toMerge: newMergeDetails(),
errs: fault.New(true),
stm: baseMigrator{path.Builder{}.Append(testTenant, path.ExchangeService.String(), "owny", category)},
}
mc := mockconnector.NewMockExchangeCollection(inboxPath, inboxPath, 1)
mc.PrevPath = mc.FullPath()
mc.ColState = data.NotMovedState
mc.Names[0] = inboxFileName2
mc.Data[0] = inboxFileData2
msw := &mockMultiSnapshotWalker{
snaps: map[string]fs.Entry{
"id1": getBaseSnapshot1(),
"id2": getBaseSnapshot2(),
},
}
collections := []data.BackupCollection{mc}
dirTree, err := inflateDirTree(
ctx,
msw,
[]IncrementalBase{
mockIncrementalBase("id1", testTenant, testUser, path.ExchangeService, path.ContactsCategory),
mockIncrementalBase("id2", testTenant, testUser, path.ExchangeService, path.EmailCategory),
},
collections,
nil,
progress,
)
require.NoError(t, err, clues.ToCore(err))
expectTree(t, ctx, expected, dirTree)
}

View File

@ -137,6 +137,7 @@ func (w Wrapper) ConsumeBackupCollections(
globalExcludeSet map[string]map[string]struct{}, globalExcludeSet map[string]map[string]struct{},
tags map[string]string, tags map[string]string,
buildTreeWithBase bool, buildTreeWithBase bool,
stm SubtreeMigrator,
errs *fault.Bus, errs *fault.Bus,
) (*BackupStats, *details.Builder, DetailsMergeInfoer, error) { ) (*BackupStats, *details.Builder, DetailsMergeInfoer, error) {
if w.c == nil { if w.c == nil {
@ -155,6 +156,7 @@ func (w Wrapper) ConsumeBackupCollections(
deets: &details.Builder{}, deets: &details.Builder{},
toMerge: newMergeDetails(), toMerge: newMergeDetails(),
errs: errs, errs: errs,
stm: stm,
} }
// When running an incremental backup, we need to pass the prior // When running an incremental backup, we need to pass the prior

View File

@ -283,6 +283,7 @@ func (suite *KopiaIntegrationSuite) TestBackupCollections() {
nil, nil,
tags, tags,
true, true,
nil,
fault.New(true)) fault.New(true))
assert.NoError(t, err, clues.ToCore(err)) assert.NoError(t, err, clues.ToCore(err))
@ -430,6 +431,7 @@ func (suite *KopiaIntegrationSuite) TestBackupCollections_NoDetailsForMeta() {
nil, nil,
tags, tags,
true, true,
nil,
fault.New(true)) fault.New(true))
assert.NoError(t, err, clues.ToCore(err)) assert.NoError(t, err, clues.ToCore(err))
@ -528,6 +530,7 @@ func (suite *KopiaIntegrationSuite) TestRestoreAfterCompressionChange() {
nil, nil,
tags, tags,
true, true,
nil,
fault.New(true)) fault.New(true))
require.NoError(t, err, clues.ToCore(err)) require.NoError(t, err, clues.ToCore(err))
@ -647,6 +650,7 @@ func (suite *KopiaIntegrationSuite) TestBackupCollections_ReaderError() {
nil, nil,
tags, tags,
true, true,
nil,
fault.New(true)) fault.New(true))
require.Error(t, err, clues.ToCore(err)) require.Error(t, err, clues.ToCore(err))
assert.Equal(t, 0, stats.ErrorCount) assert.Equal(t, 0, stats.ErrorCount)
@ -709,6 +713,7 @@ func (suite *KopiaIntegrationSuite) TestBackupCollectionsHandlesNoCollections()
nil, nil,
nil, nil,
true, true,
nil,
fault.New(true)) fault.New(true))
require.NoError(t, err, clues.ToCore(err)) require.NoError(t, err, clues.ToCore(err))
@ -869,6 +874,7 @@ func (suite *KopiaSimpleRepoIntegrationSuite) SetupTest() {
nil, nil,
tags, tags,
false, false,
nil,
fault.New(true)) fault.New(true))
require.NoError(t, err, clues.ToCore(err)) require.NoError(t, err, clues.ToCore(err))
require.Equal(t, stats.ErrorCount, 0) require.Equal(t, stats.ErrorCount, 0)
@ -1028,6 +1034,7 @@ func (suite *KopiaSimpleRepoIntegrationSuite) TestBackupExcludeItem() {
excluded, excluded,
tags, tags,
true, true,
nil,
fault.New(true)) fault.New(true))
require.NoError(t, err, clues.ToCore(err)) require.NoError(t, err, clues.ToCore(err))
assert.Equal(t, test.expectedCachedItems, stats.CachedFileCount) assert.Equal(t, test.expectedCachedItems, stats.CachedFileCount)

View File

@ -230,8 +230,8 @@ func (op *BackupOperation) do(
backupID model.StableID, backupID model.StableID,
) (*details.Builder, error) { ) (*details.Builder, error) {
var ( var (
reasons = selectorToReasons(op.Selectors, false) reasons = selectorToReasons(op.Selectors, op.Selectors.DiscreteOwner)
fallbackReasons = makeFallbackReasons(op.Selectors) fallbackReasons, stm = makeFallbackReasons(op.Selectors)
) )
logger.Ctx(ctx).With("selectors", op.Selectors).Info("backing up selection") logger.Ctx(ctx).With("selectors", op.Selectors).Info("backing up selection")
@ -274,6 +274,7 @@ func (op *BackupOperation) do(
excludes, excludes,
backupID, backupID,
op.incremental && canUseMetaData, op.incremental && canUseMetaData,
stm,
op.Errors) op.Errors)
if err != nil { if err != nil {
return nil, clues.Wrap(err, "persisting collection backups") return nil, clues.Wrap(err, "persisting collection backups")
@ -300,13 +301,25 @@ func (op *BackupOperation) do(
return deets, nil return deets, nil
} }
func makeFallbackReasons(sel selectors.Selector) []kopia.Reason { func makeFallbackReasons(sel selectors.Selector) ([]kopia.Reason, kopia.SubtreeMigrator) {
if sel.PathService() != path.SharePointService && if sel.PathService() == path.SharePointService ||
sel.DiscreteOwner != sel.DiscreteOwnerName { sel.DiscreteOwner == sel.DiscreteOwnerName {
return selectorToReasons(sel, true) return nil, nil
} }
return nil var (
fbs = selectorToReasons(sel, sel.DiscreteOwnerName)
stm kopia.SubtreeMigrator
)
switch sel.PathService() {
// only onedrive is required here. Exchange will naturally get migrated
// since all exchange folders are enumerated during backup collection aggregation.
case path.OneDriveService:
stm = kopia.NewSubtreeOwnerMigration(sel.DiscreteOwner, sel.DiscreteOwnerName)
}
return fbs, stm
} }
// checker to see if conditions are correct for incremental backup behavior such as // checker to see if conditions are correct for incremental backup behavior such as
@ -350,7 +363,7 @@ func produceBackupDataCollections(
// Consumer funcs // Consumer funcs
// --------------------------------------------------------------------------- // ---------------------------------------------------------------------------
func selectorToReasons(sel selectors.Selector, useOwnerNameForID bool) []kopia.Reason { func selectorToReasons(sel selectors.Selector, owner string) []kopia.Reason {
service := sel.PathService() service := sel.PathService()
reasons := []kopia.Reason{} reasons := []kopia.Reason{}
@ -361,11 +374,6 @@ func selectorToReasons(sel selectors.Selector, useOwnerNameForID bool) []kopia.R
return nil return nil
} }
owner := sel.DiscreteOwner
if useOwnerNameForID {
owner = sel.DiscreteOwnerName
}
for _, sl := range [][]path.CategoryType{pcs.Includes, pcs.Filters} { for _, sl := range [][]path.CategoryType{pcs.Includes, pcs.Filters} {
for _, cat := range sl { for _, cat := range sl {
reasons = append(reasons, kopia.Reason{ reasons = append(reasons, kopia.Reason{
@ -410,6 +418,7 @@ func consumeBackupCollections(
excludes map[string]map[string]struct{}, excludes map[string]map[string]struct{},
backupID model.StableID, backupID model.StableID,
isIncremental bool, isIncremental bool,
stm kopia.SubtreeMigrator,
errs *fault.Bus, errs *fault.Bus,
) (*kopia.BackupStats, *details.Builder, kopia.DetailsMergeInfoer, error) { ) (*kopia.BackupStats, *details.Builder, kopia.DetailsMergeInfoer, error) {
complete, closer := observe.MessageWithCompletion(ctx, "Backing up data") complete, closer := observe.MessageWithCompletion(ctx, "Backing up data")
@ -483,6 +492,7 @@ func consumeBackupCollections(
excludes, excludes,
tags, tags,
isIncremental, isIncremental,
stm,
errs) errs)
if err != nil { if err != nil {
if kopiaStats == nil { if kopiaStats == nil {

View File

@ -31,6 +31,7 @@ import (
"github.com/alcionai/corso/src/internal/kopia" "github.com/alcionai/corso/src/internal/kopia"
"github.com/alcionai/corso/src/internal/model" "github.com/alcionai/corso/src/internal/model"
"github.com/alcionai/corso/src/internal/operations/inject" "github.com/alcionai/corso/src/internal/operations/inject"
"github.com/alcionai/corso/src/internal/streamstore"
"github.com/alcionai/corso/src/internal/tester" "github.com/alcionai/corso/src/internal/tester"
"github.com/alcionai/corso/src/internal/version" "github.com/alcionai/corso/src/internal/version"
"github.com/alcionai/corso/src/pkg/account" "github.com/alcionai/corso/src/pkg/account"
@ -1599,6 +1600,139 @@ func (suite *BackupOpIntegrationSuite) TestBackup_Run_oneDriveIncrementals() {
} }
} }
func (suite *BackupOpIntegrationSuite) TestBackup_Run_oneDriveOwnerMigration() {
ctx, flush := tester.NewContext()
defer flush()
var (
t = suite.T()
acct = tester.NewM365Account(t)
ffs = control.Toggles{}
mb = evmock.NewBus()
// `now` has to be formatted with SimpleDateTimeOneDrive as
// some onedrive cannot have `:` in file/folder names
now = common.FormatNow(common.SimpleTimeTesting)
owners = []string{suite.user}
categories = map[path.CategoryType][]string{
path.FilesCategory: {graph.DeltaURLsFileName, graph.PreviousPathFileName},
}
container = fmt.Sprintf("%s%d_%s", incrementalsDestContainerPrefix, 1, now)
genDests = []string{container}
)
creds, err := acct.M365Config()
require.NoError(t, err, clues.ToCore(err))
gc, err := connector.NewGraphConnector(
ctx,
graph.HTTPClient(graph.NoTimeout()),
acct,
connector.Users,
fault.New(true))
require.NoError(t, err, clues.ToCore(err))
driveID := mustGetDefaultDriveID(t, ctx, gc.Service, suite.user)
fileDBF := func(id, timeStamp, subject, body string) []byte {
return []byte(id + subject)
}
// Populate initial test data.
for _, destName := range genDests {
generateContainerOfItems(
t,
ctx,
gc,
path.OneDriveService,
acct,
path.FilesCategory,
selectors.NewOneDriveRestore(owners).Selector,
creds.AzureTenantID, suite.user, driveID, destName,
2,
// Use an old backup version so we don't need metadata files.
0,
fileDBF)
}
// container3 does not exist yet. It will get created later on
// during the tests.
containers := []string{container}
sel := selectors.NewOneDriveBackup(owners)
sel.Include(sel.Folders(containers, selectors.PrefixMatch()))
bo, _, kw, ms, gc, closer := prepNewTestBackupOp(t, ctx, mb, sel.Selector, ffs)
defer closer()
// ensure the initial owner uses name in both cases
sel.SetDiscreteOwnerIDName(suite.user, suite.user)
bo.ResourceOwner = sel
// TODO: ensure this equals the PN
require.Equal(t, bo.ResourceOwner.Name(), bo.ResourceOwner.ID(), "historical representation of user")
// run the initial backup
runAndCheckBackup(t, ctx, &bo, mb, false)
var (
incMB = evmock.NewBus()
// the incremental backup op should have a proper user ID for the id.
incBO = newTestBackupOp(t, ctx, kw, ms, gc, acct, sel.Selector, incMB, ffs, closer)
)
require.NotEqual(
t,
incBO.ResourceOwner.Name(),
incBO.ResourceOwner.ID(),
"current representation of user: id should differ from PN")
err = incBO.Run(ctx)
require.NoError(t, err, clues.ToCore(err))
checkBackupIsInManifests(t, ctx, kw, &incBO, sel.Selector, suite.user, maps.Keys(categories)...)
checkMetadataFilesExist(
t,
ctx,
incBO.Results.BackupID,
kw,
ms,
creds.AzureTenantID,
suite.user,
path.OneDriveService,
categories)
// 2 on read/writes to account for metadata: 1 delta and 1 path.
assert.LessOrEqual(t, 2, incBO.Results.ItemsWritten, "items written")
assert.LessOrEqual(t, 2, incBO.Results.ItemsRead, "items read")
assert.NoError(t, incBO.Errors.Failure(), "non-recoverable error", clues.ToCore(incBO.Errors.Failure()))
assert.Empty(t, incBO.Errors.Recovered(), "recoverable/iteration errors")
assert.Equal(t, 1, incMB.TimesCalled[events.BackupStart], "backup-start events")
assert.Equal(t, 1, incMB.TimesCalled[events.BackupEnd], "backup-end events")
assert.Equal(t,
incMB.CalledWith[events.BackupStart][0][events.BackupID],
incBO.Results.BackupID, "backupID pre-declaration")
bid := incBO.Results.BackupID
bup := &backup.Backup{}
err = ms.Get(ctx, model.BackupSchema, bid, bup)
require.NoError(t, err, clues.ToCore(err))
var (
ssid = bup.StreamStoreID
deets details.Details
ss = streamstore.NewStreamer(kw, creds.AzureTenantID, path.OneDriveService)
)
err = ss.Read(ctx, ssid, streamstore.DetailsReader(details.UnmarshalTo(&deets)), fault.New(true))
require.NoError(t, err, clues.ToCore(err))
for _, ent := range deets.Entries {
assert.Contains(t, ent.RepoRef, incBO.ResourceOwner.ID())
}
}
// --------------------------------------------------------------------------- // ---------------------------------------------------------------------------
// SharePoint // SharePoint
// --------------------------------------------------------------------------- // ---------------------------------------------------------------------------

View File

@ -101,6 +101,7 @@ func (mbu mockBackupConsumer) ConsumeBackupCollections(
excluded map[string]map[string]struct{}, excluded map[string]map[string]struct{},
tags map[string]string, tags map[string]string,
buildTreeWithBase bool, buildTreeWithBase bool,
_ kopia.SubtreeMigrator,
errs *fault.Bus, errs *fault.Bus,
) (*kopia.BackupStats, *details.Builder, kopia.DetailsMergeInfoer, error) { ) (*kopia.BackupStats, *details.Builder, kopia.DetailsMergeInfoer, error) {
if mbu.checkFunc != nil { if mbu.checkFunc != nil {
@ -624,6 +625,7 @@ func (suite *BackupOpUnitSuite) TestBackupOperation_ConsumeBackupDataCollections
nil, nil,
model.StableID(""), model.StableID(""),
true, true,
nil,
fault.New(true)) fault.New(true))
}) })
} }

View File

@ -36,6 +36,7 @@ type (
excluded map[string]map[string]struct{}, excluded map[string]map[string]struct{},
tags map[string]string, tags map[string]string,
buildTreeWithBase bool, buildTreeWithBase bool,
stm kopia.SubtreeMigrator,
errs *fault.Bus, errs *fault.Bus,
) (*kopia.BackupStats, *details.Builder, kopia.DetailsMergeInfoer, error) ) (*kopia.BackupStats, *details.Builder, kopia.DetailsMergeInfoer, error)
} }

View File

@ -2,6 +2,7 @@ package operations
import ( import (
"context" "context"
"fmt"
"github.com/alcionai/clues" "github.com/alcionai/clues"
"github.com/kopia/kopia/repo/manifest" "github.com/kopia/kopia/repo/manifest"
@ -93,20 +94,6 @@ func produceManifestsAndMetadata(
return ms, nil, false, nil return ms, nil, false, nil
} }
// We only need to check that we have 1:1 reason:base if we're doing an
// incremental with associated metadata. This ensures that we're only sourcing
// data from a single Point-In-Time (base) for each incremental backup.
//
// TODO(ashmrtn): This may need updating if we start sourcing item backup
// details from previous snapshots when using kopia-assisted incrementals.
if err := verifyDistinctBases(ctx, ms); err != nil {
logger.Ctx(ctx).With("error", err).Infow(
"unioned snapshot collision, falling back to full backup",
clues.In(ctx).Slice()...)
return ms, nil, false, nil
}
for _, man := range ms { for _, man := range ms {
if len(man.IncompleteReason) > 0 { if len(man.IncompleteReason) > 0 {
continue continue
@ -163,6 +150,7 @@ func produceManifestsAndMetadata(
LogFaultErrors(ctx, fb.Errors(), "collecting metadata") LogFaultErrors(ctx, fb.Errors(), "collecting metadata")
if err != nil && !errors.Is(err, data.ErrNotFound) { if err != nil && !errors.Is(err, data.ErrNotFound) {
fmt.Printf("\n-----\nCOLLECTING METADATA %+v\n-----\n", err)
// prior metadata isn't guaranteed to exist. // prior metadata isn't guaranteed to exist.
// if it doesn't, we'll just have to do a // if it doesn't, we'll just have to do a
// full backup for that data. // full backup for that data.
@ -234,6 +222,8 @@ func unionManifests(
// backfill from the fallback where necessary // backfill from the fallback where necessary
for _, m := range fallback { for _, m := range fallback {
useReasons := []kopia.Reason{}
for _, r := range m.Reasons { for _, r := range m.Reasons {
k := r.Service.String() + r.Category.String() k := r.Service.String() + r.Category.String()
t := tups[k] t := tups[k]
@ -245,6 +235,8 @@ func unionManifests(
continue continue
} }
useReasons = append(useReasons, r)
if len(m.IncompleteReason) > 0 && t.incomplete == nil { if len(m.IncompleteReason) > 0 && t.incomplete == nil {
t.incomplete = m t.incomplete = m
} else if len(m.IncompleteReason) == 0 { } else if len(m.IncompleteReason) == 0 {
@ -253,6 +245,10 @@ func unionManifests(
tups[k] = t tups[k] = t
} }
if len(m.IncompleteReason) == 0 && len(useReasons) > 0 {
m.Reasons = useReasons
}
} }
// collect the results into a single slice of manifests // collect the results into a single slice of manifests

View File

@ -757,18 +757,20 @@ func (suite *OperationsManifestsUnitSuite) TestProduceManifestsAndMetadata_fallb
table := []struct { table := []struct {
name string name string
main []testInput man []testInput
fallback []testInput fallback []testInput
reasons []kopia.Reason reasons []kopia.Reason
fallbackReasons []kopia.Reason fallbackReasons []kopia.Reason
categories []path.CategoryType manCategories []path.CategoryType
fbCategories []path.CategoryType
assertErr assert.ErrorAssertionFunc assertErr assert.ErrorAssertionFunc
expectManIDs []string expectManIDs []string
expectNilMans bool expectNilMans bool
expectReasons map[string][]path.CategoryType
}{ }{
{ {
name: "only mans, no fallbacks", name: "only mans, no fallbacks",
main: []testInput{ man: []testInput{
{ {
id: manComplete, id: manComplete,
}, },
@ -777,8 +779,13 @@ func (suite *OperationsManifestsUnitSuite) TestProduceManifestsAndMetadata_fallb
incomplete: true, incomplete: true,
}, },
}, },
categories: []path.CategoryType{path.EmailCategory}, manCategories: []path.CategoryType{path.EmailCategory},
expectManIDs: []string{manComplete, manIncomplete}, fbCategories: []path.CategoryType{path.EmailCategory},
expectManIDs: []string{manComplete, manIncomplete},
expectReasons: map[string][]path.CategoryType{
manComplete: {path.EmailCategory},
manIncomplete: {path.EmailCategory},
},
}, },
{ {
name: "no mans, only fallbacks", name: "no mans, only fallbacks",
@ -791,12 +798,17 @@ func (suite *OperationsManifestsUnitSuite) TestProduceManifestsAndMetadata_fallb
incomplete: true, incomplete: true,
}, },
}, },
categories: []path.CategoryType{path.EmailCategory}, manCategories: []path.CategoryType{path.EmailCategory},
expectManIDs: []string{fbComplete, fbIncomplete}, fbCategories: []path.CategoryType{path.EmailCategory},
expectManIDs: []string{fbComplete, fbIncomplete},
expectReasons: map[string][]path.CategoryType{
fbComplete: {path.EmailCategory},
fbIncomplete: {path.EmailCategory},
},
}, },
{ {
name: "complete mans and fallbacks", name: "complete mans and fallbacks",
main: []testInput{ man: []testInput{
{ {
id: manComplete, id: manComplete,
}, },
@ -806,12 +818,16 @@ func (suite *OperationsManifestsUnitSuite) TestProduceManifestsAndMetadata_fallb
id: fbComplete, id: fbComplete,
}, },
}, },
categories: []path.CategoryType{path.EmailCategory}, manCategories: []path.CategoryType{path.EmailCategory},
expectManIDs: []string{manComplete}, fbCategories: []path.CategoryType{path.EmailCategory},
expectManIDs: []string{manComplete},
expectReasons: map[string][]path.CategoryType{
manComplete: {path.EmailCategory},
},
}, },
{ {
name: "incomplete mans and fallbacks", name: "incomplete mans and fallbacks",
main: []testInput{ man: []testInput{
{ {
id: manIncomplete, id: manIncomplete,
incomplete: true, incomplete: true,
@ -823,12 +839,16 @@ func (suite *OperationsManifestsUnitSuite) TestProduceManifestsAndMetadata_fallb
incomplete: true, incomplete: true,
}, },
}, },
categories: []path.CategoryType{path.EmailCategory}, manCategories: []path.CategoryType{path.EmailCategory},
expectManIDs: []string{manIncomplete}, fbCategories: []path.CategoryType{path.EmailCategory},
expectManIDs: []string{manIncomplete},
expectReasons: map[string][]path.CategoryType{
manIncomplete: {path.EmailCategory},
},
}, },
{ {
name: "complete and incomplete mans and fallbacks", name: "complete and incomplete mans and fallbacks",
main: []testInput{ man: []testInput{
{ {
id: manComplete, id: manComplete,
}, },
@ -846,12 +866,17 @@ func (suite *OperationsManifestsUnitSuite) TestProduceManifestsAndMetadata_fallb
incomplete: true, incomplete: true,
}, },
}, },
categories: []path.CategoryType{path.EmailCategory}, manCategories: []path.CategoryType{path.EmailCategory},
expectManIDs: []string{manComplete, manIncomplete}, fbCategories: []path.CategoryType{path.EmailCategory},
expectManIDs: []string{manComplete, manIncomplete},
expectReasons: map[string][]path.CategoryType{
manComplete: {path.EmailCategory},
manIncomplete: {path.EmailCategory},
},
}, },
{ {
name: "incomplete mans, complete fallbacks", name: "incomplete mans, complete fallbacks",
main: []testInput{ man: []testInput{
{ {
id: manIncomplete, id: manIncomplete,
incomplete: true, incomplete: true,
@ -862,12 +887,17 @@ func (suite *OperationsManifestsUnitSuite) TestProduceManifestsAndMetadata_fallb
id: fbComplete, id: fbComplete,
}, },
}, },
categories: []path.CategoryType{path.EmailCategory}, manCategories: []path.CategoryType{path.EmailCategory},
expectManIDs: []string{fbComplete, manIncomplete}, fbCategories: []path.CategoryType{path.EmailCategory},
expectManIDs: []string{fbComplete, manIncomplete},
expectReasons: map[string][]path.CategoryType{
fbComplete: {path.EmailCategory},
manIncomplete: {path.EmailCategory},
},
}, },
{ {
name: "complete mans, incomplete fallbacks", name: "complete mans, incomplete fallbacks",
main: []testInput{ man: []testInput{
{ {
id: manComplete, id: manComplete,
}, },
@ -878,12 +908,16 @@ func (suite *OperationsManifestsUnitSuite) TestProduceManifestsAndMetadata_fallb
incomplete: true, incomplete: true,
}, },
}, },
categories: []path.CategoryType{path.EmailCategory}, manCategories: []path.CategoryType{path.EmailCategory},
expectManIDs: []string{manComplete}, fbCategories: []path.CategoryType{path.EmailCategory},
expectManIDs: []string{manComplete},
expectReasons: map[string][]path.CategoryType{
manComplete: {path.EmailCategory},
},
}, },
{ {
name: "complete mans, complete fallbacks, multiple reasons", name: "complete mans, complete fallbacks, multiple reasons",
main: []testInput{ man: []testInput{
{ {
id: manComplete, id: manComplete,
}, },
@ -893,8 +927,52 @@ func (suite *OperationsManifestsUnitSuite) TestProduceManifestsAndMetadata_fallb
id: fbComplete, id: fbComplete,
}, },
}, },
categories: []path.CategoryType{path.EmailCategory, path.ContactsCategory}, manCategories: []path.CategoryType{path.EmailCategory, path.ContactsCategory},
expectManIDs: []string{manComplete}, fbCategories: []path.CategoryType{path.EmailCategory, path.ContactsCategory},
expectManIDs: []string{manComplete},
expectReasons: map[string][]path.CategoryType{
manComplete: {path.EmailCategory, path.ContactsCategory},
},
},
{
name: "complete mans, complete fallbacks, distinct reasons",
man: []testInput{
{
id: manComplete,
},
},
fallback: []testInput{
{
id: fbComplete,
},
},
manCategories: []path.CategoryType{path.ContactsCategory},
fbCategories: []path.CategoryType{path.EmailCategory},
expectManIDs: []string{manComplete, fbComplete},
expectReasons: map[string][]path.CategoryType{
manComplete: {path.ContactsCategory},
fbComplete: {path.EmailCategory},
},
},
{
name: "fb has superset of mans reasons",
man: []testInput{
{
id: manComplete,
},
},
fallback: []testInput{
{
id: fbComplete,
},
},
manCategories: []path.CategoryType{path.ContactsCategory},
fbCategories: []path.CategoryType{path.EmailCategory, path.ContactsCategory, path.EventsCategory},
expectManIDs: []string{manComplete, fbComplete},
expectReasons: map[string][]path.CategoryType{
manComplete: {path.ContactsCategory},
fbComplete: {path.EmailCategory, path.EventsCategory},
},
}, },
} }
for _, test := range table { for _, test := range table {
@ -907,7 +985,7 @@ func (suite *OperationsManifestsUnitSuite) TestProduceManifestsAndMetadata_fallb
mainReasons := []kopia.Reason{} mainReasons := []kopia.Reason{}
fbReasons := []kopia.Reason{} fbReasons := []kopia.Reason{}
for _, cat := range test.categories { for _, cat := range test.manCategories {
mainReasons = append( mainReasons = append(
mainReasons, mainReasons,
kopia.Reason{ kopia.Reason{
@ -915,7 +993,9 @@ func (suite *OperationsManifestsUnitSuite) TestProduceManifestsAndMetadata_fallb
Service: path.ExchangeService, Service: path.ExchangeService,
Category: cat, Category: cat,
}) })
}
for _, cat := range test.fbCategories {
fbReasons = append( fbReasons = append(
fbReasons, fbReasons,
kopia.Reason{ kopia.Reason{
@ -927,7 +1007,7 @@ func (suite *OperationsManifestsUnitSuite) TestProduceManifestsAndMetadata_fallb
mans := []*kopia.ManifestEntry{} mans := []*kopia.ManifestEntry{}
for _, m := range test.main { for _, m := range test.man {
incomplete := "" incomplete := ""
if m.incomplete { if m.incomplete {
incomplete = "ir" incomplete = "ir"
@ -959,8 +1039,19 @@ func (suite *OperationsManifestsUnitSuite) TestProduceManifestsAndMetadata_fallb
assert.False(t, b, "no-metadata is forced for this test") assert.False(t, b, "no-metadata is forced for this test")
manIDs := []string{} manIDs := []string{}
for _, m := range mans { for _, m := range mans {
manIDs = append(manIDs, string(m.ID)) manIDs = append(manIDs, string(m.ID))
reasons, ok := test.expectReasons[string(m.ID)]
assert.True(t, ok, "unexpected manifest in result: ", m.ID)
mrs := []path.CategoryType{}
for _, r := range m.Reasons {
mrs = append(mrs, r.Category)
}
assert.ElementsMatch(t, reasons, mrs)
} }
assert.ElementsMatch(t, test.expectManIDs, manIDs) assert.ElementsMatch(t, test.expectManIDs, manIDs)

View File

@ -235,6 +235,7 @@ func write(
nil, nil,
nil, nil,
false, false,
nil,
errs) errs)
if err != nil { if err != nil {
return "", clues.Wrap(err, "storing marshalled bytes in repository") return "", clues.Wrap(err, "storing marshalled bytes in repository")

View File

@ -27,7 +27,6 @@ import (
"github.com/alcionai/corso/src/pkg/control" "github.com/alcionai/corso/src/pkg/control"
"github.com/alcionai/corso/src/pkg/fault" "github.com/alcionai/corso/src/pkg/fault"
"github.com/alcionai/corso/src/pkg/logger" "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/selectors"
"github.com/alcionai/corso/src/pkg/storage" "github.com/alcionai/corso/src/pkg/storage"
"github.com/alcionai/corso/src/pkg/store" "github.com/alcionai/corso/src/pkg/store"
@ -319,11 +318,6 @@ func (r repository) NewBackupWithLookup(
return operations.BackupOperation{}, errors.Wrap(err, "resolving resource owner details") return operations.BackupOperation{}, errors.Wrap(err, "resolving resource owner details")
} }
// Exchange and OneDrive need to maintain the user PN as the ID until we're ready to migrate
if sel.PathService() != path.SharePointService {
ownerID = ownerName
}
// TODO: retrieve display name from gc // TODO: retrieve display name from gc
sel = sel.SetDiscreteOwnerIDName(ownerID, ownerName) sel = sel.SetDiscreteOwnerIDName(ownerID, ownerName)