From c74451b7ef03405a7abcf046cf4d5543cd2597a3 Mon Sep 17 00:00:00 2001 From: guyhardonag Date: Tue, 8 Dec 2020 20:14:36 +0200 Subject: [PATCH 01/14] initial implementation of Graveler --- graveler/gravel.go | 331 +++++++++++++++ graveler/gravel_iterators.go | 368 +++++++++++++++++ graveler/gravel_test.go | 779 +++++++++++++++++++++++++++++++++++ graveler/graveler.go | 3 + 4 files changed, 1481 insertions(+) create mode 100644 graveler/gravel.go create mode 100644 graveler/gravel_iterators.go create mode 100644 graveler/gravel_test.go diff --git a/graveler/gravel.go b/graveler/gravel.go new file mode 100644 index 00000000000..cea5f85e80e --- /dev/null +++ b/graveler/gravel.go @@ -0,0 +1,331 @@ +package graveler + +import ( + "context" + "errors" + "fmt" + "time" + + uuid2 "github.com/google/uuid" +) + +type Gravel struct { + CommittedManager CommittedManager + StagingManager StagingManager + RefManager RefManager +} + +func (r *Gravel) GetRepository(ctx context.Context, repositoryID RepositoryID) (*Repository, error) { + return r.RefManager.GetRepository(ctx, repositoryID) +} + +const ( + initialCommitID = "" //Todo(Guys) decide what should we do with initial commit ID +) + +func (r *Gravel) CreateRepository(ctx context.Context, repositoryID RepositoryID, storageNamespace StorageNamespace, branchID BranchID) (*Repository, error) { + repo := Repository{ + StorageNamespace: storageNamespace, + CreationDate: time.Now(), + DefaultBranchID: branchID, + } + branch := Branch{ + CommitID: initialCommitID, + stagingToken: generateStagingToken(repositoryID, branchID), + } + err := r.RefManager.CreateRepository(ctx, repositoryID, repo, branch) + if err != nil { + return nil, err + } + return &repo, nil +} + +func (r *Gravel) ListRepositories(ctx context.Context, from RepositoryID) (RepositoryIterator, error) { + return r.RefManager.ListRepositories(ctx, from) +} + +func (r *Gravel) DeleteRepository(ctx context.Context, repositoryID RepositoryID) error { + return r.RefManager.DeleteRepository(ctx, repositoryID) +} + +func (r *Gravel) GetCommit(ctx context.Context, repositoryID RepositoryID, commitID CommitID) (*Commit, error) { + return r.RefManager.GetCommit(ctx, repositoryID, commitID) +} +func generateStagingToken(repositoryID RepositoryID, branchID BranchID) StagingToken { + // Todo(Guys): initial implementation, change this + uuid := uuid2.New().String() + return StagingToken(fmt.Sprintf("%s-%s:%s", repositoryID, branchID, uuid)) +} + +func (r *Gravel) CreateBranch(ctx context.Context, repositoryID RepositoryID, branchID BranchID, ref Ref) (*Branch, error) { + // check if branch exists + _, err := r.RefManager.GetBranch(ctx, repositoryID, branchID) + if err != nil && !errors.Is(err, ErrNotFound) { + return nil, err + } + if !errors.Is(err, ErrNotFound) { + return nil, ErrBranchExists + } + + reference, err := r.RefManager.RevParse(ctx, repositoryID, ref) + if err != nil { + return nil, err + } + + newBranch := Branch{ + CommitID: reference.CommitID(), + stagingToken: generateStagingToken(repositoryID, branchID), + } + err = r.RefManager.SetBranch(ctx, repositoryID, branchID, newBranch) + if err != nil { + return nil, err + } + return &newBranch, nil +} + +func (r *Gravel) UpdateBranch(ctx context.Context, repositoryID RepositoryID, branchID BranchID, ref Ref) (*Branch, error) { + reference, err := r.RefManager.RevParse(ctx, repositoryID, ref) + if err != nil { + return nil, err + } + + curBranch, err := r.RefManager.GetBranch(ctx, repositoryID, branchID) + if err != nil { + return nil, err + } + // validate no conflict + // Todo(Guys) return error only on conflicts, currently returns error for any changes on staging + list, err := r.StagingManager.List(ctx, curBranch.stagingToken) + if err != nil { + return nil, err + } + if list.Next() { + return nil, ErrConflictFound + } + + newBranch := Branch{ + CommitID: reference.CommitID(), + stagingToken: curBranch.stagingToken, + } + err = r.RefManager.SetBranch(ctx, repositoryID, branchID, newBranch) + if err != nil { + return nil, err + } + return &newBranch, nil +} + +func (r *Gravel) GetBranch(ctx context.Context, repositoryID RepositoryID, branchID BranchID) (*Branch, error) { + return r.RefManager.GetBranch(ctx, repositoryID, branchID) +} + +func (r *Gravel) Dereference(ctx context.Context, repositoryID RepositoryID, ref Ref) (CommitID, error) { + reference, err := r.RefManager.RevParse(ctx, repositoryID, ref) + if err != nil { + return "", err + } + return reference.CommitID(), nil +} + +func (r *Gravel) Log(ctx context.Context, repositoryID RepositoryID, commitID CommitID) (CommitIterator, error) { + return r.RefManager.Log(ctx, repositoryID, commitID) +} + +func (r *Gravel) ListBranches(ctx context.Context, repositoryID RepositoryID, from BranchID) (BranchIterator, error) { + return r.RefManager.ListBranches(ctx, repositoryID, from) +} + +func (r *Gravel) DeleteBranch(ctx context.Context, repositoryID RepositoryID, branchID BranchID) error { + branch, err := r.RefManager.GetBranch(ctx, repositoryID, branchID) + if err != nil { + return err + } + err = r.StagingManager.Drop(ctx, branch.stagingToken) + if err != nil { + return err + } + return r.RefManager.DeleteBranch(ctx, repositoryID, branchID) +} + +func (r *Gravel) Get(ctx context.Context, repositoryID RepositoryID, ref Ref, key Key) (*Value, error) { + repo, err := r.RefManager.GetRepository(ctx, repositoryID) + if err != nil { + return nil, err + } + reference, err := r.RefManager.RevParse(ctx, repositoryID, ref) + if err != nil { + return nil, err + } + if reference.Type() == ReferenceTypeBranch { + // try to get from staging, if not found proceed to committed + branch := reference.Branch() + value, err := r.StagingManager.Get(ctx, branch.stagingToken, key) + if !errors.Is(err, ErrNotFound) { + if err != nil { + return nil, err + } + if value == nil { + // tombstone + return nil, ErrNotFound + } + return value, nil + } + } + commitID := reference.CommitID() + commit, err := r.RefManager.GetCommit(ctx, repositoryID, commitID) + if err != nil { + return nil, err + } + return r.CommittedManager.Get(ctx, repo.StorageNamespace, commit.TreeID, key) +} + +func (r *Gravel) Set(ctx context.Context, repositoryID RepositoryID, branchID BranchID, key Key, value Value) error { + branch, err := r.GetBranch(ctx, repositoryID, branchID) + if err != nil { + return err + } + return r.StagingManager.Set(ctx, branch.stagingToken, key, value) +} + +func (r *Gravel) Delete(ctx context.Context, repositoryID RepositoryID, branchID BranchID, key Key) error { + branch, err := r.GetBranch(ctx, repositoryID, branchID) + if err != nil { + return err + } + return r.StagingManager.Delete(ctx, branch.stagingToken, key) +} + +func (r *Gravel) List(ctx context.Context, repositoryID RepositoryID, ref Ref, prefix, from, delimiter Key) (ListingIterator, error) { + repo, err := r.RefManager.GetRepository(ctx, repositoryID) + if err != nil { + return nil, err + } + reference, err := r.RefManager.RevParse(ctx, repositoryID, ref) + if err != nil { + return nil, err + } + commit, err := r.RefManager.GetCommit(ctx, repositoryID, reference.CommitID()) + if err != nil { + return nil, err + } + + var listing ListingIterator + committedValues, err := r.CommittedManager.List(ctx, repo.StorageNamespace, commit.TreeID, from) + if err != nil { + return nil, err + } + committedListing := NewListingIter(NewPrefixIterator(committedValues, prefix), delimiter, prefix) + if reference.Type() == ReferenceTypeBranch { + stagingList, err := r.StagingManager.List(ctx, reference.Branch().stagingToken) + if err != nil { + return nil, err + } + listing = NewCombinedIterator(NewListingIter(NewPrefixIterator(stagingList, prefix), delimiter, prefix), committedListing) + } else { + listing = committedListing + } + return listing, nil +} + +func (r *Gravel) Commit(ctx context.Context, repositoryID RepositoryID, branchID BranchID, committer string, message string, metadata Metadata) (CommitID, error) { + panic("implement me") +} + +func (r *Gravel) Reset(ctx context.Context, repositoryID RepositoryID, branchID BranchID) error { + panic("implement me") // waiting for staging reset +} + +func (r *Gravel) Revert(ctx context.Context, repositoryID RepositoryID, branchID BranchID, ref Ref) (CommitID, error) { + panic("implement me") +} + +func (r *Gravel) Merge(ctx context.Context, repositoryID RepositoryID, from Ref, to BranchID) (CommitID, error) { + repo, err := r.RefManager.GetRepository(ctx, repositoryID) + if err != nil { + return "", err + } + + fromCommit, err := r.getCommitRecordFromRef(ctx, repositoryID, from) + if err != nil { + return "", err + } + toCommit, err := r.getCommitRecordFromBranchID(ctx, repositoryID, to) + if err != nil { + return "", err + } + baseCommit, err := r.RefManager.FindMergeBase(ctx, repositoryID, fromCommit.CommitID, toCommit.CommitID) + if err != nil { + return "", err + } + + treeID, err := r.CommittedManager.Merge(ctx, repo.StorageNamespace, fromCommit.TreeID, toCommit.TreeID, baseCommit.TreeID) + if err != nil { + return "", err + } + commit := Commit{ + Committer: "unknown", // Todo(Guys): pass committer or enter default value + Message: "merge message", // Todo(Guys): get merge message + TreeID: treeID, + CreationDate: time.Time{}, + Parents: []CommitID{fromCommit.CommitID, toCommit.CommitID}, + Metadata: nil, + } + return r.RefManager.AddCommit(ctx, repositoryID, commit) +} + +func (r *Gravel) DiffUncommitted(ctx context.Context, repositoryID RepositoryID, branchID BranchID, from Key) (DiffIterator, error) { + repo, err := r.RefManager.GetRepository(ctx, repositoryID) + if err != nil { + return nil, err + } + branch, err := r.RefManager.GetBranch(ctx, repositoryID, branchID) + if err != nil { + return nil, err + } + commit, err := r.RefManager.GetCommit(ctx, repositoryID, branch.CommitID) + if err != nil { + return nil, err + } + valueIterator, err := r.StagingManager.List(ctx, branch.stagingToken) + + return newUncommittedDiffIterator(r.CommittedManager, valueIterator, repo.StorageNamespace, commit.TreeID), nil +} + +func (r *Gravel) getCommitRecordFromRef(ctx context.Context, repositoryID RepositoryID, ref Ref) (*CommitRecord, error) { + reference, err := r.RefManager.RevParse(ctx, repositoryID, ref) + if err != nil { + return nil, err + } + commit, err := r.RefManager.GetCommit(ctx, repositoryID, reference.CommitID()) + if err != nil { + return nil, err + } + return &CommitRecord{ + CommitID: reference.CommitID(), + Commit: commit, + }, nil +} + +func (r *Gravel) getCommitRecordFromBranchID(ctx context.Context, repositoryID RepositoryID, branch BranchID) (*CommitRecord, error) { + return r.getCommitRecordFromRef(ctx, repositoryID, Ref(branch)) +} + +func (r *Gravel) Diff(ctx context.Context, repositoryID RepositoryID, left, right Ref, from Key) (DiffIterator, error) { + repo, err := r.RefManager.GetRepository(ctx, repositoryID) + if err != nil { + return nil, err + } + leftCommit, err := r.getCommitRecordFromRef(ctx, repositoryID, left) + if err != nil { + return nil, err + } + rightCommit, err := r.getCommitRecordFromRef(ctx, repositoryID, right) + if err != nil { + return nil, err + } + baseCommit, err := r.RefManager.FindMergeBase(ctx, repositoryID, leftCommit.CommitID, rightCommit.CommitID) + if err != nil { + return nil, err + } + + return r.CommittedManager.Diff(ctx, repo.StorageNamespace, leftCommit.TreeID, rightCommit.TreeID, baseCommit.TreeID, from) +} diff --git a/graveler/gravel_iterators.go b/graveler/gravel_iterators.go new file mode 100644 index 00000000000..813ff74be79 --- /dev/null +++ b/graveler/gravel_iterators.go @@ -0,0 +1,368 @@ +package graveler + +import ( + "bytes" + "context" + "errors" + + "github.com/treeverse/lakefs/logging" +) + +// PrefixIterator holds a ValueIterator and iterates only over values the their Key starts with the prefix +type PrefixIterator struct { + prefix Key + valueIterator ValueIterator + current *ValueRecord +} + +func NewPrefixIterator(iterator ValueIterator, prefix Key) *PrefixIterator { + iterator.SeekGE(prefix) + return &PrefixIterator{ + prefix: prefix, + valueIterator: iterator, + current: nil, + } +} + +func (p *PrefixIterator) Next() bool { + if !p.valueIterator.Next() { + p.current = nil + return false + } + val := p.valueIterator.Value() + if !bytes.HasPrefix(val.Key, p.prefix) { + p.current = nil + return false + } + p.current = val + return true +} + +func (p *PrefixIterator) SeekGE(id Key) { + if bytes.Compare(id, p.prefix) <= 0 { + p.valueIterator.SeekGE(p.prefix) + return + } + p.valueIterator.SeekGE(id) +} + +func (p *PrefixIterator) Value() *ValueRecord { + return p.current +} + +func (p *PrefixIterator) Err() error { + return p.valueIterator.Err() +} + +func (p *PrefixIterator) Close() { + p.valueIterator.Close() +} + +// ListingIter implements a listing iterator using a ValueIterator +// assumes all values in valueIterator start with prefix +type ListingIter struct { + valueIterator ValueIterator + delimiter Key + nextDelimiter Key + prefix Key + current *Listing + nextFunc func(l *ListingIter) bool + err error +} + +// getFollowingValue returns the following value (i.e will increase the last byte by 1) +// in the following cases will return received value : value is nil, value length is 0, last byte is maximum byte +func getFollowingValue(value []byte) []byte { + if value == nil || len(value) == 0 || value[len(value)-1] == 255 { + return value + } + copiedDelimiter := make([]byte, len(value)) + copy(copiedDelimiter, value) + return append(copiedDelimiter[:len(copiedDelimiter)-1], copiedDelimiter[len(copiedDelimiter)-1]+1) +} + +func NewListingIter(iterator ValueIterator, delimiter, prefix Key) *ListingIter { + var nextDelimiter Key + var nextFunc func(l *ListingIter) bool + if len(delimiter) == 0 { + nextFunc = nextNoDelimiter + } else { + nextFunc = nextWithDelimiter + nextDelimiter = getFollowingValue(delimiter) + } + + return &ListingIter{ + valueIterator: iterator, + delimiter: delimiter, + nextDelimiter: nextDelimiter, + prefix: prefix, + nextFunc: nextFunc, + } +} + +func nextNoDelimiter(l *ListingIter) bool { + hasNext := l.valueIterator.Next() + if !hasNext { + l.current = nil + return false + } + val := l.valueIterator.Value() + l.current = &Listing{ + CommonPrefix: false, + Key: val.Key, + Value: val.Value, + } + return true +} + +func nextWithDelimiter(l *ListingIter) bool { + var hasNext bool + if l.current == nil || !l.current.CommonPrefix { + hasNext = l.valueIterator.Next() + } else { + nextKey := append(l.current.Key, l.nextDelimiter...) + l.valueIterator.SeekGE(nextKey) + hasNext = l.valueIterator.Next() + } + + if hasNext { + nextValue := l.valueIterator.Value() + if !bytes.HasPrefix(nextValue.Key, l.prefix) { + l.current = nil + l.err = ErrUnexpected + return false + } + l.current = l.getListingFromValue(nextValue.Value, nextValue.Key) + } else { + l.current = nil + } + return hasNext +} + +func (l *ListingIter) Next() bool { + return l.nextFunc(l) +} + +func (l *ListingIter) getListingFromValue(value *Value, key Key) *Listing { + relevantPath := key[len(l.prefix):] + delimiterIndex := bytes.Index(relevantPath, l.delimiter) + commonPrefix := delimiterIndex >= 0 + if commonPrefix { + relevantPath = relevantPath[:delimiterIndex] + value = nil + } + newKey := append(l.prefix, relevantPath...) + return &Listing{ + Key: newKey, + CommonPrefix: commonPrefix, + Value: value, + } +} + +func (l *ListingIter) SeekGE(id Key) { + l.current = nil + l.valueIterator.SeekGE(id) +} + +func (l *ListingIter) Value() *Listing { + return l.current +} + +func (l *ListingIter) Err() error { + if l.err != nil { + return l.err + } + return l.valueIterator.Err() +} + +func (l *ListingIter) Close() { + l.valueIterator.Close() +} + +// CombinedIterator iterates over two listing iterators, +// in case of duplication (in values or in errors) returns value in iterA +type CombinedIterator struct { + iterA ListingIterator + iterB ListingIterator + p ListingIterator + err error +} + +func NewCombinedIterator(iterA, iterB ListingIterator) *CombinedIterator { + return &CombinedIterator{ + iterA: iterA, + iterB: iterB, + p: nil, + err: nil, + } +} + +func (c *CombinedIterator) Next() bool { + // call next with the relevant iterators + valA := c.iterA.Value() + valB := c.iterB.Value() + + switch { + case c.p == nil: + // first + c.iterA.Next() + c.iterB.Next() + case valA == nil && valB == nil: + // last + return false + case valA == nil: + c.p = c.iterB + return c.iterB.Next() + case valB == nil: + c.p = c.iterA + return c.iterA.Next() + case bytes.Equal(valA.Key, valB.Key): + c.iterA.Next() + c.iterB.Next() + case bytes.Compare(valA.Key, valB.Key) < 0: + c.iterA.Next() + default: + // value of iterA < value of iterB + c.iterB.Next() + } + + if c.iterA.Err() != nil || c.iterB.Err() != nil { + return false + } + // get current pointer + valA = c.iterA.Value() + valB = c.iterB.Value() + switch { + case valA == nil && valB == nil: + c.p = c.iterA // in order not to be stuck in start state + return false + + case valA == nil: + c.p = c.iterB + return true + + case valB == nil: + c.p = c.iterA + return true + + case bytes.Compare(valA.Key, valB.Key) <= 0: + c.p = c.iterA + return true + default: + c.p = c.iterB + } + return true +} + +func (c *CombinedIterator) SeekGE(id Key) { + c.p = nil + c.iterA.SeekGE(id) + c.iterB.SeekGE(id) +} + +func (c *CombinedIterator) Value() *Listing { + if c.p == nil { + return nil + } + return c.p.Value() +} + +func (c *CombinedIterator) Err() error { + if c.iterA.Err() != nil { + return c.iterA.Err() + } + return c.iterB.Err() +} + +func (c *CombinedIterator) Close() { + c.iterA.Close() + c.iterB.Close() +} + +type UncommittedDiffIterator struct { + committedManager CommittedManager + list ValueIterator + sn StorageNamespace + treeID TreeID + value *Diff + err error +} + +func newUncommittedDiffIterator(manager CommittedManager, list ValueIterator, sn StorageNamespace, treeId TreeID) *UncommittedDiffIterator { + return &UncommittedDiffIterator{ + committedManager: manager, + list: list, + sn: sn, + treeID: treeId, + } +} + +func valueExistsInCommitted(ctx context.Context, committedManager CommittedManager, sn StorageNamespace, treeID TreeID, key Key) (bool, error) { + _, err := committedManager.Get(ctx, sn, treeID, key) + if errors.Is(err, ErrNotFound) { + return false, nil + } else if err != nil { + return false, err + } + return true, nil +} + +func getDiffType(ctx context.Context, committedManager CommittedManager, sn StorageNamespace, treeID TreeID, key Key, tombstone bool) (DiffType, error) { + existsInCommitted, err := valueExistsInCommitted(ctx, committedManager, sn, treeID, key) + if err != nil { + return 0, err + } + var diffType DiffType + diffType = DiffTypeAdded + if tombstone && existsInCommitted { + diffType = DiffTypeRemoved + } + if tombstone && !existsInCommitted { + logging.Default().WithFields(logging.Fields{"treeID": treeID, "stagingToken": sn, "key": key}).Warn("tombstone for file that does not exist") + diffType = DiffTypeRemoved + } + if !tombstone && existsInCommitted { + diffType = DiffTypeChanged + } + if !tombstone && !existsInCommitted { + diffType = DiffTypeAdded + } + return diffType, nil +} + +func (d *UncommittedDiffIterator) Next() bool { + if !d.list.Next() { + d.value = nil + return false + } + val := d.list.Value() + diffType, err := getDiffType(context.Background(), d.committedManager, d.sn, d.treeID, val.Key, val.Value == nil) + if err != nil { + d.value = nil + d.err = err + return false + } + d.value = &Diff{ + Type: diffType, + Key: val.Key, + Value: val.Value, + } + return true +} + +func (d *UncommittedDiffIterator) SeekGE(id Key) { + d.value = nil + d.list.SeekGE(id) +} + +func (d *UncommittedDiffIterator) Value() *Diff { + return d.value +} + +func (d *UncommittedDiffIterator) Err() error { + return d.err +} + +func (d *UncommittedDiffIterator) Close() { + d.list.Close() +} diff --git a/graveler/gravel_test.go b/graveler/gravel_test.go new file mode 100644 index 00000000000..a6aa75da2dd --- /dev/null +++ b/graveler/gravel_test.go @@ -0,0 +1,779 @@ +package graveler_test + +import ( + "bytes" + "context" + "errors" + "testing" + + "github.com/treeverse/lakefs/graveler" + + "github.com/go-test/deep" +) + +type committedMock struct { + Value *graveler.Value + ValueIterator graveler.ValueIterator + diffIterator graveler.DiffIterator + err error + treeID graveler.TreeID +} + +func (c *committedMock) Get(_ context.Context, _ graveler.StorageNamespace, _ graveler.TreeID, _ graveler.Key) (*graveler.Value, error) { + if c.err != nil { + return nil, c.err + } + return c.Value, nil +} + +func (c *committedMock) List(_ context.Context, _ graveler.StorageNamespace, _ graveler.TreeID, _ graveler.Key) (graveler.ValueIterator, error) { + if c.err != nil { + return nil, c.err + } + return c.ValueIterator, nil +} + +func (c *committedMock) Diff(_ context.Context, _ graveler.StorageNamespace, _, _, _ graveler.TreeID, _ graveler.Key) (graveler.DiffIterator, error) { + if c.err != nil { + return nil, c.err + } + return c.diffIterator, nil +} + +func (c *committedMock) Merge(_ context.Context, _ graveler.StorageNamespace, _, _, _ graveler.TreeID) (graveler.TreeID, error) { + if c.err != nil { + return "", c.err + } + return c.treeID, nil +} + +func (c *committedMock) Apply(_ context.Context, _ graveler.StorageNamespace, _ graveler.TreeID, _ graveler.ValueIterator) (graveler.TreeID, error) { + if c.err != nil { + return "", c.err + } + return c.treeID, nil +} + +type stagingMock struct { + err error + Value *graveler.Value + ValueIterator graveler.ValueIterator + stagingToken graveler.StagingToken +} + +func (s *stagingMock) Drop(_ context.Context, _ graveler.StagingToken) error { + if s.err != nil { + return s.err + } + return nil +} + +func (s *stagingMock) Get(_ context.Context, _ graveler.StagingToken, _ graveler.Key) (*graveler.Value, error) { + if s.err != nil { + return nil, s.err + } + return s.Value, nil +} + +func (s *stagingMock) Set(_ context.Context, _ graveler.StagingToken, _ graveler.Key, _ graveler.Value) error { + if s.err != nil { + return s.err + } + return nil +} + +func (s *stagingMock) Delete(_ context.Context, _ graveler.StagingToken, _ graveler.Key) error { + return nil +} + +func (s *stagingMock) List(_ context.Context, _ graveler.StagingToken) (graveler.ValueIterator, error) { + if s.err != nil { + return nil, s.err + } + return s.ValueIterator, nil +} + +func (s *stagingMock) Snapshot(_ context.Context, _ graveler.StagingToken) (graveler.StagingToken, error) { + if s.err != nil { + return "", s.err + } + return s.stagingToken, nil +} + +func (s *stagingMock) ListSnapshot(_ context.Context, _ graveler.StagingToken, _ graveler.Key) (graveler.ValueIterator, error) { + if s.err != nil { + return nil, s.err + } + return s.ValueIterator, nil +} + +type mockRefs struct { + listRepositoriesRes graveler.RepositoryIterator + listBranchesRes graveler.BranchIterator + commitIter graveler.CommitIterator + refType graveler.ReferenceType +} + +const defaultBranchName = graveler.BranchID("master") + +func (m *mockRefs) RevParse(_ context.Context, _ graveler.RepositoryID, _ graveler.Ref) (graveler.Reference, error) { + var branch graveler.BranchID + if m.refType == graveler.ReferenceTypeBranch { + branch = defaultBranchName + } + return newMockReference(m.refType, branch, ""), nil +} + +func (m *mockRefs) GetRepository(_ context.Context, _ graveler.RepositoryID) (*graveler.Repository, error) { + return &graveler.Repository{}, nil +} + +func (m *mockRefs) CreateRepository(_ context.Context, _ graveler.RepositoryID, _ graveler.Repository, _ graveler.Branch) error { + return nil +} + +func (m *mockRefs) ListRepositories(_ context.Context, _ graveler.RepositoryID) (graveler.RepositoryIterator, error) { + return m.listRepositoriesRes, nil +} + +func (m *mockRefs) DeleteRepository(_ context.Context, _ graveler.RepositoryID) error { + return nil +} + +func (m *mockRefs) GetBranch(_ context.Context, _ graveler.RepositoryID, _ graveler.BranchID) (*graveler.Branch, error) { + return &graveler.Branch{}, nil +} + +func (m *mockRefs) SetBranch(_ context.Context, _ graveler.RepositoryID, _ graveler.BranchID, _ graveler.Branch) error { + return nil +} + +func (m *mockRefs) DeleteBranch(_ context.Context, _ graveler.RepositoryID, _ graveler.BranchID) error { + return nil +} + +func (m *mockRefs) ListBranches(_ context.Context, _ graveler.RepositoryID, _ graveler.BranchID) (graveler.BranchIterator, error) { + return m.listBranchesRes, nil +} + +func (m *mockRefs) GetCommit(_ context.Context, _ graveler.RepositoryID, _ graveler.CommitID) (*graveler.Commit, error) { + return &graveler.Commit{}, nil +} + +func (m *mockRefs) AddCommit(_ context.Context, _ graveler.RepositoryID, _ graveler.Commit) (graveler.CommitID, error) { + return "", nil +} + +func (m *mockRefs) FindMergeBase(_ context.Context, _ graveler.RepositoryID, _ ...graveler.CommitID) (*graveler.Commit, error) { + return &graveler.Commit{}, nil +} + +func (m *mockRefs) Log(_ context.Context, _ graveler.RepositoryID, _ graveler.CommitID) (graveler.CommitIterator, error) { + return m.commitIter, nil +} + +type ListingIter struct { + current int + listings []graveler.Listing + err error +} + +func newListingIter(listings []graveler.Listing) *ListingIter { + return &ListingIter{listings: listings, current: -1} +} + +func (r *ListingIter) Next() bool { + r.current++ + return r.current < len(r.listings) +} + +func (r *ListingIter) SeekGE(id graveler.Key) { + for i, listing := range r.listings { + if bytes.Compare(id, listing.Key) >= 0 { + r.current = i - 1 + } + } + r.current = len(r.listings) +} + +func (r *ListingIter) Value() *graveler.Listing { + if r.current < 0 || r.current >= len(r.listings) { + return nil + } + return &r.listings[r.current] +} + +func (r *ListingIter) Err() error { + return r.err +} + +func (r *ListingIter) Close() { + return +} + +type diffIter struct { + current int + records []graveler.Diff + err error +} + +func newDiffIter(records []graveler.Diff) *diffIter { + return &diffIter{records: records, current: -1} +} +func (r *diffIter) Next() bool { + r.current++ + return r.current < len(r.records) +} + +func (r *diffIter) SeekGE(id graveler.Key) { + for i, record := range r.records { + if bytes.Compare(id, record.Key) >= 0 { + r.current = i - 1 + } + } + r.current = len(r.records) +} + +func (r *diffIter) Value() *graveler.Diff { + if r.current < 0 || r.current >= len(r.records) { + return nil + } + return &r.records[r.current] +} + +func (r *diffIter) Err() error { + return r.err +} + +func (r *diffIter) Close() { + return +} + +type ValueIter struct { + current int + records []graveler.ValueRecord + err error +} + +func newValueIter(records []graveler.ValueRecord) *ValueIter { + return &ValueIter{records: records, current: -1} +} +func (r *ValueIter) Next() bool { + r.current++ + return r.current < len(r.records) +} + +func (r *ValueIter) SeekGE(id graveler.Key) { + for i, record := range r.records { + if bytes.Compare(record.Key, id) >= 0 { + r.current = i - 1 + return + } + } + r.current = len(r.records) +} + +func (r *ValueIter) Value() *graveler.ValueRecord { + if r.current < 0 || r.current >= len(r.records) { + return nil + } + return &r.records[r.current] +} + +func (r *ValueIter) Err() error { + return r.err +} + +func (r *ValueIter) Close() { + return +} + +type mockReference struct { + refType graveler.ReferenceType + branch graveler.Branch + commitId graveler.CommitID +} + +// newMockReference returns a mockReference +// if branch parameter is empty branch record will be nil +func newMockReference(refType graveler.ReferenceType, branchID graveler.BranchID, commitId graveler.CommitID) *mockReference { + var branch graveler.Branch + if branchID != "" { + branch = graveler.Branch{CommitID: commitId} + + } + return &mockReference{ + refType: refType, + branch: branch, + commitId: commitId, + } +} +func (m *mockReference) Type() graveler.ReferenceType { + return m.refType +} + +func (m *mockReference) Branch() graveler.Branch { + return m.branch +} + +func (m *mockReference) CommitID() graveler.CommitID { + return m.commitId +} + +func compareListingIterators(got, expected graveler.ListingIterator, t *testing.T) { + for got.Next() { + if !expected.Next() { + t.Fatalf("got next returned true where expected next returned false") + } + if diff := deep.Equal(got.Value(), expected.Value()); diff != nil { + t.Errorf("unexpected diff %s", diff) + } + } + if expected.Next() { + t.Fatalf("expected next returned true where got next returned false") + } +} + +func TestGravel_prefixIterator(t *testing.T) { + tests := []struct { + name string + valueIter graveler.ValueIterator + prefix []byte + seekTo []byte + expectedPrefixIter graveler.ValueIterator + }{ + { + name: "no prefix", + valueIter: newValueIter([]graveler.ValueRecord{{Key: []byte("foo")}}), + expectedPrefixIter: newValueIter([]graveler.ValueRecord{{Key: []byte("foo")}}), + }, + { + name: "no files ", + valueIter: newValueIter([]graveler.ValueRecord{{Key: []byte("other/path/foo")}}), + prefix: []byte("path/"), + expectedPrefixIter: newValueIter([]graveler.ValueRecord{}), + }, + { + name: "one file ", + valueIter: newValueIter([]graveler.ValueRecord{{Key: []byte("path/foo")}}), + prefix: []byte("path/"), + expectedPrefixIter: newValueIter([]graveler.ValueRecord{{Key: []byte("path/foo"), Value: nil}}), + }, + { + name: "one file in prefix ", + prefix: []byte("path/"), + valueIter: newValueIter([]graveler.ValueRecord{{Key: []byte("before/foo")}, {Key: []byte("path/foo")}, {Key: []byte("last/foo")}}), + expectedPrefixIter: newValueIter([]graveler.ValueRecord{{Key: []byte("path/foo"), Value: nil}}), + }, + { + name: "seek before", + prefix: []byte("path/"), + valueIter: newValueIter([]graveler.ValueRecord{{Key: []byte("before/foo")}, {Key: []byte("path/foo")}, {Key: []byte("last/foo")}}), + seekTo: []byte("before/"), + expectedPrefixIter: newValueIter([]graveler.ValueRecord{{Key: []byte("path/foo"), Value: nil}}), + }, + { + name: "seek after", + prefix: []byte("path/"), + valueIter: newValueIter([]graveler.ValueRecord{{Key: []byte("before/foo")}, {Key: []byte("path/foo")}, {Key: []byte("z_after/foo")}}), + seekTo: []byte("z_after/"), + expectedPrefixIter: newValueIter([]graveler.ValueRecord{}), + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + prefixIter := graveler.NewPrefixIterator(tt.valueIter, tt.prefix) + prefixIter.SeekGE(tt.seekTo) + // compare iterators + for prefixIter.Next() { + if !tt.expectedPrefixIter.Next() { + t.Fatalf("listing next returned true where expected listing next returned false") + } + if diff := deep.Equal(prefixIter.Value(), tt.expectedPrefixIter.Value()); diff != nil { + t.Errorf("unexpected diff %s", diff) + } + } + if tt.expectedPrefixIter.Next() { + t.Fatalf("expected listing next returned true where listing next returned false") + } + + }) + } +} + +func TestGravel_listingIterator(t *testing.T) { + tests := []struct { + name string + valueIter graveler.ValueIterator + delimiter []byte + prefix []byte + expectedListingIter graveler.ListingIterator + }{ + { + name: "no file", + valueIter: newValueIter([]graveler.ValueRecord{}), + delimiter: []byte("/"), + prefix: nil, + expectedListingIter: newListingIter([]graveler.Listing{}), + }, + { + name: "one file no delimiter", + valueIter: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo")}}), + delimiter: nil, + prefix: nil, + expectedListingIter: newListingIter([]graveler.Listing{{CommonPrefix: false, Key: graveler.Key("foo")}}), + }, + { + name: "one file", + valueIter: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo")}}), + delimiter: []byte("/"), + prefix: nil, + expectedListingIter: newListingIter([]graveler.Listing{{CommonPrefix: false, Key: graveler.Key("foo")}}), + }, + { + name: "one common prefix", + valueIter: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/bar")}, {Key: graveler.Key("foo/bar2")}}), + delimiter: []byte("/"), + prefix: nil, + expectedListingIter: newListingIter([]graveler.Listing{{CommonPrefix: true, Key: graveler.Key("foo")}}), + }, + { + name: "one common prefix one file", + valueIter: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/bar")}, {Key: graveler.Key("foo/bar2")}, {Key: graveler.Key("foo/bar3")}, {Key: graveler.Key("foo/bar4")}, {Key: graveler.Key("fooFighter")}}), + delimiter: []byte("/"), + prefix: nil, + expectedListingIter: newListingIter([]graveler.Listing{{CommonPrefix: true, Key: graveler.Key("foo")}, {CommonPrefix: false, Key: graveler.Key("fooFighter")}}), + }, + { + name: "one file with prefix", + valueIter: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("path/to/foo")}}), + delimiter: []byte("/"), + prefix: []byte("path/to/"), + expectedListingIter: newListingIter([]graveler.Listing{{CommonPrefix: false, Key: graveler.Key("path/to/foo")}}), + }, + { + name: "one common prefix", + valueIter: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("path/to/foo/bar")}, {Key: graveler.Key("path/to/foo/bar2")}}), + delimiter: []byte("/"), + prefix: []byte("path/to/"), + expectedListingIter: newListingIter([]graveler.Listing{{CommonPrefix: true, Key: graveler.Key("path/to/foo")}}), + }, + { + name: "one common prefix one file", + valueIter: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("path/to/foo/bar")}, {Key: graveler.Key("path/to/foo/bar2")}, {Key: graveler.Key("path/to/foo/bar3")}, {Key: graveler.Key("path/to/foo/bar4")}, {Key: graveler.Key("path/to/fooFighter")}}), + delimiter: []byte("/"), + prefix: []byte("path/to/"), + expectedListingIter: newListingIter([]graveler.Listing{{CommonPrefix: true, Key: graveler.Key("path/to/foo")}, {CommonPrefix: false, Key: graveler.Key("path/to/fooFighter")}}), + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + listingIter := graveler.NewListingIter(tt.valueIter, tt.delimiter, tt.prefix) + compareListingIterators(listingIter, tt.expectedListingIter, t) + }) + } +} + +func TestGravel_List(t *testing.T) { + tests := []struct { + name string + r graveler.Gravel + amount int + delimiter graveler.Key + from graveler.Key + prefix graveler.Key + expectedErr error + expectedHasMore bool + expectedListing graveler.ListingIterator + }{ + { + name: "one committed one staged no paths", + r: graveler.Gravel{ + CommittedManager: &committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo"), Value: &graveler.Value{}}})}, + StagingManager: &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("bar"), Value: &graveler.Value{}}})}, + RefManager: &mockRefs{refType: graveler.ReferenceTypeBranch}, + }, + delimiter: graveler.Key("/"), + prefix: graveler.Key(""), + amount: 10, + expectedListing: newListingIter([]graveler.Listing{{Key: graveler.Key("bar"), Value: &graveler.Value{}}, {Key: graveler.Key("foo"), Value: &graveler.Value{}}}), + }, + { + name: "same path different file", + r: graveler.Gravel{ + CommittedManager: &committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo"), Value: &graveler.Value{Identity: []byte("original")}}})}, + StagingManager: &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo"), Value: &graveler.Value{Identity: []byte("other")}}})}, + RefManager: &mockRefs{refType: graveler.ReferenceTypeBranch}, + }, + delimiter: graveler.Key("/"), + prefix: graveler.Key(""), + amount: 10, + expectedListing: newListingIter([]graveler.Listing{{Key: graveler.Key("foo"), Value: &graveler.Value{Identity: []byte("other")}}}), + }, + { + name: "one committed one staged no paths - with prefix", + r: graveler.Gravel{ + CommittedManager: &committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("prefix/foo"), Value: &graveler.Value{}}})}, + StagingManager: &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("prefix/bar"), Value: &graveler.Value{}}})}, + RefManager: &mockRefs{refType: graveler.ReferenceTypeBranch}, + }, + delimiter: graveler.Key("/"), + prefix: graveler.Key("prefix/"), + amount: 10, + expectedListing: newListingIter([]graveler.Listing{{Key: graveler.Key("prefix/bar"), Value: &graveler.Value{}}, {Key: graveler.Key("prefix/foo"), Value: &graveler.Value{}}}), + }, + { + name: "objects and paths in both committed and staging", + r: graveler.Gravel{ + CommittedManager: &committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("prefix/pathA/foo"), Value: &graveler.Value{}}, {Key: graveler.Key("prefix/pathA/foo2"), Value: &graveler.Value{}}, {Key: graveler.Key("prefix/pathB/foo"), Value: &graveler.Value{}}})}, + StagingManager: &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("prefix/file"), Value: &graveler.Value{}}, {Key: graveler.Key("prefix/pathA/bar"), Value: &graveler.Value{}}, {Key: graveler.Key("prefix/pathB/bar"), Value: &graveler.Value{}}})}, + RefManager: &mockRefs{refType: graveler.ReferenceTypeBranch}, + }, + delimiter: graveler.Key("/"), + prefix: graveler.Key("prefix/"), + amount: 10, + expectedListing: newListingIter([]graveler.Listing{ + { + CommonPrefix: false, + Key: graveler.Key("prefix/file"), + Value: &graveler.Value{}, + }, { + CommonPrefix: true, + Key: graveler.Key("prefix/pathA"), + Value: nil, + }, { + CommonPrefix: true, + Key: graveler.Key("prefix/pathB"), + Value: nil, + }}), + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + listing, err := tt.r.List(context.Background(), "", "", tt.prefix, tt.from, tt.delimiter) + if err != tt.expectedErr { + t.Fatalf("wrong error, expected:%s got:%s ", tt.expectedErr, err) + } + if err != nil { + return // err == tt.expectedErr + } + // compare iterators + compareListingIterators(listing, tt.expectedListing, t) + }) + } +} + +func TestGravel_Get(t *testing.T) { + var ErrTest = errors.New("some kind of err") + tests := []struct { + name string + r graveler.Gravel + expectedValueResult graveler.Value + expectedErr error + }{ + { + name: "commit - exists", + r: graveler.Gravel{ + CommittedManager: &committedMock{Value: &graveler.Value{Identity: []byte("committed")}}, + RefManager: &mockRefs{refType: graveler.ReferenceTypeCommit}, + }, + + expectedValueResult: graveler.Value{Identity: []byte("committed")}, + }, + { + name: "commit - not found", + r: graveler.Gravel{ + CommittedManager: &committedMock{err: graveler.ErrNotFound}, + RefManager: &mockRefs{refType: graveler.ReferenceTypeCommit}, + }, + expectedErr: graveler.ErrNotFound, + }, + { + name: "commit - error", + r: graveler.Gravel{ + CommittedManager: &committedMock{err: ErrTest}, + RefManager: &mockRefs{refType: graveler.ReferenceTypeCommit}, + }, + expectedErr: ErrTest, + }, + { + name: "branch - only staged", + r: graveler.Gravel{ + StagingManager: &stagingMock{Value: &graveler.Value{Identity: []byte("staged")}}, + CommittedManager: &committedMock{err: graveler.ErrNotFound}, + RefManager: &mockRefs{refType: graveler.ReferenceTypeBranch}, + }, + expectedValueResult: graveler.Value{Identity: []byte("staged")}, + }, + { + name: "branch - committed and staged", + r: graveler.Gravel{ + StagingManager: &stagingMock{Value: &graveler.Value{Identity: []byte("staged")}}, + CommittedManager: &committedMock{Value: &graveler.Value{Identity: []byte("committed")}}, + RefManager: &mockRefs{refType: graveler.ReferenceTypeBranch}, + }, + expectedValueResult: graveler.Value{Identity: []byte("staged")}, + }, + { + name: "branch - only committed", + r: graveler.Gravel{ + StagingManager: &stagingMock{err: graveler.ErrNotFound}, + CommittedManager: &committedMock{Value: &graveler.Value{Identity: []byte("committed")}}, + RefManager: &mockRefs{refType: graveler.ReferenceTypeBranch}, + }, + expectedValueResult: graveler.Value{Identity: []byte("committed")}, + }, + { + name: "branch - tombstone", + r: graveler.Gravel{ + StagingManager: &stagingMock{Value: nil}, + CommittedManager: &committedMock{Value: &graveler.Value{Identity: []byte("committed")}}, + RefManager: &mockRefs{refType: graveler.ReferenceTypeBranch}, + }, + expectedErr: graveler.ErrNotFound, + }, + { + name: "branch - staged return error", + r: graveler.Gravel{ + StagingManager: &stagingMock{err: ErrTest}, + CommittedManager: &committedMock{}, + RefManager: &mockRefs{refType: graveler.ReferenceTypeBranch}, + }, + expectedErr: ErrTest, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + Value, err := tt.r.Get(context.Background(), "", "", nil) + if err != tt.expectedErr { + t.Fatalf("wrong error, expected:%s got:%s ", tt.expectedErr, err) + } + if err != nil { + return // err == tt.expected error + } + if string(tt.expectedValueResult.Identity) != string(Value.Identity) { + t.Errorf("wrong Value address, expected:%s got:%s ", tt.expectedValueResult.Identity, Value.Identity) + } + }) + } +} + +func TestGravel_Merge(t *testing.T) { + +} + +func TestGravel_Reset(t *testing.T) { + +} + +func TestGravel_Revert(t *testing.T) { + +} + +func TestGravel_DiffUncommitted(t *testing.T) { + tests := []struct { + name string + r graveler.Gravel + amount int + expectedErr error + expectedHasMore bool + expectedDiff graveler.DiffIterator + }{ + { + name: "no changes", + r: graveler.Gravel{ + CommittedManager: &committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}}), err: graveler.ErrNotFound}, + StagingManager: &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{})}, + RefManager: &mockRefs{}, + }, + amount: 10, + expectedDiff: newDiffIter([]graveler.Diff{}), + }, + { + name: "added one", + r: graveler.Gravel{ + CommittedManager: &committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{}), err: graveler.ErrNotFound}, + StagingManager: &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}})}, + RefManager: &mockRefs{}, + }, + amount: 10, + expectedDiff: newDiffIter([]graveler.Diff{{ + Key: graveler.Key("foo/one"), + Type: graveler.DiffTypeAdded, + Value: &graveler.Value{}, + }}), + }, + { + name: "changed one", + r: graveler.Gravel{ + CommittedManager: &committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}})}, + StagingManager: &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}})}, + RefManager: &mockRefs{}, + }, + amount: 10, + expectedDiff: newDiffIter([]graveler.Diff{{ + Key: graveler.Key("foo/one"), + Type: graveler.DiffTypeChanged, + Value: &graveler.Value{}, + }}), + }, + { + name: "removed one", + r: graveler.Gravel{ + CommittedManager: &committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}})}, + StagingManager: &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: nil}})}, + RefManager: &mockRefs{}, + }, + amount: 10, + expectedDiff: newDiffIter([]graveler.Diff{{ + Key: graveler.Key("foo/one"), + Type: graveler.DiffTypeRemoved, + }}), + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + diff, err := tt.r.DiffUncommitted(context.Background(), "repo", "branch", graveler.Key("from")) + if err != tt.expectedErr { + t.Fatalf("wrong error, expected:%s got:%s ", tt.expectedErr, err) + } + if err != nil { + return // err == tt.expectedErr + } + + // compare iterators + for diff.Next() { + if !tt.expectedDiff.Next() { + t.Fatalf("listing next returned true where expected listing next returned false") + } + if diff := deep.Equal(diff.Value(), tt.expectedDiff.Value()); diff != nil { + t.Errorf("unexpected diff %s", diff) + } + } + if tt.expectedDiff.Next() { + t.Fatalf("expected listing next returned true where listing next returned false") + } + }) + } +} + +func TestGravel_UpdateBranch(t *testing.T) { + gravel := graveler.Gravel{ + CommittedManager: nil, + StagingManager: &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}})}, + RefManager: &mockRefs{}, + } + _, err := gravel.UpdateBranch(context.Background(), "", "", "") + if !errors.Is(err, graveler.ErrConflictFound) { + t.Fatal("expected update to fail on conflict") + } + + gravel.StagingManager = &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{})} + _, err = gravel.UpdateBranch(context.Background(), "", "", "") + if err != nil { + t.Fatal("did not expect to get error") + } +} diff --git a/graveler/graveler.go b/graveler/graveler.go index cb4d2af40eb..a7d40504189 100644 --- a/graveler/graveler.go +++ b/graveler/graveler.go @@ -422,6 +422,9 @@ var ( ErrInvalidCommitID = fmt.Errorf("commit id: %w", ErrInvalidValue) ErrCommitNotFound = fmt.Errorf("commit: %w", ErrNotFound) ErrCommitIDAmbiguous = fmt.Errorf("commit ID is ambiguous: %w", ErrNotFound) + ErrConflictFound = errors.New("conflict found") + ErrBranchExists = errors.New("branch already exists") + ErrUnexpected = errors.New("unexpected error") ) func NewRepositoryID(id string) (RepositoryID, error) { From ed6c13b63f6ae7db1943c9e455caa2d030810bfa Mon Sep 17 00:00:00 2001 From: guyhardonag Date: Wed, 9 Dec 2020 12:20:24 +0200 Subject: [PATCH 02/14] Change Gravel name to graveler and Extract iterators to separate files --- export/export_handler.go | 2 +- graveler/combined_iterator.go | 103 +++++ graveler/gravel.go | 331 ---------------- graveler/gravel_iterators.go | 368 ------------------ graveler/graveler.go | 333 +++++++++++++++- graveler/{gravel_test.go => graveler_test.go} | 163 ++++---- graveler/listing_iterator.go | 124 ++++++ graveler/prefix_iterator.go | 53 +++ graveler/uncommitted_iterator.go | 96 +++++ parade/action_manager.go | 2 +- 10 files changed, 779 insertions(+), 796 deletions(-) create mode 100644 graveler/combined_iterator.go delete mode 100644 graveler/gravel.go delete mode 100644 graveler/gravel_iterators.go rename graveler/{gravel_test.go => graveler_test.go} (78%) create mode 100644 graveler/listing_iterator.go create mode 100644 graveler/prefix_iterator.go create mode 100644 graveler/uncommitted_iterator.go diff --git a/export/export_handler.go b/export/export_handler.go index 014f5027a30..03f9f6d232a 100644 --- a/export/export_handler.go +++ b/export/export_handler.go @@ -92,7 +92,7 @@ func (h *Handler) generateTasks(startData StartData, config catalog.ExportConfig if diffFromBase { diffs, hasMore, err = getDiffFromBase(context.Background(), startData.Repo, startData.ToCommitRef, after, limit, h.cataloger) } else { - // Todo(guys) change this to work with diff iterator once it is available outside of cataloger + // TODO(guys) change this to work with diff iterator once it is available outside of cataloger diffs, hasMore, err = h.cataloger.Diff(context.Background(), startData.Repo, startData.ToCommitRef, startData.FromCommitRef, catalog.DiffParams{ Limit: limit, After: after, diff --git a/graveler/combined_iterator.go b/graveler/combined_iterator.go new file mode 100644 index 00000000000..0b87d4ff28e --- /dev/null +++ b/graveler/combined_iterator.go @@ -0,0 +1,103 @@ +package graveler + +import "bytes" + +// CombinedIterator iterates over two listing iterators, +// in case of duplication (in values or in errors) returns value in iterA +type CombinedIterator struct { + iterA ListingIterator + iterB ListingIterator + p ListingIterator + err error +} + +func NewCombinedIterator(iterA, iterB ListingIterator) *CombinedIterator { + return &CombinedIterator{ + iterA: iterA, + iterB: iterB, + p: nil, + err: nil, + } +} + +func (c *CombinedIterator) Next() bool { + // call next with the relevant iterators + valA := c.iterA.Value() + valB := c.iterB.Value() + + switch { + case c.p == nil: + // first + c.iterA.Next() + c.iterB.Next() + case valA == nil && valB == nil: + // last + return false + case valA == nil: + c.p = c.iterB + return c.iterB.Next() + case valB == nil: + c.p = c.iterA + return c.iterA.Next() + case bytes.Equal(valA.Key, valB.Key): + c.iterA.Next() + c.iterB.Next() + case bytes.Compare(valA.Key, valB.Key) < 0: + c.iterA.Next() + default: + // value of iterA < value of iterB + c.iterB.Next() + } + + if c.iterA.Err() != nil || c.iterB.Err() != nil { + return false + } + // get current pointer + valA = c.iterA.Value() + valB = c.iterB.Value() + switch { + case valA == nil && valB == nil: + c.p = c.iterA // in order not to be stuck in start state + return false + + case valA == nil: + c.p = c.iterB + return true + + case valB == nil: + c.p = c.iterA + return true + + case bytes.Compare(valA.Key, valB.Key) <= 0: + c.p = c.iterA + return true + default: + c.p = c.iterB + } + return true +} + +func (c *CombinedIterator) SeekGE(id Key) { + c.p = nil + c.iterA.SeekGE(id) + c.iterB.SeekGE(id) +} + +func (c *CombinedIterator) Value() *Listing { + if c.p == nil { + return nil + } + return c.p.Value() +} + +func (c *CombinedIterator) Err() error { + if c.iterA.Err() != nil { + return c.iterA.Err() + } + return c.iterB.Err() +} + +func (c *CombinedIterator) Close() { + c.iterA.Close() + c.iterB.Close() +} diff --git a/graveler/gravel.go b/graveler/gravel.go deleted file mode 100644 index cea5f85e80e..00000000000 --- a/graveler/gravel.go +++ /dev/null @@ -1,331 +0,0 @@ -package graveler - -import ( - "context" - "errors" - "fmt" - "time" - - uuid2 "github.com/google/uuid" -) - -type Gravel struct { - CommittedManager CommittedManager - StagingManager StagingManager - RefManager RefManager -} - -func (r *Gravel) GetRepository(ctx context.Context, repositoryID RepositoryID) (*Repository, error) { - return r.RefManager.GetRepository(ctx, repositoryID) -} - -const ( - initialCommitID = "" //Todo(Guys) decide what should we do with initial commit ID -) - -func (r *Gravel) CreateRepository(ctx context.Context, repositoryID RepositoryID, storageNamespace StorageNamespace, branchID BranchID) (*Repository, error) { - repo := Repository{ - StorageNamespace: storageNamespace, - CreationDate: time.Now(), - DefaultBranchID: branchID, - } - branch := Branch{ - CommitID: initialCommitID, - stagingToken: generateStagingToken(repositoryID, branchID), - } - err := r.RefManager.CreateRepository(ctx, repositoryID, repo, branch) - if err != nil { - return nil, err - } - return &repo, nil -} - -func (r *Gravel) ListRepositories(ctx context.Context, from RepositoryID) (RepositoryIterator, error) { - return r.RefManager.ListRepositories(ctx, from) -} - -func (r *Gravel) DeleteRepository(ctx context.Context, repositoryID RepositoryID) error { - return r.RefManager.DeleteRepository(ctx, repositoryID) -} - -func (r *Gravel) GetCommit(ctx context.Context, repositoryID RepositoryID, commitID CommitID) (*Commit, error) { - return r.RefManager.GetCommit(ctx, repositoryID, commitID) -} -func generateStagingToken(repositoryID RepositoryID, branchID BranchID) StagingToken { - // Todo(Guys): initial implementation, change this - uuid := uuid2.New().String() - return StagingToken(fmt.Sprintf("%s-%s:%s", repositoryID, branchID, uuid)) -} - -func (r *Gravel) CreateBranch(ctx context.Context, repositoryID RepositoryID, branchID BranchID, ref Ref) (*Branch, error) { - // check if branch exists - _, err := r.RefManager.GetBranch(ctx, repositoryID, branchID) - if err != nil && !errors.Is(err, ErrNotFound) { - return nil, err - } - if !errors.Is(err, ErrNotFound) { - return nil, ErrBranchExists - } - - reference, err := r.RefManager.RevParse(ctx, repositoryID, ref) - if err != nil { - return nil, err - } - - newBranch := Branch{ - CommitID: reference.CommitID(), - stagingToken: generateStagingToken(repositoryID, branchID), - } - err = r.RefManager.SetBranch(ctx, repositoryID, branchID, newBranch) - if err != nil { - return nil, err - } - return &newBranch, nil -} - -func (r *Gravel) UpdateBranch(ctx context.Context, repositoryID RepositoryID, branchID BranchID, ref Ref) (*Branch, error) { - reference, err := r.RefManager.RevParse(ctx, repositoryID, ref) - if err != nil { - return nil, err - } - - curBranch, err := r.RefManager.GetBranch(ctx, repositoryID, branchID) - if err != nil { - return nil, err - } - // validate no conflict - // Todo(Guys) return error only on conflicts, currently returns error for any changes on staging - list, err := r.StagingManager.List(ctx, curBranch.stagingToken) - if err != nil { - return nil, err - } - if list.Next() { - return nil, ErrConflictFound - } - - newBranch := Branch{ - CommitID: reference.CommitID(), - stagingToken: curBranch.stagingToken, - } - err = r.RefManager.SetBranch(ctx, repositoryID, branchID, newBranch) - if err != nil { - return nil, err - } - return &newBranch, nil -} - -func (r *Gravel) GetBranch(ctx context.Context, repositoryID RepositoryID, branchID BranchID) (*Branch, error) { - return r.RefManager.GetBranch(ctx, repositoryID, branchID) -} - -func (r *Gravel) Dereference(ctx context.Context, repositoryID RepositoryID, ref Ref) (CommitID, error) { - reference, err := r.RefManager.RevParse(ctx, repositoryID, ref) - if err != nil { - return "", err - } - return reference.CommitID(), nil -} - -func (r *Gravel) Log(ctx context.Context, repositoryID RepositoryID, commitID CommitID) (CommitIterator, error) { - return r.RefManager.Log(ctx, repositoryID, commitID) -} - -func (r *Gravel) ListBranches(ctx context.Context, repositoryID RepositoryID, from BranchID) (BranchIterator, error) { - return r.RefManager.ListBranches(ctx, repositoryID, from) -} - -func (r *Gravel) DeleteBranch(ctx context.Context, repositoryID RepositoryID, branchID BranchID) error { - branch, err := r.RefManager.GetBranch(ctx, repositoryID, branchID) - if err != nil { - return err - } - err = r.StagingManager.Drop(ctx, branch.stagingToken) - if err != nil { - return err - } - return r.RefManager.DeleteBranch(ctx, repositoryID, branchID) -} - -func (r *Gravel) Get(ctx context.Context, repositoryID RepositoryID, ref Ref, key Key) (*Value, error) { - repo, err := r.RefManager.GetRepository(ctx, repositoryID) - if err != nil { - return nil, err - } - reference, err := r.RefManager.RevParse(ctx, repositoryID, ref) - if err != nil { - return nil, err - } - if reference.Type() == ReferenceTypeBranch { - // try to get from staging, if not found proceed to committed - branch := reference.Branch() - value, err := r.StagingManager.Get(ctx, branch.stagingToken, key) - if !errors.Is(err, ErrNotFound) { - if err != nil { - return nil, err - } - if value == nil { - // tombstone - return nil, ErrNotFound - } - return value, nil - } - } - commitID := reference.CommitID() - commit, err := r.RefManager.GetCommit(ctx, repositoryID, commitID) - if err != nil { - return nil, err - } - return r.CommittedManager.Get(ctx, repo.StorageNamespace, commit.TreeID, key) -} - -func (r *Gravel) Set(ctx context.Context, repositoryID RepositoryID, branchID BranchID, key Key, value Value) error { - branch, err := r.GetBranch(ctx, repositoryID, branchID) - if err != nil { - return err - } - return r.StagingManager.Set(ctx, branch.stagingToken, key, value) -} - -func (r *Gravel) Delete(ctx context.Context, repositoryID RepositoryID, branchID BranchID, key Key) error { - branch, err := r.GetBranch(ctx, repositoryID, branchID) - if err != nil { - return err - } - return r.StagingManager.Delete(ctx, branch.stagingToken, key) -} - -func (r *Gravel) List(ctx context.Context, repositoryID RepositoryID, ref Ref, prefix, from, delimiter Key) (ListingIterator, error) { - repo, err := r.RefManager.GetRepository(ctx, repositoryID) - if err != nil { - return nil, err - } - reference, err := r.RefManager.RevParse(ctx, repositoryID, ref) - if err != nil { - return nil, err - } - commit, err := r.RefManager.GetCommit(ctx, repositoryID, reference.CommitID()) - if err != nil { - return nil, err - } - - var listing ListingIterator - committedValues, err := r.CommittedManager.List(ctx, repo.StorageNamespace, commit.TreeID, from) - if err != nil { - return nil, err - } - committedListing := NewListingIter(NewPrefixIterator(committedValues, prefix), delimiter, prefix) - if reference.Type() == ReferenceTypeBranch { - stagingList, err := r.StagingManager.List(ctx, reference.Branch().stagingToken) - if err != nil { - return nil, err - } - listing = NewCombinedIterator(NewListingIter(NewPrefixIterator(stagingList, prefix), delimiter, prefix), committedListing) - } else { - listing = committedListing - } - return listing, nil -} - -func (r *Gravel) Commit(ctx context.Context, repositoryID RepositoryID, branchID BranchID, committer string, message string, metadata Metadata) (CommitID, error) { - panic("implement me") -} - -func (r *Gravel) Reset(ctx context.Context, repositoryID RepositoryID, branchID BranchID) error { - panic("implement me") // waiting for staging reset -} - -func (r *Gravel) Revert(ctx context.Context, repositoryID RepositoryID, branchID BranchID, ref Ref) (CommitID, error) { - panic("implement me") -} - -func (r *Gravel) Merge(ctx context.Context, repositoryID RepositoryID, from Ref, to BranchID) (CommitID, error) { - repo, err := r.RefManager.GetRepository(ctx, repositoryID) - if err != nil { - return "", err - } - - fromCommit, err := r.getCommitRecordFromRef(ctx, repositoryID, from) - if err != nil { - return "", err - } - toCommit, err := r.getCommitRecordFromBranchID(ctx, repositoryID, to) - if err != nil { - return "", err - } - baseCommit, err := r.RefManager.FindMergeBase(ctx, repositoryID, fromCommit.CommitID, toCommit.CommitID) - if err != nil { - return "", err - } - - treeID, err := r.CommittedManager.Merge(ctx, repo.StorageNamespace, fromCommit.TreeID, toCommit.TreeID, baseCommit.TreeID) - if err != nil { - return "", err - } - commit := Commit{ - Committer: "unknown", // Todo(Guys): pass committer or enter default value - Message: "merge message", // Todo(Guys): get merge message - TreeID: treeID, - CreationDate: time.Time{}, - Parents: []CommitID{fromCommit.CommitID, toCommit.CommitID}, - Metadata: nil, - } - return r.RefManager.AddCommit(ctx, repositoryID, commit) -} - -func (r *Gravel) DiffUncommitted(ctx context.Context, repositoryID RepositoryID, branchID BranchID, from Key) (DiffIterator, error) { - repo, err := r.RefManager.GetRepository(ctx, repositoryID) - if err != nil { - return nil, err - } - branch, err := r.RefManager.GetBranch(ctx, repositoryID, branchID) - if err != nil { - return nil, err - } - commit, err := r.RefManager.GetCommit(ctx, repositoryID, branch.CommitID) - if err != nil { - return nil, err - } - valueIterator, err := r.StagingManager.List(ctx, branch.stagingToken) - - return newUncommittedDiffIterator(r.CommittedManager, valueIterator, repo.StorageNamespace, commit.TreeID), nil -} - -func (r *Gravel) getCommitRecordFromRef(ctx context.Context, repositoryID RepositoryID, ref Ref) (*CommitRecord, error) { - reference, err := r.RefManager.RevParse(ctx, repositoryID, ref) - if err != nil { - return nil, err - } - commit, err := r.RefManager.GetCommit(ctx, repositoryID, reference.CommitID()) - if err != nil { - return nil, err - } - return &CommitRecord{ - CommitID: reference.CommitID(), - Commit: commit, - }, nil -} - -func (r *Gravel) getCommitRecordFromBranchID(ctx context.Context, repositoryID RepositoryID, branch BranchID) (*CommitRecord, error) { - return r.getCommitRecordFromRef(ctx, repositoryID, Ref(branch)) -} - -func (r *Gravel) Diff(ctx context.Context, repositoryID RepositoryID, left, right Ref, from Key) (DiffIterator, error) { - repo, err := r.RefManager.GetRepository(ctx, repositoryID) - if err != nil { - return nil, err - } - leftCommit, err := r.getCommitRecordFromRef(ctx, repositoryID, left) - if err != nil { - return nil, err - } - rightCommit, err := r.getCommitRecordFromRef(ctx, repositoryID, right) - if err != nil { - return nil, err - } - baseCommit, err := r.RefManager.FindMergeBase(ctx, repositoryID, leftCommit.CommitID, rightCommit.CommitID) - if err != nil { - return nil, err - } - - return r.CommittedManager.Diff(ctx, repo.StorageNamespace, leftCommit.TreeID, rightCommit.TreeID, baseCommit.TreeID, from) -} diff --git a/graveler/gravel_iterators.go b/graveler/gravel_iterators.go deleted file mode 100644 index 813ff74be79..00000000000 --- a/graveler/gravel_iterators.go +++ /dev/null @@ -1,368 +0,0 @@ -package graveler - -import ( - "bytes" - "context" - "errors" - - "github.com/treeverse/lakefs/logging" -) - -// PrefixIterator holds a ValueIterator and iterates only over values the their Key starts with the prefix -type PrefixIterator struct { - prefix Key - valueIterator ValueIterator - current *ValueRecord -} - -func NewPrefixIterator(iterator ValueIterator, prefix Key) *PrefixIterator { - iterator.SeekGE(prefix) - return &PrefixIterator{ - prefix: prefix, - valueIterator: iterator, - current: nil, - } -} - -func (p *PrefixIterator) Next() bool { - if !p.valueIterator.Next() { - p.current = nil - return false - } - val := p.valueIterator.Value() - if !bytes.HasPrefix(val.Key, p.prefix) { - p.current = nil - return false - } - p.current = val - return true -} - -func (p *PrefixIterator) SeekGE(id Key) { - if bytes.Compare(id, p.prefix) <= 0 { - p.valueIterator.SeekGE(p.prefix) - return - } - p.valueIterator.SeekGE(id) -} - -func (p *PrefixIterator) Value() *ValueRecord { - return p.current -} - -func (p *PrefixIterator) Err() error { - return p.valueIterator.Err() -} - -func (p *PrefixIterator) Close() { - p.valueIterator.Close() -} - -// ListingIter implements a listing iterator using a ValueIterator -// assumes all values in valueIterator start with prefix -type ListingIter struct { - valueIterator ValueIterator - delimiter Key - nextDelimiter Key - prefix Key - current *Listing - nextFunc func(l *ListingIter) bool - err error -} - -// getFollowingValue returns the following value (i.e will increase the last byte by 1) -// in the following cases will return received value : value is nil, value length is 0, last byte is maximum byte -func getFollowingValue(value []byte) []byte { - if value == nil || len(value) == 0 || value[len(value)-1] == 255 { - return value - } - copiedDelimiter := make([]byte, len(value)) - copy(copiedDelimiter, value) - return append(copiedDelimiter[:len(copiedDelimiter)-1], copiedDelimiter[len(copiedDelimiter)-1]+1) -} - -func NewListingIter(iterator ValueIterator, delimiter, prefix Key) *ListingIter { - var nextDelimiter Key - var nextFunc func(l *ListingIter) bool - if len(delimiter) == 0 { - nextFunc = nextNoDelimiter - } else { - nextFunc = nextWithDelimiter - nextDelimiter = getFollowingValue(delimiter) - } - - return &ListingIter{ - valueIterator: iterator, - delimiter: delimiter, - nextDelimiter: nextDelimiter, - prefix: prefix, - nextFunc: nextFunc, - } -} - -func nextNoDelimiter(l *ListingIter) bool { - hasNext := l.valueIterator.Next() - if !hasNext { - l.current = nil - return false - } - val := l.valueIterator.Value() - l.current = &Listing{ - CommonPrefix: false, - Key: val.Key, - Value: val.Value, - } - return true -} - -func nextWithDelimiter(l *ListingIter) bool { - var hasNext bool - if l.current == nil || !l.current.CommonPrefix { - hasNext = l.valueIterator.Next() - } else { - nextKey := append(l.current.Key, l.nextDelimiter...) - l.valueIterator.SeekGE(nextKey) - hasNext = l.valueIterator.Next() - } - - if hasNext { - nextValue := l.valueIterator.Value() - if !bytes.HasPrefix(nextValue.Key, l.prefix) { - l.current = nil - l.err = ErrUnexpected - return false - } - l.current = l.getListingFromValue(nextValue.Value, nextValue.Key) - } else { - l.current = nil - } - return hasNext -} - -func (l *ListingIter) Next() bool { - return l.nextFunc(l) -} - -func (l *ListingIter) getListingFromValue(value *Value, key Key) *Listing { - relevantPath := key[len(l.prefix):] - delimiterIndex := bytes.Index(relevantPath, l.delimiter) - commonPrefix := delimiterIndex >= 0 - if commonPrefix { - relevantPath = relevantPath[:delimiterIndex] - value = nil - } - newKey := append(l.prefix, relevantPath...) - return &Listing{ - Key: newKey, - CommonPrefix: commonPrefix, - Value: value, - } -} - -func (l *ListingIter) SeekGE(id Key) { - l.current = nil - l.valueIterator.SeekGE(id) -} - -func (l *ListingIter) Value() *Listing { - return l.current -} - -func (l *ListingIter) Err() error { - if l.err != nil { - return l.err - } - return l.valueIterator.Err() -} - -func (l *ListingIter) Close() { - l.valueIterator.Close() -} - -// CombinedIterator iterates over two listing iterators, -// in case of duplication (in values or in errors) returns value in iterA -type CombinedIterator struct { - iterA ListingIterator - iterB ListingIterator - p ListingIterator - err error -} - -func NewCombinedIterator(iterA, iterB ListingIterator) *CombinedIterator { - return &CombinedIterator{ - iterA: iterA, - iterB: iterB, - p: nil, - err: nil, - } -} - -func (c *CombinedIterator) Next() bool { - // call next with the relevant iterators - valA := c.iterA.Value() - valB := c.iterB.Value() - - switch { - case c.p == nil: - // first - c.iterA.Next() - c.iterB.Next() - case valA == nil && valB == nil: - // last - return false - case valA == nil: - c.p = c.iterB - return c.iterB.Next() - case valB == nil: - c.p = c.iterA - return c.iterA.Next() - case bytes.Equal(valA.Key, valB.Key): - c.iterA.Next() - c.iterB.Next() - case bytes.Compare(valA.Key, valB.Key) < 0: - c.iterA.Next() - default: - // value of iterA < value of iterB - c.iterB.Next() - } - - if c.iterA.Err() != nil || c.iterB.Err() != nil { - return false - } - // get current pointer - valA = c.iterA.Value() - valB = c.iterB.Value() - switch { - case valA == nil && valB == nil: - c.p = c.iterA // in order not to be stuck in start state - return false - - case valA == nil: - c.p = c.iterB - return true - - case valB == nil: - c.p = c.iterA - return true - - case bytes.Compare(valA.Key, valB.Key) <= 0: - c.p = c.iterA - return true - default: - c.p = c.iterB - } - return true -} - -func (c *CombinedIterator) SeekGE(id Key) { - c.p = nil - c.iterA.SeekGE(id) - c.iterB.SeekGE(id) -} - -func (c *CombinedIterator) Value() *Listing { - if c.p == nil { - return nil - } - return c.p.Value() -} - -func (c *CombinedIterator) Err() error { - if c.iterA.Err() != nil { - return c.iterA.Err() - } - return c.iterB.Err() -} - -func (c *CombinedIterator) Close() { - c.iterA.Close() - c.iterB.Close() -} - -type UncommittedDiffIterator struct { - committedManager CommittedManager - list ValueIterator - sn StorageNamespace - treeID TreeID - value *Diff - err error -} - -func newUncommittedDiffIterator(manager CommittedManager, list ValueIterator, sn StorageNamespace, treeId TreeID) *UncommittedDiffIterator { - return &UncommittedDiffIterator{ - committedManager: manager, - list: list, - sn: sn, - treeID: treeId, - } -} - -func valueExistsInCommitted(ctx context.Context, committedManager CommittedManager, sn StorageNamespace, treeID TreeID, key Key) (bool, error) { - _, err := committedManager.Get(ctx, sn, treeID, key) - if errors.Is(err, ErrNotFound) { - return false, nil - } else if err != nil { - return false, err - } - return true, nil -} - -func getDiffType(ctx context.Context, committedManager CommittedManager, sn StorageNamespace, treeID TreeID, key Key, tombstone bool) (DiffType, error) { - existsInCommitted, err := valueExistsInCommitted(ctx, committedManager, sn, treeID, key) - if err != nil { - return 0, err - } - var diffType DiffType - diffType = DiffTypeAdded - if tombstone && existsInCommitted { - diffType = DiffTypeRemoved - } - if tombstone && !existsInCommitted { - logging.Default().WithFields(logging.Fields{"treeID": treeID, "stagingToken": sn, "key": key}).Warn("tombstone for file that does not exist") - diffType = DiffTypeRemoved - } - if !tombstone && existsInCommitted { - diffType = DiffTypeChanged - } - if !tombstone && !existsInCommitted { - diffType = DiffTypeAdded - } - return diffType, nil -} - -func (d *UncommittedDiffIterator) Next() bool { - if !d.list.Next() { - d.value = nil - return false - } - val := d.list.Value() - diffType, err := getDiffType(context.Background(), d.committedManager, d.sn, d.treeID, val.Key, val.Value == nil) - if err != nil { - d.value = nil - d.err = err - return false - } - d.value = &Diff{ - Type: diffType, - Key: val.Key, - Value: val.Value, - } - return true -} - -func (d *UncommittedDiffIterator) SeekGE(id Key) { - d.value = nil - d.list.SeekGE(id) -} - -func (d *UncommittedDiffIterator) Value() *Diff { - return d.value -} - -func (d *UncommittedDiffIterator) Err() error { - return d.err -} - -func (d *UncommittedDiffIterator) Close() { - d.list.Close() -} diff --git a/graveler/graveler.go b/graveler/graveler.go index a7d40504189..7f9b80a8f23 100644 --- a/graveler/graveler.go +++ b/graveler/graveler.go @@ -9,6 +9,7 @@ import ( "regexp" "strings" + "github.com/google/uuid" "github.com/treeverse/lakefs/ident" "github.com/treeverse/lakefs/catalog" @@ -202,13 +203,13 @@ type VersionController interface { DeleteRepository(ctx context.Context, repositoryID RepositoryID) error // CreateBranch creates branch on repository pointing to ref - CreateBranch(ctx context.Context, repositoryID RepositoryID, branchID BranchID, ref Ref) (Branch, error) + CreateBranch(ctx context.Context, repositoryID RepositoryID, branchID BranchID, ref Ref) (*Branch, error) // UpdateBranch updates branch on repository pointing to ref - UpdateBranch(ctx context.Context, repositoryID RepositoryID, branchID BranchID, ref Ref) (Branch, error) + UpdateBranch(ctx context.Context, repositoryID RepositoryID, branchID BranchID, ref Ref) (*Branch, error) // GetBranch gets branch information by branch / repository id - GetBranch(ctx context.Context, repositoryID RepositoryID, branchID BranchID) (Branch, error) + GetBranch(ctx context.Context, repositoryID RepositoryID, branchID BranchID) (*Branch, error) // Log returns an iterator starting at commit ID up to repository root Log(ctx context.Context, repositoryID RepositoryID, commitID CommitID) (CommitIterator, error) @@ -491,3 +492,329 @@ func NewCommitID(id string) (CommitID, error) { func (id CommitID) String() string { return string(id) } + +type graveler struct { + CommittedManager CommittedManager + StagingManager StagingManager + RefManager RefManager +} + +func NewGraveler(CommittedManager CommittedManager, StagingManager StagingManager, RefManager RefManager) Graveler { + return &graveler{ + CommittedManager: CommittedManager, + StagingManager: StagingManager, + RefManager: RefManager, + } +} + +func (r *graveler) GetRepository(ctx context.Context, repositoryID RepositoryID) (*Repository, error) { + return r.RefManager.GetRepository(ctx, repositoryID) +} + +func (r *graveler) CreateRepository(ctx context.Context, repositoryID RepositoryID, storageNamespace StorageNamespace, branchID BranchID) (*Repository, error) { + repo := Repository{ + StorageNamespace: storageNamespace, + CreationDate: time.Now(), + DefaultBranchID: branchID, + } + branch := Branch{ + stagingToken: generateStagingToken(repositoryID, branchID), + } + err := r.RefManager.CreateRepository(ctx, repositoryID, repo, branch) + if err != nil { + return nil, err + } + return &repo, nil +} + +func (r *graveler) ListRepositories(ctx context.Context, from RepositoryID) (RepositoryIterator, error) { + return r.RefManager.ListRepositories(ctx, from) +} + +func (r *graveler) DeleteRepository(ctx context.Context, repositoryID RepositoryID) error { + return r.RefManager.DeleteRepository(ctx, repositoryID) +} + +func (r *graveler) GetCommit(ctx context.Context, repositoryID RepositoryID, commitID CommitID) (*Commit, error) { + return r.RefManager.GetCommit(ctx, repositoryID, commitID) +} +func generateStagingToken(repositoryID RepositoryID, branchID BranchID) StagingToken { + // TODO(Guys): initial implementation, change this + uid := uuid.New().String() + return StagingToken(fmt.Sprintf("%s-%s:%s", repositoryID, branchID, uid)) +} + +func (r *graveler) CreateBranch(ctx context.Context, repositoryID RepositoryID, branchID BranchID, ref Ref) (*Branch, error) { + // check if branch exists + _, err := r.RefManager.GetBranch(ctx, repositoryID, branchID) + if err != nil && !errors.Is(err, ErrNotFound) { + return nil, err + } + if !errors.Is(err, ErrNotFound) { + return nil, ErrBranchExists + } + + reference, err := r.RefManager.RevParse(ctx, repositoryID, ref) + if err != nil { + return nil, err + } + + newBranch := Branch{ + CommitID: reference.CommitID(), + stagingToken: generateStagingToken(repositoryID, branchID), + } + err = r.RefManager.SetBranch(ctx, repositoryID, branchID, newBranch) + if err != nil { + return nil, err + } + return &newBranch, nil +} + +func (r *graveler) UpdateBranch(ctx context.Context, repositoryID RepositoryID, branchID BranchID, ref Ref) (*Branch, error) { + reference, err := r.RefManager.RevParse(ctx, repositoryID, ref) + if err != nil { + return nil, err + } + + curBranch, err := r.RefManager.GetBranch(ctx, repositoryID, branchID) + if err != nil { + return nil, err + } + // validate no conflict + // TODO(Guys) return error only on conflicts, currently returns error for any changes on staging + list, err := r.StagingManager.List(ctx, curBranch.stagingToken) + if err != nil { + return nil, err + } + if list.Next() { + return nil, ErrConflictFound + } + + newBranch := Branch{ + CommitID: reference.CommitID(), + stagingToken: curBranch.stagingToken, + } + err = r.RefManager.SetBranch(ctx, repositoryID, branchID, newBranch) + if err != nil { + return nil, err + } + return &newBranch, nil +} + +func (r *graveler) GetBranch(ctx context.Context, repositoryID RepositoryID, branchID BranchID) (*Branch, error) { + return r.RefManager.GetBranch(ctx, repositoryID, branchID) +} + +func (r *graveler) Dereference(ctx context.Context, repositoryID RepositoryID, ref Ref) (CommitID, error) { + reference, err := r.RefManager.RevParse(ctx, repositoryID, ref) + if err != nil { + return "", err + } + return reference.CommitID(), nil +} + +func (r *graveler) Log(ctx context.Context, repositoryID RepositoryID, commitID CommitID) (CommitIterator, error) { + return r.RefManager.Log(ctx, repositoryID, commitID) +} + +func (r *graveler) ListBranches(ctx context.Context, repositoryID RepositoryID, from BranchID) (BranchIterator, error) { + return r.RefManager.ListBranches(ctx, repositoryID, from) +} + +func (r *graveler) DeleteBranch(ctx context.Context, repositoryID RepositoryID, branchID BranchID) error { + branch, err := r.RefManager.GetBranch(ctx, repositoryID, branchID) + if err != nil { + return err + } + err = r.StagingManager.Drop(ctx, branch.stagingToken) + if err != nil { + return err + } + return r.RefManager.DeleteBranch(ctx, repositoryID, branchID) +} + +func (r *graveler) Get(ctx context.Context, repositoryID RepositoryID, ref Ref, key Key) (*Value, error) { + repo, err := r.RefManager.GetRepository(ctx, repositoryID) + if err != nil { + return nil, err + } + reference, err := r.RefManager.RevParse(ctx, repositoryID, ref) + if err != nil { + return nil, err + } + if reference.Type() == ReferenceTypeBranch { + // try to get from staging, if not found proceed to committed + branch := reference.Branch() + value, err := r.StagingManager.Get(ctx, branch.stagingToken, key) + if !errors.Is(err, ErrNotFound) { + if err != nil { + return nil, err + } + if value == nil { + // tombstone + return nil, ErrNotFound + } + return value, nil + } + } + commitID := reference.CommitID() + commit, err := r.RefManager.GetCommit(ctx, repositoryID, commitID) + if err != nil { + return nil, err + } + return r.CommittedManager.Get(ctx, repo.StorageNamespace, commit.TreeID, key) +} + +func (r *graveler) Set(ctx context.Context, repositoryID RepositoryID, branchID BranchID, key Key, value Value) error { + branch, err := r.GetBranch(ctx, repositoryID, branchID) + if err != nil { + return err + } + return r.StagingManager.Set(ctx, branch.stagingToken, key, value) +} + +func (r *graveler) Delete(ctx context.Context, repositoryID RepositoryID, branchID BranchID, key Key) error { + branch, err := r.GetBranch(ctx, repositoryID, branchID) + if err != nil { + return err + } + return r.StagingManager.Delete(ctx, branch.stagingToken, key) +} + +func (r *graveler) List(ctx context.Context, repositoryID RepositoryID, ref Ref, prefix, from, delimiter Key) (ListingIterator, error) { + repo, err := r.RefManager.GetRepository(ctx, repositoryID) + if err != nil { + return nil, err + } + reference, err := r.RefManager.RevParse(ctx, repositoryID, ref) + if err != nil { + return nil, err + } + commit, err := r.RefManager.GetCommit(ctx, repositoryID, reference.CommitID()) + if err != nil { + return nil, err + } + + var listing ListingIterator + committedValues, err := r.CommittedManager.List(ctx, repo.StorageNamespace, commit.TreeID, from) + if err != nil { + return nil, err + } + committedListing := NewListingIter(NewPrefixIterator(committedValues, prefix), delimiter, prefix) + if reference.Type() == ReferenceTypeBranch { + stagingList, err := r.StagingManager.List(ctx, reference.Branch().stagingToken) + if err != nil { + return nil, err + } + listing = NewCombinedIterator(NewListingIter(NewPrefixIterator(stagingList, prefix), delimiter, prefix), committedListing) + } else { + listing = committedListing + } + return listing, nil +} + +func (r *graveler) Commit(ctx context.Context, repositoryID RepositoryID, branchID BranchID, committer string, message string, metadata Metadata) (CommitID, error) { + panic("implement me") +} + +func (r *graveler) Reset(ctx context.Context, repositoryID RepositoryID, branchID BranchID) error { + panic("implement me") // waiting for staging reset +} + +func (r *graveler) Revert(ctx context.Context, repositoryID RepositoryID, branchID BranchID, ref Ref) (CommitID, error) { + panic("implement me") +} + +func (r *graveler) Merge(ctx context.Context, repositoryID RepositoryID, from Ref, to BranchID) (CommitID, error) { + repo, err := r.RefManager.GetRepository(ctx, repositoryID) + if err != nil { + return "", err + } + + fromCommit, err := r.getCommitRecordFromRef(ctx, repositoryID, from) + if err != nil { + return "", err + } + toCommit, err := r.getCommitRecordFromBranchID(ctx, repositoryID, to) + if err != nil { + return "", err + } + baseCommit, err := r.RefManager.FindMergeBase(ctx, repositoryID, fromCommit.CommitID, toCommit.CommitID) + if err != nil { + return "", err + } + + treeID, err := r.CommittedManager.Merge(ctx, repo.StorageNamespace, fromCommit.TreeID, toCommit.TreeID, baseCommit.TreeID) + if err != nil { + return "", err + } + commit := Commit{ + Committer: "unknown", // TODO(Guys): pass committer or enter default value + Message: "merge message", // TODO(Guys): get merge message + TreeID: treeID, + CreationDate: time.Time{}, + Parents: []CommitID{fromCommit.CommitID, toCommit.CommitID}, + Metadata: nil, + } + return r.RefManager.AddCommit(ctx, repositoryID, commit) +} + +func (r *graveler) DiffUncommitted(ctx context.Context, repositoryID RepositoryID, branchID BranchID, from Key) (DiffIterator, error) { + repo, err := r.RefManager.GetRepository(ctx, repositoryID) + if err != nil { + return nil, err + } + branch, err := r.RefManager.GetBranch(ctx, repositoryID, branchID) + if err != nil { + return nil, err + } + commit, err := r.RefManager.GetCommit(ctx, repositoryID, branch.CommitID) + if err != nil { + return nil, err + } + valueIterator, err := r.StagingManager.List(ctx, branch.stagingToken) + if err != nil { + return nil, err + } + return newUncommittedDiffIterator(r.CommittedManager, valueIterator, repo.StorageNamespace, commit.TreeID), nil +} + +func (r *graveler) getCommitRecordFromRef(ctx context.Context, repositoryID RepositoryID, ref Ref) (*CommitRecord, error) { + reference, err := r.RefManager.RevParse(ctx, repositoryID, ref) + if err != nil { + return nil, err + } + commit, err := r.RefManager.GetCommit(ctx, repositoryID, reference.CommitID()) + if err != nil { + return nil, err + } + return &CommitRecord{ + CommitID: reference.CommitID(), + Commit: commit, + }, nil +} + +func (r *graveler) getCommitRecordFromBranchID(ctx context.Context, repositoryID RepositoryID, branch BranchID) (*CommitRecord, error) { + return r.getCommitRecordFromRef(ctx, repositoryID, Ref(branch)) +} + +func (r *graveler) Diff(ctx context.Context, repositoryID RepositoryID, left, right Ref, from Key) (DiffIterator, error) { + repo, err := r.RefManager.GetRepository(ctx, repositoryID) + if err != nil { + return nil, err + } + leftCommit, err := r.getCommitRecordFromRef(ctx, repositoryID, left) + if err != nil { + return nil, err + } + rightCommit, err := r.getCommitRecordFromRef(ctx, repositoryID, right) + if err != nil { + return nil, err + } + baseCommit, err := r.RefManager.FindMergeBase(ctx, repositoryID, leftCommit.CommitID, rightCommit.CommitID) + if err != nil { + return nil, err + } + + return r.CommittedManager.Diff(ctx, repo.StorageNamespace, leftCommit.TreeID, rightCommit.TreeID, baseCommit.TreeID, from) +} diff --git a/graveler/gravel_test.go b/graveler/graveler_test.go similarity index 78% rename from graveler/gravel_test.go rename to graveler/graveler_test.go index a6aa75da2dd..9a286aa5bba 100644 --- a/graveler/gravel_test.go +++ b/graveler/graveler_test.go @@ -6,9 +6,8 @@ import ( "errors" "testing" - "github.com/treeverse/lakefs/graveler" - "github.com/go-test/deep" + "github.com/treeverse/lakefs/graveler" ) type committedMock struct { @@ -479,7 +478,7 @@ func TestGravel_listingIterator(t *testing.T) { func TestGravel_List(t *testing.T) { tests := []struct { name string - r graveler.Gravel + r graveler.Graveler amount int delimiter graveler.Key from graveler.Key @@ -490,11 +489,10 @@ func TestGravel_List(t *testing.T) { }{ { name: "one committed one staged no paths", - r: graveler.Gravel{ - CommittedManager: &committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo"), Value: &graveler.Value{}}})}, - StagingManager: &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("bar"), Value: &graveler.Value{}}})}, - RefManager: &mockRefs{refType: graveler.ReferenceTypeBranch}, - }, + r: graveler.NewGraveler(&committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo"), Value: &graveler.Value{}}})}, + &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("bar"), Value: &graveler.Value{}}})}, + &mockRefs{refType: graveler.ReferenceTypeBranch}, + ), delimiter: graveler.Key("/"), prefix: graveler.Key(""), amount: 10, @@ -502,11 +500,10 @@ func TestGravel_List(t *testing.T) { }, { name: "same path different file", - r: graveler.Gravel{ - CommittedManager: &committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo"), Value: &graveler.Value{Identity: []byte("original")}}})}, - StagingManager: &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo"), Value: &graveler.Value{Identity: []byte("other")}}})}, - RefManager: &mockRefs{refType: graveler.ReferenceTypeBranch}, - }, + r: graveler.NewGraveler(&committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo"), Value: &graveler.Value{Identity: []byte("original")}}})}, + &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo"), Value: &graveler.Value{Identity: []byte("other")}}})}, + &mockRefs{refType: graveler.ReferenceTypeBranch}, + ), delimiter: graveler.Key("/"), prefix: graveler.Key(""), amount: 10, @@ -514,11 +511,10 @@ func TestGravel_List(t *testing.T) { }, { name: "one committed one staged no paths - with prefix", - r: graveler.Gravel{ - CommittedManager: &committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("prefix/foo"), Value: &graveler.Value{}}})}, - StagingManager: &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("prefix/bar"), Value: &graveler.Value{}}})}, - RefManager: &mockRefs{refType: graveler.ReferenceTypeBranch}, - }, + r: graveler.NewGraveler(&committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("prefix/foo"), Value: &graveler.Value{}}})}, + &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("prefix/bar"), Value: &graveler.Value{}}})}, + &mockRefs{refType: graveler.ReferenceTypeBranch}, + ), delimiter: graveler.Key("/"), prefix: graveler.Key("prefix/"), amount: 10, @@ -526,11 +522,10 @@ func TestGravel_List(t *testing.T) { }, { name: "objects and paths in both committed and staging", - r: graveler.Gravel{ - CommittedManager: &committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("prefix/pathA/foo"), Value: &graveler.Value{}}, {Key: graveler.Key("prefix/pathA/foo2"), Value: &graveler.Value{}}, {Key: graveler.Key("prefix/pathB/foo"), Value: &graveler.Value{}}})}, - StagingManager: &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("prefix/file"), Value: &graveler.Value{}}, {Key: graveler.Key("prefix/pathA/bar"), Value: &graveler.Value{}}, {Key: graveler.Key("prefix/pathB/bar"), Value: &graveler.Value{}}})}, - RefManager: &mockRefs{refType: graveler.ReferenceTypeBranch}, - }, + r: graveler.NewGraveler(&committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("prefix/pathA/foo"), Value: &graveler.Value{}}, {Key: graveler.Key("prefix/pathA/foo2"), Value: &graveler.Value{}}, {Key: graveler.Key("prefix/pathB/foo"), Value: &graveler.Value{}}})}, + &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("prefix/file"), Value: &graveler.Value{}}, {Key: graveler.Key("prefix/pathA/bar"), Value: &graveler.Value{}}, {Key: graveler.Key("prefix/pathB/bar"), Value: &graveler.Value{}}})}, + &mockRefs{refType: graveler.ReferenceTypeBranch}, + ), delimiter: graveler.Key("/"), prefix: graveler.Key("prefix/"), amount: 10, @@ -569,78 +564,65 @@ func TestGravel_Get(t *testing.T) { var ErrTest = errors.New("some kind of err") tests := []struct { name string - r graveler.Gravel + r graveler.Graveler expectedValueResult graveler.Value expectedErr error }{ { name: "commit - exists", - r: graveler.Gravel{ - CommittedManager: &committedMock{Value: &graveler.Value{Identity: []byte("committed")}}, - RefManager: &mockRefs{refType: graveler.ReferenceTypeCommit}, - }, - + r: graveler.NewGraveler(&committedMock{Value: &graveler.Value{Identity: []byte("committed")}}, nil, + &mockRefs{refType: graveler.ReferenceTypeCommit}, + ), expectedValueResult: graveler.Value{Identity: []byte("committed")}, }, { name: "commit - not found", - r: graveler.Gravel{ - CommittedManager: &committedMock{err: graveler.ErrNotFound}, - RefManager: &mockRefs{refType: graveler.ReferenceTypeCommit}, - }, - expectedErr: graveler.ErrNotFound, + r: graveler.NewGraveler(&committedMock{err: graveler.ErrNotFound}, nil, + &mockRefs{refType: graveler.ReferenceTypeCommit}, + ), expectedErr: graveler.ErrNotFound, }, { name: "commit - error", - r: graveler.Gravel{ - CommittedManager: &committedMock{err: ErrTest}, - RefManager: &mockRefs{refType: graveler.ReferenceTypeCommit}, - }, - expectedErr: ErrTest, + r: graveler.NewGraveler(&committedMock{err: ErrTest}, nil, + &mockRefs{refType: graveler.ReferenceTypeCommit}, + ), expectedErr: ErrTest, }, { name: "branch - only staged", - r: graveler.Gravel{ - StagingManager: &stagingMock{Value: &graveler.Value{Identity: []byte("staged")}}, - CommittedManager: &committedMock{err: graveler.ErrNotFound}, - RefManager: &mockRefs{refType: graveler.ReferenceTypeBranch}, - }, + r: graveler.NewGraveler(&committedMock{err: graveler.ErrNotFound}, &stagingMock{Value: &graveler.Value{Identity: []byte("staged")}}, + &mockRefs{refType: graveler.ReferenceTypeBranch}, + ), expectedValueResult: graveler.Value{Identity: []byte("staged")}, }, { name: "branch - committed and staged", - r: graveler.Gravel{ - StagingManager: &stagingMock{Value: &graveler.Value{Identity: []byte("staged")}}, - CommittedManager: &committedMock{Value: &graveler.Value{Identity: []byte("committed")}}, - RefManager: &mockRefs{refType: graveler.ReferenceTypeBranch}, - }, + r: graveler.NewGraveler(&committedMock{Value: &graveler.Value{Identity: []byte("committed")}}, &stagingMock{Value: &graveler.Value{Identity: []byte("staged")}}, + + &mockRefs{refType: graveler.ReferenceTypeBranch}, + ), expectedValueResult: graveler.Value{Identity: []byte("staged")}, }, { name: "branch - only committed", - r: graveler.Gravel{ - StagingManager: &stagingMock{err: graveler.ErrNotFound}, - CommittedManager: &committedMock{Value: &graveler.Value{Identity: []byte("committed")}}, - RefManager: &mockRefs{refType: graveler.ReferenceTypeBranch}, - }, + r: graveler.NewGraveler(&committedMock{Value: &graveler.Value{Identity: []byte("committed")}}, &stagingMock{err: graveler.ErrNotFound}, + + &mockRefs{refType: graveler.ReferenceTypeBranch}, + ), expectedValueResult: graveler.Value{Identity: []byte("committed")}, }, { name: "branch - tombstone", - r: graveler.Gravel{ - StagingManager: &stagingMock{Value: nil}, - CommittedManager: &committedMock{Value: &graveler.Value{Identity: []byte("committed")}}, - RefManager: &mockRefs{refType: graveler.ReferenceTypeBranch}, - }, + r: graveler.NewGraveler(&committedMock{Value: &graveler.Value{Identity: []byte("committed")}}, &stagingMock{Value: nil}, + + &mockRefs{refType: graveler.ReferenceTypeBranch}, + ), expectedErr: graveler.ErrNotFound, }, { name: "branch - staged return error", - r: graveler.Gravel{ - StagingManager: &stagingMock{err: ErrTest}, - CommittedManager: &committedMock{}, - RefManager: &mockRefs{refType: graveler.ReferenceTypeBranch}, - }, + r: graveler.NewGraveler(&committedMock{}, &stagingMock{err: ErrTest}, + &mockRefs{refType: graveler.ReferenceTypeBranch}, + ), expectedErr: ErrTest, }, } @@ -675,7 +657,7 @@ func TestGravel_Revert(t *testing.T) { func TestGravel_DiffUncommitted(t *testing.T) { tests := []struct { name string - r graveler.Gravel + r graveler.Graveler amount int expectedErr error expectedHasMore bool @@ -683,21 +665,19 @@ func TestGravel_DiffUncommitted(t *testing.T) { }{ { name: "no changes", - r: graveler.Gravel{ - CommittedManager: &committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}}), err: graveler.ErrNotFound}, - StagingManager: &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{})}, - RefManager: &mockRefs{}, - }, + r: graveler.NewGraveler(&committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}}), err: graveler.ErrNotFound}, + &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{})}, + &mockRefs{}, + ), amount: 10, expectedDiff: newDiffIter([]graveler.Diff{}), }, { name: "added one", - r: graveler.Gravel{ - CommittedManager: &committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{}), err: graveler.ErrNotFound}, - StagingManager: &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}})}, - RefManager: &mockRefs{}, - }, + r: graveler.NewGraveler(&committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{}), err: graveler.ErrNotFound}, + &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}})}, + &mockRefs{}, + ), amount: 10, expectedDiff: newDiffIter([]graveler.Diff{{ Key: graveler.Key("foo/one"), @@ -707,11 +687,10 @@ func TestGravel_DiffUncommitted(t *testing.T) { }, { name: "changed one", - r: graveler.Gravel{ - CommittedManager: &committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}})}, - StagingManager: &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}})}, - RefManager: &mockRefs{}, - }, + r: graveler.NewGraveler(&committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}})}, + &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}})}, + &mockRefs{}, + ), amount: 10, expectedDiff: newDiffIter([]graveler.Diff{{ Key: graveler.Key("foo/one"), @@ -721,11 +700,10 @@ func TestGravel_DiffUncommitted(t *testing.T) { }, { name: "removed one", - r: graveler.Gravel{ - CommittedManager: &committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}})}, - StagingManager: &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: nil}})}, - RefManager: &mockRefs{}, - }, + r: graveler.NewGraveler(&committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}})}, + &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: nil}})}, + &mockRefs{}, + ), amount: 10, expectedDiff: newDiffIter([]graveler.Diff{{ Key: graveler.Key("foo/one"), @@ -761,17 +739,18 @@ func TestGravel_DiffUncommitted(t *testing.T) { } func TestGravel_UpdateBranch(t *testing.T) { - gravel := graveler.Gravel{ - CommittedManager: nil, - StagingManager: &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}})}, - RefManager: &mockRefs{}, - } + gravel := graveler.NewGraveler(nil, + &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}})}, + &mockRefs{}, + ) _, err := gravel.UpdateBranch(context.Background(), "", "", "") if !errors.Is(err, graveler.ErrConflictFound) { t.Fatal("expected update to fail on conflict") } - - gravel.StagingManager = &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{})} + gravel = graveler.NewGraveler(nil, + &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{})}, + &mockRefs{}, + ) _, err = gravel.UpdateBranch(context.Background(), "", "", "") if err != nil { t.Fatal("did not expect to get error") diff --git a/graveler/listing_iterator.go b/graveler/listing_iterator.go new file mode 100644 index 00000000000..c3e62ae8dd9 --- /dev/null +++ b/graveler/listing_iterator.go @@ -0,0 +1,124 @@ +package graveler + +import "bytes" + +// ListingIter implements a listing iterator using a ValueIterator +// assumes all values in valueIterator start with prefix +type ListingIter struct { + valueIterator ValueIterator + delimiter Key + nextDelimiter Key + prefix Key + current *Listing + nextFunc func(l *ListingIter) bool + err error +} + +// getFollowingValue returns the following value (i.e will increase the last byte by 1) +// in the following cases will return received value : value is nil, value length is 0, last byte is maximum byte +func getFollowingValue(value []byte) []byte { + if len(value) == 0 || value[len(value)-1] == 255 { + return value + } + copiedDelimiter := make([]byte, len(value)) + copy(copiedDelimiter, value) + return append(copiedDelimiter[:len(copiedDelimiter)-1], copiedDelimiter[len(copiedDelimiter)-1]+1) +} + +func NewListingIter(iterator ValueIterator, delimiter, prefix Key) *ListingIter { + var nextDelimiter Key + var nextFunc func(l *ListingIter) bool + if len(delimiter) == 0 { + nextFunc = nextNoDelimiter + } else { + nextFunc = nextWithDelimiter + nextDelimiter = getFollowingValue(delimiter) + } + + return &ListingIter{ + valueIterator: iterator, + delimiter: delimiter, + nextDelimiter: nextDelimiter, + prefix: prefix, + nextFunc: nextFunc, + } +} + +func nextNoDelimiter(l *ListingIter) bool { + hasNext := l.valueIterator.Next() + if !hasNext { + l.current = nil + return false + } + val := l.valueIterator.Value() + l.current = &Listing{ + CommonPrefix: false, + Key: val.Key, + Value: val.Value, + } + return true +} + +func nextWithDelimiter(l *ListingIter) bool { + var hasNext bool + if l.current == nil || !l.current.CommonPrefix { + hasNext = l.valueIterator.Next() + } else { + nextKey := append(l.current.Key, l.nextDelimiter...) + l.valueIterator.SeekGE(nextKey) + hasNext = l.valueIterator.Next() + } + + if hasNext { + nextValue := l.valueIterator.Value() + if !bytes.HasPrefix(nextValue.Key, l.prefix) { + l.current = nil + l.err = ErrUnexpected + return false + } + l.current = l.getListingFromValue(nextValue.Value, nextValue.Key) + } else { + l.current = nil + } + return hasNext +} + +func (l *ListingIter) Next() bool { + return l.nextFunc(l) +} + +func (l *ListingIter) getListingFromValue(value *Value, key Key) *Listing { + relevantPath := key[len(l.prefix):] + delimiterIndex := bytes.Index(relevantPath, l.delimiter) + commonPrefix := delimiterIndex >= 0 + if commonPrefix { + relevantPath = relevantPath[:delimiterIndex] + value = nil + } + newKey := append(l.prefix, relevantPath...) + return &Listing{ + Key: newKey, + CommonPrefix: commonPrefix, + Value: value, + } +} + +func (l *ListingIter) SeekGE(id Key) { + l.current = nil + l.valueIterator.SeekGE(id) +} + +func (l *ListingIter) Value() *Listing { + return l.current +} + +func (l *ListingIter) Err() error { + if l.err != nil { + return l.err + } + return l.valueIterator.Err() +} + +func (l *ListingIter) Close() { + l.valueIterator.Close() +} diff --git a/graveler/prefix_iterator.go b/graveler/prefix_iterator.go new file mode 100644 index 00000000000..ad94836e0bb --- /dev/null +++ b/graveler/prefix_iterator.go @@ -0,0 +1,53 @@ +package graveler + +import "bytes" + +// PrefixIterator holds a ValueIterator and iterates only over values the their Key starts with the prefix +type PrefixIterator struct { + prefix Key + valueIterator ValueIterator + current *ValueRecord +} + +func NewPrefixIterator(iterator ValueIterator, prefix Key) *PrefixIterator { + iterator.SeekGE(prefix) + return &PrefixIterator{ + prefix: prefix, + valueIterator: iterator, + current: nil, + } +} + +func (p *PrefixIterator) Next() bool { + if !p.valueIterator.Next() { + p.current = nil + return false + } + val := p.valueIterator.Value() + if !bytes.HasPrefix(val.Key, p.prefix) { + p.current = nil + return false + } + p.current = val + return true +} + +func (p *PrefixIterator) SeekGE(id Key) { + if bytes.Compare(id, p.prefix) <= 0 { + p.valueIterator.SeekGE(p.prefix) + return + } + p.valueIterator.SeekGE(id) +} + +func (p *PrefixIterator) Value() *ValueRecord { + return p.current +} + +func (p *PrefixIterator) Err() error { + return p.valueIterator.Err() +} + +func (p *PrefixIterator) Close() { + p.valueIterator.Close() +} diff --git a/graveler/uncommitted_iterator.go b/graveler/uncommitted_iterator.go new file mode 100644 index 00000000000..ded9d207b60 --- /dev/null +++ b/graveler/uncommitted_iterator.go @@ -0,0 +1,96 @@ +package graveler + +import ( + "context" + "errors" + + "github.com/treeverse/lakefs/logging" +) + +type UncommittedDiffIterator struct { + committedManager CommittedManager + list ValueIterator + sn StorageNamespace + treeID TreeID + value *Diff + err error +} + +func newUncommittedDiffIterator(manager CommittedManager, list ValueIterator, sn StorageNamespace, treeItreeID TreeID) *UncommittedDiffIterator { + return &UncommittedDiffIterator{ + committedManager: manager, + list: list, + sn: sn, + treeID: treeItreeID, + } +} + +func valueExistsInCommitted(ctx context.Context, committedManager CommittedManager, sn StorageNamespace, treeID TreeID, key Key) (bool, error) { + _, err := committedManager.Get(ctx, sn, treeID, key) + if errors.Is(err, ErrNotFound) { + return false, nil + } else if err != nil { + return false, err + } + return true, nil +} + +func getDiffType(ctx context.Context, committedManager CommittedManager, sn StorageNamespace, treeID TreeID, key Key, tombstone bool) (DiffType, error) { + existsInCommitted, err := valueExistsInCommitted(ctx, committedManager, sn, treeID, key) + if err != nil { + return 0, err + } + var diffType DiffType + diffType = DiffTypeAdded + if tombstone && existsInCommitted { + diffType = DiffTypeRemoved + } + if tombstone && !existsInCommitted { + logging.Default().WithFields(logging.Fields{"treeID": treeID, "stagingToken": sn, "key": key}).Warn("tombstone for file that does not exist") + diffType = DiffTypeRemoved + } + if !tombstone && existsInCommitted { + diffType = DiffTypeChanged + } + if !tombstone && !existsInCommitted { + diffType = DiffTypeAdded + } + return diffType, nil +} + +func (d *UncommittedDiffIterator) Next() bool { + if !d.list.Next() { + d.value = nil + return false + } + val := d.list.Value() + diffType, err := getDiffType(context.Background(), d.committedManager, d.sn, d.treeID, val.Key, val.Value == nil) + if err != nil { + d.value = nil + d.err = err + return false + } + d.value = &Diff{ + Type: diffType, + Key: val.Key, + Value: val.Value, + } + return true +} + +func (d *UncommittedDiffIterator) SeekGE(id Key) { + d.value = nil + d.list.SeekGE(id) +} + +func (d *UncommittedDiffIterator) Value() *Diff { + return d.value +} + +func (d *UncommittedDiffIterator) Err() error { + return d.err +} + +func (d *UncommittedDiffIterator) Close() { + d.list.Close() +} diff --git a/parade/action_manager.go b/parade/action_manager.go index 5be495865f9..15e3d7bd81d 100644 --- a/parade/action_manager.go +++ b/parade/action_manager.go @@ -16,7 +16,7 @@ const ( defaultMaxTasks = 500 defaultWaitTime = time.Millisecond * 300 defaultErrWaitTime = time.Second * 3 - defaultMaxDuration = time.Minute * 30 // Todo(guys): change this + defaultMaxDuration = time.Minute * 30 // TODO(guys): change this ) // ManagerProperties defines the configuration properties of an ActionManager From ee590dca6995b629016ae0241eb3a4806965e73c Mon Sep 17 00:00:00 2001 From: guyhardonag Date: Wed, 9 Dec 2020 12:35:15 +0200 Subject: [PATCH 03/14] Fix lint --- graveler/graveler.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/graveler/graveler.go b/graveler/graveler.go index 7f9b80a8f23..ee0825e8b05 100644 --- a/graveler/graveler.go +++ b/graveler/graveler.go @@ -499,11 +499,11 @@ type graveler struct { RefManager RefManager } -func NewGraveler(CommittedManager CommittedManager, StagingManager StagingManager, RefManager RefManager) Graveler { +func NewGraveler(committedManager CommittedManager, stagingManager StagingManager, refManager RefManager) Graveler { return &graveler{ - CommittedManager: CommittedManager, - StagingManager: StagingManager, - RefManager: RefManager, + CommittedManager: committedManager, + StagingManager: stagingManager, + RefManager: refManager, } } From 8d128935c30d4573c3e52bc85de33aada99b0745 Mon Sep 17 00:00:00 2001 From: guyhardonag Date: Wed, 9 Dec 2020 16:00:08 +0200 Subject: [PATCH 04/14] Add change requests from review --- graveler/combined_iterator.go | 37 +- graveler/graveler.go | 7 +- graveler/graveler_mock_test.go | 336 +++++++++++++ graveler/graveler_test.go | 466 +++--------------- graveler/listing_iterator.go | 87 ++-- graveler/prefix_iterator.go | 57 ++- ...erator.go => uncommitted_diff_iterator.go} | 16 +- 7 files changed, 497 insertions(+), 509 deletions(-) create mode 100644 graveler/graveler_mock_test.go rename graveler/{uncommitted_iterator.go => uncommitted_diff_iterator.go} (80%) diff --git a/graveler/combined_iterator.go b/graveler/combined_iterator.go index 0b87d4ff28e..d047cf4729a 100644 --- a/graveler/combined_iterator.go +++ b/graveler/combined_iterator.go @@ -2,17 +2,17 @@ package graveler import "bytes" -// CombinedIterator iterates over two listing iterators, +// combinedIterator iterates over two listing iterators, // in case of duplication (in values or in errors) returns value in iterA -type CombinedIterator struct { +type combinedIterator struct { iterA ListingIterator iterB ListingIterator p ListingIterator err error } -func NewCombinedIterator(iterA, iterB ListingIterator) *CombinedIterator { - return &CombinedIterator{ +func NewCombinedIterator(iterA, iterB ListingIterator) ListingIterator { + return &combinedIterator{ iterA: iterA, iterB: iterB, p: nil, @@ -20,7 +20,7 @@ func NewCombinedIterator(iterA, iterB ListingIterator) *CombinedIterator { } } -func (c *CombinedIterator) Next() bool { +func (c *combinedIterator) Next() bool { // call next with the relevant iterators valA := c.iterA.Value() valB := c.iterB.Value() @@ -50,54 +50,53 @@ func (c *CombinedIterator) Next() bool { } if c.iterA.Err() != nil || c.iterB.Err() != nil { + if c.iterA.Err() != nil { + c.p = c.iterA + } else { + c.p = c.iterB + } return false } - // get current pointer + // get the current pointer valA = c.iterA.Value() valB = c.iterB.Value() switch { case valA == nil && valB == nil: c.p = c.iterA // in order not to be stuck in start state return false - case valA == nil: c.p = c.iterB - return true - case valB == nil: c.p = c.iterA - return true - case bytes.Compare(valA.Key, valB.Key) <= 0: c.p = c.iterA - return true default: c.p = c.iterB } return true } -func (c *CombinedIterator) SeekGE(id Key) { +func (c *combinedIterator) SeekGE(id Key) { c.p = nil c.iterA.SeekGE(id) c.iterB.SeekGE(id) } -func (c *CombinedIterator) Value() *Listing { +func (c *combinedIterator) Value() *Listing { if c.p == nil { return nil } return c.p.Value() } -func (c *CombinedIterator) Err() error { - if c.iterA.Err() != nil { - return c.iterA.Err() +func (c *combinedIterator) Err() error { + if c.p == nil { + return nil } - return c.iterB.Err() + return c.p.Err() } -func (c *CombinedIterator) Close() { +func (c *combinedIterator) Close() { c.iterA.Close() c.iterB.Close() } diff --git a/graveler/graveler.go b/graveler/graveler.go index ee0825e8b05..7e51725fac1 100644 --- a/graveler/graveler.go +++ b/graveler/graveler.go @@ -425,7 +425,6 @@ var ( ErrCommitIDAmbiguous = fmt.Errorf("commit ID is ambiguous: %w", ErrNotFound) ErrConflictFound = errors.New("conflict found") ErrBranchExists = errors.New("branch already exists") - ErrUnexpected = errors.New("unexpected error") ) func NewRepositoryID(id string) (RepositoryID, error) { @@ -700,13 +699,13 @@ func (r *graveler) List(ctx context.Context, repositoryID RepositoryID, ref Ref, if err != nil { return nil, err } - committedListing := NewListingIter(NewPrefixIterator(committedValues, prefix), delimiter, prefix) + committedListing := NewListingIterator(NewPrefixIterator(committedValues, prefix), delimiter, prefix) if reference.Type() == ReferenceTypeBranch { stagingList, err := r.StagingManager.List(ctx, reference.Branch().stagingToken) if err != nil { return nil, err } - listing = NewCombinedIterator(NewListingIter(NewPrefixIterator(stagingList, prefix), delimiter, prefix), committedListing) + listing = NewCombinedIterator(NewListingIterator(NewPrefixIterator(stagingList, prefix), delimiter, prefix), committedListing) } else { listing = committedListing } @@ -776,7 +775,7 @@ func (r *graveler) DiffUncommitted(ctx context.Context, repositoryID RepositoryI if err != nil { return nil, err } - return newUncommittedDiffIterator(r.CommittedManager, valueIterator, repo.StorageNamespace, commit.TreeID), nil + return NewUncommittedDiffIterator(r.CommittedManager, valueIterator, repo.StorageNamespace, commit.TreeID), nil } func (r *graveler) getCommitRecordFromRef(ctx context.Context, repositoryID RepositoryID, ref Ref) (*CommitRecord, error) { diff --git a/graveler/graveler_mock_test.go b/graveler/graveler_mock_test.go new file mode 100644 index 00000000000..2a13ba36d36 --- /dev/null +++ b/graveler/graveler_mock_test.go @@ -0,0 +1,336 @@ +package graveler_test + +import ( + "bytes" + "context" + "testing" + + "github.com/go-test/deep" + "github.com/treeverse/lakefs/graveler" +) + +type committedMock struct { + Value *graveler.Value + ValueIterator graveler.ValueIterator + diffIterator graveler.DiffIterator + err error + treeID graveler.TreeID +} + +func (c *committedMock) Get(_ context.Context, _ graveler.StorageNamespace, _ graveler.TreeID, _ graveler.Key) (*graveler.Value, error) { + if c.err != nil { + return nil, c.err + } + return c.Value, nil +} + +func (c *committedMock) List(_ context.Context, _ graveler.StorageNamespace, _ graveler.TreeID, _ graveler.Key) (graveler.ValueIterator, error) { + if c.err != nil { + return nil, c.err + } + return c.ValueIterator, nil +} + +func (c *committedMock) Diff(_ context.Context, _ graveler.StorageNamespace, _, _, _ graveler.TreeID, _ graveler.Key) (graveler.DiffIterator, error) { + if c.err != nil { + return nil, c.err + } + return c.diffIterator, nil +} + +func (c *committedMock) Merge(_ context.Context, _ graveler.StorageNamespace, _, _, _ graveler.TreeID) (graveler.TreeID, error) { + if c.err != nil { + return "", c.err + } + return c.treeID, nil +} + +func (c *committedMock) Apply(_ context.Context, _ graveler.StorageNamespace, _ graveler.TreeID, _ graveler.ValueIterator) (graveler.TreeID, error) { + if c.err != nil { + return "", c.err + } + return c.treeID, nil +} + +type stagingMock struct { + err error + Value *graveler.Value + ValueIterator graveler.ValueIterator + stagingToken graveler.StagingToken +} + +func (s *stagingMock) Drop(_ context.Context, _ graveler.StagingToken) error { + if s.err != nil { + return s.err + } + return nil +} + +func (s *stagingMock) Get(_ context.Context, _ graveler.StagingToken, _ graveler.Key) (*graveler.Value, error) { + if s.err != nil { + return nil, s.err + } + return s.Value, nil +} + +func (s *stagingMock) Set(_ context.Context, _ graveler.StagingToken, _ graveler.Key, _ graveler.Value) error { + if s.err != nil { + return s.err + } + return nil +} + +func (s *stagingMock) Delete(_ context.Context, _ graveler.StagingToken, _ graveler.Key) error { + return nil +} + +func (s *stagingMock) List(_ context.Context, _ graveler.StagingToken) (graveler.ValueIterator, error) { + if s.err != nil { + return nil, s.err + } + return s.ValueIterator, nil +} + +func (s *stagingMock) Snapshot(_ context.Context, _ graveler.StagingToken) (graveler.StagingToken, error) { + if s.err != nil { + return "", s.err + } + return s.stagingToken, nil +} + +func (s *stagingMock) ListSnapshot(_ context.Context, _ graveler.StagingToken, _ graveler.Key) (graveler.ValueIterator, error) { + if s.err != nil { + return nil, s.err + } + return s.ValueIterator, nil +} + +type mockRefs struct { + listRepositoriesRes graveler.RepositoryIterator + listBranchesRes graveler.BranchIterator + commitIter graveler.CommitIterator + refType graveler.ReferenceType +} + +const defaultBranchName = graveler.BranchID("master") + +func (m *mockRefs) RevParse(_ context.Context, _ graveler.RepositoryID, _ graveler.Ref) (graveler.Reference, error) { + var branch graveler.BranchID + if m.refType == graveler.ReferenceTypeBranch { + branch = defaultBranchName + } + return newMockReference(m.refType, branch, ""), nil +} + +func (m *mockRefs) GetRepository(_ context.Context, _ graveler.RepositoryID) (*graveler.Repository, error) { + return &graveler.Repository{}, nil +} + +func (m *mockRefs) CreateRepository(_ context.Context, _ graveler.RepositoryID, _ graveler.Repository, _ graveler.Branch) error { + return nil +} + +func (m *mockRefs) ListRepositories(_ context.Context, _ graveler.RepositoryID) (graveler.RepositoryIterator, error) { + return m.listRepositoriesRes, nil +} + +func (m *mockRefs) DeleteRepository(_ context.Context, _ graveler.RepositoryID) error { + return nil +} + +func (m *mockRefs) GetBranch(_ context.Context, _ graveler.RepositoryID, _ graveler.BranchID) (*graveler.Branch, error) { + return &graveler.Branch{}, nil +} + +func (m *mockRefs) SetBranch(_ context.Context, _ graveler.RepositoryID, _ graveler.BranchID, _ graveler.Branch) error { + return nil +} + +func (m *mockRefs) DeleteBranch(_ context.Context, _ graveler.RepositoryID, _ graveler.BranchID) error { + return nil +} + +func (m *mockRefs) ListBranches(_ context.Context, _ graveler.RepositoryID, _ graveler.BranchID) (graveler.BranchIterator, error) { + return m.listBranchesRes, nil +} + +func (m *mockRefs) GetCommit(_ context.Context, _ graveler.RepositoryID, _ graveler.CommitID) (*graveler.Commit, error) { + return &graveler.Commit{}, nil +} + +func (m *mockRefs) AddCommit(_ context.Context, _ graveler.RepositoryID, _ graveler.Commit) (graveler.CommitID, error) { + return "", nil +} + +func (m *mockRefs) FindMergeBase(_ context.Context, _ graveler.RepositoryID, _ ...graveler.CommitID) (*graveler.Commit, error) { + return &graveler.Commit{}, nil +} + +func (m *mockRefs) Log(_ context.Context, _ graveler.RepositoryID, _ graveler.CommitID) (graveler.CommitIterator, error) { + return m.commitIter, nil +} + +type ListingIter struct { + current int + listings []graveler.Listing + err error +} + +func newListingIter(listings []graveler.Listing) *ListingIter { + return &ListingIter{listings: listings, current: -1} +} + +func (r *ListingIter) Next() bool { + r.current++ + return r.current < len(r.listings) +} + +func (r *ListingIter) SeekGE(id graveler.Key) { + for i, listing := range r.listings { + if bytes.Compare(id, listing.Key) >= 0 { + r.current = i - 1 + } + } + r.current = len(r.listings) +} + +func (r *ListingIter) Value() *graveler.Listing { + if r.current < 0 || r.current >= len(r.listings) { + return nil + } + return &r.listings[r.current] +} + +func (r *ListingIter) Err() error { + return r.err +} + +func (r *ListingIter) Close() { + return +} + +type diffIter struct { + current int + records []graveler.Diff + err error +} + +func newDiffIter(records []graveler.Diff) *diffIter { + return &diffIter{records: records, current: -1} +} +func (r *diffIter) Next() bool { + r.current++ + return r.current < len(r.records) +} + +func (r *diffIter) SeekGE(id graveler.Key) { + for i, record := range r.records { + if bytes.Compare(id, record.Key) >= 0 { + r.current = i - 1 + } + } + r.current = len(r.records) +} + +func (r *diffIter) Value() *graveler.Diff { + if r.current < 0 || r.current >= len(r.records) { + return nil + } + return &r.records[r.current] +} + +func (r *diffIter) Err() error { + return r.err +} + +func (r *diffIter) Close() { + return +} + +type mockValueIterator struct { + current int + records []graveler.ValueRecord + err error +} + +func newMockValueIterator(records []graveler.ValueRecord) graveler.ValueIterator { + return &mockValueIterator{records: records, current: -1} +} + +func (r *mockValueIterator) Next() bool { + r.current++ + return r.current < len(r.records) +} + +func (r *mockValueIterator) SeekGE(id graveler.Key) { + for i, record := range r.records { + if bytes.Compare(record.Key, id) >= 0 { + r.current = i - 1 + return + } + } + r.current = len(r.records) +} + +func (r *mockValueIterator) Value() *graveler.ValueRecord { + if r.current < 0 || r.current >= len(r.records) { + return nil + } + return &r.records[r.current] +} + +func (r *mockValueIterator) Err() error { + return r.err +} + +func (r *mockValueIterator) Close() { + return +} + +type mockReference struct { + refType graveler.ReferenceType + branch graveler.Branch + commitId graveler.CommitID +} + +// newMockReference returns a mockReference +// if branch parameter is empty branch record will be nil +func newMockReference(refType graveler.ReferenceType, branchID graveler.BranchID, commitId graveler.CommitID) *mockReference { + var branch graveler.Branch + if branchID != "" { + branch = graveler.Branch{CommitID: commitId} + + } + return &mockReference{ + refType: refType, + branch: branch, + commitId: commitId, + } +} + +func (m *mockReference) Type() graveler.ReferenceType { + return m.refType +} + +func (m *mockReference) Branch() graveler.Branch { + return m.branch +} + +func (m *mockReference) CommitID() graveler.CommitID { + return m.commitId +} + +func compareListingIterators(t *testing.T, got, expected graveler.ListingIterator) { + t.Helper() + for got.Next() { + if !expected.Next() { + t.Fatalf("got next returned true where expected next returned false") + } + if diff := deep.Equal(got.Value(), expected.Value()); diff != nil { + t.Errorf("unexpected diff %s", diff) + } + } + if expected.Next() { + t.Fatalf("expected next returned true where got next returned false") + } +} diff --git a/graveler/graveler_test.go b/graveler/graveler_test.go index 9a286aa5bba..daf946c9f42 100644 --- a/graveler/graveler_test.go +++ b/graveler/graveler_test.go @@ -1,7 +1,6 @@ package graveler_test import ( - "bytes" "context" "errors" "testing" @@ -10,330 +9,7 @@ import ( "github.com/treeverse/lakefs/graveler" ) -type committedMock struct { - Value *graveler.Value - ValueIterator graveler.ValueIterator - diffIterator graveler.DiffIterator - err error - treeID graveler.TreeID -} - -func (c *committedMock) Get(_ context.Context, _ graveler.StorageNamespace, _ graveler.TreeID, _ graveler.Key) (*graveler.Value, error) { - if c.err != nil { - return nil, c.err - } - return c.Value, nil -} - -func (c *committedMock) List(_ context.Context, _ graveler.StorageNamespace, _ graveler.TreeID, _ graveler.Key) (graveler.ValueIterator, error) { - if c.err != nil { - return nil, c.err - } - return c.ValueIterator, nil -} - -func (c *committedMock) Diff(_ context.Context, _ graveler.StorageNamespace, _, _, _ graveler.TreeID, _ graveler.Key) (graveler.DiffIterator, error) { - if c.err != nil { - return nil, c.err - } - return c.diffIterator, nil -} - -func (c *committedMock) Merge(_ context.Context, _ graveler.StorageNamespace, _, _, _ graveler.TreeID) (graveler.TreeID, error) { - if c.err != nil { - return "", c.err - } - return c.treeID, nil -} - -func (c *committedMock) Apply(_ context.Context, _ graveler.StorageNamespace, _ graveler.TreeID, _ graveler.ValueIterator) (graveler.TreeID, error) { - if c.err != nil { - return "", c.err - } - return c.treeID, nil -} - -type stagingMock struct { - err error - Value *graveler.Value - ValueIterator graveler.ValueIterator - stagingToken graveler.StagingToken -} - -func (s *stagingMock) Drop(_ context.Context, _ graveler.StagingToken) error { - if s.err != nil { - return s.err - } - return nil -} - -func (s *stagingMock) Get(_ context.Context, _ graveler.StagingToken, _ graveler.Key) (*graveler.Value, error) { - if s.err != nil { - return nil, s.err - } - return s.Value, nil -} - -func (s *stagingMock) Set(_ context.Context, _ graveler.StagingToken, _ graveler.Key, _ graveler.Value) error { - if s.err != nil { - return s.err - } - return nil -} - -func (s *stagingMock) Delete(_ context.Context, _ graveler.StagingToken, _ graveler.Key) error { - return nil -} - -func (s *stagingMock) List(_ context.Context, _ graveler.StagingToken) (graveler.ValueIterator, error) { - if s.err != nil { - return nil, s.err - } - return s.ValueIterator, nil -} - -func (s *stagingMock) Snapshot(_ context.Context, _ graveler.StagingToken) (graveler.StagingToken, error) { - if s.err != nil { - return "", s.err - } - return s.stagingToken, nil -} - -func (s *stagingMock) ListSnapshot(_ context.Context, _ graveler.StagingToken, _ graveler.Key) (graveler.ValueIterator, error) { - if s.err != nil { - return nil, s.err - } - return s.ValueIterator, nil -} - -type mockRefs struct { - listRepositoriesRes graveler.RepositoryIterator - listBranchesRes graveler.BranchIterator - commitIter graveler.CommitIterator - refType graveler.ReferenceType -} - -const defaultBranchName = graveler.BranchID("master") - -func (m *mockRefs) RevParse(_ context.Context, _ graveler.RepositoryID, _ graveler.Ref) (graveler.Reference, error) { - var branch graveler.BranchID - if m.refType == graveler.ReferenceTypeBranch { - branch = defaultBranchName - } - return newMockReference(m.refType, branch, ""), nil -} - -func (m *mockRefs) GetRepository(_ context.Context, _ graveler.RepositoryID) (*graveler.Repository, error) { - return &graveler.Repository{}, nil -} - -func (m *mockRefs) CreateRepository(_ context.Context, _ graveler.RepositoryID, _ graveler.Repository, _ graveler.Branch) error { - return nil -} - -func (m *mockRefs) ListRepositories(_ context.Context, _ graveler.RepositoryID) (graveler.RepositoryIterator, error) { - return m.listRepositoriesRes, nil -} - -func (m *mockRefs) DeleteRepository(_ context.Context, _ graveler.RepositoryID) error { - return nil -} - -func (m *mockRefs) GetBranch(_ context.Context, _ graveler.RepositoryID, _ graveler.BranchID) (*graveler.Branch, error) { - return &graveler.Branch{}, nil -} - -func (m *mockRefs) SetBranch(_ context.Context, _ graveler.RepositoryID, _ graveler.BranchID, _ graveler.Branch) error { - return nil -} - -func (m *mockRefs) DeleteBranch(_ context.Context, _ graveler.RepositoryID, _ graveler.BranchID) error { - return nil -} - -func (m *mockRefs) ListBranches(_ context.Context, _ graveler.RepositoryID, _ graveler.BranchID) (graveler.BranchIterator, error) { - return m.listBranchesRes, nil -} - -func (m *mockRefs) GetCommit(_ context.Context, _ graveler.RepositoryID, _ graveler.CommitID) (*graveler.Commit, error) { - return &graveler.Commit{}, nil -} - -func (m *mockRefs) AddCommit(_ context.Context, _ graveler.RepositoryID, _ graveler.Commit) (graveler.CommitID, error) { - return "", nil -} - -func (m *mockRefs) FindMergeBase(_ context.Context, _ graveler.RepositoryID, _ ...graveler.CommitID) (*graveler.Commit, error) { - return &graveler.Commit{}, nil -} - -func (m *mockRefs) Log(_ context.Context, _ graveler.RepositoryID, _ graveler.CommitID) (graveler.CommitIterator, error) { - return m.commitIter, nil -} - -type ListingIter struct { - current int - listings []graveler.Listing - err error -} - -func newListingIter(listings []graveler.Listing) *ListingIter { - return &ListingIter{listings: listings, current: -1} -} - -func (r *ListingIter) Next() bool { - r.current++ - return r.current < len(r.listings) -} - -func (r *ListingIter) SeekGE(id graveler.Key) { - for i, listing := range r.listings { - if bytes.Compare(id, listing.Key) >= 0 { - r.current = i - 1 - } - } - r.current = len(r.listings) -} - -func (r *ListingIter) Value() *graveler.Listing { - if r.current < 0 || r.current >= len(r.listings) { - return nil - } - return &r.listings[r.current] -} - -func (r *ListingIter) Err() error { - return r.err -} - -func (r *ListingIter) Close() { - return -} - -type diffIter struct { - current int - records []graveler.Diff - err error -} - -func newDiffIter(records []graveler.Diff) *diffIter { - return &diffIter{records: records, current: -1} -} -func (r *diffIter) Next() bool { - r.current++ - return r.current < len(r.records) -} - -func (r *diffIter) SeekGE(id graveler.Key) { - for i, record := range r.records { - if bytes.Compare(id, record.Key) >= 0 { - r.current = i - 1 - } - } - r.current = len(r.records) -} - -func (r *diffIter) Value() *graveler.Diff { - if r.current < 0 || r.current >= len(r.records) { - return nil - } - return &r.records[r.current] -} - -func (r *diffIter) Err() error { - return r.err -} - -func (r *diffIter) Close() { - return -} - -type ValueIter struct { - current int - records []graveler.ValueRecord - err error -} - -func newValueIter(records []graveler.ValueRecord) *ValueIter { - return &ValueIter{records: records, current: -1} -} -func (r *ValueIter) Next() bool { - r.current++ - return r.current < len(r.records) -} - -func (r *ValueIter) SeekGE(id graveler.Key) { - for i, record := range r.records { - if bytes.Compare(record.Key, id) >= 0 { - r.current = i - 1 - return - } - } - r.current = len(r.records) -} - -func (r *ValueIter) Value() *graveler.ValueRecord { - if r.current < 0 || r.current >= len(r.records) { - return nil - } - return &r.records[r.current] -} - -func (r *ValueIter) Err() error { - return r.err -} - -func (r *ValueIter) Close() { - return -} - -type mockReference struct { - refType graveler.ReferenceType - branch graveler.Branch - commitId graveler.CommitID -} - -// newMockReference returns a mockReference -// if branch parameter is empty branch record will be nil -func newMockReference(refType graveler.ReferenceType, branchID graveler.BranchID, commitId graveler.CommitID) *mockReference { - var branch graveler.Branch - if branchID != "" { - branch = graveler.Branch{CommitID: commitId} - - } - return &mockReference{ - refType: refType, - branch: branch, - commitId: commitId, - } -} -func (m *mockReference) Type() graveler.ReferenceType { - return m.refType -} - -func (m *mockReference) Branch() graveler.Branch { - return m.branch -} - -func (m *mockReference) CommitID() graveler.CommitID { - return m.commitId -} - -func compareListingIterators(got, expected graveler.ListingIterator, t *testing.T) { - for got.Next() { - if !expected.Next() { - t.Fatalf("got next returned true where expected next returned false") - } - if diff := deep.Equal(got.Value(), expected.Value()); diff != nil { - t.Errorf("unexpected diff %s", diff) - } - } - if expected.Next() { - t.Fatalf("expected next returned true where got next returned false") - } -} - -func TestGravel_prefixIterator(t *testing.T) { +func TestGraveler_PrefixIterator(t *testing.T) { tests := []struct { name string valueIter graveler.ValueIterator @@ -343,40 +19,40 @@ func TestGravel_prefixIterator(t *testing.T) { }{ { name: "no prefix", - valueIter: newValueIter([]graveler.ValueRecord{{Key: []byte("foo")}}), - expectedPrefixIter: newValueIter([]graveler.ValueRecord{{Key: []byte("foo")}}), + valueIter: newMockValueIterator([]graveler.ValueRecord{{Key: []byte("foo")}}), + expectedPrefixIter: newMockValueIterator([]graveler.ValueRecord{{Key: []byte("foo")}}), }, { - name: "no files ", - valueIter: newValueIter([]graveler.ValueRecord{{Key: []byte("other/path/foo")}}), + name: "no files", + valueIter: newMockValueIterator([]graveler.ValueRecord{{Key: []byte("other/path/foo")}}), prefix: []byte("path/"), - expectedPrefixIter: newValueIter([]graveler.ValueRecord{}), + expectedPrefixIter: newMockValueIterator([]graveler.ValueRecord{}), }, { - name: "one file ", - valueIter: newValueIter([]graveler.ValueRecord{{Key: []byte("path/foo")}}), + name: "one file", + valueIter: newMockValueIterator([]graveler.ValueRecord{{Key: []byte("path/foo")}}), prefix: []byte("path/"), - expectedPrefixIter: newValueIter([]graveler.ValueRecord{{Key: []byte("path/foo"), Value: nil}}), + expectedPrefixIter: newMockValueIterator([]graveler.ValueRecord{{Key: []byte("path/foo"), Value: nil}}), }, { - name: "one file in prefix ", + name: "one file in prefix", prefix: []byte("path/"), - valueIter: newValueIter([]graveler.ValueRecord{{Key: []byte("before/foo")}, {Key: []byte("path/foo")}, {Key: []byte("last/foo")}}), - expectedPrefixIter: newValueIter([]graveler.ValueRecord{{Key: []byte("path/foo"), Value: nil}}), + valueIter: newMockValueIterator([]graveler.ValueRecord{{Key: []byte("before/foo")}, {Key: []byte("path/foo")}, {Key: []byte("last/foo")}}), + expectedPrefixIter: newMockValueIterator([]graveler.ValueRecord{{Key: []byte("path/foo"), Value: nil}}), }, { name: "seek before", prefix: []byte("path/"), - valueIter: newValueIter([]graveler.ValueRecord{{Key: []byte("before/foo")}, {Key: []byte("path/foo")}, {Key: []byte("last/foo")}}), + valueIter: newMockValueIterator([]graveler.ValueRecord{{Key: []byte("before/foo")}, {Key: []byte("path/foo")}, {Key: []byte("last/foo")}}), seekTo: []byte("before/"), - expectedPrefixIter: newValueIter([]graveler.ValueRecord{{Key: []byte("path/foo"), Value: nil}}), + expectedPrefixIter: newMockValueIterator([]graveler.ValueRecord{{Key: []byte("path/foo"), Value: nil}}), }, { name: "seek after", prefix: []byte("path/"), - valueIter: newValueIter([]graveler.ValueRecord{{Key: []byte("before/foo")}, {Key: []byte("path/foo")}, {Key: []byte("z_after/foo")}}), + valueIter: newMockValueIterator([]graveler.ValueRecord{{Key: []byte("before/foo")}, {Key: []byte("path/foo")}, {Key: []byte("z_after/foo")}}), seekTo: []byte("z_after/"), - expectedPrefixIter: newValueIter([]graveler.ValueRecord{}), + expectedPrefixIter: newMockValueIterator([]graveler.ValueRecord{}), }, } @@ -401,7 +77,7 @@ func TestGravel_prefixIterator(t *testing.T) { } } -func TestGravel_listingIterator(t *testing.T) { +func TestGraveler_ListingIterator(t *testing.T) { tests := []struct { name string valueIter graveler.ValueIterator @@ -411,71 +87,71 @@ func TestGravel_listingIterator(t *testing.T) { }{ { name: "no file", - valueIter: newValueIter([]graveler.ValueRecord{}), + valueIter: newMockValueIterator([]graveler.ValueRecord{}), delimiter: []byte("/"), prefix: nil, expectedListingIter: newListingIter([]graveler.Listing{}), }, { name: "one file no delimiter", - valueIter: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo")}}), + valueIter: newMockValueIterator([]graveler.ValueRecord{{Key: graveler.Key("foo")}}), delimiter: nil, prefix: nil, expectedListingIter: newListingIter([]graveler.Listing{{CommonPrefix: false, Key: graveler.Key("foo")}}), }, { name: "one file", - valueIter: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo")}}), + valueIter: newMockValueIterator([]graveler.ValueRecord{{Key: graveler.Key("foo")}}), delimiter: []byte("/"), prefix: nil, expectedListingIter: newListingIter([]graveler.Listing{{CommonPrefix: false, Key: graveler.Key("foo")}}), }, { name: "one common prefix", - valueIter: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/bar")}, {Key: graveler.Key("foo/bar2")}}), + valueIter: newMockValueIterator([]graveler.ValueRecord{{Key: graveler.Key("foo/bar")}, {Key: graveler.Key("foo/bar2")}}), delimiter: []byte("/"), prefix: nil, - expectedListingIter: newListingIter([]graveler.Listing{{CommonPrefix: true, Key: graveler.Key("foo")}}), + expectedListingIter: newListingIter([]graveler.Listing{{CommonPrefix: true, Key: graveler.Key("foo/")}}), }, { name: "one common prefix one file", - valueIter: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/bar")}, {Key: graveler.Key("foo/bar2")}, {Key: graveler.Key("foo/bar3")}, {Key: graveler.Key("foo/bar4")}, {Key: graveler.Key("fooFighter")}}), + valueIter: newMockValueIterator([]graveler.ValueRecord{{Key: graveler.Key("foo/bar")}, {Key: graveler.Key("foo/bar2")}, {Key: graveler.Key("foo/bar3")}, {Key: graveler.Key("foo/bar4")}, {Key: graveler.Key("fooFighter")}}), delimiter: []byte("/"), prefix: nil, - expectedListingIter: newListingIter([]graveler.Listing{{CommonPrefix: true, Key: graveler.Key("foo")}, {CommonPrefix: false, Key: graveler.Key("fooFighter")}}), + expectedListingIter: newListingIter([]graveler.Listing{{CommonPrefix: true, Key: graveler.Key("foo/")}, {CommonPrefix: false, Key: graveler.Key("fooFighter")}}), }, { name: "one file with prefix", - valueIter: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("path/to/foo")}}), + valueIter: newMockValueIterator([]graveler.ValueRecord{{Key: graveler.Key("path/to/foo")}}), delimiter: []byte("/"), prefix: []byte("path/to/"), expectedListingIter: newListingIter([]graveler.Listing{{CommonPrefix: false, Key: graveler.Key("path/to/foo")}}), }, { - name: "one common prefix", - valueIter: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("path/to/foo/bar")}, {Key: graveler.Key("path/to/foo/bar2")}}), + name: "one common prefix with prefix", + valueIter: newMockValueIterator([]graveler.ValueRecord{{Key: graveler.Key("path/to/foo/bar")}, {Key: graveler.Key("path/to/foo/bar2")}}), delimiter: []byte("/"), prefix: []byte("path/to/"), - expectedListingIter: newListingIter([]graveler.Listing{{CommonPrefix: true, Key: graveler.Key("path/to/foo")}}), + expectedListingIter: newListingIter([]graveler.Listing{{CommonPrefix: true, Key: graveler.Key("path/to/foo/")}}), }, { - name: "one common prefix one file", - valueIter: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("path/to/foo/bar")}, {Key: graveler.Key("path/to/foo/bar2")}, {Key: graveler.Key("path/to/foo/bar3")}, {Key: graveler.Key("path/to/foo/bar4")}, {Key: graveler.Key("path/to/fooFighter")}}), + name: "one common prefix one file with prefix", + valueIter: newMockValueIterator([]graveler.ValueRecord{{Key: graveler.Key("path/to/foo/bar")}, {Key: graveler.Key("path/to/foo/bar2")}, {Key: graveler.Key("path/to/foo/bar3")}, {Key: graveler.Key("path/to/foo/bar4")}, {Key: graveler.Key("path/to/fooFighter")}}), delimiter: []byte("/"), prefix: []byte("path/to/"), - expectedListingIter: newListingIter([]graveler.Listing{{CommonPrefix: true, Key: graveler.Key("path/to/foo")}, {CommonPrefix: false, Key: graveler.Key("path/to/fooFighter")}}), + expectedListingIter: newListingIter([]graveler.Listing{{CommonPrefix: true, Key: graveler.Key("path/to/foo/")}, {CommonPrefix: false, Key: graveler.Key("path/to/fooFighter")}}), }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - listingIter := graveler.NewListingIter(tt.valueIter, tt.delimiter, tt.prefix) - compareListingIterators(listingIter, tt.expectedListingIter, t) + listingIter := graveler.NewListingIterator(tt.valueIter, tt.delimiter, tt.prefix) + compareListingIterators(t, listingIter, tt.expectedListingIter) }) } } -func TestGravel_List(t *testing.T) { +func TestGraveler_List(t *testing.T) { tests := []struct { name string r graveler.Graveler @@ -489,8 +165,8 @@ func TestGravel_List(t *testing.T) { }{ { name: "one committed one staged no paths", - r: graveler.NewGraveler(&committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo"), Value: &graveler.Value{}}})}, - &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("bar"), Value: &graveler.Value{}}})}, + r: graveler.NewGraveler(&committedMock{ValueIterator: newMockValueIterator([]graveler.ValueRecord{{Key: graveler.Key("foo"), Value: &graveler.Value{}}})}, + &stagingMock{ValueIterator: newMockValueIterator([]graveler.ValueRecord{{Key: graveler.Key("bar"), Value: &graveler.Value{}}})}, &mockRefs{refType: graveler.ReferenceTypeBranch}, ), delimiter: graveler.Key("/"), @@ -500,8 +176,8 @@ func TestGravel_List(t *testing.T) { }, { name: "same path different file", - r: graveler.NewGraveler(&committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo"), Value: &graveler.Value{Identity: []byte("original")}}})}, - &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo"), Value: &graveler.Value{Identity: []byte("other")}}})}, + r: graveler.NewGraveler(&committedMock{ValueIterator: newMockValueIterator([]graveler.ValueRecord{{Key: graveler.Key("foo"), Value: &graveler.Value{Identity: []byte("original")}}})}, + &stagingMock{ValueIterator: newMockValueIterator([]graveler.ValueRecord{{Key: graveler.Key("foo"), Value: &graveler.Value{Identity: []byte("other")}}})}, &mockRefs{refType: graveler.ReferenceTypeBranch}, ), delimiter: graveler.Key("/"), @@ -511,8 +187,8 @@ func TestGravel_List(t *testing.T) { }, { name: "one committed one staged no paths - with prefix", - r: graveler.NewGraveler(&committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("prefix/foo"), Value: &graveler.Value{}}})}, - &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("prefix/bar"), Value: &graveler.Value{}}})}, + r: graveler.NewGraveler(&committedMock{ValueIterator: newMockValueIterator([]graveler.ValueRecord{{Key: graveler.Key("prefix/foo"), Value: &graveler.Value{}}})}, + &stagingMock{ValueIterator: newMockValueIterator([]graveler.ValueRecord{{Key: graveler.Key("prefix/bar"), Value: &graveler.Value{}}})}, &mockRefs{refType: graveler.ReferenceTypeBranch}, ), delimiter: graveler.Key("/"), @@ -522,8 +198,8 @@ func TestGravel_List(t *testing.T) { }, { name: "objects and paths in both committed and staging", - r: graveler.NewGraveler(&committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("prefix/pathA/foo"), Value: &graveler.Value{}}, {Key: graveler.Key("prefix/pathA/foo2"), Value: &graveler.Value{}}, {Key: graveler.Key("prefix/pathB/foo"), Value: &graveler.Value{}}})}, - &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("prefix/file"), Value: &graveler.Value{}}, {Key: graveler.Key("prefix/pathA/bar"), Value: &graveler.Value{}}, {Key: graveler.Key("prefix/pathB/bar"), Value: &graveler.Value{}}})}, + r: graveler.NewGraveler(&committedMock{ValueIterator: newMockValueIterator([]graveler.ValueRecord{{Key: graveler.Key("prefix/pathA/foo"), Value: &graveler.Value{}}, {Key: graveler.Key("prefix/pathA/foo2"), Value: &graveler.Value{}}, {Key: graveler.Key("prefix/pathB/foo"), Value: &graveler.Value{}}})}, + &stagingMock{ValueIterator: newMockValueIterator([]graveler.ValueRecord{{Key: graveler.Key("prefix/file"), Value: &graveler.Value{}}, {Key: graveler.Key("prefix/pathA/bar"), Value: &graveler.Value{}}, {Key: graveler.Key("prefix/pathB/bar"), Value: &graveler.Value{}}})}, &mockRefs{refType: graveler.ReferenceTypeBranch}, ), delimiter: graveler.Key("/"), @@ -536,11 +212,11 @@ func TestGravel_List(t *testing.T) { Value: &graveler.Value{}, }, { CommonPrefix: true, - Key: graveler.Key("prefix/pathA"), + Key: graveler.Key("prefix/pathA/"), Value: nil, }, { CommonPrefix: true, - Key: graveler.Key("prefix/pathB"), + Key: graveler.Key("prefix/pathB/"), Value: nil, }}), }, @@ -549,18 +225,18 @@ func TestGravel_List(t *testing.T) { t.Run(tt.name, func(t *testing.T) { listing, err := tt.r.List(context.Background(), "", "", tt.prefix, tt.from, tt.delimiter) if err != tt.expectedErr { - t.Fatalf("wrong error, expected:%s got:%s ", tt.expectedErr, err) + t.Fatalf("wrong error, expected:%s got:%s", tt.expectedErr, err) } if err != nil { return // err == tt.expectedErr } // compare iterators - compareListingIterators(listing, tt.expectedListing, t) + compareListingIterators(t, listing, tt.expectedListing) }) } } -func TestGravel_Get(t *testing.T) { +func TestGraveler_Get(t *testing.T) { var ErrTest = errors.New("some kind of err") tests := []struct { name string @@ -603,7 +279,7 @@ func TestGravel_Get(t *testing.T) { expectedValueResult: graveler.Value{Identity: []byte("staged")}, }, { - name: "branch - only committed", + name: "branch - only committed", r: graveler.NewGraveler(&committedMock{Value: &graveler.Value{Identity: []byte("committed")}}, &stagingMock{err: graveler.ErrNotFound}, &mockRefs{refType: graveler.ReferenceTypeBranch}, @@ -611,7 +287,7 @@ func TestGravel_Get(t *testing.T) { expectedValueResult: graveler.Value{Identity: []byte("committed")}, }, { - name: "branch - tombstone", + name: "branch - tombstone", r: graveler.NewGraveler(&committedMock{Value: &graveler.Value{Identity: []byte("committed")}}, &stagingMock{Value: nil}, &mockRefs{refType: graveler.ReferenceTypeBranch}, @@ -619,7 +295,7 @@ func TestGravel_Get(t *testing.T) { expectedErr: graveler.ErrNotFound, }, { - name: "branch - staged return error", + name: "branch - staged return error", r: graveler.NewGraveler(&committedMock{}, &stagingMock{err: ErrTest}, &mockRefs{refType: graveler.ReferenceTypeBranch}, ), @@ -630,31 +306,19 @@ func TestGravel_Get(t *testing.T) { t.Run(tt.name, func(t *testing.T) { Value, err := tt.r.Get(context.Background(), "", "", nil) if err != tt.expectedErr { - t.Fatalf("wrong error, expected:%s got:%s ", tt.expectedErr, err) + t.Fatalf("wrong error, expected:%s got:%s", tt.expectedErr, err) } if err != nil { return // err == tt.expected error } if string(tt.expectedValueResult.Identity) != string(Value.Identity) { - t.Errorf("wrong Value address, expected:%s got:%s ", tt.expectedValueResult.Identity, Value.Identity) + t.Errorf("wrong Value address, expected:%s got:%s", tt.expectedValueResult.Identity, Value.Identity) } }) } } -func TestGravel_Merge(t *testing.T) { - -} - -func TestGravel_Reset(t *testing.T) { - -} - -func TestGravel_Revert(t *testing.T) { - -} - -func TestGravel_DiffUncommitted(t *testing.T) { +func TestGraveler_DiffUncommitted(t *testing.T) { tests := []struct { name string r graveler.Graveler @@ -665,8 +329,8 @@ func TestGravel_DiffUncommitted(t *testing.T) { }{ { name: "no changes", - r: graveler.NewGraveler(&committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}}), err: graveler.ErrNotFound}, - &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{})}, + r: graveler.NewGraveler(&committedMock{ValueIterator: newMockValueIterator([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}}), err: graveler.ErrNotFound}, + &stagingMock{ValueIterator: newMockValueIterator([]graveler.ValueRecord{})}, &mockRefs{}, ), amount: 10, @@ -674,8 +338,8 @@ func TestGravel_DiffUncommitted(t *testing.T) { }, { name: "added one", - r: graveler.NewGraveler(&committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{}), err: graveler.ErrNotFound}, - &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}})}, + r: graveler.NewGraveler(&committedMock{ValueIterator: newMockValueIterator([]graveler.ValueRecord{}), err: graveler.ErrNotFound}, + &stagingMock{ValueIterator: newMockValueIterator([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}})}, &mockRefs{}, ), amount: 10, @@ -687,8 +351,8 @@ func TestGravel_DiffUncommitted(t *testing.T) { }, { name: "changed one", - r: graveler.NewGraveler(&committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}})}, - &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}})}, + r: graveler.NewGraveler(&committedMock{ValueIterator: newMockValueIterator([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}})}, + &stagingMock{ValueIterator: newMockValueIterator([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}})}, &mockRefs{}, ), amount: 10, @@ -700,8 +364,8 @@ func TestGravel_DiffUncommitted(t *testing.T) { }, { name: "removed one", - r: graveler.NewGraveler(&committedMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}})}, - &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: nil}})}, + r: graveler.NewGraveler(&committedMock{ValueIterator: newMockValueIterator([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}})}, + &stagingMock{ValueIterator: newMockValueIterator([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: nil}})}, &mockRefs{}, ), amount: 10, @@ -716,7 +380,7 @@ func TestGravel_DiffUncommitted(t *testing.T) { t.Run(tt.name, func(t *testing.T) { diff, err := tt.r.DiffUncommitted(context.Background(), "repo", "branch", graveler.Key("from")) if err != tt.expectedErr { - t.Fatalf("wrong error, expected:%s got:%s ", tt.expectedErr, err) + t.Fatalf("wrong error, expected:%s got:%s", tt.expectedErr, err) } if err != nil { return // err == tt.expectedErr @@ -738,9 +402,9 @@ func TestGravel_DiffUncommitted(t *testing.T) { } } -func TestGravel_UpdateBranch(t *testing.T) { +func TestGraveler_UpdateBranch(t *testing.T) { gravel := graveler.NewGraveler(nil, - &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}})}, + &stagingMock{ValueIterator: newMockValueIterator([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}})}, &mockRefs{}, ) _, err := gravel.UpdateBranch(context.Background(), "", "", "") @@ -748,7 +412,7 @@ func TestGravel_UpdateBranch(t *testing.T) { t.Fatal("expected update to fail on conflict") } gravel = graveler.NewGraveler(nil, - &stagingMock{ValueIterator: newValueIter([]graveler.ValueRecord{})}, + &stagingMock{ValueIterator: newMockValueIterator([]graveler.ValueRecord{})}, &mockRefs{}, ) _, err = gravel.UpdateBranch(context.Background(), "", "", "") diff --git a/graveler/listing_iterator.go b/graveler/listing_iterator.go index c3e62ae8dd9..07fd4191094 100644 --- a/graveler/listing_iterator.go +++ b/graveler/listing_iterator.go @@ -2,16 +2,14 @@ package graveler import "bytes" -// ListingIter implements a listing iterator using a ValueIterator +// listingIterator implements a ListingIterator using a ValueIterator // assumes all values in valueIterator start with prefix -type ListingIter struct { +type listingIterator struct { valueIterator ValueIterator delimiter Key - nextDelimiter Key prefix Key current *Listing - nextFunc func(l *ListingIter) bool - err error + nextFunc func() bool } // getFollowingValue returns the following value (i.e will increase the last byte by 1) @@ -25,26 +23,21 @@ func getFollowingValue(value []byte) []byte { return append(copiedDelimiter[:len(copiedDelimiter)-1], copiedDelimiter[len(copiedDelimiter)-1]+1) } -func NewListingIter(iterator ValueIterator, delimiter, prefix Key) *ListingIter { - var nextDelimiter Key - var nextFunc func(l *ListingIter) bool - if len(delimiter) == 0 { - nextFunc = nextNoDelimiter - } else { - nextFunc = nextWithDelimiter - nextDelimiter = getFollowingValue(delimiter) - } - - return &ListingIter{ +func NewListingIterator(iterator ValueIterator, delimiter, prefix Key) ListingIterator { + l := &listingIterator{ valueIterator: iterator, delimiter: delimiter, - nextDelimiter: nextDelimiter, prefix: prefix, - nextFunc: nextFunc, } + if len(delimiter) == 0 { + l.nextFunc = l.nextNoDelimiter + } else { + l.nextFunc = l.nextWithDelimiter + } + return l } -func nextNoDelimiter(l *ListingIter) bool { +func (l *listingIterator) nextNoDelimiter() bool { hasNext := l.valueIterator.Next() if !hasNext { l.current = nil @@ -59,23 +52,15 @@ func nextNoDelimiter(l *ListingIter) bool { return true } -func nextWithDelimiter(l *ListingIter) bool { +func (l *listingIterator) nextWithDelimiter() bool { var hasNext bool - if l.current == nil || !l.current.CommonPrefix { - hasNext = l.valueIterator.Next() - } else { - nextKey := append(l.current.Key, l.nextDelimiter...) + if l.current != nil && l.current.CommonPrefix { + nextKey := getFollowingValue(l.current.Key) l.valueIterator.SeekGE(nextKey) - hasNext = l.valueIterator.Next() } - + hasNext = l.valueIterator.Next() if hasNext { nextValue := l.valueIterator.Value() - if !bytes.HasPrefix(nextValue.Key, l.prefix) { - l.current = nil - l.err = ErrUnexpected - return false - } l.current = l.getListingFromValue(nextValue.Value, nextValue.Key) } else { l.current = nil @@ -83,42 +68,42 @@ func nextWithDelimiter(l *ListingIter) bool { return hasNext } -func (l *ListingIter) Next() bool { - return l.nextFunc(l) +func (l *listingIterator) Next() bool { + return l.nextFunc() } -func (l *ListingIter) getListingFromValue(value *Value, key Key) *Listing { - relevantPath := key[len(l.prefix):] - delimiterIndex := bytes.Index(relevantPath, l.delimiter) - commonPrefix := delimiterIndex >= 0 - if commonPrefix { - relevantPath = relevantPath[:delimiterIndex] - value = nil +func (l *listingIterator) getListingFromValue(value *Value, key Key) *Listing { + relevantKey := key[len(l.prefix):] + delimiterIndex := bytes.Index(relevantKey, l.delimiter) + if delimiterIndex == -1 { + // return listing for non common prefix with value + return &Listing{ + CommonPrefix: false, + Key: key, + Value: value, + } } - newKey := append(l.prefix, relevantPath...) + // return listing for common prefix key + commonPrefixKey := key[:len(l.prefix)+delimiterIndex+len(l.delimiter)] return &Listing{ - Key: newKey, - CommonPrefix: commonPrefix, - Value: value, + CommonPrefix: true, + Key: commonPrefixKey, } } -func (l *ListingIter) SeekGE(id Key) { +func (l *listingIterator) SeekGE(id Key) { l.current = nil l.valueIterator.SeekGE(id) } -func (l *ListingIter) Value() *Listing { +func (l *listingIterator) Value() *Listing { return l.current } -func (l *ListingIter) Err() error { - if l.err != nil { - return l.err - } +func (l *listingIterator) Err() error { return l.valueIterator.Err() } -func (l *ListingIter) Close() { +func (l *listingIterator) Close() { l.valueIterator.Close() } diff --git a/graveler/prefix_iterator.go b/graveler/prefix_iterator.go index ad94836e0bb..bcb4af561f1 100644 --- a/graveler/prefix_iterator.go +++ b/graveler/prefix_iterator.go @@ -2,52 +2,57 @@ package graveler import "bytes" -// PrefixIterator holds a ValueIterator and iterates only over values the their Key starts with the prefix -type PrefixIterator struct { - prefix Key - valueIterator ValueIterator - current *ValueRecord +// prefixIterator holds a ValueIterator and iterates only over values the their Key starts with the prefix +type prefixIterator struct { + prefix Key + iterator ValueIterator + ended bool } -func NewPrefixIterator(iterator ValueIterator, prefix Key) *PrefixIterator { +func NewPrefixIterator(iterator ValueIterator, prefix Key) ValueIterator { iterator.SeekGE(prefix) - return &PrefixIterator{ - prefix: prefix, - valueIterator: iterator, - current: nil, + return &prefixIterator{ + prefix: prefix, + iterator: iterator, } } -func (p *PrefixIterator) Next() bool { - if !p.valueIterator.Next() { - p.current = nil +func (p *prefixIterator) Next() bool { + if p.ended { return false } - val := p.valueIterator.Value() + if !p.iterator.Next() { + p.ended = true + return false + } + val := p.iterator.Value() if !bytes.HasPrefix(val.Key, p.prefix) { - p.current = nil + p.ended = true return false } - p.current = val return true } -func (p *PrefixIterator) SeekGE(id Key) { +func (p *prefixIterator) SeekGE(id Key) { + from := id if bytes.Compare(id, p.prefix) <= 0 { - p.valueIterator.SeekGE(p.prefix) - return + from = p.prefix } - p.valueIterator.SeekGE(id) + p.iterator.SeekGE(from) + p.ended = false } -func (p *PrefixIterator) Value() *ValueRecord { - return p.current +func (p *prefixIterator) Value() *ValueRecord { + if p.ended { + return nil + } + return p.iterator.Value() } -func (p *PrefixIterator) Err() error { - return p.valueIterator.Err() +func (p *prefixIterator) Err() error { + return p.iterator.Err() } -func (p *PrefixIterator) Close() { - p.valueIterator.Close() +func (p *prefixIterator) Close() { + p.iterator.Close() } diff --git a/graveler/uncommitted_iterator.go b/graveler/uncommitted_diff_iterator.go similarity index 80% rename from graveler/uncommitted_iterator.go rename to graveler/uncommitted_diff_iterator.go index ded9d207b60..07251182322 100644 --- a/graveler/uncommitted_iterator.go +++ b/graveler/uncommitted_diff_iterator.go @@ -7,7 +7,7 @@ import ( "github.com/treeverse/lakefs/logging" ) -type UncommittedDiffIterator struct { +type uncommittedDiffIterator struct { committedManager CommittedManager list ValueIterator sn StorageNamespace @@ -16,8 +16,8 @@ type UncommittedDiffIterator struct { err error } -func newUncommittedDiffIterator(manager CommittedManager, list ValueIterator, sn StorageNamespace, treeItreeID TreeID) *UncommittedDiffIterator { - return &UncommittedDiffIterator{ +func NewUncommittedDiffIterator(manager CommittedManager, list ValueIterator, sn StorageNamespace, treeItreeID TreeID) DiffIterator { + return &uncommittedDiffIterator{ committedManager: manager, list: list, sn: sn, @@ -58,7 +58,7 @@ func getDiffType(ctx context.Context, committedManager CommittedManager, sn Stor return diffType, nil } -func (d *UncommittedDiffIterator) Next() bool { +func (d *uncommittedDiffIterator) Next() bool { if !d.list.Next() { d.value = nil return false @@ -78,19 +78,19 @@ func (d *UncommittedDiffIterator) Next() bool { return true } -func (d *UncommittedDiffIterator) SeekGE(id Key) { +func (d *uncommittedDiffIterator) SeekGE(id Key) { d.value = nil d.list.SeekGE(id) } -func (d *UncommittedDiffIterator) Value() *Diff { +func (d *uncommittedDiffIterator) Value() *Diff { return d.value } -func (d *UncommittedDiffIterator) Err() error { +func (d *uncommittedDiffIterator) Err() error { return d.err } -func (d *UncommittedDiffIterator) Close() { +func (d *uncommittedDiffIterator) Close() { d.list.Close() } From feebf51ea71d77a551e2b9911d2f246744ff1f90 Mon Sep 17 00:00:00 2001 From: guy-har <60321938+guy-har@users.noreply.github.com> Date: Thu, 10 Dec 2020 13:28:14 +0200 Subject: [PATCH 05/14] Update graveler/uncommitted_diff_iterator.go Co-authored-by: Barak Amar --- graveler/uncommitted_diff_iterator.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/graveler/uncommitted_diff_iterator.go b/graveler/uncommitted_diff_iterator.go index 07251182322..9f2242ff00f 100644 --- a/graveler/uncommitted_diff_iterator.go +++ b/graveler/uncommitted_diff_iterator.go @@ -40,8 +40,7 @@ func getDiffType(ctx context.Context, committedManager CommittedManager, sn Stor if err != nil { return 0, err } - var diffType DiffType - diffType = DiffTypeAdded + diffType := DiffTypeAdded if tombstone && existsInCommitted { diffType = DiffTypeRemoved } From dc53534d6a5c408a79cb3cf1af02bdef83a41760 Mon Sep 17 00:00:00 2001 From: guy-har <60321938+guy-har@users.noreply.github.com> Date: Thu, 10 Dec 2020 13:30:17 +0200 Subject: [PATCH 06/14] Update graveler/combined_iterator.go Co-authored-by: Barak Amar --- graveler/combined_iterator.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/graveler/combined_iterator.go b/graveler/combined_iterator.go index d047cf4729a..eccf7c66e02 100644 --- a/graveler/combined_iterator.go +++ b/graveler/combined_iterator.go @@ -49,12 +49,12 @@ func (c *combinedIterator) Next() bool { c.iterB.Next() } - if c.iterA.Err() != nil || c.iterB.Err() != nil { - if c.iterA.Err() != nil { - c.p = c.iterA - } else { - c.p = c.iterB - } + if c.iterA.Err() != nil { + c.p = c.iterA + return false + } + if c.iterB.Err() != nil { + c.p = c.iterB return false } // get the current pointer From 7204b811a680390660d8004b493e9d46886d35c5 Mon Sep 17 00:00:00 2001 From: guy-har <60321938+guy-har@users.noreply.github.com> Date: Thu, 10 Dec 2020 13:36:39 +0200 Subject: [PATCH 07/14] Update graveler/graveler.go Co-authored-by: Barak Amar --- graveler/graveler.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/graveler/graveler.go b/graveler/graveler.go index 7e51725fac1..02bac8e614b 100644 --- a/graveler/graveler.go +++ b/graveler/graveler.go @@ -546,12 +546,12 @@ func generateStagingToken(repositoryID RepositoryID, branchID BranchID) StagingT func (r *graveler) CreateBranch(ctx context.Context, repositoryID RepositoryID, branchID BranchID, ref Ref) (*Branch, error) { // check if branch exists _, err := r.RefManager.GetBranch(ctx, repositoryID, branchID) - if err != nil && !errors.Is(err, ErrNotFound) { - return nil, err - } - if !errors.Is(err, ErrNotFound) { + if errors.Is(err, ErrNotFound) { return nil, ErrBranchExists } + if err != nil { + return nil, err + } reference, err := r.RefManager.RevParse(ctx, repositoryID, ref) if err != nil { From b068659ded5022127ddb7bddad7ab910dd789e1f Mon Sep 17 00:00:00 2001 From: guy-har <60321938+guy-har@users.noreply.github.com> Date: Thu, 10 Dec 2020 14:42:00 +0200 Subject: [PATCH 08/14] Update graveler/listing_iterator.go Co-authored-by: Barak Amar --- graveler/listing_iterator.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graveler/listing_iterator.go b/graveler/listing_iterator.go index 07fd4191094..5116048ae25 100644 --- a/graveler/listing_iterator.go +++ b/graveler/listing_iterator.go @@ -13,7 +13,7 @@ type listingIterator struct { } // getFollowingValue returns the following value (i.e will increase the last byte by 1) -// in the following cases will return received value : value is nil, value length is 0, last byte is maximum byte +// in the following cases will return received value: empty value, the last byte is math.MaxUint8 func getFollowingValue(value []byte) []byte { if len(value) == 0 || value[len(value)-1] == 255 { return value From 6dd8c419b82044fb0f1420d175006562e645a048 Mon Sep 17 00:00:00 2001 From: guy-har <60321938+guy-har@users.noreply.github.com> Date: Thu, 10 Dec 2020 17:29:16 +0200 Subject: [PATCH 09/14] Update graveler/graveler.go Co-authored-by: Barak Amar --- graveler/graveler.go | 1 + 1 file changed, 1 insertion(+) diff --git a/graveler/graveler.go b/graveler/graveler.go index 02bac8e614b..57322ecf8d6 100644 --- a/graveler/graveler.go +++ b/graveler/graveler.go @@ -537,6 +537,7 @@ func (r *graveler) DeleteRepository(ctx context.Context, repositoryID Repository func (r *graveler) GetCommit(ctx context.Context, repositoryID RepositoryID, commitID CommitID) (*Commit, error) { return r.RefManager.GetCommit(ctx, repositoryID, commitID) } + func generateStagingToken(repositoryID RepositoryID, branchID BranchID) StagingToken { // TODO(Guys): initial implementation, change this uid := uuid.New().String() From 9e32e2c6b450d23ee9a270fa4105c25dd6e05fa3 Mon Sep 17 00:00:00 2001 From: guy-har <60321938+guy-har@users.noreply.github.com> Date: Thu, 10 Dec 2020 17:29:34 +0200 Subject: [PATCH 10/14] Update graveler/uncommitted_diff_iterator.go Co-authored-by: Barak Amar --- graveler/uncommitted_diff_iterator.go | 22 ++++++++++------------ 1 file changed, 10 insertions(+), 12 deletions(-) diff --git a/graveler/uncommitted_diff_iterator.go b/graveler/uncommitted_diff_iterator.go index 9f2242ff00f..8f4a7a5425c 100644 --- a/graveler/uncommitted_diff_iterator.go +++ b/graveler/uncommitted_diff_iterator.go @@ -41,20 +41,18 @@ func getDiffType(ctx context.Context, committedManager CommittedManager, sn Stor return 0, err } diffType := DiffTypeAdded - if tombstone && existsInCommitted { - diffType = DiffTypeRemoved + if tombstone { + if !existsInCommitted { + logging.Default(). + WithFields(logging.Fields{"tree_id": treeID, "storage_namespace": sn, "key": key}). + Warn("tombstone for a file that does not exist") + } + return DiffTypeRemoved } - if tombstone && !existsInCommitted { - logging.Default().WithFields(logging.Fields{"treeID": treeID, "stagingToken": sn, "key": key}).Warn("tombstone for file that does not exist") - diffType = DiffTypeRemoved + if existsInCommitted { + return DiffTypeChanged } - if !tombstone && existsInCommitted { - diffType = DiffTypeChanged - } - if !tombstone && !existsInCommitted { - diffType = DiffTypeAdded - } - return diffType, nil + return DiffTypeAdded } func (d *uncommittedDiffIterator) Next() bool { From c452f5948548a230b94ce5c85c2f74bfcd85b721 Mon Sep 17 00:00:00 2001 From: guyhardonag Date: Sun, 13 Dec 2020 09:50:26 +0200 Subject: [PATCH 11/14] Add change requests from review --- graveler/combined_iterator.go | 2 - graveler/graveler.go | 163 +++++++++++++------------- graveler/graveler_mock_test.go | 4 +- graveler/listing_iterator.go | 13 +- graveler/prefix_iterator.go | 8 +- graveler/uncommitted_diff_iterator.go | 10 +- 6 files changed, 100 insertions(+), 100 deletions(-) diff --git a/graveler/combined_iterator.go b/graveler/combined_iterator.go index d047cf4729a..4885058fe2f 100644 --- a/graveler/combined_iterator.go +++ b/graveler/combined_iterator.go @@ -8,7 +8,6 @@ type combinedIterator struct { iterA ListingIterator iterB ListingIterator p ListingIterator - err error } func NewCombinedIterator(iterA, iterB ListingIterator) ListingIterator { @@ -16,7 +15,6 @@ func NewCombinedIterator(iterA, iterB ListingIterator) ListingIterator { iterA: iterA, iterB: iterB, p: nil, - err: nil, } } diff --git a/graveler/graveler.go b/graveler/graveler.go index 7e51725fac1..f3c24f307a5 100644 --- a/graveler/graveler.go +++ b/graveler/graveler.go @@ -506,11 +506,11 @@ func NewGraveler(committedManager CommittedManager, stagingManager StagingManage } } -func (r *graveler) GetRepository(ctx context.Context, repositoryID RepositoryID) (*Repository, error) { - return r.RefManager.GetRepository(ctx, repositoryID) +func (g *graveler) GetRepository(ctx context.Context, repositoryID RepositoryID) (*Repository, error) { + return g.RefManager.GetRepository(ctx, repositoryID) } -func (r *graveler) CreateRepository(ctx context.Context, repositoryID RepositoryID, storageNamespace StorageNamespace, branchID BranchID) (*Repository, error) { +func (g *graveler) CreateRepository(ctx context.Context, repositoryID RepositoryID, storageNamespace StorageNamespace, branchID BranchID) (*Repository, error) { repo := Repository{ StorageNamespace: storageNamespace, CreationDate: time.Now(), @@ -519,33 +519,34 @@ func (r *graveler) CreateRepository(ctx context.Context, repositoryID Repository branch := Branch{ stagingToken: generateStagingToken(repositoryID, branchID), } - err := r.RefManager.CreateRepository(ctx, repositoryID, repo, branch) + err := g.RefManager.CreateRepository(ctx, repositoryID, repo, branch) if err != nil { return nil, err } return &repo, nil } -func (r *graveler) ListRepositories(ctx context.Context, from RepositoryID) (RepositoryIterator, error) { - return r.RefManager.ListRepositories(ctx, from) +func (g *graveler) ListRepositories(ctx context.Context, from RepositoryID) (RepositoryIterator, error) { + return g.RefManager.ListRepositories(ctx, from) } -func (r *graveler) DeleteRepository(ctx context.Context, repositoryID RepositoryID) error { - return r.RefManager.DeleteRepository(ctx, repositoryID) +func (g *graveler) DeleteRepository(ctx context.Context, repositoryID RepositoryID) error { + return g.RefManager.DeleteRepository(ctx, repositoryID) } -func (r *graveler) GetCommit(ctx context.Context, repositoryID RepositoryID, commitID CommitID) (*Commit, error) { - return r.RefManager.GetCommit(ctx, repositoryID, commitID) +func (g *graveler) GetCommit(ctx context.Context, repositoryID RepositoryID, commitID CommitID) (*Commit, error) { + return g.RefManager.GetCommit(ctx, repositoryID, commitID) } + func generateStagingToken(repositoryID RepositoryID, branchID BranchID) StagingToken { // TODO(Guys): initial implementation, change this uid := uuid.New().String() return StagingToken(fmt.Sprintf("%s-%s:%s", repositoryID, branchID, uid)) } -func (r *graveler) CreateBranch(ctx context.Context, repositoryID RepositoryID, branchID BranchID, ref Ref) (*Branch, error) { +func (g *graveler) CreateBranch(ctx context.Context, repositoryID RepositoryID, branchID BranchID, ref Ref) (*Branch, error) { // check if branch exists - _, err := r.RefManager.GetBranch(ctx, repositoryID, branchID) + _, err := g.RefManager.GetBranch(ctx, repositoryID, branchID) if err != nil && !errors.Is(err, ErrNotFound) { return nil, err } @@ -553,7 +554,7 @@ func (r *graveler) CreateBranch(ctx context.Context, repositoryID RepositoryID, return nil, ErrBranchExists } - reference, err := r.RefManager.RevParse(ctx, repositoryID, ref) + reference, err := g.RefManager.RevParse(ctx, repositoryID, ref) if err != nil { return nil, err } @@ -562,30 +563,31 @@ func (r *graveler) CreateBranch(ctx context.Context, repositoryID RepositoryID, CommitID: reference.CommitID(), stagingToken: generateStagingToken(repositoryID, branchID), } - err = r.RefManager.SetBranch(ctx, repositoryID, branchID, newBranch) + err = g.RefManager.SetBranch(ctx, repositoryID, branchID, newBranch) if err != nil { return nil, err } return &newBranch, nil } -func (r *graveler) UpdateBranch(ctx context.Context, repositoryID RepositoryID, branchID BranchID, ref Ref) (*Branch, error) { - reference, err := r.RefManager.RevParse(ctx, repositoryID, ref) +func (g *graveler) UpdateBranch(ctx context.Context, repositoryID RepositoryID, branchID BranchID, ref Ref) (*Branch, error) { + reference, err := g.RefManager.RevParse(ctx, repositoryID, ref) if err != nil { return nil, err } - curBranch, err := r.RefManager.GetBranch(ctx, repositoryID, branchID) + curBranch, err := g.RefManager.GetBranch(ctx, repositoryID, branchID) if err != nil { return nil, err } // validate no conflict // TODO(Guys) return error only on conflicts, currently returns error for any changes on staging - list, err := r.StagingManager.List(ctx, curBranch.stagingToken) + iter, err := g.StagingManager.List(ctx, curBranch.stagingToken) if err != nil { return nil, err } - if list.Next() { + defer iter.Close() + if iter.Next() { return nil, ErrConflictFound } @@ -593,58 +595,58 @@ func (r *graveler) UpdateBranch(ctx context.Context, repositoryID RepositoryID, CommitID: reference.CommitID(), stagingToken: curBranch.stagingToken, } - err = r.RefManager.SetBranch(ctx, repositoryID, branchID, newBranch) + err = g.RefManager.SetBranch(ctx, repositoryID, branchID, newBranch) if err != nil { return nil, err } return &newBranch, nil } -func (r *graveler) GetBranch(ctx context.Context, repositoryID RepositoryID, branchID BranchID) (*Branch, error) { - return r.RefManager.GetBranch(ctx, repositoryID, branchID) +func (g *graveler) GetBranch(ctx context.Context, repositoryID RepositoryID, branchID BranchID) (*Branch, error) { + return g.RefManager.GetBranch(ctx, repositoryID, branchID) } -func (r *graveler) Dereference(ctx context.Context, repositoryID RepositoryID, ref Ref) (CommitID, error) { - reference, err := r.RefManager.RevParse(ctx, repositoryID, ref) +func (g *graveler) Dereference(ctx context.Context, repositoryID RepositoryID, ref Ref) (CommitID, error) { + reference, err := g.RefManager.RevParse(ctx, repositoryID, ref) if err != nil { return "", err } return reference.CommitID(), nil } -func (r *graveler) Log(ctx context.Context, repositoryID RepositoryID, commitID CommitID) (CommitIterator, error) { - return r.RefManager.Log(ctx, repositoryID, commitID) +func (g *graveler) Log(ctx context.Context, repositoryID RepositoryID, commitID CommitID) (CommitIterator, error) { + return g.RefManager.Log(ctx, repositoryID, commitID) } -func (r *graveler) ListBranches(ctx context.Context, repositoryID RepositoryID, from BranchID) (BranchIterator, error) { - return r.RefManager.ListBranches(ctx, repositoryID, from) +func (g *graveler) ListBranches(ctx context.Context, repositoryID RepositoryID, from BranchID) (BranchIterator, error) { + return g.RefManager.ListBranches(ctx, repositoryID, from) } -func (r *graveler) DeleteBranch(ctx context.Context, repositoryID RepositoryID, branchID BranchID) error { - branch, err := r.RefManager.GetBranch(ctx, repositoryID, branchID) +func (g *graveler) DeleteBranch(ctx context.Context, repositoryID RepositoryID, branchID BranchID) error { + branch, err := g.RefManager.GetBranch(ctx, repositoryID, branchID) if err != nil { return err } - err = r.StagingManager.Drop(ctx, branch.stagingToken) + err = g.StagingManager.Drop(ctx, branch.stagingToken) if err != nil { return err } - return r.RefManager.DeleteBranch(ctx, repositoryID, branchID) + return g.RefManager.DeleteBranch(ctx, repositoryID, branchID) } -func (r *graveler) Get(ctx context.Context, repositoryID RepositoryID, ref Ref, key Key) (*Value, error) { - repo, err := r.RefManager.GetRepository(ctx, repositoryID) +func (g *graveler) Get(ctx context.Context, repositoryID RepositoryID, ref Ref, key Key) (*Value, error) { + repo, err := g.RefManager.GetRepository(ctx, repositoryID) if err != nil { return nil, err } - reference, err := r.RefManager.RevParse(ctx, repositoryID, ref) + reference, err := g.RefManager.RevParse(ctx, repositoryID, ref) if err != nil { return nil, err } if reference.Type() == ReferenceTypeBranch { // try to get from staging, if not found proceed to committed branch := reference.Branch() - value, err := r.StagingManager.Get(ctx, branch.stagingToken, key) + value, err := g.StagingManager.Get(ctx, branch.stagingToken, key) if !errors.Is(err, ErrNotFound) { if err != nil { return nil, err @@ -657,93 +659,92 @@ func (r *graveler) Get(ctx context.Context, repositoryID RepositoryID, ref Ref, } } commitID := reference.CommitID() - commit, err := r.RefManager.GetCommit(ctx, repositoryID, commitID) + commit, err := g.RefManager.GetCommit(ctx, repositoryID, commitID) if err != nil { return nil, err } - return r.CommittedManager.Get(ctx, repo.StorageNamespace, commit.TreeID, key) + return g.CommittedManager.Get(ctx, repo.StorageNamespace, commit.TreeID, key) } -func (r *graveler) Set(ctx context.Context, repositoryID RepositoryID, branchID BranchID, key Key, value Value) error { - branch, err := r.GetBranch(ctx, repositoryID, branchID) +func (g *graveler) Set(ctx context.Context, repositoryID RepositoryID, branchID BranchID, key Key, value Value) error { + branch, err := g.GetBranch(ctx, repositoryID, branchID) if err != nil { return err } - return r.StagingManager.Set(ctx, branch.stagingToken, key, value) + return g.StagingManager.Set(ctx, branch.stagingToken, key, value) } -func (r *graveler) Delete(ctx context.Context, repositoryID RepositoryID, branchID BranchID, key Key) error { - branch, err := r.GetBranch(ctx, repositoryID, branchID) +func (g *graveler) Delete(ctx context.Context, repositoryID RepositoryID, branchID BranchID, key Key) error { + branch, err := g.GetBranch(ctx, repositoryID, branchID) if err != nil { return err } - return r.StagingManager.Delete(ctx, branch.stagingToken, key) + return g.StagingManager.Delete(ctx, branch.stagingToken, key) } -func (r *graveler) List(ctx context.Context, repositoryID RepositoryID, ref Ref, prefix, from, delimiter Key) (ListingIterator, error) { - repo, err := r.RefManager.GetRepository(ctx, repositoryID) +func (g *graveler) List(ctx context.Context, repositoryID RepositoryID, ref Ref, prefix, from, delimiter Key) (ListingIterator, error) { + repo, err := g.RefManager.GetRepository(ctx, repositoryID) if err != nil { return nil, err } - reference, err := r.RefManager.RevParse(ctx, repositoryID, ref) + reference, err := g.RefManager.RevParse(ctx, repositoryID, ref) if err != nil { return nil, err } - commit, err := r.RefManager.GetCommit(ctx, repositoryID, reference.CommitID()) + commit, err := g.RefManager.GetCommit(ctx, repositoryID, reference.CommitID()) if err != nil { return nil, err } var listing ListingIterator - committedValues, err := r.CommittedManager.List(ctx, repo.StorageNamespace, commit.TreeID, from) + committedValues, err := g.CommittedManager.List(ctx, repo.StorageNamespace, commit.TreeID, from) if err != nil { return nil, err } - committedListing := NewListingIterator(NewPrefixIterator(committedValues, prefix), delimiter, prefix) if reference.Type() == ReferenceTypeBranch { - stagingList, err := r.StagingManager.List(ctx, reference.Branch().stagingToken) + stagingList, err := g.StagingManager.List(ctx, reference.Branch().stagingToken) if err != nil { return nil, err } - listing = NewCombinedIterator(NewListingIterator(NewPrefixIterator(stagingList, prefix), delimiter, prefix), committedListing) + listing = NewCombinedIterator(NewListingIterator(NewPrefixIterator(stagingList, prefix), delimiter, prefix), NewListingIterator(NewPrefixIterator(committedValues, prefix), delimiter, prefix)) } else { - listing = committedListing + listing = NewListingIterator(NewPrefixIterator(committedValues, prefix), delimiter, prefix) } return listing, nil } -func (r *graveler) Commit(ctx context.Context, repositoryID RepositoryID, branchID BranchID, committer string, message string, metadata Metadata) (CommitID, error) { +func (g *graveler) Commit(ctx context.Context, repositoryID RepositoryID, branchID BranchID, committer string, message string, metadata Metadata) (CommitID, error) { panic("implement me") } -func (r *graveler) Reset(ctx context.Context, repositoryID RepositoryID, branchID BranchID) error { +func (g *graveler) Reset(ctx context.Context, repositoryID RepositoryID, branchID BranchID) error { panic("implement me") // waiting for staging reset } -func (r *graveler) Revert(ctx context.Context, repositoryID RepositoryID, branchID BranchID, ref Ref) (CommitID, error) { +func (g *graveler) Revert(ctx context.Context, repositoryID RepositoryID, branchID BranchID, ref Ref) (CommitID, error) { panic("implement me") } -func (r *graveler) Merge(ctx context.Context, repositoryID RepositoryID, from Ref, to BranchID) (CommitID, error) { - repo, err := r.RefManager.GetRepository(ctx, repositoryID) +func (g *graveler) Merge(ctx context.Context, repositoryID RepositoryID, from Ref, to BranchID) (CommitID, error) { + repo, err := g.RefManager.GetRepository(ctx, repositoryID) if err != nil { return "", err } - fromCommit, err := r.getCommitRecordFromRef(ctx, repositoryID, from) + fromCommit, err := g.getCommitRecordFromRef(ctx, repositoryID, from) if err != nil { return "", err } - toCommit, err := r.getCommitRecordFromBranchID(ctx, repositoryID, to) + toCommit, err := g.getCommitRecordFromBranchID(ctx, repositoryID, to) if err != nil { return "", err } - baseCommit, err := r.RefManager.FindMergeBase(ctx, repositoryID, fromCommit.CommitID, toCommit.CommitID) + baseCommit, err := g.RefManager.FindMergeBase(ctx, repositoryID, fromCommit.CommitID, toCommit.CommitID) if err != nil { return "", err } - treeID, err := r.CommittedManager.Merge(ctx, repo.StorageNamespace, fromCommit.TreeID, toCommit.TreeID, baseCommit.TreeID) + treeID, err := g.CommittedManager.Merge(ctx, repo.StorageNamespace, fromCommit.TreeID, toCommit.TreeID, baseCommit.TreeID) if err != nil { return "", err } @@ -753,37 +754,37 @@ func (r *graveler) Merge(ctx context.Context, repositoryID RepositoryID, from Re TreeID: treeID, CreationDate: time.Time{}, Parents: []CommitID{fromCommit.CommitID, toCommit.CommitID}, - Metadata: nil, + Metadata: nil, // TODO(Guys): pass metadata } - return r.RefManager.AddCommit(ctx, repositoryID, commit) + return g.RefManager.AddCommit(ctx, repositoryID, commit) } -func (r *graveler) DiffUncommitted(ctx context.Context, repositoryID RepositoryID, branchID BranchID, from Key) (DiffIterator, error) { - repo, err := r.RefManager.GetRepository(ctx, repositoryID) +func (g *graveler) DiffUncommitted(ctx context.Context, repositoryID RepositoryID, branchID BranchID, from Key) (DiffIterator, error) { + repo, err := g.RefManager.GetRepository(ctx, repositoryID) if err != nil { return nil, err } - branch, err := r.RefManager.GetBranch(ctx, repositoryID, branchID) + branch, err := g.RefManager.GetBranch(ctx, repositoryID, branchID) if err != nil { return nil, err } - commit, err := r.RefManager.GetCommit(ctx, repositoryID, branch.CommitID) + commit, err := g.RefManager.GetCommit(ctx, repositoryID, branch.CommitID) if err != nil { return nil, err } - valueIterator, err := r.StagingManager.List(ctx, branch.stagingToken) + valueIterator, err := g.StagingManager.List(ctx, branch.stagingToken) if err != nil { return nil, err } - return NewUncommittedDiffIterator(r.CommittedManager, valueIterator, repo.StorageNamespace, commit.TreeID), nil + return NewUncommittedDiffIterator(g.CommittedManager, valueIterator, repo.StorageNamespace, commit.TreeID), nil } -func (r *graveler) getCommitRecordFromRef(ctx context.Context, repositoryID RepositoryID, ref Ref) (*CommitRecord, error) { - reference, err := r.RefManager.RevParse(ctx, repositoryID, ref) +func (g *graveler) getCommitRecordFromRef(ctx context.Context, repositoryID RepositoryID, ref Ref) (*CommitRecord, error) { + reference, err := g.RefManager.RevParse(ctx, repositoryID, ref) if err != nil { return nil, err } - commit, err := r.RefManager.GetCommit(ctx, repositoryID, reference.CommitID()) + commit, err := g.RefManager.GetCommit(ctx, repositoryID, reference.CommitID()) if err != nil { return nil, err } @@ -793,27 +794,27 @@ func (r *graveler) getCommitRecordFromRef(ctx context.Context, repositoryID Repo }, nil } -func (r *graveler) getCommitRecordFromBranchID(ctx context.Context, repositoryID RepositoryID, branch BranchID) (*CommitRecord, error) { - return r.getCommitRecordFromRef(ctx, repositoryID, Ref(branch)) +func (g *graveler) getCommitRecordFromBranchID(ctx context.Context, repositoryID RepositoryID, branch BranchID) (*CommitRecord, error) { + return g.getCommitRecordFromRef(ctx, repositoryID, Ref(branch)) } -func (r *graveler) Diff(ctx context.Context, repositoryID RepositoryID, left, right Ref, from Key) (DiffIterator, error) { - repo, err := r.RefManager.GetRepository(ctx, repositoryID) +func (g *graveler) Diff(ctx context.Context, repositoryID RepositoryID, left, right Ref, from Key) (DiffIterator, error) { + repo, err := g.RefManager.GetRepository(ctx, repositoryID) if err != nil { return nil, err } - leftCommit, err := r.getCommitRecordFromRef(ctx, repositoryID, left) + leftCommit, err := g.getCommitRecordFromRef(ctx, repositoryID, left) if err != nil { return nil, err } - rightCommit, err := r.getCommitRecordFromRef(ctx, repositoryID, right) + rightCommit, err := g.getCommitRecordFromRef(ctx, repositoryID, right) if err != nil { return nil, err } - baseCommit, err := r.RefManager.FindMergeBase(ctx, repositoryID, leftCommit.CommitID, rightCommit.CommitID) + baseCommit, err := g.RefManager.FindMergeBase(ctx, repositoryID, leftCommit.CommitID, rightCommit.CommitID) if err != nil { return nil, err } - return r.CommittedManager.Diff(ctx, repo.StorageNamespace, leftCommit.TreeID, rightCommit.TreeID, baseCommit.TreeID, from) + return g.CommittedManager.Diff(ctx, repo.StorageNamespace, leftCommit.TreeID, rightCommit.TreeID, baseCommit.TreeID, from) } diff --git a/graveler/graveler_mock_test.go b/graveler/graveler_mock_test.go index 2a13ba36d36..ce674b5458b 100644 --- a/graveler/graveler_mock_test.go +++ b/graveler/graveler_mock_test.go @@ -9,6 +9,8 @@ import ( "github.com/treeverse/lakefs/graveler" ) +const defaultBranchName = graveler.BranchID("master") + type committedMock struct { Value *graveler.Value ValueIterator graveler.ValueIterator @@ -112,8 +114,6 @@ type mockRefs struct { refType graveler.ReferenceType } -const defaultBranchName = graveler.BranchID("master") - func (m *mockRefs) RevParse(_ context.Context, _ graveler.RepositoryID, _ graveler.Ref) (graveler.Reference, error) { var branch graveler.BranchID if m.refType == graveler.ReferenceTypeBranch { diff --git a/graveler/listing_iterator.go b/graveler/listing_iterator.go index 07fd4191094..697f2b061bf 100644 --- a/graveler/listing_iterator.go +++ b/graveler/listing_iterator.go @@ -1,6 +1,9 @@ package graveler -import "bytes" +import ( + "bytes" + "math" +) // listingIterator implements a ListingIterator using a ValueIterator // assumes all values in valueIterator start with prefix @@ -15,12 +18,13 @@ type listingIterator struct { // getFollowingValue returns the following value (i.e will increase the last byte by 1) // in the following cases will return received value : value is nil, value length is 0, last byte is maximum byte func getFollowingValue(value []byte) []byte { - if len(value) == 0 || value[len(value)-1] == 255 { + if len(value) == 0 || value[len(value)-1] == math.MaxUint8 { return value } copiedDelimiter := make([]byte, len(value)) copy(copiedDelimiter, value) - return append(copiedDelimiter[:len(copiedDelimiter)-1], copiedDelimiter[len(copiedDelimiter)-1]+1) + copiedDelimiter[len(copiedDelimiter)-1]++ + return copiedDelimiter } func NewListingIterator(iterator ValueIterator, delimiter, prefix Key) ListingIterator { @@ -53,12 +57,11 @@ func (l *listingIterator) nextNoDelimiter() bool { } func (l *listingIterator) nextWithDelimiter() bool { - var hasNext bool if l.current != nil && l.current.CommonPrefix { nextKey := getFollowingValue(l.current.Key) l.valueIterator.SeekGE(nextKey) } - hasNext = l.valueIterator.Next() + hasNext := l.valueIterator.Next() if hasNext { nextValue := l.valueIterator.Value() l.current = l.getListingFromValue(nextValue.Value, nextValue.Key) diff --git a/graveler/prefix_iterator.go b/graveler/prefix_iterator.go index bcb4af561f1..77b88cb4882 100644 --- a/graveler/prefix_iterator.go +++ b/graveler/prefix_iterator.go @@ -21,12 +21,8 @@ func (p *prefixIterator) Next() bool { if p.ended { return false } - if !p.iterator.Next() { - p.ended = true - return false - } - val := p.iterator.Value() - if !bytes.HasPrefix(val.Key, p.prefix) { + // prefix iterator ends when there is no more data, or the next value doesn't match the prefix + if !p.iterator.Next() || !bytes.HasPrefix(p.iterator.Value().Key, p.prefix) { p.ended = true return false } diff --git a/graveler/uncommitted_diff_iterator.go b/graveler/uncommitted_diff_iterator.go index 07251182322..2c9017a402b 100644 --- a/graveler/uncommitted_diff_iterator.go +++ b/graveler/uncommitted_diff_iterator.go @@ -10,7 +10,7 @@ import ( type uncommittedDiffIterator struct { committedManager CommittedManager list ValueIterator - sn StorageNamespace + storageNamespace StorageNamespace treeID TreeID value *Diff err error @@ -20,7 +20,7 @@ func NewUncommittedDiffIterator(manager CommittedManager, list ValueIterator, sn return &uncommittedDiffIterator{ committedManager: manager, list: list, - sn: sn, + storageNamespace: sn, treeID: treeItreeID, } } @@ -29,7 +29,8 @@ func valueExistsInCommitted(ctx context.Context, committedManager CommittedManag _, err := committedManager.Get(ctx, sn, treeID, key) if errors.Is(err, ErrNotFound) { return false, nil - } else if err != nil { + } + if err != nil { return false, err } return true, nil @@ -64,7 +65,7 @@ func (d *uncommittedDiffIterator) Next() bool { return false } val := d.list.Value() - diffType, err := getDiffType(context.Background(), d.committedManager, d.sn, d.treeID, val.Key, val.Value == nil) + diffType, err := getDiffType(context.Background(), d.committedManager, d.storageNamespace, d.treeID, val.Key, val.Value == nil) if err != nil { d.value = nil d.err = err @@ -80,6 +81,7 @@ func (d *uncommittedDiffIterator) Next() bool { func (d *uncommittedDiffIterator) SeekGE(id Key) { d.value = nil + d.err = nil d.list.SeekGE(id) } From 7d632f3ee3c723727bd64b3d56571d4b0c9a72af Mon Sep 17 00:00:00 2001 From: guyhardonag Date: Sun, 13 Dec 2020 09:58:42 +0200 Subject: [PATCH 12/14] Add change requests from review --- graveler/uncommitted_diff_iterator.go | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/graveler/uncommitted_diff_iterator.go b/graveler/uncommitted_diff_iterator.go index 7506b6b1b2e..a44d8ed31b5 100644 --- a/graveler/uncommitted_diff_iterator.go +++ b/graveler/uncommitted_diff_iterator.go @@ -41,19 +41,18 @@ func getDiffType(ctx context.Context, committedManager CommittedManager, sn Stor if err != nil { return 0, err } - diffType := DiffTypeAdded if tombstone { if !existsInCommitted { logging.Default(). WithFields(logging.Fields{"tree_id": treeID, "storage_namespace": sn, "key": key}). Warn("tombstone for a file that does not exist") } - return DiffTypeRemoved + return DiffTypeRemoved, nil } if existsInCommitted { - return DiffTypeChanged + return DiffTypeChanged, nil } - return DiffTypeAdded + return DiffTypeAdded, nil } func (d *uncommittedDiffIterator) Next() bool { From d618229d06d4f959ca543ef7c9833abe5c79235e Mon Sep 17 00:00:00 2001 From: guyhardonag Date: Sun, 13 Dec 2020 13:38:52 +0200 Subject: [PATCH 13/14] pr changes --- graveler/graveler.go | 12 ++++-------- graveler/pgrefs.go | 2 +- graveler/uncommitted_diff_iterator.go | 20 ++++++++++++-------- 3 files changed, 17 insertions(+), 17 deletions(-) diff --git a/graveler/graveler.go b/graveler/graveler.go index 6d11e111aa7..fbb16cafa51 100644 --- a/graveler/graveler.go +++ b/graveler/graveler.go @@ -422,7 +422,7 @@ var ( ErrInvalidRef = fmt.Errorf("ref: %w", ErrInvalidValue) ErrInvalidCommitID = fmt.Errorf("commit id: %w", ErrInvalidValue) ErrCommitNotFound = fmt.Errorf("commit: %w", ErrNotFound) - ErrCommitIDAmbiguous = fmt.Errorf("commit ID is ambiguous: %w", ErrNotFound) + ErrRefAmbiguous = fmt.Errorf("reference is ambiguous: %w", ErrNotFound) ErrConflictFound = errors.New("conflict found") ErrBranchExists = errors.New("branch already exists") ) @@ -628,7 +628,7 @@ func (g *graveler) DeleteBranch(ctx context.Context, repositoryID RepositoryID, return err } err = g.StagingManager.Drop(ctx, branch.stagingToken) - if err != nil { + if err != nil && !errors.Is(err, ErrNotFound) { return err } return g.RefManager.DeleteBranch(ctx, repositoryID, branchID) @@ -735,7 +735,7 @@ func (g *graveler) Merge(ctx context.Context, repositoryID RepositoryID, from Re if err != nil { return "", err } - toCommit, err := g.getCommitRecordFromBranchID(ctx, repositoryID, to) + toCommit, err := g.getCommitRecordFromRef(ctx, repositoryID, Ref(to)) if err != nil { return "", err } @@ -776,7 +776,7 @@ func (g *graveler) DiffUncommitted(ctx context.Context, repositoryID RepositoryI if err != nil { return nil, err } - return NewUncommittedDiffIterator(g.CommittedManager, valueIterator, repo.StorageNamespace, commit.TreeID), nil + return NewUncommittedDiffIterator(ctx, g.CommittedManager, valueIterator, repo.StorageNamespace, commit.TreeID), nil } func (g *graveler) getCommitRecordFromRef(ctx context.Context, repositoryID RepositoryID, ref Ref) (*CommitRecord, error) { @@ -794,10 +794,6 @@ func (g *graveler) getCommitRecordFromRef(ctx context.Context, repositoryID Repo }, nil } -func (g *graveler) getCommitRecordFromBranchID(ctx context.Context, repositoryID RepositoryID, branch BranchID) (*CommitRecord, error) { - return g.getCommitRecordFromRef(ctx, repositoryID, Ref(branch)) -} - func (g *graveler) Diff(ctx context.Context, repositoryID RepositoryID, left, right Ref, from Key) (DiffIterator, error) { repo, err := g.RefManager.GetRepository(ctx, repositoryID) if err != nil { diff --git a/graveler/pgrefs.go b/graveler/pgrefs.go index b15a9883c5f..77bcea22235 100644 --- a/graveler/pgrefs.go +++ b/graveler/pgrefs.go @@ -196,7 +196,7 @@ func (m *PGRefManager) GetCommitByPrefix(ctx context.Context, repositoryID Repos return "", ErrNotFound } if len(startWith) > 1 { - return "", ErrCommitIDAmbiguous // more than 1 commit starts with the ID prefix + return "", ErrRefAmbiguous // more than 1 commit starts with the ID prefix } return startWith[0], nil }, db.ReadOnly(), db.WithContext(ctx)) diff --git a/graveler/uncommitted_diff_iterator.go b/graveler/uncommitted_diff_iterator.go index a44d8ed31b5..a0663e815cd 100644 --- a/graveler/uncommitted_diff_iterator.go +++ b/graveler/uncommitted_diff_iterator.go @@ -14,10 +14,12 @@ type uncommittedDiffIterator struct { treeID TreeID value *Diff err error + ctx context.Context } -func NewUncommittedDiffIterator(manager CommittedManager, list ValueIterator, sn StorageNamespace, treeItreeID TreeID) DiffIterator { +func NewUncommittedDiffIterator(ctx context.Context, manager CommittedManager, list ValueIterator, sn StorageNamespace, treeItreeID TreeID) DiffIterator { return &uncommittedDiffIterator{ + ctx: ctx, committedManager: manager, list: list, storageNamespace: sn, @@ -25,8 +27,8 @@ func NewUncommittedDiffIterator(manager CommittedManager, list ValueIterator, sn } } -func valueExistsInCommitted(ctx context.Context, committedManager CommittedManager, sn StorageNamespace, treeID TreeID, key Key) (bool, error) { - _, err := committedManager.Get(ctx, sn, treeID, key) +func (d *uncommittedDiffIterator) valueExistsInCommitted(val ValueRecord) (bool, error) { + _, err := d.committedManager.Get(d.ctx, d.storageNamespace, d.treeID, val.Key) if errors.Is(err, ErrNotFound) { return false, nil } @@ -36,15 +38,17 @@ func valueExistsInCommitted(ctx context.Context, committedManager CommittedManag return true, nil } -func getDiffType(ctx context.Context, committedManager CommittedManager, sn StorageNamespace, treeID TreeID, key Key, tombstone bool) (DiffType, error) { - existsInCommitted, err := valueExistsInCommitted(ctx, committedManager, sn, treeID, key) +func (d *uncommittedDiffIterator) getDiffType(val ValueRecord) (DiffType, error) { + existsInCommitted, err := d.valueExistsInCommitted(val) if err != nil { return 0, err } - if tombstone { + + if val.Value == nil { + // tombstone if !existsInCommitted { logging.Default(). - WithFields(logging.Fields{"tree_id": treeID, "storage_namespace": sn, "key": key}). + WithFields(logging.Fields{"tree_id": d.treeID, "storage_namespace": d.storageNamespace, "key": val.Key}). Warn("tombstone for a file that does not exist") } return DiffTypeRemoved, nil @@ -61,7 +65,7 @@ func (d *uncommittedDiffIterator) Next() bool { return false } val := d.list.Value() - diffType, err := getDiffType(context.Background(), d.committedManager, d.storageNamespace, d.treeID, val.Key, val.Value == nil) + diffType, err := d.getDiffType(*val) if err != nil { d.value = nil d.err = err From d4cbf9e08be01a647e8934ea1a14df30354f1883 Mon Sep 17 00:00:00 2001 From: guyhardonag Date: Sun, 13 Dec 2020 16:53:20 +0200 Subject: [PATCH 14/14] pr changes --- graveler/graveler.go | 8 +++---- graveler/graveler_mock_test.go | 8 ++++--- graveler/graveler_test.go | 39 ++++++++++++++++++++++++++++------ graveler/listing_iterator.go | 13 ++++++------ 4 files changed, 48 insertions(+), 20 deletions(-) diff --git a/graveler/graveler.go b/graveler/graveler.go index fbb16cafa51..1e6b78b1676 100644 --- a/graveler/graveler.go +++ b/graveler/graveler.go @@ -547,10 +547,10 @@ func generateStagingToken(repositoryID RepositoryID, branchID BranchID) StagingT func (g *graveler) CreateBranch(ctx context.Context, repositoryID RepositoryID, branchID BranchID, ref Ref) (*Branch, error) { // check if branch exists _, err := g.RefManager.GetBranch(ctx, repositoryID, branchID) - if errors.Is(err, ErrNotFound) { - return nil, ErrBranchExists - } - if err != nil { + if !errors.Is(err, ErrNotFound) { + if err == nil { + err = ErrBranchExists + } return nil, err } diff --git a/graveler/graveler_mock_test.go b/graveler/graveler_mock_test.go index ce674b5458b..ec39dae886e 100644 --- a/graveler/graveler_mock_test.go +++ b/graveler/graveler_mock_test.go @@ -9,7 +9,7 @@ import ( "github.com/treeverse/lakefs/graveler" ) -const defaultBranchName = graveler.BranchID("master") +const defaultBranchID = graveler.BranchID("master") type committedMock struct { Value *graveler.Value @@ -112,12 +112,14 @@ type mockRefs struct { listBranchesRes graveler.BranchIterator commitIter graveler.CommitIterator refType graveler.ReferenceType + branch *graveler.Branch + branchErr error } func (m *mockRefs) RevParse(_ context.Context, _ graveler.RepositoryID, _ graveler.Ref) (graveler.Reference, error) { var branch graveler.BranchID if m.refType == graveler.ReferenceTypeBranch { - branch = defaultBranchName + branch = defaultBranchID } return newMockReference(m.refType, branch, ""), nil } @@ -139,7 +141,7 @@ func (m *mockRefs) DeleteRepository(_ context.Context, _ graveler.RepositoryID) } func (m *mockRefs) GetBranch(_ context.Context, _ graveler.RepositoryID, _ graveler.BranchID) (*graveler.Branch, error) { - return &graveler.Branch{}, nil + return m.branch, m.branchErr } func (m *mockRefs) SetBranch(_ context.Context, _ graveler.RepositoryID, _ graveler.BranchID, _ graveler.Branch) error { diff --git a/graveler/graveler_test.go b/graveler/graveler_test.go index daf946c9f42..ca378d11be2 100644 --- a/graveler/graveler_test.go +++ b/graveler/graveler_test.go @@ -59,6 +59,7 @@ func TestGraveler_PrefixIterator(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { prefixIter := graveler.NewPrefixIterator(tt.valueIter, tt.prefix) + defer prefixIter.Close() prefixIter.SeekGE(tt.seekTo) // compare iterators for prefixIter.Next() { @@ -146,6 +147,7 @@ func TestGraveler_ListingIterator(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { listingIter := graveler.NewListingIterator(tt.valueIter, tt.delimiter, tt.prefix) + defer listingIter.Close() compareListingIterators(t, listingIter, tt.expectedListingIter) }) } @@ -230,6 +232,7 @@ func TestGraveler_List(t *testing.T) { if err != nil { return // err == tt.expectedErr } + defer listing.Close() // compare iterators compareListingIterators(t, listing, tt.expectedListing) }) @@ -331,7 +334,7 @@ func TestGraveler_DiffUncommitted(t *testing.T) { name: "no changes", r: graveler.NewGraveler(&committedMock{ValueIterator: newMockValueIterator([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}}), err: graveler.ErrNotFound}, &stagingMock{ValueIterator: newMockValueIterator([]graveler.ValueRecord{})}, - &mockRefs{}, + &mockRefs{branch: &graveler.Branch{}}, ), amount: 10, expectedDiff: newDiffIter([]graveler.Diff{}), @@ -340,7 +343,7 @@ func TestGraveler_DiffUncommitted(t *testing.T) { name: "added one", r: graveler.NewGraveler(&committedMock{ValueIterator: newMockValueIterator([]graveler.ValueRecord{}), err: graveler.ErrNotFound}, &stagingMock{ValueIterator: newMockValueIterator([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}})}, - &mockRefs{}, + &mockRefs{branch: &graveler.Branch{}}, ), amount: 10, expectedDiff: newDiffIter([]graveler.Diff{{ @@ -353,7 +356,7 @@ func TestGraveler_DiffUncommitted(t *testing.T) { name: "changed one", r: graveler.NewGraveler(&committedMock{ValueIterator: newMockValueIterator([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}})}, &stagingMock{ValueIterator: newMockValueIterator([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}})}, - &mockRefs{}, + &mockRefs{branch: &graveler.Branch{}}, ), amount: 10, expectedDiff: newDiffIter([]graveler.Diff{{ @@ -366,7 +369,7 @@ func TestGraveler_DiffUncommitted(t *testing.T) { name: "removed one", r: graveler.NewGraveler(&committedMock{ValueIterator: newMockValueIterator([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}})}, &stagingMock{ValueIterator: newMockValueIterator([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: nil}})}, - &mockRefs{}, + &mockRefs{branch: &graveler.Branch{}}, ), amount: 10, expectedDiff: newDiffIter([]graveler.Diff{{ @@ -402,10 +405,34 @@ func TestGraveler_DiffUncommitted(t *testing.T) { } } +func TestGraveler_CreateBranch(t *testing.T) { + gravel := graveler.NewGraveler(nil, + nil, + &mockRefs{ + branchErr: graveler.ErrNotFound, + }, + ) + _, err := gravel.CreateBranch(context.Background(), "", "", "") + if err != nil { + t.Fatal("unexpected error on create branch", err) + } + // test create branch when branch exists + gravel = graveler.NewGraveler(nil, + nil, + &mockRefs{ + branch: &graveler.Branch{}, + }, + ) + _, err = gravel.CreateBranch(context.Background(), "", "", "") + if !errors.Is(err, graveler.ErrBranchExists) { + t.Fatal("did not get expected error, expected ErrBranchExists") + } +} + func TestGraveler_UpdateBranch(t *testing.T) { gravel := graveler.NewGraveler(nil, &stagingMock{ValueIterator: newMockValueIterator([]graveler.ValueRecord{{Key: graveler.Key("foo/one"), Value: &graveler.Value{}}})}, - &mockRefs{}, + &mockRefs{branch: &graveler.Branch{}}, ) _, err := gravel.UpdateBranch(context.Background(), "", "", "") if !errors.Is(err, graveler.ErrConflictFound) { @@ -413,7 +440,7 @@ func TestGraveler_UpdateBranch(t *testing.T) { } gravel = graveler.NewGraveler(nil, &stagingMock{ValueIterator: newMockValueIterator([]graveler.ValueRecord{})}, - &mockRefs{}, + &mockRefs{branch: &graveler.Branch{}}, ) _, err = gravel.UpdateBranch(context.Background(), "", "", "") if err != nil { diff --git a/graveler/listing_iterator.go b/graveler/listing_iterator.go index 984f12fe0d7..d4e8be1e763 100644 --- a/graveler/listing_iterator.go +++ b/graveler/listing_iterator.go @@ -63,8 +63,7 @@ func (l *listingIterator) nextWithDelimiter() bool { } hasNext := l.valueIterator.Next() if hasNext { - nextValue := l.valueIterator.Value() - l.current = l.getListingFromValue(nextValue.Value, nextValue.Key) + l.current = l.getListingFromValue(*l.valueIterator.Value()) } else { l.current = nil } @@ -75,19 +74,19 @@ func (l *listingIterator) Next() bool { return l.nextFunc() } -func (l *listingIterator) getListingFromValue(value *Value, key Key) *Listing { - relevantKey := key[len(l.prefix):] +func (l *listingIterator) getListingFromValue(value ValueRecord) *Listing { + relevantKey := value.Key[len(l.prefix):] delimiterIndex := bytes.Index(relevantKey, l.delimiter) if delimiterIndex == -1 { // return listing for non common prefix with value return &Listing{ CommonPrefix: false, - Key: key, - Value: value, + Key: value.Key, + Value: value.Value, } } // return listing for common prefix key - commonPrefixKey := key[:len(l.prefix)+delimiterIndex+len(l.delimiter)] + commonPrefixKey := value.Key[:len(l.prefix)+delimiterIndex+len(l.delimiter)] return &Listing{ CommonPrefix: true, Key: commonPrefixKey,