rename data pkg Stream to Item (#3966)

A Stream is a continuous transmission of data.
An item is a single structure.  Crossing the two
definitions generates confusion.

Primarily code movement/renaming.  Though there
is also some reduction/replacement of structs
where we'd made a variety of testable Item implementations
instead of re-using the generic mock.

---

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

- [x]  No

#### Type of change

- [x] 🧹 Tech Debt/Cleanup

#### Test Plan

- [x]  Unit test
- [x] 💚 E2E
This commit is contained in:
Keepers 2023-08-11 13:33:33 -06:00 committed by GitHub
parent eb95b43010
commit 0a947386e6
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
44 changed files with 532 additions and 570 deletions

View File

@ -5,28 +5,11 @@ import (
"io" "io"
"time" "time"
"github.com/alcionai/clues"
"github.com/alcionai/corso/src/pkg/backup/details" "github.com/alcionai/corso/src/pkg/backup/details"
"github.com/alcionai/corso/src/pkg/fault" "github.com/alcionai/corso/src/pkg/fault"
"github.com/alcionai/corso/src/pkg/path" "github.com/alcionai/corso/src/pkg/path"
) )
// ------------------------------------------------------------------------------------------------
// standard ifaces
// ------------------------------------------------------------------------------------------------
var ErrNotFound = clues.New("not found")
type CollectionState int
const (
NewState = CollectionState(iota)
NotMovedState
MovedState
DeletedState
)
// A Collection represents the set of data within a single logical location // A Collection represents the set of data within a single logical location
// denoted by FullPath. // denoted by FullPath.
type Collection interface { type Collection interface {
@ -34,7 +17,7 @@ type Collection interface {
// Each returned struct contains the next item in the collection // Each returned struct contains the next item in the collection
// The channel is closed when there are no more items in the collection or if // The channel is closed when there are no more items in the collection or if
// an unrecoverable error caused an early termination in the sender. // an unrecoverable error caused an early termination in the sender.
Items(ctx context.Context, errs *fault.Bus) <-chan Stream Items(ctx context.Context, errs *fault.Bus) <-chan Item
// FullPath returns a path struct that acts as a metadata tag for this // FullPath returns a path struct that acts as a metadata tag for this
// Collection. // Collection.
FullPath() path.Path FullPath() path.Path
@ -77,32 +60,15 @@ type FetchItemByNamer interface {
// Fetch retrieves an item with the given name from the Collection if it // Fetch retrieves an item with the given name from the Collection if it
// exists. Items retrieved with Fetch may still appear in the channel returned // exists. Items retrieved with Fetch may still appear in the channel returned
// by Items(). // by Items().
FetchItemByName(ctx context.Context, name string) (Stream, error) FetchItemByName(ctx context.Context, name string) (Item, error)
} }
// NoFetchRestoreCollection is a wrapper for a Collection that returns // Item represents a single item within a Collection
// ErrNotFound for all Fetch calls. type Item interface {
type NoFetchRestoreCollection struct { // ToReader returns an io.Reader with the item's data
Collection
FetchItemByNamer
}
func (c NoFetchRestoreCollection) FetchItemByName(context.Context, string) (Stream, error) {
return nil, ErrNotFound
}
type FetchRestoreCollection struct {
Collection
FetchItemByNamer
}
// Stream represents a single item within a Collection
// that can be consumed as a stream (it embeds io.Reader)
type Stream interface {
// ToReader returns an io.Reader for the DataStream
ToReader() io.ReadCloser ToReader() io.ReadCloser
// UUID provides a unique identifier for this data // ID provides a unique identifier for this item
UUID() string ID() string
// Deleted returns true if the item represented by this Stream has been // Deleted returns true if the item represented by this Stream has been
// deleted and should be removed from the current in-progress backup. // deleted and should be removed from the current in-progress backup.
Deleted() bool Deleted() bool
@ -125,40 +91,20 @@ type PreviousLocationPather interface {
PreviousLocationPath() details.LocationIDer PreviousLocationPath() details.LocationIDer
} }
// StreamInfo is used to provide service specific // ItemInfo returns the details.ItemInfo for the item.
// information about the Stream type ItemInfo interface {
type StreamInfo interface {
Info() details.ItemInfo Info() details.ItemInfo
} }
// StreamSize is used to provide size // ItemSize returns the size of the item in bytes.
// information about the Stream type ItemSize interface {
type StreamSize interface {
Size() int64 Size() int64
} }
// StreamModTime is used to provide the modified time of the stream's data. // ItemModTime provides the last modified time of the item.
// //
// If an item implements StreamModTime and StreamInfo it should return the same // If an item implements ItemModTime and ItemInfo it should return the same
// value here as in item.Info().Modified(). // value here as in item.Info().Modified().
type StreamModTime interface { type ItemModTime interface {
ModTime() time.Time ModTime() time.Time
} }
// StateOf lets us figure out the state of the collection from the
// previous and current path
func StateOf(prev, curr path.Path) CollectionState {
if curr == nil || len(curr.String()) == 0 {
return DeletedState
}
if prev == nil || len(prev.String()) == 0 {
return NewState
}
if curr.String() != prev.String() {
return MovedState
}
return NotMovedState
}

View File

@ -12,15 +12,15 @@ import (
"github.com/alcionai/corso/src/pkg/path" "github.com/alcionai/corso/src/pkg/path"
) )
type DataCollectionSuite struct { type CollectionSuite struct {
tester.Suite tester.Suite
} }
func TestDataCollectionSuite(t *testing.T) { func TestDataCollectionSuite(t *testing.T) {
suite.Run(t, &DataCollectionSuite{Suite: tester.NewUnitSuite(t)}) suite.Run(t, &CollectionSuite{Suite: tester.NewUnitSuite(t)})
} }
func (suite *DataCollectionSuite) TestStateOf() { func (suite *CollectionSuite) TestStateOf() {
fooP, err := path.Build("t", "u", path.ExchangeService, path.EmailCategory, false, "foo") fooP, err := path.Build("t", "u", path.ExchangeService, path.EmailCategory, false, "foo")
require.NoError(suite.T(), err, clues.ToCore(err)) require.NoError(suite.T(), err, clues.ToCore(err))
barP, err := path.Build("t", "u", path.ExchangeService, path.EmailCategory, false, "bar") barP, err := path.Build("t", "u", path.ExchangeService, path.EmailCategory, false, "bar")

View File

@ -0,0 +1,54 @@
package data
import (
"context"
"github.com/alcionai/clues"
"github.com/alcionai/corso/src/pkg/path"
)
var ErrNotFound = clues.New("not found")
type CollectionState int
const (
NewState = CollectionState(iota)
NotMovedState
MovedState
DeletedState
)
type FetchRestoreCollection struct {
Collection
FetchItemByNamer
}
// NoFetchRestoreCollection is a wrapper for a Collection that returns
// ErrNotFound for all Fetch calls.
type NoFetchRestoreCollection struct {
Collection
FetchItemByNamer
}
func (c NoFetchRestoreCollection) FetchItemByName(context.Context, string) (Item, error) {
return nil, ErrNotFound
}
// StateOf lets us figure out the state of the collection from the
// previous and current path
func StateOf(prev, curr path.Path) CollectionState {
if curr == nil || len(curr.String()) == 0 {
return DeletedState
}
if prev == nil || len(prev.String()) == 0 {
return NewState
}
if curr.String() != prev.String() {
return MovedState
}
return NotMovedState
}

View File

@ -14,30 +14,30 @@ import (
) )
// --------------------------------------------------------------------------- // ---------------------------------------------------------------------------
// stream // Item
// --------------------------------------------------------------------------- // ---------------------------------------------------------------------------
var _ data.Stream = &Stream{} var _ data.Item = &Item{}
type Stream struct { type Item struct {
ID string DeletedFlag bool
Reader io.ReadCloser ItemID string
ReadErr error ItemInfo details.ItemInfo
ItemSize int64 ItemSize int64
ModifiedTime time.Time ModifiedTime time.Time
DeletedFlag bool Reader io.ReadCloser
ItemInfo details.ItemInfo ReadErr error
} }
func (s *Stream) UUID() string { func (s *Item) ID() string {
return s.ID return s.ItemID
} }
func (s Stream) Deleted() bool { func (s Item) Deleted() bool {
return s.DeletedFlag return s.DeletedFlag
} }
func (s *Stream) ToReader() io.ReadCloser { func (s *Item) ToReader() io.ReadCloser {
if s.ReadErr != nil { if s.ReadErr != nil {
return io.NopCloser(errReader{s.ReadErr}) return io.NopCloser(errReader{s.ReadErr})
} }
@ -45,15 +45,15 @@ func (s *Stream) ToReader() io.ReadCloser {
return s.Reader return s.Reader
} }
func (s *Stream) Info() details.ItemInfo { func (s *Item) Info() details.ItemInfo {
return s.ItemInfo return s.ItemInfo
} }
func (s *Stream) Size() int64 { func (s *Item) Size() int64 {
return s.ItemSize return s.ItemSize
} }
func (s *Stream) ModTime() time.Time { func (s *Item) ModTime() time.Time {
return s.ModifiedTime return s.ModifiedTime
} }
@ -77,7 +77,7 @@ var (
type Collection struct{} type Collection struct{}
func (c Collection) Items(ctx context.Context, errs *fault.Bus) <-chan data.Stream { func (c Collection) Items(ctx context.Context, errs *fault.Bus) <-chan data.Item {
return nil return nil
} }
@ -97,6 +97,6 @@ func (c Collection) DoNotMergeItems() bool {
return true return true
} }
func (c Collection) FetchItemByName(ctx context.Context, name string) (data.Stream, error) { func (c Collection) FetchItemByName(ctx context.Context, name string) (data.Item, error) {
return &Stream{}, clues.New("not implemented") return &Item{}, clues.New("not implemented")
} }

View File

@ -15,7 +15,7 @@ import (
var ( var (
_ data.RestoreCollection = &kopiaDataCollection{} _ data.RestoreCollection = &kopiaDataCollection{}
_ data.Stream = &kopiaDataStream{} _ data.Item = &kopiaDataStream{}
) )
type kopiaDataCollection struct { type kopiaDataCollection struct {
@ -29,9 +29,9 @@ type kopiaDataCollection struct {
func (kdc *kopiaDataCollection) Items( func (kdc *kopiaDataCollection) Items(
ctx context.Context, ctx context.Context,
errs *fault.Bus, errs *fault.Bus,
) <-chan data.Stream { ) <-chan data.Item {
var ( var (
res = make(chan data.Stream) res = make(chan data.Item)
el = errs.Local() el = errs.Local()
loadCount = 0 loadCount = 0
) )
@ -72,12 +72,12 @@ func (kdc kopiaDataCollection) FullPath() path.Path {
} }
// Fetch returns the file with the given name from the collection as a // Fetch returns the file with the given name from the collection as a
// data.Stream. Returns a data.ErrNotFound error if the file isn't in the // data.Item. Returns a data.ErrNotFound error if the file isn't in the
// collection. // collection.
func (kdc kopiaDataCollection) FetchItemByName( func (kdc kopiaDataCollection) FetchItemByName(
ctx context.Context, ctx context.Context,
name string, name string,
) (data.Stream, error) { ) (data.Item, error) {
ctx = clues.Add(ctx, "item_name", clues.Hide(name)) ctx = clues.Add(ctx, "item_name", clues.Hide(name))
if kdc.dir == nil { if kdc.dir == nil {
@ -119,7 +119,7 @@ func (kdc kopiaDataCollection) FetchItemByName(
} }
return &kopiaDataStream{ return &kopiaDataStream{
uuid: name, id: name,
reader: &restoreStreamReader{ reader: &restoreStreamReader{
ReadCloser: r, ReadCloser: r,
expectedVersion: kdc.expectedVersion, expectedVersion: kdc.expectedVersion,
@ -130,7 +130,7 @@ func (kdc kopiaDataCollection) FetchItemByName(
type kopiaDataStream struct { type kopiaDataStream struct {
reader io.ReadCloser reader io.ReadCloser
uuid string id string
size int64 size int64
} }
@ -138,8 +138,8 @@ func (kds kopiaDataStream) ToReader() io.ReadCloser {
return kds.reader return kds.reader
} }
func (kds kopiaDataStream) UUID() string { func (kds kopiaDataStream) ID() string {
return kds.uuid return kds.id
} }
func (kds kopiaDataStream) Deleted() bool { func (kds kopiaDataStream) Deleted() bool {

View File

@ -14,7 +14,7 @@ import (
"github.com/stretchr/testify/suite" "github.com/stretchr/testify/suite"
"github.com/alcionai/corso/src/internal/data" "github.com/alcionai/corso/src/internal/data"
exchMock "github.com/alcionai/corso/src/internal/m365/service/exchange/mock" dataMock "github.com/alcionai/corso/src/internal/data/mock"
"github.com/alcionai/corso/src/internal/tester" "github.com/alcionai/corso/src/internal/tester"
"github.com/alcionai/corso/src/pkg/fault" "github.com/alcionai/corso/src/pkg/fault"
"github.com/alcionai/corso/src/pkg/path" "github.com/alcionai/corso/src/pkg/path"
@ -240,25 +240,25 @@ func (suite *KopiaDataCollectionUnitSuite) TestReturnsStreams() {
bus = fault.New(false) bus = fault.New(false)
) )
for returnedStream := range c.Items(ctx, bus) { for item := range c.Items(ctx, bus) {
require.Less(t, len(found), len(test.expectedLoaded), "items read safety") require.Less(t, len(found), len(test.expectedLoaded), "items read safety")
found = append(found, loadedData{}) found = append(found, loadedData{})
f := &found[len(found)-1] f := &found[len(found)-1]
f.uuid = returnedStream.UUID() f.uuid = item.ID()
buf, err := io.ReadAll(returnedStream.ToReader()) buf, err := io.ReadAll(item.ToReader())
if !assert.NoError(t, err, clues.ToCore(err)) { if !assert.NoError(t, err, clues.ToCore(err)) {
continue continue
} }
f.data = buf f.data = buf
if !assert.Implements(t, (*data.StreamSize)(nil), returnedStream) { if !assert.Implements(t, (*data.ItemSize)(nil), item) {
continue continue
} }
ss := returnedStream.(data.StreamSize) ss := item.(data.ItemSize)
f.size = ss.Size() f.size = ss.Size()
} }
@ -289,7 +289,7 @@ func (suite *KopiaDataCollectionUnitSuite) TestFetchItemByName() {
errFileName2 = "error2" errFileName2 = "error2"
noErrFileData = "foo bar baz" noErrFileData = "foo bar baz"
errReader = &exchMock.Data{ errReader = &dataMock.Item{
ReadErr: assert.AnError, ReadErr: assert.AnError,
} }
) )

View File

@ -64,8 +64,8 @@ func (mc mergeCollection) FullPath() path.Path {
func (mc *mergeCollection) Items( func (mc *mergeCollection) Items(
ctx context.Context, ctx context.Context,
errs *fault.Bus, errs *fault.Bus,
) <-chan data.Stream { ) <-chan data.Item {
res := make(chan data.Stream) res := make(chan data.Item)
go func() { go func() {
defer close(res) defer close(res)
@ -98,7 +98,7 @@ func (mc *mergeCollection) Items(
func (mc *mergeCollection) FetchItemByName( func (mc *mergeCollection) FetchItemByName(
ctx context.Context, ctx context.Context,
name string, name string,
) (data.Stream, error) { ) (data.Item, error) {
logger.Ctx(ctx).Infow( logger.Ctx(ctx).Infow(
"fetching item in merged collection", "fetching item in merged collection",
"merged_collection_count", len(mc.cols)) "merged_collection_count", len(mc.cols))

View File

@ -90,7 +90,7 @@ func (suite *MergeCollectionUnitSuite) TestItems() {
gotItemNames := []string{} gotItemNames := []string{}
for item := range dc.Items(ctx, fault.New(true)) { for item := range dc.Items(ctx, fault.New(true)) {
gotItemNames = append(gotItemNames, item.UUID()) gotItemNames = append(gotItemNames, item.ID())
} }
assert.ElementsMatch(t, expectedItemNames, gotItemNames) assert.ElementsMatch(t, expectedItemNames, gotItemNames)

View File

@ -196,14 +196,16 @@ func (cp *corsoProgress) FinishedFile(relativePath string, err error) {
return return
} }
ctx := clues.Add(
cp.ctx,
"service", d.repoPath.Service().String(),
"category", d.repoPath.Category().String())
// These items were sourced from a base snapshot or were cached in kopia so we // These items were sourced from a base snapshot or were cached in kopia so we
// never had to materialize their details in-memory. // never had to materialize their details in-memory.
if d.info == nil || d.cached { if d.info == nil || d.cached {
if d.prevPath == nil { if d.prevPath == nil {
cp.errs.AddRecoverable(cp.ctx, clues.New("item sourced from previous backup with no previous path"). cp.errs.AddRecoverable(ctx, clues.New("finished file sourced from previous backup with no previous path").
With(
"service", d.repoPath.Service().String(),
"category", d.repoPath.Category().String()).
Label(fault.LabelForceNoBackupCreation)) Label(fault.LabelForceNoBackupCreation))
return return
@ -218,10 +220,7 @@ func (cp *corsoProgress) FinishedFile(relativePath string, err error) {
d.repoPath, d.repoPath,
d.locationPath) d.locationPath)
if err != nil { if err != nil {
cp.errs.AddRecoverable(cp.ctx, clues.Wrap(err, "adding item to merge list"). cp.errs.AddRecoverable(ctx, clues.Wrap(err, "adding finished file to merge list").
With(
"service", d.repoPath.Service().String(),
"category", d.repoPath.Category().String()).
Label(fault.LabelForceNoBackupCreation)) Label(fault.LabelForceNoBackupCreation))
} }
@ -233,10 +232,7 @@ func (cp *corsoProgress) FinishedFile(relativePath string, err error) {
d.locationPath, d.locationPath,
*d.info) *d.info)
if err != nil { if err != nil {
cp.errs.AddRecoverable(cp.ctx, clues.New("adding item to details"). cp.errs.AddRecoverable(ctx, clues.Wrap(err, "adding finished file to details").
With(
"service", d.repoPath.Service().String(),
"category", d.repoPath.Category().String()).
Label(fault.LabelForceNoBackupCreation)) Label(fault.LabelForceNoBackupCreation))
return return
@ -348,7 +344,7 @@ func collectionEntries(
return seen, nil return seen, nil
} }
encodedName := encodeAsPath(e.UUID()) encodedName := encodeAsPath(e.ID())
// Even if this item has been deleted and should not appear at all in // Even if this item has been deleted and should not appear at all in
// the new snapshot we need to record that we've seen it here so we know // the new snapshot we need to record that we've seen it here so we know
@ -366,7 +362,7 @@ func collectionEntries(
seen[encodedName] = struct{}{} seen[encodedName] = struct{}{}
// For now assuming that item IDs don't need escaping. // For now assuming that item IDs don't need escaping.
itemPath, err := streamedEnts.FullPath().AppendItem(e.UUID()) itemPath, err := streamedEnts.FullPath().AppendItem(e.ID())
if err != nil { if err != nil {
err = clues.Wrap(err, "getting full item path") err = clues.Wrap(err, "getting full item path")
progress.errs.AddRecoverable(ctx, err) progress.errs.AddRecoverable(ctx, err)
@ -383,7 +379,7 @@ func collectionEntries(
} }
modTime := time.Now() modTime := time.Now()
if smt, ok := e.(data.StreamModTime); ok { if smt, ok := e.(data.ItemModTime); ok {
modTime = smt.ModTime() modTime = smt.ModTime()
} }
@ -392,7 +388,7 @@ func collectionEntries(
// used for restore. If progress does not contain information about a // used for restore. If progress does not contain information about a
// finished file it just returns without an error so it's safe to skip // finished file it just returns without an error so it's safe to skip
// adding something to it. // adding something to it.
ei, ok := e.(data.StreamInfo) ei, ok := e.(data.ItemInfo)
if ok { if ok {
// Relative path given to us in the callback is missing the root // Relative path given to us in the callback is missing the root
// element. Add to pending set before calling the callback to avoid race // element. Add to pending set before calling the callback to avoid race

View File

@ -24,7 +24,7 @@ import (
pmMock "github.com/alcionai/corso/src/internal/common/prefixmatcher/mock" pmMock "github.com/alcionai/corso/src/internal/common/prefixmatcher/mock"
"github.com/alcionai/corso/src/internal/common/ptr" "github.com/alcionai/corso/src/internal/common/ptr"
"github.com/alcionai/corso/src/internal/data" "github.com/alcionai/corso/src/internal/data"
"github.com/alcionai/corso/src/internal/data/mock" dataMock "github.com/alcionai/corso/src/internal/data/mock"
"github.com/alcionai/corso/src/internal/m365/collection/drive/metadata" "github.com/alcionai/corso/src/internal/m365/collection/drive/metadata"
exchMock "github.com/alcionai/corso/src/internal/m365/service/exchange/mock" exchMock "github.com/alcionai/corso/src/internal/m365/service/exchange/mock"
"github.com/alcionai/corso/src/internal/tester" "github.com/alcionai/corso/src/internal/tester"
@ -79,7 +79,7 @@ func testForFiles(
for s := range c.Items(ctx, fault.New(true)) { for s := range c.Items(ctx, fault.New(true)) {
count++ count++
fullPath, err := c.FullPath().AppendItem(s.UUID()) fullPath, err := c.FullPath().AppendItem(s.ID())
require.NoError(t, err, clues.ToCore(err)) require.NoError(t, err, clues.ToCore(err))
expected, ok := expected[fullPath.String()] expected, ok := expected[fullPath.String()]
@ -89,9 +89,9 @@ func testForFiles(
require.NoError(t, err, "reading collection item", fullPath, clues.ToCore(err)) require.NoError(t, err, "reading collection item", fullPath, clues.ToCore(err))
assert.Equal(t, expected, buf, "comparing collection item", fullPath) assert.Equal(t, expected, buf, "comparing collection item", fullPath)
require.Implements(t, (*data.StreamSize)(nil), s) require.Implements(t, (*data.ItemSize)(nil), s)
ss := s.(data.StreamSize) ss := s.(data.ItemSize)
assert.Equal(t, len(buf), int(ss.Size())) assert.Equal(t, len(buf), int(ss.Size()))
} }
} }
@ -1108,7 +1108,7 @@ func (suite *KopiaIntegrationSuite) TestBackupCollections_NoDetailsForMeta() {
numDeetsEntries: 3, numDeetsEntries: 3,
hasMetaDeets: true, hasMetaDeets: true,
cols: func() []data.BackupCollection { cols: func() []data.BackupCollection {
streams := []data.Stream{} streams := []data.Item{}
fileNames := []string{ fileNames := []string{
testFileName, testFileName,
testFileName + metadata.MetaFileSuffix, testFileName + metadata.MetaFileSuffix,
@ -1119,8 +1119,8 @@ func (suite *KopiaIntegrationSuite) TestBackupCollections_NoDetailsForMeta() {
info := baseOneDriveItemInfo info := baseOneDriveItemInfo
info.ItemName = name info.ItemName = name
ms := &mock.Stream{ ms := &dataMock.Item{
ID: name, ItemID: name,
Reader: io.NopCloser(&bytes.Buffer{}), Reader: io.NopCloser(&bytes.Buffer{}),
ItemSize: 0, ItemSize: 0,
ItemInfo: details.ItemInfo{OneDrive: &info}, ItemInfo: details.ItemInfo{OneDrive: &info},
@ -1149,8 +1149,8 @@ func (suite *KopiaIntegrationSuite) TestBackupCollections_NoDetailsForMeta() {
info := baseOneDriveItemInfo info := baseOneDriveItemInfo
info.ItemName = testFileName info.ItemName = testFileName
ms := &mock.Stream{ ms := &dataMock.Item{
ID: testFileName, ItemID: testFileName,
Reader: io.NopCloser(&bytes.Buffer{}), Reader: io.NopCloser(&bytes.Buffer{}),
ItemSize: 0, ItemSize: 0,
ItemInfo: details.ItemInfo{OneDrive: &info}, ItemInfo: details.ItemInfo{OneDrive: &info},
@ -1159,7 +1159,7 @@ func (suite *KopiaIntegrationSuite) TestBackupCollections_NoDetailsForMeta() {
mc := &mockBackupCollection{ mc := &mockBackupCollection{
path: storePath, path: storePath,
loc: locPath, loc: locPath,
streams: []data.Stream{ms}, streams: []data.Item{ms},
state: data.NotMovedState, state: data.NotMovedState,
} }
@ -1298,12 +1298,12 @@ func (suite *KopiaIntegrationSuite) TestRestoreAfterCompressionChange() {
type mockBackupCollection struct { type mockBackupCollection struct {
path path.Path path path.Path
loc *path.Builder loc *path.Builder
streams []data.Stream streams []data.Item
state data.CollectionState state data.CollectionState
} }
func (c *mockBackupCollection) Items(context.Context, *fault.Bus) <-chan data.Stream { func (c *mockBackupCollection) Items(context.Context, *fault.Bus) <-chan data.Item {
res := make(chan data.Stream) res := make(chan data.Item)
go func() { go func() {
defer close(res) defer close(res)
@ -1347,36 +1347,42 @@ func (suite *KopiaIntegrationSuite) TestBackupCollections_ReaderError() {
&mockBackupCollection{ &mockBackupCollection{
path: suite.storePath1, path: suite.storePath1,
loc: loc1, loc: loc1,
streams: []data.Stream{ streams: []data.Item{
&exchMock.Data{ &dataMock.Item{
ID: testFileName, ItemID: testFileName,
Reader: io.NopCloser(bytes.NewReader(testFileData)), Reader: io.NopCloser(bytes.NewReader(testFileData)),
ItemInfo: exchMock.StubMailInfo(),
}, },
&exchMock.Data{ &dataMock.Item{
ID: testFileName2, ItemID: testFileName2,
Reader: io.NopCloser(bytes.NewReader(testFileData2)), Reader: io.NopCloser(bytes.NewReader(testFileData2)),
ItemInfo: exchMock.StubMailInfo(),
}, },
}, },
}, },
&mockBackupCollection{ &mockBackupCollection{
path: suite.storePath2, path: suite.storePath2,
loc: loc2, loc: loc2,
streams: []data.Stream{ streams: []data.Item{
&exchMock.Data{ &dataMock.Item{
ID: testFileName3, ItemID: testFileName3,
Reader: io.NopCloser(bytes.NewReader(testFileData3)), Reader: io.NopCloser(bytes.NewReader(testFileData3)),
ItemInfo: exchMock.StubMailInfo(),
}, },
&exchMock.Data{ &dataMock.Item{
ID: testFileName4, ItemID: testFileName4,
ReadErr: assert.AnError, ReadErr: assert.AnError,
ItemInfo: exchMock.StubMailInfo(),
}, },
&exchMock.Data{ &dataMock.Item{
ID: testFileName5, ItemID: testFileName5,
Reader: io.NopCloser(bytes.NewReader(testFileData5)), Reader: io.NopCloser(bytes.NewReader(testFileData5)),
ItemInfo: exchMock.StubMailInfo(),
}, },
&exchMock.Data{ &dataMock.Item{
ID: testFileName6, ItemID: testFileName6,
Reader: io.NopCloser(bytes.NewReader(testFileData6)), Reader: io.NopCloser(bytes.NewReader(testFileData6)),
ItemInfo: exchMock.StubMailInfo(),
}, },
}, },
}, },
@ -1603,9 +1609,10 @@ func (suite *KopiaSimpleRepoIntegrationSuite) SetupTest() {
for _, item := range suite.files[parent.String()] { for _, item := range suite.files[parent.String()] {
collection.streams = append( collection.streams = append(
collection.streams, collection.streams,
&exchMock.Data{ &dataMock.Item{
ID: item.itemPath.Item(), ItemID: item.itemPath.Item(),
Reader: io.NopCloser(bytes.NewReader(item.data)), Reader: io.NopCloser(bytes.NewReader(item.data)),
ItemInfo: exchMock.StubMailInfo(),
}, },
) )
} }

View File

@ -457,7 +457,7 @@ func (suite *SPCollectionIntgSuite) TestCreateSharePointCollection_Lists() {
t.Logf("Path: %s\n", collection.FullPath().String()) t.Logf("Path: %s\n", collection.FullPath().String())
for item := range collection.Items(ctx, fault.New(true)) { for item := range collection.Items(ctx, fault.New(true)) {
t.Log("File: " + item.UUID()) t.Log("File: " + item.ID())
bs, err := io.ReadAll(item.ToReader()) bs, err := io.ReadAll(item.ToReader())
require.NoError(t, err, clues.ToCore(err)) require.NoError(t, err, clues.ToCore(err))

View File

@ -35,11 +35,11 @@ const (
var ( var (
_ data.BackupCollection = &Collection{} _ data.BackupCollection = &Collection{}
_ data.Stream = &Item{} _ data.Item = &Item{}
_ data.StreamInfo = &Item{} _ data.ItemInfo = &Item{}
_ data.StreamModTime = &Item{} _ data.ItemModTime = &Item{}
_ data.Stream = &metadata.Item{} _ data.Item = &metadata.Item{}
_ data.StreamModTime = &metadata.Item{} _ data.ItemModTime = &metadata.Item{}
) )
// Collection represents a set of OneDrive objects retrieved from M365 // Collection represents a set of OneDrive objects retrieved from M365
@ -47,7 +47,7 @@ type Collection struct {
handler BackupHandler handler BackupHandler
// data is used to share data streams with the collection consumer // data is used to share data streams with the collection consumer
data chan data.Stream data chan data.Item
// folderPath indicates what level in the hierarchy this collection // folderPath indicates what level in the hierarchy this collection
// represents // represents
folderPath path.Path folderPath path.Path
@ -162,7 +162,7 @@ func newColl(
prevPath: prevPath, prevPath: prevPath,
driveItems: map[string]models.DriveItemable{}, driveItems: map[string]models.DriveItemable{},
driveID: driveID, driveID: driveID,
data: make(chan data.Stream, graph.Parallelism(path.OneDriveMetadataService).CollectionBufferSize()), data: make(chan data.Item, graph.Parallelism(path.OneDriveMetadataService).CollectionBufferSize()),
statusUpdater: statusUpdater, statusUpdater: statusUpdater,
ctrl: ctrlOpts, ctrl: ctrlOpts,
state: data.StateOf(prevPath, currPath), state: data.StateOf(prevPath, currPath),
@ -207,7 +207,7 @@ func (oc *Collection) IsEmpty() bool {
func (oc *Collection) Items( func (oc *Collection) Items(
ctx context.Context, ctx context.Context,
errs *fault.Bus, errs *fault.Bus,
) <-chan data.Stream { ) <-chan data.Item {
go oc.populateItems(ctx, errs) go oc.populateItems(ctx, errs)
return oc.data return oc.data
} }
@ -256,7 +256,7 @@ type Item struct {
// as deleted by adding them to the exclude list so this can always return // as deleted by adding them to the exclude list so this can always return
// false. // false.
func (i Item) Deleted() bool { return false } func (i Item) Deleted() bool { return false }
func (i *Item) UUID() string { return i.id } func (i *Item) ID() string { return i.id }
func (i *Item) ToReader() io.ReadCloser { return i.data } func (i *Item) ToReader() io.ReadCloser { return i.data }
func (i *Item) Info() details.ItemInfo { return i.info } func (i *Item) Info() details.ItemInfo { return i.info }
func (i *Item) ModTime() time.Time { return i.info.Modified() } func (i *Item) ModTime() time.Time { return i.info.Modified() }
@ -582,8 +582,8 @@ func (oc *Collection) populateDriveItem(
}) })
oc.data <- &metadata.Item{ oc.data <- &metadata.Item{
ID: metaFileName + metaSuffix, ItemID: metaFileName + metaSuffix,
Data: metaReader, Data: metaReader,
// Metadata file should always use the latest time as // Metadata file should always use the latest time as
// permissions change does not update mod time. // permissions change does not update mod time.
Mod: time.Now(), Mod: time.Now(),

View File

@ -174,7 +174,7 @@ func (suite *CollectionUnitTestSuite) TestCollection() {
var ( var (
wg = sync.WaitGroup{} wg = sync.WaitGroup{}
collStatus = support.ControllerOperationStatus{} collStatus = support.ControllerOperationStatus{}
readItems = []data.Stream{} readItems = []data.Item{}
) )
pb := path.Builder{}.Append(path.Split("drive/driveID1/root:/dir1/dir2/dir3")...) pb := path.Builder{}.Append(path.Split("drive/driveID1/root:/dir1/dir2/dir3")...)
@ -250,10 +250,10 @@ func (suite *CollectionUnitTestSuite) TestCollection() {
// Validate item info and data // Validate item info and data
readItem := readItems[0] readItem := readItems[0]
assert.Equal(t, stubItemID+metadata.DataFileSuffix, readItem.UUID()) assert.Equal(t, stubItemID+metadata.DataFileSuffix, readItem.ID())
require.Implements(t, (*data.StreamModTime)(nil), readItem) require.Implements(t, (*data.ItemModTime)(nil), readItem)
mt := readItem.(data.StreamModTime) mt := readItem.(data.ItemModTime)
assert.Equal(t, now, mt.ModTime()) assert.Equal(t, now, mt.ModTime())
readData, err := io.ReadAll(readItem.ToReader()) readData, err := io.ReadAll(readItem.ToReader())
@ -270,7 +270,7 @@ func (suite *CollectionUnitTestSuite) TestCollection() {
assert.Equal(t, stubItemContent, readData) assert.Equal(t, stubItemContent, readData)
readItemMeta := readItems[1] readItemMeta := readItems[1]
assert.Equal(t, stubItemID+metadata.MetaFileSuffix, readItemMeta.UUID()) assert.Equal(t, stubItemID+metadata.MetaFileSuffix, readItemMeta.ID())
readMeta := metadata.Metadata{} readMeta := metadata.Metadata{}
err = json.NewDecoder(readItemMeta.ToReader()).Decode(&readMeta) err = json.NewDecoder(readItemMeta.ToReader()).Decode(&readMeta)
@ -472,7 +472,7 @@ func (suite *CollectionUnitTestSuite) TestCollectionPermissionBackupLatestModTim
coll.handler = mbh coll.handler = mbh
readItems := []data.Stream{} readItems := []data.Item{}
for item := range coll.Items(ctx, fault.New(true)) { for item := range coll.Items(ctx, fault.New(true)) {
readItems = append(readItems, item) readItems = append(readItems, item)
} }
@ -484,12 +484,12 @@ func (suite *CollectionUnitTestSuite) TestCollectionPermissionBackupLatestModTim
require.Equal(t, 1, collStatus.Metrics.Successes) require.Equal(t, 1, collStatus.Metrics.Successes)
for _, i := range readItems { for _, i := range readItems {
if strings.HasSuffix(i.UUID(), metadata.MetaFileSuffix) { if strings.HasSuffix(i.ID(), metadata.MetaFileSuffix) {
content, err := io.ReadAll(i.ToReader()) content, err := io.ReadAll(i.ToReader())
require.NoError(t, err, clues.ToCore(err)) require.NoError(t, err, clues.ToCore(err))
require.Equal(t, `{"filename":"Fake Item","permissionMode":1}`, string(content)) require.Equal(t, `{"filename":"Fake Item","permissionMode":1}`, string(content))
im, ok := i.(data.StreamModTime) im, ok := i.(data.ItemModTime)
require.Equal(t, ok, true, "modtime interface") require.Equal(t, ok, true, "modtime interface")
require.Greater(t, im.ModTime(), mtime, "permissions time greater than mod time") require.Greater(t, im.ModTime(), mtime, "permissions time greater than mod time")
} }
@ -978,7 +978,7 @@ func (suite *CollectionUnitTestSuite) TestItemExtensions() {
wg.Wait() wg.Wait()
ei, ok := collItem.(data.StreamInfo) ei, ok := collItem.(data.ItemInfo)
assert.True(t, ok) assert.True(t, ok)
itemInfo := ei.Info() itemInfo := ei.Info()

View File

@ -116,10 +116,10 @@ func deserializeMetadata(
var ( var (
err error err error
ictx = clues.Add(ctx, "item_uuid", item.UUID()) ictx = clues.Add(ctx, "item_uuid", item.ID())
) )
switch item.UUID() { switch item.ID() {
case graph.PreviousPathFileName: case graph.PreviousPathFileName:
err = deserializeMap(item.ToReader(), prevFolders) err = deserializeMap(item.ToReader(), prevFolders)
@ -129,7 +129,7 @@ func deserializeMetadata(
default: default:
logger.Ctx(ictx).Infow( logger.Ctx(ictx).Infow(
"skipping unknown metadata file", "skipping unknown metadata file",
"file_name", item.UUID()) "file_name", item.ID())
continue continue
} }

View File

@ -1153,16 +1153,16 @@ func (suite *OneDriveCollectionsUnitSuite) TestDeserializeMetadata() {
type failingColl struct{} type failingColl struct{}
func (f failingColl) Items(ctx context.Context, errs *fault.Bus) <-chan data.Stream { func (f failingColl) Items(ctx context.Context, errs *fault.Bus) <-chan data.Item {
ic := make(chan data.Stream) ic := make(chan data.Item)
defer close(ic) defer close(ic)
errs.AddRecoverable(ctx, assert.AnError) errs.AddRecoverable(ctx, assert.AnError)
return ic return ic
} }
func (f failingColl) FullPath() path.Path { return nil } func (f failingColl) FullPath() path.Path { return nil }
func (f failingColl) FetchItemByName(context.Context, string) (data.Stream, error) { return nil, nil } func (f failingColl) FetchItemByName(context.Context, string) (data.Item, error) { return nil, nil }
// This check is to ensure that we don't error out, but still return // This check is to ensure that we don't error out, but still return
// canUsePreviousBackup as false on read errors // canUsePreviousBackup as false on read errors

View File

@ -43,15 +43,15 @@ type Metadata struct {
} }
type Item struct { type Item struct {
ID string ItemID string
Data io.ReadCloser Data io.ReadCloser
Mod time.Time Mod time.Time
} }
// Deleted implements an interface function. However, OneDrive items are marked // Deleted implements an interface function. However, OneDrive items are marked
// as deleted by adding them to the exclude list so this can always return // as deleted by adding them to the exclude list so this can always return
// false. // false.
func (i *Item) Deleted() bool { return false } func (i *Item) Deleted() bool { return false }
func (i *Item) UUID() string { return i.ID } func (i *Item) ID() string { return i.ItemID }
func (i *Item) ToReader() io.ReadCloser { return i.Data } func (i *Item) ToReader() io.ReadCloser { return i.Data }
func (i *Item) ModTime() time.Time { return i.Mod } func (i *Item) ModTime() time.Time { return i.Mod }

View File

@ -184,7 +184,7 @@ func RestoreCollection(
wg.Add(1) wg.Add(1)
semaphoreCh <- struct{}{} semaphoreCh <- struct{}{}
go func(ctx context.Context, itemData data.Stream) { go func(ctx context.Context, itemData data.Item) {
defer wg.Done() defer wg.Done()
defer func() { <-semaphoreCh }() defer func() { <-semaphoreCh }()
@ -192,9 +192,9 @@ func RestoreCollection(
defer caches.pool.Put(copyBufferPtr) defer caches.pool.Put(copyBufferPtr)
copyBuffer := *copyBufferPtr copyBuffer := *copyBufferPtr
ictx := clues.Add(ctx, "restore_item_id", itemData.UUID()) ictx := clues.Add(ctx, "restore_item_id", itemData.ID())
itemPath, err := dc.FullPath().AppendItem(itemData.UUID()) itemPath, err := dc.FullPath().AppendItem(itemData.ID())
if err != nil { if err != nil {
el.AddRecoverable(ctx, clues.Wrap(err, "appending item to full path").WithClues(ictx)) el.AddRecoverable(ctx, clues.Wrap(err, "appending item to full path").WithClues(ictx))
return return
@ -257,11 +257,11 @@ func restoreItem(
restoreFolderID string, restoreFolderID string,
copyBuffer []byte, copyBuffer []byte,
caches *restoreCaches, caches *restoreCaches,
itemData data.Stream, itemData data.Item,
itemPath path.Path, itemPath path.Path,
ctr *count.Bus, ctr *count.Bus,
) (details.ItemInfo, bool, error) { ) (details.ItemInfo, bool, error) {
itemUUID := itemData.UUID() itemUUID := itemData.ID()
ctx = clues.Add(ctx, "item_id", itemUUID) ctx = clues.Add(ctx, "item_id", itemUUID)
if rcc.BackupVersion < version.OneDrive1DataAndMetaFiles { if rcc.BackupVersion < version.OneDrive1DataAndMetaFiles {
@ -378,7 +378,7 @@ func restoreV0File(
restoreFolderID string, restoreFolderID string,
copyBuffer []byte, copyBuffer []byte,
collisionKeyToItemID map[string]api.DriveItemIDType, collisionKeyToItemID map[string]api.DriveItemIDType,
itemData data.Stream, itemData data.Item,
ctr *count.Bus, ctr *count.Bus,
) (details.ItemInfo, error) { ) (details.ItemInfo, error) {
_, itemInfo, err := restoreFile( _, itemInfo, err := restoreFile(
@ -386,7 +386,7 @@ func restoreV0File(
restoreCfg, restoreCfg,
rh, rh,
fibn, fibn,
itemData.UUID(), itemData.ID(),
itemData, itemData,
drivePath.DriveID, drivePath.DriveID,
restoreFolderID, restoreFolderID,
@ -410,10 +410,10 @@ func restoreV1File(
copyBuffer []byte, copyBuffer []byte,
caches *restoreCaches, caches *restoreCaches,
itemPath path.Path, itemPath path.Path,
itemData data.Stream, itemData data.Item,
ctr *count.Bus, ctr *count.Bus,
) (details.ItemInfo, error) { ) (details.ItemInfo, error) {
trimmedName := strings.TrimSuffix(itemData.UUID(), metadata.DataFileSuffix) trimmedName := strings.TrimSuffix(itemData.ID(), metadata.DataFileSuffix)
itemID, itemInfo, err := restoreFile( itemID, itemInfo, err := restoreFile(
ctx, ctx,
@ -470,10 +470,10 @@ func restoreV6File(
copyBuffer []byte, copyBuffer []byte,
caches *restoreCaches, caches *restoreCaches,
itemPath path.Path, itemPath path.Path,
itemData data.Stream, itemData data.Item,
ctr *count.Bus, ctr *count.Bus,
) (details.ItemInfo, error) { ) (details.ItemInfo, error) {
trimmedName := strings.TrimSuffix(itemData.UUID(), metadata.DataFileSuffix) trimmedName := strings.TrimSuffix(itemData.ID(), metadata.DataFileSuffix)
// Get metadata file so we can determine the file name. // Get metadata file so we can determine the file name.
metaName := trimmedName + metadata.MetaFileSuffix metaName := trimmedName + metadata.MetaFileSuffix
@ -701,26 +701,26 @@ type itemRestorer interface {
PostItemInContainerer PostItemInContainerer
} }
// restoreFile will create a new item in the specified `parentFolderID` and upload the data.Stream // restoreFile will create a new item in the specified `parentFolderID` and upload the data.Item
func restoreFile( func restoreFile(
ctx context.Context, ctx context.Context,
restoreCfg control.RestoreConfig, restoreCfg control.RestoreConfig,
ir itemRestorer, ir itemRestorer,
fibn data.FetchItemByNamer, fibn data.FetchItemByNamer,
name string, name string,
itemData data.Stream, itemData data.Item,
driveID, parentFolderID string, driveID, parentFolderID string,
collisionKeyToItemID map[string]api.DriveItemIDType, collisionKeyToItemID map[string]api.DriveItemIDType,
copyBuffer []byte, copyBuffer []byte,
ctr *count.Bus, ctr *count.Bus,
) (string, details.ItemInfo, error) { ) (string, details.ItemInfo, error) {
ctx, end := diagnostics.Span(ctx, "gc:oneDrive:restoreItem", diagnostics.Label("item_uuid", itemData.UUID())) ctx, end := diagnostics.Span(ctx, "gc:oneDrive:restoreItem", diagnostics.Label("item_uuid", itemData.ID()))
defer end() defer end()
trace.Log(ctx, "gc:oneDrive:restoreItem", itemData.UUID()) trace.Log(ctx, "gc:oneDrive:restoreItem", itemData.ID())
// Get the stream size (needed to create the upload session) // Get the stream size (needed to create the upload session)
ss, ok := itemData.(data.StreamSize) ss, ok := itemData.(data.ItemSize)
if !ok { if !ok {
return "", details.ItemInfo{}, clues.New("item does not implement DataStreamInfo").WithClues(ctx) return "", details.ItemInfo{}, clues.New("item does not implement DataStreamInfo").WithClues(ctx)
} }
@ -808,7 +808,7 @@ func restoreFile(
// If it is not the first try, we have to pull the file // If it is not the first try, we have to pull the file
// again from kopia. Ideally we could just seek the stream // again from kopia. Ideally we could just seek the stream
// but we don't have a Seeker available here. // but we don't have a Seeker available here.
itemData, err := fibn.FetchItemByName(ctx, itemData.UUID()) itemData, err := fibn.FetchItemByName(ctx, itemData.ID())
if err != nil { if err != nil {
return "", details.ItemInfo{}, clues.Wrap(err, "get data file") return "", details.ItemInfo{}, clues.Wrap(err, "get data file")
} }

View File

@ -13,9 +13,11 @@ import (
"github.com/alcionai/corso/src/internal/common/idname" "github.com/alcionai/corso/src/internal/common/idname"
"github.com/alcionai/corso/src/internal/common/ptr" "github.com/alcionai/corso/src/internal/common/ptr"
dataMock "github.com/alcionai/corso/src/internal/data/mock"
"github.com/alcionai/corso/src/internal/m365/graph" "github.com/alcionai/corso/src/internal/m365/graph"
odConsts "github.com/alcionai/corso/src/internal/m365/service/onedrive/consts" odConsts "github.com/alcionai/corso/src/internal/m365/service/onedrive/consts"
"github.com/alcionai/corso/src/internal/m365/service/onedrive/mock" odMock "github.com/alcionai/corso/src/internal/m365/service/onedrive/mock"
odStub "github.com/alcionai/corso/src/internal/m365/service/onedrive/stub"
"github.com/alcionai/corso/src/internal/operations/inject" "github.com/alcionai/corso/src/internal/operations/inject"
"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"
@ -49,7 +51,7 @@ func (suite *RestoreUnitSuite) TestRestoreItem_collisionHandling() {
onCollision control.CollisionPolicy onCollision control.CollisionPolicy
deleteErr error deleteErr error
expectSkipped assert.BoolAssertionFunc expectSkipped assert.BoolAssertionFunc
expectMock func(*testing.T, *mock.RestoreHandler) expectMock func(*testing.T, *odMock.RestoreHandler)
expectCounts counts expectCounts counts
}{ }{
{ {
@ -57,7 +59,7 @@ func (suite *RestoreUnitSuite) TestRestoreItem_collisionHandling() {
collisionKeys: map[string]api.DriveItemIDType{}, collisionKeys: map[string]api.DriveItemIDType{},
onCollision: control.Copy, onCollision: control.Copy,
expectSkipped: assert.False, expectSkipped: assert.False,
expectMock: func(t *testing.T, rh *mock.RestoreHandler) { expectMock: func(t *testing.T, rh *odMock.RestoreHandler) {
assert.True(t, rh.CalledPostItem, "new item posted") assert.True(t, rh.CalledPostItem, "new item posted")
assert.False(t, rh.CalledDeleteItem, "new item deleted") assert.False(t, rh.CalledDeleteItem, "new item deleted")
}, },
@ -68,7 +70,7 @@ func (suite *RestoreUnitSuite) TestRestoreItem_collisionHandling() {
collisionKeys: map[string]api.DriveItemIDType{}, collisionKeys: map[string]api.DriveItemIDType{},
onCollision: control.Replace, onCollision: control.Replace,
expectSkipped: assert.False, expectSkipped: assert.False,
expectMock: func(t *testing.T, rh *mock.RestoreHandler) { expectMock: func(t *testing.T, rh *odMock.RestoreHandler) {
assert.True(t, rh.CalledPostItem, "new item posted") assert.True(t, rh.CalledPostItem, "new item posted")
assert.False(t, rh.CalledDeleteItem, "new item deleted") assert.False(t, rh.CalledDeleteItem, "new item deleted")
}, },
@ -79,7 +81,7 @@ func (suite *RestoreUnitSuite) TestRestoreItem_collisionHandling() {
collisionKeys: map[string]api.DriveItemIDType{}, collisionKeys: map[string]api.DriveItemIDType{},
onCollision: control.Skip, onCollision: control.Skip,
expectSkipped: assert.False, expectSkipped: assert.False,
expectMock: func(t *testing.T, rh *mock.RestoreHandler) { expectMock: func(t *testing.T, rh *odMock.RestoreHandler) {
assert.True(t, rh.CalledPostItem, "new item posted") assert.True(t, rh.CalledPostItem, "new item posted")
assert.False(t, rh.CalledDeleteItem, "new item deleted") assert.False(t, rh.CalledDeleteItem, "new item deleted")
}, },
@ -88,11 +90,11 @@ func (suite *RestoreUnitSuite) TestRestoreItem_collisionHandling() {
{ {
name: "collision, copy", name: "collision, copy",
collisionKeys: map[string]api.DriveItemIDType{ collisionKeys: map[string]api.DriveItemIDType{
mock.DriveItemFileName: {ItemID: mndiID}, odMock.DriveItemFileName: {ItemID: mndiID},
}, },
onCollision: control.Copy, onCollision: control.Copy,
expectSkipped: assert.False, expectSkipped: assert.False,
expectMock: func(t *testing.T, rh *mock.RestoreHandler) { expectMock: func(t *testing.T, rh *odMock.RestoreHandler) {
assert.True(t, rh.CalledPostItem, "new item posted") assert.True(t, rh.CalledPostItem, "new item posted")
assert.False(t, rh.CalledDeleteItem, "new item deleted") assert.False(t, rh.CalledDeleteItem, "new item deleted")
}, },
@ -101,11 +103,11 @@ func (suite *RestoreUnitSuite) TestRestoreItem_collisionHandling() {
{ {
name: "collision, replace", name: "collision, replace",
collisionKeys: map[string]api.DriveItemIDType{ collisionKeys: map[string]api.DriveItemIDType{
mock.DriveItemFileName: {ItemID: mndiID}, odMock.DriveItemFileName: {ItemID: mndiID},
}, },
onCollision: control.Replace, onCollision: control.Replace,
expectSkipped: assert.False, expectSkipped: assert.False,
expectMock: func(t *testing.T, rh *mock.RestoreHandler) { expectMock: func(t *testing.T, rh *odMock.RestoreHandler) {
assert.True(t, rh.CalledPostItem, "new item posted") assert.True(t, rh.CalledPostItem, "new item posted")
assert.True(t, rh.CalledDeleteItem, "new item deleted") assert.True(t, rh.CalledDeleteItem, "new item deleted")
assert.Equal(t, mndiID, rh.CalledDeleteItemOn, "deleted the correct item") assert.Equal(t, mndiID, rh.CalledDeleteItemOn, "deleted the correct item")
@ -115,12 +117,12 @@ func (suite *RestoreUnitSuite) TestRestoreItem_collisionHandling() {
{ {
name: "collision, replace - err already deleted", name: "collision, replace - err already deleted",
collisionKeys: map[string]api.DriveItemIDType{ collisionKeys: map[string]api.DriveItemIDType{
mock.DriveItemFileName: {ItemID: "smarf"}, odMock.DriveItemFileName: {ItemID: "smarf"},
}, },
onCollision: control.Replace, onCollision: control.Replace,
deleteErr: graph.ErrDeletedInFlight, deleteErr: graph.ErrDeletedInFlight,
expectSkipped: assert.False, expectSkipped: assert.False,
expectMock: func(t *testing.T, rh *mock.RestoreHandler) { expectMock: func(t *testing.T, rh *odMock.RestoreHandler) {
assert.True(t, rh.CalledPostItem, "new item posted") assert.True(t, rh.CalledPostItem, "new item posted")
assert.True(t, rh.CalledDeleteItem, "new item deleted") assert.True(t, rh.CalledDeleteItem, "new item deleted")
}, },
@ -129,11 +131,11 @@ func (suite *RestoreUnitSuite) TestRestoreItem_collisionHandling() {
{ {
name: "collision, skip", name: "collision, skip",
collisionKeys: map[string]api.DriveItemIDType{ collisionKeys: map[string]api.DriveItemIDType{
mock.DriveItemFileName: {ItemID: mndiID}, odMock.DriveItemFileName: {ItemID: mndiID},
}, },
onCollision: control.Skip, onCollision: control.Skip,
expectSkipped: assert.True, expectSkipped: assert.True,
expectMock: func(t *testing.T, rh *mock.RestoreHandler) { expectMock: func(t *testing.T, rh *odMock.RestoreHandler) {
assert.False(t, rh.CalledPostItem, "new item posted") assert.False(t, rh.CalledPostItem, "new item posted")
assert.False(t, rh.CalledDeleteItem, "new item deleted") assert.False(t, rh.CalledDeleteItem, "new item deleted")
}, },
@ -142,14 +144,14 @@ func (suite *RestoreUnitSuite) TestRestoreItem_collisionHandling() {
{ {
name: "file-folder collision, copy", name: "file-folder collision, copy",
collisionKeys: map[string]api.DriveItemIDType{ collisionKeys: map[string]api.DriveItemIDType{
mock.DriveItemFileName: { odMock.DriveItemFileName: {
ItemID: mndiID, ItemID: mndiID,
IsFolder: true, IsFolder: true,
}, },
}, },
onCollision: control.Copy, onCollision: control.Copy,
expectSkipped: assert.False, expectSkipped: assert.False,
expectMock: func(t *testing.T, rh *mock.RestoreHandler) { expectMock: func(t *testing.T, rh *odMock.RestoreHandler) {
assert.True(t, rh.CalledPostItem, "new item posted") assert.True(t, rh.CalledPostItem, "new item posted")
assert.False(t, rh.CalledDeleteItem, "new item deleted") assert.False(t, rh.CalledDeleteItem, "new item deleted")
}, },
@ -158,14 +160,14 @@ func (suite *RestoreUnitSuite) TestRestoreItem_collisionHandling() {
{ {
name: "file-folder collision, replace", name: "file-folder collision, replace",
collisionKeys: map[string]api.DriveItemIDType{ collisionKeys: map[string]api.DriveItemIDType{
mock.DriveItemFileName: { odMock.DriveItemFileName: {
ItemID: mndiID, ItemID: mndiID,
IsFolder: true, IsFolder: true,
}, },
}, },
onCollision: control.Replace, onCollision: control.Replace,
expectSkipped: assert.False, expectSkipped: assert.False,
expectMock: func(t *testing.T, rh *mock.RestoreHandler) { expectMock: func(t *testing.T, rh *odMock.RestoreHandler) {
assert.True(t, rh.CalledPostItem, "new item posted") assert.True(t, rh.CalledPostItem, "new item posted")
assert.False(t, rh.CalledDeleteItem, "new item deleted") assert.False(t, rh.CalledDeleteItem, "new item deleted")
}, },
@ -174,14 +176,14 @@ func (suite *RestoreUnitSuite) TestRestoreItem_collisionHandling() {
{ {
name: "file-folder collision, skip", name: "file-folder collision, skip",
collisionKeys: map[string]api.DriveItemIDType{ collisionKeys: map[string]api.DriveItemIDType{
mock.DriveItemFileName: { odMock.DriveItemFileName: {
ItemID: mndiID, ItemID: mndiID,
IsFolder: true, IsFolder: true,
}, },
}, },
onCollision: control.Skip, onCollision: control.Skip,
expectSkipped: assert.True, expectSkipped: assert.True,
expectMock: func(t *testing.T, rh *mock.RestoreHandler) { expectMock: func(t *testing.T, rh *odMock.RestoreHandler) {
assert.False(t, rh.CalledPostItem, "new item posted") assert.False(t, rh.CalledPostItem, "new item posted")
assert.False(t, rh.CalledDeleteItem, "new item deleted") assert.False(t, rh.CalledDeleteItem, "new item deleted")
}, },
@ -200,7 +202,7 @@ func (suite *RestoreUnitSuite) TestRestoreItem_collisionHandling() {
var ( var (
caches = NewRestoreCaches(nil) caches = NewRestoreCaches(nil)
rh = &mock.RestoreHandler{ rh = &odMock.RestoreHandler{
PostItemResp: models.NewDriveItem(), PostItemResp: models.NewDriveItem(),
DeleteItemErr: test.deleteErr, DeleteItemErr: test.deleteErr,
} }
@ -228,18 +230,20 @@ func (suite *RestoreUnitSuite) TestRestoreItem_collisionHandling() {
ctx, ctx,
rh, rh,
rcc, rcc,
mock.FetchItemByName{ odMock.FetchItemByName{
Item: &mock.Data{ Item: &dataMock.Item{
Reader: mock.FileRespReadCloser(mock.DriveFileMetaData), Reader: odMock.FileRespReadCloser(odMock.DriveFileMetaData),
ItemInfo: odStub.DriveItemInfo(),
}, },
}, },
dp, dp,
"", "",
make([]byte, graph.CopyBufferSize), make([]byte, graph.CopyBufferSize),
caches, caches,
&mock.Data{ &dataMock.Item{
ID: uuid.NewString(), ItemID: uuid.NewString(),
Reader: mock.FileRespReadCloser(mock.DriveFilePayloadData), Reader: odMock.FileRespReadCloser(odMock.DriveFilePayloadData),
ItemInfo: odStub.DriveItemInfo(),
}, },
nil, nil,
ctr) ctr)

View File

@ -38,16 +38,16 @@ const (
var ( var (
_ data.BackupCollection = &Collection{} _ data.BackupCollection = &Collection{}
_ data.Stream = &Item{} _ data.Item = &Item{}
_ data.StreamInfo = &Item{} _ data.ItemInfo = &Item{}
_ data.StreamModTime = &Item{} _ data.ItemModTime = &Item{}
) )
// Collection is the SharePoint.List implementation of data.Collection. SharePoint.Libraries collections are supported // Collection is the SharePoint.List implementation of data.Collection. SharePoint.Libraries collections are supported
// by the oneDrive.Collection as the calls are identical for populating the Collection // by the oneDrive.Collection as the calls are identical for populating the Collection
type Collection struct { type Collection struct {
// data is the container for each individual SharePoint.List // data is the container for each individual SharePoint.List
data chan data.Stream data chan data.Item
// fullPath indicates the hierarchy within the collection // fullPath indicates the hierarchy within the collection
fullPath path.Path fullPath path.Path
// jobs contain the SharePoint.Site.ListIDs for the associated list(s). // jobs contain the SharePoint.Site.ListIDs for the associated list(s).
@ -71,7 +71,7 @@ func NewCollection(
c := &Collection{ c := &Collection{
fullPath: folderPath, fullPath: folderPath,
jobs: make([]string, 0), jobs: make([]string, 0),
data: make(chan data.Stream, collectionChannelBufferSize), data: make(chan data.Item, collectionChannelBufferSize),
client: ac.Sites(), client: ac.Sites(),
statusUpdater: statusUpdater, statusUpdater: statusUpdater,
category: category, category: category,
@ -111,7 +111,7 @@ func (sc Collection) DoNotMergeItems() bool {
func (sc *Collection) Items( func (sc *Collection) Items(
ctx context.Context, ctx context.Context,
errs *fault.Bus, errs *fault.Bus,
) <-chan data.Stream { ) <-chan data.Item {
go sc.populate(ctx, errs) go sc.populate(ctx, errs)
return sc.data return sc.data
} }
@ -133,7 +133,7 @@ func NewItem(name string, d io.ReadCloser) *Item {
} }
} }
func (sd *Item) UUID() string { func (sd *Item) ID() string {
return sd.id return sd.id
} }

View File

@ -171,7 +171,7 @@ func (suite *SharePointCollectionSuite) TestCollection_Items() {
control.DefaultOptions()) control.DefaultOptions())
col.data <- test.getItem(t, test.itemName) col.data <- test.getItem(t, test.itemName)
readItems := []data.Stream{} readItems := []data.Item{}
for item := range col.Items(ctx, fault.New(true)) { for item := range col.Items(ctx, fault.New(true)) {
readItems = append(readItems, item) readItems = append(readItems, item)
@ -179,7 +179,7 @@ func (suite *SharePointCollectionSuite) TestCollection_Items() {
require.Equal(t, len(readItems), 1) require.Equal(t, len(readItems), 1)
item := readItems[0] item := readItems[0]
shareInfo, ok := item.(data.StreamInfo) shareInfo, ok := item.(data.ItemInfo)
require.True(t, ok) require.True(t, ok)
require.NotNil(t, shareInfo.Info()) require.NotNil(t, shareInfo.Info())
require.NotNil(t, shareInfo.Info().SharePoint) require.NotNil(t, shareInfo.Info().SharePoint)

View File

@ -136,17 +136,17 @@ func ConsumeRestoreCollections(
func restoreListItem( func restoreListItem(
ctx context.Context, ctx context.Context,
service graph.Servicer, service graph.Servicer,
itemData data.Stream, itemData data.Item,
siteID, destName string, siteID, destName string,
) (details.ItemInfo, error) { ) (details.ItemInfo, error) {
ctx, end := diagnostics.Span(ctx, "m365:sharepoint:restoreList", diagnostics.Label("item_uuid", itemData.UUID())) ctx, end := diagnostics.Span(ctx, "m365:sharepoint:restoreList", diagnostics.Label("item_uuid", itemData.ID()))
defer end() defer end()
ctx = clues.Add(ctx, "list_item_id", itemData.UUID()) ctx = clues.Add(ctx, "list_item_id", itemData.ID())
var ( var (
dii = details.ItemInfo{} dii = details.ItemInfo{}
listName = itemData.UUID() listName = itemData.ID()
) )
byteArray, err := io.ReadAll(itemData.ToReader()) byteArray, err := io.ReadAll(itemData.ToReader())
@ -254,7 +254,7 @@ func RestoreListCollection(
metrics.Bytes += itemInfo.SharePoint.Size metrics.Bytes += itemInfo.SharePoint.Size
itemPath, err := dc.FullPath().AppendItem(itemData.UUID()) itemPath, err := dc.FullPath().AppendItem(itemData.ID())
if err != nil { if err != nil {
el.AddRecoverable(ctx, clues.Wrap(err, "appending item to full path").WithClues(ctx)) el.AddRecoverable(ctx, clues.Wrap(err, "appending item to full path").WithClues(ctx))
continue continue
@ -333,7 +333,7 @@ func RestorePageCollection(
metrics.Bytes += itemInfo.SharePoint.Size metrics.Bytes += itemInfo.SharePoint.Size
itemPath, err := dc.FullPath().AppendItem(itemData.UUID()) itemPath, err := dc.FullPath().AppendItem(itemData.ID())
if err != nil { if err != nil {
el.AddRecoverable(ctx, clues.Wrap(err, "appending item to full path").WithClues(ctx)) el.AddRecoverable(ctx, clues.Wrap(err, "appending item to full path").WithClues(ctx))
continue continue

View File

@ -23,8 +23,8 @@ type prefixCollection struct {
state data.CollectionState state data.CollectionState
} }
func (c prefixCollection) Items(ctx context.Context, _ *fault.Bus) <-chan data.Stream { func (c prefixCollection) Items(ctx context.Context, _ *fault.Bus) <-chan data.Item {
res := make(chan data.Stream) res := make(chan data.Item)
close(res) close(res)
s := support.CreateStatus(ctx, support.Backup, 0, support.CollectionMetrics{}, "") s := support.CreateStatus(ctx, support.Backup, 0, support.CollectionMetrics{}, "")

View File

@ -16,7 +16,7 @@ import (
var ( var (
_ data.BackupCollection = &MetadataCollection{} _ data.BackupCollection = &MetadataCollection{}
_ data.Stream = &MetadataItem{} _ data.Item = &MetadataItem{}
) )
// MetadataCollection in a simple collection that assumes all items to be // MetadataCollection in a simple collection that assumes all items to be
@ -134,8 +134,8 @@ func (md MetadataCollection) DoNotMergeItems() bool {
func (md MetadataCollection) Items( func (md MetadataCollection) Items(
ctx context.Context, ctx context.Context,
_ *fault.Bus, // not used, just here for interface compliance _ *fault.Bus, // not used, just here for interface compliance
) <-chan data.Stream { ) <-chan data.Item {
res := make(chan data.Stream) res := make(chan data.Item)
go func() { go func() {
totalBytes := int64(0) totalBytes := int64(0)
@ -169,8 +169,8 @@ func (md MetadataCollection) Items(
return res return res
} }
// MetadataItem is an in-memory data.Stream implementation. MetadataItem does // MetadataItem is an in-memory data.Item implementation. MetadataItem does
// not implement additional interfaces like data.StreamInfo, so it should only // not implement additional interfaces like data.ItemInfo, so it should only
// be used for items with a small amount of content that don't need to be added // be used for items with a small amount of content that don't need to be added
// to backup details. // to backup details.
// //
@ -190,7 +190,7 @@ func NewMetadataItem(uuid string, itemData []byte) MetadataItem {
} }
} }
func (mi MetadataItem) UUID() string { func (mi MetadataItem) ID() string {
return mi.uuid return mi.uuid
} }

View File

@ -92,7 +92,7 @@ func (suite *MetadataCollectionUnitSuite) TestItems() {
gotNames := []string{} gotNames := []string{}
for s := range c.Items(ctx, fault.New(true)) { for s := range c.Items(ctx, fault.New(true)) {
gotNames = append(gotNames, s.UUID()) gotNames = append(gotNames, s.ID())
buf, err := io.ReadAll(s.ToReader()) buf, err := io.ReadAll(s.ToReader())
if !assert.NoError(t, err, clues.ToCore(err)) { if !assert.NoError(t, err, clues.ToCore(err)) {
@ -181,7 +181,7 @@ func (suite *MetadataCollectionUnitSuite) TestMakeMetadataCollection() {
itemCount := 0 itemCount := 0
for item := range col.Items(ctx, fault.New(true)) { for item := range col.Items(ctx, fault.New(true)) {
assert.Equal(t, test.metadata.fileName, item.UUID()) assert.Equal(t, test.metadata.fileName, item.ID())
gotMap := map[string]string{} gotMap := map[string]string{}
decoder := json.NewDecoder(item.ToReader()) decoder := json.NewDecoder(item.ToReader())

View File

@ -578,10 +578,10 @@ func checkEvent(
func compareExchangeEmail( func compareExchangeEmail(
t *testing.T, t *testing.T,
expected map[string][]byte, expected map[string][]byte,
item data.Stream, item data.Item,
) { ) {
itemData, err := io.ReadAll(item.ToReader()) itemData, err := io.ReadAll(item.ToReader())
if !assert.NoError(t, err, "reading collection item", item.UUID(), clues.ToCore(err)) { if !assert.NoError(t, err, "reading collection item", item.ID(), clues.ToCore(err)) {
return return
} }
@ -605,10 +605,10 @@ func compareExchangeContact(
t *testing.T, t *testing.T,
colPath path.Path, colPath path.Path,
expected map[string][]byte, expected map[string][]byte,
item data.Stream, item data.Item,
) { ) {
itemData, err := io.ReadAll(item.ToReader()) itemData, err := io.ReadAll(item.ToReader())
if !assert.NoError(t, err, "reading collection item", item.UUID(), clues.ToCore(err)) { if !assert.NoError(t, err, "reading collection item", item.ID(), clues.ToCore(err)) {
return return
} }
@ -633,10 +633,10 @@ func compareExchangeContact(
func compareExchangeEvent( func compareExchangeEvent(
t *testing.T, t *testing.T,
expected map[string][]byte, expected map[string][]byte,
item data.Stream, item data.Item,
) { ) {
itemData, err := io.ReadAll(item.ToReader()) itemData, err := io.ReadAll(item.ToReader())
if !assert.NoError(t, err, "reading collection item", item.UUID(), clues.ToCore(err)) { if !assert.NoError(t, err, "reading collection item", item.ID(), clues.ToCore(err)) {
return return
} }
@ -715,13 +715,13 @@ func linkSharesEqual(expected metadata.LinkShare, got metadata.LinkShare) bool {
func compareDriveItem( func compareDriveItem(
t *testing.T, t *testing.T,
expected map[string][]byte, expected map[string][]byte,
item data.Stream, item data.Item,
mci m365Stub.ConfigInfo, mci m365Stub.ConfigInfo,
rootDir bool, rootDir bool,
) bool { ) bool {
// Skip Drive permissions in the folder that used to be the root. We don't // Skip Drive permissions in the folder that used to be the root. We don't
// have a good way to materialize these in the test right now. // have a good way to materialize these in the test right now.
if rootDir && item.UUID() == metadata.DirMetaFileSuffix { if rootDir && item.ID() == metadata.DirMetaFileSuffix {
return false return false
} }
@ -732,7 +732,7 @@ func compareDriveItem(
var ( var (
displayName string displayName string
name = item.UUID() name = item.ID()
isMeta = metadata.HasMetaSuffix(name) isMeta = metadata.HasMetaSuffix(name)
) )
@ -867,11 +867,11 @@ func compareItem(
expected map[string][]byte, expected map[string][]byte,
service path.ServiceType, service path.ServiceType,
category path.CategoryType, category path.CategoryType,
item data.Stream, item data.Item,
mci m365Stub.ConfigInfo, mci m365Stub.ConfigInfo,
rootDir bool, rootDir bool,
) bool { ) bool {
if mt, ok := item.(data.StreamModTime); ok { if mt, ok := item.(data.ItemModTime); ok {
assert.NotZero(t, mt.ModTime()) assert.NotZero(t, mt.ModTime())
} }

View File

@ -10,13 +10,13 @@ import (
type RestoreCollection struct { type RestoreCollection struct {
data.Collection data.Collection
AuxItems map[string]data.Stream AuxItems map[string]data.Item
} }
func (rc RestoreCollection) FetchItemByName( func (rc RestoreCollection) FetchItemByName(
ctx context.Context, ctx context.Context,
name string, name string,
) (data.Stream, error) { ) (data.Item, error) {
res := rc.AuxItems[name] res := rc.AuxItems[name]
if res == nil { if res == nil {
return nil, data.ErrNotFound return nil, data.ErrNotFound
@ -28,12 +28,12 @@ func (rc RestoreCollection) FetchItemByName(
type BackupCollection struct { type BackupCollection struct {
Path path.Path Path path.Path
Loc *path.Builder Loc *path.Builder
Streams []data.Stream Streams []data.Item
CState data.CollectionState CState data.CollectionState
} }
func (c *BackupCollection) Items(context.Context, *fault.Bus) <-chan data.Stream { func (c *BackupCollection) Items(context.Context, *fault.Bus) <-chan data.Item {
res := make(chan data.Stream) res := make(chan data.Item)
go func() { go func() {
defer close(res) defer close(res)

View File

@ -560,7 +560,7 @@ func parseMetadataCollections(
return nil, false, clues.New("decoding metadata json").WithClues(ctx) return nil, false, clues.New("decoding metadata json").WithClues(ctx)
} }
switch item.UUID() { switch item.ID() {
case graph.PreviousPathFileName: case graph.PreviousPathFileName:
if _, ok := found[category]["path"]; ok { if _, ok := found[category]["path"]; ok {
return nil, false, clues.Wrap(clues.New(category.String()), "multiple versions of path metadata").WithClues(ctx) return nil, false, clues.Wrap(clues.New(category.String()), "multiple versions of path metadata").WithClues(ctx)

View File

@ -331,8 +331,8 @@ type failingColl struct {
t *testing.T t *testing.T
} }
func (f failingColl) Items(ctx context.Context, errs *fault.Bus) <-chan data.Stream { func (f failingColl) Items(ctx context.Context, errs *fault.Bus) <-chan data.Item {
ic := make(chan data.Stream) ic := make(chan data.Item)
defer close(ic) defer close(ic)
errs.AddRecoverable(ctx, assert.AnError) errs.AddRecoverable(ctx, assert.AnError)
@ -353,7 +353,7 @@ func (f failingColl) FullPath() path.Path {
return tmp return tmp
} }
func (f failingColl) FetchItemByName(context.Context, string) (data.Stream, error) { func (f failingColl) FetchItemByName(context.Context, string) (data.Item, error) {
// no fetch calls will be made // no fetch calls will be made
return nil, nil return nil, nil
} }

View File

@ -26,9 +26,9 @@ import (
var ( var (
_ data.BackupCollection = &Collection{} _ data.BackupCollection = &Collection{}
_ data.Stream = &Stream{} _ data.Item = &Item{}
_ data.StreamInfo = &Stream{} _ data.ItemInfo = &Item{}
_ data.StreamModTime = &Stream{} _ data.ItemModTime = &Item{}
) )
const ( const (
@ -41,7 +41,7 @@ const (
type Collection struct { type Collection struct {
// M365 user // M365 user
user string // M365 user user string // M365 user
data chan data.Stream data chan data.Item
// added is a list of existing item IDs that were added to a container // added is a list of existing item IDs that were added to a container
added map[string]struct{} added map[string]struct{}
@ -92,7 +92,7 @@ func NewCollection(
added: make(map[string]struct{}, 0), added: make(map[string]struct{}, 0),
category: category, category: category,
ctrl: ctrlOpts, ctrl: ctrlOpts,
data: make(chan data.Stream, collectionChannelBufferSize), data: make(chan data.Item, collectionChannelBufferSize),
doNotMergeItems: doNotMergeItems, doNotMergeItems: doNotMergeItems,
fullPath: curr, fullPath: curr,
items: items, items: items,
@ -109,7 +109,7 @@ func NewCollection(
// Items utility function to asynchronously execute process to fill data channel with // Items utility function to asynchronously execute process to fill data channel with
// M365 exchange objects and returns the data channel // M365 exchange objects and returns the data channel
func (col *Collection) Items(ctx context.Context, errs *fault.Bus) <-chan data.Stream { func (col *Collection) Items(ctx context.Context, errs *fault.Bus) <-chan data.Item {
go col.streamItems(ctx, errs) go col.streamItems(ctx, errs)
return col.data return col.data
} }
@ -183,7 +183,7 @@ func (col *Collection) streamItems(ctx context.Context, errs *fault.Bus) {
defer wg.Done() defer wg.Done()
defer func() { <-semaphoreCh }() defer func() { <-semaphoreCh }()
col.data <- &Stream{ col.data <- &Item{
id: id, id: id,
modTime: time.Now().UTC(), // removed items have no modTime entry. modTime: time.Now().UTC(), // removed items have no modTime entry.
deleted: true, deleted: true,
@ -247,7 +247,7 @@ func (col *Collection) streamItems(ctx context.Context, errs *fault.Bus) {
info.ParentPath = col.locationPath.String() info.ParentPath = col.locationPath.String()
col.data <- &Stream{ col.data <- &Item{
id: id, id: id,
message: data, message: data,
info: info, info: info,
@ -292,8 +292,8 @@ func (col *Collection) finishPopulation(
col.statusUpdater(status) col.statusUpdater(status)
} }
// Stream represents a single item retrieved from exchange // Item represents a single item retrieved from exchange
type Stream struct { type Item struct {
id string id string
// TODO: We may need this to be a "oneOf" of `message`, `contact`, etc. // TODO: We may need this to be a "oneOf" of `message`, `contact`, etc.
// going forward. Using []byte for now but I assume we'll have // going forward. Using []byte for now but I assume we'll have
@ -308,29 +308,33 @@ type Stream struct {
deleted bool deleted bool
} }
func (od *Stream) UUID() string { func (i *Item) ID() string {
return od.id return i.id
} }
func (od *Stream) ToReader() io.ReadCloser { func (i *Item) ToReader() io.ReadCloser {
return io.NopCloser(bytes.NewReader(od.message)) return io.NopCloser(bytes.NewReader(i.message))
} }
func (od Stream) Deleted() bool { func (i Item) Deleted() bool {
return od.deleted return i.deleted
} }
func (od *Stream) Info() details.ItemInfo { func (i *Item) Info() details.ItemInfo {
return details.ItemInfo{Exchange: od.info} return details.ItemInfo{Exchange: i.info}
} }
func (od *Stream) ModTime() time.Time { func (i *Item) ModTime() time.Time {
return od.modTime return i.modTime
} }
// NewStream constructor for exchange.Stream object func NewItem(
func NewStream(identifier string, dataBytes []byte, detail details.ExchangeInfo, modTime time.Time) Stream { identifier string,
return Stream{ dataBytes []byte,
detail details.ExchangeInfo,
modTime time.Time,
) Item {
return Item{
id: identifier, id: identifier,
message: dataBytes, message: dataBytes,
info: &detail, info: &detail,

View File

@ -57,13 +57,13 @@ func TestCollectionSuite(t *testing.T) {
func (suite *CollectionSuite) TestReader_Valid() { func (suite *CollectionSuite) TestReader_Valid() {
m := []byte("test message") m := []byte("test message")
description := "aFile" description := "aFile"
ed := &Stream{id: description, message: m} ed := &Item{id: description, message: m}
buf := &bytes.Buffer{} buf := &bytes.Buffer{}
_, err := buf.ReadFrom(ed.ToReader()) _, err := buf.ReadFrom(ed.ToReader())
assert.NoError(suite.T(), err, clues.ToCore(err)) assert.NoError(suite.T(), err, clues.ToCore(err))
assert.Equal(suite.T(), buf.Bytes(), m) assert.Equal(suite.T(), buf.Bytes(), m)
assert.Equal(suite.T(), description, ed.UUID()) assert.Equal(suite.T(), description, ed.ID())
} }
func (suite *CollectionSuite) TestReader_Empty() { func (suite *CollectionSuite) TestReader_Empty() {
@ -73,7 +73,7 @@ func (suite *CollectionSuite) TestReader_Empty() {
t = suite.T() t = suite.T()
) )
ed := &Stream{message: empty} ed := &Item{message: empty}
buf := &bytes.Buffer{} buf := &bytes.Buffer{}
received, err := buf.ReadFrom(ed.ToReader()) received, err := buf.ReadFrom(ed.ToReader())

View File

@ -10,6 +10,7 @@ import (
"github.com/google/uuid" "github.com/google/uuid"
"github.com/alcionai/corso/src/internal/data" "github.com/alcionai/corso/src/internal/data"
dataMock "github.com/alcionai/corso/src/internal/data/mock"
"github.com/alcionai/corso/src/pkg/backup/details" "github.com/alcionai/corso/src/pkg/backup/details"
"github.com/alcionai/corso/src/pkg/fault" "github.com/alcionai/corso/src/pkg/fault"
"github.com/alcionai/corso/src/pkg/path" "github.com/alcionai/corso/src/pkg/path"
@ -29,12 +30,7 @@ type DataCollection struct {
DoNotMerge bool DoNotMerge bool
} }
var ( var _ data.BackupCollection = &DataCollection{}
_ data.BackupCollection = &DataCollection{}
_ data.Stream = &Data{}
_ data.StreamInfo = &Data{}
_ data.StreamSize = &Data{}
)
func (medc DataCollection) FullPath() path.Path { return medc.fullPath } func (medc DataCollection) FullPath() path.Path { return medc.fullPath }
@ -118,19 +114,20 @@ func NewContactCollection(pathRepresentation path.Path, numMessagesToReturn int)
func (medc *DataCollection) Items( func (medc *DataCollection) Items(
ctx context.Context, ctx context.Context,
_ *fault.Bus, // unused _ *fault.Bus, // unused
) <-chan data.Stream { ) <-chan data.Item {
res := make(chan data.Stream) res := make(chan data.Item)
go func() { go func() {
defer close(res) defer close(res)
for i := 0; i < medc.messageCount; i++ { for i := 0; i < medc.messageCount; i++ {
res <- &Data{ res <- &dataMock.Item{
ID: medc.Names[i], ItemID: medc.Names[i],
Reader: io.NopCloser(bytes.NewReader(medc.Data[i])), Reader: io.NopCloser(bytes.NewReader(medc.Data[i])),
size: int64(len(medc.Data[i])), ItemSize: int64(len(medc.Data[i])),
modifiedTime: medc.ModTimes[i], ModifiedTime: medc.ModTimes[i],
deleted: medc.DeletedItems[i], DeletedFlag: medc.DeletedItems[i],
ItemInfo: StubMailInfo(),
} }
} }
}() }()
@ -138,31 +135,7 @@ func (medc *DataCollection) Items(
return res return res
} }
// TODO: move to data/mock for service-agnostic mocking func StubMailInfo() details.ItemInfo {
// Data represents a single item retrieved from exchange
type Data struct {
ID string
Reader io.ReadCloser
ReadErr error
size int64
modifiedTime time.Time
deleted bool
}
func (med *Data) UUID() string { return med.ID }
func (med *Data) Deleted() bool { return med.deleted }
func (med *Data) Size() int64 { return med.size }
func (med *Data) ModTime() time.Time { return med.modifiedTime }
func (med *Data) ToReader() io.ReadCloser {
if med.ReadErr != nil {
return io.NopCloser(errReader{med.ReadErr})
}
return med.Reader
}
func (med *Data) Info() details.ItemInfo {
return details.ItemInfo{ return details.ItemInfo{
Exchange: &details.ExchangeInfo{ Exchange: &details.ExchangeInfo{
ItemType: details.ExchangeMail, ItemType: details.ExchangeMail,
@ -172,11 +145,3 @@ func (med *Data) Info() details.ItemInfo {
}, },
} }
} }
type errReader struct {
readErr error
}
func (er errReader) Read([]byte) (int, error) {
return 0, er.readErr
}

View File

@ -11,6 +11,7 @@ import (
"github.com/stretchr/testify/suite" "github.com/stretchr/testify/suite"
"github.com/alcionai/corso/src/internal/data" "github.com/alcionai/corso/src/internal/data"
dataMock "github.com/alcionai/corso/src/internal/data/mock"
"github.com/alcionai/corso/src/internal/tester" "github.com/alcionai/corso/src/internal/tester"
"github.com/alcionai/corso/src/pkg/fault" "github.com/alcionai/corso/src/pkg/fault"
"github.com/alcionai/corso/src/pkg/services/m365/api" "github.com/alcionai/corso/src/pkg/services/m365/api"
@ -55,8 +56,8 @@ func (suite *MockSuite) TestMockExchangeCollectionItemSize() {
buf, err := io.ReadAll(item.ToReader()) buf, err := io.ReadAll(item.ToReader())
assert.NoError(t, err, clues.ToCore(err)) assert.NoError(t, err, clues.ToCore(err))
assert.Implements(t, (*data.StreamSize)(nil), item) assert.Implements(t, (*data.ItemSize)(nil), item)
s := item.(data.StreamSize) s := item.(data.ItemSize)
assert.Equal(t, int64(len(buf)), s.Size()) assert.Equal(t, int64(len(buf)), s.Size())
} }
} }
@ -97,22 +98,24 @@ func (suite *MockExchangeDataSuite) TestMockExchangeData() {
table := []struct { table := []struct {
name string name string
reader *Data reader *dataMock.Item
check require.ErrorAssertionFunc check require.ErrorAssertionFunc
}{ }{
{ {
name: "NoError", name: "NoError",
reader: &Data{ reader: &dataMock.Item{
ID: id, ItemID: id,
Reader: io.NopCloser(bytes.NewReader(itemData)), Reader: io.NopCloser(bytes.NewReader(itemData)),
ItemInfo: StubMailInfo(),
}, },
check: require.NoError, check: require.NoError,
}, },
{ {
name: "Error", name: "Error",
reader: &Data{ reader: &dataMock.Item{
ID: id, ItemID: id,
ReadErr: assert.AnError, ReadErr: assert.AnError,
ItemInfo: StubMailInfo(),
}, },
check: require.Error, check: require.Error,
}, },
@ -122,7 +125,7 @@ func (suite *MockExchangeDataSuite) TestMockExchangeData() {
suite.Run(test.name, func() { suite.Run(test.name, func() {
t := suite.T() t := suite.T()
assert.Equal(t, id, test.reader.UUID()) assert.Equal(t, id, test.reader.ID())
buf, err := io.ReadAll(test.reader.ToReader()) buf, err := io.ReadAll(test.reader.ToReader())
test.check(t, err, clues.ToCore(err)) test.check(t, err, clues.ToCore(err))

View File

@ -168,8 +168,8 @@ func restoreCollection(
return metrics, el.Failure() return metrics, el.Failure()
} }
ictx := clues.Add(ctx, "item_id", itemData.UUID()) ictx := clues.Add(ctx, "item_id", itemData.ID())
trace.Log(ictx, "m365:exchange:restoreCollection:item", itemData.UUID()) trace.Log(ictx, "m365:exchange:restoreCollection:item", itemData.ID())
metrics.Objects++ metrics.Objects++
buf := &bytes.Buffer{} buf := &bytes.Buffer{}
@ -204,7 +204,7 @@ func restoreCollection(
// FIXME: this may be the incorrect path. If we restored within a top-level // FIXME: this may be the incorrect path. If we restored within a top-level
// destination folder, then the restore path no longer matches the fullPath. // destination folder, then the restore path no longer matches the fullPath.
itemPath, err := fullPath.AppendItem(itemData.UUID()) itemPath, err := fullPath.AppendItem(itemData.ID())
if err != nil { if err != nil {
el.AddRecoverable(ctx, clues.Wrap(err, "adding item to collection path").WithClues(ctx)) el.AddRecoverable(ctx, clues.Wrap(err, "adding item to collection path").WithClues(ctx))
continue continue

View File

@ -54,7 +54,7 @@ func items(ctx context.Context, ec exportCollection, ch chan<- export.Item) {
// There will only be a single item in the backingCollections // There will only be a single item in the backingCollections
// for OneDrive // for OneDrive
for item := range ec.backingCollection.Items(ctx, errs) { for item := range ec.backingCollection.Items(ctx, errs) {
itemUUID := item.UUID() itemUUID := item.ID()
if isMetadataFile(itemUUID, ec.backupVersion) { if isMetadataFile(itemUUID, ec.backupVersion) {
continue continue
} }

View File

@ -10,8 +10,10 @@ import (
"github.com/stretchr/testify/suite" "github.com/stretchr/testify/suite"
"github.com/alcionai/corso/src/internal/data" "github.com/alcionai/corso/src/internal/data"
dataMock "github.com/alcionai/corso/src/internal/data/mock"
"github.com/alcionai/corso/src/internal/m365/collection/drive/metadata" "github.com/alcionai/corso/src/internal/m365/collection/drive/metadata"
odConsts "github.com/alcionai/corso/src/internal/m365/service/onedrive/consts" odConsts "github.com/alcionai/corso/src/internal/m365/service/onedrive/consts"
odStub "github.com/alcionai/corso/src/internal/m365/service/onedrive/stub"
"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/control" "github.com/alcionai/corso/src/pkg/control"
@ -67,30 +69,22 @@ func (suite *ExportUnitSuite) TestIsMetadataFile() {
} }
} }
type metadataStream struct {
id string
name string
}
func (ms metadataStream) ToReader() io.ReadCloser {
return io.NopCloser(bytes.NewBufferString(`{"filename": "` + ms.name + `"}`))
}
func (ms metadataStream) UUID() string { return ms.id }
func (ms metadataStream) Deleted() bool { return false }
type finD struct { type finD struct {
id string id string
name string name string
err error err error
} }
func (fd finD) FetchItemByName(ctx context.Context, name string) (data.Stream, error) { func (fd finD) FetchItemByName(ctx context.Context, name string) (data.Item, error) {
if fd.err != nil { if fd.err != nil {
return nil, fd.err return nil, fd.err
} }
if name == fd.id { if name == fd.id {
return metadataStream{id: fd.id, name: fd.name}, nil return &dataMock.Item{
ItemID: fd.id,
Reader: io.NopCloser(bytes.NewBufferString(`{"filename": "` + fd.name + `"}`)),
}, nil
} }
return nil, assert.AnError return nil, assert.AnError
@ -148,8 +142,7 @@ func (suite *ExportUnitSuite) TestGetItemName() {
ctx, ctx,
test.id, test.id,
test.backupVersion, test.backupVersion,
test.fin, test.fin)
)
test.errFunc(t, err) test.errFunc(t, err)
assert.Equal(t, test.name, name, "name") assert.Equal(t, test.name, name, "name")
@ -159,11 +152,11 @@ func (suite *ExportUnitSuite) TestGetItemName() {
type mockRestoreCollection struct { type mockRestoreCollection struct {
path path.Path path path.Path
items []mockDataStream items []*dataMock.Item
} }
func (rc mockRestoreCollection) Items(ctx context.Context, errs *fault.Bus) <-chan data.Stream { func (rc mockRestoreCollection) Items(ctx context.Context, errs *fault.Bus) <-chan data.Item {
ch := make(chan data.Stream) ch := make(chan data.Item)
go func() { go func() {
defer close(ch) defer close(ch)
@ -171,8 +164,8 @@ func (rc mockRestoreCollection) Items(ctx context.Context, errs *fault.Bus) <-ch
el := errs.Local() el := errs.Local()
for _, item := range rc.items { for _, item := range rc.items {
if item.err != nil { if item.ReadErr != nil {
el.AddRecoverable(ctx, item.err) el.AddRecoverable(ctx, item.ReadErr)
continue continue
} }
@ -187,22 +180,6 @@ func (rc mockRestoreCollection) FullPath() path.Path {
return rc.path return rc.path
} }
type mockDataStream struct {
id string
data string
err error
}
func (ms mockDataStream) ToReader() io.ReadCloser {
if ms.data != "" {
return io.NopCloser(bytes.NewBufferString(ms.data))
}
return nil
}
func (ms mockDataStream) UUID() string { return ms.id }
func (ms mockDataStream) Deleted() bool { return false }
func (suite *ExportUnitSuite) TestGetItems() { func (suite *ExportUnitSuite) TestGetItems() {
table := []struct { table := []struct {
name string name string
@ -215,8 +192,11 @@ func (suite *ExportUnitSuite) TestGetItems() {
version: 1, version: 1,
backingCollection: data.NoFetchRestoreCollection{ backingCollection: data.NoFetchRestoreCollection{
Collection: mockRestoreCollection{ Collection: mockRestoreCollection{
items: []mockDataStream{ items: []*dataMock.Item{
{id: "name1", data: "body1"}, {
ItemID: "name1",
Reader: io.NopCloser(bytes.NewBufferString("body1")),
},
}, },
}, },
}, },
@ -235,9 +215,15 @@ func (suite *ExportUnitSuite) TestGetItems() {
version: 1, version: 1,
backingCollection: data.NoFetchRestoreCollection{ backingCollection: data.NoFetchRestoreCollection{
Collection: mockRestoreCollection{ Collection: mockRestoreCollection{
items: []mockDataStream{ items: []*dataMock.Item{
{id: "name1", data: "body1"}, {
{id: "name2", data: "body2"}, ItemID: "name1",
Reader: io.NopCloser(bytes.NewBufferString("body1")),
},
{
ItemID: "name2",
Reader: io.NopCloser(bytes.NewBufferString("body2")),
},
}, },
}, },
}, },
@ -263,8 +249,11 @@ func (suite *ExportUnitSuite) TestGetItems() {
version: 2, version: 2,
backingCollection: data.NoFetchRestoreCollection{ backingCollection: data.NoFetchRestoreCollection{
Collection: mockRestoreCollection{ Collection: mockRestoreCollection{
items: []mockDataStream{ items: []*dataMock.Item{
{id: "name1.data", data: "body1"}, {
ItemID: "name1.data",
Reader: io.NopCloser(bytes.NewBufferString("body1")),
},
}, },
}, },
}, },
@ -283,8 +272,11 @@ func (suite *ExportUnitSuite) TestGetItems() {
version: version.Backup, version: version.Backup,
backingCollection: data.FetchRestoreCollection{ backingCollection: data.FetchRestoreCollection{
Collection: mockRestoreCollection{ Collection: mockRestoreCollection{
items: []mockDataStream{ items: []*dataMock.Item{
{id: "id1.data", data: "body1"}, {
ItemID: "id1.data",
Reader: io.NopCloser(bytes.NewBufferString("body1")),
},
}, },
}, },
FetchItemByNamer: finD{id: "id1.meta", name: "name1"}, FetchItemByNamer: finD{id: "id1.meta", name: "name1"},
@ -304,8 +296,8 @@ func (suite *ExportUnitSuite) TestGetItems() {
version: version.Backup, version: version.Backup,
backingCollection: data.FetchRestoreCollection{ backingCollection: data.FetchRestoreCollection{
Collection: mockRestoreCollection{ Collection: mockRestoreCollection{
items: []mockDataStream{ items: []*dataMock.Item{
{id: "id1.data"}, {ItemID: "id1.data"},
}, },
}, },
FetchItemByNamer: finD{err: assert.AnError}, FetchItemByNamer: finD{err: assert.AnError},
@ -322,9 +314,14 @@ func (suite *ExportUnitSuite) TestGetItems() {
version: version.Backup, version: version.Backup,
backingCollection: data.FetchRestoreCollection{ backingCollection: data.FetchRestoreCollection{
Collection: mockRestoreCollection{ Collection: mockRestoreCollection{
items: []mockDataStream{ items: []*dataMock.Item{
{id: "missing.data"}, {
{id: "id1.data", data: "body1"}, ItemID: "missing.data",
},
{
ItemID: "id1.data",
Reader: io.NopCloser(bytes.NewBufferString("body1")),
},
}, },
}, },
FetchItemByNamer: finD{id: "id1.meta", name: "name1"}, FetchItemByNamer: finD{id: "id1.meta", name: "name1"},
@ -348,10 +345,19 @@ func (suite *ExportUnitSuite) TestGetItems() {
version: version.OneDrive1DataAndMetaFiles, version: version.OneDrive1DataAndMetaFiles,
backingCollection: data.FetchRestoreCollection{ backingCollection: data.FetchRestoreCollection{
Collection: mockRestoreCollection{ Collection: mockRestoreCollection{
items: []mockDataStream{ items: []*dataMock.Item{
{id: "name0", data: "body0"}, {
{id: "name1", err: assert.AnError}, ItemID: "name0",
{id: "name2", data: "body2"}, Reader: io.NopCloser(bytes.NewBufferString("body0")),
},
{
ItemID: "name1",
ReadErr: assert.AnError,
},
{
ItemID: "name2",
Reader: io.NopCloser(bytes.NewBufferString("body2")),
},
}, },
}, },
}, },
@ -419,7 +425,22 @@ func (suite *ExportUnitSuite) TestExportRestoreCollections() {
ctx, flush := tester.NewContext(t) ctx, flush := tester.NewContext(t)
defer flush() defer flush()
dpb := odConsts.DriveFolderPrefixBuilder("driveID1") var (
exportCfg = control.ExportConfig{}
dpb = odConsts.DriveFolderPrefixBuilder("driveID1")
dii = odStub.DriveItemInfo()
expectedItems = []export.Item{
{
ID: "id1.data",
Data: export.ItemData{
Name: "name1",
Body: io.NopCloser((bytes.NewBufferString("body1"))),
},
},
}
)
dii.OneDrive.ItemName = "name1"
p, err := dpb.ToDataLayerOneDrivePath("t", "u", false) p, err := dpb.ToDataLayerOneDrivePath("t", "u", false)
assert.NoError(t, err, "build path") assert.NoError(t, err, "build path")
@ -428,34 +449,31 @@ func (suite *ExportUnitSuite) TestExportRestoreCollections() {
data.FetchRestoreCollection{ data.FetchRestoreCollection{
Collection: mockRestoreCollection{ Collection: mockRestoreCollection{
path: p, path: p,
items: []mockDataStream{ items: []*dataMock.Item{
{id: "id1.data", data: "body1"}, {
ItemID: "id1.data",
Reader: io.NopCloser(bytes.NewBufferString("body1")),
ItemInfo: dii,
},
}, },
}, },
FetchItemByNamer: finD{id: "id1.meta", name: "name1"}, FetchItemByNamer: finD{id: "id1.meta", name: "name1"},
}, },
} }
expectedItems := []export.Item{ ecs, err := ProduceExportCollections(
{ ctx,
ID: "id1.data", int(version.Backup),
Data: export.ItemData{ exportCfg,
Name: "name1", control.DefaultOptions(),
Body: io.NopCloser((bytes.NewBufferString("body1"))), dcs,
}, nil,
}, fault.New(true))
}
exportCfg := control.ExportConfig{}
ecs, err := ProduceExportCollections(ctx, int(version.Backup), exportCfg, control.Options{}, dcs, nil, fault.New(true))
assert.NoError(t, err, "export collections error") assert.NoError(t, err, "export collections error")
assert.Len(t, ecs, 1, "num of collections") assert.Len(t, ecs, 1, "num of collections")
items := ecs[0].Items(ctx)
fitems := []export.Item{} fitems := []export.Item{}
for item := range items { for item := range ecs[0].Items(ctx) {
fitems = append(fitems, item) fitems = append(fitems, item)
} }

View File

@ -4,66 +4,12 @@ import (
"bytes" "bytes"
"context" "context"
"io" "io"
"time"
"github.com/alcionai/corso/src/internal/data" "github.com/alcionai/corso/src/internal/data"
"github.com/alcionai/corso/src/pkg/backup/details" "github.com/alcionai/corso/src/pkg/backup/details"
"github.com/alcionai/corso/src/pkg/extensions" "github.com/alcionai/corso/src/pkg/extensions"
) )
// ---------------------------------------------------------------------------
// data.Stream
// ---------------------------------------------------------------------------
var _ data.Stream = &Data{}
type Data struct {
ID string
DriveID string
DriveName string
Reader io.ReadCloser
ReadErr error
Sz int64
ModifiedTime time.Time
Del bool
ExtensionData *details.ExtensionData
}
func (d *Data) UUID() string { return d.ID }
func (d *Data) Deleted() bool { return d.Del }
func (d *Data) Size() int64 { return d.Sz }
func (d *Data) ModTime() time.Time { return d.ModifiedTime }
func (d *Data) ToReader() io.ReadCloser {
if d.ReadErr != nil {
return io.NopCloser(errReader{d.ReadErr})
}
return d.Reader
}
func (d *Data) Info() details.ItemInfo {
return details.ItemInfo{
OneDrive: &details.OneDriveInfo{
ItemType: details.OneDriveItem,
ItemName: "test.txt",
Size: d.Sz,
DriveID: d.DriveID,
DriveName: d.DriveName,
Modified: d.ModifiedTime,
},
Extension: d.ExtensionData,
}
}
type errReader struct {
readErr error
}
func (er errReader) Read([]byte) (int, error) {
return 0, er.readErr
}
// --------------------------------------------------------------------------- // ---------------------------------------------------------------------------
// FetchItemByNamer // FetchItemByNamer
// --------------------------------------------------------------------------- // ---------------------------------------------------------------------------
@ -71,11 +17,11 @@ func (er errReader) Read([]byte) (int, error) {
var _ data.FetchItemByNamer = &FetchItemByName{} var _ data.FetchItemByNamer = &FetchItemByName{}
type FetchItemByName struct { type FetchItemByName struct {
Item data.Stream Item data.Item
Err error Err error
} }
func (f FetchItemByName) FetchItemByName(context.Context, string) (data.Stream, error) { func (f FetchItemByName) FetchItemByName(context.Context, string) (data.Item, error) {
return f.Item, f.Err return f.Item, f.Err
} }

View File

@ -12,6 +12,7 @@ import (
odConsts "github.com/alcionai/corso/src/internal/m365/service/onedrive/consts" odConsts "github.com/alcionai/corso/src/internal/m365/service/onedrive/consts"
m365Stub "github.com/alcionai/corso/src/internal/m365/stub" m365Stub "github.com/alcionai/corso/src/internal/m365/stub"
"github.com/alcionai/corso/src/internal/version" "github.com/alcionai/corso/src/internal/version"
"github.com/alcionai/corso/src/pkg/backup/details"
"github.com/alcionai/corso/src/pkg/path" "github.com/alcionai/corso/src/pkg/path"
) )
@ -355,3 +356,13 @@ func ItemWithMetadata(
LookupKey: lookupKey, LookupKey: lookupKey,
}, nil }, nil
} }
func DriveItemInfo() details.ItemInfo {
return details.ItemInfo{
OneDrive: &details.OneDriveInfo{
ItemType: details.OneDriveItem,
ItemName: "test.txt",
Size: 1,
},
}
}

View File

@ -169,15 +169,15 @@ func retrieveSitePageOptions() *betasites.ItemPagesSitePageItemRequestBuilderGet
func RestoreSitePage( func RestoreSitePage(
ctx context.Context, ctx context.Context,
service *BetaService, service *BetaService,
itemData data.Stream, itemData data.Item,
siteID, destName string, siteID, destName string,
) (details.ItemInfo, error) { ) (details.ItemInfo, error) {
ctx, end := diagnostics.Span(ctx, "m365:sharepoint:restorePage", diagnostics.Label("item_uuid", itemData.UUID())) ctx, end := diagnostics.Span(ctx, "m365:sharepoint:restorePage", diagnostics.Label("item_uuid", itemData.ID()))
defer end() defer end()
var ( var (
dii = details.ItemInfo{} dii = details.ItemInfo{}
pageID = itemData.UUID() pageID = itemData.ID()
pageName = pageID pageName = pageID
) )

View File

@ -17,7 +17,7 @@ import (
) )
var ( var (
_ data.Stream = &ListData{} _ data.Item = &ListData{}
_ data.BackupCollection = &ListCollection{} _ data.BackupCollection = &ListCollection{}
) )
@ -50,8 +50,8 @@ func (mlc *ListCollection) PreviousPath() path.Path {
func (mlc *ListCollection) Items( func (mlc *ListCollection) Items(
ctx context.Context, ctx context.Context,
_ *fault.Bus, // unused _ *fault.Bus, // unused
) <-chan data.Stream { ) <-chan data.Item {
res := make(chan data.Stream) res := make(chan data.Item)
go func() { go func() {
defer close(res) defer close(res)
@ -65,15 +65,15 @@ func (mlc *ListCollection) Items(
} }
type ListData struct { type ListData struct {
ID string ItemID string
Reader io.ReadCloser Reader io.ReadCloser
ReadErr error ReadErr error
size int64 size int64
deleted bool deleted bool
} }
func (mld *ListData) UUID() string { func (mld *ListData) ID() string {
return mld.ID return mld.ItemID
} }
func (mld ListData) Deleted() bool { func (mld ListData) Deleted() bool {
@ -156,14 +156,14 @@ func ListBytes(title string) ([]byte, error) {
return objectWriter.GetSerializedContent() return objectWriter.GetSerializedContent()
} }
// ListStream returns the data.Stream representation // ListStream returns the data.Item representation
// of the Mocked SharePoint List // of the Mocked SharePoint List
func ListStream(t *testing.T, title string, numOfItems int) *ListData { func ListStream(t *testing.T, title string, numOfItems int) *ListData {
byteArray, err := ListBytes(title) byteArray, err := ListBytes(title)
require.NoError(t, err, clues.ToCore(err)) require.NoError(t, err, clues.ToCore(err))
listData := &ListData{ listData := &ListData{
ID: title, ItemID: title,
Reader: io.NopCloser(bytes.NewReader(byteArray)), Reader: io.NopCloser(bytes.NewReader(byteArray)),
size: int64(len(byteArray)), size: int64(len(byteArray)),
} }

View File

@ -7,6 +7,7 @@ import (
"golang.org/x/exp/maps" "golang.org/x/exp/maps"
"github.com/alcionai/corso/src/internal/data" "github.com/alcionai/corso/src/internal/data"
dataMock "github.com/alcionai/corso/src/internal/data/mock"
"github.com/alcionai/corso/src/internal/m365/collection/drive/metadata" "github.com/alcionai/corso/src/internal/m365/collection/drive/metadata"
"github.com/alcionai/corso/src/internal/m365/mock" "github.com/alcionai/corso/src/internal/m365/mock"
"github.com/alcionai/corso/src/internal/m365/resource" "github.com/alcionai/corso/src/internal/m365/resource"
@ -140,13 +141,14 @@ func CollectionsForInfo(
c := mock.RestoreCollection{ c := mock.RestoreCollection{
Collection: mc, Collection: mc,
AuxItems: map[string]data.Stream{}, AuxItems: map[string]data.Item{},
} }
for _, aux := range info.AuxItems { for _, aux := range info.AuxItems {
c.AuxItems[aux.Name] = &exchMock.Data{ c.AuxItems[aux.Name] = &dataMock.Item{
ID: aux.Name, ItemID: aux.Name,
Reader: io.NopCloser(bytes.NewReader(aux.Data)), Reader: io.NopCloser(bytes.NewReader(aux.Data)),
ItemInfo: exchMock.StubMailInfo(),
} }
} }

View File

@ -16,12 +16,14 @@ import (
"github.com/alcionai/corso/src/cli/config" "github.com/alcionai/corso/src/cli/config"
"github.com/alcionai/corso/src/internal/common/prefixmatcher" "github.com/alcionai/corso/src/internal/common/prefixmatcher"
"github.com/alcionai/corso/src/internal/data" "github.com/alcionai/corso/src/internal/data"
dataMock "github.com/alcionai/corso/src/internal/data/mock"
evmock "github.com/alcionai/corso/src/internal/events/mock" evmock "github.com/alcionai/corso/src/internal/events/mock"
"github.com/alcionai/corso/src/internal/kopia" "github.com/alcionai/corso/src/internal/kopia"
"github.com/alcionai/corso/src/internal/m365/graph" "github.com/alcionai/corso/src/internal/m365/graph"
"github.com/alcionai/corso/src/internal/m365/mock" "github.com/alcionai/corso/src/internal/m365/mock"
odConsts "github.com/alcionai/corso/src/internal/m365/service/onedrive/consts" odConsts "github.com/alcionai/corso/src/internal/m365/service/onedrive/consts"
odMock "github.com/alcionai/corso/src/internal/m365/service/onedrive/mock" odMock "github.com/alcionai/corso/src/internal/m365/service/onedrive/mock"
odStub "github.com/alcionai/corso/src/internal/m365/service/onedrive/stub"
"github.com/alcionai/corso/src/internal/m365/support" "github.com/alcionai/corso/src/internal/m365/support"
"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"
@ -1544,9 +1546,9 @@ func (mbp *mockBackupProducer) Wait() *data.CollectionStats {
func makeBackupCollection( func makeBackupCollection(
p path.Path, p path.Path,
locPath *path.Builder, locPath *path.Builder,
items []odMock.Data, items []dataMock.Item,
) data.BackupCollection { ) data.BackupCollection {
streams := make([]data.Stream, len(items)) streams := make([]data.Item, len(items))
for i := range items { for i := range items {
streams[i] = &items[i] streams[i] = &items[i]
@ -1586,29 +1588,34 @@ const (
folderID = "folder-id" folderID = "folder-id"
) )
func makeODMockData( func makeMockItem(
fileID string, fileID string,
extData *details.ExtensionData, extData *details.ExtensionData,
modTime time.Time, modTime time.Time,
del bool, del bool,
readErr error, readErr error,
) odMock.Data { ) dataMock.Item {
rc := odMock.FileRespReadCloser(odMock.DriveFilePayloadData) rc := odMock.FileRespReadCloser(odMock.DriveFilePayloadData)
if extData != nil { if extData != nil {
rc = odMock.FileRespWithExtensions(odMock.DriveFilePayloadData, extData) rc = odMock.FileRespWithExtensions(odMock.DriveFilePayloadData, extData)
} }
return odMock.Data{ dmi := dataMock.Item{
ID: fileID, DeletedFlag: del,
DriveID: driveID, ItemID: fileID,
DriveName: driveName, ItemInfo: odStub.DriveItemInfo(),
Reader: rc, ItemSize: 100,
ReadErr: readErr, ModifiedTime: modTime,
Sz: 100, Reader: rc,
ModifiedTime: modTime, ReadErr: readErr,
Del: del,
ExtensionData: extData,
} }
dmi.ItemInfo.OneDrive.DriveID = driveID
dmi.ItemInfo.OneDrive.DriveName = driveName
dmi.ItemInfo.OneDrive.Modified = modTime
dmi.ItemInfo.Extension = extData
return dmi
} }
// Check what kind of backup is produced for a given failurePolicy/observed fault // Check what kind of backup is produced for a given failurePolicy/observed fault
@ -1655,8 +1662,8 @@ func (suite *AssistBackupIntegrationSuite) TestBackupTypesForFailureModes() {
makeBackupCollection( makeBackupCollection(
tmp, tmp,
locPath, locPath,
[]odMock.Data{ []dataMock.Item{
makeODMockData("file1", nil, time.Now(), false, nil), makeMockItem("file1", nil, time.Now(), false, nil),
}), }),
} }
@ -1677,8 +1684,8 @@ func (suite *AssistBackupIntegrationSuite) TestBackupTypesForFailureModes() {
makeBackupCollection( makeBackupCollection(
tmp, tmp,
locPath, locPath,
[]odMock.Data{ []dataMock.Item{
makeODMockData("file1", nil, time.Now(), false, assert.AnError), makeMockItem("file1", nil, time.Now(), false, assert.AnError),
}), }),
} }
return bc return bc
@ -1697,8 +1704,8 @@ func (suite *AssistBackupIntegrationSuite) TestBackupTypesForFailureModes() {
makeBackupCollection( makeBackupCollection(
tmp, tmp,
locPath, locPath,
[]odMock.Data{ []dataMock.Item{
makeODMockData("file1", nil, time.Now(), false, nil), makeMockItem("file1", nil, time.Now(), false, nil),
}), }),
} }
@ -1732,8 +1739,8 @@ func (suite *AssistBackupIntegrationSuite) TestBackupTypesForFailureModes() {
makeBackupCollection( makeBackupCollection(
tmp, tmp,
locPath, locPath,
[]odMock.Data{ []dataMock.Item{
makeODMockData("file1", nil, time.Now(), false, assert.AnError), makeMockItem("file1", nil, time.Now(), false, assert.AnError),
}), }),
} }
@ -1754,9 +1761,9 @@ func (suite *AssistBackupIntegrationSuite) TestBackupTypesForFailureModes() {
makeBackupCollection( makeBackupCollection(
tmp, tmp,
locPath, locPath,
[]odMock.Data{ []dataMock.Item{
makeODMockData("file1", nil, time.Now(), false, nil), makeMockItem("file1", nil, time.Now(), false, nil),
makeODMockData("file2", nil, time.Now(), false, nil), makeMockItem("file2", nil, time.Now(), false, nil),
}), }),
} }
@ -1790,9 +1797,9 @@ func (suite *AssistBackupIntegrationSuite) TestBackupTypesForFailureModes() {
makeBackupCollection( makeBackupCollection(
tmp, tmp,
locPath, locPath,
[]odMock.Data{ []dataMock.Item{
makeODMockData("file1", nil, time.Now(), false, nil), makeMockItem("file1", nil, time.Now(), false, nil),
makeODMockData("file2", nil, time.Now(), false, assert.AnError), makeMockItem("file2", nil, time.Now(), false, assert.AnError),
}), }),
} }
@ -1847,7 +1854,6 @@ func (suite *AssistBackupIntegrationSuite) TestBackupTypesForFailureModes() {
err = bo.Run(ctx) err = bo.Run(ctx)
test.expectRunErr(t, err, clues.ToCore(err)) test.expectRunErr(t, err, clues.ToCore(err))
test.expectFaults(t, bo.Errors) test.expectFaults(t, bo.Errors)
if len(test.expectBackupTag) == 0 { if len(test.expectBackupTag) == 0 {
@ -1933,9 +1939,9 @@ func (suite *AssistBackupIntegrationSuite) TestExtensionsIncrementals() {
makeBackupCollection( makeBackupCollection(
tmp, tmp,
locPath, locPath,
[]odMock.Data{ []dataMock.Item{
makeODMockData("file1", extData[0], T1, false, nil), makeMockItem("file1", extData[0], T1, false, nil),
makeODMockData("file2", extData[1], T1, false, assert.AnError), makeMockItem("file2", extData[1], T1, false, assert.AnError),
}), }),
} }
@ -1960,10 +1966,10 @@ func (suite *AssistBackupIntegrationSuite) TestExtensionsIncrementals() {
makeBackupCollection( makeBackupCollection(
tmp, tmp,
locPath, locPath,
[]odMock.Data{ []dataMock.Item{
makeODMockData("file1", extData[0], T1, false, nil), makeMockItem("file1", extData[0], T1, false, nil),
makeODMockData("file2", extData[1], T2, false, nil), makeMockItem("file2", extData[1], T2, false, nil),
makeODMockData("file3", extData[2], T2, false, assert.AnError), makeMockItem("file3", extData[2], T2, false, assert.AnError),
}), }),
} }
@ -1995,10 +2001,10 @@ func (suite *AssistBackupIntegrationSuite) TestExtensionsIncrementals() {
makeBackupCollection( makeBackupCollection(
tmp, tmp,
locPath, locPath,
[]odMock.Data{ []dataMock.Item{
makeODMockData("file1", extData[0], T1, false, nil), makeMockItem("file1", extData[0], T1, false, nil),
makeODMockData("file2", extData[1], T2, false, nil), makeMockItem("file2", extData[1], T2, false, nil),
makeODMockData("file3", extData[2], T3, false, nil), makeMockItem("file3", extData[2], T3, false, nil),
}), }),
} }
@ -2034,10 +2040,10 @@ func (suite *AssistBackupIntegrationSuite) TestExtensionsIncrementals() {
makeBackupCollection( makeBackupCollection(
tmp, tmp,
locPath, locPath,
[]odMock.Data{ []dataMock.Item{
makeODMockData("file1", extData[0], T1, true, nil), makeMockItem("file1", extData[0], T1, true, nil),
makeODMockData("file2", extData[1], T2, true, nil), makeMockItem("file2", extData[1], T2, true, nil),
makeODMockData("file3", extData[2], T3, true, nil), makeMockItem("file3", extData[2], T3, true, nil),
}), }),
} }
@ -2056,8 +2062,8 @@ func (suite *AssistBackupIntegrationSuite) TestExtensionsIncrementals() {
makeBackupCollection( makeBackupCollection(
tmp, tmp,
locPath, locPath,
[]odMock.Data{ []dataMock.Item{
makeODMockData("file1", extData[0], T1, false, nil), makeMockItem("file1", extData[0], T1, false, nil),
}), }),
} }
@ -2087,10 +2093,10 @@ func (suite *AssistBackupIntegrationSuite) TestExtensionsIncrementals() {
makeBackupCollection( makeBackupCollection(
tmp, tmp,
locPath, locPath,
[]odMock.Data{ []dataMock.Item{
makeODMockData("file1", extData[0], T1, false, nil), makeMockItem("file1", extData[0], T1, false, nil),
makeODMockData("file2", extData[1], T2, false, nil), makeMockItem("file2", extData[1], T2, false, nil),
makeODMockData("file3", extData[2], T3, false, assert.AnError), makeMockItem("file3", extData[2], T3, false, assert.AnError),
}), }),
} }

View File

@ -29,7 +29,7 @@ type mockColl struct {
p path.Path p path.Path
} }
func (mc mockColl) Items(context.Context, *fault.Bus) <-chan data.Stream { func (mc mockColl) Items(context.Context, *fault.Bus) <-chan data.Item {
return nil return nil
} }

View File

@ -323,19 +323,19 @@ func checkMetadataFilesExist(
itemNames := []string{} itemNames := []string{}
for item := range col.Items(ctx, fault.New(true)) { for item := range col.Items(ctx, fault.New(true)) {
assert.Implements(t, (*data.StreamSize)(nil), item) assert.Implements(t, (*data.ItemSize)(nil), item)
s := item.(data.StreamSize) s := item.(data.ItemSize)
assert.Greaterf( assert.Greaterf(
t, t,
s.Size(), s.Size(),
int64(0), int64(0),
"empty metadata file: %s/%s", "empty metadata file: %s/%s",
col.FullPath(), col.FullPath(),
item.UUID(), item.ID(),
) )
itemNames = append(itemNames, item.UUID()) itemNames = append(itemNames, item.ID())
} }
assert.ElementsMatchf( assert.ElementsMatchf(

View File

@ -156,10 +156,10 @@ func (dc *streamCollection) DoNotMergeItems() bool {
return false return false
} }
// Items() always returns a channel with a single data.Stream // Items() always returns a channel with a single data.Item
// representing the object to be persisted // representing the object to be persisted
func (dc *streamCollection) Items(context.Context, *fault.Bus) <-chan data.Stream { func (dc *streamCollection) Items(context.Context, *fault.Bus) <-chan data.Item {
items := make(chan data.Stream, 1) items := make(chan data.Item, 1)
defer close(items) defer close(items)
items <- dc.item items <- dc.item
@ -175,7 +175,7 @@ type streamItem struct {
data []byte data []byte
} }
func (di *streamItem) UUID() string { func (di *streamItem) ID() string {
return di.name return di.name
} }