diff --git a/pkg/local_object_storage/engine/tree.go b/pkg/local_object_storage/engine/tree.go index 0d6f49eea..b69ab4890 100644 --- a/pkg/local_object_storage/engine/tree.go +++ b/pkg/local_object_storage/engine/tree.go @@ -51,17 +51,17 @@ func (e *StorageEngine) TreeAddByPath(d pilorama.CIDDescriptor, treeID string, a } // TreeApply implements the pilorama.Forest interface. -func (e *StorageEngine) TreeApply(d pilorama.CIDDescriptor, treeID string, m *pilorama.Move, backgroundSync bool) error { - index, lst, err := e.getTreeShard(d.CID, treeID) +func (e *StorageEngine) TreeApply(cnr cidSDK.ID, treeID string, m *pilorama.Move, backgroundSync bool) error { + index, lst, err := e.getTreeShard(cnr, treeID) if err != nil && !errors.Is(err, pilorama.ErrTreeNotFound) { return err } - err = lst[index].TreeApply(d, treeID, m, backgroundSync) + err = lst[index].TreeApply(cnr, treeID, m, backgroundSync) if err != nil { if !errors.Is(err, shard.ErrReadOnlyMode) && err != shard.ErrPiloramaDisabled { e.reportShardError(lst[index], "can't perform `TreeApply`", err, - zap.Stringer("cid", d.CID), + zap.Stringer("cid", cnr), zap.String("tree", treeID)) } return err diff --git a/pkg/local_object_storage/pilorama/boltdb.go b/pkg/local_object_storage/pilorama/boltdb.go index b47fa16e8..42dde8607 100644 --- a/pkg/local_object_storage/pilorama/boltdb.go +++ b/pkg/local_object_storage/pilorama/boltdb.go @@ -327,11 +327,7 @@ func (t *boltForest) findSpareID(bTree *bbolt.Bucket) uint64 { } // TreeApply implements the Forest interface. -func (t *boltForest) TreeApply(d CIDDescriptor, treeID string, m *Move, backgroundSync bool) error { - if !d.checkValid() { - return ErrInvalidCIDDescriptor - } - +func (t *boltForest) TreeApply(cnr cidSDK.ID, treeID string, m *Move, backgroundSync bool) error { t.modeMtx.RLock() defer t.modeMtx.RUnlock() @@ -344,7 +340,7 @@ func (t *boltForest) TreeApply(d CIDDescriptor, treeID string, m *Move, backgrou if backgroundSync { var seen bool err := t.db.View(func(tx *bbolt.Tx) error { - treeRoot := tx.Bucket(bucketName(d.CID, treeID)) + treeRoot := tx.Bucket(bucketName(cnr, treeID)) if treeRoot == nil { return nil } @@ -362,7 +358,7 @@ func (t *boltForest) TreeApply(d CIDDescriptor, treeID string, m *Move, backgrou } if t.db.MaxBatchSize == 1 { - fullID := bucketName(d.CID, treeID) + fullID := bucketName(cnr, treeID) return t.db.Update(func(tx *bbolt.Tx) error { bLog, bTree, err := t.getTreeBuckets(tx, fullID) if err != nil { @@ -375,11 +371,11 @@ func (t *boltForest) TreeApply(d CIDDescriptor, treeID string, m *Move, backgrou } ch := make(chan error, 1) - t.addBatch(d, treeID, m, ch) + t.addBatch(cnr, treeID, m, ch) return <-ch } -func (t *boltForest) addBatch(d CIDDescriptor, treeID string, m *Move, ch chan error) { +func (t *boltForest) addBatch(cnr cidSDK.ID, treeID string, m *Move, ch chan error) { t.mtx.Lock() for i := 0; i < len(t.batches); i++ { t.batches[i].mtx.Lock() @@ -391,7 +387,7 @@ func (t *boltForest) addBatch(d CIDDescriptor, treeID string, m *Move, ch chan e continue } - found := t.batches[i].cid.Equals(d.CID) && t.batches[i].treeID == treeID + found := t.batches[i].cid.Equals(cnr) && t.batches[i].treeID == treeID if found { t.batches[i].results = append(t.batches[i].results, ch) t.batches[i].operations = append(t.batches[i].operations, m) @@ -412,7 +408,7 @@ func (t *boltForest) addBatch(d CIDDescriptor, treeID string, m *Move, ch chan e } b := &batch{ forest: t, - cid: d.CID, + cid: cnr, treeID: treeID, results: []chan<- error{ch}, operations: []*Move{m}, diff --git a/pkg/local_object_storage/pilorama/forest.go b/pkg/local_object_storage/pilorama/forest.go index 743096c81..21209420a 100644 --- a/pkg/local_object_storage/pilorama/forest.go +++ b/pkg/local_object_storage/pilorama/forest.go @@ -5,6 +5,7 @@ import ( "strings" "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard/mode" + cid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/container/id" cidSDK "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/container/id" ) @@ -93,12 +94,8 @@ func (f *memoryForest) TreeAddByPath(d CIDDescriptor, treeID string, attr string } // TreeApply implements the Forest interface. -func (f *memoryForest) TreeApply(d CIDDescriptor, treeID string, op *Move, _ bool) error { - if !d.checkValid() { - return ErrInvalidCIDDescriptor - } - - fullID := d.CID.String() + "/" + treeID +func (f *memoryForest) TreeApply(cnr cid.ID, treeID string, op *Move, _ bool) error { + fullID := cnr.String() + "/" + treeID s, ok := f.treeMap[fullID] if !ok { s = newState() diff --git a/pkg/local_object_storage/pilorama/forest_test.go b/pkg/local_object_storage/pilorama/forest_test.go index 9fe372b36..8e6f12717 100644 --- a/pkg/local_object_storage/pilorama/forest_test.go +++ b/pkg/local_object_storage/pilorama/forest_test.go @@ -411,21 +411,10 @@ func TestForest_Apply(t *testing.T) { func testForestTreeApply(t *testing.T, constructor func(t testing.TB, _ ...Option) Forest) { cid := cidtest.ID() - d := CIDDescriptor{cid, 0, 1} treeID := "version" - t.Run("invalid descriptor", func(t *testing.T) { - s := constructor(t) - err := s.TreeApply(CIDDescriptor{cid, 0, 0}, treeID, &Move{ - Child: 10, - Parent: 0, - Meta: Meta{Time: 1, Items: []KeyValue{{"grand", []byte{1}}}}, - }, false) - require.ErrorIs(t, err, ErrInvalidCIDDescriptor) - }) - testApply := func(t *testing.T, s Forest, child, parent Node, meta Meta) { - require.NoError(t, s.TreeApply(d, treeID, &Move{ + require.NoError(t, s.TreeApply(cid, treeID, &Move{ Child: child, Parent: parent, Meta: meta, @@ -465,7 +454,6 @@ func TestForest_GetOpLog(t *testing.T) { func testForestTreeGetOpLog(t *testing.T, constructor func(t testing.TB, _ ...Option) Forest) { cid := cidtest.ID() - d := CIDDescriptor{cid, 0, 1} treeID := "version" logs := []Move{ { @@ -491,7 +479,7 @@ func testForestTreeGetOpLog(t *testing.T, constructor func(t testing.TB, _ ...Op }) for i := range logs { - require.NoError(t, s.TreeApply(d, treeID, &logs[i], false)) + require.NoError(t, s.TreeApply(cid, treeID, &logs[i], false)) } testGetOpLog := func(t *testing.T, height uint64, m Move) { @@ -533,13 +521,12 @@ func testForestTreeExists(t *testing.T, constructor func(t testing.TB, opts ...O cid := cidtest.ID() treeID := "version" - d := CIDDescriptor{cid, 0, 1} t.Run("empty state, no panic", func(t *testing.T) { checkExists(t, false, cid, treeID) }) - require.NoError(t, s.TreeApply(d, treeID, &Move{Parent: 0, Child: 1}, false)) + require.NoError(t, s.TreeApply(cid, treeID, &Move{Parent: 0, Child: 1}, false)) checkExists(t, true, cid, treeID) checkExists(t, false, cidtest.ID(), treeID) // different CID, same tree checkExists(t, false, cid, "another tree") // same CID, different tree @@ -570,16 +557,16 @@ func TestApplyTricky1(t *testing.T) { } treeID := "version" - d := CIDDescriptor{CID: cidtest.ID(), Position: 0, Size: 1} + cid := cidtest.ID() for i := range providers { t.Run(providers[i].name, func(t *testing.T) { s := providers[i].construct(t) for i := range ops { - require.NoError(t, s.TreeApply(d, treeID, &ops[i], false)) + require.NoError(t, s.TreeApply(cid, treeID, &ops[i], false)) } for i := range expected { - _, parent, err := s.TreeGetMeta(d.CID, treeID, expected[i].child) + _, parent, err := s.TreeGetMeta(cid, treeID, expected[i].child) require.NoError(t, err) require.Equal(t, expected[i].parent, parent) } @@ -631,16 +618,16 @@ func TestApplyTricky2(t *testing.T) { } treeID := "version" - d := CIDDescriptor{CID: cidtest.ID(), Position: 0, Size: 1} + cid := cidtest.ID() for i := range providers { t.Run(providers[i].name, func(t *testing.T) { s := providers[i].construct(t) for i := range ops { - require.NoError(t, s.TreeApply(d, treeID, &ops[i], false)) + require.NoError(t, s.TreeApply(cid, treeID, &ops[i], false)) } for i := range expected { - _, parent, err := s.TreeGetMeta(d.CID, treeID, expected[i].child) + _, parent, err := s.TreeGetMeta(cid, treeID, expected[i].child) require.NoError(t, err) require.Equal(t, expected[i].parent, parent) } @@ -746,12 +733,11 @@ func testForestTreeParallelApply(t *testing.T, constructor func(t testing.TB, _ ops := prepareRandomTree(nodeCount, opCount) cid := cidtest.ID() - d := CIDDescriptor{cid, 0, 1} treeID := "version" expected := constructor(t) for i := range ops { - require.NoError(t, expected.TreeApply(d, treeID, &ops[i], false)) + require.NoError(t, expected.TreeApply(cid, treeID, &ops[i], false)) } for i := 0; i < iterCount; i++ { @@ -766,7 +752,7 @@ func testForestTreeParallelApply(t *testing.T, constructor func(t testing.TB, _ go func() { defer wg.Done() for op := range ch { - require.NoError(t, actual.TreeApply(d, treeID, op, false)) + require.NoError(t, actual.TreeApply(cid, treeID, op, false)) } }() } @@ -792,12 +778,11 @@ func testForestTreeApplyRandom(t *testing.T, constructor func(t testing.TB, _ .. ops := prepareRandomTree(nodeCount, opCount) cid := cidtest.ID() - d := CIDDescriptor{cid, 0, 1} treeID := "version" expected := constructor(t) for i := range ops { - require.NoError(t, expected.TreeApply(d, treeID, &ops[i], false)) + require.NoError(t, expected.TreeApply(cid, treeID, &ops[i], false)) } const iterCount = 200 @@ -807,7 +792,7 @@ func testForestTreeApplyRandom(t *testing.T, constructor func(t testing.TB, _ .. actual := constructor(t) for i := range ops { - require.NoError(t, actual.TreeApply(d, treeID, &ops[i], false)) + require.NoError(t, actual.TreeApply(cid, treeID, &ops[i], false)) } compareForests(t, expected, actual, cid, treeID, nodeCount) } @@ -889,7 +874,6 @@ func benchmarkApply(b *testing.B, s Forest, genFunc func(int) []Move) { ops := genFunc(b.N) cid := cidtest.ID() - d := CIDDescriptor{cid, 0, 1} treeID := "version" ch := make(chan int, b.N) for i := 0; i < b.N; i++ { @@ -901,7 +885,7 @@ func benchmarkApply(b *testing.B, s Forest, genFunc func(int) []Move) { b.SetParallelism(10) b.RunParallel(func(pb *testing.PB) { for pb.Next() { - if err := s.TreeApply(d, treeID, &ops[<-ch], false); err != nil { + if err := s.TreeApply(cid, treeID, &ops[<-ch], false); err != nil { b.Fatalf("error in `Apply`: %v", err) } } @@ -918,7 +902,6 @@ func TestTreeGetByPath(t *testing.T) { func testTreeGetByPath(t *testing.T, s Forest) { cid := cidtest.ID() - d := CIDDescriptor{cid, 0, 1} treeID := "version" // / @@ -928,12 +911,12 @@ func testTreeGetByPath(t *testing.T, s Forest) { // |- cat1.jpg, Version=XXX (4) // |- cat1.jpg, Version=YYY (5) // |- cat2.jpg, Version=ZZZ (6) - testMove(t, s, 0, 1, 0, d, treeID, "a", "") - testMove(t, s, 1, 2, 0, d, treeID, "b", "") - testMove(t, s, 2, 3, 1, d, treeID, "cat1.jpg", "TTT") - testMove(t, s, 3, 4, 2, d, treeID, "cat1.jpg", "XXX") - testMove(t, s, 4, 5, 2, d, treeID, "cat1.jpg", "YYY") - testMove(t, s, 5, 6, 2, d, treeID, "cat2.jpg", "ZZZ") + testMove(t, s, 0, 1, 0, cid, treeID, "a", "") + testMove(t, s, 1, 2, 0, cid, treeID, "b", "") + testMove(t, s, 2, 3, 1, cid, treeID, "cat1.jpg", "TTT") + testMove(t, s, 3, 4, 2, cid, treeID, "cat1.jpg", "XXX") + testMove(t, s, 4, 5, 2, cid, treeID, "cat1.jpg", "YYY") + testMove(t, s, 5, 6, 2, cid, treeID, "cat2.jpg", "ZZZ") if mf, ok := s.(*memoryForest); ok { single := mf.treeMap[cid.String()+"/"+treeID] @@ -970,14 +953,14 @@ func testTreeGetByPath(t *testing.T, s Forest) { }) } -func testMove(t *testing.T, s Forest, ts int, node, parent Node, d CIDDescriptor, treeID, filename, version string) { +func testMove(t *testing.T, s Forest, ts int, node, parent Node, cid cidSDK.ID, treeID, filename, version string) { items := make([]KeyValue, 1, 2) items[0] = KeyValue{AttributeFilename, []byte(filename)} if version != "" { items = append(items, KeyValue{AttributeVersion, []byte(version)}) } - require.NoError(t, s.TreeApply(d, treeID, &Move{ + require.NoError(t, s.TreeApply(cid, treeID, &Move{ Parent: parent, Child: node, Meta: Meta{ diff --git a/pkg/local_object_storage/pilorama/interface.go b/pkg/local_object_storage/pilorama/interface.go index c2143de24..290f633a5 100644 --- a/pkg/local_object_storage/pilorama/interface.go +++ b/pkg/local_object_storage/pilorama/interface.go @@ -18,7 +18,7 @@ type Forest interface { TreeAddByPath(d CIDDescriptor, treeID string, attr string, path []string, meta []KeyValue) ([]Move, error) // TreeApply applies replicated operation from another node. // If background is true, TreeApply will first check whether an operation exists. - TreeApply(d CIDDescriptor, treeID string, m *Move, backgroundSync bool) error + TreeApply(cnr cidSDK.ID, treeID string, m *Move, backgroundSync bool) error // TreeGetByPath returns all nodes corresponding to the path. // The path is constructed by descending from the root using the values of the // AttributeFilename in meta. diff --git a/pkg/local_object_storage/shard/tree.go b/pkg/local_object_storage/shard/tree.go index b9f909997..684c92e66 100644 --- a/pkg/local_object_storage/shard/tree.go +++ b/pkg/local_object_storage/shard/tree.go @@ -42,7 +42,7 @@ func (s *Shard) TreeAddByPath(d pilorama.CIDDescriptor, treeID string, attr stri } // TreeApply implements the pilorama.Forest interface. -func (s *Shard) TreeApply(d pilorama.CIDDescriptor, treeID string, m *pilorama.Move, backgroundSync bool) error { +func (s *Shard) TreeApply(cnr cidSDK.ID, treeID string, m *pilorama.Move, backgroundSync bool) error { if s.pilorama == nil { return ErrPiloramaDisabled } @@ -53,7 +53,7 @@ func (s *Shard) TreeApply(d pilorama.CIDDescriptor, treeID string, m *pilorama.M if s.info.Mode.ReadOnly() { return ErrReadOnlyMode } - return s.pilorama.TreeApply(d, treeID, m, backgroundSync) + return s.pilorama.TreeApply(cnr, treeID, m, backgroundSync) } // TreeGetByPath implements the pilorama.Forest interface. diff --git a/pkg/services/tree/replicator.go b/pkg/services/tree/replicator.go index 01bd2debd..8a1180289 100644 --- a/pkg/services/tree/replicator.go +++ b/pkg/services/tree/replicator.go @@ -27,7 +27,7 @@ type replicationTask struct { type applyOp struct { treeID string - pilorama.CIDDescriptor + cid cidSDK.ID pilorama.Move } @@ -43,7 +43,7 @@ func (s *Service) localReplicationWorker() { case <-s.closeCh: return case op := <-s.replicateLocalCh: - err := s.forest.TreeApply(op.CIDDescriptor, op.treeID, &op.Move, false) + err := s.forest.TreeApply(op.cid, op.treeID, &op.Move, false) if err != nil { s.log.Error("failed to apply replicated operation", zap.String("err", err.Error())) diff --git a/pkg/services/tree/service.go b/pkg/services/tree/service.go index 3176858e2..edea450f1 100644 --- a/pkg/services/tree/service.go +++ b/pkg/services/tree/service.go @@ -468,7 +468,7 @@ func (s *Service) Apply(_ context.Context, req *ApplyRequest) (*ApplyResponse, e key := req.GetSignature().GetKey() - _, pos, size, err := s.getContainerInfo(cid, key) + _, pos, _, err := s.getContainerInfo(cid, key) if err != nil { return nil, err } @@ -485,8 +485,8 @@ func (s *Service) Apply(_ context.Context, req *ApplyRequest) (*ApplyResponse, e select { case s.replicateLocalCh <- applyOp{ - treeID: req.GetBody().GetTreeId(), - CIDDescriptor: pilorama.CIDDescriptor{CID: cid, Position: pos, Size: size}, + treeID: req.GetBody().GetTreeId(), + cid: cid, Move: pilorama.Move{ Parent: op.GetParentId(), Child: op.GetChildId(), diff --git a/pkg/services/tree/sync.go b/pkg/services/tree/sync.go index abd683b77..9fe24fed0 100644 --- a/pkg/services/tree/sync.go +++ b/pkg/services/tree/sync.go @@ -14,6 +14,7 @@ import ( "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/morph/client/netmap" "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/network" cid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/container/id" + cidSDK "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/container/id" netmapSDK "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/netmap" "github.com/panjf2000/ants/v2" "go.uber.org/zap" @@ -40,11 +41,6 @@ func (s *Service) synchronizeAllTrees(ctx context.Context, cid cid.ID) error { return ErrNotInContainer } - var d pilorama.CIDDescriptor - d.CID = cid - d.Position = pos - d.Size = len(nodes) - nodes = randomizeNodeOrder(nodes, pos) if len(nodes) == 0 { return nil @@ -87,18 +83,18 @@ func (s *Service) synchronizeAllTrees(ctx context.Context, cid cid.ID) error { } for _, tid := range treesToSync { - h, err := s.forest.TreeLastSyncHeight(d.CID, tid) + h, err := s.forest.TreeLastSyncHeight(cid, tid) if err != nil && !errors.Is(err, pilorama.ErrTreeNotFound) { s.log.Warn("could not get last synchronized height for a tree", - zap.Stringer("cid", d.CID), + zap.Stringer("cid", cid), zap.String("tree", tid)) continue } - newHeight := s.synchronizeTree(ctx, d, h, tid, nodes) + newHeight := s.synchronizeTree(ctx, cid, h, tid, nodes) if h < newHeight { - if err := s.forest.TreeUpdateLastSyncHeight(d.CID, tid, newHeight); err != nil { + if err := s.forest.TreeUpdateLastSyncHeight(cid, tid, newHeight); err != nil { s.log.Warn("could not update last synchronized height for a tree", - zap.Stringer("cid", d.CID), + zap.Stringer("cid", cid), zap.String("tree", tid)) } } @@ -118,24 +114,19 @@ func (s *Service) SynchronizeTree(ctx context.Context, cid cid.ID, treeID string return ErrNotInContainer } - var d pilorama.CIDDescriptor - d.CID = cid - d.Position = pos - d.Size = len(nodes) - nodes = randomizeNodeOrder(nodes, pos) if len(nodes) == 0 { return nil } - s.synchronizeTree(ctx, d, 0, treeID, nodes) + s.synchronizeTree(ctx, cid, 0, treeID, nodes) return nil } -func (s *Service) synchronizeTree(ctx context.Context, d pilorama.CIDDescriptor, from uint64, +func (s *Service) synchronizeTree(ctx context.Context, cid cidSDK.ID, from uint64, treeID string, nodes []netmapSDK.NodeInfo) uint64 { s.log.Debug("synchronize tree", - zap.Stringer("cid", d.CID), + zap.Stringer("cid", cid), zap.String("tree", treeID), zap.Uint64("from", from)) @@ -157,7 +148,7 @@ func (s *Service) synchronizeTree(ctx context.Context, d pilorama.CIDDescriptor, treeClient := NewTreeServiceClient(cc) for { - h, err := s.synchronizeSingle(ctx, d, treeID, height, treeClient) + h, err := s.synchronizeSingle(ctx, cid, treeID, height, treeClient) if height < h { height = h } @@ -179,9 +170,9 @@ func (s *Service) synchronizeTree(ctx context.Context, d pilorama.CIDDescriptor, return newHeight } -func (s *Service) synchronizeSingle(ctx context.Context, d pilorama.CIDDescriptor, treeID string, height uint64, treeClient TreeServiceClient) (uint64, error) { +func (s *Service) synchronizeSingle(ctx context.Context, cid cidSDK.ID, treeID string, height uint64, treeClient TreeServiceClient) (uint64, error) { rawCID := make([]byte, sha256.Size) - d.CID.Encode(rawCID) + cid.Encode(rawCID) for { newHeight := height @@ -211,7 +202,7 @@ func (s *Service) synchronizeSingle(ctx context.Context, d pilorama.CIDDescripto if err := m.Meta.FromBytes(lm.Meta); err != nil { return newHeight, err } - if err := s.forest.TreeApply(d, treeID, m, true); err != nil { + if err := s.forest.TreeApply(cid, treeID, m, true); err != nil { return newHeight, err } if m.Time > newHeight {