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"
"time"
"github.com/alcionai/clues"
"github.com/alcionai/corso/src/pkg/backup/details"
"github.com/alcionai/corso/src/pkg/fault"
"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
// denoted by FullPath.
type Collection interface {
@ -34,7 +17,7 @@ type Collection interface {
// 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
// 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
// Collection.
FullPath() path.Path
@ -77,32 +60,15 @@ type FetchItemByNamer interface {
// 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
// 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
// ErrNotFound for all Fetch calls.
type NoFetchRestoreCollection struct {
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
// Item represents a single item within a Collection
type Item interface {
// ToReader returns an io.Reader with the item's data
ToReader() io.ReadCloser
// UUID provides a unique identifier for this data
UUID() string
// ID provides a unique identifier for this item
ID() string
// Deleted returns true if the item represented by this Stream has been
// deleted and should be removed from the current in-progress backup.
Deleted() bool
@ -125,40 +91,20 @@ type PreviousLocationPather interface {
PreviousLocationPath() details.LocationIDer
}
// StreamInfo is used to provide service specific
// information about the Stream
type StreamInfo interface {
// ItemInfo returns the details.ItemInfo for the item.
type ItemInfo interface {
Info() details.ItemInfo
}
// StreamSize is used to provide size
// information about the Stream
type StreamSize interface {
// ItemSize returns the size of the item in bytes.
type ItemSize interface {
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().
type StreamModTime interface {
type ItemModTime interface {
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"
)
type DataCollectionSuite struct {
type CollectionSuite struct {
tester.Suite
}
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")
require.NoError(suite.T(), err, clues.ToCore(err))
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 {
ID string
Reader io.ReadCloser
ReadErr error
type Item struct {
DeletedFlag bool
ItemID string
ItemInfo details.ItemInfo
ItemSize int64
ModifiedTime time.Time
DeletedFlag bool
ItemInfo details.ItemInfo
Reader io.ReadCloser
ReadErr error
}
func (s *Stream) UUID() string {
return s.ID
func (s *Item) ID() string {
return s.ItemID
}
func (s Stream) Deleted() bool {
func (s Item) Deleted() bool {
return s.DeletedFlag
}
func (s *Stream) ToReader() io.ReadCloser {
func (s *Item) ToReader() io.ReadCloser {
if s.ReadErr != nil {
return io.NopCloser(errReader{s.ReadErr})
}
@ -45,15 +45,15 @@ func (s *Stream) ToReader() io.ReadCloser {
return s.Reader
}
func (s *Stream) Info() details.ItemInfo {
func (s *Item) Info() details.ItemInfo {
return s.ItemInfo
}
func (s *Stream) Size() int64 {
func (s *Item) Size() int64 {
return s.ItemSize
}
func (s *Stream) ModTime() time.Time {
func (s *Item) ModTime() time.Time {
return s.ModifiedTime
}
@ -77,7 +77,7 @@ var (
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
}
@ -97,6 +97,6 @@ func (c Collection) DoNotMergeItems() bool {
return true
}
func (c Collection) FetchItemByName(ctx context.Context, name string) (data.Stream, error) {
return &Stream{}, clues.New("not implemented")
func (c Collection) FetchItemByName(ctx context.Context, name string) (data.Item, error) {
return &Item{}, clues.New("not implemented")
}

View File

@ -15,7 +15,7 @@ import (
var (
_ data.RestoreCollection = &kopiaDataCollection{}
_ data.Stream = &kopiaDataStream{}
_ data.Item = &kopiaDataStream{}
)
type kopiaDataCollection struct {
@ -29,9 +29,9 @@ type kopiaDataCollection struct {
func (kdc *kopiaDataCollection) Items(
ctx context.Context,
errs *fault.Bus,
) <-chan data.Stream {
) <-chan data.Item {
var (
res = make(chan data.Stream)
res = make(chan data.Item)
el = errs.Local()
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
// 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.
func (kdc kopiaDataCollection) FetchItemByName(
ctx context.Context,
name string,
) (data.Stream, error) {
) (data.Item, error) {
ctx = clues.Add(ctx, "item_name", clues.Hide(name))
if kdc.dir == nil {
@ -119,7 +119,7 @@ func (kdc kopiaDataCollection) FetchItemByName(
}
return &kopiaDataStream{
uuid: name,
id: name,
reader: &restoreStreamReader{
ReadCloser: r,
expectedVersion: kdc.expectedVersion,
@ -130,7 +130,7 @@ func (kdc kopiaDataCollection) FetchItemByName(
type kopiaDataStream struct {
reader io.ReadCloser
uuid string
id string
size int64
}
@ -138,8 +138,8 @@ func (kds kopiaDataStream) ToReader() io.ReadCloser {
return kds.reader
}
func (kds kopiaDataStream) UUID() string {
return kds.uuid
func (kds kopiaDataStream) ID() string {
return kds.id
}
func (kds kopiaDataStream) Deleted() bool {

View File

@ -14,7 +14,7 @@ import (
"github.com/stretchr/testify/suite"
"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/pkg/fault"
"github.com/alcionai/corso/src/pkg/path"
@ -240,25 +240,25 @@ func (suite *KopiaDataCollectionUnitSuite) TestReturnsStreams() {
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")
found = append(found, loadedData{})
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)) {
continue
}
f.data = buf
if !assert.Implements(t, (*data.StreamSize)(nil), returnedStream) {
if !assert.Implements(t, (*data.ItemSize)(nil), item) {
continue
}
ss := returnedStream.(data.StreamSize)
ss := item.(data.ItemSize)
f.size = ss.Size()
}
@ -289,7 +289,7 @@ func (suite *KopiaDataCollectionUnitSuite) TestFetchItemByName() {
errFileName2 = "error2"
noErrFileData = "foo bar baz"
errReader = &exchMock.Data{
errReader = &dataMock.Item{
ReadErr: assert.AnError,
}
)

View File

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

View File

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

View File

@ -196,14 +196,16 @@ func (cp *corsoProgress) FinishedFile(relativePath string, err error) {
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
// never had to materialize their details in-memory.
if d.info == nil || d.cached {
if d.prevPath == nil {
cp.errs.AddRecoverable(cp.ctx, clues.New("item sourced from previous backup with no previous path").
With(
"service", d.repoPath.Service().String(),
"category", d.repoPath.Category().String()).
cp.errs.AddRecoverable(ctx, clues.New("finished file sourced from previous backup with no previous path").
Label(fault.LabelForceNoBackupCreation))
return
@ -218,10 +220,7 @@ func (cp *corsoProgress) FinishedFile(relativePath string, err error) {
d.repoPath,
d.locationPath)
if err != nil {
cp.errs.AddRecoverable(cp.ctx, clues.Wrap(err, "adding item to merge list").
With(
"service", d.repoPath.Service().String(),
"category", d.repoPath.Category().String()).
cp.errs.AddRecoverable(ctx, clues.Wrap(err, "adding finished file to merge list").
Label(fault.LabelForceNoBackupCreation))
}
@ -233,10 +232,7 @@ func (cp *corsoProgress) FinishedFile(relativePath string, err error) {
d.locationPath,
*d.info)
if err != nil {
cp.errs.AddRecoverable(cp.ctx, clues.New("adding item to details").
With(
"service", d.repoPath.Service().String(),
"category", d.repoPath.Category().String()).
cp.errs.AddRecoverable(ctx, clues.Wrap(err, "adding finished file to details").
Label(fault.LabelForceNoBackupCreation))
return
@ -348,7 +344,7 @@ func collectionEntries(
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
// 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{}{}
// 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 {
err = clues.Wrap(err, "getting full item path")
progress.errs.AddRecoverable(ctx, err)
@ -383,7 +379,7 @@ func collectionEntries(
}
modTime := time.Now()
if smt, ok := e.(data.StreamModTime); ok {
if smt, ok := e.(data.ItemModTime); ok {
modTime = smt.ModTime()
}
@ -392,7 +388,7 @@ func collectionEntries(
// 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
// adding something to it.
ei, ok := e.(data.StreamInfo)
ei, ok := e.(data.ItemInfo)
if ok {
// Relative path given to us in the callback is missing the root
// 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"
"github.com/alcionai/corso/src/internal/common/ptr"
"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"
exchMock "github.com/alcionai/corso/src/internal/m365/service/exchange/mock"
"github.com/alcionai/corso/src/internal/tester"
@ -79,7 +79,7 @@ func testForFiles(
for s := range c.Items(ctx, fault.New(true)) {
count++
fullPath, err := c.FullPath().AppendItem(s.UUID())
fullPath, err := c.FullPath().AppendItem(s.ID())
require.NoError(t, err, clues.ToCore(err))
expected, ok := expected[fullPath.String()]
@ -89,9 +89,9 @@ func testForFiles(
require.NoError(t, err, "reading collection item", fullPath, clues.ToCore(err))
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()))
}
}
@ -1108,7 +1108,7 @@ func (suite *KopiaIntegrationSuite) TestBackupCollections_NoDetailsForMeta() {
numDeetsEntries: 3,
hasMetaDeets: true,
cols: func() []data.BackupCollection {
streams := []data.Stream{}
streams := []data.Item{}
fileNames := []string{
testFileName,
testFileName + metadata.MetaFileSuffix,
@ -1119,8 +1119,8 @@ func (suite *KopiaIntegrationSuite) TestBackupCollections_NoDetailsForMeta() {
info := baseOneDriveItemInfo
info.ItemName = name
ms := &mock.Stream{
ID: name,
ms := &dataMock.Item{
ItemID: name,
Reader: io.NopCloser(&bytes.Buffer{}),
ItemSize: 0,
ItemInfo: details.ItemInfo{OneDrive: &info},
@ -1149,8 +1149,8 @@ func (suite *KopiaIntegrationSuite) TestBackupCollections_NoDetailsForMeta() {
info := baseOneDriveItemInfo
info.ItemName = testFileName
ms := &mock.Stream{
ID: testFileName,
ms := &dataMock.Item{
ItemID: testFileName,
Reader: io.NopCloser(&bytes.Buffer{}),
ItemSize: 0,
ItemInfo: details.ItemInfo{OneDrive: &info},
@ -1159,7 +1159,7 @@ func (suite *KopiaIntegrationSuite) TestBackupCollections_NoDetailsForMeta() {
mc := &mockBackupCollection{
path: storePath,
loc: locPath,
streams: []data.Stream{ms},
streams: []data.Item{ms},
state: data.NotMovedState,
}
@ -1298,12 +1298,12 @@ func (suite *KopiaIntegrationSuite) TestRestoreAfterCompressionChange() {
type mockBackupCollection struct {
path path.Path
loc *path.Builder
streams []data.Stream
streams []data.Item
state data.CollectionState
}
func (c *mockBackupCollection) Items(context.Context, *fault.Bus) <-chan data.Stream {
res := make(chan data.Stream)
func (c *mockBackupCollection) Items(context.Context, *fault.Bus) <-chan data.Item {
res := make(chan data.Item)
go func() {
defer close(res)
@ -1347,36 +1347,42 @@ func (suite *KopiaIntegrationSuite) TestBackupCollections_ReaderError() {
&mockBackupCollection{
path: suite.storePath1,
loc: loc1,
streams: []data.Stream{
&exchMock.Data{
ID: testFileName,
streams: []data.Item{
&dataMock.Item{
ItemID: testFileName,
Reader: io.NopCloser(bytes.NewReader(testFileData)),
ItemInfo: exchMock.StubMailInfo(),
},
&exchMock.Data{
ID: testFileName2,
&dataMock.Item{
ItemID: testFileName2,
Reader: io.NopCloser(bytes.NewReader(testFileData2)),
ItemInfo: exchMock.StubMailInfo(),
},
},
},
&mockBackupCollection{
path: suite.storePath2,
loc: loc2,
streams: []data.Stream{
&exchMock.Data{
ID: testFileName3,
streams: []data.Item{
&dataMock.Item{
ItemID: testFileName3,
Reader: io.NopCloser(bytes.NewReader(testFileData3)),
ItemInfo: exchMock.StubMailInfo(),
},
&exchMock.Data{
ID: testFileName4,
&dataMock.Item{
ItemID: testFileName4,
ReadErr: assert.AnError,
ItemInfo: exchMock.StubMailInfo(),
},
&exchMock.Data{
ID: testFileName5,
&dataMock.Item{
ItemID: testFileName5,
Reader: io.NopCloser(bytes.NewReader(testFileData5)),
ItemInfo: exchMock.StubMailInfo(),
},
&exchMock.Data{
ID: testFileName6,
&dataMock.Item{
ItemID: testFileName6,
Reader: io.NopCloser(bytes.NewReader(testFileData6)),
ItemInfo: exchMock.StubMailInfo(),
},
},
},
@ -1603,9 +1609,10 @@ func (suite *KopiaSimpleRepoIntegrationSuite) SetupTest() {
for _, item := range suite.files[parent.String()] {
collection.streams = append(
collection.streams,
&exchMock.Data{
ID: item.itemPath.Item(),
&dataMock.Item{
ItemID: item.itemPath.Item(),
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())
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())
require.NoError(t, err, clues.ToCore(err))

View File

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

View File

@ -174,7 +174,7 @@ func (suite *CollectionUnitTestSuite) TestCollection() {
var (
wg = sync.WaitGroup{}
collStatus = support.ControllerOperationStatus{}
readItems = []data.Stream{}
readItems = []data.Item{}
)
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
readItem := readItems[0]
assert.Equal(t, stubItemID+metadata.DataFileSuffix, readItem.UUID())
require.Implements(t, (*data.StreamModTime)(nil), readItem)
assert.Equal(t, stubItemID+metadata.DataFileSuffix, readItem.ID())
require.Implements(t, (*data.ItemModTime)(nil), readItem)
mt := readItem.(data.StreamModTime)
mt := readItem.(data.ItemModTime)
assert.Equal(t, now, mt.ModTime())
readData, err := io.ReadAll(readItem.ToReader())
@ -270,7 +270,7 @@ func (suite *CollectionUnitTestSuite) TestCollection() {
assert.Equal(t, stubItemContent, readData)
readItemMeta := readItems[1]
assert.Equal(t, stubItemID+metadata.MetaFileSuffix, readItemMeta.UUID())
assert.Equal(t, stubItemID+metadata.MetaFileSuffix, readItemMeta.ID())
readMeta := metadata.Metadata{}
err = json.NewDecoder(readItemMeta.ToReader()).Decode(&readMeta)
@ -472,7 +472,7 @@ func (suite *CollectionUnitTestSuite) TestCollectionPermissionBackupLatestModTim
coll.handler = mbh
readItems := []data.Stream{}
readItems := []data.Item{}
for item := range coll.Items(ctx, fault.New(true)) {
readItems = append(readItems, item)
}
@ -484,12 +484,12 @@ func (suite *CollectionUnitTestSuite) TestCollectionPermissionBackupLatestModTim
require.Equal(t, 1, collStatus.Metrics.Successes)
for _, i := range readItems {
if strings.HasSuffix(i.UUID(), metadata.MetaFileSuffix) {
if strings.HasSuffix(i.ID(), metadata.MetaFileSuffix) {
content, err := io.ReadAll(i.ToReader())
require.NoError(t, err, clues.ToCore(err))
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.Greater(t, im.ModTime(), mtime, "permissions time greater than mod time")
}
@ -978,7 +978,7 @@ func (suite *CollectionUnitTestSuite) TestItemExtensions() {
wg.Wait()
ei, ok := collItem.(data.StreamInfo)
ei, ok := collItem.(data.ItemInfo)
assert.True(t, ok)
itemInfo := ei.Info()

View File

@ -116,10 +116,10 @@ func deserializeMetadata(
var (
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:
err = deserializeMap(item.ToReader(), prevFolders)
@ -129,7 +129,7 @@ func deserializeMetadata(
default:
logger.Ctx(ictx).Infow(
"skipping unknown metadata file",
"file_name", item.UUID())
"file_name", item.ID())
continue
}

View File

@ -1153,8 +1153,8 @@ func (suite *OneDriveCollectionsUnitSuite) TestDeserializeMetadata() {
type failingColl struct{}
func (f failingColl) Items(ctx context.Context, errs *fault.Bus) <-chan data.Stream {
ic := make(chan data.Stream)
func (f failingColl) Items(ctx context.Context, errs *fault.Bus) <-chan data.Item {
ic := make(chan data.Item)
defer close(ic)
errs.AddRecoverable(ctx, assert.AnError)
@ -1162,7 +1162,7 @@ func (f failingColl) Items(ctx context.Context, errs *fault.Bus) <-chan data.Str
return ic
}
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
// canUsePreviousBackup as false on read errors

View File

@ -43,7 +43,7 @@ type Metadata struct {
}
type Item struct {
ID string
ItemID string
Data io.ReadCloser
Mod time.Time
}
@ -52,6 +52,6 @@ type Item struct {
// as deleted by adding them to the exclude list so this can always 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) ModTime() time.Time { return i.Mod }

View File

@ -184,7 +184,7 @@ func RestoreCollection(
wg.Add(1)
semaphoreCh <- struct{}{}
go func(ctx context.Context, itemData data.Stream) {
go func(ctx context.Context, itemData data.Item) {
defer wg.Done()
defer func() { <-semaphoreCh }()
@ -192,9 +192,9 @@ func RestoreCollection(
defer caches.pool.Put(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 {
el.AddRecoverable(ctx, clues.Wrap(err, "appending item to full path").WithClues(ictx))
return
@ -257,11 +257,11 @@ func restoreItem(
restoreFolderID string,
copyBuffer []byte,
caches *restoreCaches,
itemData data.Stream,
itemData data.Item,
itemPath path.Path,
ctr *count.Bus,
) (details.ItemInfo, bool, error) {
itemUUID := itemData.UUID()
itemUUID := itemData.ID()
ctx = clues.Add(ctx, "item_id", itemUUID)
if rcc.BackupVersion < version.OneDrive1DataAndMetaFiles {
@ -378,7 +378,7 @@ func restoreV0File(
restoreFolderID string,
copyBuffer []byte,
collisionKeyToItemID map[string]api.DriveItemIDType,
itemData data.Stream,
itemData data.Item,
ctr *count.Bus,
) (details.ItemInfo, error) {
_, itemInfo, err := restoreFile(
@ -386,7 +386,7 @@ func restoreV0File(
restoreCfg,
rh,
fibn,
itemData.UUID(),
itemData.ID(),
itemData,
drivePath.DriveID,
restoreFolderID,
@ -410,10 +410,10 @@ func restoreV1File(
copyBuffer []byte,
caches *restoreCaches,
itemPath path.Path,
itemData data.Stream,
itemData data.Item,
ctr *count.Bus,
) (details.ItemInfo, error) {
trimmedName := strings.TrimSuffix(itemData.UUID(), metadata.DataFileSuffix)
trimmedName := strings.TrimSuffix(itemData.ID(), metadata.DataFileSuffix)
itemID, itemInfo, err := restoreFile(
ctx,
@ -470,10 +470,10 @@ func restoreV6File(
copyBuffer []byte,
caches *restoreCaches,
itemPath path.Path,
itemData data.Stream,
itemData data.Item,
ctr *count.Bus,
) (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.
metaName := trimmedName + metadata.MetaFileSuffix
@ -701,26 +701,26 @@ type itemRestorer interface {
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(
ctx context.Context,
restoreCfg control.RestoreConfig,
ir itemRestorer,
fibn data.FetchItemByNamer,
name string,
itemData data.Stream,
itemData data.Item,
driveID, parentFolderID string,
collisionKeyToItemID map[string]api.DriveItemIDType,
copyBuffer []byte,
ctr *count.Bus,
) (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()
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)
ss, ok := itemData.(data.StreamSize)
ss, ok := itemData.(data.ItemSize)
if !ok {
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
// again from kopia. Ideally we could just seek the stream
// 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 {
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/ptr"
dataMock "github.com/alcionai/corso/src/internal/data/mock"
"github.com/alcionai/corso/src/internal/m365/graph"
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/tester"
"github.com/alcionai/corso/src/internal/version"
@ -49,7 +51,7 @@ func (suite *RestoreUnitSuite) TestRestoreItem_collisionHandling() {
onCollision control.CollisionPolicy
deleteErr error
expectSkipped assert.BoolAssertionFunc
expectMock func(*testing.T, *mock.RestoreHandler)
expectMock func(*testing.T, *odMock.RestoreHandler)
expectCounts counts
}{
{
@ -57,7 +59,7 @@ func (suite *RestoreUnitSuite) TestRestoreItem_collisionHandling() {
collisionKeys: map[string]api.DriveItemIDType{},
onCollision: control.Copy,
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.False(t, rh.CalledDeleteItem, "new item deleted")
},
@ -68,7 +70,7 @@ func (suite *RestoreUnitSuite) TestRestoreItem_collisionHandling() {
collisionKeys: map[string]api.DriveItemIDType{},
onCollision: control.Replace,
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.False(t, rh.CalledDeleteItem, "new item deleted")
},
@ -79,7 +81,7 @@ func (suite *RestoreUnitSuite) TestRestoreItem_collisionHandling() {
collisionKeys: map[string]api.DriveItemIDType{},
onCollision: control.Skip,
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.False(t, rh.CalledDeleteItem, "new item deleted")
},
@ -88,11 +90,11 @@ func (suite *RestoreUnitSuite) TestRestoreItem_collisionHandling() {
{
name: "collision, copy",
collisionKeys: map[string]api.DriveItemIDType{
mock.DriveItemFileName: {ItemID: mndiID},
odMock.DriveItemFileName: {ItemID: mndiID},
},
onCollision: control.Copy,
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.False(t, rh.CalledDeleteItem, "new item deleted")
},
@ -101,11 +103,11 @@ func (suite *RestoreUnitSuite) TestRestoreItem_collisionHandling() {
{
name: "collision, replace",
collisionKeys: map[string]api.DriveItemIDType{
mock.DriveItemFileName: {ItemID: mndiID},
odMock.DriveItemFileName: {ItemID: mndiID},
},
onCollision: control.Replace,
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.CalledDeleteItem, "new item deleted")
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",
collisionKeys: map[string]api.DriveItemIDType{
mock.DriveItemFileName: {ItemID: "smarf"},
odMock.DriveItemFileName: {ItemID: "smarf"},
},
onCollision: control.Replace,
deleteErr: graph.ErrDeletedInFlight,
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.CalledDeleteItem, "new item deleted")
},
@ -129,11 +131,11 @@ func (suite *RestoreUnitSuite) TestRestoreItem_collisionHandling() {
{
name: "collision, skip",
collisionKeys: map[string]api.DriveItemIDType{
mock.DriveItemFileName: {ItemID: mndiID},
odMock.DriveItemFileName: {ItemID: mndiID},
},
onCollision: control.Skip,
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.CalledDeleteItem, "new item deleted")
},
@ -142,14 +144,14 @@ func (suite *RestoreUnitSuite) TestRestoreItem_collisionHandling() {
{
name: "file-folder collision, copy",
collisionKeys: map[string]api.DriveItemIDType{
mock.DriveItemFileName: {
odMock.DriveItemFileName: {
ItemID: mndiID,
IsFolder: true,
},
},
onCollision: control.Copy,
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.False(t, rh.CalledDeleteItem, "new item deleted")
},
@ -158,14 +160,14 @@ func (suite *RestoreUnitSuite) TestRestoreItem_collisionHandling() {
{
name: "file-folder collision, replace",
collisionKeys: map[string]api.DriveItemIDType{
mock.DriveItemFileName: {
odMock.DriveItemFileName: {
ItemID: mndiID,
IsFolder: true,
},
},
onCollision: control.Replace,
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.False(t, rh.CalledDeleteItem, "new item deleted")
},
@ -174,14 +176,14 @@ func (suite *RestoreUnitSuite) TestRestoreItem_collisionHandling() {
{
name: "file-folder collision, skip",
collisionKeys: map[string]api.DriveItemIDType{
mock.DriveItemFileName: {
odMock.DriveItemFileName: {
ItemID: mndiID,
IsFolder: true,
},
},
onCollision: control.Skip,
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.CalledDeleteItem, "new item deleted")
},
@ -200,7 +202,7 @@ func (suite *RestoreUnitSuite) TestRestoreItem_collisionHandling() {
var (
caches = NewRestoreCaches(nil)
rh = &mock.RestoreHandler{
rh = &odMock.RestoreHandler{
PostItemResp: models.NewDriveItem(),
DeleteItemErr: test.deleteErr,
}
@ -228,18 +230,20 @@ func (suite *RestoreUnitSuite) TestRestoreItem_collisionHandling() {
ctx,
rh,
rcc,
mock.FetchItemByName{
Item: &mock.Data{
Reader: mock.FileRespReadCloser(mock.DriveFileMetaData),
odMock.FetchItemByName{
Item: &dataMock.Item{
Reader: odMock.FileRespReadCloser(odMock.DriveFileMetaData),
ItemInfo: odStub.DriveItemInfo(),
},
},
dp,
"",
make([]byte, graph.CopyBufferSize),
caches,
&mock.Data{
ID: uuid.NewString(),
Reader: mock.FileRespReadCloser(mock.DriveFilePayloadData),
&dataMock.Item{
ItemID: uuid.NewString(),
Reader: odMock.FileRespReadCloser(odMock.DriveFilePayloadData),
ItemInfo: odStub.DriveItemInfo(),
},
nil,
ctr)

View File

@ -38,16 +38,16 @@ const (
var (
_ data.BackupCollection = &Collection{}
_ data.Stream = &Item{}
_ data.StreamInfo = &Item{}
_ data.StreamModTime = &Item{}
_ data.Item = &Item{}
_ data.ItemInfo = &Item{}
_ data.ItemModTime = &Item{}
)
// 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
type Collection struct {
// 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 path.Path
// jobs contain the SharePoint.Site.ListIDs for the associated list(s).
@ -71,7 +71,7 @@ func NewCollection(
c := &Collection{
fullPath: folderPath,
jobs: make([]string, 0),
data: make(chan data.Stream, collectionChannelBufferSize),
data: make(chan data.Item, collectionChannelBufferSize),
client: ac.Sites(),
statusUpdater: statusUpdater,
category: category,
@ -111,7 +111,7 @@ func (sc Collection) DoNotMergeItems() bool {
func (sc *Collection) Items(
ctx context.Context,
errs *fault.Bus,
) <-chan data.Stream {
) <-chan data.Item {
go sc.populate(ctx, errs)
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
}

View File

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

View File

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

View File

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

View File

@ -16,7 +16,7 @@ import (
var (
_ data.BackupCollection = &MetadataCollection{}
_ data.Stream = &MetadataItem{}
_ data.Item = &MetadataItem{}
)
// MetadataCollection in a simple collection that assumes all items to be
@ -134,8 +134,8 @@ func (md MetadataCollection) DoNotMergeItems() bool {
func (md MetadataCollection) Items(
ctx context.Context,
_ *fault.Bus, // not used, just here for interface compliance
) <-chan data.Stream {
res := make(chan data.Stream)
) <-chan data.Item {
res := make(chan data.Item)
go func() {
totalBytes := int64(0)
@ -169,8 +169,8 @@ func (md MetadataCollection) Items(
return res
}
// MetadataItem is an in-memory data.Stream implementation. MetadataItem does
// not implement additional interfaces like data.StreamInfo, so it should only
// MetadataItem is an in-memory data.Item implementation. MetadataItem does
// 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
// 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
}

View File

@ -92,7 +92,7 @@ func (suite *MetadataCollectionUnitSuite) TestItems() {
gotNames := []string{}
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())
if !assert.NoError(t, err, clues.ToCore(err)) {
@ -181,7 +181,7 @@ func (suite *MetadataCollectionUnitSuite) TestMakeMetadataCollection() {
itemCount := 0
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{}
decoder := json.NewDecoder(item.ToReader())

View File

@ -578,10 +578,10 @@ func checkEvent(
func compareExchangeEmail(
t *testing.T,
expected map[string][]byte,
item data.Stream,
item data.Item,
) {
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
}
@ -605,10 +605,10 @@ func compareExchangeContact(
t *testing.T,
colPath path.Path,
expected map[string][]byte,
item data.Stream,
item data.Item,
) {
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
}
@ -633,10 +633,10 @@ func compareExchangeContact(
func compareExchangeEvent(
t *testing.T,
expected map[string][]byte,
item data.Stream,
item data.Item,
) {
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
}
@ -715,13 +715,13 @@ func linkSharesEqual(expected metadata.LinkShare, got metadata.LinkShare) bool {
func compareDriveItem(
t *testing.T,
expected map[string][]byte,
item data.Stream,
item data.Item,
mci m365Stub.ConfigInfo,
rootDir bool,
) bool {
// 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.
if rootDir && item.UUID() == metadata.DirMetaFileSuffix {
if rootDir && item.ID() == metadata.DirMetaFileSuffix {
return false
}
@ -732,7 +732,7 @@ func compareDriveItem(
var (
displayName string
name = item.UUID()
name = item.ID()
isMeta = metadata.HasMetaSuffix(name)
)
@ -867,11 +867,11 @@ func compareItem(
expected map[string][]byte,
service path.ServiceType,
category path.CategoryType,
item data.Stream,
item data.Item,
mci m365Stub.ConfigInfo,
rootDir bool,
) bool {
if mt, ok := item.(data.StreamModTime); ok {
if mt, ok := item.(data.ItemModTime); ok {
assert.NotZero(t, mt.ModTime())
}

View File

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

View File

@ -560,7 +560,7 @@ func parseMetadataCollections(
return nil, false, clues.New("decoding metadata json").WithClues(ctx)
}
switch item.UUID() {
switch item.ID() {
case graph.PreviousPathFileName:
if _, ok := found[category]["path"]; ok {
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
}
func (f failingColl) Items(ctx context.Context, errs *fault.Bus) <-chan data.Stream {
ic := make(chan data.Stream)
func (f failingColl) Items(ctx context.Context, errs *fault.Bus) <-chan data.Item {
ic := make(chan data.Item)
defer close(ic)
errs.AddRecoverable(ctx, assert.AnError)
@ -353,7 +353,7 @@ func (f failingColl) FullPath() path.Path {
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
return nil, nil
}

View File

@ -26,9 +26,9 @@ import (
var (
_ data.BackupCollection = &Collection{}
_ data.Stream = &Stream{}
_ data.StreamInfo = &Stream{}
_ data.StreamModTime = &Stream{}
_ data.Item = &Item{}
_ data.ItemInfo = &Item{}
_ data.ItemModTime = &Item{}
)
const (
@ -41,7 +41,7 @@ const (
type Collection struct {
// 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 map[string]struct{}
@ -92,7 +92,7 @@ func NewCollection(
added: make(map[string]struct{}, 0),
category: category,
ctrl: ctrlOpts,
data: make(chan data.Stream, collectionChannelBufferSize),
data: make(chan data.Item, collectionChannelBufferSize),
doNotMergeItems: doNotMergeItems,
fullPath: curr,
items: items,
@ -109,7 +109,7 @@ func NewCollection(
// Items utility function to asynchronously execute process to fill data channel with
// 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)
return col.data
}
@ -183,7 +183,7 @@ func (col *Collection) streamItems(ctx context.Context, errs *fault.Bus) {
defer wg.Done()
defer func() { <-semaphoreCh }()
col.data <- &Stream{
col.data <- &Item{
id: id,
modTime: time.Now().UTC(), // removed items have no modTime entry.
deleted: true,
@ -247,7 +247,7 @@ func (col *Collection) streamItems(ctx context.Context, errs *fault.Bus) {
info.ParentPath = col.locationPath.String()
col.data <- &Stream{
col.data <- &Item{
id: id,
message: data,
info: info,
@ -292,8 +292,8 @@ func (col *Collection) finishPopulation(
col.statusUpdater(status)
}
// Stream represents a single item retrieved from exchange
type Stream struct {
// Item represents a single item retrieved from exchange
type Item struct {
id string
// 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
@ -308,29 +308,33 @@ type Stream struct {
deleted bool
}
func (od *Stream) UUID() string {
return od.id
func (i *Item) ID() string {
return i.id
}
func (od *Stream) ToReader() io.ReadCloser {
return io.NopCloser(bytes.NewReader(od.message))
func (i *Item) ToReader() io.ReadCloser {
return io.NopCloser(bytes.NewReader(i.message))
}
func (od Stream) Deleted() bool {
return od.deleted
func (i Item) Deleted() bool {
return i.deleted
}
func (od *Stream) Info() details.ItemInfo {
return details.ItemInfo{Exchange: od.info}
func (i *Item) Info() details.ItemInfo {
return details.ItemInfo{Exchange: i.info}
}
func (od *Stream) ModTime() time.Time {
return od.modTime
func (i *Item) ModTime() time.Time {
return i.modTime
}
// NewStream constructor for exchange.Stream object
func NewStream(identifier string, dataBytes []byte, detail details.ExchangeInfo, modTime time.Time) Stream {
return Stream{
func NewItem(
identifier string,
dataBytes []byte,
detail details.ExchangeInfo,
modTime time.Time,
) Item {
return Item{
id: identifier,
message: dataBytes,
info: &detail,

View File

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

View File

@ -10,6 +10,7 @@ import (
"github.com/google/uuid"
"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/fault"
"github.com/alcionai/corso/src/pkg/path"
@ -29,12 +30,7 @@ type DataCollection struct {
DoNotMerge bool
}
var (
_ data.BackupCollection = &DataCollection{}
_ data.Stream = &Data{}
_ data.StreamInfo = &Data{}
_ data.StreamSize = &Data{}
)
var _ data.BackupCollection = &DataCollection{}
func (medc DataCollection) FullPath() path.Path { return medc.fullPath }
@ -118,19 +114,20 @@ func NewContactCollection(pathRepresentation path.Path, numMessagesToReturn int)
func (medc *DataCollection) Items(
ctx context.Context,
_ *fault.Bus, // unused
) <-chan data.Stream {
res := make(chan data.Stream)
) <-chan data.Item {
res := make(chan data.Item)
go func() {
defer close(res)
for i := 0; i < medc.messageCount; i++ {
res <- &Data{
ID: medc.Names[i],
res <- &dataMock.Item{
ItemID: medc.Names[i],
Reader: io.NopCloser(bytes.NewReader(medc.Data[i])),
size: int64(len(medc.Data[i])),
modifiedTime: medc.ModTimes[i],
deleted: medc.DeletedItems[i],
ItemSize: int64(len(medc.Data[i])),
ModifiedTime: medc.ModTimes[i],
DeletedFlag: medc.DeletedItems[i],
ItemInfo: StubMailInfo(),
}
}
}()
@ -138,31 +135,7 @@ func (medc *DataCollection) Items(
return res
}
// TODO: move to data/mock for service-agnostic mocking
// 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 {
func StubMailInfo() details.ItemInfo {
return details.ItemInfo{
Exchange: &details.ExchangeInfo{
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/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/pkg/fault"
"github.com/alcionai/corso/src/pkg/services/m365/api"
@ -55,8 +56,8 @@ func (suite *MockSuite) TestMockExchangeCollectionItemSize() {
buf, err := io.ReadAll(item.ToReader())
assert.NoError(t, err, clues.ToCore(err))
assert.Implements(t, (*data.StreamSize)(nil), item)
s := item.(data.StreamSize)
assert.Implements(t, (*data.ItemSize)(nil), item)
s := item.(data.ItemSize)
assert.Equal(t, int64(len(buf)), s.Size())
}
}
@ -97,22 +98,24 @@ func (suite *MockExchangeDataSuite) TestMockExchangeData() {
table := []struct {
name string
reader *Data
reader *dataMock.Item
check require.ErrorAssertionFunc
}{
{
name: "NoError",
reader: &Data{
ID: id,
reader: &dataMock.Item{
ItemID: id,
Reader: io.NopCloser(bytes.NewReader(itemData)),
ItemInfo: StubMailInfo(),
},
check: require.NoError,
},
{
name: "Error",
reader: &Data{
ID: id,
reader: &dataMock.Item{
ItemID: id,
ReadErr: assert.AnError,
ItemInfo: StubMailInfo(),
},
check: require.Error,
},
@ -122,7 +125,7 @@ func (suite *MockExchangeDataSuite) TestMockExchangeData() {
suite.Run(test.name, func() {
t := suite.T()
assert.Equal(t, id, test.reader.UUID())
assert.Equal(t, id, test.reader.ID())
buf, err := io.ReadAll(test.reader.ToReader())
test.check(t, err, clues.ToCore(err))

View File

@ -168,8 +168,8 @@ func restoreCollection(
return metrics, el.Failure()
}
ictx := clues.Add(ctx, "item_id", itemData.UUID())
trace.Log(ictx, "m365:exchange:restoreCollection:item", itemData.UUID())
ictx := clues.Add(ctx, "item_id", itemData.ID())
trace.Log(ictx, "m365:exchange:restoreCollection:item", itemData.ID())
metrics.Objects++
buf := &bytes.Buffer{}
@ -204,7 +204,7 @@ func restoreCollection(
// 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.
itemPath, err := fullPath.AppendItem(itemData.UUID())
itemPath, err := fullPath.AppendItem(itemData.ID())
if err != nil {
el.AddRecoverable(ctx, clues.Wrap(err, "adding item to collection path").WithClues(ctx))
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
// for OneDrive
for item := range ec.backingCollection.Items(ctx, errs) {
itemUUID := item.UUID()
itemUUID := item.ID()
if isMetadataFile(itemUUID, ec.backupVersion) {
continue
}

View File

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

View File

@ -4,66 +4,12 @@ import (
"bytes"
"context"
"io"
"time"
"github.com/alcionai/corso/src/internal/data"
"github.com/alcionai/corso/src/pkg/backup/details"
"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
// ---------------------------------------------------------------------------
@ -71,11 +17,11 @@ func (er errReader) Read([]byte) (int, error) {
var _ data.FetchItemByNamer = &FetchItemByName{}
type FetchItemByName struct {
Item data.Stream
Item data.Item
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
}

View File

@ -12,6 +12,7 @@ import (
odConsts "github.com/alcionai/corso/src/internal/m365/service/onedrive/consts"
m365Stub "github.com/alcionai/corso/src/internal/m365/stub"
"github.com/alcionai/corso/src/internal/version"
"github.com/alcionai/corso/src/pkg/backup/details"
"github.com/alcionai/corso/src/pkg/path"
)
@ -355,3 +356,13 @@ func ItemWithMetadata(
LookupKey: lookupKey,
}, 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(
ctx context.Context,
service *BetaService,
itemData data.Stream,
itemData data.Item,
siteID, destName string,
) (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()
var (
dii = details.ItemInfo{}
pageID = itemData.UUID()
pageID = itemData.ID()
pageName = pageID
)

View File

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

View File

@ -7,6 +7,7 @@ import (
"golang.org/x/exp/maps"
"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/mock"
"github.com/alcionai/corso/src/internal/m365/resource"
@ -140,13 +141,14 @@ func CollectionsForInfo(
c := mock.RestoreCollection{
Collection: mc,
AuxItems: map[string]data.Stream{},
AuxItems: map[string]data.Item{},
}
for _, aux := range info.AuxItems {
c.AuxItems[aux.Name] = &exchMock.Data{
ID: aux.Name,
c.AuxItems[aux.Name] = &dataMock.Item{
ItemID: aux.Name,
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/internal/common/prefixmatcher"
"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"
"github.com/alcionai/corso/src/internal/kopia"
"github.com/alcionai/corso/src/internal/m365/graph"
"github.com/alcionai/corso/src/internal/m365/mock"
odConsts "github.com/alcionai/corso/src/internal/m365/service/onedrive/consts"
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/model"
"github.com/alcionai/corso/src/internal/operations/inject"
@ -1544,9 +1546,9 @@ func (mbp *mockBackupProducer) Wait() *data.CollectionStats {
func makeBackupCollection(
p path.Path,
locPath *path.Builder,
items []odMock.Data,
items []dataMock.Item,
) data.BackupCollection {
streams := make([]data.Stream, len(items))
streams := make([]data.Item, len(items))
for i := range items {
streams[i] = &items[i]
@ -1586,29 +1588,34 @@ const (
folderID = "folder-id"
)
func makeODMockData(
func makeMockItem(
fileID string,
extData *details.ExtensionData,
modTime time.Time,
del bool,
readErr error,
) odMock.Data {
) dataMock.Item {
rc := odMock.FileRespReadCloser(odMock.DriveFilePayloadData)
if extData != nil {
rc = odMock.FileRespWithExtensions(odMock.DriveFilePayloadData, extData)
}
return odMock.Data{
ID: fileID,
DriveID: driveID,
DriveName: driveName,
dmi := dataMock.Item{
DeletedFlag: del,
ItemID: fileID,
ItemInfo: odStub.DriveItemInfo(),
ItemSize: 100,
ModifiedTime: modTime,
Reader: rc,
ReadErr: readErr,
Sz: 100,
ModifiedTime: modTime,
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
@ -1655,8 +1662,8 @@ func (suite *AssistBackupIntegrationSuite) TestBackupTypesForFailureModes() {
makeBackupCollection(
tmp,
locPath,
[]odMock.Data{
makeODMockData("file1", nil, time.Now(), false, nil),
[]dataMock.Item{
makeMockItem("file1", nil, time.Now(), false, nil),
}),
}
@ -1677,8 +1684,8 @@ func (suite *AssistBackupIntegrationSuite) TestBackupTypesForFailureModes() {
makeBackupCollection(
tmp,
locPath,
[]odMock.Data{
makeODMockData("file1", nil, time.Now(), false, assert.AnError),
[]dataMock.Item{
makeMockItem("file1", nil, time.Now(), false, assert.AnError),
}),
}
return bc
@ -1697,8 +1704,8 @@ func (suite *AssistBackupIntegrationSuite) TestBackupTypesForFailureModes() {
makeBackupCollection(
tmp,
locPath,
[]odMock.Data{
makeODMockData("file1", nil, time.Now(), false, nil),
[]dataMock.Item{
makeMockItem("file1", nil, time.Now(), false, nil),
}),
}
@ -1732,8 +1739,8 @@ func (suite *AssistBackupIntegrationSuite) TestBackupTypesForFailureModes() {
makeBackupCollection(
tmp,
locPath,
[]odMock.Data{
makeODMockData("file1", nil, time.Now(), false, assert.AnError),
[]dataMock.Item{
makeMockItem("file1", nil, time.Now(), false, assert.AnError),
}),
}
@ -1754,9 +1761,9 @@ func (suite *AssistBackupIntegrationSuite) TestBackupTypesForFailureModes() {
makeBackupCollection(
tmp,
locPath,
[]odMock.Data{
makeODMockData("file1", nil, time.Now(), false, nil),
makeODMockData("file2", nil, time.Now(), false, nil),
[]dataMock.Item{
makeMockItem("file1", nil, time.Now(), false, nil),
makeMockItem("file2", nil, time.Now(), false, nil),
}),
}
@ -1790,9 +1797,9 @@ func (suite *AssistBackupIntegrationSuite) TestBackupTypesForFailureModes() {
makeBackupCollection(
tmp,
locPath,
[]odMock.Data{
makeODMockData("file1", nil, time.Now(), false, nil),
makeODMockData("file2", nil, time.Now(), false, assert.AnError),
[]dataMock.Item{
makeMockItem("file1", nil, time.Now(), false, nil),
makeMockItem("file2", nil, time.Now(), false, assert.AnError),
}),
}
@ -1847,7 +1854,6 @@ func (suite *AssistBackupIntegrationSuite) TestBackupTypesForFailureModes() {
err = bo.Run(ctx)
test.expectRunErr(t, err, clues.ToCore(err))
test.expectFaults(t, bo.Errors)
if len(test.expectBackupTag) == 0 {
@ -1933,9 +1939,9 @@ func (suite *AssistBackupIntegrationSuite) TestExtensionsIncrementals() {
makeBackupCollection(
tmp,
locPath,
[]odMock.Data{
makeODMockData("file1", extData[0], T1, false, nil),
makeODMockData("file2", extData[1], T1, false, assert.AnError),
[]dataMock.Item{
makeMockItem("file1", extData[0], T1, false, nil),
makeMockItem("file2", extData[1], T1, false, assert.AnError),
}),
}
@ -1960,10 +1966,10 @@ func (suite *AssistBackupIntegrationSuite) TestExtensionsIncrementals() {
makeBackupCollection(
tmp,
locPath,
[]odMock.Data{
makeODMockData("file1", extData[0], T1, false, nil),
makeODMockData("file2", extData[1], T2, false, nil),
makeODMockData("file3", extData[2], T2, false, assert.AnError),
[]dataMock.Item{
makeMockItem("file1", extData[0], T1, false, nil),
makeMockItem("file2", extData[1], T2, false, nil),
makeMockItem("file3", extData[2], T2, false, assert.AnError),
}),
}
@ -1995,10 +2001,10 @@ func (suite *AssistBackupIntegrationSuite) TestExtensionsIncrementals() {
makeBackupCollection(
tmp,
locPath,
[]odMock.Data{
makeODMockData("file1", extData[0], T1, false, nil),
makeODMockData("file2", extData[1], T2, false, nil),
makeODMockData("file3", extData[2], T3, false, nil),
[]dataMock.Item{
makeMockItem("file1", extData[0], T1, false, nil),
makeMockItem("file2", extData[1], T2, false, nil),
makeMockItem("file3", extData[2], T3, false, nil),
}),
}
@ -2034,10 +2040,10 @@ func (suite *AssistBackupIntegrationSuite) TestExtensionsIncrementals() {
makeBackupCollection(
tmp,
locPath,
[]odMock.Data{
makeODMockData("file1", extData[0], T1, true, nil),
makeODMockData("file2", extData[1], T2, true, nil),
makeODMockData("file3", extData[2], T3, true, nil),
[]dataMock.Item{
makeMockItem("file1", extData[0], T1, true, nil),
makeMockItem("file2", extData[1], T2, true, nil),
makeMockItem("file3", extData[2], T3, true, nil),
}),
}
@ -2056,8 +2062,8 @@ func (suite *AssistBackupIntegrationSuite) TestExtensionsIncrementals() {
makeBackupCollection(
tmp,
locPath,
[]odMock.Data{
makeODMockData("file1", extData[0], T1, false, nil),
[]dataMock.Item{
makeMockItem("file1", extData[0], T1, false, nil),
}),
}
@ -2087,10 +2093,10 @@ func (suite *AssistBackupIntegrationSuite) TestExtensionsIncrementals() {
makeBackupCollection(
tmp,
locPath,
[]odMock.Data{
makeODMockData("file1", extData[0], T1, false, nil),
makeODMockData("file2", extData[1], T2, false, nil),
makeODMockData("file3", extData[2], T3, false, assert.AnError),
[]dataMock.Item{
makeMockItem("file1", extData[0], T1, false, nil),
makeMockItem("file2", extData[1], T2, false, nil),
makeMockItem("file3", extData[2], T3, false, assert.AnError),
}),
}

View File

@ -29,7 +29,7 @@ type mockColl struct {
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
}

View File

@ -323,19 +323,19 @@ func checkMetadataFilesExist(
itemNames := []string{}
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(
t,
s.Size(),
int64(0),
"empty metadata file: %s/%s",
col.FullPath(),
item.UUID(),
item.ID(),
)
itemNames = append(itemNames, item.UUID())
itemNames = append(itemNames, item.ID())
}
assert.ElementsMatchf(

View File

@ -156,10 +156,10 @@ func (dc *streamCollection) DoNotMergeItems() bool {
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
func (dc *streamCollection) Items(context.Context, *fault.Bus) <-chan data.Stream {
items := make(chan data.Stream, 1)
func (dc *streamCollection) Items(context.Context, *fault.Bus) <-chan data.Item {
items := make(chan data.Item, 1)
defer close(items)
items <- dc.item
@ -175,7 +175,7 @@ type streamItem struct {
data []byte
}
func (di *streamItem) UUID() string {
func (di *streamItem) ID() string {
return di.name
}