diff --git a/activation/activation.go b/activation/activation.go index 3c2315b3b0..67044a1968 100644 --- a/activation/activation.go +++ b/activation/activation.go @@ -11,6 +11,7 @@ import ( "github.com/spacemeshos/post/shared" "go.uber.org/zap" + "golang.org/x/exp/maps" "golang.org/x/sync/errgroup" "github.com/spacemeshos/go-spacemesh/activation/metrics" @@ -65,32 +66,30 @@ type Config struct { // it is responsible for initializing post, receiving poet proof and orchestrating nipst. after which it will // calculate total weight and providing relevant view as proof. type Builder struct { - eg errgroup.Group - - signer *signing.EdSigner - accountLock sync.RWMutex - coinbaseAccount types.Address - goldenATXID types.ATXID - regossipInterval time.Duration - cdb *datastore.CachedDB - localDB *localsql.Database - publisher pubsub.Publisher - nipostBuilder nipostBuilder - validator nipostValidator - - // smeshingMutex protects `StartSmeshing` and `StopSmeshing` from concurrent access - smeshingMutex sync.Mutex - started bool - + accountLock sync.RWMutex + coinbaseAccount types.Address + goldenATXID types.ATXID + regossipInterval time.Duration + cdb *datastore.CachedDB + localDB *localsql.Database + publisher pubsub.Publisher + nipostBuilder nipostBuilder + validator nipostValidator layerClock layerClock syncer syncer log *zap.Logger parentCtx context.Context - stop context.CancelFunc poetCfg PoetConfig poetRetryInterval time.Duration // delay before PoST in ATX is considered valid (counting from the time it was received) postValidityDelay time.Duration + + // smeshingMutex protects methods like `StartSmeshing` and `StopSmeshing` from concurrent execution + // since they (can) modify the fields below. + smeshingMutex sync.Mutex + signers map[types.NodeID]*signing.EdSigner + eg errgroup.Group + stop context.CancelFunc } type BuilderOption func(*Builder) @@ -132,7 +131,6 @@ func WithValidator(v nipostValidator) BuilderOption { // NewBuilder returns an atx builder that will start a routine that will attempt to create an atx upon each new layer. func NewBuilder( conf Config, - signer *signing.EdSigner, cdb *datastore.CachedDB, localDB *localsql.Database, publisher pubsub.Publisher, @@ -144,7 +142,7 @@ func NewBuilder( ) *Builder { b := &Builder{ parentCtx: context.Background(), - signer: signer, + signers: make(map[types.NodeID]*signing.EdSigner), goldenATXID: conf.GoldenATXID, regossipInterval: conf.RegossipInterval, cdb: cdb, @@ -163,11 +161,27 @@ func NewBuilder( return b } +func (b *Builder) Register(sig *signing.EdSigner) { + b.smeshingMutex.Lock() + defer b.smeshingMutex.Unlock() + if _, exists := b.signers[sig.NodeID()]; exists { + b.log.Error("signing key already registered", zap.Stringer("id", sig.NodeID())) + return + } + + b.log.Info("registered signing key", zap.Stringer("id", sig.NodeID())) + b.signers[sig.NodeID()] = sig + + if b.stop != nil { + b.startID(b.parentCtx, sig) + } +} + // Smeshing returns true iff atx builder is smeshing. func (b *Builder) Smeshing() bool { b.smeshingMutex.Lock() defer b.smeshingMutex.Unlock() - return b.started + return b.stop != nil } // StartSmeshing is the main entry point of the atx builder. It runs the main @@ -180,82 +194,96 @@ func (b *Builder) StartSmeshing(coinbase types.Address) error { b.smeshingMutex.Lock() defer b.smeshingMutex.Unlock() - if b.started { + if b.stop != nil { return errors.New("already started") } - b.started = true b.coinbaseAccount = coinbase ctx, stop := context.WithCancel(b.parentCtx) b.stop = stop + for _, sig := range b.signers { + b.startID(ctx, sig) + } + return nil +} + +func (b *Builder) startID(ctx context.Context, sig *signing.EdSigner) { b.eg.Go(func() error { - b.run(ctx) + b.run(ctx, sig) return nil }) - if b.regossipInterval != 0 { - b.eg.Go(func() error { - ticker := time.NewTicker(b.regossipInterval) - defer ticker.Stop() - for { - select { - case <-ctx.Done(): - return ctx.Err() - case <-ticker.C: - if err := b.Regossip(ctx); err != nil { - b.log.Warn("failed to re-gossip", zap.Error(err)) - } + if b.regossipInterval == 0 { + return + } + b.eg.Go(func() error { + ticker := time.NewTicker(b.regossipInterval) + defer ticker.Stop() + for { + select { + case <-ctx.Done(): + return ctx.Err() + case <-ticker.C: + if err := b.Regossip(ctx, sig.NodeID()); err != nil { + b.log.Warn("failed to re-gossip", zap.Error(err)) } } - }) - } - return nil + } + }) } // StopSmeshing stops the atx builder. -// It doesn't wait for the smeshing to stop. func (b *Builder) StopSmeshing(deleteFiles bool) error { b.smeshingMutex.Lock() defer b.smeshingMutex.Unlock() - if !b.started { + if b.stop == nil { return errors.New("not started") } b.stop() err := b.eg.Wait() - b.started = false + b.eg = errgroup.Group{} + b.stop = nil switch { case err == nil || errors.Is(err, context.Canceled): if !deleteFiles { return nil } - if err := b.nipostBuilder.ResetState(); err != nil { - b.log.Error("failed to delete builder state", zap.Error(err)) - return err - } - if err := nipost.RemoveChallenge(b.localDB, b.signer.NodeID()); err != nil { - b.log.Error("failed to remove nipost challenge", zap.Error(err)) - return err + var resetErr error + for _, sig := range b.signers { + if err := b.nipostBuilder.ResetState(sig.NodeID()); err != nil { + b.log.Error("failed to reset builder state", log.ZShortStringer("nodeId", sig.NodeID()), zap.Error(err)) + err = fmt.Errorf("reset builder state for id %s: %w", sig.NodeID().ShortString(), err) + resetErr = errors.Join(resetErr, err) + continue + } + if err := nipost.RemoveChallenge(b.localDB, sig.NodeID()); err != nil { + b.log.Error("failed to remove nipost challenge", zap.Error(err)) + err = fmt.Errorf("remove nipost challenge for id %s: %w", sig.NodeID().ShortString(), err) + resetErr = errors.Join(resetErr, err) + } } - return nil + return resetErr default: return fmt.Errorf("failed to stop smeshing: %w", err) } } // SmesherID returns the ID of the smesher that created this activation. -func (b *Builder) SmesherID() types.NodeID { - return b.signer.NodeID() +func (b *Builder) SmesherIDs() []types.NodeID { + b.smeshingMutex.Lock() + defer b.smeshingMutex.Unlock() + return maps.Keys(b.signers) } -func (b *Builder) buildInitialPost(ctx context.Context) error { +func (b *Builder) buildInitialPost(ctx context.Context, nodeId types.NodeID) error { // Generate the initial POST if we don't have an ATX... - if _, err := b.cdb.GetLastAtx(b.signer.NodeID()); err == nil { + if _, err := b.cdb.GetLastAtx(nodeId); err == nil { return nil } // ...and if we haven't stored an initial post yet. - _, err := nipost.InitialPost(b.localDB, b.signer.NodeID()) + _, err := nipost.InitialPost(b.localDB, nodeId) switch { case err == nil: b.log.Info("load initial post from db") @@ -268,7 +296,7 @@ func (b *Builder) buildInitialPost(ctx context.Context) error { // Create the initial post and save it. startTime := time.Now() - post, postInfo, err := b.nipostBuilder.Proof(ctx, shared.ZeroChallenge) + post, postInfo, err := b.nipostBuilder.Proof(ctx, nodeId, shared.ZeroChallenge) if err != nil { return fmt.Errorf("post execution: %w", err) } @@ -285,14 +313,14 @@ func (b *Builder) buildInitialPost(ctx context.Context) error { CommitmentATX: postInfo.CommitmentATX, VRFNonce: *postInfo.Nonce, } - return nipost.AddInitialPost(b.localDB, b.signer.NodeID(), initialPost) + return nipost.AddInitialPost(b.localDB, nodeId, initialPost) } -func (b *Builder) run(ctx context.Context) { +func (b *Builder) run(ctx context.Context, sig *signing.EdSigner) { defer b.log.Info("atx builder stopped") for { - err := b.buildInitialPost(ctx) + err := b.buildInitialPost(ctx, sig.NodeID()) if err == nil { break } @@ -306,7 +334,7 @@ func (b *Builder) run(ctx context.Context) { } for { - err := b.PublishActivationTx(ctx) + err := b.PublishActivationTx(ctx, sig) if err == nil { continue } else if errors.Is(err, context.Canceled) { @@ -318,10 +346,10 @@ func (b *Builder) run(ctx context.Context) { switch { case errors.Is(err, ErrATXChallengeExpired): b.log.Debug("retrying with new challenge after waiting for a layer") - if err := b.nipostBuilder.ResetState(); err != nil { + if err := b.nipostBuilder.ResetState(sig.NodeID()); err != nil { b.log.Error("failed to reset nipost builder state", zap.Error(err)) } - if err := nipost.RemoveChallenge(b.localDB, b.signer.NodeID()); err != nil { + if err := nipost.RemoveChallenge(b.localDB, sig.NodeID()); err != nil { b.log.Error("failed to discard challenge", zap.Error(err)) } // give node some time to sync in case selecting the positioning ATX caused the challenge to expire @@ -351,7 +379,7 @@ func (b *Builder) run(ctx context.Context) { } } -func (b *Builder) buildNIPostChallenge(ctx context.Context) (*types.NIPostChallenge, error) { +func (b *Builder) buildNIPostChallenge(ctx context.Context, nodeID types.NodeID) (*types.NIPostChallenge, error) { select { case <-ctx.Done(): return nil, ctx.Err() @@ -359,7 +387,7 @@ func (b *Builder) buildNIPostChallenge(ctx context.Context) (*types.NIPostChalle } current := b.layerClock.CurrentLayer().GetEpoch() - challenge, err := nipost.Challenge(b.localDB, b.signer.NodeID()) + challenge, err := nipost.Challenge(b.localDB, nodeID) switch { case errors.Is(err, sql.ErrNotFound): // build new challenge @@ -367,10 +395,10 @@ func (b *Builder) buildNIPostChallenge(ctx context.Context) (*types.NIPostChalle return nil, fmt.Errorf("get nipost challenge: %w", err) case challenge.PublishEpoch < current: // challenge is stale - if err := b.nipostBuilder.ResetState(); err != nil { + if err := b.nipostBuilder.ResetState(nodeID); err != nil { return nil, fmt.Errorf("reset nipost builder state: %w", err) } - if err := nipost.RemoveChallenge(b.localDB, b.signer.NodeID()); err != nil { + if err := nipost.RemoveChallenge(b.localDB, nodeID); err != nil { return nil, fmt.Errorf("remove stale nipost challenge: %w", err) } default: @@ -378,7 +406,7 @@ func (b *Builder) buildNIPostChallenge(ctx context.Context) (*types.NIPostChalle return challenge, nil } - prev, err := b.cdb.GetLastAtx(b.signer.NodeID()) + prev, err := b.cdb.GetLastAtx(nodeID) switch { case err == nil: current = max(current, prev.PublishEpoch) @@ -410,16 +438,16 @@ func (b *Builder) buildNIPostChallenge(ctx context.Context) (*types.NIPostChalle } } - posAtx, err := b.getPositioningAtx(ctx) + posAtx, err := b.getPositioningAtx(ctx, nodeID) if err != nil { return nil, fmt.Errorf("failed to get positioning ATX: %w", err) } - prevAtx, err := b.cdb.GetLastAtx(b.signer.NodeID()) + prevAtx, err := b.cdb.GetLastAtx(nodeID) switch { case errors.Is(err, sql.ErrNotFound): // initial ATX challenge - post, err := nipost.InitialPost(b.localDB, b.signer.NodeID()) + post, err := nipost.InitialPost(b.localDB, nodeID) if err != nil { return nil, fmt.Errorf("get initial post: %w", err) } @@ -447,7 +475,7 @@ func (b *Builder) buildNIPostChallenge(ctx context.Context) (*types.NIPostChalle } } - if err := nipost.AddChallenge(b.localDB, b.signer.NodeID(), challenge); err != nil { + if err := nipost.AddChallenge(b.localDB, nodeID, challenge); err != nil { return nil, fmt.Errorf("add nipost challenge: %w", err) } return challenge, nil @@ -469,8 +497,8 @@ func (b *Builder) Coinbase() types.Address { } // PublishActivationTx attempts to publish an atx, it returns an error if an atx cannot be created. -func (b *Builder) PublishActivationTx(ctx context.Context) error { - challenge, err := b.buildNIPostChallenge(ctx) +func (b *Builder) PublishActivationTx(ctx context.Context, sig *signing.EdSigner) error { + challenge, err := b.buildNIPostChallenge(ctx, sig.NodeID()) if err != nil { return err } @@ -482,7 +510,7 @@ func (b *Builder) PublishActivationTx(ctx context.Context) error { ) ctx, cancel := context.WithDeadline(ctx, b.layerClock.LayerToTime((challenge.TargetEpoch()).FirstLayer())) defer cancel() - atx, err := b.createAtx(ctx, challenge) + atx, err := b.createAtx(ctx, sig, challenge) if err != nil { return fmt.Errorf("create ATX: %w", err) } @@ -502,10 +530,10 @@ func (b *Builder) PublishActivationTx(ctx context.Context) error { } } - if err := b.nipostBuilder.ResetState(); err != nil { + if err := b.nipostBuilder.ResetState(sig.NodeID()); err != nil { return fmt.Errorf("reset nipost builder state: %w", err) } - if err := nipost.RemoveChallenge(b.localDB, b.signer.NodeID()); err != nil { + if err := nipost.RemoveChallenge(b.localDB, sig.NodeID()); err != nil { return fmt.Errorf("discarding challenge after published ATX: %w", err) } events.EmitAtxPublished( @@ -520,10 +548,14 @@ func (b *Builder) poetRoundStart(epoch types.EpochID) time.Time { return b.layerClock.LayerToTime(epoch.FirstLayer()).Add(b.poetCfg.PhaseShift) } -func (b *Builder) createAtx(ctx context.Context, challenge *types.NIPostChallenge) (*types.ActivationTx, error) { +func (b *Builder) createAtx( + ctx context.Context, + sig *signing.EdSigner, + challenge *types.NIPostChallenge, +) (*types.ActivationTx, error) { pubEpoch := challenge.PublishEpoch - nipostState, err := b.nipostBuilder.BuildNIPost(ctx, challenge) + nipostState, err := b.nipostBuilder.BuildNIPost(ctx, sig, challenge) if err != nil { return nil, fmt.Errorf("build NIPost: %w", err) } @@ -532,6 +564,7 @@ func (b *Builder) createAtx(ctx context.Context, challenge *types.NIPostChalleng zap.Stringer("pub_epoch", pubEpoch), zap.Stringer("pub_epoch_first_layer", pubEpoch.FirstLayer()), zap.Stringer("current_layer", b.layerClock.CurrentLayer()), + zap.Stringer("node_id", sig.NodeID()), ) select { case <-ctx.Done(): @@ -549,14 +582,14 @@ func (b *Builder) createAtx(ctx context.Context, challenge *types.NIPostChalleng } var nonce *types.VRFPostIndex - var nodeID *types.NodeID + var atxNodeID *types.NodeID switch { case challenge.PrevATXID == types.EmptyATXID: - nodeID = new(types.NodeID) - *nodeID = b.signer.NodeID() + atxNodeID = new(types.NodeID) + *atxNodeID = sig.NodeID() nonce = &nipostState.VRFNonce default: - oldNonce, err := atxs.VRFNonce(b.cdb, b.signer.NodeID(), challenge.PublishEpoch) + oldNonce, err := atxs.VRFNonce(b.cdb, sig.NodeID(), challenge.PublishEpoch) if err != nil { b.log.Warn("failed to get VRF nonce for ATX", zap.Error(err)) break @@ -573,8 +606,8 @@ func (b *Builder) createAtx(ctx context.Context, challenge *types.NIPostChalleng nipostState.NumUnits, nonce, ) - atx.InnerActivationTx.NodeID = nodeID - if err = SignAndFinalizeAtx(b.signer, atx); err != nil { + atx.InnerActivationTx.NodeID = atxNodeID + if err = SignAndFinalizeAtx(sig, atx); err != nil { return nil, fmt.Errorf("sign atx: %w", err) } return atx, nil @@ -592,16 +625,16 @@ func (b *Builder) broadcast(ctx context.Context, atx *types.ActivationTx) (int, } // getPositioningAtx returns atx id with the highest tick height. -func (b *Builder) getPositioningAtx(ctx context.Context) (types.ATXID, error) { +func (b *Builder) getPositioningAtx(ctx context.Context, nodeID types.NodeID) (types.ATXID, error) { id, err := findFullyValidHighTickAtx( ctx, b.cdb, - b.signer.NodeID(), + nodeID, b.goldenATXID, b.validator, b.log, VerifyChainOpts.AssumeValidBefore(time.Now().Add(-b.postValidityDelay)), - VerifyChainOpts.WithTrustedID(b.signer.NodeID()), + VerifyChainOpts.WithTrustedID(nodeID), VerifyChainOpts.WithLogger(b.log), ) if errors.Is(err, sql.ErrNotFound) { @@ -611,15 +644,15 @@ func (b *Builder) getPositioningAtx(ctx context.Context) (types.ATXID, error) { return id, err } -func (b *Builder) Regossip(ctx context.Context) error { +func (b *Builder) Regossip(ctx context.Context, nodeID types.NodeID) error { epoch := b.layerClock.CurrentLayer().GetEpoch() - atx, err := atxs.GetIDByEpochAndNodeID(b.cdb, epoch, b.signer.NodeID()) + atx, err := atxs.GetIDByEpochAndNodeID(b.cdb, epoch, nodeID) if errors.Is(err, sql.ErrNotFound) { return nil } else if err != nil { return err } - blob, err := atxs.GetBlob(b.cdb, atx[:]) + blob, err := atxs.GetBlob(b.cdb, atx.Bytes()) if err != nil { return fmt.Errorf("get blob %s: %w", atx.ShortString(), err) } @@ -629,7 +662,7 @@ func (b *Builder) Regossip(ctx context.Context) error { if err := b.publisher.Publish(ctx, pubsub.AtxProtocol, blob); err != nil { return fmt.Errorf("republish %s: %w", atx.ShortString(), err) } - b.log.Debug("regossipped atx", log.ZShortStringer("atx", atx)) + b.log.Debug("re-gossipped atx", log.ZShortStringer("atx", atx)) return nil } diff --git a/activation/activation_multi_test.go b/activation/activation_multi_test.go new file mode 100644 index 0000000000..38d7fe265d --- /dev/null +++ b/activation/activation_multi_test.go @@ -0,0 +1,480 @@ +package activation + +import ( + "context" + "math/rand" + "sync" + "testing" + "time" + + "github.com/spacemeshos/post/shared" + "github.com/stretchr/testify/require" + "go.uber.org/mock/gomock" + "golang.org/x/sync/errgroup" + + "github.com/spacemeshos/go-spacemesh/codec" + "github.com/spacemeshos/go-spacemesh/common/types" + "github.com/spacemeshos/go-spacemesh/p2p/pubsub" + "github.com/spacemeshos/go-spacemesh/sql" + "github.com/spacemeshos/go-spacemesh/sql/atxs" + "github.com/spacemeshos/go-spacemesh/sql/localsql/nipost" +) + +func Test_Builder_Multi_StartSmeshingCoinbase(t *testing.T) { + tab := newTestBuilder(t, 5) + coinbase := types.Address{1, 1, 1} + + for _, sig := range tab.signers { + tab.mnipost.EXPECT().Proof(gomock.Any(), sig.NodeID(), shared.ZeroChallenge).DoAndReturn( + func(ctx context.Context, _ types.NodeID, _ []byte) (*types.Post, *types.PostInfo, error) { + <-ctx.Done() + return nil, nil, ctx.Err() + }) + } + tab.mclock.EXPECT().CurrentLayer().Return(types.LayerID(0)).AnyTimes() + tab.mclock.EXPECT().AwaitLayer(gomock.Any()).Return(make(chan struct{})).AnyTimes() + require.NoError(t, tab.StartSmeshing(coinbase)) + require.Equal(t, coinbase, tab.Coinbase()) + + // calling StartSmeshing more than once before calling StopSmeshing is an error + require.ErrorContains(t, tab.StartSmeshing(coinbase), "already started") + + for _, sig := range tab.signers { + tab.mnipost.EXPECT().ResetState(sig.NodeID()).Return(nil) + } + require.NoError(t, tab.StopSmeshing(true)) +} + +func Test_Builder_Multi_RestartSmeshing(t *testing.T) { + getBuilder := func(t *testing.T) *Builder { + tab := newTestBuilder(t, 5) + + for _, sig := range tab.signers { + tab.mnipost.EXPECT().Proof(gomock.Any(), sig.NodeID(), shared.ZeroChallenge).AnyTimes().DoAndReturn( + func(ctx context.Context, _ types.NodeID, _ []byte) (*types.Post, *types.PostInfo, error) { + <-ctx.Done() + return nil, nil, ctx.Err() + }) + } + + ch := make(chan struct{}) + close(ch) + tab.mclock.EXPECT().AwaitLayer(gomock.Any()).Return(ch).AnyTimes() + tab.mclock.EXPECT().CurrentLayer().Return(types.LayerID(0)).AnyTimes() + return tab.Builder + } + + t.Run("Single threaded", func(t *testing.T) { + builder := getBuilder(t) + for i := 0; i < 50; i++ { + require.NoError(t, builder.StartSmeshing(types.Address{})) + require.True(t, builder.Smeshing()) + require.NoError(t, builder.StopSmeshing(false)) + require.False(t, builder.Smeshing()) + } + }) + + t.Run("Multi threaded", func(t *testing.T) { + // Meant to be run with -race to detect races. + // It cannot check `builder.Smeshing()` as Start/Stop is happening from many goroutines simultaneously. + // Both Start and Stop can fail as it is not known if builder is smeshing or not. + builder := getBuilder(t) + var eg errgroup.Group + for worker := 0; worker < 10; worker += 1 { + eg.Go(func() error { + for i := 0; i < 50; i++ { + builder.StartSmeshing(types.Address{}) + builder.StopSmeshing(false) + } + return nil + }) + } + require.NoError(t, eg.Wait()) + }) +} + +func Test_Builder_Multi_StopSmeshing_Delete(t *testing.T) { + numIds := 5 + tab := newTestBuilder(t, numIds) + + atx := types.RandomATXID() + refChallenge := &types.NIPostChallenge{ + PublishEpoch: postGenesisEpoch + 2, + CommitmentATX: &atx, + } + + currLayer := (postGenesisEpoch + 1).FirstLayer() + tab.mclock.EXPECT().CurrentLayer().Return(currLayer).Times(numIds) + tab.mclock.EXPECT().AwaitLayer(gomock.Any()).Return(make(chan struct{})).Times(numIds) + + for _, sig := range tab.signers { + tab.mnipost.EXPECT().Proof(gomock.Any(), sig.NodeID(), shared.ZeroChallenge).DoAndReturn( + func(ctx context.Context, _ types.NodeID, _ []byte) (*types.Post, *types.PostInfo, error) { + <-ctx.Done() + return nil, nil, ctx.Err() + }) + + // add challenge to DB + require.NoError(t, nipost.AddChallenge(tab.localDb, sig.NodeID(), refChallenge)) + } + + require.NoError(t, tab.StartSmeshing(types.Address{})) + require.NoError(t, tab.StopSmeshing(false)) + require.True(t, tab.mctrl.Satisfied(), "failed to assert all mocks were called the expected number of times") + + tab.mclock.EXPECT().CurrentLayer().Return(currLayer).Times(numIds) + tab.mclock.EXPECT().AwaitLayer(gomock.Any()).Return(make(chan struct{})).Times(numIds) + + for _, sig := range tab.signers { + challenge, err := nipost.Challenge(tab.localDb, sig.NodeID()) + require.NoError(t, err) + require.Equal(t, refChallenge, challenge) // challenge still present + + tab.mnipost.EXPECT().ResetState(sig.NodeID()).Return(nil) + tab.mnipost.EXPECT().Proof(gomock.Any(), sig.NodeID(), shared.ZeroChallenge).DoAndReturn( + func(ctx context.Context, _ types.NodeID, _ []byte) (*types.Post, *types.PostInfo, error) { + <-ctx.Done() + return nil, nil, ctx.Err() + }) + } + + require.NoError(t, tab.StartSmeshing(types.Address{})) + require.NoError(t, tab.StopSmeshing(true)) + require.True(t, tab.mctrl.Satisfied(), "failed to assert all mocks were called the expected number of times") + + tab.mclock.EXPECT().CurrentLayer().Return(currLayer).Times(numIds) + tab.mclock.EXPECT().AwaitLayer(gomock.Any()).Return(make(chan struct{})).Times(numIds) + + for _, sig := range tab.signers { + challenge, err := nipost.Challenge(tab.localDb, sig.NodeID()) + require.ErrorIs(t, err, sql.ErrNotFound) + require.Nil(t, challenge) // challenge deleted + + tab.mnipost.EXPECT().ResetState(sig.NodeID()).Return(nil) + tab.mnipost.EXPECT().Proof(gomock.Any(), sig.NodeID(), shared.ZeroChallenge).DoAndReturn( + func(ctx context.Context, _ types.NodeID, _ []byte) (*types.Post, *types.PostInfo, error) { + <-ctx.Done() + return nil, nil, ctx.Err() + }) + } + + require.NoError(t, tab.StartSmeshing(types.Address{})) + require.NoError(t, tab.StopSmeshing(true)) // no-op + require.True(t, tab.mctrl.Satisfied(), "failed to assert all mocks were called the expected number of times") + + for _, sig := range tab.signers { + challenge, err := nipost.Challenge(tab.localDb, sig.NodeID()) + require.ErrorIs(t, err, sql.ErrNotFound) + require.Nil(t, challenge) // challenge still deleted + } +} + +func TestRegossip(t *testing.T) { + layer := types.LayerID(10) + + t.Run("not found", func(t *testing.T) { + tab := newTestBuilder(t, 5) + for _, sig := range tab.signers { + tab.mclock.EXPECT().CurrentLayer().Return(layer) + require.NoError(t, tab.Regossip(context.Background(), sig.NodeID())) + } + }) + + t.Run("success", func(t *testing.T) { + tab := newTestBuilder(t, 5) + var refAtx *types.VerifiedActivationTx + + for _, sig := range tab.signers { + atx := newActivationTx(t, + sig, 0, types.EmptyATXID, types.EmptyATXID, nil, + layer.GetEpoch(), 0, 1, types.Address{}, 1, &types.NIPost{}) + require.NoError(t, atxs.Add(tab.cdb, atx)) + + if refAtx == nil { + refAtx = atx + } + } + + blob, err := atxs.GetBlob(tab.cdb, refAtx.ID().Bytes()) + require.NoError(t, err) + + // atx will be regossiped once (by the smesher) + tab.mclock.EXPECT().CurrentLayer().Return(layer) + ctx := context.Background() + tab.mpub.EXPECT().Publish(ctx, pubsub.AtxProtocol, blob) + require.NoError(t, tab.Regossip(ctx, refAtx.SmesherID)) + }) + + t.Run("checkpointed", func(t *testing.T) { + tab := newTestBuilder(t, 5) + for _, sig := range tab.signers { + atx := atxs.CheckpointAtx{ + ID: types.RandomATXID(), + Epoch: layer.GetEpoch(), + SmesherID: sig.NodeID(), + } + require.NoError(t, atxs.AddCheckpointed(tab.cdb, &atx)) + tab.mclock.EXPECT().CurrentLayer().Return(layer) + require.NoError(t, tab.Regossip(context.Background(), sig.NodeID())) + } + }) +} + +func Test_Builder_Multi_InitialPost(t *testing.T) { + tab := newTestBuilder(t, 5, WithPoetConfig(PoetConfig{PhaseShift: layerDuration * 4})) + var eg errgroup.Group + for _, sig := range tab.signers { + sig := sig + eg.Go(func() error { + tab.mnipost.EXPECT().Proof(gomock.Any(), sig.NodeID(), shared.ZeroChallenge).Return( + &types.Post{Indices: make([]byte, 10)}, + &types.PostInfo{ + CommitmentATX: types.RandomATXID(), + Nonce: new(types.VRFPostIndex), + }, + nil, + ) + + require.NoError(t, tab.buildInitialPost(context.Background(), sig.NodeID())) + + // postClient.Proof() should not be called again + require.NoError(t, tab.buildInitialPost(context.Background(), sig.NodeID())) + return nil + }) + } + + eg.Wait() +} + +func Test_Builder_Multi_HappyPath(t *testing.T) { + layerDuration := 2 * time.Second + tab := newTestBuilder(t, 3, WithPoetConfig(PoetConfig{PhaseShift: layerDuration * 4, CycleGap: layerDuration})) + tab.regossipInterval = 0 // disable regossip for testing + + // step 1: build initial posts + initialPostChan := make(chan struct{}) + initialPostStep := make(map[types.NodeID]chan struct{}) + initialPost := make(map[types.NodeID]*nipost.Post) + for _, sig := range tab.signers { + ch := make(chan struct{}) + initialPostStep[sig.NodeID()] = ch + + nipost := nipost.Post{ + Indices: types.RandomBytes(10), + Nonce: rand.Uint32(), + Pow: rand.Uint64(), + + NumUnits: 4, + CommitmentATX: types.RandomATXID(), + VRFNonce: types.VRFPostIndex(rand.Uint64()), + } + initialPost[sig.NodeID()] = &nipost + + tab.mnipost.EXPECT().Proof(gomock.Any(), sig.NodeID(), shared.ZeroChallenge).DoAndReturn( + func(ctx context.Context, _ types.NodeID, _ []byte) (*types.Post, *types.PostInfo, error) { + <-initialPostChan + close(ch) + post := &types.Post{ + Indices: nipost.Indices, + Nonce: nipost.Nonce, + Pow: nipost.Pow, + } + postInfo := &types.PostInfo{ + NumUnits: nipost.NumUnits, + CommitmentATX: nipost.CommitmentATX, + Nonce: &nipost.VRFNonce, + } + + return post, postInfo, nil + }, + ) + } + + // step 2: build nipost challenge + nipostChallengeChan := make(chan struct{}) + nipostChallengeStep := make(map[types.NodeID]chan struct{}) + poetRoundEnd := time.Now().Add(1 * time.Second).Add(-tab.poetCfg.PhaseShift) // poetRoundEnd is in 100ms + for _, sig := range tab.signers { + ch := make(chan struct{}) + nipostChallengeStep[sig.NodeID()] = ch + + tab.mclock.EXPECT().CurrentLayer().DoAndReturn( + func() types.LayerID { + <-nipostChallengeChan + return postGenesisEpoch.FirstLayer() + 1 + }, + ) + + // called twice per id + tab.mclock.EXPECT().LayerToTime(postGenesisEpoch.FirstLayer()).Return(poetRoundEnd).Times(2) + + // logged once per id + tab.mclock.EXPECT().CurrentLayer().DoAndReturn( + func() types.LayerID { + close(ch) + return postGenesisEpoch.FirstLayer() + 1 + }, + ) + } + + // step 3: create ATX + nipostChan := make(chan struct{}) + nipostStep := make(map[types.NodeID]chan struct{}) + nipostState := make(map[types.NodeID]*nipost.NIPostState) + for _, sig := range tab.signers { + ch := make(chan struct{}) + nipostStep[sig.NodeID()] = ch + + // deadline for create ATX + tab.mclock.EXPECT().LayerToTime(postGenesisEpoch.Add(2).FirstLayer()).DoAndReturn( + func(_ types.LayerID) time.Time { + <-nipostChan + return time.Now().Add(5 * time.Second) + }, + ) + + post := &types.Post{ + Indices: initialPost[sig.NodeID()].Indices, + Nonce: initialPost[sig.NodeID()].Nonce, + Pow: initialPost[sig.NodeID()].Pow, + } + ref := &types.NIPostChallenge{ + PublishEpoch: postGenesisEpoch + 1, + CommitmentATX: &initialPost[sig.NodeID()].CommitmentATX, + Sequence: 0, + PrevATXID: types.EmptyATXID, + PositioningATX: tab.goldenATXID, + InitialPost: post, + } + + state := &nipost.NIPostState{ + NIPost: &types.NIPost{ + Membership: types.MerkleProof{}, + Post: &types.Post{ + Indices: types.RandomBytes(10), + Nonce: rand.Uint32(), + Pow: rand.Uint64(), + }, + PostMetadata: &types.PostMetadata{ + LabelsPerUnit: 128, + Challenge: shared.ZeroChallenge, + }, + }, + NumUnits: 4, + VRFNonce: types.VRFPostIndex(rand.Uint64()), + } + nipostState[sig.NodeID()] = state + tab.mnipost.EXPECT().BuildNIPost(gomock.Any(), sig, ref).Return(state, nil) + + // awaiting atx publication epoch log + tab.mclock.EXPECT().CurrentLayer().DoAndReturn( + func() types.LayerID { + close(ch) + return postGenesisEpoch.Add(1).FirstLayer() + }, + ) + } + + // step 4: build and broadcast atx + atxChan := make(chan struct{}) + atxStep := make(map[types.NodeID]chan struct{}) + var atxMtx sync.Mutex + atxs := make(map[types.NodeID]types.ActivationTx) + endChan := make(chan struct{}) + for _, sig := range tab.signers { + ch := make(chan struct{}) + atxStep[sig.NodeID()] = ch + + tab.mclock.EXPECT().AwaitLayer(postGenesisEpoch.Add(1).FirstLayer()).DoAndReturn( + func(_ types.LayerID) <-chan struct{} { + <-atxChan + ch := make(chan struct{}) + close(ch) + return ch + }, + ) + tab.mclock.EXPECT().CurrentLayer().Return(postGenesisEpoch.Add(1).FirstLayer()) + + tab.mpub.EXPECT().Publish(gomock.Any(), pubsub.AtxProtocol, gomock.Any()).DoAndReturn( + func(ctx context.Context, _ string, got []byte) error { + atxMtx.Lock() + defer atxMtx.Unlock() + var gotAtx types.ActivationTx + require.NoError(t, codec.Decode(got, &gotAtx)) + atxs[gotAtx.SmesherID] = gotAtx + return nil + }, + ) + + // shutdown builder + tab.mnipost.EXPECT().ResetState(sig.NodeID()).DoAndReturn( + func(_ types.NodeID) error { + close(ch) + <-endChan + return context.Canceled + }, + ) + } + + // start smeshing + require.NoError(t, tab.StartSmeshing(types.Address{})) + + close(initialPostChan) // signal initial post to complete + for id, ch := range initialPostStep { + select { + case <-ch: + case <-time.After(5 * time.Second): + require.FailNowf(t, "timed out waiting for initial post", "node %s", id) + } + } + + close(nipostChallengeChan) + for id, ch := range nipostChallengeStep { + select { + case <-ch: + case <-time.After(5 * time.Second): + require.FailNowf(t, "timed out waiting for nipost challenge", "node %s", id) + } + } + + close(nipostChan) + for id, ch := range nipostStep { + select { + case <-ch: + case <-time.After(5 * time.Second): + require.FailNowf(t, "timed out waiting for nipost", "node %s", id) + } + } + + close(atxChan) + for id, ch := range atxStep { + select { + case <-ch: + case <-time.After(5 * time.Second): + require.FailNowf(t, "timed out waiting for atx publication", "node %s", id) + } + } + close(endChan) + + for _, sig := range tab.signers { + atx := atxs[sig.NodeID()] + require.Equal(t, initialPost[sig.NodeID()].Nonce, atx.NIPostChallenge.InitialPost.Nonce) + require.Equal(t, initialPost[sig.NodeID()].Pow, atx.NIPostChallenge.InitialPost.Pow) + require.Equal(t, initialPost[sig.NodeID()].Indices, atx.NIPostChallenge.InitialPost.Indices) + + require.Equal(t, initialPost[sig.NodeID()].CommitmentATX, *atx.NIPostChallenge.CommitmentATX) + require.Equal(t, postGenesisEpoch+1, atx.NIPostChallenge.PublishEpoch) + require.Equal(t, types.EmptyATXID, atx.NIPostChallenge.PrevATXID) + require.Equal(t, tab.goldenATXID, atx.NIPostChallenge.PositioningATX) + require.Equal(t, uint64(0), atx.NIPostChallenge.Sequence) + + require.Equal(t, types.Address{}, atx.Coinbase) + require.Equal(t, nipostState[sig.NodeID()].NumUnits, atx.NumUnits) + require.Equal(t, nipostState[sig.NodeID()].NIPost, atx.NIPost) + require.Equal(t, sig.NodeID(), *atx.NodeID) + require.Equal(t, nipostState[sig.NodeID()].VRFNonce, *atx.VRFNonce) + } + + // stop smeshing + require.NoError(t, tab.StopSmeshing(false)) +} diff --git a/activation/activation_test.go b/activation/activation_test.go index c329a6cbca..20f182a239 100644 --- a/activation/activation_test.go +++ b/activation/activation_test.go @@ -13,6 +13,8 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "go.uber.org/mock/gomock" + "go.uber.org/zap/zapcore" + "golang.org/x/exp/maps" "golang.org/x/sync/errgroup" "github.com/spacemeshos/go-spacemesh/codec" @@ -46,24 +48,7 @@ func TestMain(m *testing.M) { // ========== Helper functions ========== -func newChallenge( - sequence uint64, - prevAtxID, posAtxID types.ATXID, - PublishEpoch types.EpochID, - cATX *types.ATXID, -) types.NIPostChallenge { - return types.NIPostChallenge{ - Sequence: sequence, - PrevATXID: prevAtxID, - PublishEpoch: PublishEpoch, - PositioningATX: posAtxID, - CommitmentATX: cATX, - } -} - func newAtx( - t testing.TB, - sig *signing.EdSigner, challenge types.NIPostChallenge, nipost *types.NIPost, numUnits uint32, @@ -77,7 +62,7 @@ func newAtx( } func newActivationTx( - t testing.TB, + tb testing.TB, sig *signing.EdSigner, sequence uint64, prevATX types.ATXID, @@ -89,8 +74,14 @@ func newActivationTx( numUnits uint32, nipost *types.NIPost, ) *types.VerifiedActivationTx { - challenge := newChallenge(sequence, prevATX, positioningATX, publishEpoch, cATX) - atx := newAtx(t, sig, challenge, nipost, numUnits, coinbase) + challenge := types.NIPostChallenge{ + Sequence: sequence, + PrevATXID: prevATX, + PublishEpoch: publishEpoch, + PositioningATX: positioningATX, + CommitmentATX: cATX, + } + atx := newAtx(challenge, nipost, numUnits, coinbase) if sequence == 0 { nodeID := sig.NodeID() atx.NodeID = &nodeID @@ -98,9 +89,9 @@ func newActivationTx( atx.SetEffectiveNumUnits(numUnits) atx.SetReceived(time.Now()) - require.NoError(t, SignAndFinalizeAtx(sig, atx)) + require.NoError(tb, SignAndFinalizeAtx(sig, atx)) vAtx, err := atx.Verify(startTick, numTicks) - require.NoError(t, err) + require.NoError(tb, err) return vAtx } @@ -108,9 +99,9 @@ type testAtxBuilder struct { *Builder cdb *datastore.CachedDB localDb *localsql.Database - sig *signing.EdSigner goldenATXID types.ATXID + mctrl *gomock.Controller mpub *mocks.MockPublisher mnipost *MocknipostBuilder mpostClient *MockPostClient @@ -119,16 +110,16 @@ type testAtxBuilder struct { mValidator *MocknipostValidator } -func newTestBuilder(tb testing.TB, opts ...BuilderOption) *testAtxBuilder { - lg := logtest.New(tb) - edSigner, err := signing.NewEdSigner() - require.NoError(tb, err) +func newTestBuilder(tb testing.TB, numSigners int, opts ...BuilderOption) *testAtxBuilder { + lg := logtest.New(tb, zapcore.DebugLevel) + ctrl := gomock.NewController(tb) tab := &testAtxBuilder{ cdb: datastore.NewCachedDB(sql.InMemory(), lg), - localDb: localsql.InMemory(), - sig: edSigner, + localDb: localsql.InMemory(sql.WithConnections(numSigners)), goldenATXID: types.ATXID(types.HexToHash32("77777")), + + mctrl: ctrl, mpub: mocks.NewMockPublisher(ctrl), mnipost: NewMocknipostBuilder(ctrl), mpostClient: NewMockPostClient(ctrl), @@ -147,7 +138,6 @@ func newTestBuilder(tb testing.TB, opts ...BuilderOption) *testAtxBuilder { b := NewBuilder( cfg, - tab.sig, tab.cdb, tab.localDb, tab.mpub, @@ -158,43 +148,29 @@ func newTestBuilder(tb testing.TB, opts ...BuilderOption) *testAtxBuilder { opts..., ) tab.Builder = b - return tab -} -func assertLastAtx( - r *require.Assertions, - nodeID types.NodeID, - poetRef types.Hash32, - newAtx *types.ActivationTx, - posAtx, prevAtx *types.VerifiedActivationTx, - layersPerEpoch uint32, -) { - atx := newAtx - r.Equal(nodeID, atx.SmesherID) - if prevAtx != nil { - r.Equal(prevAtx.Sequence+1, atx.Sequence) - r.Equal(prevAtx.ID(), atx.PrevATXID) - r.Nil(atx.InitialPost) - r.Nil(atx.VRFNonce) - } else { - r.Zero(atx.Sequence) - r.Equal(types.EmptyATXID, atx.PrevATXID) - r.NotNil(atx.InitialPost) - r.NotNil(atx.VRFNonce) + for i := 0; i < numSigners; i++ { + sig, err := signing.NewEdSigner() + require.NoError(tb, err) + tab.Register(sig) } - r.Equal(posAtx.ID(), atx.PositioningATX) - r.Equal(posAtx.PublishEpoch+1, atx.PublishEpoch) - r.Equal(poetRef, atx.GetPoetProofRef()) + + return tab } func publishAtx( - t *testing.T, + tb testing.TB, tab *testAtxBuilder, + nodeID types.NodeID, posEpoch types.EpochID, currLayer *types.LayerID, // pointer to keep current layer consistent across calls buildNIPostLayerDuration uint32, ) (*types.ActivationTx, error) { - t.Helper() + tb.Helper() + + if _, ok := tab.signers[nodeID]; !ok { + return nil, fmt.Errorf("node %v not registered", nodeID) + } publishEpoch := posEpoch + 1 tab.mclock.EXPECT().LayerToTime(gomock.Any()).DoAndReturn( @@ -206,17 +182,17 @@ func publishAtx( nonce := types.VRFPostIndex(123) commitmentATX := types.RandomATXID() tab.mpostClient.EXPECT().Info(gomock.Any()).Return(&types.PostInfo{ - NodeID: tab.sig.NodeID(), + NodeID: nodeID, CommitmentATX: commitmentATX, Nonce: &nonce, NumUnits: DefaultPostSetupOpts().NumUnits, LabelsPerUnit: DefaultPostConfig().LabelsPerUnit, }, nil).AnyTimes() - tab.mnipost.EXPECT().BuildNIPost(gomock.Any(), gomock.Any()).DoAndReturn( - func(_ context.Context, challenge *types.NIPostChallenge) (*nipost.NIPostState, error) { + tab.mnipost.EXPECT().BuildNIPost(gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( + func(_ context.Context, _ *signing.EdSigner, challenge *types.NIPostChallenge) (*nipost.NIPostState, error) { *currLayer = currLayer.Add(buildNIPostLayerDuration) - return newNIPostWithChallenge(t, challenge.Hash(), []byte("66666")), nil + return newNIPostWithChallenge(tb, challenge.Hash(), []byte("66666")), nil }) ch := make(chan struct{}) close(ch) @@ -232,50 +208,37 @@ func publishAtx( tab.mpub.EXPECT().Publish(gomock.Any(), pubsub.AtxProtocol, gomock.Any()).DoAndReturn( func(_ context.Context, _ string, got []byte) error { var gotAtx types.ActivationTx - require.NoError(t, codec.Decode(got, &gotAtx)) + require.NoError(tb, codec.Decode(got, &gotAtx)) gotAtx.SetReceived(time.Now().Local()) built = &gotAtx - require.NoError(t, built.Initialize()) + require.NoError(tb, built.Initialize()) built.SetEffectiveNumUnits(gotAtx.NumUnits) vatx, err := built.Verify(0, 1) - require.NoError(t, err) - require.NoError(t, atxs.Add(tab.cdb, vatx)) + require.NoError(tb, err) + require.NoError(tb, atxs.Add(tab.cdb, vatx)) return nil }) - tab.mnipost.EXPECT().ResetState().Return(nil) + tab.mnipost.EXPECT().ResetState(nodeID).Return(nil) // Expect verification of positioning ATX candidate chain. tab.mValidator.EXPECT().VerifyChain(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()) // create and publish ATX - err := tab.PublishActivationTx(context.Background()) + err := tab.PublishActivationTx(context.Background(), tab.signers[nodeID]) return built, err } -func addAtx(t *testing.T, db sql.Executor, sig *signing.EdSigner, atx *types.ActivationTx) *types.VerifiedActivationTx { - require.NoError(t, SignAndFinalizeAtx(sig, atx)) - atx.SetEffectiveNumUnits(atx.NumUnits) - atx.SetReceived(time.Now()) - vAtx, err := atx.Verify(0, 1) - require.NoError(t, err) - require.NoError(t, atxs.Add(db, vAtx)) - return vAtx -} - // ========== Tests ========== -func TestBuilder_StartSmeshingCoinbase(t *testing.T) { - tab := newTestBuilder(t) +func Test_Builder_StartSmeshingCoinbase(t *testing.T) { + tab := newTestBuilder(t, 1) + sig := maps.Values(tab.signers)[0] coinbase := types.Address{1, 1, 1} - tab.mnipost.EXPECT().Proof(gomock.Any(), shared.ZeroChallenge).DoAndReturn( - func(ctx context.Context, b []byte) (*types.Post, *types.PostInfo, error) { + tab.mnipost.EXPECT().Proof(gomock.Any(), sig.NodeID(), shared.ZeroChallenge).DoAndReturn( + func(ctx context.Context, _ types.NodeID, _ []byte) (*types.Post, *types.PostInfo, error) { <-ctx.Done() return nil, nil, ctx.Err() }) - tab.mValidator.EXPECT(). - Post(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()). - AnyTimes(). - Return(nil) tab.mclock.EXPECT().CurrentLayer().Return(types.LayerID(0)).AnyTimes() tab.mclock.EXPECT().AwaitLayer(gomock.Any()).Return(make(chan struct{})).AnyTimes() require.NoError(t, tab.StartSmeshing(coinbase)) @@ -284,15 +247,17 @@ func TestBuilder_StartSmeshingCoinbase(t *testing.T) { // calling StartSmeshing more than once before calling StopSmeshing is an error require.ErrorContains(t, tab.StartSmeshing(coinbase), "already started") - tab.mnipost.EXPECT().ResetState().Return(nil) + tab.mnipost.EXPECT().ResetState(sig.NodeID()).Return(nil) require.NoError(t, tab.StopSmeshing(true)) } func TestBuilder_RestartSmeshing(t *testing.T) { getBuilder := func(t *testing.T) *Builder { - tab := newTestBuilder(t) - tab.mnipost.EXPECT().Proof(gomock.Any(), shared.ZeroChallenge).AnyTimes().DoAndReturn( - func(ctx context.Context, b []byte) (*types.Post, *types.PostInfo, error) { + tab := newTestBuilder(t, 1) + sig := maps.Values(tab.signers)[0] + + tab.mnipost.EXPECT().Proof(gomock.Any(), sig.NodeID(), shared.ZeroChallenge).AnyTimes().DoAndReturn( + func(ctx context.Context, _ types.NodeID, _ []byte) (*types.Post, *types.PostInfo, error) { <-ctx.Done() return nil, nil, ctx.Err() }) @@ -334,95 +299,114 @@ func TestBuilder_RestartSmeshing(t *testing.T) { } func TestBuilder_StopSmeshing_Delete(t *testing.T) { - tab := newTestBuilder(t) + tab := newTestBuilder(t, 1) + sig := maps.Values(tab.signers)[0] + + atx := types.RandomATXID() + refChallenge := &types.NIPostChallenge{ + PublishEpoch: postGenesisEpoch + 2, + CommitmentATX: &atx, + } currLayer := (postGenesisEpoch + 1).FirstLayer() - tab.mclock.EXPECT().AwaitLayer(gomock.Any()).Return(make(chan struct{})).AnyTimes() - tab.mclock.EXPECT().CurrentLayer().Return(currLayer).AnyTimes() - tab.mclock.EXPECT().LayerToTime(gomock.Any()).DoAndReturn( - func(got types.LayerID) time.Time { - // time.Now() ~= currentLayer - genesis := time.Now().Add(-time.Duration(currLayer) * layerDuration) - return genesis.Add(layerDuration * time.Duration(got)) - }).AnyTimes() - tab.mnipost.EXPECT(). - BuildNIPost(gomock.Any(), gomock.Any()). - DoAndReturn(func(ctx context.Context, _ *types.NIPostChallenge) (*nipost.NIPostState, error) { - <-ctx.Done() - return nil, ctx.Err() - }). - AnyTimes() - tab.mnipost.EXPECT().Proof(gomock.Any(), shared.ZeroChallenge).DoAndReturn( - func(ctx context.Context, b []byte) (*types.Post, *types.PostInfo, error) { + tab.mclock.EXPECT().CurrentLayer().Return(currLayer) + tab.mclock.EXPECT().AwaitLayer(gomock.Any()).Return(make(chan struct{})) + + tab.mnipost.EXPECT().Proof(gomock.Any(), sig.NodeID(), shared.ZeroChallenge).DoAndReturn( + func(ctx context.Context, _ types.NodeID, _ []byte) (*types.Post, *types.PostInfo, error) { <-ctx.Done() return nil, nil, ctx.Err() - }).AnyTimes() + }) // add challenge to DB - refChallenge := &types.NIPostChallenge{ - PublishEpoch: postGenesisEpoch + 2, - CommitmentATX: &types.ATXID{1, 2, 3}, - } - err := nipost.AddChallenge(tab.localDb, tab.sig.NodeID(), refChallenge) - require.NoError(t, err) + require.NoError(t, nipost.AddChallenge(tab.localDb, sig.NodeID(), refChallenge)) require.NoError(t, tab.StartSmeshing(types.Address{})) require.NoError(t, tab.StopSmeshing(false)) + require.True(t, tab.mctrl.Satisfied(), "failed to assert all mocks were called the expected number of times") + + tab.mclock.EXPECT().CurrentLayer().Return(currLayer) + tab.mclock.EXPECT().AwaitLayer(gomock.Any()).Return(make(chan struct{})) - challenge, err := nipost.Challenge(tab.localDb, tab.sig.NodeID()) + challenge, err := nipost.Challenge(tab.localDb, sig.NodeID()) require.NoError(t, err) require.Equal(t, refChallenge, challenge) // challenge still present - tab.mnipost.EXPECT().ResetState().Return(nil) + tab.mnipost.EXPECT().ResetState(sig.NodeID()).Return(nil) + tab.mnipost.EXPECT().Proof(gomock.Any(), sig.NodeID(), shared.ZeroChallenge).DoAndReturn( + func(ctx context.Context, _ types.NodeID, _ []byte) (*types.Post, *types.PostInfo, error) { + <-ctx.Done() + return nil, nil, ctx.Err() + }) + require.NoError(t, tab.StartSmeshing(types.Address{})) require.NoError(t, tab.StopSmeshing(true)) + require.True(t, tab.mctrl.Satisfied(), "failed to assert all mocks were called the expected number of times") + + tab.mclock.EXPECT().CurrentLayer().Return(currLayer) + tab.mclock.EXPECT().AwaitLayer(gomock.Any()).Return(make(chan struct{})) - challenge, err = nipost.Challenge(tab.localDb, tab.sig.NodeID()) + challenge, err = nipost.Challenge(tab.localDb, sig.NodeID()) require.ErrorIs(t, err, sql.ErrNotFound) require.Nil(t, challenge) // challenge deleted - tab.mnipost.EXPECT().ResetState().Return(nil) + tab.mnipost.EXPECT().ResetState(sig.NodeID()).Return(nil) + tab.mnipost.EXPECT().Proof(gomock.Any(), sig.NodeID(), shared.ZeroChallenge).DoAndReturn( + func(ctx context.Context, _ types.NodeID, _ []byte) (*types.Post, *types.PostInfo, error) { + <-ctx.Done() + return nil, nil, ctx.Err() + }) require.NoError(t, tab.StartSmeshing(types.Address{})) require.NoError(t, tab.StopSmeshing(true)) // no-op + require.True(t, tab.mctrl.Satisfied(), "failed to assert all mocks were called the expected number of times") - challenge, err = nipost.Challenge(tab.localDb, tab.sig.NodeID()) + challenge, err = nipost.Challenge(tab.localDb, sig.NodeID()) require.ErrorIs(t, err, sql.ErrNotFound) require.Nil(t, challenge) // challenge still deleted } func TestBuilder_StopSmeshing_failsWhenNotStarted(t *testing.T) { - tab := newTestBuilder(t) + tab := newTestBuilder(t, 1) require.ErrorContains(t, tab.StopSmeshing(true), "not started") } func TestBuilder_PublishActivationTx_HappyFlow(t *testing.T) { - tab := newTestBuilder(t, WithPoetConfig(PoetConfig{PhaseShift: layerDuration})) + tab := newTestBuilder(t, 1, WithPoetConfig(PoetConfig{PhaseShift: layerDuration})) + sig := maps.Values(tab.signers)[0] + posEpoch := postGenesisEpoch currLayer := posEpoch.FirstLayer() - ch := newChallenge(1, types.EmptyATXID, tab.goldenATXID, posEpoch, &tab.goldenATXID) + ch := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: types.EmptyATXID, + PublishEpoch: posEpoch, + PositioningATX: tab.goldenATXID, + CommitmentATX: &tab.goldenATXID, + } nipostData := newNIPostWithChallenge(t, types.HexToHash32("55555"), []byte("66666")) - prevAtx := newAtx(t, tab.sig, ch, nipostData.NIPost, 2, types.Address{}) - require.NoError(t, SignAndFinalizeAtx(tab.sig, prevAtx)) + prevAtx := newAtx(ch, nipostData.NIPost, 2, types.Address{}) + require.NoError(t, SignAndFinalizeAtx(sig, prevAtx)) vPrevAtx, err := prevAtx.Verify(0, 1) require.NoError(t, err) require.NoError(t, atxs.Add(tab.cdb, vPrevAtx)) // create and publish ATX tab.mclock.EXPECT().CurrentLayer().Return(currLayer).Times(4) - atx1, err := publishAtx(t, tab, posEpoch, &currLayer, layersPerEpoch) + atx1, err := publishAtx(t, tab, sig.NodeID(), posEpoch, &currLayer, layersPerEpoch) require.NoError(t, err) require.NotNil(t, atx1) // create and publish another ATX currLayer = (posEpoch + 1).FirstLayer() tab.mclock.EXPECT().CurrentLayer().Return(currLayer).Times(4) - atx2, err := publishAtx(t, tab, atx1.PublishEpoch, &currLayer, layersPerEpoch) + atx2, err := publishAtx(t, tab, sig.NodeID(), atx1.PublishEpoch, &currLayer, layersPerEpoch) require.NoError(t, err) + require.NotNil(t, atx2) require.NotEqual(t, atx1, atx2) require.Equal(t, atx1.TargetEpoch()+1, atx2.TargetEpoch()) // state is cleaned up - _, err = nipost.Challenge(tab.localDB, tab.sig.NodeID()) + _, err = nipost.Challenge(tab.localDB, sig.NodeID()) require.ErrorIs(t, err, sql.ErrNotFound) } @@ -430,13 +414,21 @@ func TestBuilder_PublishActivationTx_HappyFlow(t *testing.T) { // failing with ErrATXChallengeExpired. func TestBuilder_Loop_WaitsOnStaleChallenge(t *testing.T) { // Arrange - tab := newTestBuilder(t, WithPoetConfig(PoetConfig{PhaseShift: layerDuration * 4})) + tab := newTestBuilder(t, 1, WithPoetConfig(PoetConfig{PhaseShift: layerDuration * 4})) + sig := maps.Values(tab.signers)[0] + // current layer is too late to be able to build a nipost on time currLayer := (postGenesisEpoch + 1).FirstLayer() - ch := newChallenge(1, types.EmptyATXID, tab.goldenATXID, postGenesisEpoch, &tab.goldenATXID) + ch := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: types.EmptyATXID, + PublishEpoch: postGenesisEpoch, + PositioningATX: tab.goldenATXID, + CommitmentATX: &tab.goldenATXID, + } nipostData := newNIPostWithChallenge(t, types.HexToHash32("55555"), []byte("66666")) - prevAtx := newAtx(t, tab.sig, ch, nipostData.NIPost, 2, types.Address{}) - require.NoError(t, SignAndFinalizeAtx(tab.sig, prevAtx)) + prevAtx := newAtx(ch, nipostData.NIPost, 2, types.Address{}) + require.NoError(t, SignAndFinalizeAtx(sig, prevAtx)) vPrevAtx, err := prevAtx.Verify(0, 1) require.NoError(t, err) require.NoError(t, atxs.Add(tab.cdb, vPrevAtx)) @@ -449,8 +441,8 @@ func TestBuilder_Loop_WaitsOnStaleChallenge(t *testing.T) { return genesis.Add(layerDuration * time.Duration(got)) }).AnyTimes() - tab.mnipost.EXPECT().BuildNIPost(gomock.Any(), gomock.Any()).Return(nil, ErrATXChallengeExpired) - tab.mnipost.EXPECT().ResetState().Return(nil) + tab.mnipost.EXPECT().BuildNIPost(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, ErrATXChallengeExpired) + tab.mnipost.EXPECT().ResetState(sig.NodeID()).Return(nil) ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -461,30 +453,38 @@ func TestBuilder_Loop_WaitsOnStaleChallenge(t *testing.T) { return ch }) - tab.mValidator.EXPECT().VerifyChain(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).MinTimes(1) + tab.mValidator.EXPECT().VerifyChain(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()) // Act & Verify var eg errgroup.Group eg.Go(func() error { - tab.run(ctx) + tab.run(ctx, sig) return nil }) require.NoError(t, eg.Wait()) // state is cleaned up - _, err = nipost.Challenge(tab.localDB, tab.sig.NodeID()) + _, err = nipost.Challenge(tab.localDB, sig.NodeID()) require.ErrorIs(t, err, sql.ErrNotFound) } func TestBuilder_PublishActivationTx_FaultyNet(t *testing.T) { - tab := newTestBuilder(t, WithPoetConfig(PoetConfig{PhaseShift: layerDuration * 4})) + tab := newTestBuilder(t, 1, WithPoetConfig(PoetConfig{PhaseShift: layerDuration * 4})) + sig := maps.Values(tab.signers)[0] + posEpoch := postGenesisEpoch currLayer := postGenesisEpoch.FirstLayer() - ch := newChallenge(1, types.EmptyATXID, tab.goldenATXID, postGenesisEpoch, &tab.goldenATXID) + ch := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: types.EmptyATXID, + PublishEpoch: postGenesisEpoch, + PositioningATX: tab.goldenATXID, + CommitmentATX: &tab.goldenATXID, + } nipostData := newNIPostWithChallenge(t, types.HexToHash32("55555"), []byte("66666")) - prevAtx := newAtx(t, tab.sig, ch, nipostData.NIPost, 2, types.Address{}) - require.NoError(t, SignAndFinalizeAtx(tab.sig, prevAtx)) + prevAtx := newAtx(ch, nipostData.NIPost, 2, types.Address{}) + require.NoError(t, SignAndFinalizeAtx(sig, prevAtx)) vPrevAtx, err := prevAtx.Verify(0, 1) require.NoError(t, err) require.NoError(t, atxs.Add(tab.cdb, vPrevAtx)) @@ -500,15 +500,15 @@ func TestBuilder_PublishActivationTx_FaultyNet(t *testing.T) { nonce := types.VRFPostIndex(123) commitmentATX := types.RandomATXID() tab.mpostClient.EXPECT().Info(gomock.Any()).Return(&types.PostInfo{ - NodeID: tab.sig.NodeID(), + NodeID: sig.NodeID(), CommitmentATX: commitmentATX, Nonce: &nonce, NumUnits: DefaultPostSetupOpts().NumUnits, LabelsPerUnit: DefaultPostConfig().LabelsPerUnit, }, nil).AnyTimes() - tab.mnipost.EXPECT().BuildNIPost(gomock.Any(), gomock.Any()).DoAndReturn( - func(_ context.Context, challenge *types.NIPostChallenge) (*nipost.NIPostState, error) { + tab.mnipost.EXPECT().BuildNIPost(gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( + func(_ context.Context, _ *signing.EdSigner, challenge *types.NIPostChallenge) (*nipost.NIPostState, error) { currLayer = currLayer.Add(layersPerEpoch) return newNIPostWithChallenge(t, challenge.Hash(), []byte("66666")), nil }) @@ -536,7 +536,7 @@ func TestBuilder_PublishActivationTx_FaultyNet(t *testing.T) { ) // after successful publish, state is cleaned up - tab.mnipost.EXPECT().ResetState().Return(nil) + tab.mnipost.EXPECT().ResetState(sig.NodeID()).Return(nil) tab.mValidator.EXPECT().VerifyChain(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()) tab.mpub.EXPECT().Publish(gomock.Any(), pubsub.AtxProtocol, gomock.Any()).DoAndReturn( // second publish succeeds @@ -550,10 +550,10 @@ func TestBuilder_PublishActivationTx_FaultyNet(t *testing.T) { }, ) // create and publish ATX - require.NoError(t, tab.PublishActivationTx(context.Background())) + require.NoError(t, tab.PublishActivationTx(context.Background(), sig)) // state is cleaned up - _, err = nipost.Challenge(tab.localDB, tab.sig.NodeID()) + _, err = nipost.Challenge(tab.localDB, sig.NodeID()) require.ErrorIs(t, err, sql.ErrNotFound) } @@ -561,13 +561,21 @@ func TestBuilder_PublishActivationTx_UsesExistingChallengeOnLatePublish(t *testi poetCfg := PoetConfig{ PhaseShift: layerDuration * 4, } - tab := newTestBuilder(t, WithPoetConfig(poetCfg)) + tab := newTestBuilder(t, 1, WithPoetConfig(poetCfg)) + sig := maps.Values(tab.signers)[0] + posEpoch := postGenesisEpoch currLayer := (postGenesisEpoch + 1).FirstLayer().Add(5) // late for poet round start - challenge := newChallenge(1, types.ATXID{1, 2, 3}, types.ATXID{1, 2, 3}, postGenesisEpoch, nil) + challenge := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: types.ATXID{1, 2, 3}, + PublishEpoch: postGenesisEpoch, + PositioningATX: types.ATXID{1, 2, 3}, + CommitmentATX: nil, + } nipostData := newNIPostWithChallenge(t, types.HexToHash32("55555"), []byte("66666")) - prevAtx := newAtx(t, tab.sig, challenge, nipostData.NIPost, posEpoch.Uint32(), types.Address{}) - require.NoError(t, SignAndFinalizeAtx(tab.sig, prevAtx)) + prevAtx := newAtx(challenge, nipostData.NIPost, posEpoch.Uint32(), types.Address{}) + require.NoError(t, SignAndFinalizeAtx(sig, prevAtx)) vPrevAtx, err := prevAtx.Verify(0, 1) require.NoError(t, err) require.NoError(t, atxs.Add(tab.cdb, vPrevAtx)) @@ -583,15 +591,15 @@ func TestBuilder_PublishActivationTx_UsesExistingChallengeOnLatePublish(t *testi nonce := types.VRFPostIndex(123) commitmentATX := types.RandomATXID() tab.mpostClient.EXPECT().Info(gomock.Any()).Return(&types.PostInfo{ - NodeID: tab.sig.NodeID(), + NodeID: sig.NodeID(), CommitmentATX: commitmentATX, Nonce: &nonce, NumUnits: DefaultPostSetupOpts().NumUnits, LabelsPerUnit: DefaultPostConfig().LabelsPerUnit, }, nil).AnyTimes() - tab.mnipost.EXPECT().BuildNIPost(gomock.Any(), gomock.Any()).DoAndReturn( - func(_ context.Context, challenge *types.NIPostChallenge) (*nipost.NIPostState, error) { + tab.mnipost.EXPECT().BuildNIPost(gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( + func(_ context.Context, _ *signing.EdSigner, challenge *types.NIPostChallenge) (*nipost.NIPostState, error) { currLayer = currLayer.Add(1) return newNIPostWithChallenge(t, challenge.Hash(), []byte("66666")), nil }) @@ -614,9 +622,9 @@ func TestBuilder_PublishActivationTx_UsesExistingChallengeOnLatePublish(t *testi PositioningATX: vPrevAtx.ID(), } - require.NoError(t, nipost.AddChallenge(tab.localDb, tab.sig.NodeID(), ch)) + require.NoError(t, nipost.AddChallenge(tab.localDb, sig.NodeID(), ch)) - tab.mnipost.EXPECT().ResetState().Return(nil) + tab.mnipost.EXPECT().ResetState(sig.NodeID()).Return(nil) tab.mpub.EXPECT().Publish(gomock.Any(), pubsub.AtxProtocol, gomock.Any()).DoAndReturn( // publish succeeds @@ -631,21 +639,29 @@ func TestBuilder_PublishActivationTx_UsesExistingChallengeOnLatePublish(t *testi ) // create and publish ATX - require.NoError(t, tab.PublishActivationTx(context.Background())) + require.NoError(t, tab.PublishActivationTx(context.Background(), sig)) // state is cleaned up - _, err = nipost.Challenge(tab.localDB, tab.sig.NodeID()) + _, err = nipost.Challenge(tab.localDB, sig.NodeID()) require.ErrorIs(t, err, sql.ErrNotFound) } func TestBuilder_PublishActivationTx_RebuildNIPostWhenTargetEpochPassed(t *testing.T) { - tab := newTestBuilder(t, WithPoetConfig(PoetConfig{PhaseShift: layerDuration * 4})) + tab := newTestBuilder(t, 1, WithPoetConfig(PoetConfig{PhaseShift: layerDuration * 4})) + sig := maps.Values(tab.signers)[0] + posEpoch := types.EpochID(2) currLayer := posEpoch.FirstLayer() - ch := newChallenge(1, types.ATXID{1, 2, 3}, types.ATXID{1, 2, 3}, posEpoch, nil) + ch := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: types.ATXID{1, 2, 3}, + PublishEpoch: posEpoch, + PositioningATX: types.ATXID{1, 2, 3}, + CommitmentATX: nil, + } nipostData := newNIPostWithChallenge(t, types.HexToHash32("55555"), []byte("66666")) - prevAtx := newAtx(t, tab.sig, ch, nipostData.NIPost, 2, types.Address{}) - require.NoError(t, SignAndFinalizeAtx(tab.sig, prevAtx)) + prevAtx := newAtx(ch, nipostData.NIPost, 2, types.Address{}) + require.NoError(t, SignAndFinalizeAtx(sig, prevAtx)) vPrevAtx, err := prevAtx.Verify(0, 1) require.NoError(t, err) require.NoError(t, atxs.Add(tab.cdb, vPrevAtx)) @@ -661,8 +677,8 @@ func TestBuilder_PublishActivationTx_RebuildNIPostWhenTargetEpochPassed(t *testi genesis := time.Now().Add(-time.Duration(currLayer) * layerDuration) return genesis.Add(layerDuration * time.Duration(got)) }).AnyTimes() - tab.mnipost.EXPECT().BuildNIPost(gomock.Any(), gomock.Any()).DoAndReturn( - func(_ context.Context, challenge *types.NIPostChallenge) (*nipost.NIPostState, error) { + tab.mnipost.EXPECT().BuildNIPost(gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( + func(_ context.Context, _ *signing.EdSigner, challenge *types.NIPostChallenge) (*nipost.NIPostState, error) { currLayer = currLayer.Add(layersPerEpoch) return newNIPostWithChallenge(t, challenge.Hash(), []byte("66666")), nil }) @@ -695,12 +711,12 @@ func TestBuilder_PublishActivationTx_RebuildNIPostWhenTargetEpochPassed(t *testi }, ) // create and publish ATX - err = tab.PublishActivationTx(ctx) + err = tab.PublishActivationTx(ctx, sig) require.ErrorIs(t, err, context.Canceled) // publish returning an error will just cause a retry if not canceled require.NotNil(t, built) // state is preserved for a retry - challenge, err := nipost.Challenge(tab.localDB, tab.sig.NodeID()) + challenge, err := nipost.Challenge(tab.localDB, sig.NodeID()) require.NoError(t, err) require.Equal(t, built.NIPostChallenge, *challenge) @@ -709,34 +725,48 @@ func TestBuilder_PublishActivationTx_RebuildNIPostWhenTargetEpochPassed(t *testi // a stale challenge and builds a new NIPost. posEpoch = types.EpochID(4) currLayer = posEpoch.FirstLayer() - ch = newChallenge(1, types.ATXID{1, 2, 3}, types.ATXID{1, 2, 3}, posEpoch, nil) - posAtx := newAtx(t, tab.sig, ch, nipostData.NIPost, 2, types.Address{}) - require.NoError(t, SignAndFinalizeAtx(tab.sig, posAtx)) + ch = types.NIPostChallenge{ + Sequence: 1, + PrevATXID: types.ATXID{1, 2, 3}, + PublishEpoch: posEpoch, + PositioningATX: types.ATXID{1, 2, 3}, + CommitmentATX: nil, + } + posAtx := newAtx(ch, nipostData.NIPost, 2, types.Address{}) + require.NoError(t, SignAndFinalizeAtx(sig, posAtx)) vPosAtx, err := posAtx.Verify(0, 1) require.NoError(t, err) require.NoError(t, atxs.Add(tab.cdb, vPosAtx)) tab.mclock.EXPECT().CurrentLayer().DoAndReturn(func() types.LayerID { return currLayer }).AnyTimes() - tab.mnipost.EXPECT().ResetState().Return(nil) - built2, err := publishAtx(t, tab, posEpoch, &currLayer, layersPerEpoch) + tab.mnipost.EXPECT().ResetState(sig.NodeID()).Return(nil) + built2, err := publishAtx(t, tab, sig.NodeID(), posEpoch, &currLayer, layersPerEpoch) require.NoError(t, err) require.NotNil(t, built2) require.NotEqual(t, built.NIPostChallenge, built2.NIPostChallenge) require.Equal(t, posEpoch+2, built2.TargetEpoch()) // state is cleaned up after successful publish - _, err = nipost.Challenge(tab.localDB, tab.sig.NodeID()) + _, err = nipost.Challenge(tab.localDB, sig.NodeID()) require.ErrorIs(t, err, sql.ErrNotFound) } func TestBuilder_PublishActivationTx_NoPrevATX(t *testing.T) { - tab := newTestBuilder(t, WithPoetConfig(PoetConfig{PhaseShift: layerDuration * 4})) + tab := newTestBuilder(t, 1, WithPoetConfig(PoetConfig{PhaseShift: layerDuration * 4})) + sig := maps.Values(tab.signers)[0] + posEpoch := postGenesisEpoch currLayer := posEpoch.FirstLayer() - challenge := newChallenge(1, types.ATXID{1, 2, 3}, types.ATXID{1, 2, 3}, posEpoch, nil) + challenge := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: types.ATXID{1, 2, 3}, + PublishEpoch: posEpoch, + PositioningATX: types.ATXID{1, 2, 3}, + CommitmentATX: nil, + } nipostData := newNIPostWithChallenge(t, types.HexToHash32("55555"), []byte("66666")) otherSigner, err := signing.NewEdSigner() require.NoError(t, err) - posAtx := newAtx(t, otherSigner, challenge, nipostData.NIPost, 2, types.Address{}) + posAtx := newAtx(challenge, nipostData.NIPost, 2, types.Address{}) require.NoError(t, SignAndFinalizeAtx(otherSigner, posAtx)) vPosAtx, err := posAtx.Verify(0, 1) require.NoError(t, err) @@ -745,30 +775,38 @@ func TestBuilder_PublishActivationTx_NoPrevATX(t *testing.T) { // generate and store initial post in state require.NoError(t, nipost.AddInitialPost( tab.localDb, - tab.sig.NodeID(), + sig.NodeID(), nipost.Post{Indices: make([]byte, 10)}, )) // create and publish ATX tab.mclock.EXPECT().CurrentLayer().Return(currLayer).AnyTimes() - atx, err := publishAtx(t, tab, posEpoch, &currLayer, layersPerEpoch) + atx, err := publishAtx(t, tab, sig.NodeID(), posEpoch, &currLayer, layersPerEpoch) require.NoError(t, err) require.NotNil(t, atx) // state is cleaned up - _, err = nipost.Challenge(tab.localDB, tab.sig.NodeID()) + _, err = nipost.Challenge(tab.localDB, sig.NodeID()) require.ErrorIs(t, err, sql.ErrNotFound) } func TestBuilder_PublishActivationTx_NoPrevATX_PublishFails_InitialPost_preserved(t *testing.T) { - tab := newTestBuilder(t, WithPoetConfig(PoetConfig{PhaseShift: layerDuration * 4})) + tab := newTestBuilder(t, 1, WithPoetConfig(PoetConfig{PhaseShift: layerDuration * 4})) + sig := maps.Values(tab.signers)[0] + posEpoch := postGenesisEpoch currLayer := posEpoch.FirstLayer() - challenge := newChallenge(1, types.ATXID{1, 2, 3}, types.ATXID{1, 2, 3}, posEpoch, nil) + challenge := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: types.ATXID{1, 2, 3}, + PublishEpoch: posEpoch, + PositioningATX: types.ATXID{1, 2, 3}, + CommitmentATX: nil, + } nipostData := newNIPostWithChallenge(t, types.HexToHash32("55555"), []byte("66666")) otherSigner, err := signing.NewEdSigner() require.NoError(t, err) - posAtx := newAtx(t, otherSigner, challenge, nipostData.NIPost, 2, types.Address{}) + posAtx := newAtx(challenge, nipostData.NIPost, 2, types.Address{}) require.NoError(t, SignAndFinalizeAtx(otherSigner, posAtx)) vPosAtx, err := posAtx.Verify(0, 1) require.NoError(t, err) @@ -779,11 +817,7 @@ func TestBuilder_PublishActivationTx_NoPrevATX_PublishFails_InitialPost_preserve Indices: make([]byte, 10), CommitmentATX: types.RandomATXID(), } - require.NoError(t, nipost.AddInitialPost( - tab.localDb, - tab.sig.NodeID(), - refPost, - )) + require.NoError(t, nipost.AddInitialPost(tab.localDb, sig.NodeID(), refPost)) // create and publish ATX tab.mclock.EXPECT().CurrentLayer().Return(currLayer).AnyTimes() @@ -794,8 +828,8 @@ func TestBuilder_PublishActivationTx_NoPrevATX_PublishFails_InitialPost_preserve return genesis.Add(layerDuration * time.Duration(got)) }).AnyTimes() - tab.mnipost.EXPECT().BuildNIPost(gomock.Any(), gomock.Any()).Return(nil, ErrATXChallengeExpired) - tab.mnipost.EXPECT().ResetState().Return(nil) + tab.mnipost.EXPECT().BuildNIPost(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, ErrATXChallengeExpired) + tab.mnipost.EXPECT().ResetState(sig.NodeID()).Return(nil) ch := make(chan struct{}) tab.mclock.EXPECT().AwaitLayer(currLayer.Add(1)).Do(func(got types.LayerID) <-chan struct{} { @@ -807,7 +841,7 @@ func TestBuilder_PublishActivationTx_NoPrevATX_PublishFails_InitialPost_preserve ctx, cancel := context.WithCancel(context.Background()) var eg errgroup.Group eg.Go(func() error { - tab.run(ctx) + tab.run(ctx, sig) return nil }) t.Cleanup(func() { @@ -822,13 +856,13 @@ func TestBuilder_PublishActivationTx_NoPrevATX_PublishFails_InitialPost_preserve } // initial post is preserved - post, err := nipost.InitialPost(tab.localDB, tab.sig.NodeID()) + post, err := nipost.InitialPost(tab.localDB, sig.NodeID()) require.NoError(t, err) require.NotNil(t, post) require.Equal(t, refPost, *post) // state is cleaned up - _, err = nipost.Challenge(tab.localDB, tab.sig.NodeID()) + _, err = nipost.Challenge(tab.localDB, sig.NodeID()) require.ErrorIs(t, err, sql.ErrNotFound) } @@ -836,7 +870,9 @@ func TestBuilder_PublishActivationTx_PrevATXWithoutPrevATX(t *testing.T) { r := require.New(t) // Arrange - tab := newTestBuilder(t, WithPoetConfig(PoetConfig{PhaseShift: layerDuration * 4})) + tab := newTestBuilder(t, 1, WithPoetConfig(PoetConfig{PhaseShift: layerDuration * 4})) + sig := maps.Values(tab.signers)[0] + otherSigner, err := signing.NewEdSigner() r.NoError(err) @@ -849,20 +885,32 @@ func TestBuilder_PublishActivationTx_PrevATXWithoutPrevATX(t *testing.T) { prevAtxPostEpoch := postGenesisEpoch postAtxPubEpoch := postGenesisEpoch - challenge := newChallenge(1, types.EmptyATXID, tab.goldenATXID, postAtxPubEpoch, &tab.goldenATXID) + challenge := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: types.EmptyATXID, + PublishEpoch: postAtxPubEpoch, + PositioningATX: tab.goldenATXID, + CommitmentATX: &tab.goldenATXID, + } poetBytes := []byte("66666") nipostData := newNIPostWithChallenge(t, types.HexToHash32("55555"), poetBytes) - posAtx := newAtx(t, otherSigner, challenge, nipostData.NIPost, 2, types.Address{}) + posAtx := newAtx(challenge, nipostData.NIPost, 2, types.Address{}) require.NoError(t, SignAndFinalizeAtx(otherSigner, posAtx)) vPosAtx, err := posAtx.Verify(0, 2) r.NoError(err) r.NoError(atxs.Add(tab.cdb, vPosAtx)) - challenge = newChallenge(0, types.EmptyATXID, posAtx.ID(), prevAtxPostEpoch, nil) + challenge = types.NIPostChallenge{ + Sequence: 0, + PrevATXID: types.EmptyATXID, + PublishEpoch: prevAtxPostEpoch, + PositioningATX: posAtx.ID(), + CommitmentATX: nil, + } challenge.InitialPost = initialPost - prevAtx := newAtx(t, tab.sig, challenge, nipostData.NIPost, 2, types.Address{}) + prevAtx := newAtx(challenge, nipostData.NIPost, 2, types.Address{}) prevAtx.InitialPost = initialPost - require.NoError(t, SignAndFinalizeAtx(tab.sig, prevAtx)) + require.NoError(t, SignAndFinalizeAtx(sig, prevAtx)) vPrevAtx, err := prevAtx.Verify(0, 1) r.NoError(err) r.NoError(atxs.Add(tab.cdb, vPrevAtx)) @@ -877,19 +925,18 @@ func TestBuilder_PublishActivationTx_PrevATXWithoutPrevATX(t *testing.T) { genesis := time.Now().Add(-time.Duration(currentLayer) * layerDuration) return genesis.Add(layerDuration * time.Duration(layer)) }).AnyTimes() - tab.mclock.EXPECT(). - AwaitLayer(vPosAtx.PublishEpoch.FirstLayer().Add(layersPerEpoch)). - DoAndReturn(func(layer types.LayerID) <-chan struct{} { + tab.mclock.EXPECT().AwaitLayer(vPosAtx.PublishEpoch.FirstLayer().Add(layersPerEpoch)).DoAndReturn( + func(layer types.LayerID) <-chan struct{} { ch := make(chan struct{}) close(ch) return ch - }). - Times(1) + }, + ) nonce := types.VRFPostIndex(123) commitmentATX := types.RandomATXID() tab.mpostClient.EXPECT().Info(gomock.Any()).Return(&types.PostInfo{ - NodeID: tab.sig.NodeID(), + NodeID: sig.NodeID(), CommitmentATX: commitmentATX, Nonce: &nonce, @@ -897,8 +944,8 @@ func TestBuilder_PublishActivationTx_PrevATXWithoutPrevATX(t *testing.T) { LabelsPerUnit: DefaultPostConfig().LabelsPerUnit, }, nil).AnyTimes() - tab.mnipost.EXPECT().BuildNIPost(gomock.Any(), gomock.Any()). - DoAndReturn(func(_ context.Context, challenge *types.NIPostChallenge) (*nipost.NIPostState, error) { + tab.mnipost.EXPECT().BuildNIPost(gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( + func(_ context.Context, _ *signing.EdSigner, challenge *types.NIPostChallenge) (*nipost.NIPostState, error) { currentLayer = currentLayer.Add(5) return newNIPostWithChallenge(t, challenge.Hash(), poetBytes), nil }) @@ -915,7 +962,7 @@ func TestBuilder_PublishActivationTx_PrevATXWithoutPrevATX(t *testing.T) { atx.SetEffectiveNumUnits(atx.NumUnits) vAtx, err := atx.Verify(0, 1) r.NoError(err) - r.Equal(tab.sig.NodeID(), vAtx.SmesherID) + r.Equal(sig.NodeID(), vAtx.SmesherID) r.NoError(atxs.Add(tab.cdb, vAtx)) @@ -932,12 +979,12 @@ func TestBuilder_PublishActivationTx_PrevATXWithoutPrevATX(t *testing.T) { return nil }) - tab.mnipost.EXPECT().ResetState().Return(nil) + tab.mnipost.EXPECT().ResetState(sig.NodeID()).Return(nil) - r.NoError(tab.PublishActivationTx(context.Background())) + r.NoError(tab.PublishActivationTx(context.Background(), sig)) // state is cleaned up - _, err = nipost.Challenge(tab.localDB, tab.sig.NodeID()) + _, err = nipost.Challenge(tab.localDB, sig.NodeID()) require.ErrorIs(t, err, sql.ErrNotFound) } @@ -945,16 +992,24 @@ func TestBuilder_PublishActivationTx_TargetsEpochBasedOnPosAtx(t *testing.T) { r := require.New(t) // Arrange - tab := newTestBuilder(t, WithPoetConfig(PoetConfig{PhaseShift: layerDuration * 4})) + tab := newTestBuilder(t, 1, WithPoetConfig(PoetConfig{PhaseShift: layerDuration * 4})) + sig := maps.Values(tab.signers)[0] + otherSigner, err := signing.NewEdSigner() r.NoError(err) currentLayer := postGenesisEpoch.FirstLayer().Add(3) posEpoch := postGenesisEpoch - challenge := newChallenge(1, types.ATXID{1, 2, 3}, types.ATXID{1, 2, 3}, posEpoch, &types.ATXID{4, 5, 6}) + challenge := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: types.ATXID{1, 2, 3}, + PublishEpoch: posEpoch, + PositioningATX: types.ATXID{1, 2, 3}, + CommitmentATX: &types.ATXID{4, 5, 6}, + } poetBytes := []byte("66666") nipostData := newNIPostWithChallenge(t, types.HexToHash32("55555"), poetBytes) - posAtx := newAtx(t, otherSigner, challenge, nipostData.NIPost, 2, types.Address{}) + posAtx := newAtx(challenge, nipostData.NIPost, 2, types.Address{}) require.NoError(t, SignAndFinalizeAtx(otherSigner, posAtx)) vPosAtx, err := posAtx.Verify(0, 1) r.NoError(err) @@ -970,19 +1025,18 @@ func TestBuilder_PublishActivationTx_TargetsEpochBasedOnPosAtx(t *testing.T) { genesis := time.Now().Add(-time.Duration(currentLayer) * layerDuration) return genesis.Add(layerDuration * time.Duration(layer)) }).AnyTimes() - tab.mclock.EXPECT(). - AwaitLayer(vPosAtx.PublishEpoch.FirstLayer().Add(layersPerEpoch)). - DoAndReturn(func(types.LayerID) <-chan struct{} { + tab.mclock.EXPECT().AwaitLayer(vPosAtx.PublishEpoch.FirstLayer().Add(layersPerEpoch)).DoAndReturn( + func(types.LayerID) <-chan struct{} { ch := make(chan struct{}) close(ch) return ch - }). - Times(1) + }, + ) nonce := types.VRFPostIndex(123) commitmentATX := types.RandomATXID() tab.mpostClient.EXPECT().Info(gomock.Any()).Return(&types.PostInfo{ - NodeID: tab.sig.NodeID(), + NodeID: sig.NodeID(), CommitmentATX: commitmentATX, Nonce: &nonce, @@ -990,14 +1044,13 @@ func TestBuilder_PublishActivationTx_TargetsEpochBasedOnPosAtx(t *testing.T) { LabelsPerUnit: DefaultPostConfig().LabelsPerUnit, }, nil).AnyTimes() - tab.mnipost.EXPECT().BuildNIPost(gomock.Any(), gomock.Any()). - DoAndReturn(func(_ context.Context, challenge *types.NIPostChallenge) (*nipost.NIPostState, error) { + tab.mnipost.EXPECT().BuildNIPost(gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( + func(_ context.Context, _ *signing.EdSigner, challenge *types.NIPostChallenge) (*nipost.NIPostState, error) { currentLayer = currentLayer.Add(layersPerEpoch) return newNIPostWithChallenge(t, challenge.Hash(), poetBytes), nil }) tab.mValidator.EXPECT().VerifyChain(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()) - tab.mpub.EXPECT(). Publish(gomock.Any(), gomock.Any(), gomock.Any()). DoAndReturn(func(ctx context.Context, _ string, msg []byte) error { @@ -1008,7 +1061,7 @@ func TestBuilder_PublishActivationTx_TargetsEpochBasedOnPosAtx(t *testing.T) { atx.SetEffectiveNumUnits(atx.NumUnits) vAtx, err := atx.Verify(0, 1) r.NoError(err) - r.Equal(tab.sig.NodeID(), vAtx.SmesherID) + r.Equal(sig.NodeID(), vAtx.SmesherID) r.NoError(atxs.Add(tab.cdb, vAtx)) @@ -1023,29 +1076,33 @@ func TestBuilder_PublishActivationTx_TargetsEpochBasedOnPosAtx(t *testing.T) { return nil }) - require.NoError(t, nipost.AddInitialPost( - tab.localDb, - tab.sig.NodeID(), - nipost.Post{Indices: make([]byte, 10)}, - )) + require.NoError(t, nipost.AddInitialPost(tab.localDb, sig.NodeID(), nipost.Post{Indices: make([]byte, 10)})) - tab.mnipost.EXPECT().ResetState().Return(nil) + tab.mnipost.EXPECT().ResetState(sig.NodeID()).Return(nil) - r.NoError(tab.PublishActivationTx(context.Background())) + r.NoError(tab.PublishActivationTx(context.Background(), sig)) // state is cleaned up - _, err = nipost.Challenge(tab.localDB, tab.sig.NodeID()) + _, err = nipost.Challenge(tab.localDB, sig.NodeID()) require.ErrorIs(t, err, sql.ErrNotFound) } func TestBuilder_PublishActivationTx_FailsWhenNIPostBuilderFails(t *testing.T) { - tab := newTestBuilder(t, WithPoetConfig(PoetConfig{PhaseShift: layerDuration * 4})) + tab := newTestBuilder(t, 1, WithPoetConfig(PoetConfig{PhaseShift: layerDuration * 4})) + sig := maps.Values(tab.signers)[0] + posEpoch := postGenesisEpoch currLayer := posEpoch.FirstLayer() - ch := newChallenge(1, types.ATXID{1, 2, 3}, types.ATXID{1, 2, 3}, posEpoch, nil) + ch := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: types.ATXID{1, 2, 3}, + PublishEpoch: posEpoch, + PositioningATX: types.ATXID{1, 2, 3}, + CommitmentATX: nil, + } nipostData := newNIPostWithChallenge(t, types.HexToHash32("55555"), []byte("66666")) - posAtx := newAtx(t, tab.sig, ch, nipostData.NIPost, 2, types.Address{}) - require.NoError(t, SignAndFinalizeAtx(tab.sig, posAtx)) + posAtx := newAtx(ch, nipostData.NIPost, 2, types.Address{}) + require.NoError(t, SignAndFinalizeAtx(sig, posAtx)) vPosAtx, err := posAtx.Verify(0, 1) require.NoError(t, err) require.NoError(t, atxs.Add(tab.cdb, vPosAtx)) @@ -1057,13 +1114,13 @@ func TestBuilder_PublishActivationTx_FailsWhenNIPostBuilderFails(t *testing.T) { genesis := time.Now().Add(-time.Duration(currLayer) * layerDuration) return genesis.Add(layerDuration * time.Duration(got)) }).AnyTimes() - nipostErr := fmt.Errorf("NIPost builder error") - tab.mnipost.EXPECT().BuildNIPost(gomock.Any(), gomock.Any()).Return(nil, nipostErr) + nipostErr := errors.New("NIPost builder error") + tab.mnipost.EXPECT().BuildNIPost(gomock.Any(), sig, gomock.Any()).Return(nil, nipostErr) tab.mValidator.EXPECT().VerifyChain(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()) - require.ErrorIs(t, tab.PublishActivationTx(context.Background()), nipostErr) + require.ErrorIs(t, tab.PublishActivationTx(context.Background(), sig), nipostErr) // state is preserved - challenge, err := nipost.Challenge(tab.localDB, tab.sig.NodeID()) + challenge, err := nipost.Challenge(tab.localDB, sig.NodeID()) require.NoError(t, err) require.NotNil(t, challenge) } @@ -1119,16 +1176,24 @@ func TestBuilder_PublishActivationTx_Serialize(t *testing.T) { } func TestBuilder_SignAtx(t *testing.T) { - tab := newTestBuilder(t) + sig, err := signing.NewEdSigner() + require.NoError(t, err) + prevAtx := types.ATXID(types.HexToHash32("0x111")) - challenge := newChallenge(1, prevAtx, prevAtx, types.EpochID(15), nil) + challenge := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: prevAtx, + PublishEpoch: types.EpochID(15), + PositioningATX: prevAtx, + CommitmentATX: nil, + } nipost := newNIPostWithChallenge(t, types.HexToHash32("55555"), []byte("66666")) - atx := newAtx(t, tab.sig, challenge, nipost.NIPost, 100, types.Address{}) - require.NoError(t, SignAndFinalizeAtx(tab.signer, atx)) + atx := newAtx(challenge, nipost.NIPost, 100, types.Address{}) + require.NoError(t, SignAndFinalizeAtx(sig, atx)) - ok := signing.NewEdVerifier().Verify(signing.ATX, tab.sig.NodeID(), atx.SignedBytes(), atx.Signature) + ok := signing.NewEdVerifier().Verify(signing.ATX, sig.NodeID(), atx.SignedBytes(), atx.Signature) require.True(t, ok) - require.Equal(t, tab.sig.NodeID(), atx.SmesherID) + require.Equal(t, sig.NodeID(), atx.SmesherID) } func TestBuilder_RetryPublishActivationTx(t *testing.T) { @@ -1146,15 +1211,23 @@ func TestBuilder_RetryPublishActivationTx(t *testing.T) { retryInterval := 50 * time.Microsecond tab := newTestBuilder( t, + 1, WithPoetConfig(PoetConfig{PhaseShift: 150 * time.Millisecond}), WithPoetRetryInterval(retryInterval), ) + sig := maps.Values(tab.signers)[0] posEpoch := types.EpochID(0) - challenge := newChallenge(1, types.ATXID{1, 2, 3}, types.ATXID{1, 2, 3}, posEpoch, nil) + challenge := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: types.ATXID{1, 2, 3}, + PublishEpoch: posEpoch, + PositioningATX: types.ATXID{1, 2, 3}, + CommitmentATX: nil, + } poetBytes := []byte("66666") nipostData := newNIPostWithChallenge(t, types.HexToHash32("55555"), poetBytes) - prevAtx := newAtx(t, tab.sig, challenge, nipostData.NIPost, 2, types.Address{}) - require.NoError(t, SignAndFinalizeAtx(tab.sig, prevAtx)) + prevAtx := newAtx(challenge, nipostData.NIPost, 2, types.Address{}) + require.NoError(t, SignAndFinalizeAtx(sig, prevAtx)) vPrevAtx, err := prevAtx.Verify(0, 1) require.NoError(t, err) require.NoError(t, atxs.Add(tab.cdb, vPrevAtx)) @@ -1184,8 +1257,8 @@ func TestBuilder_RetryPublishActivationTx(t *testing.T) { tries := 0 var last time.Time builderConfirmation := make(chan struct{}) - tab.mnipost.EXPECT().BuildNIPost(gomock.Any(), gomock.Any()).Times(expectedTries).DoAndReturn( - func(_ context.Context, challenge *types.NIPostChallenge) (*nipost.NIPostState, error) { + tab.mnipost.EXPECT().BuildNIPost(gomock.Any(), gomock.Any(), gomock.Any()).Times(expectedTries).DoAndReturn( + func(_ context.Context, _ *signing.EdSigner, challenge *types.NIPostChallenge) (*nipost.NIPostState, error) { now := time.Now() if now.Sub(last) < retryInterval { require.FailNow(t, "retry interval not respected") @@ -1201,13 +1274,13 @@ func TestBuilder_RetryPublishActivationTx(t *testing.T) { }, ) - tab.mnipost.EXPECT().ResetState().Return(nil) + tab.mnipost.EXPECT().ResetState(sig.NodeID()).Return(nil) tab.mValidator.EXPECT().VerifyChain(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()) nonce := types.VRFPostIndex(123) commitmentATX := types.RandomATXID() tab.mpostClient.EXPECT().Info(gomock.Any()).Return(&types.PostInfo{ - NodeID: tab.sig.NodeID(), + NodeID: sig.NodeID(), CommitmentATX: commitmentATX, Nonce: &nonce, @@ -1232,7 +1305,7 @@ func TestBuilder_RetryPublishActivationTx(t *testing.T) { defer cancel() var eg errgroup.Group eg.Go(func() error { - tab.run(ctx) + tab.run(ctx, sig) return nil }) t.Cleanup(func() { assert.NoError(t, eg.Wait()) }) @@ -1253,16 +1326,18 @@ func TestBuilder_RetryPublishActivationTx(t *testing.T) { } // state is cleaned up - _, err = nipost.InitialPost(tab.localDB, tab.sig.NodeID()) + _, err = nipost.InitialPost(tab.localDB, sig.NodeID()) require.ErrorIs(t, err, sql.ErrNotFound) - _, err = nipost.Challenge(tab.localDB, tab.sig.NodeID()) + _, err = nipost.Challenge(tab.localDB, sig.NodeID()) require.ErrorIs(t, err, sql.ErrNotFound) } func TestBuilder_InitialProofGeneratedOnce(t *testing.T) { - tab := newTestBuilder(t, WithPoetConfig(PoetConfig{PhaseShift: layerDuration * 4})) - tab.mnipost.EXPECT().Proof(gomock.Any(), shared.ZeroChallenge).Return( + tab := newTestBuilder(t, 1, WithPoetConfig(PoetConfig{PhaseShift: layerDuration * 4})) + sig := maps.Values(tab.signers)[0] + + tab.mnipost.EXPECT().Proof(gomock.Any(), sig.NodeID(), shared.ZeroChallenge).Return( &types.Post{Indices: make([]byte, 10)}, &types.PostInfo{ CommitmentATX: types.RandomATXID(), @@ -1270,32 +1345,48 @@ func TestBuilder_InitialProofGeneratedOnce(t *testing.T) { }, nil, ) - require.NoError(t, tab.buildInitialPost(context.Background())) + require.NoError(t, tab.buildInitialPost(context.Background(), sig.NodeID())) posEpoch := postGenesisEpoch + 1 - challenge := newChallenge(1, types.EmptyATXID, tab.goldenATXID, posEpoch, &tab.goldenATXID) + challenge := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: types.EmptyATXID, + PublishEpoch: posEpoch, + PositioningATX: tab.goldenATXID, + CommitmentATX: &tab.goldenATXID, + } poetByte := []byte("66666") nipost := newNIPostWithChallenge(t, types.HexToHash32("55555"), poetByte) - prevAtx := newAtx(t, tab.sig, challenge, nipost.NIPost, 2, types.Address{}) - require.NoError(t, SignAndFinalizeAtx(tab.sig, prevAtx)) + prevAtx := newAtx(challenge, nipost.NIPost, 2, types.Address{}) + require.NoError(t, SignAndFinalizeAtx(sig, prevAtx)) vPrevAtx, err := prevAtx.Verify(0, 1) require.NoError(t, err) require.NoError(t, atxs.Add(tab.cdb, vPrevAtx)) currLayer := posEpoch.FirstLayer().Add(1) tab.mclock.EXPECT().CurrentLayer().Return(currLayer).AnyTimes() - atx, err := publishAtx(t, tab, posEpoch, &currLayer, layersPerEpoch) + atx, err := publishAtx(t, tab, sig.NodeID(), posEpoch, &currLayer, layersPerEpoch) require.NoError(t, err) require.NotNil(t, atx) - assertLastAtx(require.New(t), tab.sig.NodeID(), types.BytesToHash(poetByte), atx, vPrevAtx, vPrevAtx, layersPerEpoch) + + require.Equal(t, sig.NodeID(), atx.SmesherID) + require.Equal(t, vPrevAtx.Sequence+1, atx.Sequence) + require.Equal(t, vPrevAtx.ID(), atx.PrevATXID) + require.Nil(t, atx.InitialPost) + require.Nil(t, atx.VRFNonce) + require.Equal(t, vPrevAtx.ID(), atx.PositioningATX) + require.Equal(t, vPrevAtx.PublishEpoch+1, atx.PublishEpoch) + require.Equal(t, types.BytesToHash(poetByte), atx.GetPoetProofRef()) // postClient.Proof() should not be called again - require.NoError(t, tab.buildInitialPost(context.Background())) + require.NoError(t, tab.buildInitialPost(context.Background(), sig.NodeID())) } func TestBuilder_InitialPostIsPersisted(t *testing.T) { - tab := newTestBuilder(t, WithPoetConfig(PoetConfig{PhaseShift: layerDuration * 4})) - tab.mnipost.EXPECT().Proof(gomock.Any(), shared.ZeroChallenge).Return( + tab := newTestBuilder(t, 1, WithPoetConfig(PoetConfig{PhaseShift: layerDuration * 4})) + sig := maps.Values(tab.signers)[0] + + tab.mnipost.EXPECT().Proof(gomock.Any(), sig.NodeID(), shared.ZeroChallenge).Return( &types.Post{Indices: make([]byte, 10)}, &types.PostInfo{ CommitmentATX: types.RandomATXID(), @@ -1303,14 +1394,10 @@ func TestBuilder_InitialPostIsPersisted(t *testing.T) { }, nil, ) - tab.mValidator.EXPECT(). - Post(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()). - AnyTimes(). - Return(nil) - require.NoError(t, tab.buildInitialPost(context.Background())) + require.NoError(t, tab.buildInitialPost(context.Background(), sig.NodeID())) // postClient.Proof() should not be called again - require.NoError(t, tab.buildInitialPost(context.Background())) + require.NoError(t, tab.buildInitialPost(context.Background(), sig.NodeID())) } func TestWaitPositioningAtx(t *testing.T) { @@ -1327,10 +1414,12 @@ func TestWaitPositioningAtx(t *testing.T) { } { tc := tc t.Run(tc.desc, func(t *testing.T) { - tab := newTestBuilder(t, WithPoetConfig(PoetConfig{ + tab := newTestBuilder(t, 1, WithPoetConfig(PoetConfig{ PhaseShift: tc.shift, GracePeriod: tc.grace, })) + sig := maps.Values(tab.signers)[0] + tab.mclock.EXPECT().CurrentLayer().Return(types.LayerID(0)).AnyTimes() tab.mclock.EXPECT().LayerToTime(gomock.Any()).DoAndReturn(func(lid types.LayerID) time.Time { // layer duration is 10ms to speed up test @@ -1339,8 +1428,8 @@ func TestWaitPositioningAtx(t *testing.T) { // everything else are stubs that are irrelevant for the test tab.mpostClient.EXPECT().Info(gomock.Any()).Return(&types.PostInfo{}, nil).AnyTimes() - tab.mnipost.EXPECT().ResetState().Return(nil) - tab.mnipost.EXPECT().BuildNIPost(gomock.Any(), gomock.Any()).Return(&nipost.NIPostState{}, nil) + tab.mnipost.EXPECT().ResetState(sig.NodeID()).Return(nil) + tab.mnipost.EXPECT().BuildNIPost(gomock.Any(), gomock.Any(), gomock.Any()).Return(&nipost.NIPostState{}, nil) closed := make(chan struct{}) close(closed) tab.mclock.EXPECT().AwaitLayer(types.EpochID(1).FirstLayer()).Return(closed).AnyTimes() @@ -1355,46 +1444,15 @@ func TestWaitPositioningAtx(t *testing.T) { require.NoError(t, nipost.AddInitialPost( tab.localDb, - tab.sig.NodeID(), + sig.NodeID(), nipost.Post{Indices: make([]byte, 10)}, )) - err := tab.PublishActivationTx(context.Background()) - require.NoError(t, err) + require.NoError(t, tab.PublishActivationTx(context.Background(), sig)) }) } } -func TestRegossip(t *testing.T) { - layer := types.LayerID(10) - t.Run("not found", func(t *testing.T) { - h := newTestBuilder(t) - h.mclock.EXPECT().CurrentLayer().Return(layer) - require.NoError(t, h.Regossip(context.Background())) - }) - t.Run("success", func(t *testing.T) { - h := newTestBuilder(t) - atx := newActivationTx(t, - h.signer, 0, types.EmptyATXID, types.EmptyATXID, nil, - layer.GetEpoch(), 0, 1, types.Address{}, 1, &types.NIPost{}) - require.NoError(t, atxs.Add(h.cdb, atx)) - blob, err := atxs.GetBlob(h.cdb, atx.ID().Bytes()) - require.NoError(t, err) - h.mclock.EXPECT().CurrentLayer().Return(layer) - - ctx := context.Background() - h.mpub.EXPECT().Publish(ctx, pubsub.AtxProtocol, blob) - require.NoError(t, h.Regossip(ctx)) - }) - t.Run("checkpointed", func(t *testing.T) { - h := newTestBuilder(t) - atx := atxs.CheckpointAtx{ID: types.ATXID{1}, Epoch: layer.GetEpoch(), SmesherID: h.sig.NodeID()} - require.NoError(t, atxs.AddCheckpointed(h.cdb, &atx)) - h.mclock.EXPECT().CurrentLayer().Return(layer) - require.NoError(t, h.Regossip(context.Background())) - }) -} - func TestWaitingToBuildNipostChallengeWithJitter(t *testing.T) { t.Run("before grace period", func(t *testing.T) { // ┌──grace period──┐ @@ -1431,14 +1489,21 @@ func TestWaitingToBuildNipostChallengeWithJitter(t *testing.T) { // Test if GetPositioningAtx disregards ATXs with invalid POST in their chain. // It should pick an ATX with valid POST even though it's a lower height. func TestGetPositioningAtxPicksAtxWithValidChain(t *testing.T) { - tab := newTestBuilder(t) + tab := newTestBuilder(t, 1) + sig := maps.Values(tab.signers)[0] // Invalid chain with high height sigInvalid, err := signing.NewEdSigner() require.NoError(t, err) - ch := newChallenge(1, types.EmptyATXID, tab.goldenATXID, postGenesisEpoch, &tab.goldenATXID) + ch := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: types.EmptyATXID, + PublishEpoch: postGenesisEpoch, + PositioningATX: tab.goldenATXID, + CommitmentATX: &tab.goldenATXID, + } nipostData := newNIPostWithChallenge(t, types.HexToHash32(""), []byte("0")) - invalidAtx := newAtx(t, sigInvalid, ch, nipostData.NIPost, 2, types.Address{}) + invalidAtx := newAtx(ch, nipostData.NIPost, 2, types.Address{}) require.NoError(t, SignAndFinalizeAtx(sigInvalid, invalidAtx)) vInvalidAtx, err := invalidAtx.Verify(0, 100) require.NoError(t, err) @@ -1447,9 +1512,8 @@ func TestGetPositioningAtxPicksAtxWithValidChain(t *testing.T) { // Valid chain with lower height sigValid, err := signing.NewEdSigner() require.NoError(t, err) - ch = newChallenge(1, types.EmptyATXID, tab.goldenATXID, postGenesisEpoch, &tab.goldenATXID) nipostData = newNIPostWithChallenge(t, types.HexToHash32(""), []byte("1")) - validAtx := newAtx(t, sigValid, ch, nipostData.NIPost, 2, types.Address{}) + validAtx := newAtx(ch, nipostData.NIPost, 2, types.Address{}) require.NoError(t, SignAndFinalizeAtx(sigValid, validAtx)) vValidAtx, err := validAtx.Verify(0, 1) require.NoError(t, err) @@ -1461,19 +1525,21 @@ func TestGetPositioningAtxPicksAtxWithValidChain(t *testing.T) { tab.mValidator.EXPECT(). VerifyChain(gomock.Any(), validAtx.ID(), tab.goldenATXID, gomock.Any()) - posAtxID, err := tab.getPositioningAtx(context.Background()) + posAtxID, err := tab.getPositioningAtx(context.Background(), sig.NodeID()) require.NoError(t, err) require.Equal(t, posAtxID, vValidAtx.ID()) } func TestGetPositioningAtxDbFailed(t *testing.T) { - tab := newTestBuilder(t) - db := datastoremocks.NewMockExecutor(gomock.NewController(t)) + tab := newTestBuilder(t, 1) + sig := maps.Values(tab.signers)[0] + + db := datastoremocks.NewMockExecutor(tab.mctrl) tab.Builder.cdb = datastore.NewCachedDB(db, logtest.New(t)) expected := errors.New("db error") db.EXPECT().Exec(gomock.Any(), gomock.Any(), gomock.Any()).Return(0, expected) - none, err := tab.getPositioningAtx(context.Background()) + none, err := tab.getPositioningAtx(context.Background(), sig.NodeID()) require.ErrorIs(t, err, expected) require.Equal(t, types.ATXID{}, none) } diff --git a/activation/e2e/activation_test.go b/activation/e2e/activation_test.go new file mode 100644 index 0000000000..d27b25ca57 --- /dev/null +++ b/activation/e2e/activation_test.go @@ -0,0 +1,207 @@ +package activation_test + +import ( + "context" + "sync" + "testing" + "time" + + "github.com/spacemeshos/post/initialization" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "go.uber.org/mock/gomock" + "go.uber.org/zap/zaptest" + "golang.org/x/sync/errgroup" + + "github.com/spacemeshos/go-spacemesh/activation" + "github.com/spacemeshos/go-spacemesh/api/grpcserver" + "github.com/spacemeshos/go-spacemesh/codec" + "github.com/spacemeshos/go-spacemesh/common/types" + "github.com/spacemeshos/go-spacemesh/datastore" + "github.com/spacemeshos/go-spacemesh/log" + "github.com/spacemeshos/go-spacemesh/p2p/pubsub" + "github.com/spacemeshos/go-spacemesh/p2p/pubsub/mocks" + "github.com/spacemeshos/go-spacemesh/signing" + "github.com/spacemeshos/go-spacemesh/sql" + "github.com/spacemeshos/go-spacemesh/sql/localsql" + "github.com/spacemeshos/go-spacemesh/timesync" +) + +func Test_BuilderWithMultipleClients(t *testing.T) { + ctrl := gomock.NewController(t) + + numSigners := 3 + signers := make(map[types.NodeID]*signing.EdSigner, numSigners) + for i := 0; i < numSigners; i++ { + sig, err := signing.NewEdSigner() + require.NoError(t, err) + + signers[sig.NodeID()] = sig + } + + logger := zaptest.NewLogger(t) + goldenATX := types.ATXID{2, 3, 4} + cfg := activation.DefaultPostConfig() + db := sql.InMemory() + cdb := datastore.NewCachedDB(db, log.NewFromLog(logger)) + + syncer := activation.NewMocksyncer(ctrl) + syncer.EXPECT().RegisterForATXSynced().DoAndReturn(func() <-chan struct{} { + synced := make(chan struct{}) + close(synced) + return synced + }).AnyTimes() + + svc := grpcserver.NewPostService(logger) + grpcCfg, cleanup := launchServer(t, svc) + t.Cleanup(cleanup) + + opts := activation.DefaultPostSetupOpts() + opts.ProviderID.SetUint32(initialization.CPUProviderID()) + opts.Scrypt.N = 2 // Speedup initialization in tests. + + var eg errgroup.Group + for _, sig := range signers { + sig := sig + opts := opts + eg.Go(func() error { + validator := activation.NewMocknipostValidator(ctrl) + mgr, err := activation.NewPostSetupManager(sig.NodeID(), cfg, logger, cdb, goldenATX, syncer, validator) + require.NoError(t, err) + + opts.DataDir = t.TempDir() + initPost(t, mgr, opts) + t.Cleanup(launchPostSupervisor(t, logger, mgr, grpcCfg, opts)) + + require.Eventually(t, func() bool { + _, err := svc.Client(sig.NodeID()) + return err == nil + }, 10*time.Second, 100*time.Millisecond, "timed out waiting for connection") + return nil + }) + } + require.NoError(t, eg.Wait()) + + // ensure that genesis aligns with layer timings + genesis := time.Now().Add(layerDuration).Round(layerDuration) + layerDuration := 3 * time.Second + epoch := layersPerEpoch * layerDuration + poetCfg := activation.PoetConfig{ + PhaseShift: epoch / 2, + CycleGap: epoch / 4, + GracePeriod: epoch / 5, + RequestTimeout: epoch / 5, + RequestRetryDelay: epoch / 50, + MaxRequestRetries: 10, + } + poetProver := spawnPoet( + t, + WithGenesis(genesis), + WithEpochDuration(epoch), + WithPhaseShift(poetCfg.PhaseShift), + WithCycleGap(poetCfg.CycleGap), + ) + + clock, err := timesync.NewClock( + timesync.WithGenesisTime(genesis), + timesync.WithLayerDuration(layerDuration), + timesync.WithTickInterval(100*time.Millisecond), + timesync.WithLogger(logger), + ) + require.NoError(t, err) + t.Cleanup(clock.Close) + + poetDb := activation.NewPoetDb(db, log.NewFromLog(logger).Named("poetDb")) + + localDB := localsql.InMemory() + nb, err := activation.NewNIPostBuilder( + localDB, + poetDb, + svc, + []types.PoetServer{{Address: poetProver.RestURL().String()}}, + logger.Named("nipostBuilder"), + poetCfg, + clock, + ) + require.NoError(t, err) + + conf := activation.Config{ + GoldenATXID: goldenATX, + RegossipInterval: 0, + } + + var atxMtx sync.Mutex + atxs := make(map[types.NodeID]types.ActivationTx) + endChan := make(chan struct{}) + mpub := mocks.NewMockPublisher(ctrl) + mpub.EXPECT().Publish(gomock.Any(), pubsub.AtxProtocol, gomock.Any()).DoAndReturn( + func(ctx context.Context, topic string, got []byte) error { + atxMtx.Lock() + defer atxMtx.Unlock() + var gotAtx types.ActivationTx + require.NoError(t, codec.Decode(got, &gotAtx)) + atxs[gotAtx.SmesherID] = gotAtx + if len(atxs) == numSigners { + close(endChan) + } + return nil + }, + ).Times(numSigners) + + tab := activation.NewBuilder( + conf, + cdb, + localDB, + mpub, + nb, + clock, + syncer, + logger, + activation.WithPoetConfig(poetCfg), + ) + for _, sig := range signers { + tab.Register(sig) + } + + require.NoError(t, tab.StartSmeshing(types.Address{})) + + <-endChan + + require.NoError(t, tab.StopSmeshing(false)) + + verifier, err := activation.NewPostVerifier(cfg, logger.Named("verifier")) + require.NoError(t, err) + t.Cleanup(func() { assert.NoError(t, verifier.Close()) }) + + v := activation.NewValidator(nil, poetDb, cfg, opts.Scrypt, verifier) + for _, sig := range signers { + atx := atxs[sig.NodeID()] + + _, err = v.NIPost( + context.Background(), + sig.NodeID(), + *atx.CommitmentATX, + atx.NIPost, + atx.NIPostChallenge.Hash(), + atx.NumUnits, + ) + require.NoError(t, err) + + err := v.VRFNonce( + sig.NodeID(), + *atx.CommitmentATX, + atx.VRFNonce, + atx.NIPost.PostMetadata, + atx.NumUnits, + ) + require.NoError(t, err) + + require.Equal(t, postGenesisEpoch, atx.NIPostChallenge.TargetEpoch()) + require.Equal(t, types.EmptyATXID, atx.NIPostChallenge.PrevATXID) + require.Equal(t, goldenATX, atx.NIPostChallenge.PositioningATX) + require.Equal(t, uint64(0), atx.NIPostChallenge.Sequence) + + require.Equal(t, types.Address{}, atx.Coinbase) + require.Equal(t, sig.NodeID(), *atx.NodeID) + } +} diff --git a/activation/e2e/nipost_test.go b/activation/e2e/nipost_test.go index df3a344d60..1ba62cef65 100644 --- a/activation/e2e/nipost_test.go +++ b/activation/e2e/nipost_test.go @@ -117,16 +117,17 @@ func TestNIPostBuilderWithClients(t *testing.T) { logger := zaptest.NewLogger(t) goldenATX := types.ATXID{2, 3, 4} cfg := activation.DefaultPostConfig() - cdb := datastore.NewCachedDB(sql.InMemory(), log.NewFromLog(logger)) - validator := activation.NewMocknipostValidator(ctrl) + db := sql.InMemory() + cdb := datastore.NewCachedDB(db, log.NewFromLog(logger)) - syncer := activation.NewMocksyncer(gomock.NewController(t)) + syncer := activation.NewMocksyncer(ctrl) syncer.EXPECT().RegisterForATXSynced().AnyTimes().DoAndReturn(func() <-chan struct{} { synced := make(chan struct{}) close(synced) return synced }) + validator := activation.NewMocknipostValidator(ctrl) mgr, err := activation.NewPostSetupManager(sig.NodeID(), cfg, logger, cdb, goldenATX, syncer, validator) require.NoError(t, err) @@ -166,7 +167,7 @@ func TestNIPostBuilderWithClients(t *testing.T) { require.NoError(t, err) t.Cleanup(func() { assert.NoError(t, verifier.Close()) }) - poetDb := activation.NewPoetDb(sql.InMemory(), log.NewFromLog(logger).Named("poetDb")) + poetDb := activation.NewPoetDb(db, log.NewFromLog(logger).Named("poetDb")) svc := grpcserver.NewPostService(logger) grpcCfg, cleanup := launchServer(t, svc) @@ -179,14 +180,13 @@ func TestNIPostBuilderWithClients(t *testing.T) { return err == nil }, 10*time.Second, 100*time.Millisecond, "timed out waiting for connection") - db := localsql.InMemory() + localDB := localsql.InMemory() nb, err := activation.NewNIPostBuilder( - db, + localDB, poetDb, svc, []types.PoetServer{{Address: poetProver.RestURL().String()}}, logger.Named("nipostBuilder"), - sig, poetCfg, mclock, ) @@ -195,7 +195,7 @@ func TestNIPostBuilderWithClients(t *testing.T) { challenge := types.NIPostChallenge{ PublishEpoch: postGenesisEpoch + 2, } - nipost, err := nb.BuildNIPost(context.Background(), &challenge) + nipost, err := nb.BuildNIPost(context.Background(), sig, &challenge) require.NoError(t, err) v := activation.NewValidator(nil, poetDb, cfg, opts.Scrypt, verifier) @@ -239,7 +239,6 @@ func TestNIPostBuilder_Close(t *testing.T) { svc, []types.PoetServer{{Address: poetProver.RestURL().String()}}, logger.Named("nipostBuilder"), - sig, activation.PoetConfig{}, mclock, ) @@ -250,7 +249,7 @@ func TestNIPostBuilder_Close(t *testing.T) { } ctx, cancel := context.WithCancel(context.Background()) cancel() - nipost, err := nb.BuildNIPost(ctx, &challenge) + nipost, err := nb.BuildNIPost(ctx, sig, &challenge) require.ErrorIs(t, err, context.Canceled) require.Nil(t, nipost) } @@ -264,16 +263,17 @@ func TestNewNIPostBuilderNotInitialized(t *testing.T) { logger := zaptest.NewLogger(t) goldenATX := types.ATXID{2, 3, 4} cfg := activation.DefaultPostConfig() - cdb := datastore.NewCachedDB(sql.InMemory(), log.NewFromLog(logger)) + db := sql.InMemory() + cdb := datastore.NewCachedDB(db, log.NewFromLog(logger)) - syncer := activation.NewMocksyncer(gomock.NewController(t)) + syncer := activation.NewMocksyncer(ctrl) syncer.EXPECT().RegisterForATXSynced().AnyTimes().DoAndReturn(func() <-chan struct{} { synced := make(chan struct{}) close(synced) return synced }) - validator := activation.NewMocknipostValidator(gomock.NewController(t)) + validator := activation.NewMocknipostValidator(ctrl) mgr, err := activation.NewPostSetupManager(sig.NodeID(), cfg, logger, cdb, goldenATX, syncer, validator) require.NoError(t, err) @@ -303,20 +303,19 @@ func TestNewNIPostBuilderNotInitialized(t *testing.T) { }, ) - poetDb := activation.NewPoetDb(sql.InMemory(), log.NewFromLog(logger).Named("poetDb")) + poetDb := activation.NewPoetDb(db, log.NewFromLog(logger).Named("poetDb")) svc := grpcserver.NewPostService(logger) grpcCfg, cleanup := launchServer(t, svc) t.Cleanup(cleanup) - db := localsql.InMemory() + localDB := localsql.InMemory() nb, err := activation.NewNIPostBuilder( - db, + localDB, poetDb, svc, []types.PoetServer{{Address: poetProver.RestURL().String()}}, logger.Named("nipostBuilder"), - sig, poetCfg, mclock, ) @@ -336,7 +335,7 @@ func TestNewNIPostBuilderNotInitialized(t *testing.T) { challenge := types.NIPostChallenge{ PublishEpoch: postGenesisEpoch + 2, } - nipost, err := nb.BuildNIPost(context.Background(), &challenge) + nipost, err := nb.BuildNIPost(context.Background(), sig, &challenge) require.NoError(t, err) require.NotNil(t, nipost) @@ -355,3 +354,126 @@ func TestNewNIPostBuilderNotInitialized(t *testing.T) { ) require.NoError(t, err) } + +func Test_NIPostBuilderWithMultipleClients(t *testing.T) { + ctrl := gomock.NewController(t) + + signers := make(map[types.NodeID]*signing.EdSigner, 3) + for i := 0; i < 3; i++ { + sig, err := signing.NewEdSigner() + require.NoError(t, err) + + signers[sig.NodeID()] = sig + } + + logger := zaptest.NewLogger(t) + goldenATX := types.ATXID{2, 3, 4} + cfg := activation.DefaultPostConfig() + db := sql.InMemory() + cdb := datastore.NewCachedDB(db, log.NewFromLog(logger)) + + syncer := activation.NewMocksyncer(ctrl) + syncer.EXPECT().RegisterForATXSynced().AnyTimes().DoAndReturn(func() <-chan struct{} { + synced := make(chan struct{}) + close(synced) + return synced + }) + + svc := grpcserver.NewPostService(logger) + grpcCfg, cleanup := launchServer(t, svc) + t.Cleanup(cleanup) + + opts := activation.DefaultPostSetupOpts() + opts.ProviderID.SetUint32(initialization.CPUProviderID()) + opts.Scrypt.N = 2 // Speedup initialization in tests. + + validator := activation.NewMocknipostValidator(ctrl) + var eg errgroup.Group + for _, sig := range signers { + sig := sig + opts := opts + eg.Go(func() error { + mgr, err := activation.NewPostSetupManager(sig.NodeID(), cfg, logger, cdb, goldenATX, syncer, validator) + require.NoError(t, err) + + opts.DataDir = t.TempDir() + initPost(t, mgr, opts) + t.Cleanup(launchPostSupervisor(t, logger, mgr, grpcCfg, opts)) + + require.Eventually(t, func() bool { + _, err := svc.Client(sig.NodeID()) + return err == nil + }, 10*time.Second, 100*time.Millisecond, "timed out waiting for connection") + return nil + }) + } + require.NoError(t, eg.Wait()) + + // ensure that genesis aligns with layer timings + genesis := time.Now().Add(layerDuration).Round(layerDuration) + epoch := layersPerEpoch * layerDuration + poetCfg := activation.PoetConfig{ + PhaseShift: epoch / 2, + CycleGap: epoch / 4, + GracePeriod: epoch / 5, + RequestTimeout: epoch / 5, + RequestRetryDelay: epoch / 50, + MaxRequestRetries: 10, + } + poetProver := spawnPoet( + t, + WithGenesis(genesis), + WithEpochDuration(epoch), + WithPhaseShift(poetCfg.PhaseShift), + WithCycleGap(poetCfg.CycleGap), + ) + + mclock := activation.NewMocklayerClock(ctrl) + mclock.EXPECT().LayerToTime(gomock.Any()).AnyTimes().DoAndReturn( + func(got types.LayerID) time.Time { + return genesis.Add(layerDuration * time.Duration(got)) + }, + ) + + verifier, err := activation.NewPostVerifier(cfg, logger.Named("verifier")) + require.NoError(t, err) + t.Cleanup(func() { assert.NoError(t, verifier.Close()) }) + + poetDb := activation.NewPoetDb(db, log.NewFromLog(logger).Named("poetDb")) + + localDB := localsql.InMemory() + nb, err := activation.NewNIPostBuilder( + localDB, + poetDb, + svc, + []types.PoetServer{{Address: poetProver.RestURL().String()}}, + logger.Named("nipostBuilder"), + poetCfg, + mclock, + ) + require.NoError(t, err) + + challenge := types.NIPostChallenge{ + PublishEpoch: postGenesisEpoch + 2, + } + for _, sig := range signers { + sig := sig + eg.Go(func() error { + nipost, err := nb.BuildNIPost(context.Background(), sig, &challenge) + require.NoError(t, err) + + v := activation.NewValidator(nil, poetDb, cfg, opts.Scrypt, verifier) + _, err = v.NIPost( + context.Background(), + sig.NodeID(), + goldenATX, + nipost.NIPost, + challenge.Hash(), + nipost.NumUnits, + ) + require.NoError(t, err) + return nil + }) + } + require.NoError(t, eg.Wait()) +} diff --git a/activation/e2e/validation_test.go b/activation/e2e/validation_test.go index 08b2c0467f..376a059f91 100644 --- a/activation/e2e/validation_test.go +++ b/activation/e2e/validation_test.go @@ -106,13 +106,12 @@ func TestValidator_Validate(t *testing.T) { svc, []types.PoetServer{{Address: poetProver.RestURL().String()}}, logger.Named("nipostBuilder"), - sig, poetCfg, mclock, ) require.NoError(t, err) - nipost, err := nb.BuildNIPost(context.Background(), &challenge) + nipost, err := nb.BuildNIPost(context.Background(), sig, &challenge) require.NoError(t, err) v := activation.NewValidator(cdb, poetDb, cfg, opts.Scrypt, verifier) diff --git a/activation/handler_test.go b/activation/handler_test.go index c673976f75..124a5de7e3 100644 --- a/activation/handler_test.go +++ b/activation/handler_test.go @@ -351,8 +351,14 @@ func TestHandler_SyntacticallyValidateAtx(t *testing.T) { atxHdlr := newTestHandler(t, goldenATXID) require.NoError(t, atxs.Add(atxHdlr.cdb, prevAtx)) - challenge := newChallenge(prevAtx.Sequence+1, prevAtx.ID(), prevAtx.ID(), currentLayer.GetEpoch(), nil) - atx := newAtx(t, sig, challenge, &types.NIPost{}, 100, types.GenerateAddress([]byte("aaaa"))) + challenge := types.NIPostChallenge{ + Sequence: prevAtx.Sequence + 1, + PrevATXID: prevAtx.ID(), + PublishEpoch: currentLayer.GetEpoch(), + PositioningATX: prevAtx.ID(), + CommitmentATX: nil, + } + atx := newAtx(challenge, &types.NIPost{}, 100, types.GenerateAddress([]byte("aaaa"))) atx.NIPost = newNIPostWithChallenge(t, atx.NIPostChallenge.Hash(), poetRef).NIPost require.NoError(t, SignAndFinalizeAtx(sig, atx)) @@ -374,9 +380,15 @@ func TestHandler_SyntacticallyValidateAtx(t *testing.T) { atxHdlr := newTestHandler(t, goldenATXID) require.NoError(t, atxs.Add(atxHdlr.cdb, prevAtx)) - challenge := newChallenge(prevAtx.Sequence+1, prevAtx.ID(), prevAtx.ID(), currentLayer.GetEpoch(), nil) + challenge := types.NIPostChallenge{ + Sequence: prevAtx.Sequence + 1, + PrevATXID: prevAtx.ID(), + PublishEpoch: currentLayer.GetEpoch(), + PositioningATX: prevAtx.ID(), + CommitmentATX: nil, + } nonce := types.VRFPostIndex(999) - atx := newAtx(t, sig, challenge, &types.NIPost{}, 100, types.GenerateAddress([]byte("aaaa"))) + atx := newAtx(challenge, &types.NIPost{}, 100, types.GenerateAddress([]byte("aaaa"))) atx.NIPost = newNIPostWithChallenge(t, atx.NIPostChallenge.Hash(), poetRef).NIPost atx.VRFNonce = &nonce require.NoError(t, SignAndFinalizeAtx(sig, atx)) @@ -403,15 +415,15 @@ func TestHandler_SyntacticallyValidateAtx(t *testing.T) { atxHdlr := newTestHandler(t, goldenATXID) require.NoError(t, atxs.Add(atxHdlr.cdb, prevAtx)) - challenge := newChallenge(prevAtx.Sequence+1, prevAtx.ID(), prevAtx.ID(), currentLayer.GetEpoch(), nil) - atx := newAtx( - t, - sig, - challenge, - &types.NIPost{}, - 90, - types.GenerateAddress([]byte("aaaa")), - ) // numunits decreased from 100 to 90 between atx and prevAtx + challenge := types.NIPostChallenge{ + Sequence: prevAtx.Sequence + 1, + PrevATXID: prevAtx.ID(), + PublishEpoch: currentLayer.GetEpoch(), + PositioningATX: prevAtx.ID(), + CommitmentATX: nil, + } + // numunits decreased from 100 to 90 between atx and prevAtx + atx := newAtx(challenge, &types.NIPost{}, 90, types.GenerateAddress([]byte("aaaa"))) atx.NIPost = newNIPostWithChallenge(t, atx.NIPostChallenge.Hash(), poetRef).NIPost require.NoError(t, SignAndFinalizeAtx(sig, atx)) @@ -435,15 +447,15 @@ func TestHandler_SyntacticallyValidateAtx(t *testing.T) { atxHdlr := newTestHandler(t, goldenATXID) require.NoError(t, atxs.Add(atxHdlr.cdb, prevAtx)) - challenge := newChallenge(prevAtx.Sequence+1, prevAtx.ID(), prevAtx.ID(), currentLayer.GetEpoch(), nil) - atx := newAtx( - t, - sig, - challenge, - &types.NIPost{}, - 110, - types.GenerateAddress([]byte("aaaa")), - ) // numunits increased from 100 to 110 between atx and prevAtx + challenge := types.NIPostChallenge{ + Sequence: prevAtx.Sequence + 1, + PrevATXID: prevAtx.ID(), + PublishEpoch: currentLayer.GetEpoch(), + PositioningATX: prevAtx.ID(), + CommitmentATX: nil, + } + // numunits increased from 100 to 110 between atx and prevAtx + atx := newAtx(challenge, &types.NIPost{}, 110, types.GenerateAddress([]byte("aaaa"))) atx.NIPost = newNIPostWithChallenge(t, atx.NIPostChallenge.Hash(), poetRef).NIPost require.NoError(t, SignAndFinalizeAtx(sig, atx)) @@ -470,15 +482,15 @@ func TestHandler_SyntacticallyValidateAtx(t *testing.T) { atxHdlr := newTestHandler(t, goldenATXID) require.NoError(t, atxs.Add(atxHdlr.cdb, prevAtx)) - challenge := newChallenge(prevAtx.Sequence+1, prevAtx.ID(), prevAtx.ID(), currentLayer.GetEpoch(), nil) - atx := newAtx( - t, - sig, - challenge, - &types.NIPost{}, - 110, - types.GenerateAddress([]byte("aaaa")), - ) // numunits increased from 100 to 110 between atx and prevAtx + challenge := types.NIPostChallenge{ + Sequence: prevAtx.Sequence + 1, + PrevATXID: prevAtx.ID(), + PublishEpoch: currentLayer.GetEpoch(), + PositioningATX: prevAtx.ID(), + CommitmentATX: nil, + } + // numunits increased from 100 to 110 between atx and prevAtx + atx := newAtx(challenge, &types.NIPost{}, 110, types.GenerateAddress([]byte("aaaa"))) atx.NIPost = newNIPostWithChallenge(t, atx.NIPostChallenge.Hash(), poetRef).NIPost require.NoError(t, SignAndFinalizeAtx(sig, atx)) @@ -499,8 +511,14 @@ func TestHandler_SyntacticallyValidateAtx(t *testing.T) { require.NoError(t, atxs.Add(atxHdlr.cdb, posAtx)) ctxID := posAtx.ID() - challenge := newChallenge(0, types.EmptyATXID, posAtx.ID(), currentLayer.GetEpoch(), &ctxID) - atx := newAtx(t, sig, challenge, &types.NIPost{}, 100, types.GenerateAddress([]byte("aaaa"))) + challenge := types.NIPostChallenge{ + Sequence: 0, + PrevATXID: types.EmptyATXID, + PublishEpoch: currentLayer.GetEpoch(), + PositioningATX: posAtx.ID(), + CommitmentATX: &ctxID, + } + atx := newAtx(challenge, &types.NIPost{}, 100, types.GenerateAddress([]byte("aaaa"))) atx.InitialPost = &types.Post{ Nonce: 0, Indices: make([]byte, 10), @@ -535,8 +553,14 @@ func TestHandler_SyntacticallyValidateAtx(t *testing.T) { atxHdlr := newTestHandler(t, goldenATXID) - challenge := newChallenge(prevAtx.Sequence+1, prevAtx.ID(), prevAtx.ID(), currentLayer.GetEpoch().Add(2), nil) - atx := newAtx(t, sig, challenge, &types.NIPost{}, 100, types.GenerateAddress([]byte("aaaa"))) + challenge := types.NIPostChallenge{ + Sequence: prevAtx.Sequence + 1, + PrevATXID: prevAtx.ID(), + PublishEpoch: currentLayer.GetEpoch().Add(2), + PositioningATX: prevAtx.ID(), + CommitmentATX: nil, + } + atx := newAtx(challenge, &types.NIPost{}, 100, types.GenerateAddress([]byte("aaaa"))) atx.NIPost = newNIPostWithChallenge(t, atx.NIPostChallenge.Hash(), poetRef).NIPost require.NoError(t, SignAndFinalizeAtx(sig, atx)) @@ -551,8 +575,14 @@ func TestHandler_SyntacticallyValidateAtx(t *testing.T) { atxHdlr := newTestHandler(t, goldenATXID) require.NoError(t, atxs.Add(atxHdlr.cdb, prevAtx)) - challenge := newChallenge(prevAtx.Sequence+1, prevAtx.ID(), prevAtx.ID(), currentLayer.GetEpoch(), nil) - atx := newAtx(t, sig, challenge, &types.NIPost{}, 100, types.GenerateAddress([]byte("aaaa"))) + challenge := types.NIPostChallenge{ + Sequence: prevAtx.Sequence + 1, + PrevATXID: prevAtx.ID(), + PublishEpoch: currentLayer.GetEpoch(), + PositioningATX: prevAtx.ID(), + CommitmentATX: nil, + } + atx := newAtx(challenge, &types.NIPost{}, 100, types.GenerateAddress([]byte("aaaa"))) require.NoError(t, SignAndFinalizeAtx(sig, atx)) atxHdlr.mclock.EXPECT().CurrentLayer().Return(currentLayer) @@ -570,8 +600,14 @@ func TestHandler_SyntacticallyValidateAtx(t *testing.T) { atxHdlr := newTestHandler(t, goldenATXID) require.NoError(t, atxs.Add(atxHdlr.cdb, prevAtx)) - challenge := newChallenge(prevAtx.Sequence+1, prevAtx.ID(), prevAtx.ID(), currentLayer.GetEpoch(), nil) - atx := newAtx(t, sig, challenge, &types.NIPost{}, 100, types.GenerateAddress([]byte("aaaa"))) + challenge := types.NIPostChallenge{ + Sequence: prevAtx.Sequence + 1, + PrevATXID: prevAtx.ID(), + PublishEpoch: currentLayer.GetEpoch(), + PositioningATX: prevAtx.ID(), + CommitmentATX: nil, + } + atx := newAtx(challenge, &types.NIPost{}, 100, types.GenerateAddress([]byte("aaaa"))) require.NoError(t, SignAndFinalizeAtx(sig, atx)) atxHdlr.mclock.EXPECT().CurrentLayer().Return(currentLayer) @@ -590,8 +626,14 @@ func TestHandler_SyntacticallyValidateAtx(t *testing.T) { atxHdlr := newTestHandler(t, goldenATXID) cATX := posAtx.ID() - challenge := newChallenge(0, types.EmptyATXID, posAtx.ID(), currentLayer.GetEpoch(), &cATX) - atx := newAtx(t, sig, challenge, npst.NIPost, npst.NumUnits, types.GenerateAddress([]byte("aaaa"))) + challenge := types.NIPostChallenge{ + Sequence: 0, + PrevATXID: types.EmptyATXID, + PublishEpoch: currentLayer.GetEpoch(), + PositioningATX: posAtx.ID(), + CommitmentATX: &cATX, + } + atx := newAtx(challenge, npst.NIPost, npst.NumUnits, types.GenerateAddress([]byte("aaaa"))) atx.InitialPost = &types.Post{ Nonce: 0, Indices: make([]byte, 10), @@ -620,8 +662,14 @@ func TestHandler_SyntacticallyValidateAtx(t *testing.T) { atxHdlr := newTestHandler(t, goldenATXID) ctxID := posAtx.ID() - challenge := newChallenge(0, types.EmptyATXID, posAtx.ID(), currentLayer.GetEpoch(), &ctxID) - atx := newAtx(t, sig, challenge, &types.NIPost{}, 100, types.GenerateAddress([]byte("aaaa"))) + challenge := types.NIPostChallenge{ + Sequence: 0, + PrevATXID: types.EmptyATXID, + PublishEpoch: currentLayer.GetEpoch(), + PositioningATX: posAtx.ID(), + CommitmentATX: &ctxID, + } + atx := newAtx(challenge, &types.NIPost{}, 100, types.GenerateAddress([]byte("aaaa"))) atx.InitialPost = &types.Post{ Nonce: 0, Indices: make([]byte, 10), @@ -637,8 +685,14 @@ func TestHandler_SyntacticallyValidateAtx(t *testing.T) { atxHdlr := newTestHandler(t, goldenATXID) cATX := posAtx.ID() - challenge := newChallenge(0, types.EmptyATXID, posAtx.ID(), currentLayer.GetEpoch(), &cATX) - atx := newAtx(t, sig, challenge, npst.NIPost, npst.NumUnits, types.GenerateAddress([]byte("aaaa"))) + challenge := types.NIPostChallenge{ + Sequence: 0, + PrevATXID: types.EmptyATXID, + PublishEpoch: currentLayer.GetEpoch(), + PositioningATX: posAtx.ID(), + CommitmentATX: &cATX, + } + atx := newAtx(challenge, npst.NIPost, npst.NumUnits, types.GenerateAddress([]byte("aaaa"))) atx.InitialPost = &types.Post{ Nonce: 0, Indices: make([]byte, 10), @@ -657,8 +711,14 @@ func TestHandler_SyntacticallyValidateAtx(t *testing.T) { atxHdlr := newTestHandler(t, goldenATXID) ctxID := posAtx.ID() - challenge := newChallenge(0, types.EmptyATXID, posAtx.ID(), currentLayer.GetEpoch(), &ctxID) - atx := newAtx(t, sig, challenge, &types.NIPost{}, 100, types.GenerateAddress([]byte("aaaa"))) + challenge := types.NIPostChallenge{ + Sequence: 0, + PrevATXID: types.EmptyATXID, + PublishEpoch: currentLayer.GetEpoch(), + PositioningATX: posAtx.ID(), + CommitmentATX: &ctxID, + } + atx := newAtx(challenge, &types.NIPost{}, 100, types.GenerateAddress([]byte("aaaa"))) atx.InitialPost = &types.Post{ Nonce: 0, Indices: make([]byte, 10), @@ -683,8 +743,14 @@ func TestHandler_SyntacticallyValidateAtx(t *testing.T) { atxHdlr := newTestHandler(t, goldenATXID) - challenge := newChallenge(0, types.EmptyATXID, posAtx.ID(), currentLayer.GetEpoch(), nil) - atx := newAtx(t, sig, challenge, &types.NIPost{}, 100, types.GenerateAddress([]byte("aaaa"))) + challenge := types.NIPostChallenge{ + Sequence: 0, + PrevATXID: types.EmptyATXID, + PublishEpoch: currentLayer.GetEpoch(), + PositioningATX: posAtx.ID(), + CommitmentATX: nil, + } + atx := newAtx(challenge, &types.NIPost{}, 100, types.GenerateAddress([]byte("aaaa"))) atx.CommitmentATX = &goldenATXID atxHdlr.mclock.EXPECT().CurrentLayer().Return(currentLayer) @@ -698,8 +764,14 @@ func TestHandler_SyntacticallyValidateAtx(t *testing.T) { atxHdlr := newTestHandler(t, goldenATXID) cATX := posAtx.ID() - challenge := newChallenge(0, types.EmptyATXID, posAtx.ID(), currentLayer.GetEpoch(), &cATX) - atx := newAtx(t, sig, challenge, npst.NIPost, npst.NumUnits, types.GenerateAddress([]byte("aaaa"))) + challenge := types.NIPostChallenge{ + Sequence: 0, + PrevATXID: types.EmptyATXID, + PublishEpoch: currentLayer.GetEpoch(), + PositioningATX: posAtx.ID(), + CommitmentATX: &cATX, + } + atx := newAtx(challenge, npst.NIPost, npst.NumUnits, types.GenerateAddress([]byte("aaaa"))) atx.InitialPost = &types.Post{ Nonce: 0, Indices: make([]byte, 10), @@ -724,8 +796,14 @@ func TestHandler_SyntacticallyValidateAtx(t *testing.T) { atxHdlr := newTestHandler(t, goldenATXID) require.NoError(t, atxs.Add(atxHdlr.cdb, prevAtx)) - challenge := newChallenge(1, prevAtx.ID(), prevAtx.ID(), currentLayer.GetEpoch(), nil) - atx := newAtx(t, sig, challenge, &types.NIPost{}, 100, types.GenerateAddress([]byte("aaaa"))) + challenge := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: prevAtx.ID(), + PublishEpoch: currentLayer.GetEpoch(), + PositioningATX: prevAtx.ID(), + CommitmentATX: nil, + } + atx := newAtx(challenge, &types.NIPost{}, 100, types.GenerateAddress([]byte("aaaa"))) atx.InitialPost = &types.Post{ Nonce: 0, Indices: make([]byte, 10), @@ -743,8 +821,14 @@ func TestHandler_SyntacticallyValidateAtx(t *testing.T) { atxHdlr := newTestHandler(t, goldenATXID) require.NoError(t, atxs.Add(atxHdlr.cdb, prevAtx)) - challenge := newChallenge(1, prevAtx.ID(), prevAtx.ID(), currentLayer.GetEpoch(), nil) - atx := newAtx(t, sig, challenge, &types.NIPost{}, 100, types.GenerateAddress([]byte("aaaa"))) + challenge := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: prevAtx.ID(), + PublishEpoch: currentLayer.GetEpoch(), + PositioningATX: prevAtx.ID(), + CommitmentATX: nil, + } + atx := newAtx(challenge, &types.NIPost{}, 100, types.GenerateAddress([]byte("aaaa"))) atx.NIPost = newNIPostWithChallenge(t, atx.NIPostChallenge.Hash(), poetRef).NIPost atx.InnerActivationTx.NodeID = new(types.NodeID) *atx.InnerActivationTx.NodeID = sig.NodeID() @@ -794,8 +878,14 @@ func TestHandler_ContextuallyValidateAtx(t *testing.T) { atxHdlr := newTestHandler(t, goldenATXID) - challenge := newChallenge(1, types.EmptyATXID, goldenATXID, 0, nil) - atx := newAtx(t, sig, challenge, nil, 2, types.Address{}) + challenge := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: types.EmptyATXID, + PublishEpoch: 0, + PositioningATX: goldenATXID, + CommitmentATX: nil, + } + atx := newAtx(challenge, nil, 2, types.Address{}) require.NoError(t, SignAndFinalizeAtx(sig, atx)) vAtx, err := atx.Verify(0, 1) @@ -809,15 +899,27 @@ func TestHandler_ContextuallyValidateAtx(t *testing.T) { atxHdlr := newTestHandler(t, goldenATXID) require.NoError(t, atxs.Add(atxHdlr.cdb, prevAtx)) - challenge := newChallenge(1, types.EmptyATXID, goldenATXID, currentLayer.GetEpoch(), nil) - atx := newAtx(t, sig, challenge, &types.NIPost{}, 100, coinbase) + challenge := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: types.EmptyATXID, + PublishEpoch: currentLayer.GetEpoch(), + PositioningATX: goldenATXID, + CommitmentATX: nil, + } + atx := newAtx(challenge, &types.NIPost{}, 100, coinbase) require.NoError(t, SignAndFinalizeAtx(sig, atx)) vAtx, err := atx.Verify(0, 1) require.NoError(t, err) require.NoError(t, atxs.Add(atxHdlr.cdb, vAtx)) - challenge = newChallenge(1, prevAtx.ID(), goldenATXID, currentLayer.GetEpoch()-1, nil) - atx = newAtx(t, sig, challenge, &types.NIPost{}, 100, coinbase) + challenge = types.NIPostChallenge{ + Sequence: 1, + PrevATXID: prevAtx.ID(), + PublishEpoch: currentLayer.GetEpoch() - 1, + PositioningATX: goldenATXID, + CommitmentATX: nil, + } + atx = newAtx(challenge, &types.NIPost{}, 100, coinbase) require.NoError(t, SignAndFinalizeAtx(sig, atx)) vAtx, err = atx.Verify(0, 1) require.NoError(t, err) @@ -831,8 +933,14 @@ func TestHandler_ContextuallyValidateAtx(t *testing.T) { atxHdlr := newTestHandler(t, goldenATXID) - challenge := newChallenge(1, types.RandomATXID(), prevAtx.ID(), 0, nil) - atx := newAtx(t, sig, challenge, nil, 2, types.Address{}) + challenge := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: types.RandomATXID(), + PublishEpoch: 0, + PositioningATX: prevAtx.ID(), + CommitmentATX: nil, + } + atx := newAtx(challenge, nil, 2, types.Address{}) require.NoError(t, SignAndFinalizeAtx(sig, atx)) vAtx, err := atx.Verify(0, 1) require.NoError(t, err) @@ -997,7 +1105,7 @@ func TestHandler_ProcessAtx(t *testing.T) { atxHdlr.log, atxHdlr.cdb, atxHdlr.edVerifier, - NewMockPostVerifier(gomock.NewController(t)), + nil, &got, ) require.NoError(t, err) @@ -1085,11 +1193,17 @@ func TestHandler_PublishesPostMalfeasanceProofs(t *testing.T) { require.ErrorIs(t, err, sql.ErrNotFound) require.Nil(t, proof) - ch := newChallenge(0, types.EmptyATXID, goldenATXID, 1, &goldenATXID) - ch.InitialPost = &types.Post{} + ch := types.NIPostChallenge{ + Sequence: 0, + PrevATXID: types.EmptyATXID, + PublishEpoch: 1, + PositioningATX: goldenATXID, + CommitmentATX: &goldenATXID, + InitialPost: &types.Post{}, + } nipost := newNIPostWithChallenge(t, types.HexToHash32("0x3333"), []byte{0x76, 0x45}) - atx := newAtx(t, sig, ch, nipost.NIPost, 100, types.GenerateAddress([]byte("aaaa"))) + atx := newAtx(ch, nipost.NIPost, 100, types.GenerateAddress([]byte("aaaa"))) atx.NodeID = &nodeID vrfNonce := types.VRFPostIndex(0) atx.VRFNonce = &vrfNonce @@ -1237,9 +1351,15 @@ func BenchmarkNewActivationDb(b *testing.B) { eStart := time.Now() for epoch := postGenesisEpoch; epoch < postGenesisEpoch+numOfEpochs; epoch++ { for i := 0; i < numOfMiners; i++ { - challenge := newChallenge(1, prevAtxs[i], posAtx, epoch, nil) + challenge := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: prevAtxs[i], + PublishEpoch: epoch, + PositioningATX: posAtx, + CommitmentATX: nil, + } npst := newNIPostWithChallenge(b, challenge.Hash(), poetBytes) - atx = newAtx(b, sigs[i], challenge, npst.NIPost, npst.NumUnits, coinbase) + atx = newAtx(challenge, npst.NIPost, npst.NumUnits, coinbase) SignAndFinalizeAtx(sigs[i], atx) vAtx, err := atx.Verify(0, 1) r.NoError(err) @@ -1398,21 +1518,12 @@ func TestHandler_HandleParallelGossipAtx(t *testing.T) { atxHdlr.mclock.EXPECT().CurrentLayer().Return(atx.PublishEpoch.FirstLayer()) atxHdlr.mValidator.EXPECT().VRFNonce(nodeID, goldenATXID, &vrfNonce, gomock.Any(), atx.NumUnits) atxHdlr.mValidator.EXPECT(). - Post(gomock.Any(), atx.SmesherID, goldenATXID, atx.InitialPost, gomock.Any(), atx.NumUnits). - DoAndReturn( - func( - _ context.Context, - _ types.NodeID, - _ types.ATXID, - _ *types.Post, - _ *types.PostMetadata, - _ uint32, - _ ...validatorOption, - ) error { - time.Sleep(100 * time.Millisecond) - return nil - }, - ) + Post(gomock.Any(), atx.SmesherID, goldenATXID, atx.InitialPost, gomock.Any(), atx.NumUnits).DoAndReturn( + func(context.Context, types.NodeID, types.ATXID, *types.Post, *types.PostMetadata, uint32, ...validatorOption) error { + time.Sleep(100 * time.Millisecond) + return nil + }, + ) atxHdlr.mockFetch.EXPECT().RegisterPeerHashes(gomock.Any(), gomock.Any()) atxHdlr.mockFetch.EXPECT().GetPoetProof(gomock.Any(), atx.GetPoetProofRef()) atxHdlr.mValidator.EXPECT().InitialNIPostChallenge(&atx.NIPostChallenge, gomock.Any(), goldenATXID) @@ -1578,10 +1689,16 @@ func BenchmarkGetAtxHeaderWithConcurrentProcessAtx(b *testing.B) { go func() { defer wg.Done() for i := 0; ; i++ { - challenge := newChallenge(uint64(i), types.EmptyATXID, goldenATXID, 0, nil) + challenge := types.NIPostChallenge{ + Sequence: uint64(i), + PrevATXID: types.EmptyATXID, + PublishEpoch: 0, + PositioningATX: goldenATXID, + CommitmentATX: nil, + } sig, err := signing.NewEdSigner() require.NoError(b, err) - atx := newAtx(b, sig, challenge, nil, 1, types.Address{}) + atx := newAtx(challenge, nil, 1, types.Address{}) require.NoError(b, SignAndFinalizeAtx(sig, atx)) vAtx, err := atx.Verify(0, 1) if !assert.NoError(b, err) { @@ -1610,9 +1727,6 @@ func BenchmarkGetAtxHeaderWithConcurrentProcessAtx(b *testing.B) { func TestHandler_FetchReferences(t *testing.T) { goldenATXID := types.ATXID{2, 3, 4} - sig, err := signing.NewEdSigner() - require.NoError(t, err) - posATX := types.ATXID{1, 2, 3} prevATX := types.ATXID{4, 5, 6} commitATX := types.ATXID{7, 8, 9} @@ -1622,9 +1736,15 @@ func TestHandler_FetchReferences(t *testing.T) { atxHdlr := newTestHandler(t, goldenATXID) coinbase := types.Address{2, 4, 5} - challenge := newChallenge(1, prevATX, posATX, types.LayerID(22).GetEpoch(), nil) + challenge := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: prevATX, + PublishEpoch: postGenesisEpoch, + PositioningATX: posATX, + CommitmentATX: nil, + } nipost := newNIPostWithChallenge(t, types.HexToHash32("55555"), []byte("66666")) - atx := newAtx(t, sig, challenge, nipost.NIPost, 2, coinbase) + atx := newAtx(challenge, nipost.NIPost, 2, coinbase) atxHdlr.mockFetch.EXPECT().GetPoetProof(gomock.Any(), atx.GetPoetProofRef()) atxHdlr.mockFetch.EXPECT(). @@ -1638,9 +1758,15 @@ func TestHandler_FetchReferences(t *testing.T) { atxHdlr := newTestHandler(t, goldenATXID) coinbase := types.Address{2, 4, 5} - challenge := newChallenge(1, prevATX, goldenATXID, types.LayerID(22).GetEpoch(), nil) + challenge := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: prevATX, + PublishEpoch: postGenesisEpoch, + PositioningATX: goldenATXID, + CommitmentATX: nil, + } nipost := newNIPostWithChallenge(t, types.HexToHash32("55555"), []byte("66666")) - atx := newAtx(t, sig, challenge, nipost.NIPost, 2, coinbase) + atx := newAtx(challenge, nipost.NIPost, 2, coinbase) atxHdlr.mockFetch.EXPECT().GetPoetProof(gomock.Any(), atx.GetPoetProofRef()) atxHdlr.mockFetch.EXPECT().GetAtxs(gomock.Any(), []types.ATXID{atx.PrevATXID}, gomock.Any()).Return(nil) @@ -1652,9 +1778,15 @@ func TestHandler_FetchReferences(t *testing.T) { atxHdlr := newTestHandler(t, goldenATXID) coinbase := types.Address{2, 4, 5} - challenge := newChallenge(1, prevATX, types.EmptyATXID, types.LayerID(22).GetEpoch(), nil) + challenge := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: prevATX, + PublishEpoch: postGenesisEpoch, + PositioningATX: types.EmptyATXID, + CommitmentATX: nil, + } nipost := newNIPostWithChallenge(t, types.HexToHash32("55555"), []byte("66666")) - atx := newAtx(t, sig, challenge, nipost.NIPost, 2, coinbase) + atx := newAtx(challenge, nipost.NIPost, 2, coinbase) atxHdlr.mockFetch.EXPECT().GetPoetProof(gomock.Any(), atx.GetPoetProofRef()) atxHdlr.mockFetch.EXPECT().GetAtxs(gomock.Any(), []types.ATXID{atx.PrevATXID}, gomock.Any()).Return(nil) @@ -1666,9 +1798,15 @@ func TestHandler_FetchReferences(t *testing.T) { atxHdlr := newTestHandler(t, goldenATXID) coinbase := types.Address{2, 4, 5} - challenge := newChallenge(1, types.EmptyATXID, posATX, types.LayerID(22).GetEpoch(), nil) + challenge := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: types.EmptyATXID, + PublishEpoch: postGenesisEpoch, + PositioningATX: posATX, + CommitmentATX: nil, + } nipost := newNIPostWithChallenge(t, types.HexToHash32("55555"), []byte("66666")) - atx := newAtx(t, sig, challenge, nipost.NIPost, 2, coinbase) + atx := newAtx(challenge, nipost.NIPost, 2, coinbase) atx.CommitmentATX = &commitATX atxHdlr.mockFetch.EXPECT().GetPoetProof(gomock.Any(), atx.GetPoetProofRef()) @@ -1682,9 +1820,15 @@ func TestHandler_FetchReferences(t *testing.T) { atxHdlr := newTestHandler(t, goldenATXID) coinbase := types.Address{2, 4, 5} - challenge := newChallenge(1, types.EmptyATXID, posATX, types.LayerID(22).GetEpoch(), nil) + challenge := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: types.EmptyATXID, + PublishEpoch: postGenesisEpoch, + PositioningATX: posATX, + CommitmentATX: nil, + } nipost := newNIPostWithChallenge(t, types.HexToHash32("55555"), []byte("66666")) - atx := newAtx(t, sig, challenge, nipost.NIPost, 2, coinbase) + atx := newAtx(challenge, nipost.NIPost, 2, coinbase) atx.CommitmentATX = &goldenATXID atxHdlr.mockFetch.EXPECT().GetPoetProof(gomock.Any(), atx.GetPoetProofRef()) @@ -1697,9 +1841,15 @@ func TestHandler_FetchReferences(t *testing.T) { atxHdlr := newTestHandler(t, goldenATXID) coinbase := types.Address{2, 4, 5} - challenge := newChallenge(1, types.EmptyATXID, types.EmptyATXID, types.LayerID(22).GetEpoch(), nil) + challenge := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: types.EmptyATXID, + PublishEpoch: postGenesisEpoch, + PositioningATX: types.EmptyATXID, + CommitmentATX: nil, + } nipost := newNIPostWithChallenge(t, types.HexToHash32("55555"), []byte("66666")) - atx := newAtx(t, sig, challenge, nipost.NIPost, 2, coinbase) + atx := newAtx(challenge, nipost.NIPost, 2, coinbase) atxHdlr.mockFetch.EXPECT().GetPoetProof(gomock.Any(), atx.GetPoetProofRef()) require.NoError(t, atxHdlr.FetchReferences(context.Background(), atx)) @@ -1710,9 +1860,15 @@ func TestHandler_FetchReferences(t *testing.T) { atxHdlr := newTestHandler(t, goldenATXID) coinbase := types.Address{2, 4, 5} - challenge := newChallenge(1, prevATX, prevATX, types.LayerID(22).GetEpoch(), nil) + challenge := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: prevATX, + PublishEpoch: postGenesisEpoch, + PositioningATX: prevATX, + CommitmentATX: nil, + } nipost := newNIPostWithChallenge(t, types.HexToHash32("55555"), []byte("66666")) - atx := newAtx(t, sig, challenge, nipost.NIPost, 2, coinbase) + atx := newAtx(challenge, nipost.NIPost, 2, coinbase) atxHdlr.mockFetch.EXPECT().GetPoetProof(gomock.Any(), atx.GetPoetProofRef()) atxHdlr.mockFetch.EXPECT().GetAtxs(gomock.Any(), []types.ATXID{atx.PrevATXID}, gomock.Any()).Return(nil) @@ -1724,9 +1880,15 @@ func TestHandler_FetchReferences(t *testing.T) { atxHdlr := newTestHandler(t, goldenATXID) coinbase := types.Address{2, 4, 5} - challenge := newChallenge(1, types.EmptyATXID, types.EmptyATXID, types.LayerID(22).GetEpoch(), nil) + challenge := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: types.EmptyATXID, + PublishEpoch: postGenesisEpoch, + PositioningATX: types.EmptyATXID, + CommitmentATX: nil, + } nipost := newNIPostWithChallenge(t, types.HexToHash32("55555"), []byte("66666")) - atx := newAtx(t, sig, challenge, nipost.NIPost, 2, coinbase) + atx := newAtx(challenge, nipost.NIPost, 2, coinbase) atxHdlr.mockFetch.EXPECT().GetPoetProof(gomock.Any(), atx.GetPoetProofRef()).Return(errors.New("pooh")) require.Error(t, atxHdlr.FetchReferences(context.Background(), atx)) @@ -1737,9 +1899,15 @@ func TestHandler_FetchReferences(t *testing.T) { atxHdlr := newTestHandler(t, goldenATXID) coinbase := types.Address{2, 4, 5} - challenge := newChallenge(1, prevATX, prevATX, types.LayerID(22).GetEpoch(), nil) + challenge := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: prevATX, + PublishEpoch: postGenesisEpoch, + PositioningATX: prevATX, + CommitmentATX: nil, + } nipost := newNIPostWithChallenge(t, types.HexToHash32("55555"), []byte("66666")) - atx := newAtx(t, sig, challenge, nipost.NIPost, 2, coinbase) + atx := newAtx(challenge, nipost.NIPost, 2, coinbase) atxHdlr.mockFetch.EXPECT().GetPoetProof(gomock.Any(), atx.GetPoetProofRef()) atxHdlr.mockFetch.EXPECT().GetAtxs(gomock.Any(), []types.ATXID{atx.PrevATXID}, gomock.Any()).Return(errors.New("oh")) @@ -1916,7 +2084,13 @@ func TestHandler_MarksAtxValid(t *testing.T) { require.NoError(t, err) goldenATXID := types.ATXID{2, 3, 4} - challenge := newChallenge(0, types.EmptyATXID, goldenATXID, 2, &goldenATXID) + challenge := types.NIPostChallenge{ + Sequence: 0, + PrevATXID: types.EmptyATXID, + PublishEpoch: 2, + PositioningATX: goldenATXID, + CommitmentATX: &goldenATXID, + } nipost := newNIPostWithChallenge(t, challenge.Hash(), []byte("poet")).NIPost t.Run("post verified fully", func(t *testing.T) { @@ -1929,7 +2103,7 @@ func TestHandler_MarksAtxValid(t *testing.T) { handler.mValidator.EXPECT().PositioningAtx(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()) handler.mValidator.EXPECT().IsVerifyingFullPost().Return(true) - atx := newAtx(t, sig, challenge, nipost, 2, types.Address{1, 2, 3, 4}) + atx := newAtx(challenge, nipost, 2, types.Address{1, 2, 3, 4}) atx.SetValidity(types.Unknown) require.NoError(t, SignAndFinalizeAtx(sig, atx)) _, err := handler.SyntacticallyValidateDeps(context.Background(), atx) @@ -1946,7 +2120,7 @@ func TestHandler_MarksAtxValid(t *testing.T) { handler.mValidator.EXPECT().PositioningAtx(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()) handler.mValidator.EXPECT().IsVerifyingFullPost().Return(false) - atx := newAtx(t, sig, challenge, nipost, 2, types.Address{1, 2, 3, 4}) + atx := newAtx(challenge, nipost, 2, types.Address{1, 2, 3, 4}) atx.SetValidity(types.Unknown) require.NoError(t, SignAndFinalizeAtx(sig, atx)) _, err := handler.SyntacticallyValidateDeps(context.Background(), atx) diff --git a/activation/interface.go b/activation/interface.go index e546044221..e196f9eeeb 100644 --- a/activation/interface.go +++ b/activation/interface.go @@ -10,6 +10,7 @@ import ( "github.com/spacemeshos/post/verifying" "github.com/spacemeshos/go-spacemesh/common/types" + "github.com/spacemeshos/go-spacemesh/signing" "github.com/spacemeshos/go-spacemesh/sql/localsql/nipost" ) @@ -79,9 +80,9 @@ type layerClock interface { } type nipostBuilder interface { - BuildNIPost(ctx context.Context, challenge *types.NIPostChallenge) (*nipost.NIPostState, error) - Proof(ctx context.Context, challenge []byte) (*types.Post, *types.PostInfo, error) - ResetState() error + BuildNIPost(ctx context.Context, sig *signing.EdSigner, challenge *types.NIPostChallenge) (*nipost.NIPostState, error) + Proof(ctx context.Context, nodeID types.NodeID, challenge []byte) (*types.Post, *types.PostInfo, error) + ResetState(types.NodeID) error } type syncer interface { @@ -107,7 +108,7 @@ type SmeshingProvider interface { Smeshing() bool StartSmeshing(types.Address) error StopSmeshing(bool) error - SmesherID() types.NodeID + SmesherIDs() []types.NodeID Coinbase() types.Address SetCoinbase(coinbase types.Address) } diff --git a/activation/mocks.go b/activation/mocks.go index d8cabb115a..d59cb7a83e 100644 --- a/activation/mocks.go +++ b/activation/mocks.go @@ -14,6 +14,7 @@ import ( time "time" types "github.com/spacemeshos/go-spacemesh/common/types" + signing "github.com/spacemeshos/go-spacemesh/signing" nipost "github.com/spacemeshos/go-spacemesh/sql/localsql/nipost" shared "github.com/spacemeshos/post/shared" verifying "github.com/spacemeshos/post/verifying" @@ -822,18 +823,18 @@ func (m *MocknipostBuilder) EXPECT() *MocknipostBuilderMockRecorder { } // BuildNIPost mocks base method. -func (m *MocknipostBuilder) BuildNIPost(ctx context.Context, challenge *types.NIPostChallenge) (*nipost.NIPostState, error) { +func (m *MocknipostBuilder) BuildNIPost(ctx context.Context, sig *signing.EdSigner, challenge *types.NIPostChallenge) (*nipost.NIPostState, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "BuildNIPost", ctx, challenge) + ret := m.ctrl.Call(m, "BuildNIPost", ctx, sig, challenge) ret0, _ := ret[0].(*nipost.NIPostState) ret1, _ := ret[1].(error) return ret0, ret1 } // BuildNIPost indicates an expected call of BuildNIPost. -func (mr *MocknipostBuilderMockRecorder) BuildNIPost(ctx, challenge any) *nipostBuilderBuildNIPostCall { +func (mr *MocknipostBuilderMockRecorder) BuildNIPost(ctx, sig, challenge any) *nipostBuilderBuildNIPostCall { mr.mock.ctrl.T.Helper() - call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BuildNIPost", reflect.TypeOf((*MocknipostBuilder)(nil).BuildNIPost), ctx, challenge) + call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BuildNIPost", reflect.TypeOf((*MocknipostBuilder)(nil).BuildNIPost), ctx, sig, challenge) return &nipostBuilderBuildNIPostCall{Call: call} } @@ -849,21 +850,21 @@ func (c *nipostBuilderBuildNIPostCall) Return(arg0 *nipost.NIPostState, arg1 err } // Do rewrite *gomock.Call.Do -func (c *nipostBuilderBuildNIPostCall) Do(f func(context.Context, *types.NIPostChallenge) (*nipost.NIPostState, error)) *nipostBuilderBuildNIPostCall { +func (c *nipostBuilderBuildNIPostCall) Do(f func(context.Context, *signing.EdSigner, *types.NIPostChallenge) (*nipost.NIPostState, error)) *nipostBuilderBuildNIPostCall { c.Call = c.Call.Do(f) return c } // DoAndReturn rewrite *gomock.Call.DoAndReturn -func (c *nipostBuilderBuildNIPostCall) DoAndReturn(f func(context.Context, *types.NIPostChallenge) (*nipost.NIPostState, error)) *nipostBuilderBuildNIPostCall { +func (c *nipostBuilderBuildNIPostCall) DoAndReturn(f func(context.Context, *signing.EdSigner, *types.NIPostChallenge) (*nipost.NIPostState, error)) *nipostBuilderBuildNIPostCall { c.Call = c.Call.DoAndReturn(f) return c } // Proof mocks base method. -func (m *MocknipostBuilder) Proof(ctx context.Context, challenge []byte) (*types.Post, *types.PostInfo, error) { +func (m *MocknipostBuilder) Proof(ctx context.Context, nodeID types.NodeID, challenge []byte) (*types.Post, *types.PostInfo, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Proof", ctx, challenge) + ret := m.ctrl.Call(m, "Proof", ctx, nodeID, challenge) ret0, _ := ret[0].(*types.Post) ret1, _ := ret[1].(*types.PostInfo) ret2, _ := ret[2].(error) @@ -871,9 +872,9 @@ func (m *MocknipostBuilder) Proof(ctx context.Context, challenge []byte) (*types } // Proof indicates an expected call of Proof. -func (mr *MocknipostBuilderMockRecorder) Proof(ctx, challenge any) *nipostBuilderProofCall { +func (mr *MocknipostBuilderMockRecorder) Proof(ctx, nodeID, challenge any) *nipostBuilderProofCall { mr.mock.ctrl.T.Helper() - call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Proof", reflect.TypeOf((*MocknipostBuilder)(nil).Proof), ctx, challenge) + call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Proof", reflect.TypeOf((*MocknipostBuilder)(nil).Proof), ctx, nodeID, challenge) return &nipostBuilderProofCall{Call: call} } @@ -889,29 +890,29 @@ func (c *nipostBuilderProofCall) Return(arg0 *types.Post, arg1 *types.PostInfo, } // Do rewrite *gomock.Call.Do -func (c *nipostBuilderProofCall) Do(f func(context.Context, []byte) (*types.Post, *types.PostInfo, error)) *nipostBuilderProofCall { +func (c *nipostBuilderProofCall) Do(f func(context.Context, types.NodeID, []byte) (*types.Post, *types.PostInfo, error)) *nipostBuilderProofCall { c.Call = c.Call.Do(f) return c } // DoAndReturn rewrite *gomock.Call.DoAndReturn -func (c *nipostBuilderProofCall) DoAndReturn(f func(context.Context, []byte) (*types.Post, *types.PostInfo, error)) *nipostBuilderProofCall { +func (c *nipostBuilderProofCall) DoAndReturn(f func(context.Context, types.NodeID, []byte) (*types.Post, *types.PostInfo, error)) *nipostBuilderProofCall { c.Call = c.Call.DoAndReturn(f) return c } // ResetState mocks base method. -func (m *MocknipostBuilder) ResetState() error { +func (m *MocknipostBuilder) ResetState(arg0 types.NodeID) error { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "ResetState") + ret := m.ctrl.Call(m, "ResetState", arg0) ret0, _ := ret[0].(error) return ret0 } // ResetState indicates an expected call of ResetState. -func (mr *MocknipostBuilderMockRecorder) ResetState() *nipostBuilderResetStateCall { +func (mr *MocknipostBuilderMockRecorder) ResetState(arg0 any) *nipostBuilderResetStateCall { mr.mock.ctrl.T.Helper() - call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ResetState", reflect.TypeOf((*MocknipostBuilder)(nil).ResetState)) + call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ResetState", reflect.TypeOf((*MocknipostBuilder)(nil).ResetState), arg0) return &nipostBuilderResetStateCall{Call: call} } @@ -927,13 +928,13 @@ func (c *nipostBuilderResetStateCall) Return(arg0 error) *nipostBuilderResetStat } // Do rewrite *gomock.Call.Do -func (c *nipostBuilderResetStateCall) Do(f func() error) *nipostBuilderResetStateCall { +func (c *nipostBuilderResetStateCall) Do(f func(types.NodeID) error) *nipostBuilderResetStateCall { c.Call = c.Call.Do(f) return c } // DoAndReturn rewrite *gomock.Call.DoAndReturn -func (c *nipostBuilderResetStateCall) DoAndReturn(f func() error) *nipostBuilderResetStateCall { +func (c *nipostBuilderResetStateCall) DoAndReturn(f func(types.NodeID) error) *nipostBuilderResetStateCall { c.Call = c.Call.DoAndReturn(f) return c } @@ -1333,40 +1334,40 @@ func (c *SmeshingProviderSetCoinbaseCall) DoAndReturn(f func(types.Address)) *Sm return c } -// SmesherID mocks base method. -func (m *MockSmeshingProvider) SmesherID() types.NodeID { +// SmesherIDs mocks base method. +func (m *MockSmeshingProvider) SmesherIDs() []types.NodeID { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "SmesherID") - ret0, _ := ret[0].(types.NodeID) + ret := m.ctrl.Call(m, "SmesherIDs") + ret0, _ := ret[0].([]types.NodeID) return ret0 } -// SmesherID indicates an expected call of SmesherID. -func (mr *MockSmeshingProviderMockRecorder) SmesherID() *SmeshingProviderSmesherIDCall { +// SmesherIDs indicates an expected call of SmesherIDs. +func (mr *MockSmeshingProviderMockRecorder) SmesherIDs() *SmeshingProviderSmesherIDsCall { mr.mock.ctrl.T.Helper() - call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SmesherID", reflect.TypeOf((*MockSmeshingProvider)(nil).SmesherID)) - return &SmeshingProviderSmesherIDCall{Call: call} + call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SmesherIDs", reflect.TypeOf((*MockSmeshingProvider)(nil).SmesherIDs)) + return &SmeshingProviderSmesherIDsCall{Call: call} } -// SmeshingProviderSmesherIDCall wrap *gomock.Call -type SmeshingProviderSmesherIDCall struct { +// SmeshingProviderSmesherIDsCall wrap *gomock.Call +type SmeshingProviderSmesherIDsCall struct { *gomock.Call } // Return rewrite *gomock.Call.Return -func (c *SmeshingProviderSmesherIDCall) Return(arg0 types.NodeID) *SmeshingProviderSmesherIDCall { +func (c *SmeshingProviderSmesherIDsCall) Return(arg0 []types.NodeID) *SmeshingProviderSmesherIDsCall { c.Call = c.Call.Return(arg0) return c } // Do rewrite *gomock.Call.Do -func (c *SmeshingProviderSmesherIDCall) Do(f func() types.NodeID) *SmeshingProviderSmesherIDCall { +func (c *SmeshingProviderSmesherIDsCall) Do(f func() []types.NodeID) *SmeshingProviderSmesherIDsCall { c.Call = c.Call.Do(f) return c } // DoAndReturn rewrite *gomock.Call.DoAndReturn -func (c *SmeshingProviderSmesherIDCall) DoAndReturn(f func() types.NodeID) *SmeshingProviderSmesherIDCall { +func (c *SmeshingProviderSmesherIDsCall) DoAndReturn(f func() []types.NodeID) *SmeshingProviderSmesherIDsCall { c.Call = c.Call.DoAndReturn(f) return c } diff --git a/activation/nipost.go b/activation/nipost.go index fa81130213..bfb8976747 100644 --- a/activation/nipost.go +++ b/activation/nipost.go @@ -45,7 +45,6 @@ const ( // NIPostBuilder holds the required state and dependencies to create Non-Interactive Proofs of Space-Time (NIPost). type NIPostBuilder struct { localDB *localsql.Database - signer *signing.EdSigner poetProvers map[string]poetClient poetDB poetDbAPI @@ -74,7 +73,6 @@ func NewNIPostBuilder( postService postService, poetServers []types.PoetServer, lg *zap.Logger, - signer *signing.EdSigner, poetCfg PoetConfig, layerClock layerClock, opts ...NIPostBuilderOption, @@ -90,7 +88,6 @@ func NewNIPostBuilder( b := &NIPostBuilder{ localDB: db, - signer: signer, poetProvers: poetClients, poetDB: poetDB, @@ -106,26 +103,30 @@ func NewNIPostBuilder( return b, nil } -func (nb *NIPostBuilder) ResetState() error { - if err := nipost.ClearPoetRegistrations(nb.localDB, nb.signer.NodeID()); err != nil { +func (nb *NIPostBuilder) ResetState(nodeId types.NodeID) error { + if err := nipost.ClearPoetRegistrations(nb.localDB, nodeId); err != nil { return fmt.Errorf("clear poet registrations: %w", err) } - if err := nipost.RemoveNIPost(nb.localDB, nb.signer.NodeID()); err != nil { + if err := nipost.RemoveNIPost(nb.localDB, nodeId); err != nil { return fmt.Errorf("remove nipost: %w", err) } return nil } -func (nb *NIPostBuilder) Proof(ctx context.Context, challenge []byte) (*types.Post, *types.PostInfo, error) { +func (nb *NIPostBuilder) Proof( + ctx context.Context, + nodeID types.NodeID, + challenge []byte, +) (*types.Post, *types.PostInfo, error) { started := false retries := 0 for { - client, err := nb.postService.Client(nb.signer.NodeID()) + client, err := nb.postService.Client(nodeID) if err != nil { select { case <-ctx.Done(): if started { - events.EmitPostFailure() + events.EmitPostFailure(nodeID) } return nil, nil, ctx.Err() case <-time.After(2 * time.Second): // Wait a few seconds and try connecting again @@ -133,7 +134,7 @@ func (nb *NIPostBuilder) Proof(ctx context.Context, challenge []byte) (*types.Po if retries%10 == 0 { // every 20 seconds inform user about lost connection (for remote post service) // TODO(mafa): emit event warning user about lost connection nb.log.Warn("post service not connected - waiting for reconnection", - zap.Stringer("service id", nb.signer.NodeID()), + zap.Stringer("service id", nodeID), zap.Error(err), ) } @@ -141,7 +142,7 @@ func (nb *NIPostBuilder) Proof(ctx context.Context, challenge []byte) (*types.Po } } if !started { - events.EmitPostStart(challenge) + events.EmitPostStart(nodeID, challenge) started = true } @@ -151,10 +152,10 @@ func (nb *NIPostBuilder) Proof(ctx context.Context, challenge []byte) (*types.Po case errors.Is(err, ErrPostClientClosed): continue case err != nil: - events.EmitPostFailure() + events.EmitPostFailure(nodeID) return nil, nil, err default: // err == nil - events.EmitPostComplete(challenge) + events.EmitPostComplete(nodeID, challenge) return post, postInfo, err } } @@ -165,6 +166,7 @@ func (nb *NIPostBuilder) Proof(ctx context.Context, challenge []byte) (*types.Po // publish a proof - a process that takes about an epoch. func (nb *NIPostBuilder) BuildNIPost( ctx context.Context, + signer *signing.EdSigner, challenge *types.NIPostChallenge, ) (*nipost.NIPostState, error) { logger := nb.log.With(log.ZContext(ctx)) @@ -204,7 +206,7 @@ func (nb *NIPostBuilder) BuildNIPost( ) // Phase 0: Submit challenge to PoET services. - count, err := nipost.PoetRegistrationCount(nb.localDB, nb.signer.NodeID()) // TODO(mafa): when are these cleared? + count, err := nipost.PoetRegistrationCount(nb.localDB, signer.NodeID()) if err != nil { return nil, fmt.Errorf("failed to get poet registration count: %w", err) } @@ -222,10 +224,10 @@ func (nb *NIPostBuilder) BuildNIPost( submitCtx, cancel := context.WithDeadline(ctx, poetRoundStart) defer cancel() - if err := nb.submitPoetChallenges(submitCtx, poetProofDeadline, challenge.Hash().Bytes()); err != nil { + if err := nb.submitPoetChallenges(submitCtx, signer, poetProofDeadline, challenge.Hash().Bytes()); err != nil { return nil, fmt.Errorf("submitting to poets: %w", err) } - count, err := nipost.PoetRegistrationCount(nb.localDB, nb.signer.NodeID()) + count, err := nipost.PoetRegistrationCount(nb.localDB, signer.NodeID()) if err != nil { return nil, fmt.Errorf("failed to get poet registration count: %w", err) } @@ -235,7 +237,7 @@ func (nb *NIPostBuilder) BuildNIPost( } // Phase 1: query PoET services for proofs - poetProofRef, membership, err := nipost.PoetProofRef(nb.localDB, nb.signer.NodeID()) + poetProofRef, membership, err := nipost.PoetProofRef(nb.localDB, signer.NodeID()) if err != nil && !errors.Is(err, sql.ErrNotFound) { nb.log.Warn("cannot get poet proof ref", zap.Error(err)) } @@ -254,20 +256,20 @@ func (nb *NIPostBuilder) BuildNIPost( } events.EmitPoetWaitProof(challenge.PublishEpoch, challenge.TargetEpoch(), poetRoundEnd) - poetProofRef, membership, err = nb.getBestProof(ctx, challenge.Hash(), challenge.PublishEpoch) + poetProofRef, membership, err = nb.getBestProof(ctx, signer.NodeID(), challenge.Hash(), challenge.PublishEpoch) if err != nil { return nil, &PoetSvcUnstableError{msg: "getBestProof failed", source: err} } if poetProofRef == types.EmptyPoetProofRef { return nil, &PoetSvcUnstableError{source: ErrPoetProofNotReceived} } - if err := nipost.UpdatePoetProofRef(nb.localDB, nb.signer.NodeID(), poetProofRef, membership); err != nil { + if err := nipost.UpdatePoetProofRef(nb.localDB, signer.NodeID(), poetProofRef, membership); err != nil { nb.log.Warn("cannot persist poet proof ref", zap.Error(err)) } } // Phase 2: Post execution. - nipostState, err := nipost.NIPost(nb.localDB, nb.signer.NodeID()) + nipostState, err := nipost.NIPost(nb.localDB, signer.NodeID()) if err != nil && !errors.Is(err, sql.ErrNotFound) { nb.log.Warn("cannot get nipost", zap.Error(err)) } @@ -289,7 +291,7 @@ func (nb *NIPostBuilder) BuildNIPost( nb.log.Info("starting post execution", zap.Binary("challenge", poetProofRef[:])) startTime := time.Now() - proof, postInfo, err := nb.Proof(postCtx, poetProofRef[:]) + proof, postInfo, err := nb.Proof(postCtx, signer.NodeID(), poetProofRef[:]) if err != nil { return nil, fmt.Errorf("failed to generate Post: %w", err) } @@ -310,7 +312,7 @@ func (nb *NIPostBuilder) BuildNIPost( NumUnits: postInfo.NumUnits, VRFNonce: *postInfo.Nonce, } - if err := nipost.AddNIPost(nb.localDB, nb.signer.NodeID(), nipostState); err != nil { + if err := nipost.AddNIPost(nb.localDB, signer.NodeID(), nipostState); err != nil { nb.log.Warn("cannot persist nipost state", zap.Error(err)) } } @@ -331,6 +333,7 @@ func withConditionalTimeout(ctx context.Context, timeout time.Duration) (context // Submit the challenge to a single PoET. func (nb *NIPostBuilder) submitPoetChallenge( ctx context.Context, + nodeID types.NodeID, deadline time.Time, client poetClient, prefix, challenge []byte, @@ -352,7 +355,7 @@ func (nb *NIPostBuilder) submitPoetChallenge( ctx, powParams.Challenge, challenge, - nb.signer.NodeID().Bytes(), + nodeID.Bytes(), powParams.Difficulty, ) metrics.PoetPowDuration.Set(float64(time.Since(startTime).Nanoseconds())) @@ -364,7 +367,7 @@ func (nb *NIPostBuilder) submitPoetChallenge( submitCtx, cancel := withConditionalTimeout(ctx, nb.poetCfg.RequestTimeout) defer cancel() - round, err := client.Submit(submitCtx, deadline, prefix, challenge, signature, nb.signer.NodeID(), PoetPoW{ + round, err := client.Submit(submitCtx, deadline, prefix, challenge, signature, nodeID, PoetPoW{ Nonce: nonce, Params: *powParams, }) @@ -373,7 +376,7 @@ func (nb *NIPostBuilder) submitPoetChallenge( } logger.Info("challenge submitted to poet proving service", zap.String("round", round.ID)) - return nipost.AddPoetRegistration(nb.localDB, nb.signer.NodeID(), nipost.PoETRegistration{ + return nipost.AddPoetRegistration(nb.localDB, nodeID, nipost.PoETRegistration{ ChallengeHash: types.Hash32(challenge), Address: client.Address(), RoundID: round.ID, @@ -384,18 +387,19 @@ func (nb *NIPostBuilder) submitPoetChallenge( // Submit the challenge to all registered PoETs. func (nb *NIPostBuilder) submitPoetChallenges( ctx context.Context, + signer *signing.EdSigner, deadline time.Time, challenge []byte, ) error { - signature := nb.signer.Sign(signing.POET, challenge) - prefix := bytes.Join([][]byte{nb.signer.Prefix(), {byte(signing.POET)}}, nil) + signature := signer.Sign(signing.POET, challenge) + prefix := bytes.Join([][]byte{signer.Prefix(), {byte(signing.POET)}}, nil) g, ctx := errgroup.WithContext(ctx) errChan := make(chan error, len(nb.poetProvers)) for _, poetClient := range nb.poetProvers { client := poetClient g.Go(func() error { - errChan <- nb.submitPoetChallenge(ctx, deadline, client, prefix, challenge, signature) + errChan <- nb.submitPoetChallenge(ctx, signer.NodeID(), deadline, client, prefix, challenge, signature) return nil }) } @@ -442,6 +446,7 @@ func membersContainChallenge(members []types.Member, challenge types.Hash32) (ui func (nb *NIPostBuilder) getBestProof( ctx context.Context, + nodeID types.NodeID, challenge types.Hash32, publishEpoch types.EpochID, ) (types.PoetProofRef, *types.MerkleProof, error) { @@ -449,7 +454,7 @@ func (nb *NIPostBuilder) getBestProof( poet *types.PoetProofMessage membership *types.MerkleProof } - registrations, err := nipost.PoetRegistrations(nb.localDB, nb.signer.NodeID()) + registrations, err := nipost.PoetRegistrations(nb.localDB, nodeID) if err != nil { return types.PoetProofRef{}, nil, fmt.Errorf("getting poet registrations: %w", err) } diff --git a/activation/nipost_test.go b/activation/nipost_test.go index 7fced292f8..ed35b7c7a6 100644 --- a/activation/nipost_test.go +++ b/activation/nipost_test.go @@ -14,6 +14,7 @@ import ( "go.uber.org/zap/zapcore" "go.uber.org/zap/zaptest" "go.uber.org/zap/zaptest/observer" + "golang.org/x/sync/errgroup" "github.com/spacemeshos/go-spacemesh/common/types" "github.com/spacemeshos/go-spacemesh/events" @@ -49,7 +50,6 @@ func defaultLayerClockMock(ctrl *gomock.Controller) *MocklayerClock { type testNIPostBuilder struct { *NIPostBuilder - sig *signing.EdSigner observedLogs *observer.ObservedLogs eventSub <-chan events.UserEvent @@ -62,9 +62,6 @@ type testNIPostBuilder struct { } func newTestNIPostBuilder(tb testing.TB) *testNIPostBuilder { - sig, err := signing.NewEdSigner() - require.NoError(tb, err) - observer, observedLogs := observer.New(zapcore.WarnLevel) logger := zap.New(observer) @@ -79,7 +76,6 @@ func newTestNIPostBuilder(tb testing.TB) *testNIPostBuilder { ctrl := gomock.NewController(tb) tnb := &testNIPostBuilder{ - sig: sig, observedLogs: observedLogs, eventSub: sub.Out(), @@ -97,7 +93,6 @@ func newTestNIPostBuilder(tb testing.TB) *testNIPostBuilder { tnb.mPostService, []types.PoetServer{}, tnb.mLogger, - tnb.sig, PoetConfig{}, tnb.mClock, ) @@ -109,12 +104,14 @@ func newTestNIPostBuilder(tb testing.TB) *testNIPostBuilder { func Test_NIPost_PostClientHandling(t *testing.T) { t.Run("connect then complete", func(t *testing.T) { // post client connects, starts post, then completes successfully - tnb := newTestNIPostBuilder(t) + sig, err := signing.NewEdSigner() + require.NoError(t, err) - tnb.mPostService.EXPECT().Client(tnb.sig.NodeID()).Return(tnb.mPostClient, nil) + tnb := newTestNIPostBuilder(t) + tnb.mPostService.EXPECT().Client(sig.NodeID()).Return(tnb.mPostClient, nil) tnb.mPostClient.EXPECT().Proof(gomock.Any(), gomock.Any()).Return(&types.Post{}, &types.PostInfo{}, nil) - nipost, nipostInfo, err := tnb.Proof(context.Background(), shared.ZeroChallenge) + nipost, nipostInfo, err := tnb.Proof(context.Background(), sig.NodeID(), shared.ZeroChallenge) require.NoError(t, err) require.NotNil(t, nipost) require.NotNil(t, nipostInfo) @@ -142,13 +139,15 @@ func Test_NIPost_PostClientHandling(t *testing.T) { t.Run("connect then error", func(t *testing.T) { // post client connects, starts post, then fails with an error that is not a disconnect + sig, err := signing.NewEdSigner() + require.NoError(t, err) + tnb := newTestNIPostBuilder(t) + tnb.mPostService.EXPECT().Client(sig.NodeID()).Return(tnb.mPostClient, nil) expectedErr := errors.New("some error") - - tnb.mPostService.EXPECT().Client(tnb.sig.NodeID()).Return(tnb.mPostClient, nil) tnb.mPostClient.EXPECT().Proof(gomock.Any(), gomock.Any()).Return(nil, nil, expectedErr) - nipost, nipostInfo, err := tnb.Proof(context.Background(), shared.ZeroChallenge) + nipost, nipostInfo, err := tnb.Proof(context.Background(), sig.NodeID(), shared.ZeroChallenge) require.ErrorIs(t, err, expectedErr) require.Nil(t, nipost) require.Nil(t, nipostInfo) @@ -175,14 +174,16 @@ func Test_NIPost_PostClientHandling(t *testing.T) { t.Run("connect, disconnect, reconnect then complete", func(t *testing.T) { // post client connects, starts post, disconnects in between but completes successfully - tnb := newTestNIPostBuilder(t) + sig, err := signing.NewEdSigner() + require.NoError(t, err) - tnb.mPostService.EXPECT().Client(tnb.sig.NodeID()).Return(tnb.mPostClient, nil) + tnb := newTestNIPostBuilder(t) + tnb.mPostService.EXPECT().Client(sig.NodeID()).Return(tnb.mPostClient, nil) tnb.mPostClient.EXPECT().Proof(gomock.Any(), gomock.Any()).Return(nil, nil, ErrPostClientClosed) - tnb.mPostService.EXPECT().Client(tnb.sig.NodeID()).Return(tnb.mPostClient, nil) + tnb.mPostService.EXPECT().Client(sig.NodeID()).Return(tnb.mPostClient, nil) tnb.mPostClient.EXPECT().Proof(gomock.Any(), gomock.Any()).Return(&types.Post{}, &types.PostInfo{}, nil) - nipost, nipostInfo, err := tnb.Proof(context.Background(), shared.ZeroChallenge) + nipost, nipostInfo, err := tnb.Proof(context.Background(), sig.NodeID(), shared.ZeroChallenge) require.NoError(t, err) require.NotNil(t, nipost) require.NotNil(t, nipostInfo) @@ -208,18 +209,70 @@ func Test_NIPost_PostClientHandling(t *testing.T) { } }) + t.Run("connect, disconnect, then cancel before reconnect", func(t *testing.T) { + // post client connects, starts post, disconnects in between and proofing is canceled before reconnection + sig, err := signing.NewEdSigner() + require.NoError(t, err) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + tnb := newTestNIPostBuilder(t) + + var eg errgroup.Group + eg.Go(func() error { + select { + case e := <-tnb.eventSub: + event := e.Event.GetPostStart() + require.NotNil(t, event, "wrong event type") + require.EqualValues(t, shared.ZeroChallenge, event.Challenge) + case <-time.After(5 * time.Second): + require.Fail(t, "timeout waiting for event") + } + + cancel() + + select { + case e := <-tnb.eventSub: + event := e.Event.GetPostComplete() + require.NotNil(t, event, "wrong event type") + require.Equal(t, true, e.Event.Failure) + require.Equal(t, "Node failed PoST execution.", e.Event.Help) + case <-time.After(5 * time.Second): + require.Fail(t, "timeout waiting for event") + } + return nil + }) + + tnb.mPostService.EXPECT().Client(sig.NodeID()).Return(tnb.mPostClient, nil) + tnb.mPostClient.EXPECT().Proof(gomock.Any(), gomock.Any()).Return(nil, nil, ErrPostClientClosed) + tnb.mPostService.EXPECT().Client(sig.NodeID()).DoAndReturn( + func(types.NodeID) (PostClient, error) { + <-ctx.Done() + return nil, ErrPostClientNotConnected + }) + + nipost, nipostInfo, err := tnb.Proof(ctx, sig.NodeID(), shared.ZeroChallenge) + require.ErrorIs(t, err, context.Canceled) + require.Nil(t, nipost) + require.Nil(t, nipostInfo) + + require.Nil(t, eg.Wait()) + }) + t.Run("connect, disconnect, reconnect then error", func(t *testing.T) { // post client connects, starts post, disconnects in between and then fails to complete - tnb := newTestNIPostBuilder(t) + sig, err := signing.NewEdSigner() + require.NoError(t, err) - tnb.mPostService.EXPECT().Client(tnb.sig.NodeID()).Return(tnb.mPostClient, nil) + tnb := newTestNIPostBuilder(t) + tnb.mPostService.EXPECT().Client(sig.NodeID()).Return(tnb.mPostClient, nil) tnb.mPostClient.EXPECT().Proof(gomock.Any(), gomock.Any()).Return(nil, nil, ErrPostClientClosed).Times(1) - tnb.mPostService.EXPECT().Client(tnb.sig.NodeID()).Return(tnb.mPostClient, nil) + tnb.mPostService.EXPECT().Client(sig.NodeID()).Return(tnb.mPostClient, nil) expectedErr := errors.New("some error") tnb.mPostClient.EXPECT().Proof(gomock.Any(), gomock.Any()).Return(nil, nil, expectedErr) - nipost, nipostInfo, err := tnb.Proof(context.Background(), shared.ZeroChallenge) + nipost, nipostInfo, err := tnb.Proof(context.Background(), sig.NodeID(), shared.ZeroChallenge) require.ErrorIs(t, err, expectedErr) require.Nil(t, nipost) require.Nil(t, nipostInfo) @@ -245,17 +298,19 @@ func Test_NIPost_PostClientHandling(t *testing.T) { }) t.Run("repeated connection failure", func(t *testing.T) { - tnb := newTestNIPostBuilder(t) + sig, err := signing.NewEdSigner() + require.NoError(t, err) + tnb := newTestNIPostBuilder(t) ctx, cancel := context.WithCancel(context.Background()) - tnb.mPostService.EXPECT().Client(tnb.sig.NodeID()).Return(nil, ErrPostClientNotConnected).Times(10) - tnb.mPostService.EXPECT().Client(tnb.sig.NodeID()).DoAndReturn( + tnb.mPostService.EXPECT().Client(sig.NodeID()).Return(nil, ErrPostClientNotConnected).Times(10) + tnb.mPostService.EXPECT().Client(sig.NodeID()).DoAndReturn( func(types.NodeID) (PostClient, error) { cancel() return nil, ErrPostClientNotConnected }) - nipost, nipostInfo, err := tnb.Proof(ctx, shared.ZeroChallenge) + nipost, nipostInfo, err := tnb.Proof(ctx, sig.NodeID(), shared.ZeroChallenge) require.ErrorIs(t, err, context.Canceled) require.Nil(t, nipost) require.Nil(t, nipostInfo) @@ -263,7 +318,7 @@ func Test_NIPost_PostClientHandling(t *testing.T) { require.Equal(t, 1, tnb.observedLogs.Len(), "expected 1 log message") require.Equal(t, zapcore.WarnLevel, tnb.observedLogs.All()[0].Level) require.Equal(t, "post service not connected - waiting for reconnection", tnb.observedLogs.All()[0].Message) - require.Equal(t, tnb.sig.NodeID().String(), tnb.observedLogs.All()[0].ContextMap()["service id"]) + require.Equal(t, sig.NodeID().String(), tnb.observedLogs.All()[0].ContextMap()["service id"]) }) } @@ -286,7 +341,6 @@ func Test_NIPostBuilder_ResetState(t *testing.T) { postService, []types.PoetServer{}, zaptest.NewLogger(t), - sig, PoetConfig{}, mclock, ) @@ -312,7 +366,7 @@ func Test_NIPostBuilder_ResetState(t *testing.T) { VRFNonce: types.VRFPostIndex(1024), }) - err = nb.ResetState() + err = nb.ResetState(sig.NodeID()) require.NoError(t, err) _, err = nipost.NIPost(db, sig.NodeID()) @@ -354,14 +408,13 @@ func Test_NIPostBuilder_WithMocks(t *testing.T) { postService, []types.PoetServer{}, zaptest.NewLogger(t), - sig, PoetConfig{}, mclock, withPoetClients([]poetClient{poetProvider}), ) require.NoError(t, err) - nipost, err := nb.BuildNIPost(context.Background(), &challenge) + nipost, err := nb.BuildNIPost(context.Background(), sig, &challenge) require.NoError(t, err) require.NotNil(t, nipost) } @@ -397,14 +450,13 @@ func TestPostSetup(t *testing.T) { postService, []types.PoetServer{}, zaptest.NewLogger(t), - sig, PoetConfig{}, mclock, withPoetClients([]poetClient{poetProvider}), ) require.NoError(t, err) - nipost, err := nb.BuildNIPost(context.Background(), &challenge) + nipost, err := nb.BuildNIPost(context.Background(), sig, &challenge) require.NoError(t, err) require.NotNil(t, nipost) } @@ -462,28 +514,26 @@ func TestNIPostBuilder_BuildNIPost(t *testing.T) { postService, []types.PoetServer{}, zaptest.NewLogger(t), - sig, PoetConfig{}, mclock, withPoetClients([]poetClient{poetProver}), ) require.NoError(t, err) - nipost, err := nb.BuildNIPost(context.Background(), &challenge) + nipost, err := nb.BuildNIPost(context.Background(), sig, &challenge) require.NoError(t, err) require.NotNil(t, nipost) poetDb = NewMockpoetDbAPI(ctrl) // fail post exec - require.NoError(t, nb.ResetState()) + require.NoError(t, nb.ResetState(sig.NodeID())) nb, err = NewNIPostBuilder( db, poetDb, postService, []types.PoetServer{}, zaptest.NewLogger(t), - sig, PoetConfig{}, mclock, withPoetClients([]poetClient{poetProver}), @@ -493,7 +543,7 @@ func TestNIPostBuilder_BuildNIPost(t *testing.T) { postClient.EXPECT().Proof(gomock.Any(), gomock.Any()).Return(nil, nil, fmt.Errorf("error")) // check that proof ref is not called again - nipost, err = nb.BuildNIPost(context.Background(), &challenge) + nipost, err = nb.BuildNIPost(context.Background(), sig, &challenge) require.Nil(t, nipost) require.Error(t, err) @@ -504,7 +554,6 @@ func TestNIPostBuilder_BuildNIPost(t *testing.T) { postService, []types.PoetServer{}, zaptest.NewLogger(t), - sig, PoetConfig{}, mclock, withPoetClients([]poetClient{poetProver}), @@ -519,11 +568,25 @@ func TestNIPostBuilder_BuildNIPost(t *testing.T) { ) // check that proof ref is not called again - nipost, err = nb.BuildNIPost(context.Background(), &challenge) + nipost, err = nb.BuildNIPost(context.Background(), sig, &challenge) require.NoError(t, err) require.NotNil(t, nipost) } +func Test_NIPostBuilder_InvalidPoetAddresses(t *testing.T) { + nb, err := NewNIPostBuilder( + nil, + nil, + nil, + []types.PoetServer{{Address: ":invalid"}}, + zaptest.NewLogger(t).Named("nipostBuilder"), + PoetConfig{}, + nil, + ) + require.ErrorContains(t, err, "cannot create poet client") + require.Nil(t, nb) +} + func TestNIPostBuilder_ManyPoETs_SubmittingChallenge_DeadlineReached(t *testing.T) { t.Parallel() // Arrange @@ -592,7 +655,6 @@ func TestNIPostBuilder_ManyPoETs_SubmittingChallenge_DeadlineReached(t *testing. postService, []types.PoetServer{}, zaptest.NewLogger(t), - sig, poetCfg, mclock, withPoetClients(poets), @@ -600,7 +662,7 @@ func TestNIPostBuilder_ManyPoETs_SubmittingChallenge_DeadlineReached(t *testing. require.NoError(t, err) // Act - nipost, err := nb.BuildNIPost(context.Background(), &challenge) + nipost, err := nb.BuildNIPost(context.Background(), sig, &challenge) require.NoError(t, err) // Verify @@ -661,7 +723,6 @@ func TestNIPostBuilder_ManyPoETs_AllFinished(t *testing.T) { postService, []types.PoetServer{}, zaptest.NewLogger(t), - sig, PoetConfig{}, mclock, withPoetClients(poets), @@ -669,7 +730,7 @@ func TestNIPostBuilder_ManyPoETs_AllFinished(t *testing.T) { require.NoError(t, err) // Act - nipost, err := nb.BuildNIPost(context.Background(), &challenge) + nipost, err := nb.BuildNIPost(context.Background(), sig, &challenge) require.NoError(t, err) // Verify @@ -708,14 +769,13 @@ func TestNIPSTBuilder_PoetUnstable(t *testing.T) { postService, []types.PoetServer{}, zaptest.NewLogger(t), - sig, poetCfg, mclock, withPoetClients([]poetClient{poetProver}), ) require.NoError(t, err) - nipst, err := nb.BuildNIPost(context.Background(), &challenge) + nipst, err := nb.BuildNIPost(context.Background(), sig, &challenge) require.ErrorIs(t, err, ErrPoetServiceUnstable) require.Nil(t, nipst) }) @@ -748,13 +808,12 @@ func TestNIPSTBuilder_PoetUnstable(t *testing.T) { postService, []types.PoetServer{}, zaptest.NewLogger(t), - sig, poetCfg, mclock, withPoetClients([]poetClient{poetProver}), ) require.NoError(t, err) - nipst, err := nb.BuildNIPost(context.Background(), &challenge) + nipst, err := nb.BuildNIPost(context.Background(), sig, &challenge) require.ErrorIs(t, err, ErrPoetServiceUnstable) require.Nil(t, nipst) }) @@ -773,13 +832,12 @@ func TestNIPSTBuilder_PoetUnstable(t *testing.T) { postService, []types.PoetServer{}, zaptest.NewLogger(t), - sig, poetCfg, mclock, withPoetClients([]poetClient{poetProver}), ) require.NoError(t, err) - nipst, err := nb.BuildNIPost(context.Background(), &challenge) + nipst, err := nb.BuildNIPost(context.Background(), sig, &challenge) require.ErrorIs(t, err, ErrPoetProofNotReceived) require.Nil(t, nipst) }) @@ -801,13 +859,12 @@ func TestNIPSTBuilder_PoetUnstable(t *testing.T) { postService, []types.PoetServer{}, zaptest.NewLogger(t), - sig, poetCfg, mclock, withPoetClients([]poetClient{poetProver}), ) require.NoError(t, err) - nipst, err := nb.BuildNIPost(context.Background(), &challenge) + nipst, err := nb.BuildNIPost(context.Background(), sig, &challenge) require.ErrorIs(t, err, ErrPoetProofNotReceived) require.Nil(t, nipst) }) @@ -844,7 +901,6 @@ func TestNIPoSTBuilder_StaleChallenge(t *testing.T) { postService, []types.PoetServer{}, zaptest.NewLogger(t), - sig, PoetConfig{}, mclock, withPoetClients([]poetClient{poetProver}), @@ -852,7 +908,7 @@ func TestNIPoSTBuilder_StaleChallenge(t *testing.T) { require.NoError(t, err) challenge := types.NIPostChallenge{PublishEpoch: currLayer.GetEpoch()} - nipost, err := nb.BuildNIPost(context.Background(), &challenge) + nipost, err := nb.BuildNIPost(context.Background(), sig, &challenge) require.ErrorIs(t, err, ErrATXChallengeExpired) require.ErrorContains(t, err, "poet round has already started") require.Nil(t, nipost) @@ -876,7 +932,6 @@ func TestNIPoSTBuilder_StaleChallenge(t *testing.T) { postService, []types.PoetServer{}, zaptest.NewLogger(t), - sig, PoetConfig{}, mclock, withPoetClients([]poetClient{poetProver}), @@ -896,7 +951,7 @@ func TestNIPoSTBuilder_StaleChallenge(t *testing.T) { }) require.NoError(t, err) - nipost, err := nb.BuildNIPost(context.Background(), challenge) + nipost, err := nb.BuildNIPost(context.Background(), sig, challenge) require.ErrorIs(t, err, ErrATXChallengeExpired) require.ErrorContains(t, err, "poet proof for pub epoch") require.Nil(t, nipost) @@ -920,7 +975,6 @@ func TestNIPoSTBuilder_StaleChallenge(t *testing.T) { postService, []types.PoetServer{}, zaptest.NewLogger(t), - sig, PoetConfig{}, mclock, withPoetClients([]poetClient{poetProver}), @@ -944,7 +998,7 @@ func TestNIPoSTBuilder_StaleChallenge(t *testing.T) { err = nipost.UpdatePoetProofRef(db, sig.NodeID(), [32]byte{1, 2, 3}, &types.MerkleProof{}) require.NoError(t, err) - nipost, err := nb.BuildNIPost(context.Background(), challenge) + nipost, err := nb.BuildNIPost(context.Background(), sig, challenge) require.ErrorIs(t, err, ErrATXChallengeExpired) require.ErrorContains(t, err, "deadline to publish ATX for pub epoch") require.Nil(t, nipost) @@ -1013,7 +1067,6 @@ func TestNIPoSTBuilder_Continues_After_Interrupted(t *testing.T) { postService, []types.PoetServer{}, zaptest.NewLogger(t), - sig, poetCfg, mclock, withPoetClients([]poetClient{poet}), @@ -1021,7 +1074,7 @@ func TestNIPoSTBuilder_Continues_After_Interrupted(t *testing.T) { require.NoError(t, err) // Act - nipost, err := nb.BuildNIPost(buildCtx, &challenge) + nipost, err := nb.BuildNIPost(buildCtx, sig, &challenge) require.ErrorIs(t, err, context.Canceled) require.Nil(t, nipost) @@ -1030,7 +1083,7 @@ func TestNIPoSTBuilder_Continues_After_Interrupted(t *testing.T) { Submit(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()). Return(&types.PoetRound{}, nil) - nipost, err = nb.BuildNIPost(context.Background(), &challenge) + nipost, err = nb.BuildNIPost(context.Background(), sig, &challenge) require.NoError(t, err) // Verify @@ -1164,14 +1217,13 @@ func TestNIPostBuilder_Mainnet_Poet_Workaround(t *testing.T) { postService, []types.PoetServer{}, zaptest.NewLogger(t), - sig, poetCfg, mclock, withPoetClients(poets), ) require.NoError(t, err) - nipost, err := nb.BuildNIPost(context.Background(), &challenge) + nipost, err := nb.BuildNIPost(context.Background(), sig, &challenge) require.NoError(t, err) require.NotNil(t, nipost) }) diff --git a/activation/poet.go b/activation/poet.go index 31e2ea9ebd..0c83704083 100644 --- a/activation/poet.go +++ b/activation/poet.go @@ -221,10 +221,6 @@ func (c *HTTPPoetClient) Proof(ctx context.Context, roundID string) (*types.Poet RoundID: roundID, Statement: types.BytesToHash(statement), } - if c.poetServiceID == nil { - c.poetServiceID = proof.PoetServiceID - } - return &proof, members, nil } diff --git a/activation/post_test.go b/activation/post_test.go index d13a61e492..065528228c 100644 --- a/activation/post_test.go +++ b/activation/post_test.go @@ -18,6 +18,7 @@ import ( "github.com/spacemeshos/go-spacemesh/log/logtest" "github.com/spacemeshos/go-spacemesh/signing" "github.com/spacemeshos/go-spacemesh/sql" + "github.com/spacemeshos/go-spacemesh/sql/atxs" ) func TestPostSetupManager(t *testing.T) { @@ -274,14 +275,20 @@ func TestPostSetupManager_Stop_WhileInProgress(t *testing.T) { func TestPostSetupManager_findCommitmentAtx_UsesLatestAtx(t *testing.T) { mgr := newTestPostManager(t) - ch := newChallenge(0, types.EmptyATXID, mgr.goldenATXID, 2, &mgr.goldenATXID) - nipostData := newNIPostWithChallenge(t, types.HexToHash32("55555"), []byte("66666")) - latestAtx := newAtx(t, mgr.signer, ch, nipostData.NIPost, 2, types.Address{}) - addAtx(t, mgr.db, mgr.signer, latestAtx) + challenge := types.NIPostChallenge{ + PublishEpoch: 1, + } + atx := types.NewActivationTx(challenge, types.Address{}, nil, 2, nil) + require.NoError(t, SignAndFinalizeAtx(mgr.signer, atx)) + atx.SetEffectiveNumUnits(atx.NumUnits) + atx.SetReceived(time.Now()) + vAtx, err := atx.Verify(0, 1) + require.NoError(t, err) + require.NoError(t, atxs.Add(mgr.db, vAtx)) - atx, err := mgr.findCommitmentAtx(context.Background()) + commitmentAtx, err := mgr.findCommitmentAtx(context.Background()) require.NoError(t, err) - require.Equal(t, latestAtx.ID(), atx) + require.Equal(t, vAtx.ID(), commitmentAtx) } func TestPostSetupManager_findCommitmentAtx_DefaultsToGoldenAtx(t *testing.T) { @@ -316,7 +323,12 @@ func TestPostSetupManager_getCommitmentAtx_getsCommitmentAtxFromInitialAtx(t *te commitmentAtx := types.RandomATXID() atx := types.NewActivationTx(types.NIPostChallenge{}, types.Address{}, nil, 1, nil) atx.CommitmentATX = &commitmentAtx - addAtx(t, mgr.cdb, mgr.signer, atx) + require.NoError(t, SignAndFinalizeAtx(mgr.signer, atx)) + atx.SetEffectiveNumUnits(atx.NumUnits) + atx.SetReceived(time.Now()) + vAtx, err := atx.Verify(0, 1) + require.NoError(t, err) + require.NoError(t, atxs.Add(mgr.cdb, vAtx)) atxid, err := mgr.commitmentAtx(context.Background(), mgr.opts.DataDir) require.NoError(t, err) diff --git a/activation/post_verifier_scaling_test.go b/activation/post_verifier_scaling_test.go index 5915f66f98..15a37cc05a 100644 --- a/activation/post_verifier_scaling_test.go +++ b/activation/post_verifier_scaling_test.go @@ -12,6 +12,7 @@ import ( "go.uber.org/zap/zaptest" "golang.org/x/sync/errgroup" + "github.com/spacemeshos/go-spacemesh/common/types" "github.com/spacemeshos/go-spacemesh/events" ) @@ -36,9 +37,9 @@ func TestAutoScaling(t *testing.T) { return nil }) - events.EmitPostStart(nil) - events.EmitPostComplete(nil) - events.EmitPostFailure() + events.EmitPostStart(types.EmptyNodeID, nil) + events.EmitPostComplete(types.EmptyNodeID, nil) + events.EmitPostFailure(types.EmptyNodeID) require.Eventually(t, done.Load, time.Second, 10*time.Millisecond) close(stop) diff --git a/activation/validation_test.go b/activation/validation_test.go index 3674685b38..3277d88753 100644 --- a/activation/validation_test.go +++ b/activation/validation_test.go @@ -100,7 +100,13 @@ func Test_Validation_InitialNIPostChallenge(t *testing.T) { posAtxId := types.ATXID{1, 2, 3} commitmentAtxId := types.ATXID{5, 6, 7} - challenge := newChallenge(0, types.EmptyATXID, posAtxId, 2, &commitmentAtxId) + challenge := types.NIPostChallenge{ + Sequence: 0, + PrevATXID: types.EmptyATXID, + PublishEpoch: 2, + PositioningATX: posAtxId, + CommitmentATX: &commitmentAtxId, + } challenge.InitialPost = &types.Post{} atxProvider := NewMockatxProvider(ctrl) @@ -119,7 +125,13 @@ func Test_Validation_InitialNIPostChallenge(t *testing.T) { posAtxId := types.ATXID{1, 2, 3} - challenge := newChallenge(0, types.EmptyATXID, posAtxId, types.LayerID(2).GetEpoch(), &goldenATXID) + challenge := types.NIPostChallenge{ + Sequence: 0, + PrevATXID: types.EmptyATXID, + PublishEpoch: 0, + PositioningATX: posAtxId, + CommitmentATX: &goldenATXID, + } challenge.InitialPost = &types.Post{} atxProvider := NewMockatxProvider(ctrl) @@ -134,7 +146,13 @@ func Test_Validation_InitialNIPostChallenge(t *testing.T) { posAtxId := types.ATXID{1, 2, 3} commitmentAtxId := types.ATXID{5, 6, 7} - challenge := newChallenge(0, types.EmptyATXID, posAtxId, 1, &commitmentAtxId) + challenge := types.NIPostChallenge{ + Sequence: 0, + PrevATXID: types.EmptyATXID, + PublishEpoch: 1, + PositioningATX: posAtxId, + CommitmentATX: &commitmentAtxId, + } challenge.InitialPost = &types.Post{} atxProvider := NewMockatxProvider(ctrl) @@ -170,7 +188,13 @@ func Test_Validation_NIPostChallenge(t *testing.T) { prevAtxId := types.ATXID{3, 2, 1} posAtxId := types.ATXID{1, 2, 3} - challenge := newChallenge(10, prevAtxId, posAtxId, 2, nil) + challenge := types.NIPostChallenge{ + Sequence: 10, + PrevATXID: prevAtxId, + PublishEpoch: 2, + PositioningATX: posAtxId, + CommitmentATX: nil, + } atxProvider := NewMockatxProvider(ctrl) atxProvider.EXPECT().GetAtxHeader(prevAtxId).Return(&types.ActivationTxHeader{ @@ -193,7 +217,13 @@ func Test_Validation_NIPostChallenge(t *testing.T) { prevAtxId := types.ATXID{3, 2, 1} posAtxId := types.ATXID{1, 2, 3} - challenge := newChallenge(10, prevAtxId, posAtxId, types.LayerID(1012).GetEpoch(), nil) + challenge := types.NIPostChallenge{ + Sequence: 10, + PrevATXID: prevAtxId, + PublishEpoch: 101, + PositioningATX: posAtxId, + CommitmentATX: nil, + } atxProvider := NewMockatxProvider(ctrl) atxProvider.EXPECT().GetAtxHeader(prevAtxId).Return(nil, errors.New("not found")) @@ -212,7 +242,13 @@ func Test_Validation_NIPostChallenge(t *testing.T) { prevAtxId := types.ATXID{3, 2, 1} posAtxId := types.ATXID{1, 2, 3} - challenge := newChallenge(10, prevAtxId, posAtxId, types.LayerID(1012).GetEpoch(), nil) + challenge := types.NIPostChallenge{ + Sequence: 10, + PrevATXID: prevAtxId, + PublishEpoch: 101, + PositioningATX: posAtxId, + CommitmentATX: nil, + } atxProvider := NewMockatxProvider(ctrl) atxProvider.EXPECT().GetAtxHeader(prevAtxId).Return(&types.ActivationTxHeader{ @@ -235,7 +271,13 @@ func Test_Validation_NIPostChallenge(t *testing.T) { prevAtxId := types.ATXID{3, 2, 1} posAtxId := types.ATXID{1, 2, 3} - challenge := newChallenge(10, prevAtxId, posAtxId, 2, nil) + challenge := types.NIPostChallenge{ + Sequence: 10, + PrevATXID: prevAtxId, + PublishEpoch: 2, + PositioningATX: posAtxId, + CommitmentATX: nil, + } atxProvider := NewMockatxProvider(ctrl) atxProvider.EXPECT().GetAtxHeader(prevAtxId).Return(&types.ActivationTxHeader{ @@ -258,7 +300,13 @@ func Test_Validation_NIPostChallenge(t *testing.T) { prevAtxId := types.ATXID{3, 2, 1} posAtxId := types.ATXID{1, 2, 3} - challenge := newChallenge(10, prevAtxId, posAtxId, 2, nil) + challenge := types.NIPostChallenge{ + Sequence: 10, + PrevATXID: prevAtxId, + PublishEpoch: 2, + PositioningATX: posAtxId, + CommitmentATX: nil, + } atxProvider := NewMockatxProvider(ctrl) atxProvider.EXPECT().GetAtxHeader(prevAtxId).Return(&types.ActivationTxHeader{ @@ -530,9 +578,15 @@ func TestVerifyChainDeps(t *testing.T) { signer, err := signing.NewEdSigner() require.NoError(t, err) - ch := newChallenge(1, types.EmptyATXID, goldenATXID, postGenesisEpoch, &goldenATXID) + ch := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: types.EmptyATXID, + PublishEpoch: postGenesisEpoch, + PositioningATX: goldenATXID, + CommitmentATX: &goldenATXID, + } nipostData := newNIPostWithChallenge(t, types.HexToHash32(""), []byte("00")) - invalidAtx := newAtx(t, signer, ch, nipostData.NIPost, 2, types.Address{}) + invalidAtx := newAtx(ch, nipostData.NIPost, 2, types.Address{}) require.NoError(t, SignAndFinalizeAtx(signer, invalidAtx)) vInvalidAtx, err := invalidAtx.Verify(0, 1) require.NoError(t, err) @@ -540,9 +594,15 @@ func TestVerifyChainDeps(t *testing.T) { require.NoError(t, atxs.Add(db, vInvalidAtx)) t.Run("invalid prev ATX", func(t *testing.T) { - ch = newChallenge(1, vInvalidAtx.ID(), goldenATXID, postGenesisEpoch, nil) + ch := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: vInvalidAtx.ID(), + PublishEpoch: postGenesisEpoch, + PositioningATX: goldenATXID, + CommitmentATX: nil, + } nipostData = newNIPostWithChallenge(t, types.HexToHash32(""), []byte("01")) - atx := newAtx(t, signer, ch, nipostData.NIPost, 2, types.Address{}) + atx := newAtx(ch, nipostData.NIPost, 2, types.Address{}) require.NoError(t, SignAndFinalizeAtx(signer, atx)) vAtx, err := atx.Verify(0, 1) require.NoError(t, err) @@ -558,9 +618,15 @@ func TestVerifyChainDeps(t *testing.T) { }) t.Run("invalid pos ATX", func(t *testing.T) { - ch = newChallenge(1, types.EmptyATXID, vInvalidAtx.ID(), postGenesisEpoch, nil) + ch := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: types.EmptyATXID, + PublishEpoch: postGenesisEpoch, + PositioningATX: vInvalidAtx.ID(), + CommitmentATX: nil, + } nipostData = newNIPostWithChallenge(t, types.HexToHash32(""), []byte("02")) - atx := newAtx(t, signer, ch, nipostData.NIPost, 2, types.Address{}) + atx := newAtx(ch, nipostData.NIPost, 2, types.Address{}) require.NoError(t, SignAndFinalizeAtx(signer, atx)) vAtx, err := atx.Verify(0, 1) require.NoError(t, err) @@ -577,9 +643,15 @@ func TestVerifyChainDeps(t *testing.T) { t.Run("invalid commitment ATX", func(t *testing.T) { commitmentAtxID := vInvalidAtx.ID() - ch = newChallenge(1, types.EmptyATXID, goldenATXID, postGenesisEpoch, &commitmentAtxID) + ch := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: types.EmptyATXID, + PublishEpoch: postGenesisEpoch, + PositioningATX: goldenATXID, + CommitmentATX: &commitmentAtxID, + } nipostData = newNIPostWithChallenge(t, types.HexToHash32(""), []byte("03")) - atx := newAtx(t, signer, ch, nipostData.NIPost, 2, types.Address{}) + atx := newAtx(ch, nipostData.NIPost, 2, types.Address{}) require.NoError(t, SignAndFinalizeAtx(signer, atx)) vAtx, err := atx.Verify(0, 1) require.NoError(t, err) @@ -594,9 +666,15 @@ func TestVerifyChainDeps(t *testing.T) { }) t.Run("with trusted node ID", func(t *testing.T) { - ch = newChallenge(1, types.EmptyATXID, vInvalidAtx.ID(), postGenesisEpoch, nil) + ch := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: types.EmptyATXID, + PublishEpoch: postGenesisEpoch, + PositioningATX: vInvalidAtx.ID(), + CommitmentATX: nil, + } nipostData = newNIPostWithChallenge(t, types.HexToHash32(""), []byte("04")) - atx := newAtx(t, signer, ch, nipostData.NIPost, 2, types.Address{}) + atx := newAtx(ch, nipostData.NIPost, 2, types.Address{}) require.NoError(t, SignAndFinalizeAtx(signer, atx)) vAtx, err := atx.Verify(0, 1) require.NoError(t, err) @@ -610,9 +688,15 @@ func TestVerifyChainDeps(t *testing.T) { }) t.Run("assume valid if older than X", func(t *testing.T) { - ch = newChallenge(1, types.EmptyATXID, vInvalidAtx.ID(), postGenesisEpoch, nil) + ch := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: types.EmptyATXID, + PublishEpoch: postGenesisEpoch, + PositioningATX: vInvalidAtx.ID(), + CommitmentATX: nil, + } nipostData = newNIPostWithChallenge(t, types.HexToHash32(""), []byte("05")) - atx := newAtx(t, signer, ch, nipostData.NIPost, 2, types.Address{}) + atx := newAtx(ch, nipostData.NIPost, 2, types.Address{}) require.NoError(t, SignAndFinalizeAtx(signer, atx)) vAtx, err := atx.Verify(0, 1) require.NoError(t, err) @@ -626,9 +710,15 @@ func TestVerifyChainDeps(t *testing.T) { }) t.Run("invalid top-level", func(t *testing.T) { - ch = newChallenge(1, types.EmptyATXID, vInvalidAtx.ID(), postGenesisEpoch, nil) + ch := types.NIPostChallenge{ + Sequence: 1, + PrevATXID: types.EmptyATXID, + PublishEpoch: postGenesisEpoch, + PositioningATX: vInvalidAtx.ID(), + CommitmentATX: nil, + } nipostData = newNIPostWithChallenge(t, types.HexToHash32(""), []byte("06")) - atx := newAtx(t, signer, ch, nipostData.NIPost, 2, types.Address{}) + atx := newAtx(ch, nipostData.NIPost, 2, types.Address{}) require.NoError(t, SignAndFinalizeAtx(signer, atx)) vAtx, err := atx.Verify(0, 1) require.NoError(t, err) diff --git a/api/grpcserver/grpcserver_test.go b/api/grpcserver/grpcserver_test.go index 07837d9f7b..cc181bcb6f 100644 --- a/api/grpcserver/grpcserver_test.go +++ b/api/grpcserver/grpcserver_test.go @@ -630,14 +630,15 @@ func TestSmesherService(t *testing.T) { require.Equal(t, int32(code.Code_OK), res.Status.Code) }) - t.Run("SmesherID", func(t *testing.T) { + t.Run("SmesherIDs", func(t *testing.T) { t.Parallel() c, ctx := setupSmesherService(t) nodeId := types.RandomNodeID() - c.smeshingProvider.EXPECT().SmesherID().Return(nodeId) - res, err := c.SmesherID(ctx, &emptypb.Empty{}) + c.smeshingProvider.EXPECT().SmesherIDs().Return([]types.NodeID{nodeId}) + res, err := c.SmesherIDs(ctx, &emptypb.Empty{}) require.NoError(t, err) - require.Equal(t, nodeId.Bytes(), res.PublicKey) + require.Equal(t, 1, len(res.PublicKeys)) + require.Equal(t, nodeId.Bytes(), res.PublicKeys[0]) }) t.Run("SetCoinbaseMissingArgs", func(t *testing.T) { diff --git a/api/grpcserver/smesher_service.go b/api/grpcserver/smesher_service.go index df438bb9ba..e6df58f953 100644 --- a/api/grpcserver/smesher_service.go +++ b/api/grpcserver/smesher_service.go @@ -140,7 +140,16 @@ func (s SmesherService) StopSmeshing( // SmesherID returns the smesher ID of this node. func (s SmesherService) SmesherID(context.Context, *emptypb.Empty) (*pb.SmesherIDResponse, error) { - return &pb.SmesherIDResponse{PublicKey: s.smeshingProvider.SmesherID().Bytes()}, nil + return nil, status.Errorf(codes.Unimplemented, "this endpoint has been deprecated, use `SmesherIDs` instead") +} + +func (s SmesherService) SmesherIDs(context.Context, *emptypb.Empty) (*pb.SmesherIDsResponse, error) { + ids := s.smeshingProvider.SmesherIDs() + res := &pb.SmesherIDsResponse{} + for _, id := range ids { + res.PublicKeys = append(res.PublicKeys, id.Bytes()) + } + return res, nil } // Coinbase returns the current coinbase setting of this node. diff --git a/api/grpcserver/smesher_service_test.go b/api/grpcserver/smesher_service_test.go index 221d65bdcc..9c8b4a8ff9 100644 --- a/api/grpcserver/smesher_service_test.go +++ b/api/grpcserver/smesher_service_test.go @@ -10,6 +10,8 @@ import ( "github.com/spacemeshos/post/config" "github.com/stretchr/testify/require" "go.uber.org/mock/gomock" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" "google.golang.org/protobuf/types/known/emptypb" "github.com/spacemeshos/go-spacemesh/activation" @@ -228,3 +230,22 @@ func TestSmesherService_PostSetupStatus(t *testing.T) { require.False(t, resp.Status.Opts.Throttle) }) } + +func TestSmesherService_SmesherID(t *testing.T) { + ctrl := gomock.NewController(t) + smeshingProvider := activation.NewMockSmeshingProvider(ctrl) + postSupervisor := grpcserver.NewMockpostSupervisor(ctrl) + svc := grpcserver.NewSmesherService( + smeshingProvider, + postSupervisor, + time.Second, + activation.DefaultPostSetupOpts(), + ) + + resp, err := svc.SmesherID(context.Background(), &emptypb.Empty{}) + require.Error(t, err) + require.Nil(t, resp) + statusErr, ok := status.FromError(err) + require.True(t, ok) + require.Equal(t, codes.Unimplemented, statusErr.Code()) +} diff --git a/blocks/certifier_test.go b/blocks/certifier_test.go index 947607cd16..7a0d23f7e8 100644 --- a/blocks/certifier_test.go +++ b/blocks/certifier_test.go @@ -664,7 +664,7 @@ func Test_OldLayersPruned(t *testing.T) { func Test_CertifyIfEligible(t *testing.T) { numSigners := 3 - tc := newTestCertifier(t, 3) + tc := newTestCertifier(t, numSigners) b := generateBlock(t, tc.db) tc.mb.EXPECT().GetBeacon(b.LayerIndex.GetEpoch()).Return(types.RandomBeacon(), nil) diff --git a/common/types/beacon.go b/common/types/beacon.go index 43f9bb4320..66444a39f2 100644 --- a/common/types/beacon.go +++ b/common/types/beacon.go @@ -1,6 +1,8 @@ package types import ( + "fmt" + "github.com/spacemeshos/go-spacemesh/common/util" "github.com/spacemeshos/go-spacemesh/log" ) @@ -25,11 +27,11 @@ func (b Beacon) Hex() string { return util.Encode(b[:]) } // doing full logging into a file. func (b Beacon) String() string { return b.Hex() } -// ShortString returns the first 5 characters of the Beacon, usually for logging purposes. +// ShortString returns the first 10 characters of the Beacon, usually for logging purposes. func (b Beacon) ShortString() string { str := b.Hex() l := len(str) - return Shorten(str[min(2, l):], 10) + return fmt.Sprintf("%.10s", str[min(2, l):]) } // Bytes gets the byte representation of the underlying hash. diff --git a/common/types/hashes.go b/common/types/hashes.go index d4554157f1..3d1aae2484 100644 --- a/common/types/hashes.go +++ b/common/types/hashes.go @@ -47,10 +47,9 @@ func (h Hash20) String() string { return h.Hex() } -// ShortString returns a the first 5 characters of the hash, for logging purposes. +// ShortString returns a the first 5 hex-encoded bytes of the hash, for logging purposes. func (h Hash20) ShortString() string { - l := len(h.Hex()) - return Shorten(h.Hex()[min(2, l):], 10) + return hex.EncodeToString(h[:5]) } // Format implements fmt.Formatter, forcing the byte slice to be formatted as is, @@ -178,16 +177,9 @@ func (h Hash32) String() string { return h.ShortString() } -// ShortString returns the first 5 characters of the hash, for logging purposes. +// ShortString returns the first 5 hex-encoded bytes of the hash, for logging purposes. func (h Hash32) ShortString() string { - l := len(h.Hex()) - return Shorten(h.Hex()[min(2, l):], 10) -} - -// Shorten shortens a string to a specified length. -func Shorten(s string, maxlen int) string { - l := len(s) - return s[:min(maxlen, l)] + return hex.EncodeToString(h[:5]) } // Format implements fmt.Formatter, forcing the byte slice to be formatted as is, diff --git a/common/types/nodeid.go b/common/types/nodeid.go index ef9bc6f401..13d16aaf2a 100644 --- a/common/types/nodeid.go +++ b/common/types/nodeid.go @@ -34,9 +34,9 @@ func (id NodeID) Bytes() []byte { return id[:] } -// ShortString returns a the first 5 characters of the ID, for logging purposes. +// ShortString returns a the first 3 hex-encoded bytes of the ID, for logging purposes. func (id NodeID) ShortString() string { - return Shorten(id.String(), 5) + return hex.EncodeToString(id[:3]) } // Field returns a log field. Implements the LoggableField interface. diff --git a/config/config.go b/config/config.go index 61ef6a4f0c..e1e622ba87 100644 --- a/config/config.go +++ b/config/config.go @@ -128,7 +128,7 @@ type BaseConfig struct { RegossipAtxInterval time.Duration `mapstructure:"regossip-atx-interval"` // ATXGradeDelay is used to grade ATXs for selection in tortoise active set. - // See grading fuction in miner/proposals_builder.go + // See grading function in miner/proposals_builder.go ATXGradeDelay time.Duration `mapstructure:"atx-grade-delay"` // PostValidDelay is the time after which a PoST is considered valid diff --git a/events/events.go b/events/events.go index cf99f720c0..bc5c42fab6 100644 --- a/events/events.go +++ b/events/events.go @@ -127,30 +127,44 @@ func EmitPostServiceStopped() { ) } -func EmitPostStart(challenge []byte) { +func EmitPostStart(nodeID types.NodeID, challenge []byte) { const help = "Node started PoST execution using the challenge from PoET." emitUserEvent( help, false, - &pb.Event_PostStart{PostStart: &pb.EventPostStart{Challenge: challenge}}, + &pb.Event_PostStart{ + PostStart: &pb.EventPostStart{ + Challenge: challenge, + Smesher: nodeID.Bytes(), + }, + }, ) } -func EmitPostComplete(challenge []byte) { +func EmitPostComplete(nodeID types.NodeID, challenge []byte) { const help = "Node finished PoST execution using PoET challenge." emitUserEvent( help, false, - &pb.Event_PostComplete{PostComplete: &pb.EventPostComplete{Challenge: challenge}}, + &pb.Event_PostComplete{ + PostComplete: &pb.EventPostComplete{ + Challenge: challenge, + Smesher: nodeID.Bytes(), + }, + }, ) } -func EmitPostFailure() { +func EmitPostFailure(nodeID types.NodeID) { const help = "Node failed PoST execution." emitUserEvent( help, true, - &pb.Event_PostComplete{PostComplete: &pb.EventPostComplete{}}, + &pb.Event_PostComplete{ + PostComplete: &pb.EventPostComplete{ + Smesher: nodeID.Bytes(), + }, + }, ) } diff --git a/go.mod b/go.mod index dbc541112b..3fe7787c5d 100644 --- a/go.mod +++ b/go.mod @@ -5,7 +5,7 @@ go 1.21.6 require ( cloud.google.com/go/storage v1.37.0 github.com/ALTree/bigfloat v0.2.0 - github.com/chaos-mesh/chaos-mesh/api v0.0.0-20240104130649-f55576898805 + github.com/chaos-mesh/chaos-mesh/api v0.0.0-20240130121751-5a08714d8838 github.com/cosmos/btcutil v1.0.5 github.com/go-llsqlite/crawshaw v0.5.0 github.com/gofrs/flock v0.8.1 @@ -36,7 +36,7 @@ require ( github.com/quic-go/quic-go v0.41.0 github.com/santhosh-tekuri/jsonschema/v5 v5.3.1 github.com/seehuhn/mt19937 v1.0.0 - github.com/spacemeshos/api/release/go v1.27.0 + github.com/spacemeshos/api/release/go v1.28.0 github.com/spacemeshos/economics v0.1.2 github.com/spacemeshos/fixed v0.1.1 github.com/spacemeshos/go-scale v1.1.12 @@ -52,7 +52,7 @@ require ( github.com/zeebo/blake3 v0.2.3 go.uber.org/mock v0.4.0 go.uber.org/zap v1.26.0 - golang.org/x/exp v0.0.0-20240112132812-db7319d0e0e3 + golang.org/x/exp v0.0.0-20240119083558-1b970713d09a golang.org/x/sync v0.6.0 golang.org/x/time v0.5.0 google.golang.org/genproto/googleapis/rpc v0.0.0-20240125205218-1f4bbc51befe diff --git a/go.sum b/go.sum index ea6ec48fab..4591f0cc65 100644 --- a/go.sum +++ b/go.sum @@ -88,8 +88,8 @@ github.com/c0mm4nd/go-ripemd v0.0.0-20200326052756-bd1759ad7d10/go.mod h1:mYPR+a github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= -github.com/chaos-mesh/chaos-mesh/api v0.0.0-20240104130649-f55576898805 h1:QjGJQDB+rcJkUEPjn4z3Ur6DgJHBbGobwdOX2i4WQ14= -github.com/chaos-mesh/chaos-mesh/api v0.0.0-20240104130649-f55576898805/go.mod h1:x11iCbZV6hzzSQWMq610B6Wl5Lg1dhwqcVfeiWQQnQQ= +github.com/chaos-mesh/chaos-mesh/api v0.0.0-20240130121751-5a08714d8838 h1:f5Jyh/fAlM3W/zYbecj1BrmRjcN+DDYBiHS0VQJsUlo= +github.com/chaos-mesh/chaos-mesh/api v0.0.0-20240130121751-5a08714d8838/go.mod h1:x11iCbZV6hzzSQWMq610B6Wl5Lg1dhwqcVfeiWQQnQQ= github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= @@ -622,8 +622,8 @@ github.com/sourcegraph/annotate v0.0.0-20160123013949-f4cad6c6324d/go.mod h1:Udh github.com/sourcegraph/conc v0.3.0 h1:OQTbbt6P72L20UqAkXXuLOj79LfEanQ+YQFNpLA9ySo= github.com/sourcegraph/conc v0.3.0/go.mod h1:Sdozi7LEKbFPqYX2/J+iBAM6HpqSLTASQIKqDmF7Mt0= github.com/sourcegraph/syntaxhighlight v0.0.0-20170531221838-bd320f5d308e/go.mod h1:HuIsMU8RRBOtsCgI77wP899iHVBQpCmg4ErYMZB+2IA= -github.com/spacemeshos/api/release/go v1.27.0 h1:LPWgr70NC1aNd4MLqv2TD/bq4qqyH2O8RyCrsR+NLwI= -github.com/spacemeshos/api/release/go v1.27.0/go.mod h1:fK9RBD8eTVXHrqkkal2bwQB4N8M9sOhPs4rnVmWqEc0= +github.com/spacemeshos/api/release/go v1.28.0 h1:HmrNf0kV7U9o2rwqrmZ0tfPgQA8JQWY8EYGoiCB4bnI= +github.com/spacemeshos/api/release/go v1.28.0/go.mod h1:fK9RBD8eTVXHrqkkal2bwQB4N8M9sOhPs4rnVmWqEc0= github.com/spacemeshos/economics v0.1.2 h1:kw8cE5SMa/7svHOGorCd2w8ef1y8iP0p47/2VDOK8Ns= github.com/spacemeshos/economics v0.1.2/go.mod h1:ngeWn5E/jy9dJP1MHyuk3ehF8NBMTYhchqVDhAHUUNk= github.com/spacemeshos/fixed v0.1.1 h1:N1y4SUpq1EV+IdJrWJwUCt1oBFzeru/VKVcBsvPc2Fk= @@ -763,8 +763,8 @@ golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u0 golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= -golang.org/x/exp v0.0.0-20240112132812-db7319d0e0e3 h1:hNQpMuAJe5CtcUqCXaWga3FHu+kQvCqcsoVaQgSV60o= -golang.org/x/exp v0.0.0-20240112132812-db7319d0e0e3/go.mod h1:idGWGoKP1toJGkd5/ig9ZLuPcZBC3ewk7SzmH0uou08= +golang.org/x/exp v0.0.0-20240119083558-1b970713d09a h1:Q8/wZp0KX97QFTc2ywcOE0YRjZPVIx+MXInMzdvQqcA= +golang.org/x/exp v0.0.0-20240119083558-1b970713d09a/go.mod h1:idGWGoKP1toJGkd5/ig9ZLuPcZBC3ewk7SzmH0uou08= golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= golang.org/x/lint v0.0.0-20180702182130-06c8688daad7/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= diff --git a/node/node.go b/node/node.go index 05bf0a1f1b..2067715745 100644 --- a/node/node.go +++ b/node/node.go @@ -950,7 +950,6 @@ func (app *App) initServices(ctx context.Context) error { grpcPostService.(*grpcserver.PostService), app.Config.PoetServers, app.addLogger(NipostBuilderLogger, lg).Zap(), - app.edSgn, app.Config.POET, app.clock, ) @@ -964,7 +963,6 @@ func (app *App) initServices(ctx context.Context) error { } atxBuilder := activation.NewBuilder( builderConfig, - app.edSgn, app.cachedDB, app.localDB, app.host, @@ -979,6 +977,7 @@ func (app *App) initServices(ctx context.Context) error { activation.WithValidator(app.validator), activation.WithPostValidityDelay(app.Config.PostValidDelay), ) + atxBuilder.Register(app.edSgn) malfeasanceHandler := malfeasance.NewHandler( app.cachedDB, @@ -1835,7 +1834,7 @@ func (app *App) startSynchronous(ctx context.Context) (err error) { timesync.WithLayerDuration(app.Config.LayerDuration), timesync.WithTickInterval(1*time.Second), timesync.WithGenesisTime(gTime), - timesync.WithLogger(app.addLogger(ClockLogger, lg)), + timesync.WithLogger(app.addLogger(ClockLogger, lg).Zap()), ) if err != nil { return fmt.Errorf("cannot create clock: %w", err) diff --git a/signing/signer.go b/signing/signer.go index 4184f207db..93b07f20b5 100644 --- a/signing/signer.go +++ b/signing/signer.go @@ -161,3 +161,15 @@ func (es *EdSigner) VRFSigner() *VRFSigner { func (es *EdSigner) Prefix() []byte { return es.prefix } + +// Matches implements the gomock.Matcher interface for testing. +func (es *EdSigner) Matches(x any) bool { + if other, ok := x.(*EdSigner); ok { + return bytes.Equal(es.priv, other.priv) + } + return false +} + +func (es *EdSigner) String() string { + return es.NodeID().ShortString() +} diff --git a/systest/tests/distributed_post_verification_test.go b/systest/tests/distributed_post_verification_test.go index cb3792e8e9..9805c976a8 100644 --- a/systest/tests/distributed_post_verification_test.go +++ b/systest/tests/distributed_post_verification_test.go @@ -143,9 +143,10 @@ func TestPostMalfeasanceProof(t *testing.T) { timesync.WithLayerDuration(cfg.LayerDuration), timesync.WithTickInterval(1*time.Second), timesync.WithGenesisTime(cl.Genesis()), - timesync.WithLogger(log.NewFromLog(logger.Named("clock"))), + timesync.WithLogger(logger.Named("clock")), ) require.NoError(t, err) + t.Cleanup(clock.Close) grpcPostService := grpcserver.NewPostService(logger.Named("grpc-post-service")) grpczap.SetGrpcLoggerV2(grpclog, logger.Named("grpc")) @@ -165,7 +166,6 @@ func TestPostMalfeasanceProof(t *testing.T) { grpcPostService, cfg.PoetServers, logger.Named("nipostBuilder"), - signer, cfg.POET, clock, ) @@ -194,7 +194,7 @@ func TestPostMalfeasanceProof(t *testing.T) { break } - nipost, err := nipostBuilder.BuildNIPost(ctx, challenge) + nipost, err := nipostBuilder.BuildNIPost(ctx, signer, challenge) require.NoError(t, err) // 2.2 Create ATX with invalid POST diff --git a/timesync/clock.go b/timesync/clock.go index 5ce9325666..a42fd86a23 100644 --- a/timesync/clock.go +++ b/timesync/clock.go @@ -6,10 +6,10 @@ import ( "github.com/jonboulle/clockwork" "github.com/prometheus/client_golang/prometheus" + "go.uber.org/zap" "golang.org/x/sync/errgroup" "github.com/spacemeshos/go-spacemesh/common/types" - "github.com/spacemeshos/go-spacemesh/log" "github.com/spacemeshos/go-spacemesh/metrics" ) @@ -37,7 +37,7 @@ type NodeClock struct { stop chan struct{} once sync.Once - log log.Log + log *zap.Logger eg errgroup.Group } @@ -55,9 +55,9 @@ func NewClock(opts ...OptionFunc) (*NodeClock, error) { } gtime := cfg.genesisTime.Local() - cfg.log.With().Info("converting genesis time to local time", - log.Time("genesis", cfg.genesisTime), - log.Time("local", gtime), + cfg.log.Info("converting genesis time to local time", + zap.Time("genesis", cfg.genesisTime), + zap.Time("local", gtime), ) t := &NodeClock{ LayerConverter: LayerConverter{duration: cfg.layerDuration, genesis: gtime}, @@ -66,7 +66,7 @@ func NewClock(opts ...OptionFunc) (*NodeClock, error) { layerChannels: make(map[types.LayerID]chan struct{}), genesis: gtime, stop: make(chan struct{}), - log: *cfg.log, + log: cfg.log, } t.eg.Go(t.startClock) @@ -74,18 +74,18 @@ func NewClock(opts ...OptionFunc) (*NodeClock, error) { } func (t *NodeClock) startClock() error { - t.log.With().Info("starting global clock", - log.Time("now", t.clock.Now()), - log.Time("genesis", t.genesis), - log.Duration("layer_duration", t.duration), - log.Duration("tick_interval", t.tickInterval), + t.log.Info("starting global clock", + zap.Time("now", t.clock.Now()), + zap.Time("genesis", t.genesis), + zap.Duration("layer_duration", t.duration), + zap.Duration("tick_interval", t.tickInterval), ) ticker := t.clock.NewTicker(t.tickInterval) for { currLayer := t.TimeToLayer(t.clock.Now()) - t.log.With().Debug("global clock going to sleep before next tick", - log.Stringer("curr_layer", currLayer), + t.log.Debug("global clock going to sleep before next tick", + zap.Stringer("curr_layer", currLayer), ) select { @@ -111,7 +111,7 @@ func (t *NodeClock) Close() { t.log.Info("stopping clock") close(t.stop) if err := t.eg.Wait(); err != nil { - t.log.Error("errgroup: %v", err) + t.log.Error("failed to stop clock", zap.Error(err)) } t.log.Info("clock stopped") }) @@ -130,17 +130,17 @@ func (t *NodeClock) tick() { layer := t.TimeToLayer(t.clock.Now()) switch { case layer.Before(t.lastTicked): - t.log.With().Info("clock ticked back in time", - log.Stringer("layer", layer), - log.Stringer("last_ticked_layer", t.lastTicked), + t.log.Info("clock ticked back in time", + zap.Stringer("layer", layer), + zap.Stringer("last_ticked_layer", t.lastTicked), ) d := t.lastTicked.Difference(layer) tickDistance.Observe(float64(-d)) // don't warn right after fresh startup case layer.Difference(t.lastTicked) > 1 && t.lastTicked > 0: - t.log.With().Warning("clock skipped layers", - log.Stringer("layer", layer), - log.Stringer("last_ticked_layer", t.lastTicked), + t.log.Warn("clock skipped layers", + zap.Stringer("layer", layer), + zap.Stringer("last_ticked_layer", t.lastTicked), ) d := layer.Difference(t.lastTicked) tickDistance.Observe(float64(d)) diff --git a/timesync/clock_options.go b/timesync/clock_options.go index 442d75837f..8b15fc1629 100644 --- a/timesync/clock_options.go +++ b/timesync/clock_options.go @@ -5,8 +5,7 @@ import ( "time" "github.com/jonboulle/clockwork" - - "github.com/spacemeshos/go-spacemesh/log" + "go.uber.org/zap" ) type option struct { @@ -15,7 +14,7 @@ type option struct { layerDuration time.Duration tickInterval time.Duration - log *log.Log + log *zap.Logger } func (o *option) validate() error { @@ -75,9 +74,9 @@ func WithTickInterval(d time.Duration) OptionFunc { } // WithLogger sets the logger for the NodeClock. -func WithLogger(logger log.Log) OptionFunc { +func WithLogger(logger *zap.Logger) OptionFunc { return func(opts *option) error { - opts.log = &logger + opts.log = logger return nil } } diff --git a/timesync/clock_test.go b/timesync/clock_test.go index 889bd7ac3a..3799446ed8 100644 --- a/timesync/clock_test.go +++ b/timesync/clock_test.go @@ -9,17 +9,17 @@ import ( "github.com/jonboulle/clockwork" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "go.uber.org/zap/zaptest" "golang.org/x/sync/errgroup" "github.com/spacemeshos/go-spacemesh/common/types" - "github.com/spacemeshos/go-spacemesh/log/logtest" ) func Test_NodeClock_NewClock(t *testing.T) { clock, err := NewClock( WithGenesisTime(time.Now()), WithTickInterval(time.Second), - WithLogger(logtest.New(t)), + WithLogger(zaptest.NewLogger(t)), ) require.ErrorContains(t, err, "layer duration is zero") require.Nil(t, clock) @@ -27,7 +27,7 @@ func Test_NodeClock_NewClock(t *testing.T) { clock, err = NewClock( WithLayerDuration(time.Second), WithTickInterval(time.Second), - WithLogger(logtest.New(t)), + WithLogger(zaptest.NewLogger(t)), ) require.ErrorContains(t, err, "genesis time is zero") require.Nil(t, clock) @@ -43,7 +43,7 @@ func Test_NodeClock_NewClock(t *testing.T) { clock, err = NewClock( WithLayerDuration(time.Second), WithGenesisTime(time.Now()), - WithLogger(logtest.New(t)), + WithLogger(zaptest.NewLogger(t)), ) require.ErrorContains(t, err, "tick interval is zero") require.Nil(t, clock) @@ -52,7 +52,7 @@ func Test_NodeClock_NewClock(t *testing.T) { WithLayerDuration(time.Second), WithTickInterval(2*time.Second), WithGenesisTime(time.Now()), - WithLogger(logtest.New(t)), + WithLogger(zaptest.NewLogger(t)), ) require.ErrorContains(t, err, "tick interval must be between 0 and layer duration") require.Nil(t, clock) @@ -65,10 +65,11 @@ func Test_NodeClock_GenesisTime(t *testing.T) { WithLayerDuration(time.Second), WithTickInterval(time.Second/10), WithGenesisTime(genesis), - WithLogger(logtest.New(t)), + WithLogger(zaptest.NewLogger(t)), ) require.NoError(t, err) require.NotNil(t, clock) + t.Cleanup(clock.Close) require.Equal(t, genesis.Local(), clock.GenesisTime()) } @@ -78,10 +79,11 @@ func Test_NodeClock_Close(t *testing.T) { WithLayerDuration(time.Second), WithTickInterval(time.Second/10), WithGenesisTime(time.Now()), - WithLogger(logtest.New(t)), + WithLogger(zaptest.NewLogger(t)), ) require.NoError(t, err) require.NotNil(t, clock) + t.Cleanup(clock.Close) var eg errgroup.Group eg.Go(func() error { @@ -105,10 +107,11 @@ func Test_NodeClock_NoRaceOnTick(t *testing.T) { WithLayerDuration(time.Second), WithTickInterval(time.Second/10), WithGenesisTime(time.Now()), - WithLogger(logtest.New(t)), + WithLogger(zaptest.NewLogger(t)), ) require.NoError(t, err) require.NotNil(t, clock) + t.Cleanup(clock.Close) ctx, cancel := context.WithCancel(context.Background()) eg, egCtx := errgroup.WithContext(ctx) @@ -148,10 +151,11 @@ func Test_NodeClock_Await_BeforeGenesis(t *testing.T) { WithLayerDuration(layerDuration), WithTickInterval(tickInterval), WithGenesisTime(genesis), - WithLogger(logtest.New(t)), + WithLogger(zaptest.NewLogger(t)), ) require.NoError(t, err) require.NotNil(t, clock) + t.Cleanup(clock.Close) select { case <-clock.AwaitLayer(types.LayerID(0)): @@ -173,10 +177,11 @@ func Test_NodeClock_Await_PassedLayer(t *testing.T) { WithLayerDuration(layerDuration), WithTickInterval(tickInterval), WithGenesisTime(genesis), - WithLogger(logtest.New(t)), + WithLogger(zaptest.NewLogger(t)), ) require.NoError(t, err) require.NotNil(t, clock) + t.Cleanup(clock.Close) select { case <-clock.AwaitLayer(types.LayerID(4)): @@ -199,10 +204,11 @@ func Test_NodeClock_Await_WithClockMovingBackwards(t *testing.T) { WithLayerDuration(layerDuration), WithTickInterval(tickInterval), WithGenesisTime(genesis), - WithLogger(logtest.New(t)), + WithLogger(zaptest.NewLogger(t)), ) require.NoError(t, err) require.NotNil(t, clock) + t.Cleanup(clock.Close) // make the clock tick clock.tick() @@ -248,10 +254,11 @@ func Test_NodeClock_NonMonotonicTick_Forward(t *testing.T) { WithLayerDuration(layerDuration), WithTickInterval(tickInterval), WithGenesisTime(genesis), - WithLogger(logtest.New(t)), + WithLogger(zaptest.NewLogger(t)), ) require.NoError(t, err) require.NotNil(t, clock) + t.Cleanup(clock.Close) ch := clock.AwaitLayer(types.LayerID(6)) select { @@ -286,10 +293,11 @@ func Test_NodeClock_NonMonotonicTick_Backward(t *testing.T) { WithLayerDuration(layerDuration), WithTickInterval(tickInterval), WithGenesisTime(genesis), - WithLogger(logtest.New(t)), + WithLogger(zaptest.NewLogger(t)), ) require.NoError(t, err) require.NotNil(t, clock) + t.Cleanup(clock.Close) ch6 := clock.AwaitLayer(types.LayerID(6)) ch7 := clock.AwaitLayer(types.LayerID(7)) @@ -382,10 +390,11 @@ func Fuzz_NodeClock_CurrentLayer(f *testing.F) { WithLayerDuration(layerTime), WithTickInterval(tickInterval), WithGenesisTime(genesisTime), - WithLogger(logtest.New(t)), + WithLogger(zaptest.NewLogger(t)), ) require.NoError(t, err) require.NotNil(t, clock) + t.Cleanup(clock.Close) expectedLayer := uint32(0) if nowTime.After(genesisTime) {