From 4828b6794fa73e1024b85edf1da3f093ed6978e5 Mon Sep 17 00:00:00 2001 From: Ivan Shvedunov Date: Mon, 11 Nov 2024 13:37:39 +0400 Subject: [PATCH 1/6] sync2: ATX integration This adds set reconciliation for ATXs. There are per-epoch syncers, with lower FPTree depth (16 by default) used for older epochs and greater FPTree depth (21 by default) used for current epoch. Both active syncv2 and passive (server-only) syncv2 are disabled by default. It is possible to enable syncv2 in server-only or full (active) mode. --- config/mainnet.go | 14 ++ config/presets/testnet.go | 13 + fetch/fetch.go | 9 + fetch/mesh_data.go | 16 +- fetch/mesh_data_test.go | 21 +- sync2/atxs.go | 336 +++++++++++++++++++++++++ sync2/atxs_test.go | 414 +++++++++++++++++++++++++++++++ sync2/interface.go | 36 +++ sync2/mocks_test.go | 498 ++++++++++++++++++++++++++++++++++++++ syncer/interface.go | 5 + syncer/mocks/mocks.go | 99 ++++++++ syncer/syncer.go | 172 +++++++++++-- syncer/syncer_test.go | 156 ++++++++++-- system/fetcher.go | 10 + 14 files changed, 1749 insertions(+), 50 deletions(-) create mode 100644 sync2/atxs.go create mode 100644 sync2/atxs_test.go create mode 100644 sync2/interface.go create mode 100644 sync2/mocks_test.go diff --git a/config/mainnet.go b/config/mainnet.go index 65312d2877..58d0a77cac 100644 --- a/config/mainnet.go +++ b/config/mainnet.go @@ -24,6 +24,7 @@ import ( "github.com/spacemeshos/go-spacemesh/hare4" "github.com/spacemeshos/go-spacemesh/miner" "github.com/spacemeshos/go-spacemesh/p2p" + "github.com/spacemeshos/go-spacemesh/sync2" "github.com/spacemeshos/go-spacemesh/syncer" "github.com/spacemeshos/go-spacemesh/syncer/atxsync" "github.com/spacemeshos/go-spacemesh/syncer/malsync" @@ -77,6 +78,14 @@ func MainnetConfig() Config { hare4conf := hare4.DefaultConfig() hare4conf.Enable = false + + oldAtxSyncCfg := sync2.DefaultConfig() + oldAtxSyncCfg.MultiPeerReconcilerConfig.SyncInterval = time.Hour + oldAtxSyncCfg.MaxDepth = 16 + newAtxSyncCfg := sync2.DefaultConfig() + newAtxSyncCfg.MaxDepth = 21 + newAtxSyncCfg.MultiPeerReconcilerConfig.SyncInterval = 5 * time.Minute + return Config{ BaseConfig: BaseConfig{ DataDirParent: defaultDataDir, @@ -212,6 +221,11 @@ func MainnetConfig() Config { DisableMeshAgreement: true, AtxSync: atxsync.DefaultConfig(), MalSync: malsync.DefaultConfig(), + V2: syncer.SyncV2Config{ + OldAtxSyncCfg: oldAtxSyncCfg, + NewAtxSyncCfg: newAtxSyncCfg, + ParallelLoadLimit: 10, + }, }, Recovery: checkpoint.DefaultConfig(), Cache: datastore.DefaultConfig(), diff --git a/config/presets/testnet.go b/config/presets/testnet.go index bb066c352e..713cea25d8 100644 --- a/config/presets/testnet.go +++ b/config/presets/testnet.go @@ -25,6 +25,7 @@ import ( "github.com/spacemeshos/go-spacemesh/hare4" "github.com/spacemeshos/go-spacemesh/miner" "github.com/spacemeshos/go-spacemesh/p2p" + "github.com/spacemeshos/go-spacemesh/sync2" "github.com/spacemeshos/go-spacemesh/syncer" "github.com/spacemeshos/go-spacemesh/syncer/atxsync" "github.com/spacemeshos/go-spacemesh/syncer/malsync" @@ -65,6 +66,13 @@ func testnet() config.Config { hare4conf := hare4.DefaultConfig() hare4conf.Enable = false defaultdir := filepath.Join(home, "spacemesh-testnet", "/") + + oldAtxSyncCfg := sync2.DefaultConfig() + oldAtxSyncCfg.MaxDepth = 16 + newAtxSyncCfg := sync2.DefaultConfig() + newAtxSyncCfg.MaxDepth = 21 + newAtxSyncCfg.MultiPeerReconcilerConfig.SyncInterval = 5 * time.Minute + return config.Config{ Preset: "testnet", BaseConfig: config.BaseConfig{ @@ -163,6 +171,11 @@ func testnet() config.Config { OutOfSyncThresholdLayers: 10, AtxSync: atxsync.DefaultConfig(), MalSync: malsync.DefaultConfig(), + V2: syncer.SyncV2Config{ + OldAtxSyncCfg: oldAtxSyncCfg, + NewAtxSyncCfg: newAtxSyncCfg, + ParallelLoadLimit: 10, + }, }, Recovery: checkpoint.DefaultConfig(), Cache: datastore.DefaultConfig(), diff --git a/fetch/fetch.go b/fetch/fetch.go index 396342a4bb..abee69cbdb 100644 --- a/fetch/fetch.go +++ b/fetch/fetch.go @@ -11,6 +11,7 @@ import ( "sync" "time" + corehost "github.com/libp2p/go-libp2p/core/host" "github.com/libp2p/go-libp2p/core/network" "go.uber.org/zap" "golang.org/x/sync/errgroup" @@ -1013,3 +1014,11 @@ func (f *Fetch) SelectBestShuffled(n int) []p2p.Peer { }) return peers } + +func (f *Fetch) Host() corehost.Host { + return f.host.(corehost.Host) +} + +func (f *Fetch) Peers() *peers.Peers { + return f.peers +} diff --git a/fetch/mesh_data.go b/fetch/mesh_data.go index ad271f307b..28382808ba 100644 --- a/fetch/mesh_data.go +++ b/fetch/mesh_data.go @@ -30,7 +30,7 @@ func (f *Fetch) GetAtxs(ctx context.Context, ids []types.ATXID, opts ...system.G return nil } - options := system.GetAtxOpts{} + var options system.GetAtxOpts for _, opt := range opts { opt(&options) } @@ -41,10 +41,20 @@ func (f *Fetch) GetAtxs(ctx context.Context, ids []types.ATXID, opts ...system.G zap.Bool("limiting", !options.LimitingOff), ) hashes := types.ATXIDsToHashes(ids) + handler := f.validators.atx.HandleMessage + if options.RecvChannel != nil { + handler = func(ctx context.Context, id types.Hash32, p p2p.Peer, data []byte) error { + if err := f.validators.atx.HandleMessage(ctx, id, p, data); err != nil { + return err + } + options.RecvChannel <- types.ATXID(id) + return nil + } + } if options.LimitingOff { - return f.getHashes(ctx, hashes, datastore.ATXDB, f.validators.atx.HandleMessage) + return f.getHashes(ctx, hashes, datastore.ATXDB, handler) } - return f.getHashes(ctx, hashes, datastore.ATXDB, f.validators.atx.HandleMessage, withLimiter(f.getAtxsLimiter)) + return f.getHashes(ctx, hashes, datastore.ATXDB, handler, withLimiter(f.getAtxsLimiter)) } type dataReceiver func(context.Context, types.Hash32, p2p.Peer, []byte) error diff --git a/fetch/mesh_data_test.go b/fetch/mesh_data_test.go index 713c96a7b7..56b1b83e91 100644 --- a/fetch/mesh_data_test.go +++ b/fetch/mesh_data_test.go @@ -86,7 +86,7 @@ func startTestLoop(tb testing.TB, f *Fetch, eg *errgroup.Group, stop chan struct default: f.mu.Lock() for h, req := range f.unprocessed { - require.NoError(tb, req.validator(req.ctx, types.Hash32{}, p2p.NoPeer, []byte{})) + require.NoError(tb, req.validator(req.ctx, h, p2p.NoPeer, []byte{})) close(req.promise.completed) delete(f.unprocessed, h) } @@ -591,7 +591,7 @@ func genATXs(tb testing.TB, num uint32) []*types.ActivationTx { } func TestGetATXs(t *testing.T) { - atxs := genATXs(t, 2) + atxs := genATXs(t, 4) f := createFetch(t) f.mAtxH.EXPECT(). HandleMessage(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()). @@ -602,10 +602,23 @@ func TestGetATXs(t *testing.T) { var eg errgroup.Group startTestLoop(t, f.Fetch, &eg, stop) - atxIDs := types.ToATXIDs(atxs) - require.NoError(t, f.GetAtxs(context.Background(), atxIDs)) + atxIDs1 := types.ToATXIDs(atxs[:2]) + require.NoError(t, f.GetAtxs(context.Background(), atxIDs1)) + + recvCh := make(chan types.ATXID) + atxIDs2 := types.ToATXIDs(atxs[2:]) + var recvIDs []types.ATXID + eg.Go(func() error { + for id := range recvCh { + recvIDs = append(recvIDs, id) + } + return nil + }) + require.NoError(t, f.GetAtxs(context.Background(), atxIDs2, system.WithRecvChannel(recvCh))) + close(recvCh) close(stop) require.NoError(t, eg.Wait()) + require.ElementsMatch(t, atxIDs2, recvIDs) } func TestGetActiveSet(t *testing.T) { diff --git a/sync2/atxs.go b/sync2/atxs.go new file mode 100644 index 0000000000..11feba1399 --- /dev/null +++ b/sync2/atxs.go @@ -0,0 +1,336 @@ +package sync2 + +import ( + "context" + "errors" + "fmt" + "time" + + "github.com/jonboulle/clockwork" + "go.uber.org/zap" + "golang.org/x/sync/errgroup" + + "github.com/spacemeshos/go-spacemesh/common/types" + "github.com/spacemeshos/go-spacemesh/fetch" + "github.com/spacemeshos/go-spacemesh/p2p" + "github.com/spacemeshos/go-spacemesh/p2p/pubsub" + "github.com/spacemeshos/go-spacemesh/p2p/server" + "github.com/spacemeshos/go-spacemesh/sql" + "github.com/spacemeshos/go-spacemesh/sql/expr" + "github.com/spacemeshos/go-spacemesh/sync2/dbset" + "github.com/spacemeshos/go-spacemesh/sync2/multipeer" + "github.com/spacemeshos/go-spacemesh/sync2/rangesync" + "github.com/spacemeshos/go-spacemesh/sync2/sqlstore" + "github.com/spacemeshos/go-spacemesh/system" +) + +const ( + proto = "sync/2" +) + +type ATXHandler struct { + logger *zap.Logger + f Fetcher + clock clockwork.Clock + batchSize int + maxAttempts int + maxBatchRetries int + failedBatchDelay time.Duration +} + +var _ multipeer.SyncKeyHandler = &ATXHandler{} + +func NewATXHandler( + logger *zap.Logger, + f Fetcher, + batchSize, maxAttempts, maxBatchRetries int, + failedBatchDelay time.Duration, + clock clockwork.Clock, +) *ATXHandler { + if clock == nil { + clock = clockwork.NewRealClock() + } + return &ATXHandler{ + f: f, + logger: logger, + clock: clock, + batchSize: batchSize, + maxAttempts: maxAttempts, + maxBatchRetries: maxBatchRetries, + failedBatchDelay: failedBatchDelay, + } +} + +func (h *ATXHandler) Receive(k rangesync.KeyBytes, peer p2p.Peer) (bool, error) { + var id types.ATXID + copy(id[:], k) + h.f.RegisterPeerHash(peer, id.Hash32()) + return false, nil +} + +func (h *ATXHandler) Commit(ctx context.Context, peer p2p.Peer, base, new rangesync.OrderedSet) error { + h.logger.Debug("begin atx commit") + defer h.logger.Debug("end atx commit") + sr := new.Received() + var firstK rangesync.KeyBytes + numDownloaded := 0 + state := make(map[types.ATXID]int) + for k := range sr.Seq { + if firstK == nil { + firstK = k + } else if firstK.Compare(k) == 0 { + break + } + found, err := base.Has(k) + if err != nil { + return fmt.Errorf("check if ATX exists: %w", err) + } + if found { + continue + } + state[types.BytesToATXID(k)] = 0 + } + if err := sr.Error(); err != nil { + return fmt.Errorf("get item: %w", err) + } + total := len(state) + items := make([]types.ATXID, 0, h.batchSize) + startTime := time.Now() + batchAttemptsRemaining := h.maxBatchRetries + for len(state) > 0 { + items = items[:0] + for id, n := range state { + if n >= h.maxAttempts { + h.logger.Debug("failed to download ATX: max attempts reached", + zap.String("atx", id.ShortString())) + delete(state, id) + continue + } + items = append(items, id) + if len(items) == h.batchSize { + break + } + } + if len(items) == 0 { + break + } + + var eg errgroup.Group + recvCh := make(chan types.ATXID) + someSucceeded := false + eg.Go(func() error { + for id := range recvCh { + numDownloaded++ + someSucceeded = true + delete(state, id) + } + return nil + }) + err := h.f.GetAtxs(ctx, items, system.WithRecvChannel(recvCh)) + close(recvCh) + eg.Wait() + if err != nil { + if errors.Is(err, context.Canceled) { + return err + } + batchError := &fetch.BatchError{} + if errors.As(err, &batchError) { + h.logger.Debug("QQQQQ: batch error", zap.Error(err)) + for hash, err := range batchError.Errors { + if _, exists := state[types.ATXID(hash)]; !exists { + continue + } + if errors.Is(err, pubsub.ErrValidationReject) { + // if the atx invalid there's no point downloading it again + state[types.ATXID(hash)] = h.maxAttempts + } else { + state[types.ATXID(hash)]++ + } + } + } else { + h.logger.Debug("failed to download ATXs", zap.Error(err)) + } + } + if !someSucceeded { + if batchAttemptsRemaining == 0 { + return errors.New("failed to download ATXs: max batch retries reached") + } + batchAttemptsRemaining-- + h.logger.Debug("failed to download any ATXs: will retry batch", + zap.Int("remaining", batchAttemptsRemaining), + zap.Duration("delay", h.failedBatchDelay)) + select { + case <-ctx.Done(): + return ctx.Err() + case <-h.clock.After(h.failedBatchDelay): + } + } else { + batchAttemptsRemaining = h.maxBatchRetries + h.logger.Debug("fetched atxs", + zap.Int("total", total), + zap.Int("downloaded", numDownloaded), + zap.Float64("rate per sec", float64(numDownloaded)/time.Since(startTime).Seconds())) + } + } + return nil +} + +type MultiEpochATXSyncer struct { + logger *zap.Logger + oldCfg Config + newCfg Config + parallelLoadLimit int + hss HashSyncSource + newEpoch types.EpochID + atxSyncers []HashSync +} + +func NewMultiEpochATXSyncer( + logger *zap.Logger, + hss HashSyncSource, + oldCfg, newCfg Config, + parallelLoadLimit int, +) *MultiEpochATXSyncer { + return &MultiEpochATXSyncer{ + logger: logger, + oldCfg: oldCfg, + newCfg: newCfg, + parallelLoadLimit: parallelLoadLimit, + hss: hss, + } +} + +func (s *MultiEpochATXSyncer) load(newEpoch types.EpochID) error { + if len(s.atxSyncers) < int(newEpoch) { + s.atxSyncers = append(s.atxSyncers, make([]HashSync, int(newEpoch)-len(s.atxSyncers))...) + } + s.newEpoch = newEpoch + var eg errgroup.Group + if s.parallelLoadLimit > 0 { + eg.SetLimit(s.parallelLoadLimit) + } + for epoch := types.EpochID(1); epoch <= newEpoch; epoch++ { + if s.atxSyncers[epoch-1] != nil { + continue + } + eg.Go(func() error { + name := fmt.Sprintf("atx-sync-%d", epoch) + cfg := s.oldCfg + if epoch == newEpoch { + cfg = s.newCfg + } + hs := s.hss.CreateHashSync(name, cfg, epoch) + if err := hs.Load(); err != nil { + return fmt.Errorf("load ATX syncer for epoch %d: %w", epoch, err) + } + s.atxSyncers[epoch-1] = hs + return nil + }) + } + return eg.Wait() +} + +// EnsureSync ensures that ATX sync is active for all the epochs up to and including +// currentEpoch, and that all ATXs are +// synced up to and including lastWaitEpoch. +// If newEpoch argument is non-zero, faster but less memory efficient sync is used for +// that epoch, based on the newCfg (larger maxDepth). +// For other epochs, oldCfg is used which corresponds to slower but more memory efficient +// sync (smaller maxDepth). +// It returns the last epoch that was synced synchronously. +func (s *MultiEpochATXSyncer) EnsureSync( + ctx context.Context, + lastWaitEpoch, newEpoch types.EpochID, +) (lastSynced types.EpochID, err error) { + if newEpoch != s.newEpoch && int(s.newEpoch) <= len(s.atxSyncers) && s.newEpoch > 0 { + s.atxSyncers[s.newEpoch-1].Stop() + s.atxSyncers[s.newEpoch-1] = nil + } + if err := s.load(newEpoch); err != nil { + return lastSynced, err + } + for epoch := types.EpochID(1); epoch <= newEpoch; epoch++ { + syncer := s.atxSyncers[epoch-1] + if epoch <= lastWaitEpoch { + s.logger.Info("waiting for epoch to sync", zap.Uint32("epoch", epoch.Uint32())) + if err := syncer.StartAndSync(ctx); err != nil { + return lastSynced, fmt.Errorf("error syncing old ATXs: %w", err) + } + lastSynced = epoch + } else { + syncer.Start() + } + } + return lastSynced, nil +} + +// Stop stops all ATX syncers. +func (s *MultiEpochATXSyncer) Stop() { + for _, hs := range s.atxSyncers { + hs.Stop() + } + s.atxSyncers = nil + s.newEpoch = 0 +} + +func atxsTable(epoch types.EpochID) *sqlstore.SyncedTable { + return &sqlstore.SyncedTable{ + TableName: "atxs", + IDColumn: "id", + TimestampColumn: "received", + Filter: expr.MustParse("epoch = ?"), + Binder: func(s *sql.Statement) { + s.BindInt64(1, int64(epoch)) + }, + } +} + +func NewATXSyncer( + logger *zap.Logger, + d *rangesync.Dispatcher, + name string, + cfg Config, + db sql.StateDatabase, + f Fetcher, + epoch types.EpochID, + enableActiveSync bool, +) *P2PHashSync { + curSet := dbset.NewDBSet(db, atxsTable(epoch), 32, cfg.MaxDepth) + return NewP2PHashSync( + logger, d, name, curSet, 32, f.Peers(), + NewATXHandler( + logger, f, cfg.BatchSize, cfg.MaxAttempts, + cfg.MaxBatchRetries, cfg.FailedBatchDelay, nil), + cfg, enableActiveSync) +} + +func NewDispatcher(logger *zap.Logger, f Fetcher) *rangesync.Dispatcher { + d := rangesync.NewDispatcher(logger) + d.SetupServer(f.Host(), proto, server.WithHardTimeout(20*time.Minute)) + return d +} + +type ATXSyncSource struct { + logger *zap.Logger + d *rangesync.Dispatcher + db sql.StateDatabase + f Fetcher + enableActiveSync bool +} + +var _ HashSyncSource = &ATXSyncSource{} + +func NewATXSyncSource( + logger *zap.Logger, + d *rangesync.Dispatcher, + db sql.StateDatabase, + f Fetcher, + enableActiveSync bool, +) *ATXSyncSource { + return &ATXSyncSource{logger: logger, d: d, db: db, f: f, enableActiveSync: enableActiveSync} +} + +// CreateHashSync implements HashSyncSource. +func (as *ATXSyncSource) CreateHashSync(name string, cfg Config, epoch types.EpochID) HashSync { + return NewATXSyncer(as.logger.Named(name), as.d, name, cfg, as.db, as.f, epoch, as.enableActiveSync) +} diff --git a/sync2/atxs_test.go b/sync2/atxs_test.go new file mode 100644 index 0000000000..7dcc780ebe --- /dev/null +++ b/sync2/atxs_test.go @@ -0,0 +1,414 @@ +package sync2_test + +import ( + "context" + "errors" + "fmt" + "testing" + "time" + + "github.com/jonboulle/clockwork" + "github.com/stretchr/testify/require" + gomock "go.uber.org/mock/gomock" + "go.uber.org/zap/zaptest" + "golang.org/x/sync/errgroup" + + "github.com/spacemeshos/go-spacemesh/common/types" + "github.com/spacemeshos/go-spacemesh/fetch" + "github.com/spacemeshos/go-spacemesh/p2p" + "github.com/spacemeshos/go-spacemesh/p2p/pubsub" + "github.com/spacemeshos/go-spacemesh/sync2" + "github.com/spacemeshos/go-spacemesh/sync2/rangesync" + "github.com/spacemeshos/go-spacemesh/sync2/rangesync/mocks" + "github.com/spacemeshos/go-spacemesh/system" +) + +func TestAtxHandler_Success(t *testing.T) { + const ( + batchSize = 4 + maxAttempts = 3 + maxBatchRetries = 2 + batchRetryDelay = 10 * time.Second + ) + ctrl := gomock.NewController(t) + allAtxs := make([]types.ATXID, 10) + logger := zaptest.NewLogger(t) + peer := p2p.Peer("foobar") + for i := range allAtxs { + allAtxs[i] = types.RandomATXID() + } + f := NewMockFetcher(ctrl) + h := sync2.NewATXHandler(logger, f, batchSize, maxAttempts, maxBatchRetries, batchRetryDelay, nil) + baseSet := mocks.NewMockOrderedSet(ctrl) + newSet := mocks.NewMockOrderedSet(ctrl) + for _, id := range allAtxs { + f.EXPECT().RegisterPeerHash(peer, id.Hash32()) + baseSet.EXPECT().Has(rangesync.KeyBytes(id[:])).Return(false, nil) + add, err := h.Receive(id.Bytes(), peer) + require.False(t, add) + require.NoError(t, err) + } + toFetch := make(map[types.ATXID]bool) + for _, id := range allAtxs { + toFetch[id] = true + } + var batches []int + f.EXPECT().GetAtxs(gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( + func(_ context.Context, atxs []types.ATXID, opts ...system.GetAtxOpt) error { + batches = append(batches, len(atxs)) + var atxOpts system.GetAtxOpts + for _, opt := range opts { + opt(&atxOpts) + } + require.NotNil(t, atxOpts.RecvChannel) + for _, id := range atxs { + require.True(t, toFetch[id], "already fetched or bad ID") + delete(toFetch, id) + select { + case <-time.After(100 * time.Millisecond): + t.Error("timeout sending recvd id") + case atxOpts.RecvChannel <- id: + } + } + return nil + }).Times(3) + newSet.EXPECT().Received().Return(rangesync.SeqResult{ + Seq: func(yield func(k rangesync.KeyBytes) bool) { + // Received sequence may be cyclic and the handler should stop + // when it sees the first key again. + for { + for _, atx := range allAtxs { + if !yield(atx.Bytes()) { + return + } + } + } + }, + Error: rangesync.NoSeqError, + }) + require.NoError(t, h.Commit(context.Background(), peer, baseSet, newSet)) + require.Empty(t, toFetch) + require.Equal(t, []int{4, 4, 2}, batches) +} + +func TestAtxHandler_Retry(t *testing.T) { + const ( + batchSize = 4 + maxAttempts = 3 + maxBatchRetries = 2 + batchRetryDelay = 10 * time.Second + ) + ctrl := gomock.NewController(t) + allAtxs := make([]types.ATXID, 10) + logger := zaptest.NewLogger(t) + peer := p2p.Peer("foobar") + for i := range allAtxs { + allAtxs[i] = types.RandomATXID() + } + f := NewMockFetcher(ctrl) + h := sync2.NewATXHandler(logger, f, batchSize, maxAttempts, maxBatchRetries, batchRetryDelay, nil) + baseSet := mocks.NewMockOrderedSet(ctrl) + newSet := mocks.NewMockOrderedSet(ctrl) + for _, id := range allAtxs { + f.EXPECT().RegisterPeerHash(peer, id.Hash32()) + baseSet.EXPECT().Has(rangesync.KeyBytes(id[:])).Return(false, nil) + add, err := h.Receive(id.Bytes(), peer) + require.False(t, add) + require.NoError(t, err) + } + failCount := 0 + var fetched []types.ATXID + validationFailed := false + f.EXPECT().GetAtxs(gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( + func(_ context.Context, atxs []types.ATXID, opts ...system.GetAtxOpt) error { + errs := make(map[types.Hash32]error) + var atxOpts system.GetAtxOpts + for _, opt := range opts { + opt(&atxOpts) + } + require.NotNil(t, atxOpts.RecvChannel) + for _, id := range atxs { + switch { + case id == allAtxs[0]: + require.False(t, validationFailed, "retried after validation error") + errs[id.Hash32()] = pubsub.ErrValidationReject + validationFailed = true + case id == allAtxs[1] && failCount < 2: + errs[id.Hash32()] = errors.New("fetch failed") + failCount++ + default: + fetched = append(fetched, id) + select { + case <-time.After(100 * time.Millisecond): + t.Error("timeout sending recvd id") + case atxOpts.RecvChannel <- id: + } + } + } + if len(errs) > 0 { + var bErr fetch.BatchError + for h, err := range errs { + bErr.Add(h, err) + } + return &bErr + } + return nil + }).AnyTimes() + newSet.EXPECT().Received().Return(rangesync.SeqResult{ + Seq: func(yield func(k rangesync.KeyBytes) bool) { + for _, atx := range allAtxs { + if !yield(atx.Bytes()) { + return + } + } + }, + Error: rangesync.NoSeqError, + }) + require.NoError(t, h.Commit(context.Background(), peer, baseSet, newSet)) + require.ElementsMatch(t, allAtxs[1:], fetched) +} + +func TestAtxHandler_Cancel(t *testing.T) { + const ( + batchSize = 4 + maxAttempts = 3 + maxBatchRetries = 2 + batchRetryDelay = 10 * time.Second + ) + atxID := types.RandomATXID() + ctrl := gomock.NewController(t) + logger := zaptest.NewLogger(t) + peer := p2p.Peer("foobar") + f := NewMockFetcher(ctrl) + h := sync2.NewATXHandler(logger, f, batchSize, maxAttempts, maxBatchRetries, batchRetryDelay, nil) + baseSet := mocks.NewMockOrderedSet(ctrl) + newSet := mocks.NewMockOrderedSet(ctrl) + baseSet.EXPECT().Has(rangesync.KeyBytes(atxID[:])).Return(false, nil) + f.EXPECT().GetAtxs(gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( + func(_ context.Context, atxs []types.ATXID, opts ...system.GetAtxOpt) error { + return context.Canceled + }) + newSet.EXPECT().Received().Return(rangesync.SeqResult{ + Seq: func(yield func(k rangesync.KeyBytes) bool) { + yield(atxID.Bytes()) + }, + Error: rangesync.NoSeqError, + }) + require.ErrorIs(t, h.Commit(context.Background(), peer, baseSet, newSet), context.Canceled) +} + +func TestAtxHandler_BatchRetry(t *testing.T) { + const ( + batchSize = 4 + maxAttempts = 3 + maxBatchRetries = 2 + batchRetryDelay = 10 * time.Second + ) + ctrl := gomock.NewController(t) + allAtxs := make([]types.ATXID, 10) + logger := zaptest.NewLogger(t) + peer := p2p.Peer("foobar") + for i := range allAtxs { + allAtxs[i] = types.RandomATXID() + } + clock := clockwork.NewFakeClock() + f := NewMockFetcher(ctrl) + h := sync2.NewATXHandler(logger, f, batchSize, maxAttempts, maxBatchRetries, batchRetryDelay, clock) + baseSet := mocks.NewMockOrderedSet(ctrl) + newSet := mocks.NewMockOrderedSet(ctrl) + for _, id := range allAtxs { + f.EXPECT().RegisterPeerHash(peer, id.Hash32()) + baseSet.EXPECT().Has(rangesync.KeyBytes(id[:])).Return(false, nil) + add, err := h.Receive(id.Bytes(), peer) + require.False(t, add) + require.NoError(t, err) + } + newSet.EXPECT().Received().Return(rangesync.SeqResult{ + Seq: func(yield func(k rangesync.KeyBytes) bool) { + // Received sequence may be cyclic and the handler should stop + // when it sees the first key again. + for { + for _, atx := range allAtxs { + if !yield(atx.Bytes()) { + return + } + } + } + }, + Error: rangesync.NoSeqError, + }) + f.EXPECT().GetAtxs(gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( + func(_ context.Context, atxs []types.ATXID, opts ...system.GetAtxOpt) error { + return errors.New("fetch failed") + }) + var eg errgroup.Group + eg.Go(func() error { + return h.Commit(context.Background(), peer, baseSet, newSet) + }) + // wait for delay after 1st batch failure + clock.BlockUntil(1) + toFetch := make(map[types.ATXID]bool) + for _, id := range allAtxs { + toFetch[id] = true + } + f.EXPECT().GetAtxs(gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( + func(_ context.Context, atxs []types.ATXID, opts ...system.GetAtxOpt) error { + var atxOpts system.GetAtxOpts + for _, opt := range opts { + opt(&atxOpts) + } + require.NotNil(t, atxOpts.RecvChannel) + for _, id := range atxs { + require.True(t, toFetch[id], "already fetched or bad ID") + delete(toFetch, id) + select { + case <-time.After(100 * time.Millisecond): + t.Error("timeout sending recvd id") + case atxOpts.RecvChannel <- id: + } + } + return nil + }).Times(3) + clock.Advance(batchRetryDelay) + require.NoError(t, eg.Wait()) + require.Empty(t, toFetch) +} + +func TestAtxHandler_BatchRetry_Fail(t *testing.T) { + const ( + batchSize = 4 + maxAttempts = 3 + maxBatchRetries = 2 + batchRetryDelay = 10 * time.Second + ) + ctrl := gomock.NewController(t) + allAtxs := make([]types.ATXID, 10) + logger := zaptest.NewLogger(t) + peer := p2p.Peer("foobar") + for i := range allAtxs { + allAtxs[i] = types.RandomATXID() + } + clock := clockwork.NewFakeClock() + f := NewMockFetcher(ctrl) + h := sync2.NewATXHandler(logger, f, batchSize, maxAttempts, maxBatchRetries, batchRetryDelay, clock) + baseSet := mocks.NewMockOrderedSet(ctrl) + newSet := mocks.NewMockOrderedSet(ctrl) + for _, id := range allAtxs { + f.EXPECT().RegisterPeerHash(peer, id.Hash32()) + baseSet.EXPECT().Has(rangesync.KeyBytes(id[:])).Return(false, nil) + add, err := h.Receive(id.Bytes(), peer) + require.False(t, add) + require.NoError(t, err) + } + newSet.EXPECT().Received().Return(rangesync.SeqResult{ + Seq: func(yield func(k rangesync.KeyBytes) bool) { + // Received sequence may be cyclic and the handler should stop + // when it sees the first key again. + for { + for _, atx := range allAtxs { + if !yield(atx.Bytes()) { + return + } + } + } + }, + Error: rangesync.NoSeqError, + }) + f.EXPECT().GetAtxs(gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( + func(_ context.Context, atxs []types.ATXID, opts ...system.GetAtxOpt) error { + return errors.New("fetch failed") + }).Times(3) + var eg errgroup.Group + eg.Go(func() error { + return h.Commit(context.Background(), peer, baseSet, newSet) + }) + for range 2 { + clock.BlockUntil(1) + clock.Advance(batchRetryDelay) + } + require.Error(t, eg.Wait()) +} + +func TestMultiEpochATXSyncer(t *testing.T) { + ctrl := gomock.NewController(t) + logger := zaptest.NewLogger(t) + oldCfg := sync2.DefaultConfig() + oldCfg.MaxDepth = 16 + newCfg := sync2.DefaultConfig() + newCfg.MaxDepth = 24 + hss := NewMockHashSyncSource(ctrl) + mhs := sync2.NewMultiEpochATXSyncer(logger, hss, oldCfg, newCfg, 1) + ctx := context.Background() + + lastSynced, err := mhs.EnsureSync(ctx, 0, 0) + require.NoError(t, err) + require.Zero(t, lastSynced) + + var syncActions []string + curIdx := 0 + hss.EXPECT().CreateHashSync(gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( + func(name string, cfg sync2.Config, epoch types.EpochID) sync2.HashSync { + idx := curIdx + curIdx++ + syncActions = append(syncActions, + fmt.Sprintf("new %s epoch %d maxDepth %d -> %d", name, epoch, cfg.MaxDepth, idx)) + hs := NewMockHashSync(ctrl) + hs.EXPECT().Load().DoAndReturn(func() error { + syncActions = append(syncActions, fmt.Sprintf("load %d %s", idx, name)) + return nil + }).AnyTimes() + hs.EXPECT().StartAndSync(ctx).DoAndReturn(func(_ context.Context) error { + syncActions = append(syncActions, fmt.Sprintf("start+sync %d %s", idx, name)) + return nil + }).AnyTimes() + hs.EXPECT().Start().DoAndReturn(func() { + syncActions = append(syncActions, fmt.Sprintf("start %d %s", idx, name)) + }).AnyTimes() + hs.EXPECT().Stop().DoAndReturn(func() { + syncActions = append(syncActions, fmt.Sprintf("stop %d %s", idx, name)) + }).AnyTimes() + return hs + }).AnyTimes() + + // Last wait epoch 3, new epoch 3 + lastSynced, err = mhs.EnsureSync(ctx, 3, 3) + require.NoError(t, err) + require.Equal(t, []string{ + "new atx-sync-1 epoch 1 maxDepth 16 -> 0", + "load 0 atx-sync-1", + "new atx-sync-2 epoch 2 maxDepth 16 -> 1", + "load 1 atx-sync-2", + "new atx-sync-3 epoch 3 maxDepth 24 -> 2", + "load 2 atx-sync-3", + "start+sync 0 atx-sync-1", + "start+sync 1 atx-sync-2", + "start+sync 2 atx-sync-3", + }, syncActions) + syncActions = nil + require.Equal(t, types.EpochID(3), lastSynced) + + // Advance to epoch 4 w/o wait + lastSynced, err = mhs.EnsureSync(ctx, 0, 4) + require.NoError(t, err) + require.Equal(t, []string{ + "stop 2 atx-sync-3", + "new atx-sync-3 epoch 3 maxDepth 16 -> 3", + "load 3 atx-sync-3", + "new atx-sync-4 epoch 4 maxDepth 24 -> 4", + "load 4 atx-sync-4", + "start 0 atx-sync-1", + "start 1 atx-sync-2", + "start 3 atx-sync-3", + "start 4 atx-sync-4", + }, syncActions) + syncActions = nil + require.Equal(t, types.EpochID(0), lastSynced) + + mhs.Stop() + require.Equal(t, []string{ + "stop 0 atx-sync-1", + "stop 1 atx-sync-2", + "stop 3 atx-sync-3", + "stop 4 atx-sync-4", + }, syncActions) +} diff --git a/sync2/interface.go b/sync2/interface.go new file mode 100644 index 0000000000..624c373d2d --- /dev/null +++ b/sync2/interface.go @@ -0,0 +1,36 @@ +package sync2 + +import ( + "context" + + "github.com/libp2p/go-libp2p/core/host" + + "github.com/spacemeshos/go-spacemesh/common/types" + "github.com/spacemeshos/go-spacemesh/fetch/peers" + "github.com/spacemeshos/go-spacemesh/p2p" + "github.com/spacemeshos/go-spacemesh/system" +) + +//go:generate mockgen -typed -package=sync2_test -destination=./mocks_test.go -source=./interface.go + +type Fetcher interface { + system.AtxFetcher + Host() host.Host + Peers() *peers.Peers + RegisterPeerHash(peer p2p.Peer, hash types.Hash32) +} + +type HashSync interface { + Load() error + Start() + Stop() + StartAndSync(ctx context.Context) error +} + +type HashSyncSource interface { + CreateHashSync(name string, cfg Config, epoch types.EpochID) HashSync +} + +type LayerTicker interface { + CurrentLayer() types.LayerID +} diff --git a/sync2/mocks_test.go b/sync2/mocks_test.go new file mode 100644 index 0000000000..640dde1143 --- /dev/null +++ b/sync2/mocks_test.go @@ -0,0 +1,498 @@ +// Code generated by MockGen. DO NOT EDIT. +// Source: ./interface.go +// +// Generated by this command: +// +// mockgen -typed -package=sync2_test -destination=./mocks_test.go -source=./interface.go +// + +// Package sync2_test is a generated GoMock package. +package sync2_test + +import ( + context "context" + reflect "reflect" + + host "github.com/libp2p/go-libp2p/core/host" + types "github.com/spacemeshos/go-spacemesh/common/types" + peers "github.com/spacemeshos/go-spacemesh/fetch/peers" + p2p "github.com/spacemeshos/go-spacemesh/p2p" + sync2 "github.com/spacemeshos/go-spacemesh/sync2" + system "github.com/spacemeshos/go-spacemesh/system" + gomock "go.uber.org/mock/gomock" +) + +// MockFetcher is a mock of Fetcher interface. +type MockFetcher struct { + ctrl *gomock.Controller + recorder *MockFetcherMockRecorder + isgomock struct{} +} + +// MockFetcherMockRecorder is the mock recorder for MockFetcher. +type MockFetcherMockRecorder struct { + mock *MockFetcher +} + +// NewMockFetcher creates a new mock instance. +func NewMockFetcher(ctrl *gomock.Controller) *MockFetcher { + mock := &MockFetcher{ctrl: ctrl} + mock.recorder = &MockFetcherMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockFetcher) EXPECT() *MockFetcherMockRecorder { + return m.recorder +} + +// GetAtxs mocks base method. +func (m *MockFetcher) GetAtxs(arg0 context.Context, arg1 []types.ATXID, arg2 ...system.GetAtxOpt) error { + m.ctrl.T.Helper() + varargs := []any{arg0, arg1} + for _, a := range arg2 { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "GetAtxs", varargs...) + ret0, _ := ret[0].(error) + return ret0 +} + +// GetAtxs indicates an expected call of GetAtxs. +func (mr *MockFetcherMockRecorder) GetAtxs(arg0, arg1 any, arg2 ...any) *MockFetcherGetAtxsCall { + mr.mock.ctrl.T.Helper() + varargs := append([]any{arg0, arg1}, arg2...) + call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetAtxs", reflect.TypeOf((*MockFetcher)(nil).GetAtxs), varargs...) + return &MockFetcherGetAtxsCall{Call: call} +} + +// MockFetcherGetAtxsCall wrap *gomock.Call +type MockFetcherGetAtxsCall struct { + *gomock.Call +} + +// Return rewrite *gomock.Call.Return +func (c *MockFetcherGetAtxsCall) Return(arg0 error) *MockFetcherGetAtxsCall { + c.Call = c.Call.Return(arg0) + return c +} + +// Do rewrite *gomock.Call.Do +func (c *MockFetcherGetAtxsCall) Do(f func(context.Context, []types.ATXID, ...system.GetAtxOpt) error) *MockFetcherGetAtxsCall { + c.Call = c.Call.Do(f) + return c +} + +// DoAndReturn rewrite *gomock.Call.DoAndReturn +func (c *MockFetcherGetAtxsCall) DoAndReturn(f func(context.Context, []types.ATXID, ...system.GetAtxOpt) error) *MockFetcherGetAtxsCall { + c.Call = c.Call.DoAndReturn(f) + return c +} + +// Host mocks base method. +func (m *MockFetcher) Host() host.Host { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Host") + ret0, _ := ret[0].(host.Host) + return ret0 +} + +// Host indicates an expected call of Host. +func (mr *MockFetcherMockRecorder) Host() *MockFetcherHostCall { + mr.mock.ctrl.T.Helper() + call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Host", reflect.TypeOf((*MockFetcher)(nil).Host)) + return &MockFetcherHostCall{Call: call} +} + +// MockFetcherHostCall wrap *gomock.Call +type MockFetcherHostCall struct { + *gomock.Call +} + +// Return rewrite *gomock.Call.Return +func (c *MockFetcherHostCall) Return(arg0 host.Host) *MockFetcherHostCall { + c.Call = c.Call.Return(arg0) + return c +} + +// Do rewrite *gomock.Call.Do +func (c *MockFetcherHostCall) Do(f func() host.Host) *MockFetcherHostCall { + c.Call = c.Call.Do(f) + return c +} + +// DoAndReturn rewrite *gomock.Call.DoAndReturn +func (c *MockFetcherHostCall) DoAndReturn(f func() host.Host) *MockFetcherHostCall { + c.Call = c.Call.DoAndReturn(f) + return c +} + +// Peers mocks base method. +func (m *MockFetcher) Peers() *peers.Peers { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Peers") + ret0, _ := ret[0].(*peers.Peers) + return ret0 +} + +// Peers indicates an expected call of Peers. +func (mr *MockFetcherMockRecorder) Peers() *MockFetcherPeersCall { + mr.mock.ctrl.T.Helper() + call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Peers", reflect.TypeOf((*MockFetcher)(nil).Peers)) + return &MockFetcherPeersCall{Call: call} +} + +// MockFetcherPeersCall wrap *gomock.Call +type MockFetcherPeersCall struct { + *gomock.Call +} + +// Return rewrite *gomock.Call.Return +func (c *MockFetcherPeersCall) Return(arg0 *peers.Peers) *MockFetcherPeersCall { + c.Call = c.Call.Return(arg0) + return c +} + +// Do rewrite *gomock.Call.Do +func (c *MockFetcherPeersCall) Do(f func() *peers.Peers) *MockFetcherPeersCall { + c.Call = c.Call.Do(f) + return c +} + +// DoAndReturn rewrite *gomock.Call.DoAndReturn +func (c *MockFetcherPeersCall) DoAndReturn(f func() *peers.Peers) *MockFetcherPeersCall { + c.Call = c.Call.DoAndReturn(f) + return c +} + +// RegisterPeerHash mocks base method. +func (m *MockFetcher) RegisterPeerHash(peer p2p.Peer, hash types.Hash32) { + m.ctrl.T.Helper() + m.ctrl.Call(m, "RegisterPeerHash", peer, hash) +} + +// RegisterPeerHash indicates an expected call of RegisterPeerHash. +func (mr *MockFetcherMockRecorder) RegisterPeerHash(peer, hash any) *MockFetcherRegisterPeerHashCall { + mr.mock.ctrl.T.Helper() + call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RegisterPeerHash", reflect.TypeOf((*MockFetcher)(nil).RegisterPeerHash), peer, hash) + return &MockFetcherRegisterPeerHashCall{Call: call} +} + +// MockFetcherRegisterPeerHashCall wrap *gomock.Call +type MockFetcherRegisterPeerHashCall struct { + *gomock.Call +} + +// Return rewrite *gomock.Call.Return +func (c *MockFetcherRegisterPeerHashCall) Return() *MockFetcherRegisterPeerHashCall { + c.Call = c.Call.Return() + return c +} + +// Do rewrite *gomock.Call.Do +func (c *MockFetcherRegisterPeerHashCall) Do(f func(p2p.Peer, types.Hash32)) *MockFetcherRegisterPeerHashCall { + c.Call = c.Call.Do(f) + return c +} + +// DoAndReturn rewrite *gomock.Call.DoAndReturn +func (c *MockFetcherRegisterPeerHashCall) DoAndReturn(f func(p2p.Peer, types.Hash32)) *MockFetcherRegisterPeerHashCall { + c.Call = c.Call.DoAndReturn(f) + return c +} + +// MockHashSync is a mock of HashSync interface. +type MockHashSync struct { + ctrl *gomock.Controller + recorder *MockHashSyncMockRecorder + isgomock struct{} +} + +// MockHashSyncMockRecorder is the mock recorder for MockHashSync. +type MockHashSyncMockRecorder struct { + mock *MockHashSync +} + +// NewMockHashSync creates a new mock instance. +func NewMockHashSync(ctrl *gomock.Controller) *MockHashSync { + mock := &MockHashSync{ctrl: ctrl} + mock.recorder = &MockHashSyncMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockHashSync) EXPECT() *MockHashSyncMockRecorder { + return m.recorder +} + +// Load mocks base method. +func (m *MockHashSync) Load() error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Load") + ret0, _ := ret[0].(error) + return ret0 +} + +// Load indicates an expected call of Load. +func (mr *MockHashSyncMockRecorder) Load() *MockHashSyncLoadCall { + mr.mock.ctrl.T.Helper() + call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Load", reflect.TypeOf((*MockHashSync)(nil).Load)) + return &MockHashSyncLoadCall{Call: call} +} + +// MockHashSyncLoadCall wrap *gomock.Call +type MockHashSyncLoadCall struct { + *gomock.Call +} + +// Return rewrite *gomock.Call.Return +func (c *MockHashSyncLoadCall) Return(arg0 error) *MockHashSyncLoadCall { + c.Call = c.Call.Return(arg0) + return c +} + +// Do rewrite *gomock.Call.Do +func (c *MockHashSyncLoadCall) Do(f func() error) *MockHashSyncLoadCall { + c.Call = c.Call.Do(f) + return c +} + +// DoAndReturn rewrite *gomock.Call.DoAndReturn +func (c *MockHashSyncLoadCall) DoAndReturn(f func() error) *MockHashSyncLoadCall { + c.Call = c.Call.DoAndReturn(f) + return c +} + +// Start mocks base method. +func (m *MockHashSync) Start() { + m.ctrl.T.Helper() + m.ctrl.Call(m, "Start") +} + +// Start indicates an expected call of Start. +func (mr *MockHashSyncMockRecorder) Start() *MockHashSyncStartCall { + mr.mock.ctrl.T.Helper() + call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Start", reflect.TypeOf((*MockHashSync)(nil).Start)) + return &MockHashSyncStartCall{Call: call} +} + +// MockHashSyncStartCall wrap *gomock.Call +type MockHashSyncStartCall struct { + *gomock.Call +} + +// Return rewrite *gomock.Call.Return +func (c *MockHashSyncStartCall) Return() *MockHashSyncStartCall { + c.Call = c.Call.Return() + return c +} + +// Do rewrite *gomock.Call.Do +func (c *MockHashSyncStartCall) Do(f func()) *MockHashSyncStartCall { + c.Call = c.Call.Do(f) + return c +} + +// DoAndReturn rewrite *gomock.Call.DoAndReturn +func (c *MockHashSyncStartCall) DoAndReturn(f func()) *MockHashSyncStartCall { + c.Call = c.Call.DoAndReturn(f) + return c +} + +// StartAndSync mocks base method. +func (m *MockHashSync) StartAndSync(ctx context.Context) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "StartAndSync", ctx) + ret0, _ := ret[0].(error) + return ret0 +} + +// StartAndSync indicates an expected call of StartAndSync. +func (mr *MockHashSyncMockRecorder) StartAndSync(ctx any) *MockHashSyncStartAndSyncCall { + mr.mock.ctrl.T.Helper() + call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StartAndSync", reflect.TypeOf((*MockHashSync)(nil).StartAndSync), ctx) + return &MockHashSyncStartAndSyncCall{Call: call} +} + +// MockHashSyncStartAndSyncCall wrap *gomock.Call +type MockHashSyncStartAndSyncCall struct { + *gomock.Call +} + +// Return rewrite *gomock.Call.Return +func (c *MockHashSyncStartAndSyncCall) Return(arg0 error) *MockHashSyncStartAndSyncCall { + c.Call = c.Call.Return(arg0) + return c +} + +// Do rewrite *gomock.Call.Do +func (c *MockHashSyncStartAndSyncCall) Do(f func(context.Context) error) *MockHashSyncStartAndSyncCall { + c.Call = c.Call.Do(f) + return c +} + +// DoAndReturn rewrite *gomock.Call.DoAndReturn +func (c *MockHashSyncStartAndSyncCall) DoAndReturn(f func(context.Context) error) *MockHashSyncStartAndSyncCall { + c.Call = c.Call.DoAndReturn(f) + return c +} + +// Stop mocks base method. +func (m *MockHashSync) Stop() { + m.ctrl.T.Helper() + m.ctrl.Call(m, "Stop") +} + +// Stop indicates an expected call of Stop. +func (mr *MockHashSyncMockRecorder) Stop() *MockHashSyncStopCall { + mr.mock.ctrl.T.Helper() + call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Stop", reflect.TypeOf((*MockHashSync)(nil).Stop)) + return &MockHashSyncStopCall{Call: call} +} + +// MockHashSyncStopCall wrap *gomock.Call +type MockHashSyncStopCall struct { + *gomock.Call +} + +// Return rewrite *gomock.Call.Return +func (c *MockHashSyncStopCall) Return() *MockHashSyncStopCall { + c.Call = c.Call.Return() + return c +} + +// Do rewrite *gomock.Call.Do +func (c *MockHashSyncStopCall) Do(f func()) *MockHashSyncStopCall { + c.Call = c.Call.Do(f) + return c +} + +// DoAndReturn rewrite *gomock.Call.DoAndReturn +func (c *MockHashSyncStopCall) DoAndReturn(f func()) *MockHashSyncStopCall { + c.Call = c.Call.DoAndReturn(f) + return c +} + +// MockHashSyncSource is a mock of HashSyncSource interface. +type MockHashSyncSource struct { + ctrl *gomock.Controller + recorder *MockHashSyncSourceMockRecorder + isgomock struct{} +} + +// MockHashSyncSourceMockRecorder is the mock recorder for MockHashSyncSource. +type MockHashSyncSourceMockRecorder struct { + mock *MockHashSyncSource +} + +// NewMockHashSyncSource creates a new mock instance. +func NewMockHashSyncSource(ctrl *gomock.Controller) *MockHashSyncSource { + mock := &MockHashSyncSource{ctrl: ctrl} + mock.recorder = &MockHashSyncSourceMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockHashSyncSource) EXPECT() *MockHashSyncSourceMockRecorder { + return m.recorder +} + +// CreateHashSync mocks base method. +func (m *MockHashSyncSource) CreateHashSync(name string, cfg sync2.Config, epoch types.EpochID) sync2.HashSync { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "CreateHashSync", name, cfg, epoch) + ret0, _ := ret[0].(sync2.HashSync) + return ret0 +} + +// CreateHashSync indicates an expected call of CreateHashSync. +func (mr *MockHashSyncSourceMockRecorder) CreateHashSync(name, cfg, epoch any) *MockHashSyncSourceCreateHashSyncCall { + mr.mock.ctrl.T.Helper() + call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CreateHashSync", reflect.TypeOf((*MockHashSyncSource)(nil).CreateHashSync), name, cfg, epoch) + return &MockHashSyncSourceCreateHashSyncCall{Call: call} +} + +// MockHashSyncSourceCreateHashSyncCall wrap *gomock.Call +type MockHashSyncSourceCreateHashSyncCall struct { + *gomock.Call +} + +// Return rewrite *gomock.Call.Return +func (c *MockHashSyncSourceCreateHashSyncCall) Return(arg0 sync2.HashSync) *MockHashSyncSourceCreateHashSyncCall { + c.Call = c.Call.Return(arg0) + return c +} + +// Do rewrite *gomock.Call.Do +func (c *MockHashSyncSourceCreateHashSyncCall) Do(f func(string, sync2.Config, types.EpochID) sync2.HashSync) *MockHashSyncSourceCreateHashSyncCall { + c.Call = c.Call.Do(f) + return c +} + +// DoAndReturn rewrite *gomock.Call.DoAndReturn +func (c *MockHashSyncSourceCreateHashSyncCall) DoAndReturn(f func(string, sync2.Config, types.EpochID) sync2.HashSync) *MockHashSyncSourceCreateHashSyncCall { + c.Call = c.Call.DoAndReturn(f) + return c +} + +// MockLayerTicker is a mock of LayerTicker interface. +type MockLayerTicker struct { + ctrl *gomock.Controller + recorder *MockLayerTickerMockRecorder + isgomock struct{} +} + +// MockLayerTickerMockRecorder is the mock recorder for MockLayerTicker. +type MockLayerTickerMockRecorder struct { + mock *MockLayerTicker +} + +// NewMockLayerTicker creates a new mock instance. +func NewMockLayerTicker(ctrl *gomock.Controller) *MockLayerTicker { + mock := &MockLayerTicker{ctrl: ctrl} + mock.recorder = &MockLayerTickerMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockLayerTicker) EXPECT() *MockLayerTickerMockRecorder { + return m.recorder +} + +// CurrentLayer mocks base method. +func (m *MockLayerTicker) CurrentLayer() types.LayerID { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "CurrentLayer") + ret0, _ := ret[0].(types.LayerID) + return ret0 +} + +// CurrentLayer indicates an expected call of CurrentLayer. +func (mr *MockLayerTickerMockRecorder) CurrentLayer() *MockLayerTickerCurrentLayerCall { + mr.mock.ctrl.T.Helper() + call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CurrentLayer", reflect.TypeOf((*MockLayerTicker)(nil).CurrentLayer)) + return &MockLayerTickerCurrentLayerCall{Call: call} +} + +// MockLayerTickerCurrentLayerCall wrap *gomock.Call +type MockLayerTickerCurrentLayerCall struct { + *gomock.Call +} + +// Return rewrite *gomock.Call.Return +func (c *MockLayerTickerCurrentLayerCall) Return(arg0 types.LayerID) *MockLayerTickerCurrentLayerCall { + c.Call = c.Call.Return(arg0) + return c +} + +// Do rewrite *gomock.Call.Do +func (c *MockLayerTickerCurrentLayerCall) Do(f func() types.LayerID) *MockLayerTickerCurrentLayerCall { + c.Call = c.Call.Do(f) + return c +} + +// DoAndReturn rewrite *gomock.Call.DoAndReturn +func (c *MockLayerTickerCurrentLayerCall) DoAndReturn(f func() types.LayerID) *MockLayerTickerCurrentLayerCall { + c.Call = c.Call.DoAndReturn(f) + return c +} diff --git a/syncer/interface.go b/syncer/interface.go index 6c9d2244d9..e1d9c5205c 100644 --- a/syncer/interface.go +++ b/syncer/interface.go @@ -73,3 +73,8 @@ type forkFinder interface { FindFork(context.Context, p2p.Peer, types.LayerID, types.Hash32) (types.LayerID, error) Purge(bool, ...p2p.Peer) } + +type multiEpochAtxSyncerV2 interface { + EnsureSync(ctx context.Context, lastWaitEpoch, newEpoch types.EpochID) (lastSynced types.EpochID, err error) + Stop() +} diff --git a/syncer/mocks/mocks.go b/syncer/mocks/mocks.go index 5cf1f95dd6..7980900806 100644 --- a/syncer/mocks/mocks.go +++ b/syncer/mocks/mocks.go @@ -1681,3 +1681,102 @@ func (c *MockforkFinderUpdateAgreementCall) DoAndReturn(f func(p2p.Peer, types.L c.Call = c.Call.DoAndReturn(f) return c } + +// MockmultiEpochAtxSyncerV2 is a mock of multiEpochAtxSyncerV2 interface. +type MockmultiEpochAtxSyncerV2 struct { + ctrl *gomock.Controller + recorder *MockmultiEpochAtxSyncerV2MockRecorder + isgomock struct{} +} + +// MockmultiEpochAtxSyncerV2MockRecorder is the mock recorder for MockmultiEpochAtxSyncerV2. +type MockmultiEpochAtxSyncerV2MockRecorder struct { + mock *MockmultiEpochAtxSyncerV2 +} + +// NewMockmultiEpochAtxSyncerV2 creates a new mock instance. +func NewMockmultiEpochAtxSyncerV2(ctrl *gomock.Controller) *MockmultiEpochAtxSyncerV2 { + mock := &MockmultiEpochAtxSyncerV2{ctrl: ctrl} + mock.recorder = &MockmultiEpochAtxSyncerV2MockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockmultiEpochAtxSyncerV2) EXPECT() *MockmultiEpochAtxSyncerV2MockRecorder { + return m.recorder +} + +// EnsureSync mocks base method. +func (m *MockmultiEpochAtxSyncerV2) EnsureSync(ctx context.Context, lastWaitEpoch, newEpoch types.EpochID) (types.EpochID, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "EnsureSync", ctx, lastWaitEpoch, newEpoch) + ret0, _ := ret[0].(types.EpochID) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// EnsureSync indicates an expected call of EnsureSync. +func (mr *MockmultiEpochAtxSyncerV2MockRecorder) EnsureSync(ctx, lastWaitEpoch, newEpoch any) *MockmultiEpochAtxSyncerV2EnsureSyncCall { + mr.mock.ctrl.T.Helper() + call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "EnsureSync", reflect.TypeOf((*MockmultiEpochAtxSyncerV2)(nil).EnsureSync), ctx, lastWaitEpoch, newEpoch) + return &MockmultiEpochAtxSyncerV2EnsureSyncCall{Call: call} +} + +// MockmultiEpochAtxSyncerV2EnsureSyncCall wrap *gomock.Call +type MockmultiEpochAtxSyncerV2EnsureSyncCall struct { + *gomock.Call +} + +// Return rewrite *gomock.Call.Return +func (c *MockmultiEpochAtxSyncerV2EnsureSyncCall) Return(lastSynced types.EpochID, err error) *MockmultiEpochAtxSyncerV2EnsureSyncCall { + c.Call = c.Call.Return(lastSynced, err) + return c +} + +// Do rewrite *gomock.Call.Do +func (c *MockmultiEpochAtxSyncerV2EnsureSyncCall) Do(f func(context.Context, types.EpochID, types.EpochID) (types.EpochID, error)) *MockmultiEpochAtxSyncerV2EnsureSyncCall { + c.Call = c.Call.Do(f) + return c +} + +// DoAndReturn rewrite *gomock.Call.DoAndReturn +func (c *MockmultiEpochAtxSyncerV2EnsureSyncCall) DoAndReturn(f func(context.Context, types.EpochID, types.EpochID) (types.EpochID, error)) *MockmultiEpochAtxSyncerV2EnsureSyncCall { + c.Call = c.Call.DoAndReturn(f) + return c +} + +// Stop mocks base method. +func (m *MockmultiEpochAtxSyncerV2) Stop() { + m.ctrl.T.Helper() + m.ctrl.Call(m, "Stop") +} + +// Stop indicates an expected call of Stop. +func (mr *MockmultiEpochAtxSyncerV2MockRecorder) Stop() *MockmultiEpochAtxSyncerV2StopCall { + mr.mock.ctrl.T.Helper() + call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Stop", reflect.TypeOf((*MockmultiEpochAtxSyncerV2)(nil).Stop)) + return &MockmultiEpochAtxSyncerV2StopCall{Call: call} +} + +// MockmultiEpochAtxSyncerV2StopCall wrap *gomock.Call +type MockmultiEpochAtxSyncerV2StopCall struct { + *gomock.Call +} + +// Return rewrite *gomock.Call.Return +func (c *MockmultiEpochAtxSyncerV2StopCall) Return() *MockmultiEpochAtxSyncerV2StopCall { + c.Call = c.Call.Return() + return c +} + +// Do rewrite *gomock.Call.Do +func (c *MockmultiEpochAtxSyncerV2StopCall) Do(f func()) *MockmultiEpochAtxSyncerV2StopCall { + c.Call = c.Call.Do(f) + return c +} + +// DoAndReturn rewrite *gomock.Call.DoAndReturn +func (c *MockmultiEpochAtxSyncerV2StopCall) DoAndReturn(f func()) *MockmultiEpochAtxSyncerV2StopCall { + c.Call = c.Call.DoAndReturn(f) + return c +} diff --git a/syncer/syncer.go b/syncer/syncer.go index ff52123950..6f4e80a979 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -18,6 +18,9 @@ import ( "github.com/spacemeshos/go-spacemesh/log" "github.com/spacemeshos/go-spacemesh/mesh" "github.com/spacemeshos/go-spacemesh/p2p" + "github.com/spacemeshos/go-spacemesh/sql" + "github.com/spacemeshos/go-spacemesh/sync2" + "github.com/spacemeshos/go-spacemesh/sync2/rangesync" "github.com/spacemeshos/go-spacemesh/syncer/atxsync" "github.com/spacemeshos/go-spacemesh/syncer/malsync" "github.com/spacemeshos/go-spacemesh/system" @@ -39,10 +42,25 @@ type Config struct { OutOfSyncThresholdLayers uint32 `mapstructure:"out-of-sync-threshold"` AtxSync atxsync.Config `mapstructure:"atx-sync"` MalSync malsync.Config `mapstructure:"malfeasance-sync"` + V2 SyncV2Config `mapstructure:"v2"` +} + +type SyncV2Config struct { + Enable bool `mapstructure:"enable"` + EnableActiveSync bool `mapstructure:"enable-active-sync"` + OldAtxSyncCfg sync2.Config `mapstructure:"old-atx-sync"` + NewAtxSyncCfg sync2.Config `mapstructure:"new-atx-sync"` + ParallelLoadLimit int `mapstructure:"parallel-load-limit"` } // DefaultConfig for the syncer. func DefaultConfig() Config { + oldAtxSyncCfg := sync2.DefaultConfig() + oldAtxSyncCfg.MaxDepth = 16 + oldAtxSyncCfg.MultiPeerReconcilerConfig.SyncInterval = time.Hour + newAtxSyncCfg := sync2.DefaultConfig() + newAtxSyncCfg.MaxDepth = 21 + newAtxSyncCfg.MultiPeerReconcilerConfig.SyncInterval = 5 * time.Minute return Config{ Interval: 10 * time.Second, EpochEndFraction: 0.5, @@ -54,6 +72,13 @@ func DefaultConfig() Config { OutOfSyncThresholdLayers: 3, AtxSync: atxsync.DefaultConfig(), MalSync: malsync.DefaultConfig(), + V2: SyncV2Config{ + Enable: false, + EnableActiveSync: false, + OldAtxSyncCfg: oldAtxSyncCfg, + NewAtxSyncCfg: newAtxSyncCfg, + ParallelLoadLimit: 10, + }, } } @@ -119,6 +144,12 @@ func withForkFinder(f forkFinder) Option { } } +func withAtxSyncerV2(asv2 multiEpochAtxSyncerV2) Option { + return func(s *Syncer) { + s.asv2 = asv2 + } +} + // Syncer is responsible to keep the node in sync with the network. type Syncer struct { logger *zap.Logger @@ -162,6 +193,9 @@ type Syncer struct { eg errgroup.Group stop context.CancelFunc + + asv2 multiEpochAtxSyncerV2 + dispatcher *rangesync.Dispatcher } // NewSyncer creates a new Syncer instance. @@ -207,6 +241,15 @@ func NewSyncer( s.isBusy.Store(false) s.lastLayerSynced.Store(s.mesh.LatestLayer().Uint32()) s.lastEpochSynced.Store(types.GetEffectiveGenesis().GetEpoch().Uint32() - 1) + if s.cfg.V2.Enable && s.asv2 == nil { + s.dispatcher = sync2.NewDispatcher(s.logger, fetcher.(sync2.Fetcher)) + hss := sync2.NewATXSyncSource( + s.logger, s.dispatcher, cdb.Database.(sql.StateDatabase), + fetcher.(sync2.Fetcher), s.cfg.V2.EnableActiveSync) + s.asv2 = sync2.NewMultiEpochATXSyncer( + s.logger, hss, s.cfg.V2.OldAtxSyncCfg, s.cfg.V2.NewAtxSyncCfg, + s.cfg.V2.ParallelLoadLimit) + } return s } @@ -218,6 +261,9 @@ func (s *Syncer) Close() { s.stop() s.logger.Debug("waiting for syncer goroutines to finish") err := s.eg.Wait() + if s.asv2 != nil { + s.asv2.Stop() + } s.logger.Debug("all syncer goroutines finished", zap.Error(err)) } @@ -251,7 +297,13 @@ func (s *Syncer) Start() { s.syncOnce.Do(func() { ctx, cancel := context.WithCancel(context.Background()) s.stop = cancel + s.logger.Info("starting syncer loop", log.ZContext(ctx)) + if s.dispatcher != nil { + s.eg.Go(func() error { + return s.dispatcher.Server.Run(ctx) + }) + } s.eg.Go(func() error { if s.ticker.CurrentLayer() <= types.GetEffectiveGenesis() { s.setSyncState(ctx, synced) @@ -413,7 +465,7 @@ func (s *Syncer) synchronize(ctx context.Context) bool { return false } - if err := s.syncAtx(ctx); err != nil { + if err := s.syncAtxAndMalfeasance(ctx); err != nil { if !errors.Is(err, context.Canceled) { s.logger.Error("failed to sync atxs", log.ZContext(ctx), zap.Error(err)) } @@ -423,6 +475,7 @@ func (s *Syncer) synchronize(ctx context.Context) bool { if s.ticker.CurrentLayer() <= types.GetEffectiveGenesis() { return true } + // always sync to currentLayer-1 to reduce race with gossip and hare/tortoise for layer := s.getLastSyncedLayer().Add(1); layer.Before(s.ticker.CurrentLayer()); layer = layer.Add(1) { if err := s.syncLayer(ctx, layer); err != nil { @@ -471,8 +524,18 @@ func (s *Syncer) synchronize(ctx context.Context) bool { return success } -func (s *Syncer) syncAtx(ctx context.Context) error { +func (s *Syncer) ensureATXsInSync(ctx context.Context) error { current := s.ticker.CurrentLayer() + publish := current.GetEpoch() + if publish == 0 { + return nil // nothing to sync in epoch 0 + } + + // if we are not advanced enough sync previous epoch, otherwise start syncing activations published in this epoch + if current.OrdinalInEpoch() <= uint32(float64(types.GetLayersPerEpoch())*s.cfg.EpochEndFraction) { + publish -= 1 + } + // on startup always download all activations that were published before current epoch if !s.ListenToATXGossip() { s.logger.Debug("syncing atx from genesis", @@ -486,31 +549,15 @@ func (s *Syncer) syncAtx(ctx context.Context) error { } } s.logger.Debug("atxs synced to epoch", log.ZContext(ctx), zap.Stringer("last epoch", s.lastAtxEpoch())) - - // FIXME https://github.com/spacemeshos/go-spacemesh/issues/3987 - s.logger.Info("syncing malicious proofs", log.ZContext(ctx)) - if err := s.syncMalfeasance(ctx, current.GetEpoch()); err != nil { - return err - } - s.logger.Info("malicious IDs synced", log.ZContext(ctx)) - s.setATXSynced() - } - - publish := current.GetEpoch() - if publish == 0 { - return nil // nothing to sync in epoch 0 } - // if we are not advanced enough sync previous epoch, otherwise start syncing activations published in this epoch - if current.OrdinalInEpoch() <= uint32(float64(types.GetLayersPerEpoch())*s.cfg.EpochEndFraction) { - publish -= 1 - } if epoch := s.backgroundSync.epoch.Load(); epoch != 0 && epoch != publish.Uint32() { s.backgroundSync.cancel() s.backgroundSync.eg.Wait() s.backgroundSync.epoch.Store(0) } if s.backgroundSync.epoch.Load() == 0 && publish.Uint32() != 0 { + // TODO: syncv2 s.logger.Debug("download atx for epoch in background", zap.Stringer("publish", publish), log.ZContext(ctx)) s.backgroundSync.epoch.Store(publish.Uint32()) ctx, cancel := context.WithCancel(ctx) @@ -533,7 +580,75 @@ func (s *Syncer) syncAtx(ctx context.Context) error { return err }) } - if !s.malSync.started { + return nil +} + +// ensureATXsInSyncV2 ensures that the ATXs are in sync and being synchronized +// continuously using syncv2. +func (s *Syncer) ensureATXsInSyncV2(ctx context.Context) error { + current := s.ticker.CurrentLayer() + currentEpoch := current.GetEpoch() + if currentEpoch == 0 { + return nil // nothing to sync in epoch 0 + } + publish := currentEpoch + if current.OrdinalInEpoch() <= uint32(float64(types.GetLayersPerEpoch())*s.cfg.EpochEndFraction) { + publish-- + } + + if !s.ListenToATXGossip() && s.cfg.V2.EnableActiveSync { + // ATXs are not in sync yet, to we need to sync them synchronously + lastWaitEpoch := types.EpochID(0) + if currentEpoch > 1 { + lastWaitEpoch = currentEpoch - 1 + } + s.logger.Debug("syncing atx from genesis", + log.ZContext(ctx), + zap.Stringer("current layer", current), + zap.Stringer("last synced epoch", s.lastAtxEpoch()), + zap.Stringer("lastWaitEpoch", lastWaitEpoch), + zap.Stringer("publish", publish), + ) + lastAtxEpoch, err := s.asv2.EnsureSync(ctx, lastWaitEpoch, publish) + if lastAtxEpoch > 0 { + s.setLastAtxEpoch(lastAtxEpoch) + } + if err != nil { + return fmt.Errorf("syncing atxs: %w", err) + } + s.logger.Debug("atxs synced to epoch", + log.ZContext(ctx), zap.Stringer("last epoch", s.lastAtxEpoch())) + return nil + } + + // When active syncv2 is not enabled, this will only cause the per-epoch sync + // servers (multiplexed via dispatcher) to be activated, without attempting to + // initiate sync against the peers + s.logger.Debug("activating sync2", zap.Uint32("new epoch", publish.Uint32())) + if _, err := s.asv2.EnsureSync(ctx, 0, publish); err != nil { + return fmt.Errorf("activating sync: %w", err) + } + + return nil +} + +func (s *Syncer) ensureMalfeasanceInSync(ctx context.Context) error { + // TODO: use syncv2 for malfeasance proofs: + // https://github.com/spacemeshos/go-spacemesh/issues/3987 + current := s.ticker.CurrentLayer() + if !s.ListenToATXGossip() { + s.logger.Info("syncing malicious proofs", log.ZContext(ctx)) + if err := s.syncMalfeasance(ctx, current.GetEpoch()); err != nil { + return err + } + s.logger.Info("malicious IDs synced", log.ZContext(ctx)) + // Malfeasance proofs are synced after the actual ATXs. + // We set ATX synced status after both ATXs and malfeascance proofs + // are in sync. + s.setATXSynced() + } + + if current.GetEpoch() > 0 && !s.malSync.started { s.malSync.started = true s.malSync.eg.Go(func() error { select { @@ -548,9 +663,26 @@ func (s *Syncer) syncAtx(ctx context.Context) error { } }) } + return nil } +func (s *Syncer) syncAtxAndMalfeasance(ctx context.Context) error { + if s.cfg.V2.Enable { + if err := s.ensureATXsInSyncV2(ctx); err != nil { + return err + } + } + if !s.cfg.V2.Enable || !s.cfg.V2.EnableActiveSync { + // If syncv2 is being used in server-only mode, we still need to run + // active syncv1. + if err := s.ensureATXsInSync(ctx); err != nil { + return err + } + } + return s.ensureMalfeasanceInSync(ctx) +} + func isTooFarBehind( ctx context.Context, logger *zap.Logger, diff --git a/syncer/syncer_test.go b/syncer/syncer_test.go index cb74d04344..1b979ab89a 100644 --- a/syncer/syncer_test.go +++ b/syncer/syncer_test.go @@ -82,6 +82,7 @@ type testSyncer struct { mTortoise *smocks.MockTortoise mCertHdr *mocks.MockcertHandler mForkFinder *mocks.MockforkFinder + mASV2 *mocks.MockmultiEpochAtxSyncerV2 } func (ts *testSyncer) expectMalEnsureInSync(current types.LayerID) { @@ -92,7 +93,7 @@ func (ts *testSyncer) expectMalEnsureInSync(current types.LayerID) { ) } -func (ts *testSyncer) expectDownloadLoop() chan struct{} { +func (ts *testSyncer) expectMalDownloadLoop() chan struct{} { ch := make(chan struct{}) ts.mMalSyncer.EXPECT().DownloadLoop(gomock.Any()). DoAndReturn(func(context.Context) error { @@ -109,7 +110,7 @@ func (ts *testSyncer) expectDownloadLoop() chan struct{} { return ch } -func newTestSyncer(tb testing.TB, interval time.Duration) *testSyncer { +func newTestSyncerWithConfig(tb testing.TB, cfg Config) *testSyncer { lg := zaptest.NewLogger(tb) mt := newMockLayerTicker() ctrl := gomock.NewController(tb) @@ -127,6 +128,7 @@ func newTestSyncer(tb testing.TB, interval time.Duration) *testSyncer { mTortoise: smocks.NewMockTortoise(ctrl), mCertHdr: mocks.NewMockcertHandler(ctrl), mForkFinder: mocks.NewMockforkFinder(ctrl), + mASV2: mocks.NewMockmultiEpochAtxSyncerV2(ctrl), } db := statesql.InMemoryTest(tb) ts.cdb = datastore.NewCachedDB(db, lg) @@ -137,14 +139,6 @@ func newTestSyncer(tb testing.TB, interval time.Duration) *testSyncer { ts.msh, err = mesh.NewMesh(db, atxsdata, ts.mTortoise, exec, ts.mConState, lg) require.NoError(tb, err) - cfg := Config{ - Interval: interval, - GossipDuration: 5 * time.Millisecond, - EpochEndFraction: 0.66, - SyncCertDistance: 4, - HareDelayLayers: 5, - OutOfSyncThresholdLayers: outOfSyncThreshold, - } ts.syncer = NewSyncer( ts.cdb, ts.mTicker, @@ -160,16 +154,39 @@ func newTestSyncer(tb testing.TB, interval time.Duration) *testSyncer { WithLogger(lg), withDataFetcher(ts.mDataFetcher), withForkFinder(ts.mForkFinder), + withAtxSyncerV2(ts.mASV2), ) return ts } +func defaultTestConfig(interval time.Duration) Config { + return Config{ + Interval: interval, + GossipDuration: 5 * time.Millisecond, + EpochEndFraction: 0.66, + SyncCertDistance: 4, + HareDelayLayers: 5, + OutOfSyncThresholdLayers: outOfSyncThreshold, + } +} + +func newTestSyncer(tb testing.TB, interval time.Duration) *testSyncer { + return newTestSyncerWithConfig(tb, defaultTestConfig(interval)) +} + func newSyncerWithoutPeriodicRuns(tb testing.TB) *testSyncer { ts := newTestSyncer(tb, never) ts.mDataFetcher.EXPECT().SelectBestShuffled(gomock.Any()).Return([]p2p.Peer{"non-empty"}).AnyTimes() return ts } +func newSyncerWithoutPeriodicRunsWithConfig(tb testing.TB, cfg Config) *testSyncer { + cfg.Interval = never + ts := newTestSyncerWithConfig(tb, cfg) + ts.mDataFetcher.EXPECT().SelectBestShuffled(gomock.Any()).Return([]p2p.Peer{"non-empty"}).AnyTimes() + return ts +} + func newTestSyncerForState(tb testing.TB) *testSyncer { ts := newTestSyncer(tb, never) return ts @@ -194,6 +211,7 @@ func TestStartAndShutdown(t *testing.T) { ts.syncer.IsSynced(ctx) }, time.Second, 10*time.Millisecond) + ts.mASV2.EXPECT().Stop() cancel() require.False(t, ts.syncer.synchronize(ctx)) ts.syncer.Close() @@ -210,7 +228,7 @@ func TestSynchronize_OnlyOneSynchronize(t *testing.T) { ts.mTicker.advanceToLayer(current) ctx, cancel := context.WithCancel(context.Background()) defer cancel() - dlCh := ts.expectDownloadLoop() + dlCh := ts.expectMalDownloadLoop() ts.syncer.Start() ts.mAtxSyncer.EXPECT().Download(gomock.Any(), gomock.Any(), gomock.Any()).AnyTimes() @@ -243,6 +261,7 @@ func TestSynchronize_OnlyOneSynchronize(t *testing.T) { require.NoError(t, eg.Wait()) <-dlCh + ts.mASV2.EXPECT().Stop() cancel() ts.syncer.Close() } @@ -271,7 +290,7 @@ func advanceState(tb testing.TB, ts *testSyncer, from, to types.LayerID) { func TestSynchronize_AllGood(t *testing.T) { ts := newSyncerWithoutPeriodicRuns(t) - ts.expectDownloadLoop() + ts.expectMalDownloadLoop() gLayer := types.GetEffectiveGenesis() current1 := gLayer.Add(10) ts.mTicker.advanceToLayer(current1) @@ -346,7 +365,7 @@ func TestSynchronize_AllGood(t *testing.T) { func TestSynchronize_FetchLayerDataFailed(t *testing.T) { ts := newSyncerWithoutPeriodicRuns(t) - ts.expectDownloadLoop() + ts.expectMalDownloadLoop() gLayer := types.GetEffectiveGenesis() current := gLayer.Add(2) ts.mTicker.advanceToLayer(current) @@ -459,7 +478,7 @@ func TestSyncAtxs_Genesis(t *testing.T) { }) t.Run("first atx epoch", func(t *testing.T) { ts := newSyncerWithoutPeriodicRuns(t) - ts.expectDownloadLoop() + ts.expectMalDownloadLoop() epoch := types.EpochID(1) current := epoch.FirstLayer() + 2 ts.mTicker.advanceToLayer(current) // to pass epoch end fraction threshold @@ -487,6 +506,33 @@ func TestSyncAtxs_Genesis(t *testing.T) { }) } +func TestSyncAtxs_Genesis_SyncV2(t *testing.T) { + cfg := defaultTestConfig(never) + cfg.V2.Enable = true + cfg.V2.EnableActiveSync = true + + t.Run("no atx expected", func(t *testing.T) { + ts := newSyncerWithoutPeriodicRunsWithConfig(t, cfg) + ts.mTicker.advanceToLayer(1) + require.True(t, ts.syncer.synchronize(context.Background())) + require.True(t, ts.syncer.ListenToATXGossip()) + require.Equal(t, types.EpochID(0), ts.syncer.lastAtxEpoch()) + }) + + t.Run("first atx epoch", func(t *testing.T) { + ts := newSyncerWithoutPeriodicRunsWithConfig(t, cfg) + ts.expectMalDownloadLoop() + epoch := types.EpochID(1) + current := epoch.FirstLayer() + 2 + ts.mTicker.advanceToLayer(current) // to pass epoch end fraction threshold + require.False(t, ts.syncer.ListenToATXGossip()) + ts.mASV2.EXPECT().EnsureSync(gomock.Any(), types.EpochID(0), epoch) + ts.expectMalEnsureInSync(current) + require.True(t, ts.syncer.synchronize(context.Background())) + require.True(t, ts.syncer.ListenToATXGossip()) + }) +} + func TestSyncAtxs(t *testing.T) { tcs := []struct { desc string @@ -507,7 +553,7 @@ func TestSyncAtxs(t *testing.T) { for _, tc := range tcs { t.Run(tc.desc, func(t *testing.T) { ts := newSyncerWithoutPeriodicRuns(t) - ts.expectDownloadLoop() + ts.expectMalDownloadLoop() lyr := startWithSyncedState(t, ts) require.LessOrEqual(t, lyr, tc.current) ts.mTicker.advanceToLayer(tc.current) @@ -523,9 +569,72 @@ func TestSyncAtxs(t *testing.T) { } } +func startWithSyncedState_SyncV2(tb testing.TB, ts *testSyncer) types.LayerID { + tb.Helper() + + gLayer := types.GetEffectiveGenesis() + ts.mTicker.advanceToLayer(gLayer) + ts.expectMalEnsureInSync(gLayer) + ts.mASV2.EXPECT().EnsureSync(gomock.Any(), types.EpochID(0), types.EpochID(1)).MinTimes(1) + // ts.mAtxSyncer.EXPECT().Download(gomock.Any(), gLayer.GetEpoch(), gomock.Any()) + require.True(tb, ts.syncer.synchronize(context.Background())) + ts.syncer.waitBackgroundSync() + require.True(tb, ts.syncer.ListenToATXGossip()) + require.True(tb, ts.syncer.ListenToGossip()) + require.True(tb, ts.syncer.IsSynced(context.Background())) + + current := gLayer.Add(2) + ts.mTicker.advanceToLayer(current) + lyr := current.Sub(1) + ts.mDataFetcher.EXPECT().PollLayerData(gomock.Any(), lyr) + + require.True(tb, ts.syncer.synchronize(context.Background())) + require.True(tb, ts.syncer.ListenToATXGossip()) + require.True(tb, ts.syncer.ListenToGossip()) + require.True(tb, ts.syncer.IsSynced(context.Background())) + return current +} + +func TestSyncAtxs_SyncV2(t *testing.T) { + cfg := defaultTestConfig(never) + cfg.V2.Enable = true + cfg.V2.EnableActiveSync = true + tcs := []struct { + desc string + current types.LayerID + downloaded types.EpochID + }{ + { + desc: "start of epoch", + current: 13, + downloaded: 3, + }, + { + desc: "end of epoch", + current: 14, + downloaded: 4, + }, + } + for _, tc := range tcs { + t.Run(tc.desc, func(t *testing.T) { + ts := newSyncerWithoutPeriodicRunsWithConfig(t, cfg) + ts.expectMalDownloadLoop() + lyr := startWithSyncedState_SyncV2(t, ts) + require.LessOrEqual(t, lyr, tc.current) + ts.mTicker.advanceToLayer(tc.current) + + ts.mASV2.EXPECT().EnsureSync(gomock.Any(), types.EpochID(0), tc.downloaded) + for lid := lyr; lid < tc.current; lid++ { + ts.mDataFetcher.EXPECT().PollLayerData(gomock.Any(), lid) + } + require.True(t, ts.syncer.synchronize(context.Background())) + }) + } +} + func TestSynchronize_StaySyncedUponFailure(t *testing.T) { ts := newSyncerWithoutPeriodicRuns(t) - ts.expectDownloadLoop() + ts.expectMalDownloadLoop() lyr := startWithSyncedState(t, ts) current := lyr.Add(1) ts.mTicker.advanceToLayer(current) @@ -542,7 +651,7 @@ func TestSynchronize_StaySyncedUponFailure(t *testing.T) { func TestSynchronize_BecomeNotSyncedUponFailureIfNoGossip(t *testing.T) { ts := newSyncerWithoutPeriodicRuns(t) - ts.expectDownloadLoop() + ts.expectMalDownloadLoop() lyr := startWithSyncedState(t, ts) current := lyr.Add(outOfSyncThreshold) ts.mTicker.advanceToLayer(current) @@ -561,7 +670,7 @@ func TestSynchronize_BecomeNotSyncedUponFailureIfNoGossip(t *testing.T) { // test the case where the node originally starts from notSynced and eventually becomes synced. func TestFromNotSyncedToSynced(t *testing.T) { ts := newSyncerWithoutPeriodicRuns(t) - ts.expectDownloadLoop() + ts.expectMalDownloadLoop() ts.mAtxSyncer.EXPECT().Download(gomock.Any(), gomock.Any(), gomock.Any()).AnyTimes() lyr := types.GetEffectiveGenesis().Add(1) current := lyr.Add(5) @@ -596,7 +705,7 @@ func TestFromNotSyncedToSynced(t *testing.T) { // to notSynced. func TestFromGossipSyncToNotSynced(t *testing.T) { ts := newSyncerWithoutPeriodicRuns(t) - ts.expectDownloadLoop() + ts.expectMalDownloadLoop() ts.mAtxSyncer.EXPECT().Download(gomock.Any(), gomock.Any(), gomock.Any()).AnyTimes() lyr := types.GetEffectiveGenesis().Add(1) current := lyr.Add(1) @@ -628,7 +737,7 @@ func TestFromGossipSyncToNotSynced(t *testing.T) { func TestNetworkHasNoData(t *testing.T) { ts := newSyncerWithoutPeriodicRuns(t) - ts.expectDownloadLoop() + ts.expectMalDownloadLoop() lyr := startWithSyncedState(t, ts) require.True(t, ts.syncer.IsSynced(context.Background())) @@ -654,7 +763,7 @@ func TestNetworkHasNoData(t *testing.T) { // eventually become synced again. func TestFromSyncedToNotSynced(t *testing.T) { ts := newSyncerWithoutPeriodicRuns(t) - ts.expectDownloadLoop() + ts.expectMalDownloadLoop() ts.mAtxSyncer.EXPECT().Download(gomock.Any(), gomock.Any(), gomock.Any()).AnyTimes() require.True(t, ts.syncer.synchronize(context.Background())) @@ -705,7 +814,7 @@ func waitOutGossipSync(tb testing.TB, ts *testSyncer) { func TestSync_AlsoSyncProcessedLayer(t *testing.T) { ts := newSyncerWithoutPeriodicRuns(t) - ts.expectDownloadLoop() + ts.expectMalDownloadLoop() ts.mAtxSyncer.EXPECT().Download(gomock.Any(), gomock.Any(), gomock.Any()).AnyTimes() lyr := types.GetEffectiveGenesis().Add(1) current := lyr.Add(1) @@ -766,7 +875,7 @@ func TestSyncer_IsBeaconSynced(t *testing.T) { func TestSynchronize_RecoverFromCheckpoint(t *testing.T) { ts := newSyncerWithoutPeriodicRuns(t) - ts.expectDownloadLoop() + ts.expectMalDownloadLoop() current := types.GetEffectiveGenesis().Add(types.GetLayersPerEpoch() * 5) // recover from a checkpoint types.SetEffectiveGenesis(current.Uint32()) @@ -786,6 +895,7 @@ func TestSynchronize_RecoverFromCheckpoint(t *testing.T) { WithLogger(ts.syncer.logger), withDataFetcher(ts.mDataFetcher), withForkFinder(ts.mForkFinder), + withAtxSyncerV2(ts.mASV2), ) // should not sync any atxs before current epoch ts.mAtxSyncer.EXPECT().Download(gomock.Any(), current.GetEpoch(), gomock.Any()) diff --git a/system/fetcher.go b/system/fetcher.go index dec2655835..61f2d5c451 100644 --- a/system/fetcher.go +++ b/system/fetcher.go @@ -28,16 +28,26 @@ type BlockFetcher interface { type GetAtxOpts struct { LimitingOff bool + RecvChannel chan<- types.ATXID } type GetAtxOpt func(*GetAtxOpts) +// WithoutLimiting disables rate limiting when downloading ATXs. func WithoutLimiting() GetAtxOpt { return func(opts *GetAtxOpts) { opts.LimitingOff = true } } +// WithRecvChannel sets the channel to receive successfully downloaded and validated ATXs +// IDs on. +func WithRecvChannel(ch chan<- types.ATXID) GetAtxOpt { + return func(opts *GetAtxOpts) { + opts.RecvChannel = ch + } +} + // AtxFetcher defines an interface for fetching ATXs from remote peers. type AtxFetcher interface { GetAtxs(context.Context, []types.ATXID, ...GetAtxOpt) error From ce73f548af3fb06947159d022dfaf53a70710c93 Mon Sep 17 00:00:00 2001 From: Ivan Shvedunov Date: Wed, 20 Nov 2024 19:05:11 +0400 Subject: [PATCH 2/6] sync2: use saner retry scheme for fetched ATXs --- fetch/limiter.go | 5 +++- fetch/mesh_data.go | 40 ++++++++++++++++++--------- fetch/mesh_data_test.go | 16 +++++------ sync2/atxs.go | 43 ++++++++++++----------------- sync2/atxs_test.go | 60 ++++++++++++++++++++++++++--------------- system/fetcher.go | 14 ++++++---- 6 files changed, 104 insertions(+), 74 deletions(-) diff --git a/fetch/limiter.go b/fetch/limiter.go index 3222433d5b..11993ddb38 100644 --- a/fetch/limiter.go +++ b/fetch/limiter.go @@ -2,6 +2,8 @@ package fetch import ( "context" + + "github.com/spacemeshos/go-spacemesh/common/types" ) type limiter interface { @@ -10,7 +12,8 @@ type limiter interface { } type getHashesOpts struct { - limiter limiter + limiter limiter + callback func(types.Hash32, error) } type noLimit struct{} diff --git a/fetch/mesh_data.go b/fetch/mesh_data.go index 28382808ba..0f2760dc89 100644 --- a/fetch/mesh_data.go +++ b/fetch/mesh_data.go @@ -42,19 +42,16 @@ func (f *Fetch) GetAtxs(ctx context.Context, ids []types.ATXID, opts ...system.G ) hashes := types.ATXIDsToHashes(ids) handler := f.validators.atx.HandleMessage - if options.RecvChannel != nil { - handler = func(ctx context.Context, id types.Hash32, p p2p.Peer, data []byte) error { - if err := f.validators.atx.HandleMessage(ctx, id, p, data); err != nil { - return err - } - options.RecvChannel <- types.ATXID(id) - return nil - } + var ghOpts []getHashesOpt + if !options.LimitingOff { + ghOpts = append(ghOpts, withLimiter(f.getAtxsLimiter)) } - if options.LimitingOff { - return f.getHashes(ctx, hashes, datastore.ATXDB, handler) + if options.Callback != nil { + ghOpts = append(ghOpts, withHashCallback(func(hash types.Hash32, err error) { + options.Callback(types.ATXID(hash), err) + })) } - return f.getHashes(ctx, hashes, datastore.ATXDB, handler, withLimiter(f.getAtxsLimiter)) + return f.getHashes(ctx, hashes, datastore.ATXDB, handler, ghOpts...) } type dataReceiver func(context.Context, types.Hash32, p2p.Peer, []byte) error @@ -67,6 +64,12 @@ func withLimiter(l limiter) getHashesOpt { } } +func withHashCallback(callback func(types.Hash32, error)) getHashesOpt { + return func(o *getHashesOpts) { + o.callback = callback + } +} + func (f *Fetch) getHashes( ctx context.Context, hashes []types.Hash32, @@ -75,7 +78,8 @@ func (f *Fetch) getHashes( opts ...getHashesOpt, ) error { options := getHashesOpts{ - limiter: noLimit{}, + limiter: noLimit{}, + callback: func(types.Hash32, error) {}, } for _, opt := range opts { opt(&options) @@ -92,18 +96,26 @@ func (f *Fetch) getHashes( for i, hash := range hashes { if err := options.limiter.Acquire(ctx, 1); err != nil { pendingMetric.Add(float64(i - len(hashes))) - return fmt.Errorf("acquiring slot to get hash: %w", err) + err = fmt.Errorf("acquiring slot to get hash: %w", err) + for _, h := range hashes[i:] { + options.callback(h, err) + } + return err } p, err := f.getHash(ctx, hash, hint, receiver) if err != nil { options.limiter.Release(1) pendingMetric.Add(float64(i - len(hashes))) + for _, h := range hashes[i:] { + options.callback(h, err) + } return err } if p == nil { // data is available locally options.limiter.Release(1) pendingMetric.Add(-1) + options.callback(hash, nil) continue } @@ -112,6 +124,7 @@ func (f *Fetch) getHashes( case <-ctx.Done(): options.limiter.Release(1) pendingMetric.Add(-1) + options.callback(hash, ctx.Err()) return ctx.Err() case <-p.completed: options.limiter.Release(1) @@ -127,6 +140,7 @@ func (f *Fetch) getHashes( bfailure.Add(hash, p.err) mu.Unlock() } + options.callback(hash, p.err) return nil } }) diff --git a/fetch/mesh_data_test.go b/fetch/mesh_data_test.go index 56b1b83e91..d5f277b65d 100644 --- a/fetch/mesh_data_test.go +++ b/fetch/mesh_data_test.go @@ -5,6 +5,7 @@ import ( "context" "errors" "fmt" + "sync" "testing" p2phost "github.com/libp2p/go-libp2p/core/host" @@ -605,17 +606,16 @@ func TestGetATXs(t *testing.T) { atxIDs1 := types.ToATXIDs(atxs[:2]) require.NoError(t, f.GetAtxs(context.Background(), atxIDs1)) - recvCh := make(chan types.ATXID) atxIDs2 := types.ToATXIDs(atxs[2:]) var recvIDs []types.ATXID - eg.Go(func() error { - for id := range recvCh { + var mtx sync.Mutex + require.NoError(t, f.GetAtxs(context.Background(), atxIDs2, + system.WithATXCallback(func(id types.ATXID, err error) { + mtx.Lock() + defer mtx.Unlock() + require.NoError(t, err) recvIDs = append(recvIDs, id) - } - return nil - }) - require.NoError(t, f.GetAtxs(context.Background(), atxIDs2, system.WithRecvChannel(recvCh))) - close(recvCh) + }))) close(stop) require.NoError(t, eg.Wait()) require.ElementsMatch(t, atxIDs2, recvIDs) diff --git a/sync2/atxs.go b/sync2/atxs.go index 11feba1399..45fec8d9b7 100644 --- a/sync2/atxs.go +++ b/sync2/atxs.go @@ -4,6 +4,7 @@ import ( "context" "errors" "fmt" + "sync" "time" "github.com/jonboulle/clockwork" @@ -95,7 +96,7 @@ func (h *ATXHandler) Commit(ctx context.Context, peer p2p.Peer, base, new ranges } total := len(state) items := make([]types.ATXID, 0, h.batchSize) - startTime := time.Now() + startTime := h.clock.Now() batchAttemptsRemaining := h.maxBatchRetries for len(state) > 0 { items = items[:0] @@ -115,39 +116,28 @@ func (h *ATXHandler) Commit(ctx context.Context, peer p2p.Peer, base, new ranges break } - var eg errgroup.Group - recvCh := make(chan types.ATXID) someSucceeded := false - eg.Go(func() error { - for id := range recvCh { + var mtx sync.Mutex + err := h.f.GetAtxs(ctx, items, system.WithATXCallback(func(id types.ATXID, err error) { + mtx.Lock() + defer mtx.Unlock() + switch { + case err == nil: numDownloaded++ someSucceeded = true delete(state, id) + case errors.Is(err, pubsub.ErrValidationReject): + // if the atx invalid there's no point downloading it again + state[id] = h.maxAttempts + default: + state[id]++ } - return nil - }) - err := h.f.GetAtxs(ctx, items, system.WithRecvChannel(recvCh)) - close(recvCh) - eg.Wait() + })) if err != nil { if errors.Is(err, context.Canceled) { return err } - batchError := &fetch.BatchError{} - if errors.As(err, &batchError) { - h.logger.Debug("QQQQQ: batch error", zap.Error(err)) - for hash, err := range batchError.Errors { - if _, exists := state[types.ATXID(hash)]; !exists { - continue - } - if errors.Is(err, pubsub.ErrValidationReject) { - // if the atx invalid there's no point downloading it again - state[types.ATXID(hash)] = h.maxAttempts - } else { - state[types.ATXID(hash)]++ - } - } - } else { + if !errors.Is(err, &fetch.BatchError{}) { h.logger.Debug("failed to download ATXs", zap.Error(err)) } } @@ -166,10 +156,11 @@ func (h *ATXHandler) Commit(ctx context.Context, peer p2p.Peer, base, new ranges } } else { batchAttemptsRemaining = h.maxBatchRetries + elapsed := h.clock.Since(startTime) h.logger.Debug("fetched atxs", zap.Int("total", total), zap.Int("downloaded", numDownloaded), - zap.Float64("rate per sec", float64(numDownloaded)/time.Since(startTime).Seconds())) + zap.Float64("rate per sec", float64(numDownloaded)/elapsed.Seconds())) } } return nil diff --git a/sync2/atxs_test.go b/sync2/atxs_test.go index 7dcc780ebe..ebfe3451a3 100644 --- a/sync2/atxs_test.go +++ b/sync2/atxs_test.go @@ -38,7 +38,8 @@ func TestAtxHandler_Success(t *testing.T) { allAtxs[i] = types.RandomATXID() } f := NewMockFetcher(ctrl) - h := sync2.NewATXHandler(logger, f, batchSize, maxAttempts, maxBatchRetries, batchRetryDelay, nil) + clock := clockwork.NewFakeClock() + h := sync2.NewATXHandler(logger, f, batchSize, maxAttempts, maxBatchRetries, batchRetryDelay, clock) baseSet := mocks.NewMockOrderedSet(ctrl) newSet := mocks.NewMockOrderedSet(ctrl) for _, id := range allAtxs { @@ -60,15 +61,11 @@ func TestAtxHandler_Success(t *testing.T) { for _, opt := range opts { opt(&atxOpts) } - require.NotNil(t, atxOpts.RecvChannel) + require.NotNil(t, atxOpts.Callback) for _, id := range atxs { require.True(t, toFetch[id], "already fetched or bad ID") delete(toFetch, id) - select { - case <-time.After(100 * time.Millisecond): - t.Error("timeout sending recvd id") - case atxOpts.RecvChannel <- id: - } + atxOpts.Callback(id, nil) } return nil }).Times(3) @@ -106,7 +103,8 @@ func TestAtxHandler_Retry(t *testing.T) { allAtxs[i] = types.RandomATXID() } f := NewMockFetcher(ctrl) - h := sync2.NewATXHandler(logger, f, batchSize, maxAttempts, maxBatchRetries, batchRetryDelay, nil) + clock := clockwork.NewFakeClock() + h := sync2.NewATXHandler(logger, f, batchSize, maxAttempts, maxBatchRetries, batchRetryDelay, clock) baseSet := mocks.NewMockOrderedSet(ctrl) newSet := mocks.NewMockOrderedSet(ctrl) for _, id := range allAtxs { @@ -126,23 +124,21 @@ func TestAtxHandler_Retry(t *testing.T) { for _, opt := range opts { opt(&atxOpts) } - require.NotNil(t, atxOpts.RecvChannel) + require.NotNil(t, atxOpts.Callback) for _, id := range atxs { switch { case id == allAtxs[0]: require.False(t, validationFailed, "retried after validation error") errs[id.Hash32()] = pubsub.ErrValidationReject + atxOpts.Callback(id, errs[id.Hash32()]) validationFailed = true case id == allAtxs[1] && failCount < 2: errs[id.Hash32()] = errors.New("fetch failed") + atxOpts.Callback(id, errs[id.Hash32()]) failCount++ default: fetched = append(fetched, id) - select { - case <-time.After(100 * time.Millisecond): - t.Error("timeout sending recvd id") - case atxOpts.RecvChannel <- id: - } + atxOpts.Callback(id, nil) } } if len(errs) > 0 { @@ -164,6 +160,31 @@ func TestAtxHandler_Retry(t *testing.T) { }, Error: rangesync.NoSeqError, }) + + // If it so happens that a full batch fails, we need to advance the clock to + // trigger the retry. + ctx, cancel := context.WithCancel(context.Background()) + var eg errgroup.Group + eg.Go(func() error { + for { + // FIXME: BlockUntilContext is not included in FakeClock interface. + // This will be fixed in a post-0.4.0 clockwork release, but with a breaking change that + // makes FakeClock a struct instead of an interface. + // See: https://github.com/jonboulle/clockwork/pull/71 + clock.(interface { + BlockUntilContext(ctx context.Context, n int) error + }).BlockUntilContext(ctx, 1) + if ctx.Err() != nil { + return nil + } + clock.Advance(batchRetryDelay) + } + }) + defer func() { + cancel() + eg.Wait() + }() + require.NoError(t, h.Commit(context.Background(), peer, baseSet, newSet)) require.ElementsMatch(t, allAtxs[1:], fetched) } @@ -180,7 +201,8 @@ func TestAtxHandler_Cancel(t *testing.T) { logger := zaptest.NewLogger(t) peer := p2p.Peer("foobar") f := NewMockFetcher(ctrl) - h := sync2.NewATXHandler(logger, f, batchSize, maxAttempts, maxBatchRetries, batchRetryDelay, nil) + clock := clockwork.NewFakeClock() + h := sync2.NewATXHandler(logger, f, batchSize, maxAttempts, maxBatchRetries, batchRetryDelay, clock) baseSet := mocks.NewMockOrderedSet(ctrl) newSet := mocks.NewMockOrderedSet(ctrl) baseSet.EXPECT().Has(rangesync.KeyBytes(atxID[:])).Return(false, nil) @@ -257,15 +279,11 @@ func TestAtxHandler_BatchRetry(t *testing.T) { for _, opt := range opts { opt(&atxOpts) } - require.NotNil(t, atxOpts.RecvChannel) + require.NotNil(t, atxOpts.Callback) for _, id := range atxs { require.True(t, toFetch[id], "already fetched or bad ID") delete(toFetch, id) - select { - case <-time.After(100 * time.Millisecond): - t.Error("timeout sending recvd id") - case atxOpts.RecvChannel <- id: - } + atxOpts.Callback(id, nil) } return nil }).Times(3) diff --git a/system/fetcher.go b/system/fetcher.go index 61f2d5c451..343f874516 100644 --- a/system/fetcher.go +++ b/system/fetcher.go @@ -28,7 +28,7 @@ type BlockFetcher interface { type GetAtxOpts struct { LimitingOff bool - RecvChannel chan<- types.ATXID + Callback func(types.ATXID, error) } type GetAtxOpt func(*GetAtxOpts) @@ -40,11 +40,15 @@ func WithoutLimiting() GetAtxOpt { } } -// WithRecvChannel sets the channel to receive successfully downloaded and validated ATXs -// IDs on. -func WithRecvChannel(ch chan<- types.ATXID) GetAtxOpt { +// WithATXCallback sets a callback function to be called after each ATX is downloaded, +// found locally or failed to download. +// The callback is guaranteed to be called exactly once for each ATX ID passed to GetAtxs. +// The callback is guaranteed not to be invoked after GetAtxs returns. +// The callback may be called concurrently from multiple goroutines. +// A non-nil error is passed in case the ATX cannot be found locally and failed to download. +func WithATXCallback(callback func(types.ATXID, error)) GetAtxOpt { return func(opts *GetAtxOpts) { - opts.RecvChannel = ch + opts.Callback = callback } } From f411f7ef34ef280b481ae31660c945d0d9136313 Mon Sep 17 00:00:00 2001 From: Ivan Shvedunov Date: Thu, 21 Nov 2024 00:26:41 +0400 Subject: [PATCH 3/6] syncer: rename "v2" field to "reconcSync" in the config --- config/mainnet.go | 2 +- config/presets/testnet.go | 2 +- syncer/syncer.go | 30 +++++++++++++++--------------- syncer/syncer_test.go | 8 ++++---- 4 files changed, 21 insertions(+), 21 deletions(-) diff --git a/config/mainnet.go b/config/mainnet.go index 58d0a77cac..bf29839606 100644 --- a/config/mainnet.go +++ b/config/mainnet.go @@ -221,7 +221,7 @@ func MainnetConfig() Config { DisableMeshAgreement: true, AtxSync: atxsync.DefaultConfig(), MalSync: malsync.DefaultConfig(), - V2: syncer.SyncV2Config{ + ReconcSync: syncer.ReconcSyncConfig{ OldAtxSyncCfg: oldAtxSyncCfg, NewAtxSyncCfg: newAtxSyncCfg, ParallelLoadLimit: 10, diff --git a/config/presets/testnet.go b/config/presets/testnet.go index 713cea25d8..e7856e509c 100644 --- a/config/presets/testnet.go +++ b/config/presets/testnet.go @@ -171,7 +171,7 @@ func testnet() config.Config { OutOfSyncThresholdLayers: 10, AtxSync: atxsync.DefaultConfig(), MalSync: malsync.DefaultConfig(), - V2: syncer.SyncV2Config{ + ReconcSync: syncer.ReconcSyncConfig{ OldAtxSyncCfg: oldAtxSyncCfg, NewAtxSyncCfg: newAtxSyncCfg, ParallelLoadLimit: 10, diff --git a/syncer/syncer.go b/syncer/syncer.go index 32089d3820..78de264603 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -37,15 +37,15 @@ type Config struct { TallyVotesFrequency float64 MaxStaleDuration time.Duration `mapstructure:"maxstaleduration"` Standalone bool - GossipDuration time.Duration `mapstructure:"gossipduration"` - DisableMeshAgreement bool `mapstructure:"disable-mesh-agreement"` - OutOfSyncThresholdLayers uint32 `mapstructure:"out-of-sync-threshold"` - AtxSync atxsync.Config `mapstructure:"atx-sync"` - MalSync malsync.Config `mapstructure:"malfeasance-sync"` - V2 SyncV2Config `mapstructure:"v2"` + GossipDuration time.Duration `mapstructure:"gossipduration"` + DisableMeshAgreement bool `mapstructure:"disable-mesh-agreement"` + OutOfSyncThresholdLayers uint32 `mapstructure:"out-of-sync-threshold"` + AtxSync atxsync.Config `mapstructure:"atx-sync"` + MalSync malsync.Config `mapstructure:"malfeasance-sync"` + ReconcSync ReconcSyncConfig `mapstructure:"reconc-sync"` } -type SyncV2Config struct { +type ReconcSyncConfig struct { Enable bool `mapstructure:"enable"` EnableActiveSync bool `mapstructure:"enable-active-sync"` OldAtxSyncCfg sync2.Config `mapstructure:"old-atx-sync"` @@ -72,7 +72,7 @@ func DefaultConfig() Config { OutOfSyncThresholdLayers: 3, AtxSync: atxsync.DefaultConfig(), MalSync: malsync.DefaultConfig(), - V2: SyncV2Config{ + ReconcSync: ReconcSyncConfig{ Enable: false, EnableActiveSync: false, OldAtxSyncCfg: oldAtxSyncCfg, @@ -238,14 +238,14 @@ func NewSyncer( s.isBusy.Store(false) s.lastLayerSynced.Store(s.mesh.LatestLayer().Uint32()) s.lastEpochSynced.Store(types.GetEffectiveGenesis().GetEpoch().Uint32() - 1) - if s.cfg.V2.Enable && s.asv2 == nil { + if s.cfg.ReconcSync.Enable && s.asv2 == nil { s.dispatcher = sync2.NewDispatcher(s.logger, fetcher.(sync2.Fetcher)) hss := sync2.NewATXSyncSource( s.logger, s.dispatcher, cdb.Database.(sql.StateDatabase), - fetcher.(sync2.Fetcher), s.cfg.V2.EnableActiveSync) + fetcher.(sync2.Fetcher), s.cfg.ReconcSync.EnableActiveSync) s.asv2 = sync2.NewMultiEpochATXSyncer( - s.logger, hss, s.cfg.V2.OldAtxSyncCfg, s.cfg.V2.NewAtxSyncCfg, - s.cfg.V2.ParallelLoadLimit) + s.logger, hss, s.cfg.ReconcSync.OldAtxSyncCfg, s.cfg.ReconcSync.NewAtxSyncCfg, + s.cfg.ReconcSync.ParallelLoadLimit) } return s } @@ -588,7 +588,7 @@ func (s *Syncer) ensureATXsInSyncV2(ctx context.Context) error { publish-- } - if !s.ListenToATXGossip() && s.cfg.V2.EnableActiveSync { + if !s.ListenToATXGossip() && s.cfg.ReconcSync.EnableActiveSync { // ATXs are not in sync yet, to we need to sync them synchronously lastWaitEpoch := types.EpochID(0) if currentEpoch > 1 { @@ -660,12 +660,12 @@ func (s *Syncer) ensureMalfeasanceInSync(ctx context.Context) error { } func (s *Syncer) syncAtxAndMalfeasance(ctx context.Context) error { - if s.cfg.V2.Enable { + if s.cfg.ReconcSync.Enable { if err := s.ensureATXsInSyncV2(ctx); err != nil { return err } } - if !s.cfg.V2.Enable || !s.cfg.V2.EnableActiveSync { + if !s.cfg.ReconcSync.Enable || !s.cfg.ReconcSync.EnableActiveSync { // If syncv2 is being used in server-only mode, we still need to run // active syncv1. if err := s.ensureATXsInSync(ctx); err != nil { diff --git a/syncer/syncer_test.go b/syncer/syncer_test.go index 3bd479a677..057ac29311 100644 --- a/syncer/syncer_test.go +++ b/syncer/syncer_test.go @@ -505,8 +505,8 @@ func TestSyncAtxs_Genesis(t *testing.T) { func TestSyncAtxs_Genesis_SyncV2(t *testing.T) { cfg := defaultTestConfig(never) - cfg.V2.Enable = true - cfg.V2.EnableActiveSync = true + cfg.ReconcSync.Enable = true + cfg.ReconcSync.EnableActiveSync = true t.Run("no atx expected", func(t *testing.T) { ts := newSyncerWithoutPeriodicRunsWithConfig(t, cfg) @@ -594,8 +594,8 @@ func startWithSyncedState_SyncV2(tb testing.TB, ts *testSyncer) types.LayerID { func TestSyncAtxs_SyncV2(t *testing.T) { cfg := defaultTestConfig(never) - cfg.V2.Enable = true - cfg.V2.EnableActiveSync = true + cfg.ReconcSync.Enable = true + cfg.ReconcSync.EnableActiveSync = true tcs := []struct { desc string current types.LayerID From bb7226fcf659a0cb7fef52f7f1e9aaac7c950784 Mon Sep 17 00:00:00 2001 From: Ivan Shvedunov Date: Thu, 21 Nov 2024 00:38:47 +0400 Subject: [PATCH 4/6] sync2: add server options and request rate limits --- fetch/fetch.go | 4 ++-- sync2/atxs.go | 4 ++-- syncer/syncer.go | 24 ++++++++++++++++++------ 3 files changed, 22 insertions(+), 10 deletions(-) diff --git a/fetch/fetch.go b/fetch/fetch.go index 479b4f4106..0aee4ed636 100644 --- a/fetch/fetch.go +++ b/fetch/fetch.go @@ -116,7 +116,7 @@ type ServerConfig struct { Interval time.Duration `mapstructure:"interval"` } -func (s ServerConfig) toOpts() []server.Opt { +func (s ServerConfig) ToOpts() []server.Opt { opts := []server.Opt{} if s.Queue != 0 { opts = append(opts, server.WithQueueSize(s.Queue)) @@ -366,7 +366,7 @@ func (f *Fetch) registerServer( if f.cfg.EnableServerMetrics { opts = append(opts, server.WithMetrics()) } - opts = append(opts, f.cfg.getServerConfig(protocol).toOpts()...) + opts = append(opts, f.cfg.getServerConfig(protocol).ToOpts()...) f.servers[protocol] = server.New(host, protocol, handler, opts...) } diff --git a/sync2/atxs.go b/sync2/atxs.go index 45fec8d9b7..90663f61e6 100644 --- a/sync2/atxs.go +++ b/sync2/atxs.go @@ -295,9 +295,9 @@ func NewATXSyncer( cfg, enableActiveSync) } -func NewDispatcher(logger *zap.Logger, f Fetcher) *rangesync.Dispatcher { +func NewDispatcher(logger *zap.Logger, f Fetcher, opts []server.Opt) *rangesync.Dispatcher { d := rangesync.NewDispatcher(logger) - d.SetupServer(f.Host(), proto, server.WithHardTimeout(20*time.Minute)) + d.SetupServer(f.Host(), proto, opts...) return d } diff --git a/syncer/syncer.go b/syncer/syncer.go index 78de264603..65fd3042f8 100644 --- a/syncer/syncer.go +++ b/syncer/syncer.go @@ -18,6 +18,7 @@ import ( "github.com/spacemeshos/go-spacemesh/log" "github.com/spacemeshos/go-spacemesh/mesh" "github.com/spacemeshos/go-spacemesh/p2p" + "github.com/spacemeshos/go-spacemesh/p2p/server" "github.com/spacemeshos/go-spacemesh/sql" "github.com/spacemeshos/go-spacemesh/sync2" "github.com/spacemeshos/go-spacemesh/sync2/rangesync" @@ -46,11 +47,13 @@ type Config struct { } type ReconcSyncConfig struct { - Enable bool `mapstructure:"enable"` - EnableActiveSync bool `mapstructure:"enable-active-sync"` - OldAtxSyncCfg sync2.Config `mapstructure:"old-atx-sync"` - NewAtxSyncCfg sync2.Config `mapstructure:"new-atx-sync"` - ParallelLoadLimit int `mapstructure:"parallel-load-limit"` + Enable bool `mapstructure:"enable"` + EnableActiveSync bool `mapstructure:"enable-active-sync"` + OldAtxSyncCfg sync2.Config `mapstructure:"old-atx-sync"` + NewAtxSyncCfg sync2.Config `mapstructure:"new-atx-sync"` + ParallelLoadLimit int `mapstructure:"parallel-load-limit"` + HardTimeout time.Duration `mapstructure:"hard-timeout"` + ServerConfig fetch.ServerConfig `mapstructure:"server-config"` } // DefaultConfig for the syncer. @@ -78,6 +81,12 @@ func DefaultConfig() Config { OldAtxSyncCfg: oldAtxSyncCfg, NewAtxSyncCfg: newAtxSyncCfg, ParallelLoadLimit: 10, + HardTimeout: 10 * time.Minute, + ServerConfig: fetch.ServerConfig{ + Queue: 200, + Requests: 100, + Interval: time.Second, + }, }, } } @@ -239,7 +248,10 @@ func NewSyncer( s.lastLayerSynced.Store(s.mesh.LatestLayer().Uint32()) s.lastEpochSynced.Store(types.GetEffectiveGenesis().GetEpoch().Uint32() - 1) if s.cfg.ReconcSync.Enable && s.asv2 == nil { - s.dispatcher = sync2.NewDispatcher(s.logger, fetcher.(sync2.Fetcher)) + serverOpts := append( + s.cfg.ReconcSync.ServerConfig.ToOpts(), + server.WithHardTimeout(s.cfg.ReconcSync.HardTimeout)) + s.dispatcher = sync2.NewDispatcher(s.logger, fetcher.(sync2.Fetcher), serverOpts) hss := sync2.NewATXSyncSource( s.logger, s.dispatcher, cdb.Database.(sql.StateDatabase), fetcher.(sync2.Fetcher), s.cfg.ReconcSync.EnableActiveSync) From 284f836bc0275f926a28d815bce7e8390d527732 Mon Sep 17 00:00:00 2001 From: Ivan Shvedunov Date: Mon, 25 Nov 2024 00:37:07 +0400 Subject: [PATCH 5/6] sync2: fix mainnet/testnet configs --- config/mainnet.go | 6 ++++++ config/presets/testnet.go | 6 ++++++ 2 files changed, 12 insertions(+) diff --git a/config/mainnet.go b/config/mainnet.go index bf29839606..2af24825c9 100644 --- a/config/mainnet.go +++ b/config/mainnet.go @@ -225,6 +225,12 @@ func MainnetConfig() Config { OldAtxSyncCfg: oldAtxSyncCfg, NewAtxSyncCfg: newAtxSyncCfg, ParallelLoadLimit: 10, + HardTimeout: 10 * time.Minute, + ServerConfig: fetch.ServerConfig{ + Queue: 200, + Requests: 100, + Interval: time.Second, + }, }, }, Recovery: checkpoint.DefaultConfig(), diff --git a/config/presets/testnet.go b/config/presets/testnet.go index e7856e509c..476b4fe16b 100644 --- a/config/presets/testnet.go +++ b/config/presets/testnet.go @@ -175,6 +175,12 @@ func testnet() config.Config { OldAtxSyncCfg: oldAtxSyncCfg, NewAtxSyncCfg: newAtxSyncCfg, ParallelLoadLimit: 10, + HardTimeout: time.Minute, + ServerConfig: fetch.ServerConfig{ + Queue: 200, + Requests: 100, + Interval: time.Second, + }, }, }, Recovery: checkpoint.DefaultConfig(), From a8d87f119965d90580279e3b6e34d50bc6c55c9c Mon Sep 17 00:00:00 2001 From: Ivan Shvedunov Date: Mon, 25 Nov 2024 16:32:19 +0400 Subject: [PATCH 6/6] sync2: only handle synced keys during commit It turned out that sync interactions are happening rather quickly, and thus it is not really practical to try and begin handling arriving keys (e.g. ATX IDs) during sync itself. Moreover, on-the-fly key handling was actually only used to register peer IDs for each ATX ID to fetch the actual blob from, and that can be done in the handler's Commit() method just as well. --- sync2/atxs.go | 23 ++- sync2/atxs_test.go | 97 ++++------- sync2/dbset/dbset.go | 7 +- sync2/dbset/dbset_test.go | 37 ++-- sync2/dbset/p2p_test.go | 4 +- sync2/multipeer/interface.go | 30 ++-- sync2/multipeer/mocks_test.go | 230 +++---------------------- sync2/multipeer/multipeer.go | 38 ++--- sync2/multipeer/multipeer_test.go | 54 +----- sync2/multipeer/setsyncbase.go | 154 ++++------------- sync2/multipeer/setsyncbase_test.go | 250 +++++----------------------- sync2/multipeer/split_sync.go | 57 +++---- sync2/multipeer/split_sync_test.go | 74 +++----- sync2/p2p.go | 14 +- sync2/p2p_test.go | 44 ++--- sync2/rangesync/dumbset.go | 6 +- sync2/rangesync/interface.go | 12 +- sync2/rangesync/mocks/mocks.go | 13 +- sync2/rangesync/seq.go | 19 ++- 19 files changed, 299 insertions(+), 864 deletions(-) diff --git a/sync2/atxs.go b/sync2/atxs.go index 90663f61e6..4bb2a57abb 100644 --- a/sync2/atxs.go +++ b/sync2/atxs.go @@ -62,21 +62,18 @@ func NewATXHandler( } } -func (h *ATXHandler) Receive(k rangesync.KeyBytes, peer p2p.Peer) (bool, error) { - var id types.ATXID - copy(id[:], k) - h.f.RegisterPeerHash(peer, id.Hash32()) - return false, nil -} - -func (h *ATXHandler) Commit(ctx context.Context, peer p2p.Peer, base, new rangesync.OrderedSet) error { +func (h *ATXHandler) Commit( + ctx context.Context, + peer p2p.Peer, + base rangesync.OrderedSet, + received rangesync.SeqResult, +) error { h.logger.Debug("begin atx commit") defer h.logger.Debug("end atx commit") - sr := new.Received() var firstK rangesync.KeyBytes numDownloaded := 0 state := make(map[types.ATXID]int) - for k := range sr.Seq { + for k := range received.Seq { if firstK == nil { firstK = k } else if firstK.Compare(k) == 0 { @@ -89,9 +86,11 @@ func (h *ATXHandler) Commit(ctx context.Context, peer p2p.Peer, base, new ranges if found { continue } - state[types.BytesToATXID(k)] = 0 + id := types.BytesToATXID(k) + h.f.RegisterPeerHash(peer, id.Hash32()) + state[id] = 0 } - if err := sr.Error(); err != nil { + if err := received.Error(); err != nil { return fmt.Errorf("get item: %w", err) } total := len(state) diff --git a/sync2/atxs_test.go b/sync2/atxs_test.go index ebfe3451a3..46db742898 100644 --- a/sync2/atxs_test.go +++ b/sync2/atxs_test.go @@ -23,6 +23,21 @@ import ( "github.com/spacemeshos/go-spacemesh/system" ) +func atxSeqResult(atxs []types.ATXID) rangesync.SeqResult { + return rangesync.SeqResult{ + Seq: func(yield func(k rangesync.KeyBytes) bool) { + // Received sequence may be cyclic and the handler should stop + // when it sees the first key again. + for _, atx := range atxs { + if !yield(atx.Bytes()) { + return + } + } + }, + Error: rangesync.NoSeqError, + } +} + func TestAtxHandler_Success(t *testing.T) { const ( batchSize = 4 @@ -41,13 +56,9 @@ func TestAtxHandler_Success(t *testing.T) { clock := clockwork.NewFakeClock() h := sync2.NewATXHandler(logger, f, batchSize, maxAttempts, maxBatchRetries, batchRetryDelay, clock) baseSet := mocks.NewMockOrderedSet(ctrl) - newSet := mocks.NewMockOrderedSet(ctrl) for _, id := range allAtxs { + baseSet.EXPECT().Has(rangesync.KeyBytes(id[:])) f.EXPECT().RegisterPeerHash(peer, id.Hash32()) - baseSet.EXPECT().Has(rangesync.KeyBytes(id[:])).Return(false, nil) - add, err := h.Receive(id.Bytes(), peer) - require.False(t, add) - require.NoError(t, err) } toFetch := make(map[types.ATXID]bool) for _, id := range allAtxs { @@ -69,21 +80,7 @@ func TestAtxHandler_Success(t *testing.T) { } return nil }).Times(3) - newSet.EXPECT().Received().Return(rangesync.SeqResult{ - Seq: func(yield func(k rangesync.KeyBytes) bool) { - // Received sequence may be cyclic and the handler should stop - // when it sees the first key again. - for { - for _, atx := range allAtxs { - if !yield(atx.Bytes()) { - return - } - } - } - }, - Error: rangesync.NoSeqError, - }) - require.NoError(t, h.Commit(context.Background(), peer, baseSet, newSet)) + require.NoError(t, h.Commit(context.Background(), peer, baseSet, atxSeqResult(allAtxs))) require.Empty(t, toFetch) require.Equal(t, []int{4, 4, 2}, batches) } @@ -106,13 +103,9 @@ func TestAtxHandler_Retry(t *testing.T) { clock := clockwork.NewFakeClock() h := sync2.NewATXHandler(logger, f, batchSize, maxAttempts, maxBatchRetries, batchRetryDelay, clock) baseSet := mocks.NewMockOrderedSet(ctrl) - newSet := mocks.NewMockOrderedSet(ctrl) for _, id := range allAtxs { + baseSet.EXPECT().Has(rangesync.KeyBytes(id[:])) f.EXPECT().RegisterPeerHash(peer, id.Hash32()) - baseSet.EXPECT().Has(rangesync.KeyBytes(id[:])).Return(false, nil) - add, err := h.Receive(id.Bytes(), peer) - require.False(t, add) - require.NoError(t, err) } failCount := 0 var fetched []types.ATXID @@ -150,16 +143,6 @@ func TestAtxHandler_Retry(t *testing.T) { } return nil }).AnyTimes() - newSet.EXPECT().Received().Return(rangesync.SeqResult{ - Seq: func(yield func(k rangesync.KeyBytes) bool) { - for _, atx := range allAtxs { - if !yield(atx.Bytes()) { - return - } - } - }, - Error: rangesync.NoSeqError, - }) // If it so happens that a full batch fails, we need to advance the clock to // trigger the retry. @@ -185,7 +168,7 @@ func TestAtxHandler_Retry(t *testing.T) { eg.Wait() }() - require.NoError(t, h.Commit(context.Background(), peer, baseSet, newSet)) + require.NoError(t, h.Commit(context.Background(), peer, baseSet, atxSeqResult(allAtxs))) require.ElementsMatch(t, allAtxs[1:], fetched) } @@ -204,19 +187,19 @@ func TestAtxHandler_Cancel(t *testing.T) { clock := clockwork.NewFakeClock() h := sync2.NewATXHandler(logger, f, batchSize, maxAttempts, maxBatchRetries, batchRetryDelay, clock) baseSet := mocks.NewMockOrderedSet(ctrl) - newSet := mocks.NewMockOrderedSet(ctrl) baseSet.EXPECT().Has(rangesync.KeyBytes(atxID[:])).Return(false, nil) + f.EXPECT().RegisterPeerHash(peer, atxID.Hash32()) f.EXPECT().GetAtxs(gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( func(_ context.Context, atxs []types.ATXID, opts ...system.GetAtxOpt) error { return context.Canceled }) - newSet.EXPECT().Received().Return(rangesync.SeqResult{ + sr := rangesync.SeqResult{ Seq: func(yield func(k rangesync.KeyBytes) bool) { yield(atxID.Bytes()) }, Error: rangesync.NoSeqError, - }) - require.ErrorIs(t, h.Commit(context.Background(), peer, baseSet, newSet), context.Canceled) + } + require.ErrorIs(t, h.Commit(context.Background(), peer, baseSet, sr), context.Canceled) } func TestAtxHandler_BatchRetry(t *testing.T) { @@ -237,35 +220,17 @@ func TestAtxHandler_BatchRetry(t *testing.T) { f := NewMockFetcher(ctrl) h := sync2.NewATXHandler(logger, f, batchSize, maxAttempts, maxBatchRetries, batchRetryDelay, clock) baseSet := mocks.NewMockOrderedSet(ctrl) - newSet := mocks.NewMockOrderedSet(ctrl) for _, id := range allAtxs { + baseSet.EXPECT().Has(rangesync.KeyBytes(id[:])) f.EXPECT().RegisterPeerHash(peer, id.Hash32()) - baseSet.EXPECT().Has(rangesync.KeyBytes(id[:])).Return(false, nil) - add, err := h.Receive(id.Bytes(), peer) - require.False(t, add) - require.NoError(t, err) } - newSet.EXPECT().Received().Return(rangesync.SeqResult{ - Seq: func(yield func(k rangesync.KeyBytes) bool) { - // Received sequence may be cyclic and the handler should stop - // when it sees the first key again. - for { - for _, atx := range allAtxs { - if !yield(atx.Bytes()) { - return - } - } - } - }, - Error: rangesync.NoSeqError, - }) f.EXPECT().GetAtxs(gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( func(_ context.Context, atxs []types.ATXID, opts ...system.GetAtxOpt) error { return errors.New("fetch failed") }) var eg errgroup.Group eg.Go(func() error { - return h.Commit(context.Background(), peer, baseSet, newSet) + return h.Commit(context.Background(), peer, baseSet, atxSeqResult(allAtxs)) }) // wait for delay after 1st batch failure clock.BlockUntil(1) @@ -310,15 +275,11 @@ func TestAtxHandler_BatchRetry_Fail(t *testing.T) { f := NewMockFetcher(ctrl) h := sync2.NewATXHandler(logger, f, batchSize, maxAttempts, maxBatchRetries, batchRetryDelay, clock) baseSet := mocks.NewMockOrderedSet(ctrl) - newSet := mocks.NewMockOrderedSet(ctrl) for _, id := range allAtxs { + baseSet.EXPECT().Has(rangesync.KeyBytes(id[:])) f.EXPECT().RegisterPeerHash(peer, id.Hash32()) - baseSet.EXPECT().Has(rangesync.KeyBytes(id[:])).Return(false, nil) - add, err := h.Receive(id.Bytes(), peer) - require.False(t, add) - require.NoError(t, err) } - newSet.EXPECT().Received().Return(rangesync.SeqResult{ + sr := rangesync.SeqResult{ Seq: func(yield func(k rangesync.KeyBytes) bool) { // Received sequence may be cyclic and the handler should stop // when it sees the first key again. @@ -331,14 +292,14 @@ func TestAtxHandler_BatchRetry_Fail(t *testing.T) { } }, Error: rangesync.NoSeqError, - }) + } f.EXPECT().GetAtxs(gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( func(_ context.Context, atxs []types.ATXID, opts ...system.GetAtxOpt) error { return errors.New("fetch failed") }).Times(3) var eg errgroup.Group eg.Go(func() error { - return h.Commit(context.Background(), peer, baseSet, newSet) + return h.Commit(context.Background(), peer, baseSet, sr) }) for range 2 { clock.BlockUntil(1) diff --git a/sync2/dbset/dbset.go b/sync2/dbset/dbset.go index 795ca24f1f..ed20293827 100644 --- a/sync2/dbset/dbset.go +++ b/sync2/dbset/dbset.go @@ -80,9 +80,10 @@ func (d *DBSet) EnsureLoaded() error { return d.snapshot.Load(d.db, d.handleIDfromDB) } -// Received returns a sequence of all items that have been received. +// Received returns a sequence of all items that have been received and the number of +// these items. // Implements rangesync.OrderedSet. -func (d *DBSet) Received() rangesync.SeqResult { +func (d *DBSet) Received() (rangesync.SeqResult, int) { return rangesync.SeqResult{ Seq: func(yield func(k rangesync.KeyBytes) bool) { for k := range d.received { @@ -92,7 +93,7 @@ func (d *DBSet) Received() rangesync.SeqResult { } }, Error: rangesync.NoSeqError, - } + }, len(d.received) } // Add adds an item to the DBSet. diff --git a/sync2/dbset/dbset_test.go b/sync2/dbset/dbset_test.go index 7970cb4830..1e0600a629 100644 --- a/sync2/dbset/dbset_test.go +++ b/sync2/dbset/dbset_test.go @@ -42,7 +42,9 @@ func TestDBSet_Empty(t *testing.T) { require.NoError(t, err) require.True(t, empty) requireEmpty(t, s.Items()) - requireEmpty(t, s.Received()) + sr, n := s.Received() + requireEmpty(t, sr) + require.Zero(t, n) info, err := s.GetRangeInfo(nil, nil) require.NoError(t, err) @@ -190,11 +192,12 @@ func TestDBSet_Receive(t *testing.T) { newID := rangesync.MustParseHexKeyBytes("abcdef1234567890000000000000000000000000000000000000000000000000") require.NoError(t, s.Receive(newID)) - recvd := s.Received() + recvd, n := s.Received() items, err := recvd.FirstN(1) require.NoError(t, err) require.NoError(t, err) require.Equal(t, []rangesync.KeyBytes{newID}, items) + require.Equal(t, 1, n) info, err := s.GetRangeInfo(ids[2], ids[0]) require.NoError(t, err) @@ -235,9 +238,9 @@ func TestDBSet_Copy(t *testing.T) { require.Equal(t, "dddddddddddddddddddddddd", info.Fingerprint.String()) require.Equal(t, ids[2], firstKey(t, info.Items)) - items, err := s.Received().FirstN(100) - require.NoError(t, err) - require.Empty(t, items) + sr, n := s.Received() + requireEmpty(t, sr) + require.Zero(t, n) info, err = s.GetRangeInfo(ids[2], ids[0]) require.NoError(t, err) @@ -245,7 +248,9 @@ func TestDBSet_Copy(t *testing.T) { require.Equal(t, "dddddddddddddddddddddddd", info.Fingerprint.String()) require.Equal(t, ids[2], firstKey(t, info.Items)) - items, err = copy.(*dbset.DBSet).Received().FirstN(100) + sr, n = copy.(*dbset.DBSet).Received() + require.Equal(t, 1, n) + items, err := sr.FirstN(100) require.NoError(t, err) require.Equal(t, []rangesync.KeyBytes{newID}, items) @@ -356,29 +361,35 @@ func TestDBSet_Added(t *testing.T) { IDColumn: "id", } s := dbset.NewDBSet(db, st, testKeyLen, testDepth) - requireEmpty(t, s.Received()) + sr, n := s.Received() + requireEmpty(t, sr) + require.Zero(t, n) - add := []rangesync.KeyBytes{ + recv := []rangesync.KeyBytes{ rangesync.MustParseHexKeyBytes("3333333333333333333333333333333333333333333333333333333333333333"), rangesync.MustParseHexKeyBytes("4444444444444444444444444444444444444444444444444444444444444444"), } - for _, item := range add { + for _, item := range recv { require.NoError(t, s.Receive(item)) } require.NoError(t, s.EnsureLoaded()) - added, err := s.Received().FirstN(3) + sr, n = s.Received() + require.Equal(t, 2, n) + recvd, err := sr.FirstN(3) require.NoError(t, err) require.ElementsMatch(t, []rangesync.KeyBytes{ rangesync.MustParseHexKeyBytes("3333333333333333333333333333333333333333333333333333333333333333"), rangesync.MustParseHexKeyBytes("4444444444444444444444444444444444444444444444444444444444444444"), - }, added) + }, recvd) require.NoError(t, s.WithCopy(context.Background(), func(copy rangesync.OrderedSet) error { - added1, err := copy.(*dbset.DBSet).Received().FirstN(3) + sr, n := copy.(*dbset.DBSet).Received() + require.Equal(t, 2, n) + recvd1, err := sr.FirstN(3) require.NoError(t, err) - require.ElementsMatch(t, added, added1) + require.ElementsMatch(t, recvd, recvd1) return nil })) } diff --git a/sync2/dbset/p2p_test.go b/sync2/dbset/p2p_test.go index c636c461ba..457b538352 100644 --- a/sync2/dbset/p2p_test.go +++ b/sync2/dbset/p2p_test.go @@ -81,8 +81,9 @@ func (tr *syncTracer) OnRecent(receivedItems, sentItems int) { } func addReceived(t testing.TB, db sql.Executor, to, from *dbset.DBSet) { - sr := from.Received() + sr, n := from.Received() for k := range sr.Seq { + n-- has, err := to.Has(k) require.NoError(t, err) if !has { @@ -91,6 +92,7 @@ func addReceived(t testing.TB, db sql.Executor, to, from *dbset.DBSet) { } require.NoError(t, sr.Error()) require.NoError(t, to.Advance()) + require.Zero(t, n) } type startStopTimer interface { diff --git a/sync2/multipeer/interface.go b/sync2/multipeer/interface.go index b3a5236171..2d65699256 100644 --- a/sync2/multipeer/interface.go +++ b/sync2/multipeer/interface.go @@ -11,38 +11,28 @@ import ( //go:generate mockgen -typed -package=multipeer_test -destination=./mocks_test.go -source=./interface.go // SyncBase is a synchronization base which holds the original OrderedSet. -// It is used to derive per-peer PeerSyncers with their own copies of the OrderedSet, -// copy operation being O(1) in terms of memory and time complexity. +// It is used to sync against peers using derived OrderedSets. // It can also probe peers to decide on the synchronization strategy. type SyncBase interface { // Count returns the number of items in the set. Count() (int, error) - // WithPeerSyncer creates a Syncer for the specified peer and passes it to the specified function. - // When the function returns, the syncer is discarded, releasing the resources associated with it. - WithPeerSyncer(ctx context.Context, p p2p.Peer, toCall func(PeerSyncer) error) error + // Sync synchronizes the set with the peer. + // It returns a sequence of new keys that were received from the peer and the + // number of received items. + Sync(ctx context.Context, p p2p.Peer, x, y rangesync.KeyBytes) error + // Serve serves a synchronization request on the specified stream. + // It returns a sequence of new keys that were received from the peer and the + // number of received items. + Serve(ctx context.Context, p p2p.Peer, stream io.ReadWriter) error // Probe probes the specified peer, obtaining its set fingerprint, // the number of items and the similarity value. Probe(ctx context.Context, p p2p.Peer) (rangesync.ProbeResult, error) - // Wait waits for all the derived syncers' handlers to finish. - Wait() error -} - -// PeerSyncer is a synchronization interface for a single peer. -type PeerSyncer interface { - // Peer returns the peer this syncer is for. - Peer() p2p.Peer - // Sync synchronizes the set with the peer. - Sync(ctx context.Context, x, y rangesync.KeyBytes) error - // Serve serves a synchronization request on the specified stream. - Serve(ctx context.Context, stream io.ReadWriter) error } // SyncKeyHandler is a handler for keys that are received from peers. type SyncKeyHandler interface { - // Receive handles a key that was received from a peer. - Receive(k rangesync.KeyBytes, peer p2p.Peer) (bool, error) // Commit is invoked at the end of synchronization to apply the changes. - Commit(ctx context.Context, peer p2p.Peer, base, new rangesync.OrderedSet) error + Commit(ctx context.Context, peer p2p.Peer, base rangesync.OrderedSet, received rangesync.SeqResult) error } // PairwiseSyncer is used to probe a peer or sync against a single peer. diff --git a/sync2/multipeer/mocks_test.go b/sync2/multipeer/mocks_test.go index 48c0401b5b..dd78ac583d 100644 --- a/sync2/multipeer/mocks_test.go +++ b/sync2/multipeer/mocks_test.go @@ -15,7 +15,6 @@ import ( reflect "reflect" p2p "github.com/spacemeshos/go-spacemesh/p2p" - multipeer "github.com/spacemeshos/go-spacemesh/sync2/multipeer" rangesync "github.com/spacemeshos/go-spacemesh/sync2/rangesync" gomock "go.uber.org/mock/gomock" ) @@ -122,216 +121,78 @@ func (c *MockSyncBaseProbeCall) DoAndReturn(f func(context.Context, p2p.Peer) (r return c } -// Wait mocks base method. -func (m *MockSyncBase) Wait() error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Wait") - ret0, _ := ret[0].(error) - return ret0 -} - -// Wait indicates an expected call of Wait. -func (mr *MockSyncBaseMockRecorder) Wait() *MockSyncBaseWaitCall { - mr.mock.ctrl.T.Helper() - call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Wait", reflect.TypeOf((*MockSyncBase)(nil).Wait)) - return &MockSyncBaseWaitCall{Call: call} -} - -// MockSyncBaseWaitCall wrap *gomock.Call -type MockSyncBaseWaitCall struct { - *gomock.Call -} - -// Return rewrite *gomock.Call.Return -func (c *MockSyncBaseWaitCall) Return(arg0 error) *MockSyncBaseWaitCall { - c.Call = c.Call.Return(arg0) - return c -} - -// Do rewrite *gomock.Call.Do -func (c *MockSyncBaseWaitCall) Do(f func() error) *MockSyncBaseWaitCall { - c.Call = c.Call.Do(f) - return c -} - -// DoAndReturn rewrite *gomock.Call.DoAndReturn -func (c *MockSyncBaseWaitCall) DoAndReturn(f func() error) *MockSyncBaseWaitCall { - c.Call = c.Call.DoAndReturn(f) - return c -} - -// WithPeerSyncer mocks base method. -func (m *MockSyncBase) WithPeerSyncer(ctx context.Context, p p2p.Peer, toCall func(multipeer.PeerSyncer) error) error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "WithPeerSyncer", ctx, p, toCall) - ret0, _ := ret[0].(error) - return ret0 -} - -// WithPeerSyncer indicates an expected call of WithPeerSyncer. -func (mr *MockSyncBaseMockRecorder) WithPeerSyncer(ctx, p, toCall any) *MockSyncBaseWithPeerSyncerCall { - mr.mock.ctrl.T.Helper() - call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WithPeerSyncer", reflect.TypeOf((*MockSyncBase)(nil).WithPeerSyncer), ctx, p, toCall) - return &MockSyncBaseWithPeerSyncerCall{Call: call} -} - -// MockSyncBaseWithPeerSyncerCall wrap *gomock.Call -type MockSyncBaseWithPeerSyncerCall struct { - *gomock.Call -} - -// Return rewrite *gomock.Call.Return -func (c *MockSyncBaseWithPeerSyncerCall) Return(arg0 error) *MockSyncBaseWithPeerSyncerCall { - c.Call = c.Call.Return(arg0) - return c -} - -// Do rewrite *gomock.Call.Do -func (c *MockSyncBaseWithPeerSyncerCall) Do(f func(context.Context, p2p.Peer, func(multipeer.PeerSyncer) error) error) *MockSyncBaseWithPeerSyncerCall { - c.Call = c.Call.Do(f) - return c -} - -// DoAndReturn rewrite *gomock.Call.DoAndReturn -func (c *MockSyncBaseWithPeerSyncerCall) DoAndReturn(f func(context.Context, p2p.Peer, func(multipeer.PeerSyncer) error) error) *MockSyncBaseWithPeerSyncerCall { - c.Call = c.Call.DoAndReturn(f) - return c -} - -// MockPeerSyncer is a mock of PeerSyncer interface. -type MockPeerSyncer struct { - ctrl *gomock.Controller - recorder *MockPeerSyncerMockRecorder - isgomock struct{} -} - -// MockPeerSyncerMockRecorder is the mock recorder for MockPeerSyncer. -type MockPeerSyncerMockRecorder struct { - mock *MockPeerSyncer -} - -// NewMockPeerSyncer creates a new mock instance. -func NewMockPeerSyncer(ctrl *gomock.Controller) *MockPeerSyncer { - mock := &MockPeerSyncer{ctrl: ctrl} - mock.recorder = &MockPeerSyncerMockRecorder{mock} - return mock -} - -// EXPECT returns an object that allows the caller to indicate expected use. -func (m *MockPeerSyncer) EXPECT() *MockPeerSyncerMockRecorder { - return m.recorder -} - -// Peer mocks base method. -func (m *MockPeerSyncer) Peer() p2p.Peer { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Peer") - ret0, _ := ret[0].(p2p.Peer) - return ret0 -} - -// Peer indicates an expected call of Peer. -func (mr *MockPeerSyncerMockRecorder) Peer() *MockPeerSyncerPeerCall { - mr.mock.ctrl.T.Helper() - call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Peer", reflect.TypeOf((*MockPeerSyncer)(nil).Peer)) - return &MockPeerSyncerPeerCall{Call: call} -} - -// MockPeerSyncerPeerCall wrap *gomock.Call -type MockPeerSyncerPeerCall struct { - *gomock.Call -} - -// Return rewrite *gomock.Call.Return -func (c *MockPeerSyncerPeerCall) Return(arg0 p2p.Peer) *MockPeerSyncerPeerCall { - c.Call = c.Call.Return(arg0) - return c -} - -// Do rewrite *gomock.Call.Do -func (c *MockPeerSyncerPeerCall) Do(f func() p2p.Peer) *MockPeerSyncerPeerCall { - c.Call = c.Call.Do(f) - return c -} - -// DoAndReturn rewrite *gomock.Call.DoAndReturn -func (c *MockPeerSyncerPeerCall) DoAndReturn(f func() p2p.Peer) *MockPeerSyncerPeerCall { - c.Call = c.Call.DoAndReturn(f) - return c -} - // Serve mocks base method. -func (m *MockPeerSyncer) Serve(ctx context.Context, stream io.ReadWriter) error { +func (m *MockSyncBase) Serve(ctx context.Context, p p2p.Peer, stream io.ReadWriter) error { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Serve", ctx, stream) + ret := m.ctrl.Call(m, "Serve", ctx, p, stream) ret0, _ := ret[0].(error) return ret0 } // Serve indicates an expected call of Serve. -func (mr *MockPeerSyncerMockRecorder) Serve(ctx, stream any) *MockPeerSyncerServeCall { +func (mr *MockSyncBaseMockRecorder) Serve(ctx, p, stream any) *MockSyncBaseServeCall { mr.mock.ctrl.T.Helper() - call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Serve", reflect.TypeOf((*MockPeerSyncer)(nil).Serve), ctx, stream) - return &MockPeerSyncerServeCall{Call: call} + call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Serve", reflect.TypeOf((*MockSyncBase)(nil).Serve), ctx, p, stream) + return &MockSyncBaseServeCall{Call: call} } -// MockPeerSyncerServeCall wrap *gomock.Call -type MockPeerSyncerServeCall struct { +// MockSyncBaseServeCall wrap *gomock.Call +type MockSyncBaseServeCall struct { *gomock.Call } // Return rewrite *gomock.Call.Return -func (c *MockPeerSyncerServeCall) Return(arg0 error) *MockPeerSyncerServeCall { +func (c *MockSyncBaseServeCall) Return(arg0 error) *MockSyncBaseServeCall { c.Call = c.Call.Return(arg0) return c } // Do rewrite *gomock.Call.Do -func (c *MockPeerSyncerServeCall) Do(f func(context.Context, io.ReadWriter) error) *MockPeerSyncerServeCall { +func (c *MockSyncBaseServeCall) Do(f func(context.Context, p2p.Peer, io.ReadWriter) error) *MockSyncBaseServeCall { c.Call = c.Call.Do(f) return c } // DoAndReturn rewrite *gomock.Call.DoAndReturn -func (c *MockPeerSyncerServeCall) DoAndReturn(f func(context.Context, io.ReadWriter) error) *MockPeerSyncerServeCall { +func (c *MockSyncBaseServeCall) DoAndReturn(f func(context.Context, p2p.Peer, io.ReadWriter) error) *MockSyncBaseServeCall { c.Call = c.Call.DoAndReturn(f) return c } // Sync mocks base method. -func (m *MockPeerSyncer) Sync(ctx context.Context, x, y rangesync.KeyBytes) error { +func (m *MockSyncBase) Sync(ctx context.Context, p p2p.Peer, x, y rangesync.KeyBytes) error { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Sync", ctx, x, y) + ret := m.ctrl.Call(m, "Sync", ctx, p, x, y) ret0, _ := ret[0].(error) return ret0 } // Sync indicates an expected call of Sync. -func (mr *MockPeerSyncerMockRecorder) Sync(ctx, x, y any) *MockPeerSyncerSyncCall { +func (mr *MockSyncBaseMockRecorder) Sync(ctx, p, x, y any) *MockSyncBaseSyncCall { mr.mock.ctrl.T.Helper() - call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Sync", reflect.TypeOf((*MockPeerSyncer)(nil).Sync), ctx, x, y) - return &MockPeerSyncerSyncCall{Call: call} + call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Sync", reflect.TypeOf((*MockSyncBase)(nil).Sync), ctx, p, x, y) + return &MockSyncBaseSyncCall{Call: call} } -// MockPeerSyncerSyncCall wrap *gomock.Call -type MockPeerSyncerSyncCall struct { +// MockSyncBaseSyncCall wrap *gomock.Call +type MockSyncBaseSyncCall struct { *gomock.Call } // Return rewrite *gomock.Call.Return -func (c *MockPeerSyncerSyncCall) Return(arg0 error) *MockPeerSyncerSyncCall { +func (c *MockSyncBaseSyncCall) Return(arg0 error) *MockSyncBaseSyncCall { c.Call = c.Call.Return(arg0) return c } // Do rewrite *gomock.Call.Do -func (c *MockPeerSyncerSyncCall) Do(f func(context.Context, rangesync.KeyBytes, rangesync.KeyBytes) error) *MockPeerSyncerSyncCall { +func (c *MockSyncBaseSyncCall) Do(f func(context.Context, p2p.Peer, rangesync.KeyBytes, rangesync.KeyBytes) error) *MockSyncBaseSyncCall { c.Call = c.Call.Do(f) return c } // DoAndReturn rewrite *gomock.Call.DoAndReturn -func (c *MockPeerSyncerSyncCall) DoAndReturn(f func(context.Context, rangesync.KeyBytes, rangesync.KeyBytes) error) *MockPeerSyncerSyncCall { +func (c *MockSyncBaseSyncCall) DoAndReturn(f func(context.Context, p2p.Peer, rangesync.KeyBytes, rangesync.KeyBytes) error) *MockSyncBaseSyncCall { c.Call = c.Call.DoAndReturn(f) return c } @@ -361,17 +222,17 @@ func (m *MockSyncKeyHandler) EXPECT() *MockSyncKeyHandlerMockRecorder { } // Commit mocks base method. -func (m *MockSyncKeyHandler) Commit(ctx context.Context, peer p2p.Peer, base, new rangesync.OrderedSet) error { +func (m *MockSyncKeyHandler) Commit(ctx context.Context, peer p2p.Peer, base rangesync.OrderedSet, received rangesync.SeqResult) error { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Commit", ctx, peer, base, new) + ret := m.ctrl.Call(m, "Commit", ctx, peer, base, received) ret0, _ := ret[0].(error) return ret0 } // Commit indicates an expected call of Commit. -func (mr *MockSyncKeyHandlerMockRecorder) Commit(ctx, peer, base, new any) *MockSyncKeyHandlerCommitCall { +func (mr *MockSyncKeyHandlerMockRecorder) Commit(ctx, peer, base, received any) *MockSyncKeyHandlerCommitCall { mr.mock.ctrl.T.Helper() - call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Commit", reflect.TypeOf((*MockSyncKeyHandler)(nil).Commit), ctx, peer, base, new) + call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Commit", reflect.TypeOf((*MockSyncKeyHandler)(nil).Commit), ctx, peer, base, received) return &MockSyncKeyHandlerCommitCall{Call: call} } @@ -387,52 +248,13 @@ func (c *MockSyncKeyHandlerCommitCall) Return(arg0 error) *MockSyncKeyHandlerCom } // Do rewrite *gomock.Call.Do -func (c *MockSyncKeyHandlerCommitCall) Do(f func(context.Context, p2p.Peer, rangesync.OrderedSet, rangesync.OrderedSet) error) *MockSyncKeyHandlerCommitCall { - c.Call = c.Call.Do(f) - return c -} - -// DoAndReturn rewrite *gomock.Call.DoAndReturn -func (c *MockSyncKeyHandlerCommitCall) DoAndReturn(f func(context.Context, p2p.Peer, rangesync.OrderedSet, rangesync.OrderedSet) error) *MockSyncKeyHandlerCommitCall { - c.Call = c.Call.DoAndReturn(f) - return c -} - -// Receive mocks base method. -func (m *MockSyncKeyHandler) Receive(k rangesync.KeyBytes, peer p2p.Peer) (bool, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Receive", k, peer) - ret0, _ := ret[0].(bool) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// Receive indicates an expected call of Receive. -func (mr *MockSyncKeyHandlerMockRecorder) Receive(k, peer any) *MockSyncKeyHandlerReceiveCall { - mr.mock.ctrl.T.Helper() - call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Receive", reflect.TypeOf((*MockSyncKeyHandler)(nil).Receive), k, peer) - return &MockSyncKeyHandlerReceiveCall{Call: call} -} - -// MockSyncKeyHandlerReceiveCall wrap *gomock.Call -type MockSyncKeyHandlerReceiveCall struct { - *gomock.Call -} - -// Return rewrite *gomock.Call.Return -func (c *MockSyncKeyHandlerReceiveCall) Return(arg0 bool, arg1 error) *MockSyncKeyHandlerReceiveCall { - c.Call = c.Call.Return(arg0, arg1) - return c -} - -// Do rewrite *gomock.Call.Do -func (c *MockSyncKeyHandlerReceiveCall) Do(f func(rangesync.KeyBytes, p2p.Peer) (bool, error)) *MockSyncKeyHandlerReceiveCall { +func (c *MockSyncKeyHandlerCommitCall) Do(f func(context.Context, p2p.Peer, rangesync.OrderedSet, rangesync.SeqResult) error) *MockSyncKeyHandlerCommitCall { c.Call = c.Call.Do(f) return c } // DoAndReturn rewrite *gomock.Call.DoAndReturn -func (c *MockSyncKeyHandlerReceiveCall) DoAndReturn(f func(rangesync.KeyBytes, p2p.Peer) (bool, error)) *MockSyncKeyHandlerReceiveCall { +func (c *MockSyncKeyHandlerCommitCall) DoAndReturn(f func(context.Context, p2p.Peer, rangesync.OrderedSet, rangesync.SeqResult) error) *MockSyncKeyHandlerCommitCall { c.Call = c.Call.DoAndReturn(f) return c } diff --git a/sync2/multipeer/multipeer.go b/sync2/multipeer/multipeer.go index 00832edc38..f0b961f9d8 100644 --- a/sync2/multipeer/multipeer.go +++ b/sync2/multipeer/multipeer.go @@ -3,7 +3,6 @@ package multipeer import ( "context" "errors" - "fmt" "math" "math/rand/v2" "sync/atomic" @@ -277,24 +276,17 @@ func (mpr *MultiPeerReconciler) fullSync(ctx context.Context, syncPeers []p2p.Pe var someSucceeded atomic.Bool for _, p := range syncPeers { eg.Go(func() error { - if err := mpr.syncBase.WithPeerSyncer(ctx, p, func(ps PeerSyncer) error { - err := ps.Sync(ctx, nil, nil) - switch { - case err == nil: - someSucceeded.Store(true) - mpr.sl.NoteSync() - case errors.Is(err, context.Canceled): - return err - default: - // failing to sync against a particular peer is not considered - // a fatal sync failure, so we just log the error - mpr.logger.Error("error syncing peer", - zap.Stringer("peer", p), - zap.Error(err)) - } - return nil - }); err != nil { - return fmt.Errorf("sync %s: %w", p, err) + err := mpr.syncBase.Sync(ctx, p, nil, nil) + switch { + case err == nil: + someSucceeded.Store(true) + mpr.sl.NoteSync() + case errors.Is(err, context.Canceled): + return err + default: + // failing to sync against a particular peer is not considered + // a fatal sync failure, so we just log the error + mpr.logger.Error("error syncing peer", zap.Stringer("peer", p), zap.Error(err)) } return nil }) @@ -356,11 +348,6 @@ func (mpr *MultiPeerReconciler) syncOnce(ctx context.Context, lastWasSplit bool) } } - // handler errors are not fatal - if handlerErr := mpr.syncBase.Wait(); handlerErr != nil { - mpr.logger.Error("error handling synced keys", zap.Error(handlerErr)) - } - return full, err } @@ -431,9 +418,6 @@ LOOP: case <-kickCh: } } - // The loop is only exited upon context cancellation. - // Thus, syncBase.Wait() is guaranteed not to block indefinitely here. - mpr.syncBase.Wait() return err } diff --git a/sync2/multipeer/multipeer_test.go b/sync2/multipeer/multipeer_test.go index 4bd05227fe..70061e94f0 100644 --- a/sync2/multipeer/multipeer_test.go +++ b/sync2/multipeer/multipeer_test.go @@ -157,23 +157,17 @@ func (mt *multiPeerSyncTester) expectFullSync(pl *peerList, times, numFails int) // delegate to the real fullsync return mt.reconciler.FullSync(ctx, peers) }) - mt.syncBase.EXPECT().WithPeerSyncer(gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( - func( - _ context.Context, - p p2p.Peer, - toCall func(multipeer.PeerSyncer) error, - ) error { + mt.syncBase.EXPECT(). + Sync(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()). + DoAndReturn(func(ctx context.Context, p p2p.Peer, x, y rangesync.KeyBytes) error { mt.mtx.Lock() defer mt.mtx.Unlock() require.Contains(mt, pl.get(), p) - s := NewMockPeerSyncer(mt.ctrl) - s.EXPECT().Peer().Return(p).AnyTimes() - expSync := s.EXPECT().Sync(gomock.Any(), gomock.Nil(), gomock.Nil()) if numFails != 0 { - expSync.Return(errors.New("sync failed")) numFails-- + return errors.New("sync failed") } - return toCall(s) + return nil }).Times(times) } @@ -210,7 +204,6 @@ func TestMultiPeerSync(t *testing.T) { require.ElementsMatch(t, plSplit.get(), peers) return nil }) - mt.syncBase.EXPECT().Wait() mt.clock.BlockUntilContext(ctx, 1) plFull := mt.expectProbe(numSyncPeers, rangesync.ProbeResult{ FP: "foo", @@ -218,7 +211,6 @@ func TestMultiPeerSync(t *testing.T) { Sim: 1, // after sync }) mt.expectFullSync(plFull, numSyncPeers, 0) - mt.syncBase.EXPECT().Wait() if i > 0 { mt.clock.Advance(time.Minute) } else if i < numSyncs-1 { @@ -226,7 +218,6 @@ func TestMultiPeerSync(t *testing.T) { } mt.satisfy() } - mt.syncBase.EXPECT().Wait() }) t.Run("full sync", func(t *testing.T) { @@ -240,7 +231,6 @@ func TestMultiPeerSync(t *testing.T) { Sim: 0.99, // high enough for full sync }) mt.expectFullSync(pl, numSyncPeers, 0) - mt.syncBase.EXPECT().Wait() } expect() // first full sync happens immediately @@ -254,7 +244,6 @@ func TestMultiPeerSync(t *testing.T) { mt.satisfy() } require.True(t, mt.reconciler.Synced()) - mt.syncBase.EXPECT().Wait() }) t.Run("sync after kick", func(t *testing.T) { @@ -268,7 +257,6 @@ func TestMultiPeerSync(t *testing.T) { Sim: 0.99, // high enough for full sync }) mt.expectFullSync(pl, numSyncPeers, 0) - mt.syncBase.EXPECT().Wait() } expect() // first full sync happens immediately @@ -282,7 +270,6 @@ func TestMultiPeerSync(t *testing.T) { mt.satisfy() } require.True(t, mt.reconciler.Synced()) - mt.syncBase.EXPECT().Wait() }) t.Run("full sync, peers with low count ignored", func(t *testing.T) { @@ -306,7 +293,6 @@ func TestMultiPeerSync(t *testing.T) { Sim: 0.9, }) mt.expectFullSync(&pl, 5, 0) - mt.syncBase.EXPECT().Wait() } expect() // first full sync happens immediately @@ -320,7 +306,6 @@ func TestMultiPeerSync(t *testing.T) { mt.satisfy() } require.True(t, mt.reconciler.Synced()) - mt.syncBase.EXPECT().Wait() }) t.Run("full sync due to low peer count", func(t *testing.T) { @@ -333,7 +318,6 @@ func TestMultiPeerSync(t *testing.T) { Sim: 0.5, // too low for full sync, but will have it anyway }) mt.expectFullSync(pl, 1, 0) - mt.syncBase.EXPECT().Wait() } expect() ctx := mt.start() @@ -346,7 +330,6 @@ func TestMultiPeerSync(t *testing.T) { mt.satisfy() } require.True(t, mt.reconciler.Synced()) - mt.syncBase.EXPECT().Wait() }) t.Run("probe failure", func(t *testing.T) { @@ -357,7 +340,6 @@ func TestMultiPeerSync(t *testing.T) { pl := mt.expectProbe(5, rangesync.ProbeResult{FP: "foo", Count: 100, Sim: 0.99}) // just 5 peers for which the probe worked will be checked mt.expectFullSync(pl, 5, 0) - mt.syncBase.EXPECT().Wait().Times(2) ctx := mt.start() mt.clock.BlockUntilContext(ctx, 1) }) @@ -369,7 +351,6 @@ func TestMultiPeerSync(t *testing.T) { expect := func() { pl := mt.expectProbe(numSyncPeers, rangesync.ProbeResult{FP: "foo", Count: 100, Sim: 0.99}) mt.expectFullSync(pl, numSyncPeers, numFails) - mt.syncBase.EXPECT().Wait() } expect() ctx := mt.start() @@ -382,7 +363,6 @@ func TestMultiPeerSync(t *testing.T) { mt.satisfy() } require.True(t, mt.reconciler.Synced()) - mt.syncBase.EXPECT().Wait() }) t.Run("all peers failed during full sync", func(t *testing.T) { @@ -391,7 +371,6 @@ func TestMultiPeerSync(t *testing.T) { pl := mt.expectProbe(numSyncPeers, rangesync.ProbeResult{FP: "foo", Count: 100, Sim: 0.99}) mt.expectFullSync(pl, numSyncPeers, numSyncPeers) - mt.syncBase.EXPECT().Wait().AnyTimes() ctx := mt.start() mt.clock.BlockUntilContext(ctx, 1) @@ -407,28 +386,6 @@ func TestMultiPeerSync(t *testing.T) { require.True(t, mt.reconciler.Synced()) }) - t.Run("failed synced key handling during full sync", func(t *testing.T) { - mt := newMultiPeerSyncTester(t, 10) - mt.syncBase.EXPECT().Count().Return(100, nil).AnyTimes() - expect := func() { - pl := mt.expectProbe(numSyncPeers, rangesync.ProbeResult{FP: "foo", Count: 100, Sim: 0.99}) - mt.expectFullSync(pl, numSyncPeers, 0) - mt.syncBase.EXPECT().Wait().Return(errors.New("some handlers failed")) - } - expect() - ctx := mt.start() - mt.clock.BlockUntilContext(ctx, 1) - mt.satisfy() - for i := 0; i < numSyncs; i++ { - expect() - mt.clock.Advance(time.Minute) - mt.clock.BlockUntilContext(ctx, 1) - mt.satisfy() - } - require.True(t, mt.reconciler.Synced()) - mt.syncBase.EXPECT().Wait() - }) - t.Run("cancellation during sync", func(t *testing.T) { mt := newMultiPeerSyncTester(t, 10) mt.syncBase.EXPECT().Count().Return(100, nil).AnyTimes() @@ -438,7 +395,6 @@ func TestMultiPeerSync(t *testing.T) { mt.cancel() return ctx.Err() }) - mt.syncBase.EXPECT().Wait().Times(2) ctx := mt.start() mt.clock.BlockUntilContext(ctx, 1) require.ErrorIs(t, mt.eg.Wait(), context.Canceled) diff --git a/sync2/multipeer/setsyncbase.go b/sync2/multipeer/setsyncbase.go index 4c4b95a261..d0b44225d0 100644 --- a/sync2/multipeer/setsyncbase.go +++ b/sync2/multipeer/setsyncbase.go @@ -2,14 +2,10 @@ package multipeer import ( "context" - "errors" "fmt" "io" "sync" - "go.uber.org/zap" - "golang.org/x/sync/singleflight" - "github.com/spacemeshos/go-spacemesh/p2p" "github.com/spacemeshos/go-spacemesh/sync2/rangesync" ) @@ -20,25 +16,20 @@ import ( // has not been yet received and validated. type SetSyncBase struct { mtx sync.Mutex - logger *zap.Logger ps PairwiseSyncer os rangesync.OrderedSet handler SyncKeyHandler - waiting []<-chan singleflight.Result - g singleflight.Group } var _ SyncBase = &SetSyncBase{} // NewSetSyncBase creates a new SetSyncBase. func NewSetSyncBase( - logger *zap.Logger, ps PairwiseSyncer, os rangesync.OrderedSet, handler SyncKeyHandler, ) *SetSyncBase { return &SetSyncBase{ - logger: logger, ps: ps, os: os, handler: handler, @@ -66,133 +57,56 @@ func (ssb *SetSyncBase) Count() (int, error) { return info.Count, nil } -// WithPeerSyncer implements SyncBase. -func (ssb *SetSyncBase) WithPeerSyncer(ctx context.Context, p p2p.Peer, toCall func(PeerSyncer) error) error { - return ssb.os.WithCopy(ctx, func(os rangesync.OrderedSet) error { - return toCall(&peerSetSyncer{ - SetSyncBase: ssb, - OrderedSet: os, - p: p, - handler: ssb.handler, - }) - }) -} - -// Probe implements SyncBase. -func (ssb *SetSyncBase) Probe(ctx context.Context, p p2p.Peer) (pr rangesync.ProbeResult, err error) { - // Use a snapshot of the store to avoid holding the mutex for a long time +func (ssb *SetSyncBase) syncPeer( + ctx context.Context, + p p2p.Peer, + toCall func(rangesync.OrderedSet) error, +) error { + sr := rangesync.EmptySeqResult() + var n int if err := ssb.os.WithCopy(ctx, func(os rangesync.OrderedSet) error { - pr, err = ssb.ps.Probe(ctx, p, os, nil, nil) - if err != nil { - return fmt.Errorf("probing peer %s: %w", p, err) + if err := toCall(os); err != nil { + return err } + sr, n = os.Received() return nil }); err != nil { - return rangesync.ProbeResult{}, fmt.Errorf("using set copy for probe: %w", err) - } - - return pr, nil -} - -func (ssb *SetSyncBase) receiveKey(k rangesync.KeyBytes, p p2p.Peer) error { - ssb.mtx.Lock() - defer ssb.mtx.Unlock() - key := k.String() - has, err := ssb.os.Has(k) - if err != nil { - return fmt.Errorf("checking if the key is present: %w", err) - } - if !has { - ssb.waiting = append(ssb.waiting, - ssb.g.DoChan(key, func() (any, error) { - addToOrig, err := ssb.handler.Receive(k, p) - if err == nil && addToOrig { - ssb.mtx.Lock() - defer ssb.mtx.Unlock() - err = ssb.os.Receive(k) - } - return key, err - })) + return fmt.Errorf("sync: %w", err) } - return nil -} - -// Wait waits for all the handlers used by derived syncers to finish. -func (ssb *SetSyncBase) Wait() error { - // At this point, the derived syncers should be done syncing, and we only want to - // wait for the remaining handlers to complete. In case if some syncers happen to - // be still running at this point, let's not fail too badly. - // TODO: wait for any derived running syncers here, too - ssb.mtx.Lock() - waiting := ssb.waiting - ssb.waiting = nil - ssb.mtx.Unlock() - gotError := false - for _, w := range waiting { - r := <-w - key := r.Val.(string) - ssb.g.Forget(key) - if r.Err != nil { - gotError = true - ssb.logger.Error("error from key handler", zap.String("key", key), zap.Error(r.Err)) + if n > 0 { + if err := ssb.handler.Commit(ctx, p, ssb.os, sr); err != nil { + return fmt.Errorf("commit: %w", err) } } - if gotError { - return errors.New("some key handlers failed") - } - return nil -} - -func (ssb *SetSyncBase) advance() error { ssb.mtx.Lock() defer ssb.mtx.Unlock() return ssb.os.Advance() } -type peerSetSyncer struct { - *SetSyncBase - rangesync.OrderedSet - p p2p.Peer - handler SyncKeyHandler -} - -var ( - _ PeerSyncer = &peerSetSyncer{} - _ rangesync.OrderedSet = &peerSetSyncer{} -) - -// Peer implements Syncer. -func (pss *peerSetSyncer) Peer() p2p.Peer { - return pss.p -} - -// Sync implements Syncer. -func (pss *peerSetSyncer) Sync(ctx context.Context, x, y rangesync.KeyBytes) error { - if err := pss.ps.Sync(ctx, pss.p, pss, x, y); err != nil { - return err - } - return pss.commit(ctx) +func (ssb *SetSyncBase) Sync(ctx context.Context, p p2p.Peer, x, y rangesync.KeyBytes) error { + return ssb.syncPeer(ctx, p, func(os rangesync.OrderedSet) error { + return ssb.ps.Sync(ctx, p, os, x, y) + }) } -// Serve implements Syncer. -func (pss *peerSetSyncer) Serve(ctx context.Context, stream io.ReadWriter) error { - if err := pss.ps.Serve(ctx, stream, pss); err != nil { - return err - } - return pss.commit(ctx) +func (ssb *SetSyncBase) Serve(ctx context.Context, p p2p.Peer, stream io.ReadWriter) error { + return ssb.syncPeer(ctx, p, func(os rangesync.OrderedSet) error { + return ssb.ps.Serve(ctx, stream, os) + }) } -// Receive implements OrderedSet. -func (pss *peerSetSyncer) Receive(k rangesync.KeyBytes) error { - if err := pss.receiveKey(k, pss.p); err != nil { - return err +// Probe implements SyncBase. +func (ssb *SetSyncBase) Probe(ctx context.Context, p p2p.Peer) (pr rangesync.ProbeResult, err error) { + // Use a snapshot of the store to avoid holding the mutex for a long time + if err := ssb.os.WithCopy(ctx, func(os rangesync.OrderedSet) error { + pr, err = ssb.ps.Probe(ctx, p, os, nil, nil) + if err != nil { + return fmt.Errorf("probing peer %s: %w", p, err) + } + return nil + }); err != nil { + return rangesync.ProbeResult{}, fmt.Errorf("using set copy for probe: %w", err) } - return pss.OrderedSet.Receive(k) -} -func (pss *peerSetSyncer) commit(ctx context.Context) error { - if err := pss.handler.Commit(ctx, pss.p, pss.SetSyncBase.os, pss.OrderedSet); err != nil { - return err - } - return pss.SetSyncBase.advance() + return pr, nil } diff --git a/sync2/multipeer/setsyncbase_test.go b/sync2/multipeer/setsyncbase_test.go index 234f454cc9..4a2ae3d365 100644 --- a/sync2/multipeer/setsyncbase_test.go +++ b/sync2/multipeer/setsyncbase_test.go @@ -2,14 +2,10 @@ package multipeer_test import ( "context" - "errors" - "sync" "testing" "github.com/stretchr/testify/require" gomock "go.uber.org/mock/gomock" - "go.uber.org/zap/zaptest" - "golang.org/x/sync/errgroup" "github.com/spacemeshos/go-spacemesh/p2p" "github.com/spacemeshos/go-spacemesh/sync2/multipeer" @@ -24,97 +20,33 @@ type setSyncBaseTester struct { handler *MockSyncKeyHandler os *mocks.MockOrderedSet ssb *multipeer.SetSyncBase - waitMtx sync.Mutex - waitChs map[string]chan error - doneCh chan rangesync.KeyBytes } func newSetSyncBaseTester(t *testing.T, os rangesync.OrderedSet) *setSyncBaseTester { ctrl := gomock.NewController(t) st := &setSyncBaseTester{ - T: t, - ctrl: ctrl, - ps: NewMockPairwiseSyncer(ctrl), - waitChs: make(map[string]chan error), - doneCh: make(chan rangesync.KeyBytes), + T: t, + ctrl: ctrl, + ps: NewMockPairwiseSyncer(ctrl), } if os == nil { st.os = mocks.NewMockOrderedSet(ctrl) - st.os.EXPECT().Items().DoAndReturn(func() rangesync.SeqResult { - return rangesync.EmptySeqResult() - }).AnyTimes() os = st.os } st.handler = NewMockSyncKeyHandler(ctrl) - st.handler.EXPECT().Receive(gomock.Any(), gomock.Any()). - DoAndReturn(func(k rangesync.KeyBytes, p p2p.Peer) (bool, error) { - err := <-st.getWaitCh(k) - st.doneCh <- k - return true, err - }).AnyTimes() - st.ssb = multipeer.NewSetSyncBase(zaptest.NewLogger(t), st.ps, os, st.handler) + st.ssb = multipeer.NewSetSyncBase(st.ps, os, st.handler) return st } -func (st *setSyncBaseTester) getWaitCh(k rangesync.KeyBytes) chan error { - st.waitMtx.Lock() - defer st.waitMtx.Unlock() - ch, found := st.waitChs[string(k)] - if !found { - ch = make(chan error) - st.waitChs[string(k)] = ch - } - return ch -} - -func (st *setSyncBaseTester) expectCopy(addedKeys ...rangesync.KeyBytes) *mocks.MockOrderedSet { +func (st *setSyncBaseTester) expectCopy() *mocks.MockOrderedSet { copy := mocks.NewMockOrderedSet(st.ctrl) st.os.EXPECT().WithCopy(gomock.Any(), gomock.Any()).DoAndReturn( func(_ context.Context, toCall func(rangesync.OrderedSet) error) error { - copy.EXPECT().Items().DoAndReturn(func() rangesync.SeqResult { - return rangesync.EmptySeqResult() - }).AnyTimes() - for _, k := range addedKeys { - copy.EXPECT().Receive(k) - } return toCall(copy) }) return copy } -func (st *setSyncBaseTester) expectSync( - p p2p.Peer, - ss multipeer.PeerSyncer, - addedKeys ...rangesync.KeyBytes, -) { - st.ps.EXPECT().Sync(gomock.Any(), p, ss, nil, nil). - DoAndReturn(func( - _ context.Context, - p p2p.Peer, - os rangesync.OrderedSet, - x, y rangesync.KeyBytes, - ) error { - for _, k := range addedKeys { - require.NoError(st, os.Receive(k)) - } - return nil - }) -} - -func (st *setSyncBaseTester) wait(count int) ([]rangesync.KeyBytes, error) { - var eg errgroup.Group - eg.Go(st.ssb.Wait) - var handledKeys []rangesync.KeyBytes - for k := range st.doneCh { - handledKeys = append(handledKeys, k.Clone()) - count-- - if count == 0 { - break - } - } - return handledKeys, eg.Wait() -} - func TestSetSyncBase(t *testing.T) { t.Run("probe", func(t *testing.T) { t.Parallel() @@ -131,160 +63,56 @@ func TestSetSyncBase(t *testing.T) { require.Equal(t, expPr, pr) }) - t.Run("single key one-time sync", func(t *testing.T) { - t.Parallel() - st := newSetSyncBaseTester(t, nil) - - addedKey := rangesync.RandomKeyBytes(32) - st.expectCopy(addedKey) - require.NoError(t, st.ssb.WithPeerSyncer( - context.Background(), p2p.Peer("p1"), - func(ps multipeer.PeerSyncer) error { - require.Equal(t, p2p.Peer("p1"), ps.Peer()) - - x := rangesync.RandomKeyBytes(32) - y := rangesync.RandomKeyBytes(32) - st.handler.EXPECT().Commit(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()) - st.os.EXPECT().Advance() - st.ps.EXPECT().Sync(gomock.Any(), p2p.Peer("p1"), ps, x, y) - require.NoError(t, ps.Sync(context.Background(), x, y)) - - st.os.EXPECT().Has(addedKey) - st.os.EXPECT().Receive(addedKey) - st.expectSync(p2p.Peer("p1"), ps, addedKey) - st.handler.EXPECT().Commit(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()) - st.os.EXPECT().Advance() - require.NoError(t, ps.Sync(context.Background(), nil, nil)) - close(st.getWaitCh(addedKey)) - return nil - })) - - handledKeys, err := st.wait(1) - require.NoError(t, err) - require.ElementsMatch(t, []rangesync.KeyBytes{addedKey}, handledKeys) - }) - - t.Run("single key synced multiple times", func(t *testing.T) { + t.Run("sync", func(t *testing.T) { t.Parallel() st := newSetSyncBaseTester(t, nil) - addedKey := rangesync.RandomKeyBytes(32) - st.expectCopy(addedKey, addedKey, addedKey) - require.NoError(t, st.ssb.WithPeerSyncer( - context.Background(), p2p.Peer("p1"), - func(ps multipeer.PeerSyncer) error { - require.Equal(t, p2p.Peer("p1"), ps.Peer()) - // added just once - st.os.EXPECT().Receive(addedKey) - for i := 0; i < 3; i++ { - st.os.EXPECT().Has(addedKey) - st.expectSync(p2p.Peer("p1"), ps, addedKey) - st.handler.EXPECT().Commit(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()) - st.os.EXPECT().Advance() - require.NoError(t, ps.Sync(context.Background(), nil, nil)) - } - close(st.getWaitCh(addedKey)) + os := st.expectCopy() + x := rangesync.RandomKeyBytes(32) + y := rangesync.RandomKeyBytes(32) + st.ps.EXPECT().Sync(gomock.Any(), p2p.Peer("p1"), os, x, y) + addedKeys := []rangesync.KeyBytes{rangesync.RandomKeyBytes(32)} + sr := rangesync.MakeSeqResult(addedKeys) + os.EXPECT().Received().Return(sr, 1) + st.handler.EXPECT().Commit(gomock.Any(), p2p.Peer("p1"), st.os, gomock.Any()). + DoAndReturn(func( + _ context.Context, + _ p2p.Peer, + _ rangesync.OrderedSet, + sr rangesync.SeqResult, + ) error { + items, err := sr.Collect() + require.NoError(t, err) + require.ElementsMatch(t, addedKeys, items) return nil - })) - - handledKeys, err := st.wait(1) - require.NoError(t, err) - require.ElementsMatch(t, []rangesync.KeyBytes{addedKey}, handledKeys) + }) + st.os.EXPECT().Advance() + st.ssb.Sync(context.Background(), p2p.Peer("p1"), x, y) }) - t.Run("multiple keys", func(t *testing.T) { + t.Run("count empty", func(t *testing.T) { t.Parallel() st := newSetSyncBaseTester(t, nil) - k1 := rangesync.RandomKeyBytes(32) - k2 := rangesync.RandomKeyBytes(32) - st.expectCopy(k1, k2) - require.NoError(t, st.ssb.WithPeerSyncer( - context.Background(), p2p.Peer("p1"), - func(ps multipeer.PeerSyncer) error { - require.Equal(t, p2p.Peer("p1"), ps.Peer()) - - st.os.EXPECT().Has(k1) - st.os.EXPECT().Has(k2) - st.os.EXPECT().Receive(k1) - st.os.EXPECT().Receive(k2) - st.expectSync(p2p.Peer("p1"), ps, k1, k2) - st.handler.EXPECT().Commit(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()) - st.os.EXPECT().Advance() - require.NoError(t, ps.Sync(context.Background(), nil, nil)) - close(st.getWaitCh(k1)) - close(st.getWaitCh(k2)) - return nil - })) - handledKeys, err := st.wait(2) + st.os.EXPECT().Empty().Return(true, nil) + count, err := st.ssb.Count() require.NoError(t, err) - require.ElementsMatch(t, []rangesync.KeyBytes{k1, k2}, handledKeys) + require.Zero(t, count) }) - t.Run("handler failure", func(t *testing.T) { + t.Run("count non-empty", func(t *testing.T) { t.Parallel() st := newSetSyncBaseTester(t, nil) - k1 := rangesync.RandomKeyBytes(32) - k2 := rangesync.RandomKeyBytes(32) - st.expectCopy(k1, k2) - require.NoError(t, st.ssb.WithPeerSyncer( - context.Background(), p2p.Peer("p1"), - func(ps multipeer.PeerSyncer) error { - require.Equal(t, p2p.Peer("p1"), ps.Peer()) - - st.os.EXPECT().Has(k1) - st.os.EXPECT().Has(k2) - // k1 is not propagated to syncBase due to the handler failure - st.os.EXPECT().Receive(k2) - st.expectSync(p2p.Peer("p1"), ps, k1, k2) - st.handler.EXPECT().Commit(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()) - st.os.EXPECT().Advance() - require.NoError(t, ps.Sync(context.Background(), nil, nil)) - st.getWaitCh(k1) <- errors.New("fail") - close(st.getWaitCh(k2)) - return nil - })) - - handledKeys, err := st.wait(2) - require.ErrorContains(t, err, "some key handlers failed") - require.ElementsMatch(t, []rangesync.KeyBytes{k1, k2}, handledKeys) - }) - - t.Run("real item set", func(t *testing.T) { - t.Parallel() - hs := make([]rangesync.KeyBytes, 4) - for n := range hs { - hs[n] = rangesync.RandomKeyBytes(32) + st.os.EXPECT().Empty().Return(false, nil) + items := []rangesync.KeyBytes{ + rangesync.RandomKeyBytes(32), + rangesync.RandomKeyBytes(32), } - var os rangesync.DumbSet - os.AddUnchecked(hs[0]) - os.AddUnchecked(hs[1]) - st := newSetSyncBaseTester(t, &os) - require.NoError(t, st.ssb.WithPeerSyncer( - context.Background(), p2p.Peer("p1"), - func(ps multipeer.PeerSyncer) error { - ps.(rangesync.OrderedSet).Receive(hs[2]) - ps.(rangesync.OrderedSet).Add(hs[2]) - ps.(rangesync.OrderedSet).Receive(hs[3]) - ps.(rangesync.OrderedSet).Add(hs[3]) - // syncer's cloned set has new key immediately - has, err := ps.(rangesync.OrderedSet).Has(hs[2]) - require.NoError(t, err) - require.True(t, has) - has, err = ps.(rangesync.OrderedSet).Has(hs[3]) - require.NoError(t, err) - require.True(t, has) - return nil - })) - st.getWaitCh(hs[2]) <- errors.New("fail") - close(st.getWaitCh(hs[3])) - handledKeys, err := st.wait(2) - require.ErrorContains(t, err, "some key handlers failed") - require.ElementsMatch(t, hs[2:], handledKeys) - // only successfully handled keys propagate the syncBase - received, err := os.Received().Collect() + st.os.EXPECT().Items().Return(rangesync.MakeSeqResult(items)) + st.os.EXPECT().GetRangeInfo(items[0], items[0]).Return(rangesync.RangeInfo{Count: 2}, nil) + count, err := st.ssb.Count() require.NoError(t, err) - require.ElementsMatch(t, hs[3:], received) + require.Equal(t, 2, count) }) } diff --git a/sync2/multipeer/split_sync.go b/sync2/multipeer/split_sync.go index 9ea1bd437f..d530894ab9 100644 --- a/sync2/multipeer/split_sync.go +++ b/sync2/multipeer/split_sync.go @@ -3,7 +3,6 @@ package multipeer import ( "context" "errors" - "fmt" "slices" "time" @@ -17,8 +16,8 @@ import ( ) type syncResult struct { - ps PeerSyncer - err error + peer p2p.Peer + err error } // splitSync is a synchronization implementation that synchronizes the set against @@ -87,19 +86,14 @@ func (s *splitSync) startPeerSync(ctx context.Context, p p2p.Peer, sr *syncRange s.numRunning++ doneCh := make(chan struct{}) s.eg.Go(func() error { - if err := s.syncBase.WithPeerSyncer(ctx, p, func(ps PeerSyncer) error { - err := ps.Sync(ctx, sr.X, sr.Y) - close(doneCh) - select { - case <-ctx.Done(): - return ctx.Err() - case s.resCh <- syncResult{ps: ps, err: err}: - return nil - } - }); err != nil { - return fmt.Errorf("sync peer %s: %w", p, err) + err := s.syncBase.Sync(ctx, p, sr.X, sr.Y) + close(doneCh) + select { + case <-ctx.Done(): + return ctx.Err() + case s.resCh <- syncResult{peer: p, err: err}: + return nil } - return nil }) gpTimer := s.clock.After(s.gracePeriod) s.eg.Go(func() error { @@ -122,18 +116,18 @@ func (s *splitSync) startPeerSync(ctx context.Context, p p2p.Peer, sr *syncRange } func (s *splitSync) handleSyncResult(r syncResult) error { - sr, found := s.syncMap[r.ps.Peer()] + sr, found := s.syncMap[r.peer] if !found { panic("BUG: error in split sync syncMap handling") } s.numRunning-- - delete(s.syncMap, r.ps.Peer()) + delete(s.syncMap, r.peer) sr.NumSyncers-- if r.err != nil { s.numPeers-- - s.failedPeers[r.ps.Peer()] = struct{}{} + s.failedPeers[r.peer] = struct{}{} s.logger.Debug("remove failed peer", - zap.Stringer("peer", r.ps.Peer()), + zap.Stringer("peer", r.peer), zap.Int("numPeers", s.numPeers), zap.Int("numRemaining", s.numRemaining), zap.Int("numRunning", s.numRunning), @@ -146,20 +140,21 @@ func (s *splitSync) handleSyncResult(r syncResult) error { // sync with no active syncs remaining s.sq.Update(sr, time.Time{}) } - } else { - sr.Done = true - s.syncPeers = append(s.syncPeers, r.ps.Peer()) - s.numRemaining-- - s.logger.Debug("peer synced successfully", - log.ZShortStringer("x", sr.X), - log.ZShortStringer("y", sr.Y), - zap.Stringer("peer", r.ps.Peer()), - zap.Int("numPeers", s.numPeers), - zap.Int("numRemaining", s.numRemaining), - zap.Int("numRunning", s.numRunning), - zap.Int("availPeers", len(s.syncPeers))) + return nil } + sr.Done = true + s.syncPeers = append(s.syncPeers, r.peer) + s.numRemaining-- + s.logger.Debug("peer synced successfully", + log.ZShortStringer("x", sr.X), + log.ZShortStringer("y", sr.Y), + zap.Stringer("peer", r.peer), + zap.Int("numPeers", s.numPeers), + zap.Int("numRemaining", s.numRemaining), + zap.Int("numRunning", s.numRunning), + zap.Int("availPeers", len(s.syncPeers))) + return nil } diff --git a/sync2/multipeer/split_sync_test.go b/sync2/multipeer/split_sync_test.go index 4dbfdb8980..6e1c3bc173 100644 --- a/sync2/multipeer/split_sync_test.go +++ b/sync2/multipeer/split_sync_test.go @@ -23,7 +23,6 @@ import ( type splitSyncTester struct { testing.TB - ctrl *gomock.Controller syncPeers []p2p.Peer clock clockwork.FakeClock mtx sync.Mutex @@ -58,7 +57,6 @@ func newTestSplitSync(t testing.TB) *splitSyncTester { ctrl := gomock.NewController(t) tst := &splitSyncTester{ TB: t, - ctrl: ctrl, syncPeers: make([]p2p.Peer, 4), clock: clockwork.NewFakeClock(), fail: make(map[hexRange]bool), @@ -92,38 +90,27 @@ func newTestSplitSync(t testing.TB) *splitSyncTester { func (tst *splitSyncTester) expectPeerSync(p p2p.Peer) { tst.syncBase.EXPECT(). - WithPeerSyncer(gomock.Any(), p, gomock.Any()). - DoAndReturn(func( - _ context.Context, - peer p2p.Peer, - toCall func(multipeer.PeerSyncer) error, - ) error { - s := NewMockPeerSyncer(tst.ctrl) - s.EXPECT().Peer().Return(p).AnyTimes() - s.EXPECT(). - Sync(gomock.Any(), gomock.Any(), gomock.Any()). - DoAndReturn(func(_ context.Context, x, y rangesync.KeyBytes) error { - tst.mtx.Lock() - defer tst.mtx.Unlock() - require.NotNil(tst, x) - require.NotNil(tst, y) - k := hexRange{x.String(), y.String()} - tst.peerRanges[k] = append(tst.peerRanges[k], peer) - count, found := tst.expPeerRanges[k] - require.True(tst, found, "peer range not found: x %s y %s", x, y) - if tst.fail[k] { - tst.Logf("ERR: peer %s x %s y %s", - string(p), x.String(), y.String()) - tst.fail[k] = false - return errors.New("injected fault") - } else { - tst.Logf("OK: peer %s x %s y %s", - string(p), x.String(), y.String()) - tst.expPeerRanges[k] = count + 1 - } - return nil - }) - return toCall(s) + Sync(gomock.Any(), p, gomock.Any(), gomock.Any()). + DoAndReturn(func(_ context.Context, p p2p.Peer, x, y rangesync.KeyBytes) error { + tst.mtx.Lock() + defer tst.mtx.Unlock() + require.NotNil(tst, x) + require.NotNil(tst, y) + k := hexRange{x.String(), y.String()} + tst.peerRanges[k] = append(tst.peerRanges[k], p) + count, found := tst.expPeerRanges[k] + require.True(tst, found, "peer range not found: x %s y %s", x, y) + if tst.fail[k] { + tst.Logf("ERR: peer %s x %s y %s", + string(p), x.String(), y.String()) + tst.fail[k] = false + return errors.New("injected fault") + } else { + tst.Logf("OK: peer %s x %s y %s", + string(p), x.String(), y.String()) + tst.expPeerRanges[k] = count + 1 + } + return nil }).AnyTimes() } @@ -169,21 +156,10 @@ func TestSplitSync_SlowPeers(t *testing.T) { for _, p := range tst.syncPeers[2:] { tst.syncBase.EXPECT(). - WithPeerSyncer(gomock.Any(), p, gomock.Any()). - DoAndReturn(func( - _ context.Context, - peer p2p.Peer, - toCall func(multipeer.PeerSyncer) error, - ) error { - s := NewMockPeerSyncer(tst.ctrl) - s.EXPECT().Peer().Return(p).AnyTimes() - s.EXPECT(). - Sync(gomock.Any(), gomock.Any(), gomock.Any()). - DoAndReturn(func(ctx context.Context, x, y rangesync.KeyBytes) error { - <-ctx.Done() - return nil - }) - return toCall(s) + Sync(gomock.Any(), p, gomock.Any(), gomock.Any()). + DoAndReturn(func(ctx context.Context, p p2p.Peer, x, y rangesync.KeyBytes) error { + <-ctx.Done() + return nil }) } diff --git a/sync2/p2p.go b/sync2/p2p.go index e43155155b..0f173d85eb 100644 --- a/sync2/p2p.go +++ b/sync2/p2p.go @@ -4,7 +4,6 @@ import ( "context" "errors" "fmt" - "io" "sync" "sync/atomic" "time" @@ -13,7 +12,6 @@ import ( "golang.org/x/sync/errgroup" "github.com/spacemeshos/go-spacemesh/fetch/peers" - "github.com/spacemeshos/go-spacemesh/p2p" "github.com/spacemeshos/go-spacemesh/sync2/multipeer" "github.com/spacemeshos/go-spacemesh/sync2/rangesync" ) @@ -81,22 +79,14 @@ func NewP2PHashSync( enableActiveSync: enableActiveSync, } ps := rangesync.NewPairwiseSetSyncer(logger, d, name, cfg.RangeSetReconcilerConfig) - s.syncBase = multipeer.NewSetSyncBase(logger, ps, s.os, handler) + s.syncBase = multipeer.NewSetSyncBase(ps, s.os, handler) s.reconciler = multipeer.NewMultiPeerReconciler( logger, cfg.MultiPeerReconcilerConfig, s.syncBase, peers, keyLen, cfg.MaxDepth) - d.Register(name, s.serve) + d.Register(name, s.syncBase.Serve) return s } -func (s *P2PHashSync) serve(ctx context.Context, peer p2p.Peer, stream io.ReadWriter) error { - // We derive a dedicated Syncer for the peer being served to pass all the received - // items through the handler before adding them to the main OrderedSet. - return s.syncBase.WithPeerSyncer(ctx, peer, func(syncer multipeer.PeerSyncer) error { - return syncer.Serve(ctx, stream) - }) -} - // Set returns the OrderedSet that is being synchronized. func (s *P2PHashSync) Set() rangesync.OrderedSet { return s.os diff --git a/sync2/p2p_test.go b/sync2/p2p_test.go index dd2d1ba7d2..61197aeb96 100644 --- a/sync2/p2p_test.go +++ b/sync2/p2p_test.go @@ -21,40 +21,22 @@ import ( "github.com/spacemeshos/go-spacemesh/sync2/rangesync" ) -type addedKey struct { - // The fields are actually used to make sure each key is synced just once between - // each pair of peers. - //nolint:unused - fromPeer, toPeer p2p.Peer - //nolint:unused - key string -} - type fakeHandler struct { - mtx *sync.Mutex - localPeerID p2p.Peer - synced map[addedKey]struct{} - committed map[string]struct{} -} - -func (fh *fakeHandler) Receive(k rangesync.KeyBytes, peer p2p.Peer) (bool, error) { - fh.mtx.Lock() - defer fh.mtx.Unlock() - ak := addedKey{ - toPeer: fh.localPeerID, - key: string(k), - } - fh.synced[ak] = struct{}{} - return true, nil + mtx *sync.Mutex + committed map[string]struct{} } -func (fh *fakeHandler) Commit(ctx context.Context, peer p2p.Peer, base, new rangesync.OrderedSet) error { +func (fh *fakeHandler) Commit( + ctx context.Context, + peer p2p.Peer, + base rangesync.OrderedSet, + received rangesync.SeqResult, +) error { fh.mtx.Lock() defer fh.mtx.Unlock() - for k := range fh.synced { - fh.committed[k.key] = struct{}{} + for k := range received.Seq { + fh.committed[string(k)] = struct{}{} } - clear(fh.synced) return nil } @@ -100,10 +82,8 @@ func TestP2P(t *testing.T) { cfg.MaxDepth = maxDepth host := mesh.Hosts()[n] handlers[n] = &fakeHandler{ - mtx: &mtx, - localPeerID: host.ID(), - synced: make(map[addedKey]struct{}), - committed: make(map[string]struct{}), + mtx: &mtx, + committed: make(map[string]struct{}), } var os rangesync.DumbSet d := rangesync.NewDispatcher(logger) diff --git a/sync2/rangesync/dumbset.go b/sync2/rangesync/dumbset.go index 5c4abfc782..3d67d1aac8 100644 --- a/sync2/rangesync/dumbset.go +++ b/sync2/rangesync/dumbset.go @@ -141,7 +141,7 @@ func (ds *DumbSet) AddUnchecked(id KeyBytes) { // AddReceived adds all the received items to the set. func (ds *DumbSet) AddReceived() { - sr := ds.Received() + sr, _ := ds.Received() for k := range sr.Seq { ds.AddUnchecked(KeyBytes(k)) } @@ -187,7 +187,7 @@ func (ds *DumbSet) Receive(id KeyBytes) error { } // Received implements the OrderedSet. -func (ds *DumbSet) Received() SeqResult { +func (ds *DumbSet) Received() (SeqResult, int) { return SeqResult{ Seq: func(yield func(KeyBytes) bool) { for k := range ds.received { @@ -197,7 +197,7 @@ func (ds *DumbSet) Received() SeqResult { } }, Error: NoSeqError, - } + }, len(ds.received) } // seq returns an endless sequence as a SeqResult starting from the given index. diff --git a/sync2/rangesync/interface.go b/sync2/rangesync/interface.go index 517efc18ad..9a941ee352 100644 --- a/sync2/rangesync/interface.go +++ b/sync2/rangesync/interface.go @@ -31,6 +31,10 @@ type SplitInfo struct { // OrderedSet represents the set that can be synced against a remote peer. // OrderedSet methods are non-threadsafe except for WithCopy, Loaded and EnsureLoaded. +// SeqResult values obtained by method calls on an OrderedSet passed to WithCopy +// callback are valid only within the callback and should not be used outside of it, +// with exception of SeqResult returned by Received, which is expected to be valid +// outside of the callback as well. type OrderedSet interface { // Add adds a new key to the set. // It should not perform any additional actions related to handling @@ -39,8 +43,12 @@ type OrderedSet interface { // Receive handles a new key received from the peer. // It should not add the key to the set. Receive(k KeyBytes) error - // Received returns the sequence containing all the items received from the peer. - Received() SeqResult + // Received returns the sequence containing all the items received from the peer, + // and the total number of received items. + // Unlike other methods, SeqResult returned by Received called on a copy of the + // OrderedSet passed to WithCopy callback is expected to be valid outside of the + // callback as well. + Received() (SeqResult, int) // GetRangeInfo returns RangeInfo for the item range in the ordered set, // bounded by [x, y). // x == y indicates the whole set. diff --git a/sync2/rangesync/mocks/mocks.go b/sync2/rangesync/mocks/mocks.go index 304aa1d9a4..7201c59d66 100644 --- a/sync2/rangesync/mocks/mocks.go +++ b/sync2/rangesync/mocks/mocks.go @@ -388,11 +388,12 @@ func (c *MockOrderedSetReceiveCall) DoAndReturn(f func(rangesync.KeyBytes) error } // Received mocks base method. -func (m *MockOrderedSet) Received() rangesync.SeqResult { +func (m *MockOrderedSet) Received() (rangesync.SeqResult, int) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Received") ret0, _ := ret[0].(rangesync.SeqResult) - return ret0 + ret1, _ := ret[1].(int) + return ret0, ret1 } // Received indicates an expected call of Received. @@ -408,19 +409,19 @@ type MockOrderedSetReceivedCall struct { } // Return rewrite *gomock.Call.Return -func (c *MockOrderedSetReceivedCall) Return(arg0 rangesync.SeqResult) *MockOrderedSetReceivedCall { - c.Call = c.Call.Return(arg0) +func (c *MockOrderedSetReceivedCall) Return(arg0 rangesync.SeqResult, arg1 int) *MockOrderedSetReceivedCall { + c.Call = c.Call.Return(arg0, arg1) return c } // Do rewrite *gomock.Call.Do -func (c *MockOrderedSetReceivedCall) Do(f func() rangesync.SeqResult) *MockOrderedSetReceivedCall { +func (c *MockOrderedSetReceivedCall) Do(f func() (rangesync.SeqResult, int)) *MockOrderedSetReceivedCall { c.Call = c.Call.Do(f) return c } // DoAndReturn rewrite *gomock.Call.DoAndReturn -func (c *MockOrderedSetReceivedCall) DoAndReturn(f func() rangesync.SeqResult) *MockOrderedSetReceivedCall { +func (c *MockOrderedSetReceivedCall) DoAndReturn(f func() (rangesync.SeqResult, int)) *MockOrderedSetReceivedCall { c.Call = c.Call.DoAndReturn(f) return c } diff --git a/sync2/rangesync/seq.go b/sync2/rangesync/seq.go index f1695114cc..86be62772b 100644 --- a/sync2/rangesync/seq.go +++ b/sync2/rangesync/seq.go @@ -129,7 +129,7 @@ func (s SeqResult) Collect() ([]KeyBytes, error) { func EmptySeqResult() SeqResult { return SeqResult{ Seq: EmptySeq(), - Error: func() error { return nil }, + Error: NoSeqError, } } @@ -140,3 +140,20 @@ func ErrorSeqResult(err error) SeqResult { Error: SeqError(err), } } + +// MakeSeqResult makes a SeqResult out of a slice. +// The sequence is made cyclic, starting over after the last element. +func MakeSeqResult(items []KeyBytes) SeqResult { + return SeqResult{ + Seq: func(yield func(k KeyBytes) bool) { + for { + for _, item := range items { + if !yield(item) { + return + } + } + } + }, + Error: NoSeqError, + } +}