From 47e8c5bf23eedfc688828a3b874cb861ee4c6adb Mon Sep 17 00:00:00 2001
From: Evgenii Stratonikov <e.stratonikov@yadro.com>
Date: Tue, 21 Mar 2023 15:43:12 +0300
Subject: [PATCH] [#156] pilorama: Remove CIDDescriptor from TreeApply()

Initially it was there to check whether an update is being initiated by
a proper node. It is now obsolete for 2 reasons:
1. Background synchronization fetches all operations from a single node.
2. There are a lot more problems with trust in the tree service, it is
   only used in controlled environments.

Signed-off-by: Evgenii Stratonikov <e.stratonikov@yadro.com>
---
 pkg/local_object_storage/engine/tree.go       |  8 +--
 pkg/local_object_storage/pilorama/boltdb.go   | 18 +++---
 pkg/local_object_storage/pilorama/forest.go   |  9 +--
 .../pilorama/forest_test.go                   | 61 +++++++------------
 .../pilorama/interface.go                     |  2 +-
 pkg/local_object_storage/shard/tree.go        |  4 +-
 pkg/services/tree/replicator.go               |  4 +-
 pkg/services/tree/service.go                  |  6 +-
 pkg/services/tree/sync.go                     | 35 ++++-------
 9 files changed, 57 insertions(+), 90 deletions(-)

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 {