forked from TrueCloudLab/frostfs-node
[#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>
This commit is contained in:
parent
ec2c5d45b4
commit
47e8c5bf23
9 changed files with 57 additions and 90 deletions
|
@ -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
|
||||
|
|
|
@ -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},
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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{
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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()))
|
||||
|
|
|
@ -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(),
|
||||
|
|
|
@ -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 {
|
||||
|
|
Loading…
Reference in a new issue