From 4251c4e10c211fc554ba7b16ac8100a9bc9a8b23 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bartosz=20R=C3=B3=C5=BCa=C5=84ski?= Date: Tue, 24 Sep 2024 09:45:10 +0200 Subject: [PATCH 01/10] Abstract away DB in AtxBuilder --- activation/activation.go | 204 +++++-------- activation/activation_multi_test.go | 44 +-- activation/activation_test.go | 283 ++++++------------- activation/atx_service_db.go | 141 +++++++++ activation/atx_service_db_test.go | 104 +++++++ activation/builder_v2_test.go | 1 - activation/e2e/activation_test.go | 14 +- activation/e2e/builds_atx_v2_test.go | 13 +- activation/e2e/checkpoint_test.go | 26 +- activation/interface.go | 9 + activation/mocks.go | 140 +++++++++ common/errors.go | 5 + node/node.go | 18 +- sql/database.go | 3 +- sql/localsql/atxs/atxs.go | 37 +++ sql/localsql/schema/migrations/0010_atxs.sql | 12 + sql/localsql/schema/schema.sql | 12 +- 17 files changed, 672 insertions(+), 394 deletions(-) create mode 100644 activation/atx_service_db.go create mode 100644 activation/atx_service_db_test.go create mode 100644 common/errors.go create mode 100644 sql/localsql/atxs/atxs.go create mode 100644 sql/localsql/schema/migrations/0010_atxs.sql diff --git a/activation/activation.go b/activation/activation.go index 8c1188f70f..bd27a313ce 100644 --- a/activation/activation.go +++ b/activation/activation.go @@ -18,8 +18,8 @@ import ( "github.com/spacemeshos/go-spacemesh/activation/metrics" "github.com/spacemeshos/go-spacemesh/activation/wire" - "github.com/spacemeshos/go-spacemesh/atxsdata" "github.com/spacemeshos/go-spacemesh/codec" + "github.com/spacemeshos/go-spacemesh/common" "github.com/spacemeshos/go-spacemesh/common/types" "github.com/spacemeshos/go-spacemesh/events" "github.com/spacemeshos/go-spacemesh/log" @@ -27,7 +27,7 @@ import ( "github.com/spacemeshos/go-spacemesh/p2p/pubsub" "github.com/spacemeshos/go-spacemesh/signing" "github.com/spacemeshos/go-spacemesh/sql" - "github.com/spacemeshos/go-spacemesh/sql/atxs" + "github.com/spacemeshos/go-spacemesh/sql/localsql/atxs" "github.com/spacemeshos/go-spacemesh/sql/localsql/nipost" ) @@ -79,11 +79,11 @@ type Config struct { // it is responsible for initializing post, receiving poet proof and orchestrating nipost. after which it will // calculate total weight and providing relevant view as proof. type Builder struct { - accountLock sync.RWMutex - coinbaseAccount types.Address - conf Config - db sql.Executor - atxsdata *atxsdata.Data + accountLock sync.RWMutex + coinbaseAccount types.Address + conf Config + atxSvc AtxService + localDB sql.LocalDatabase publisher pubsub.Publisher nipostBuilder nipostBuilder @@ -95,8 +95,6 @@ type Builder struct { poets []PoetService poetCfg PoetConfig poetRetryInterval time.Duration - // delay before PoST in ATX is considered valid (counting from the time it was received) - postValidityDelay time.Duration // ATX versions versions []atxVersion @@ -119,16 +117,12 @@ type positioningAtxFinder struct { id types.ATXID forPublish types.EpochID } + golden types.ATXID + logger *zap.Logger } type BuilderOption func(*Builder) -func WithPostValidityDelay(delay time.Duration) BuilderOption { - return func(b *Builder) { - b.postValidityDelay = delay - } -} - // WithPoetRetryInterval modifies time that builder will have to wait before retrying ATX build process // if it failed due to issues with PoET server. func WithPoetRetryInterval(interval time.Duration) BuilderOption { @@ -157,12 +151,6 @@ func WithPoets(poets ...PoetService) BuilderOption { } } -func WithValidator(v nipostValidator) BuilderOption { - return func(b *Builder) { - b.validator = v - } -} - func WithPostStates(ps PostStates) BuilderOption { return func(b *Builder) { b.postStates = ps @@ -178,10 +166,10 @@ func BuilderAtxVersions(v AtxVersions) 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, - db sql.Executor, - atxsdata *atxsdata.Data, localDB sql.LocalDatabase, + atxService AtxService, publisher pubsub.Publisher, + nipostValidator nipostValidator, nipostBuilder nipostBuilder, layerClock layerClock, syncer syncer, @@ -192,18 +180,21 @@ func NewBuilder( parentCtx: context.Background(), signers: make(map[types.NodeID]*signing.EdSigner), conf: conf, - db: db, - atxsdata: atxsdata, localDB: localDB, publisher: publisher, + atxSvc: atxService, + validator: nipostValidator, nipostBuilder: nipostBuilder, layerClock: layerClock, syncer: syncer, logger: log, poetRetryInterval: defaultPoetRetryInterval, - postValidityDelay: 12 * time.Hour, postStates: NewPostStates(log), versions: []atxVersion{{0, types.AtxV1}}, + posAtxFinder: positioningAtxFinder{ + golden: conf.GoldenATXID, + logger: log, + }, } for _, opt := range opts { opt(b) @@ -346,7 +337,7 @@ func (b *Builder) SmesherIDs() []types.NodeID { func (b *Builder) BuildInitialPost(ctx context.Context, nodeID types.NodeID) error { // Generate the initial POST if we don't have an ATX... - if _, err := atxs.GetLastIDByNodeID(b.db, nodeID); err == nil { + if _, err := b.atxSvc.LastATX(ctx, nodeID); err == nil { return nil } // ...and if we haven't stored an initial post yet. @@ -355,7 +346,7 @@ func (b *Builder) BuildInitialPost(ctx context.Context, nodeID types.NodeID) err case err == nil: b.logger.Info("load initial post from db") return nil - case errors.Is(err, sql.ErrNotFound): + case errors.Is(err, common.ErrNotFound): b.logger.Info("creating initial post") default: return fmt.Errorf("get initial post: %w", err) @@ -527,11 +518,11 @@ func (b *Builder) BuildNIPostChallenge(ctx context.Context, nodeID types.NodeID) // Start building new challenge: // 1. get previous ATX - prevAtx, err := b.GetPrevAtx(nodeID) + prevAtx, err := b.atxSvc.LastATX(ctx, nodeID) switch { case err == nil: currentEpochId = max(currentEpochId, prevAtx.PublishEpoch) - case errors.Is(err, sql.ErrNotFound): + case errors.Is(err, common.ErrNotFound): // no previous ATX case err != nil: return nil, fmt.Errorf("get last ATX: %w", err) @@ -576,11 +567,11 @@ func (b *Builder) BuildNIPostChallenge(ctx context.Context, nodeID types.NodeID) // 4. build new challenge logger.Info("building new NiPOST challenge", zap.Uint32("current_epoch", currentEpochId.Uint32())) - prevAtx, err = b.GetPrevAtx(nodeID) + prevAtx, err = b.atxSvc.LastATX(ctx, nodeID) var challenge *types.NIPostChallenge switch { - case errors.Is(err, sql.ErrNotFound): + case errors.Is(err, common.ErrNotFound): logger.Info("no previous ATX found, creating an initial nipost challenge") challenge, err = b.buildInitialNIPostChallenge(ctx, logger, nodeID, publishEpochId) @@ -618,7 +609,7 @@ func (b *Builder) getExistingChallenge( challenge, err := nipost.Challenge(b.localDB, nodeID) switch { - case errors.Is(err, sql.ErrNotFound): + case errors.Is(err, common.ErrNotFound): return nil, nil case err != nil: @@ -692,14 +683,6 @@ func (b *Builder) buildInitialNIPostChallenge( }, nil } -func (b *Builder) GetPrevAtx(nodeID types.NodeID) (*types.ActivationTx, error) { - id, err := atxs.GetLastIDByNodeID(b.db, nodeID) - if err != nil { - return nil, fmt.Errorf("getting last ATXID: %w", err) - } - return atxs.Get(b.db, id) -} - // SetCoinbase sets the address rewardAddress to be the coinbase account written into the activation transaction // the rewards for blocks made by this miner will go to this address. func (b *Builder) SetCoinbase(rewardAddress types.Address) { @@ -747,6 +730,11 @@ func (b *Builder) PublishActivationTx(ctx context.Context, sig *signing.EdSigner case <-b.layerClock.AwaitLayer(challenge.PublishEpoch.FirstLayer()): } + err = atxs.AddBlob(b.localDB, challenge.PublishEpoch, atx.ID(), sig.NodeID(), codec.MustEncode(atx)) + if err != nil { + b.logger.Warn("failed to persist built ATX into the local DB - regossiping won't work", zap.Error(err)) + } + for { b.logger.Info( "broadcasting ATX", @@ -839,7 +827,7 @@ func (b *Builder) createAtx( case challenge.PrevATXID == types.EmptyATXID: atx.VRFNonce = (*uint64)(&nipostState.VRFNonce) default: - oldNonce, err := atxs.NonceByID(b.db, challenge.PrevATXID) + prevAtx, err := b.atxSvc.Atx(ctx, challenge.PrevATXID) if err != nil { b.logger.Warn("failed to get VRF nonce for ATX", zap.Error(err), @@ -847,12 +835,12 @@ func (b *Builder) createAtx( ) break } - if nipostState.VRFNonce != oldNonce { + if nipostState.VRFNonce != prevAtx.VRFNonce { b.logger.Info( "attaching a new VRF nonce in ATX", log.ZShortStringer("smesherID", sig.NodeID()), zap.Uint64("new nonce", uint64(nipostState.VRFNonce)), - zap.Uint64("old nonce", uint64(oldNonce)), + zap.Uint64("old nonce", uint64(prevAtx.VRFNonce)), ) atx.VRFNonce = (*uint64)(&nipostState.VRFNonce) } @@ -911,54 +899,44 @@ func (b *Builder) broadcast(ctx context.Context, atx scale.Encodable) (int, erro return len(buf), nil } -// searchPositioningAtx returns atx id with the highest tick height. -// publish epoch is used for caching the positioning atx. -func (b *Builder) searchPositioningAtx( +// find returns atx id with the highest tick height. +// The publish epoch (of the built ATX) is used for: +// - caching the positioning atx, +// - filtering candidates for positioning atx (it must be published in an earlier epoch than built ATX). +// +// It always returns an ATX, falling back to the golden one as the last resort. +func (f *positioningAtxFinder) find( ctx context.Context, - nodeID types.NodeID, + atxs AtxService, publish types.EpochID, -) (types.ATXID, error) { - logger := b.logger.With(log.ZShortStringer("smesherID", nodeID), zap.Uint32("publish epoch", publish.Uint32())) +) types.ATXID { + logger := f.logger.With(zap.Uint32("publish epoch", publish.Uint32())) - b.posAtxFinder.finding.Lock() - defer b.posAtxFinder.finding.Unlock() + f.finding.Lock() + defer f.finding.Unlock() - if found := b.posAtxFinder.found; found != nil && found.forPublish == publish { + if found := f.found; found != nil && found.forPublish == publish { logger.Debug("using cached positioning atx", log.ZShortStringer("atx_id", found.id)) - return found.id, nil + return found.id } - latestPublished, err := atxs.LatestEpoch(b.db) - if err != nil { - return types.EmptyATXID, fmt.Errorf("get latest epoch: %w", err) - } - - logger.Info("searching for positioning atx", zap.Uint32("latest_epoch", latestPublished.Uint32())) - - // positioning ATX publish epoch must be lower than the publish epoch of built ATX - positioningAtxPublished := min(latestPublished, publish-1) - id, err := findFullyValidHighTickAtx( - ctx, - b.atxsdata, - positioningAtxPublished, - b.conf.GoldenATXID, - b.validator, - logger, - VerifyChainOpts.AssumeValidBefore(time.Now().Add(-b.postValidityDelay)), - VerifyChainOpts.WithTrustedID(nodeID), - VerifyChainOpts.WithLogger(b.logger), - ) + id, err := atxs.PositioningATX(ctx, publish-1) if err != nil { - logger.Info("search failed - using golden atx as positioning atx", zap.Error(err)) - id = b.conf.GoldenATXID + logger.Warn("failed to get positioning ATX - falling back to golden", zap.Error(err)) + f.found = &struct { + id types.ATXID + forPublish types.EpochID + }{f.golden, publish} + return f.golden } - b.posAtxFinder.found = &struct { + logger.Debug("found candidate positioning atx", log.ZShortStringer("id", id)) + + f.found = &struct { id types.ATXID forPublish types.EpochID }{id, publish} - - return id, nil + return id } // getPositioningAtx returns the positioning ATX. @@ -970,15 +948,7 @@ func (b *Builder) getPositioningAtx( publish types.EpochID, previous *types.ActivationTx, ) (types.ATXID, error) { - id, err := b.searchPositioningAtx(ctx, nodeID, publish) - if err != nil { - return types.EmptyATXID, err - } - - b.logger.Debug("found candidate positioning atx", - log.ZShortStringer("id", id), - log.ZShortStringer("smesherID", nodeID), - ) + id := b.posAtxFinder.find(ctx, b.atxSvc, publish) if previous == nil { b.logger.Info("selected positioning atx", @@ -996,7 +966,7 @@ func (b *Builder) getPositioningAtx( return id, nil } - candidate, err := atxs.Get(b.db, id) + candidate, err := b.atxSvc.Atx(ctx, id) if err != nil { return types.EmptyATXID, fmt.Errorf("get candidate pos ATX %s: %w", id.ShortString(), err) } @@ -1016,23 +986,17 @@ func (b *Builder) getPositioningAtx( func (b *Builder) Regossip(ctx context.Context, nodeID types.NodeID) error { epoch := b.layerClock.CurrentLayer().GetEpoch() - atx, err := atxs.GetIDByEpochAndNodeID(b.db, epoch, nodeID) - if errors.Is(err, sql.ErrNotFound) { + id, blob, err := atxs.AtxBlob(b.localDB, epoch, nodeID) + if errors.Is(err, common.ErrNotFound) { return nil } else if err != nil { return err } - var blob sql.Blob - if _, err := atxs.LoadBlob(ctx, b.db, atx.Bytes(), &blob); err != nil { - return fmt.Errorf("get blob %s: %w", atx.ShortString(), err) - } - if len(blob.Bytes) == 0 { - return nil // checkpoint - } - if err := b.publisher.Publish(ctx, pubsub.AtxProtocol, blob.Bytes); err != nil { - return fmt.Errorf("republish %s: %w", atx.ShortString(), err) + + if err := b.publisher.Publish(ctx, pubsub.AtxProtocol, blob); err != nil { + return fmt.Errorf("republishing ATX %s: %w", id, err) } - b.logger.Debug("re-gossipped atx", log.ZShortStringer("smesherID", nodeID), log.ZShortStringer("atx", atx)) + b.logger.Debug("re-gossipped atx", log.ZShortStringer("smesherID", nodeID), log.ZShortStringer("atx ID", id)) return nil } @@ -1045,41 +1009,3 @@ func (b *Builder) version(publish types.EpochID) types.AtxVersion { } return version } - -func findFullyValidHighTickAtx( - ctx context.Context, - atxdata *atxsdata.Data, - publish types.EpochID, - goldenATXID types.ATXID, - validator nipostValidator, - logger *zap.Logger, - opts ...VerifyChainOption, -) (types.ATXID, error) { - var found *types.ATXID - - // iterate trough epochs, to get first valid, not malicious ATX with the biggest height - atxdata.IterateHighTicksInEpoch(publish+1, func(id types.ATXID) (contSearch bool) { - logger.Debug("found candidate for high-tick atx", log.ZShortStringer("id", id)) - if ctx.Err() != nil { - return false - } - // verify ATX-candidate by getting their dependencies (previous Atx, positioning ATX etc.) - // and verifying PoST for every dependency - if err := validator.VerifyChain(ctx, id, goldenATXID, opts...); err != nil { - logger.Debug("rejecting candidate for high-tick atx", zap.Error(err), log.ZShortStringer("id", id)) - return true - } - found = &id - return false - }) - - if ctx.Err() != nil { - return types.ATXID{}, ctx.Err() - } - - if found == nil { - return types.ATXID{}, ErrNotFound - } - - return *found, nil -} diff --git a/activation/activation_multi_test.go b/activation/activation_multi_test.go index ce795e53b5..56736a7c51 100644 --- a/activation/activation_multi_test.go +++ b/activation/activation_multi_test.go @@ -18,7 +18,7 @@ import ( "github.com/spacemeshos/go-spacemesh/p2p/pubsub" "github.com/spacemeshos/go-spacemesh/signing" "github.com/spacemeshos/go-spacemesh/sql" - "github.com/spacemeshos/go-spacemesh/sql/atxs" + localatxs "github.com/spacemeshos/go-spacemesh/sql/localsql/atxs" "github.com/spacemeshos/go-spacemesh/sql/localsql/nipost" ) @@ -188,46 +188,22 @@ func TestRegossip(t *testing.T) { }) t.Run("success", func(t *testing.T) { - goldenATXID := types.RandomATXID() tab := newTestBuilder(t, 5) - var refAtx *types.ActivationTx - + var ( + smesher types.NodeID + blob []byte + ) for _, sig := range tab.signers { - atx := newInitialATXv1(t, goldenATXID) - atx.PublishEpoch = layer.GetEpoch() - atx.Sign(sig) - vAtx := toAtx(t, atx) - require.NoError(t, atxs.Add(tab.db, vAtx, atx.Blob())) - - if refAtx == nil { - refAtx = vAtx - } + smesher = sig.NodeID() + blob = types.RandomBytes(20) + localatxs.AddBlob(tab.localDb, layer.GetEpoch(), types.RandomATXID(), smesher, blob) } - var blob sql.Blob - ver, err := atxs.LoadBlob(context.Background(), tab.db, refAtx.ID().Bytes(), &blob) - require.NoError(t, err) - require.Equal(t, types.AtxV1, ver) - // 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.Bytes) - 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.db, &atx)) - tab.mclock.EXPECT().CurrentLayer().Return(layer) - require.NoError(t, tab.Regossip(context.Background(), sig.NodeID())) - } + tab.mpub.EXPECT().Publish(ctx, pubsub.AtxProtocol, blob) + require.NoError(t, tab.Regossip(ctx, smesher)) }) } diff --git a/activation/activation_test.go b/activation/activation_test.go index 4f3550d7dd..620e5afb5a 100644 --- a/activation/activation_test.go +++ b/activation/activation_test.go @@ -32,7 +32,6 @@ import ( "github.com/spacemeshos/go-spacemesh/sql/atxs" "github.com/spacemeshos/go-spacemesh/sql/localsql" "github.com/spacemeshos/go-spacemesh/sql/localsql/nipost" - sqlmocks "github.com/spacemeshos/go-spacemesh/sql/mocks" "github.com/spacemeshos/go-spacemesh/sql/statesql" ) @@ -78,8 +77,8 @@ func newTestBuilder(tb testing.TB, numSigners int, opts ...BuilderOption) *testA ctrl := gomock.NewController(tb) tab := &testAtxBuilder{ - db: statesql.InMemory(), - localDb: localsql.InMemory(sql.WithConnections(numSigners)), + db: statesql.InMemoryTest(tb), + localDb: localsql.InMemoryTest(tb, sql.WithConnections(numSigners)), goldenATXID: types.ATXID(types.HexToHash32("77777")), observedLogs: observedLogs, @@ -92,20 +91,26 @@ func newTestBuilder(tb testing.TB, numSigners int, opts ...BuilderOption) *testA mValidator: NewMocknipostValidator(ctrl), } - opts = append(opts, WithValidator(tab.mValidator)) - cfg := Config{ GoldenATXID: tab.goldenATXID, } tab.msync.EXPECT().RegisterForATXSynced().DoAndReturn(closedChan).AnyTimes() - b := NewBuilder( - cfg, + atxService := NewDBAtxService( tab.db, + tab.goldenATXID, atxsdata.New(), + tab.mValidator, + logger, + ) + + b := NewBuilder( + cfg, tab.localDb, + atxService, tab.mpub, + tab.mValidator, tab.mnipost, tab.mclock, tab.msync, @@ -142,7 +147,6 @@ func publishAtxV1( tb, atxs.SetPost(tab.db, watx.ID(), watx.PrevATXID, 0, watx.SmesherID, watx.NumUnits, watx.PublishEpoch), ) - tab.atxsdata.AddFromAtx(toAtx(tb, &watx), false) return &watx } @@ -354,14 +358,17 @@ func TestBuilder_PublishActivationTx_HappyFlow(t *testing.T) { posEpoch := postGenesisEpoch currLayer := posEpoch.FirstLayer() - prevAtx := newInitialATXv1(t, tab.goldenATXID) - prevAtx.Sign(sig) - require.NoError(t, atxs.Add(tab.db, toAtx(t, prevAtx), prevAtx.Blob())) - tab.atxsdata.AddFromAtx(toAtx(t, prevAtx), false) + prevAtx := &types.ActivationTx{ + CommitmentATX: &tab.goldenATXID, + Coinbase: tab.Coinbase(), + NumUnits: 100, + TickCount: 10, + SmesherID: sig.NodeID(), + } + require.NoError(t, atxs.Add(tab.db, prevAtx, types.AtxBlob{})) // create and publish ATX tab.mclock.EXPECT().CurrentLayer().Return(currLayer).Times(4) - tab.mValidator.EXPECT().VerifyChain(gomock.Any(), prevAtx.ID(), tab.goldenATXID, gomock.Any()) atx1 := publishAtxV1(t, tab, sig.NodeID(), posEpoch, &currLayer, layersPerEpoch) require.NotNil(t, atx1) require.Equal(t, prevAtx.ID(), atx1.PositioningATXID) @@ -369,7 +376,6 @@ func TestBuilder_PublishActivationTx_HappyFlow(t *testing.T) { // create and publish another ATX currLayer = (posEpoch + 1).FirstLayer() tab.mclock.EXPECT().CurrentLayer().Return(currLayer).Times(4) - tab.mValidator.EXPECT().VerifyChain(gomock.Any(), atx1.ID(), tab.goldenATXID, gomock.Any()) atx2 := publishAtxV1(t, tab, sig.NodeID(), atx1.PublishEpoch, &currLayer, layersPerEpoch) require.NotNil(t, atx2) require.NotEqual(t, atx1, atx2) @@ -393,7 +399,6 @@ func TestBuilder_Loop_WaitsOnStaleChallenge(t *testing.T) { prevAtx := newInitialATXv1(t, tab.goldenATXID) prevAtx.Sign(sig) require.NoError(t, atxs.Add(tab.db, toAtx(t, prevAtx), prevAtx.Blob())) - tab.atxsdata.AddFromAtx(toAtx(t, prevAtx), false) tab.mclock.EXPECT().CurrentLayer().Return(currLayer).AnyTimes() tab.mclock.EXPECT().LayerToTime(gomock.Any()).DoAndReturn( @@ -417,8 +422,6 @@ func TestBuilder_Loop_WaitsOnStaleChallenge(t *testing.T) { return ch }) - tab.mValidator.EXPECT().VerifyChain(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()) - // Act & Verify var eg errgroup.Group eg.Go(func() error { @@ -442,7 +445,6 @@ func TestBuilder_PublishActivationTx_FaultyNet(t *testing.T) { prevAtx := newInitialATXv1(t, tab.goldenATXID) prevAtx.Sign(sig) require.NoError(t, atxs.Add(tab.db, toAtx(t, prevAtx), prevAtx.Blob())) - tab.atxsdata.AddFromAtx(toAtx(t, prevAtx), false) publishEpoch := posEpoch + 1 tab.mclock.EXPECT().CurrentLayer().DoAndReturn(func() types.LayerID { return currLayer }).AnyTimes() @@ -489,7 +491,6 @@ func TestBuilder_PublishActivationTx_FaultyNet(t *testing.T) { // after successful publish, state is cleaned up 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 func(_ context.Context, _ string, got []byte) error { @@ -517,7 +518,6 @@ func TestBuilder_PublishActivationTx_UsesExistingChallengeOnLatePublish(t *testi prevAtx.Sign(sig) vPrevAtx := toAtx(t, prevAtx) require.NoError(t, atxs.Add(tab.db, vPrevAtx, prevAtx.Blob())) - tab.atxsdata.AddFromAtx(toAtx(t, prevAtx), false) publishEpoch := currLayer.GetEpoch() tab.mclock.EXPECT().CurrentLayer().DoAndReturn(func() types.LayerID { return currLayer }).AnyTimes() @@ -537,12 +537,14 @@ func TestBuilder_PublishActivationTx_UsesExistingChallengeOnLatePublish(t *testi NumUnits: DefaultPostSetupOpts().NumUnits, LabelsPerUnit: DefaultPostConfig().LabelsPerUnit, }, nil).AnyTimes() - tab.mnipost.EXPECT().BuildNIPost(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( - func(_ context.Context, _ *signing.EdSigner, _ types.Hash32, _ *types.NIPostChallenge, - ) (*nipost.NIPostState, error) { - currLayer = currLayer.Add(1) - return newNIPostWithPoet(t, []byte("66666")), nil - }) + tab.mnipost.EXPECT(). + BuildNIPost(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()). + DoAndReturn( + func(_ context.Context, _ *signing.EdSigner, _ types.Hash32, _ *types.NIPostChallenge, + ) (*nipost.NIPostState, error) { + currLayer = currLayer.Add(1) + return newNIPostWithPoet(t, []byte("66666")), nil + }) done := make(chan struct{}) close(done) tab.mclock.EXPECT().AwaitLayer(publishEpoch.FirstLayer()).DoAndReturn( @@ -594,7 +596,6 @@ func TestBuilder_PublishActivationTx_RebuildNIPostWhenTargetEpochPassed(t *testi prevAtx.Sign(sig) vPrevAtx := toAtx(t, prevAtx) require.NoError(t, atxs.Add(tab.db, vPrevAtx, prevAtx.Blob())) - tab.atxsdata.AddFromAtx(toAtx(t, prevAtx), false) publishEpoch := posEpoch + 1 tab.mclock.EXPECT().CurrentLayer().DoAndReturn( @@ -623,7 +624,6 @@ func TestBuilder_PublishActivationTx_RebuildNIPostWhenTargetEpochPassed(t *testi } return done }) - tab.mValidator.EXPECT().VerifyChain(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()) ctx, cancel := context.WithCancel(context.Background()) defer cancel() var built *wire.ActivationTxV1 @@ -655,10 +655,8 @@ func TestBuilder_PublishActivationTx_RebuildNIPostWhenTargetEpochPassed(t *testi posAtx := newInitialATXv1(t, tab.goldenATXID, func(atx *wire.ActivationTxV1) { atx.PublishEpoch = posEpoch }) posAtx.Sign(sig) require.NoError(t, atxs.Add(tab.db, toAtx(t, posAtx), posAtx.Blob())) - tab.atxsdata.AddFromAtx(toAtx(t, posAtx), false) tab.mclock.EXPECT().CurrentLayer().DoAndReturn(func() types.LayerID { return currLayer }).AnyTimes() tab.mnipost.EXPECT().ResetState(sig.NodeID()).Return(nil) - tab.mValidator.EXPECT().VerifyChain(gomock.Any(), posAtx.ID(), tab.goldenATXID, gomock.Any()) built2 := publishAtxV1(t, tab, sig.NodeID(), posEpoch, &currLayer, layersPerEpoch) require.NotNil(t, built2) require.NotEqual(t, built.NIPostChallengeV1, built2.NIPostChallengeV1) @@ -784,31 +782,35 @@ func TestBuilder_PublishActivationTx_PrevATXWithoutPrevATX(t *testing.T) { r := require.New(t) // Arrange - tab := newTestBuilder(t, 1, WithPoetConfig(PoetConfig{PhaseShift: layerDuration * 4})) + actSvc := NewMockAtxService(gomock.NewController(t)) + tab := newTestBuilder(t, 1, + WithPoetConfig(PoetConfig{PhaseShift: layerDuration * 4}), + ) + tab.atxSvc = actSvc sig := maps.Values(tab.signers)[0] - otherSigner, err := signing.NewEdSigner() - r.NoError(err) - poetBytes := []byte("poet") - currentLayer := postGenesisEpoch.FirstLayer().Add(3) - posAtx := newInitialATXv1(t, tab.goldenATXID) - posAtx.Sign(otherSigner) - vPosAtx := toAtx(t, posAtx) - vPosAtx.TickCount = 100 - r.NoError(atxs.Add(tab.db, vPosAtx, posAtx.Blob())) - tab.atxsdata.AddFromAtx(vPosAtx, false) - - nonce := types.VRFPostIndex(123) - prevAtx := newInitialATXv1(t, tab.goldenATXID, func(atx *wire.ActivationTxV1) { - atx.VRFNonce = (*uint64)(&nonce) - }) - prevAtx.Sign(sig) - vPrevAtx := toAtx(t, prevAtx) - r.NoError(atxs.Add(tab.db, vPrevAtx, prevAtx.Blob())) - tab.atxsdata.AddFromAtx(vPrevAtx, false) + posAtx := &types.ActivationTx{ + PublishEpoch: 1, + TickCount: 100, + SmesherID: types.RandomNodeID(), + } + posAtx.SetID(types.RandomATXID()) + actSvc.EXPECT().PositioningATX(gomock.Any(), gomock.Any()).Return(posAtx.ID(), nil) + actSvc.EXPECT().Atx(gomock.Any(), posAtx.ID()).Return(posAtx, nil) + + prevAtx := &types.ActivationTx{ + PublishEpoch: 1, + TickCount: 10, + SmesherID: types.RandomNodeID(), + VRFNonce: types.VRFPostIndex(123), + } + prevAtx.SetID(types.RandomATXID()) + actSvc.EXPECT().LastATX(gomock.Any(), sig.NodeID()).Return(prevAtx, nil).Times(2) + actSvc.EXPECT().Atx(gomock.Any(), prevAtx.ID()).Return(prevAtx, nil) // Act + currentLayer := prevAtx.PublishEpoch.FirstLayer().Add(3) tab.msync.EXPECT().RegisterForATXSynced().DoAndReturn(closedChan).AnyTimes() tab.mclock.EXPECT().CurrentLayer().Return(currentLayer).AnyTimes() @@ -818,7 +820,7 @@ 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( + tab.mclock.EXPECT().AwaitLayer(prevAtx.PublishEpoch.FirstLayer().Add(layersPerEpoch)).DoAndReturn( func(layer types.LayerID) <-chan struct{} { ch := make(chan struct{}) close(ch) @@ -830,7 +832,7 @@ func TestBuilder_PublishActivationTx_PrevATXWithoutPrevATX(t *testing.T) { tab.mpostClient.EXPECT().Info(gomock.Any()).Return(&types.PostInfo{ NodeID: sig.NodeID(), CommitmentATX: commitmentATX, - Nonce: &nonce, + Nonce: &prevAtx.VRFNonce, NumUnits: DefaultPostSetupOpts().NumUnits, LabelsPerUnit: DefaultPostConfig().LabelsPerUnit, @@ -845,8 +847,6 @@ func TestBuilder_PublishActivationTx_PrevATXWithoutPrevATX(t *testing.T) { return newNIPostWithPoet(t, 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 { @@ -870,7 +870,7 @@ func TestBuilder_PublishActivationTx_PrevATXWithoutPrevATX(t *testing.T) { r.NoError(tab.PublishActivationTx(context.Background(), sig)) // state is cleaned up - _, err = nipost.Challenge(tab.localDB, sig.NodeID()) + _, err := nipost.Challenge(tab.localDB, sig.NodeID()) require.ErrorIs(t, err, sql.ErrNotFound) } @@ -878,19 +878,23 @@ func TestBuilder_PublishActivationTx_TargetsEpochBasedOnPosAtx(t *testing.T) { r := require.New(t) // Arrange - tab := newTestBuilder(t, 1, WithPoetConfig(PoetConfig{PhaseShift: layerDuration * 4})) + atxSvc := NewMockAtxService(gomock.NewController(t)) + tab := newTestBuilder(t, 1, + WithPoetConfig(PoetConfig{PhaseShift: layerDuration * 4}), + ) + tab.atxSvc = atxSvc sig := maps.Values(tab.signers)[0] - otherSigner, err := signing.NewEdSigner() - r.NoError(err) - poetBytes := []byte("poet") currentLayer := postGenesisEpoch.FirstLayer().Add(3) - posEpoch := postGenesisEpoch - posAtx := newInitialATXv1(t, tab.goldenATXID) - posAtx.Sign(otherSigner) - r.NoError(atxs.Add(tab.db, toAtx(t, posAtx), posAtx.Blob())) - tab.atxsdata.AddFromAtx(toAtx(t, posAtx), false) + posAtx := &types.ActivationTx{ + PublishEpoch: 2, + TickCount: 100, + SmesherID: types.RandomNodeID(), + } + posAtx.SetID(types.RandomATXID()) + atxSvc.EXPECT().PositioningATX(gomock.Any(), gomock.Any()).Return(posAtx.ID(), nil) + atxSvc.EXPECT().LastATX(gomock.Any(), sig.NodeID()).Return(nil, sql.ErrNotFound).Times(2) // Act & Assert tab.msync.EXPECT().RegisterForATXSynced().DoAndReturn(closedChan).AnyTimes() @@ -928,7 +932,6 @@ func TestBuilder_PublishActivationTx_TargetsEpochBasedOnPosAtx(t *testing.T) { return newNIPostWithPoet(t, 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 { @@ -940,7 +943,7 @@ func TestBuilder_PublishActivationTx_TargetsEpochBasedOnPosAtx(t *testing.T) { r.Equal(types.EmptyATXID, atx.PrevATXID) r.NotNil(atx.InitialPost) r.Equal(posAtx.ID(), atx.PositioningATXID) - r.Equal(posEpoch+1, atx.PublishEpoch) + r.Equal(posAtx.PublishEpoch+1, atx.PublishEpoch) r.Equal(poetBytes, atx.NIPost.PostMetadata.Challenge) return nil @@ -970,7 +973,7 @@ func TestBuilder_PublishActivationTx_TargetsEpochBasedOnPosAtx(t *testing.T) { r.NoError(tab.PublishActivationTx(context.Background(), sig)) // state is cleaned up - _, err = nipost.Challenge(tab.localDB, sig.NodeID()) + _, err := nipost.Challenge(tab.localDB, sig.NodeID()) require.ErrorIs(t, err, sql.ErrNotFound) } @@ -983,7 +986,6 @@ func TestBuilder_PublishActivationTx_FailsWhenNIPostBuilderFails(t *testing.T) { prevAtx := newInitialATXv1(t, tab.goldenATXID) prevAtx.Sign(sig) require.NoError(t, atxs.Add(tab.db, toAtx(t, prevAtx), prevAtx.Blob())) - tab.atxsdata.AddFromAtx(toAtx(t, prevAtx), false) tab.mclock.EXPECT().CurrentLayer().Return(posEpoch.FirstLayer()).AnyTimes() tab.mclock.EXPECT().LayerToTime(gomock.Any()).DoAndReturn( @@ -996,7 +998,6 @@ func TestBuilder_PublishActivationTx_FailsWhenNIPostBuilderFails(t *testing.T) { tab.mnipost.EXPECT(). BuildNIPost(gomock.Any(), sig, gomock.Any(), gomock.Any()). Return(nil, nipostErr) - tab.mValidator.EXPECT().VerifyChain(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()) require.ErrorIs(t, tab.PublishActivationTx(context.Background(), sig), nipostErr) // state is preserved @@ -1041,7 +1042,6 @@ func TestBuilder_RetryPublishActivationTx(t *testing.T) { prevAtx := newInitialATXv1(t, tab.goldenATXID) prevAtx.Sign(sig) require.NoError(t, atxs.Add(tab.db, toAtx(t, prevAtx), prevAtx.Blob())) - tab.atxsdata.AddFromAtx(toAtx(t, prevAtx), false) currLayer := prevAtx.PublishEpoch.FirstLayer() tab.mclock.EXPECT().CurrentLayer().DoAndReturn(func() types.LayerID { return currLayer }).AnyTimes() @@ -1089,7 +1089,6 @@ func TestBuilder_RetryPublishActivationTx(t *testing.T) { ) 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() @@ -1336,74 +1335,20 @@ func TestWaitPositioningAtx(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, 1) - sig := maps.Values(tab.signers)[0] - - // Invalid chain with high height - sigInvalid, err := signing.NewEdSigner() - require.NoError(t, err) - invalidAtx := newInitialATXv1(t, tab.goldenATXID) - invalidAtx.Sign(sigInvalid) - vInvalidAtx := toAtx(t, invalidAtx) - vInvalidAtx.TickCount = 100 - require.NoError(t, err) - require.NoError(t, atxs.Add(tab.db, vInvalidAtx, invalidAtx.Blob())) - tab.atxsdata.AddFromAtx(vInvalidAtx, false) - - // Valid chain with lower height - sigValid, err := signing.NewEdSigner() - require.NoError(t, err) - validAtx := newInitialATXv1(t, tab.goldenATXID) - validAtx.NumUnits += 10 - validAtx.Sign(sigValid) - vValidAtx := toAtx(t, validAtx) - require.NoError(t, atxs.Add(tab.db, vValidAtx, validAtx.Blob())) - tab.atxsdata.AddFromAtx(vValidAtx, false) - - tab.mValidator.EXPECT(). - VerifyChain(gomock.Any(), invalidAtx.ID(), tab.goldenATXID, gomock.Any()). - Return(errors.New("")) - tab.mValidator.EXPECT(). - VerifyChain(gomock.Any(), validAtx.ID(), tab.goldenATXID, gomock.Any()) - - posAtxID, err := tab.getPositioningAtx(context.Background(), sig.NodeID(), 77, nil) - require.NoError(t, err) - require.Equal(t, posAtxID, vValidAtx.ID()) - - // should use the cached positioning ATX when asked for the same publish epoch - posAtxID, err = tab.getPositioningAtx(context.Background(), sig.NodeID(), 77, nil) - require.NoError(t, err) - require.Equal(t, posAtxID, vValidAtx.ID()) - - // should lookup again when asked for a different publish epoch - tab.mValidator.EXPECT(). - VerifyChain(gomock.Any(), invalidAtx.ID(), tab.goldenATXID, gomock.Any()). - Return(errors.New("")) - tab.mValidator.EXPECT(). - VerifyChain(gomock.Any(), validAtx.ID(), tab.goldenATXID, gomock.Any()) - - posAtxID, err = tab.getPositioningAtx(context.Background(), sig.NodeID(), 99, nil) - require.NoError(t, err) - require.Equal(t, posAtxID, vValidAtx.ID()) -} - func TestGetPositioningAtx(t *testing.T) { t.Parallel() - t.Run("db failed", func(t *testing.T) { + t.Run("picks golden when failed", func(t *testing.T) { t.Parallel() + atxSvc := NewMockAtxService(gomock.NewController(t)) tab := newTestBuilder(t, 1) + tab.atxSvc = atxSvc - db := sqlmocks.NewMockExecutor(gomock.NewController(t)) - tab.Builder.db = db - expected := errors.New("db error") - db.EXPECT().Exec(gomock.Any(), gomock.Any(), gomock.Any()).Return(0, expected) + expected := errors.New("expected error") + atxSvc.EXPECT().PositioningATX(gomock.Any(), gomock.Any()).Return(types.ATXID{}, expected) - none, err := tab.getPositioningAtx(context.Background(), types.EmptyNodeID, 99, nil) - require.ErrorIs(t, err, expected) - require.Equal(t, types.ATXID{}, none) + posATX, err := tab.getPositioningAtx(context.Background(), types.EmptyNodeID, 99, nil) + require.NoError(t, err) + require.Equal(t, tab.goldenATXID, posATX) }) t.Run("picks golden if no ATXs", func(t *testing.T) { tab := newTestBuilder(t, 1) @@ -1420,21 +1365,18 @@ func TestGetPositioningAtx(t *testing.T) { require.Equal(t, prev.ID(), atx) }) t.Run("prefers own previous when it has GTE ticks", func(t *testing.T) { + atxSvc := NewMockAtxService(gomock.NewController(t)) tab := newTestBuilder(t, 1) + tab.atxSvc = atxSvc atxInDb := &types.ActivationTx{TickCount: 10} atxInDb.SetID(types.RandomATXID()) - require.NoError(t, atxs.Add(tab.db, atxInDb, types.AtxBlob{})) - tab.atxsdata.AddFromAtx(atxInDb, false) + atxSvc.EXPECT().PositioningATX(gomock.Any(), types.EpochID(98)).Return(atxInDb.ID(), nil) + atxSvc.EXPECT().Atx(context.Background(), atxInDb.ID()).Return(atxInDb, nil).Times(2) prev := &types.ActivationTx{TickCount: 100} prev.SetID(types.RandomATXID()) - tab.mValidator.EXPECT().VerifyChain(gomock.Any(), atxInDb.ID(), tab.goldenATXID, gomock.Any()) - found, err := tab.searchPositioningAtx(context.Background(), types.EmptyNodeID, 99) - require.NoError(t, err) - require.Equal(t, atxInDb.ID(), found) - // prev.Height > found.Height selected, err := tab.getPositioningAtx(context.Background(), types.EmptyNodeID, 99, prev) require.NoError(t, err) @@ -1446,67 +1388,6 @@ func TestGetPositioningAtx(t *testing.T) { require.NoError(t, err) require.Equal(t, prev.ID(), selected) }) - t.Run("prefers own previous or golded when positioning ATX selection timout expired", func(t *testing.T) { - tab := newTestBuilder(t, 1) - - atxInDb := &types.ActivationTx{TickCount: 100} - atxInDb.SetID(types.RandomATXID()) - require.NoError(t, atxs.Add(tab.db, atxInDb, types.AtxBlob{})) - tab.atxsdata.AddFromAtx(atxInDb, false) - - prev := &types.ActivationTx{TickCount: 90} - prev.SetID(types.RandomATXID()) - - // no timeout set up - tab.mValidator.EXPECT().VerifyChain(gomock.Any(), atxInDb.ID(), tab.goldenATXID, gomock.Any()) - found, err := tab.getPositioningAtx(context.Background(), types.EmptyNodeID, 99, prev) - require.NoError(t, err) - require.Equal(t, atxInDb.ID(), found) - - tab.posAtxFinder.found = nil - - // timeout set up, prev ATX exists - ctx, cancel := context.WithCancel(context.Background()) - cancel() - - selected, err := tab.getPositioningAtx(ctx, types.EmptyNodeID, 99, prev) - require.NoError(t, err) - require.Equal(t, prev.ID(), selected) - - tab.posAtxFinder.found = nil - - // timeout set up, prev ATX do not exists - ctx, cancel = context.WithCancel(context.Background()) - cancel() - - selected, err = tab.getPositioningAtx(ctx, types.EmptyNodeID, 99, nil) - require.NoError(t, err) - require.Equal(t, tab.goldenATXID, selected) - }) -} - -func TestFindFullyValidHighTickAtx(t *testing.T) { - t.Parallel() - golden := types.RandomATXID() - - t.Run("skips malicious ATXs", func(t *testing.T) { - data := atxsdata.New() - atxMal := &types.ActivationTx{TickCount: 100, SmesherID: types.RandomNodeID()} - atxMal.SetID(types.RandomATXID()) - data.AddFromAtx(atxMal, true) - - atxLower := &types.ActivationTx{TickCount: 10, SmesherID: types.RandomNodeID()} - atxLower.SetID(types.RandomATXID()) - data.AddFromAtx(atxLower, false) - - mValidator := NewMocknipostValidator(gomock.NewController(t)) - mValidator.EXPECT().VerifyChain(gomock.Any(), atxLower.ID(), golden, gomock.Any()) - - lg := zaptest.NewLogger(t) - found, err := findFullyValidHighTickAtx(context.Background(), data, 0, golden, mValidator, lg) - require.NoError(t, err) - require.Equal(t, atxLower.ID(), found) - }) } // Test_Builder_RegenerateInitialPost tests the coverage for the edge case diff --git a/activation/atx_service_db.go b/activation/atx_service_db.go new file mode 100644 index 0000000000..94e40b17ba --- /dev/null +++ b/activation/atx_service_db.go @@ -0,0 +1,141 @@ +package activation + +import ( + "context" + "fmt" + "time" + + "go.uber.org/zap" + + "github.com/spacemeshos/go-spacemesh/atxsdata" + "github.com/spacemeshos/go-spacemesh/common/types" + "github.com/spacemeshos/go-spacemesh/log" + "github.com/spacemeshos/go-spacemesh/sql" + "github.com/spacemeshos/go-spacemesh/sql/atxs" +) + +// dbAtxService implements AtxService by accessing the state database. +type dbAtxService struct { + golden types.ATXID + logger *zap.Logger + db sql.Executor + atxsdata *atxsdata.Data + validator nipostValidator + cfg dbAtxServiceConfig +} + +type dbAtxServiceConfig struct { + // delay before PoST in ATX is considered valid (counting from the time it was received) + postValidityDelay time.Duration +} + +type dbAtxServiceOption func(*dbAtxServiceConfig) + +func WithPostValidityDelay(delay time.Duration) dbAtxServiceOption { + return func(cfg *dbAtxServiceConfig) { + cfg.postValidityDelay = delay + } +} + +func NewDBAtxService( + db sql.Executor, + golden types.ATXID, + atxsdata *atxsdata.Data, + validator nipostValidator, + logger *zap.Logger, + opts ...dbAtxServiceOption, +) *dbAtxService { + cfg := dbAtxServiceConfig{ + postValidityDelay: time.Hour * 12, + } + + for _, opt := range opts { + opt(&cfg) + } + + return &dbAtxService{ + golden: golden, + logger: logger, + db: db, + atxsdata: atxsdata, + validator: validator, + cfg: cfg, + } +} + +func (s *dbAtxService) Atx(_ context.Context, id types.ATXID) (*types.ActivationTx, error) { + return atxs.Get(s.db, id) +} + +func (s *dbAtxService) LastATX(ctx context.Context, id types.NodeID) (*types.ActivationTx, error) { + atxid, err := atxs.GetLastIDByNodeID(s.db, id) + if err != nil { + return nil, fmt.Errorf("getting last ATXID: %w", err) + } + return atxs.Get(s.db, atxid) +} + +func (s *dbAtxService) PositioningATX(ctx context.Context, maxPublish types.EpochID) (types.ATXID, error) { + latestPublished, err := atxs.LatestEpoch(s.db) + if err != nil { + return types.EmptyATXID, fmt.Errorf("get latest epoch: %w", err) + } + s.logger.Info("searching for positioning atx", zap.Uint32("latest_epoch", latestPublished.Uint32())) + + // positioning ATX publish epoch must be lower than the publish epoch of built ATX + positioningAtxPublished := min(latestPublished, maxPublish) + id, err := findFullyValidHighTickAtx( + ctx, + s.atxsdata, + positioningAtxPublished, + s.golden, + s.validator, s.logger, + VerifyChainOpts.AssumeValidBefore(time.Now().Add(-s.cfg.postValidityDelay)), + // VerifyChainOpts.WithTrustedID(nodeID), + VerifyChainOpts.WithLogger(s.logger), + ) + if err != nil { + s.logger.Info("search failed - using golden atx as positioning atx", zap.Error(err)) + id = s.golden + } + + return id, nil +} + +func findFullyValidHighTickAtx( + ctx context.Context, + atxdata *atxsdata.Data, + publish types.EpochID, + goldenATXID types.ATXID, + validator nipostValidator, + logger *zap.Logger, + opts ...VerifyChainOption, +) (types.ATXID, error) { + var found *types.ATXID + + // iterate trough epochs, to get first valid, not malicious ATX with the biggest height + atxdata.IterateHighTicksInEpoch(publish+1, func(id types.ATXID) (contSearch bool) { + logger.Debug("found candidate for high-tick atx", log.ZShortStringer("id", id)) + if ctx.Err() != nil { + return false + } + // verify ATX-candidate by getting their dependencies (previous Atx, positioning ATX etc.) + // and verifying PoST for every dependency + if err := validator.VerifyChain(ctx, id, goldenATXID, opts...); err != nil { + logger.Debug("rejecting candidate for high-tick atx", zap.Error(err), log.ZShortStringer("id", id)) + return true + } + found = &id + return false + }) + + if ctx.Err() != nil { + return types.ATXID{}, ctx.Err() + } + + if found == nil { + return types.ATXID{}, ErrNotFound + } + + return *found, nil +} diff --git a/activation/atx_service_db_test.go b/activation/atx_service_db_test.go new file mode 100644 index 0000000000..f2e916a595 --- /dev/null +++ b/activation/atx_service_db_test.go @@ -0,0 +1,104 @@ +package activation + +import ( + "context" + "errors" + "testing" + + "github.com/stretchr/testify/require" + "go.uber.org/mock/gomock" + "go.uber.org/zap/zaptest" + + "github.com/spacemeshos/go-spacemesh/atxsdata" + "github.com/spacemeshos/go-spacemesh/common/types" + "github.com/spacemeshos/go-spacemesh/signing" + "github.com/spacemeshos/go-spacemesh/sql/atxs" + "github.com/spacemeshos/go-spacemesh/sql/statesql" +) + +func newTestDbAtxService(t *testing.T) *dbAtxService { + return NewDBAtxService( + statesql.InMemoryTest(t), + types.RandomATXID(), + atxsdata.New(), + NewMocknipostValidator(gomock.NewController(t)), + zaptest.NewLogger(t), + ) +} + +// Test if PositioningAtx 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) { + atxSvc := newTestDbAtxService(t) + + // Invalid chain with high height + sigInvalid, err := signing.NewEdSigner() + require.NoError(t, err) + invalidAtx := newInitialATXv1(t, atxSvc.golden) + invalidAtx.Sign(sigInvalid) + vInvalidAtx := toAtx(t, invalidAtx) + vInvalidAtx.TickCount = 100 + require.NoError(t, err) + require.NoError(t, atxs.Add(atxSvc.db, vInvalidAtx, invalidAtx.Blob())) + atxSvc.atxsdata.AddFromAtx(vInvalidAtx, false) + + // Valid chain with lower height + sigValid, err := signing.NewEdSigner() + require.NoError(t, err) + validAtx := newInitialATXv1(t, atxSvc.golden) + validAtx.NumUnits += 10 + validAtx.Sign(sigValid) + vValidAtx := toAtx(t, validAtx) + require.NoError(t, atxs.Add(atxSvc.db, vValidAtx, validAtx.Blob())) + atxSvc.atxsdata.AddFromAtx(vValidAtx, false) + + atxSvc.validator.(*MocknipostValidator).EXPECT(). + VerifyChain(gomock.Any(), invalidAtx.ID(), atxSvc.golden, gomock.Any()). + Return(errors.New("this is invalid")) + atxSvc.validator.(*MocknipostValidator).EXPECT(). + VerifyChain(gomock.Any(), validAtx.ID(), atxSvc.golden, gomock.Any()) + + posAtxID, err := atxSvc.PositioningATX(context.Background(), validAtx.PublishEpoch) + require.NoError(t, err) + require.Equal(t, vValidAtx.ID(), posAtxID) + + // look in a later epoch, it should return the same one (there is no newer one). + atxSvc.validator.(*MocknipostValidator).EXPECT(). + VerifyChain(gomock.Any(), invalidAtx.ID(), atxSvc.golden, gomock.Any()). + Return(errors.New("")) + atxSvc.validator.(*MocknipostValidator).EXPECT(). + VerifyChain(gomock.Any(), validAtx.ID(), atxSvc.golden, gomock.Any()) + + posAtxID, err = atxSvc.PositioningATX(context.Background(), validAtx.PublishEpoch+1) + require.NoError(t, err) + require.Equal(t, vValidAtx.ID(), posAtxID) + + // it returns the golden ATX if couldn't find a better one + posAtxID, err = atxSvc.PositioningATX(context.Background(), validAtx.PublishEpoch-1) + require.NoError(t, err) + require.Equal(t, atxSvc.golden, posAtxID) +} + +func TestFindFullyValidHighTickAtx(t *testing.T) { + t.Parallel() + golden := types.RandomATXID() + + t.Run("skips malicious ATXs", func(t *testing.T) { + data := atxsdata.New() + atxMal := &types.ActivationTx{TickCount: 100, SmesherID: types.RandomNodeID()} + atxMal.SetID(types.RandomATXID()) + data.AddFromAtx(atxMal, true) + + atxLower := &types.ActivationTx{TickCount: 10, SmesherID: types.RandomNodeID()} + atxLower.SetID(types.RandomATXID()) + data.AddFromAtx(atxLower, false) + + mValidator := NewMocknipostValidator(gomock.NewController(t)) + mValidator.EXPECT().VerifyChain(gomock.Any(), atxLower.ID(), golden, gomock.Any()) + + lg := zaptest.NewLogger(t) + found, err := findFullyValidHighTickAtx(context.Background(), data, 0, golden, mValidator, lg) + require.NoError(t, err) + require.Equal(t, atxLower.ID(), found) + }) +} diff --git a/activation/builder_v2_test.go b/activation/builder_v2_test.go index 0054147f4e..d7170c12bf 100644 --- a/activation/builder_v2_test.go +++ b/activation/builder_v2_test.go @@ -93,7 +93,6 @@ func TestBuilder_SwitchesToBuildV2(t *testing.T) { posEpoch += 1 layer = posEpoch.FirstLayer() tab.mclock.EXPECT().CurrentLayer().Return(layer).Times(4) - tab.mValidator.EXPECT().VerifyChain(gomock.Any(), atx1.ID(), tab.goldenATXID, gomock.Any()) var atx2 wire.ActivationTxV2 publishAtx(t, tab, sig.NodeID(), posEpoch, &layer, layersPerEpoch, func(_ context.Context, _ string, got []byte) error { diff --git a/activation/e2e/activation_test.go b/activation/e2e/activation_test.go index 1472f4453c..72fe351dce 100644 --- a/activation/e2e/activation_test.go +++ b/activation/e2e/activation_test.go @@ -188,18 +188,26 @@ func Test_BuilderWithMultipleClients(t *testing.T) { ).Times(totalAtxs) t.Cleanup(func() { assert.NoError(t, verifier.Close()) }) - tab := activation.NewBuilder( - conf, + + atxService := activation.NewDBAtxService( db, + conf.GoldenATXID, data, + validator, + logger, + ) + + tab := activation.NewBuilder( + conf, localDB, + atxService, mpub, + validator, nb, clock, syncedSyncer(t), logger, activation.WithPoetConfig(poetCfg), - activation.WithValidator(validator), activation.WithPoets(client), ) for _, sig := range signers { diff --git a/activation/e2e/builds_atx_v2_test.go b/activation/e2e/builds_atx_v2_test.go index af8fad6a55..9a16eecf13 100644 --- a/activation/e2e/builds_atx_v2_test.go +++ b/activation/e2e/builds_atx_v2_test.go @@ -198,18 +198,25 @@ func TestBuilder_SwitchesToBuildV2(t *testing.T) { ).Times(2), ) - tab := activation.NewBuilder( - conf, + atxService := activation.NewDBAtxService( db, + conf.GoldenATXID, atxsdata, + validator, + logger, + ) + + tab := activation.NewBuilder( + conf, localDB, + atxService, mpub, + validator, nb, clock, syncedSyncer(t), logger, activation.WithPoetConfig(poetCfg), - activation.WithValidator(validator), activation.BuilderAtxVersions(atxVersions), ) tab.Register(sig) diff --git a/activation/e2e/checkpoint_test.go b/activation/e2e/checkpoint_test.go index 0e28ee72d9..d3bfad8525 100644 --- a/activation/e2e/checkpoint_test.go +++ b/activation/e2e/checkpoint_test.go @@ -121,18 +121,25 @@ func TestCheckpoint_PublishingSoloATXs(t *testing.T) { activation.WithAtxVersions(atxVersions), ) - tab := activation.NewBuilder( - activation.Config{GoldenATXID: goldenATX}, + atxService := activation.NewDBAtxService( db, + goldenATX, atxdata, + validator, + logger, + ) + + tab := activation.NewBuilder( + activation.Config{GoldenATXID: goldenATX}, localDB, + atxService, mpub, + validator, nb, clock, syncer, logger, activation.WithPoetConfig(poetCfg), - activation.WithValidator(validator), activation.BuilderAtxVersions(atxVersions), ) tab.Register(sig) @@ -219,18 +226,25 @@ func TestCheckpoint_PublishingSoloATXs(t *testing.T) { ) require.NoError(t, err) - tab = activation.NewBuilder( - activation.Config{GoldenATXID: goldenATX}, + atxService = activation.NewDBAtxService( newDB, + goldenATX, atxdata, + validator, + logger, + ) + + tab = activation.NewBuilder( + activation.Config{GoldenATXID: goldenATX}, localDB, + atxService, mpub, + validator, nb, clock, syncer, logger, activation.WithPoetConfig(poetCfg), - activation.WithValidator(validator), activation.BuilderAtxVersions(atxVersions), ) tab.Register(sig) diff --git a/activation/interface.go b/activation/interface.go index 5bd649f5ae..4e642d24e7 100644 --- a/activation/interface.go +++ b/activation/interface.go @@ -108,6 +108,15 @@ type atxProvider interface { GetAtx(id types.ATXID) (*types.ActivationTx, error) } +// AtxService provides ATXs needed by the ATX Builder. +type AtxService interface { + Atx(ctx context.Context, id types.ATXID) (*types.ActivationTx, error) + LastATX(ctx context.Context, nodeID types.NodeID) (*types.ActivationTx, error) + // PositioningATX returns atx id with the highest tick height. + // The maxPublish epoch is the maximum publish epoch of the returned ATX. + PositioningATX(ctx context.Context, maxPublish types.EpochID) (types.ATXID, error) +} + // PostSetupProvider defines the functionality required for Post setup. // This interface is used by the atx builder and currently implemented by the PostSetupManager. // Eventually most of the functionality will be moved to the PoSTClient. diff --git a/activation/mocks.go b/activation/mocks.go index 38a1a47206..bdeee5b2b8 100644 --- a/activation/mocks.go +++ b/activation/mocks.go @@ -1208,6 +1208,146 @@ func (c *MockatxProviderGetAtxCall) DoAndReturn(f func(types.ATXID) (*types.Acti return c } +// MockAtxService is a mock of AtxService interface. +type MockAtxService struct { + ctrl *gomock.Controller + recorder *MockAtxServiceMockRecorder +} + +// MockAtxServiceMockRecorder is the mock recorder for MockAtxService. +type MockAtxServiceMockRecorder struct { + mock *MockAtxService +} + +// NewMockAtxService creates a new mock instance. +func NewMockAtxService(ctrl *gomock.Controller) *MockAtxService { + mock := &MockAtxService{ctrl: ctrl} + mock.recorder = &MockAtxServiceMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockAtxService) EXPECT() *MockAtxServiceMockRecorder { + return m.recorder +} + +// Atx mocks base method. +func (m *MockAtxService) Atx(ctx context.Context, id types.ATXID) (*types.ActivationTx, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Atx", ctx, id) + ret0, _ := ret[0].(*types.ActivationTx) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// Atx indicates an expected call of Atx. +func (mr *MockAtxServiceMockRecorder) Atx(ctx, id any) *MockAtxServiceAtxCall { + mr.mock.ctrl.T.Helper() + call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Atx", reflect.TypeOf((*MockAtxService)(nil).Atx), ctx, id) + return &MockAtxServiceAtxCall{Call: call} +} + +// MockAtxServiceAtxCall wrap *gomock.Call +type MockAtxServiceAtxCall struct { + *gomock.Call +} + +// Return rewrite *gomock.Call.Return +func (c *MockAtxServiceAtxCall) Return(arg0 *types.ActivationTx, arg1 error) *MockAtxServiceAtxCall { + c.Call = c.Call.Return(arg0, arg1) + return c +} + +// Do rewrite *gomock.Call.Do +func (c *MockAtxServiceAtxCall) Do(f func(context.Context, types.ATXID) (*types.ActivationTx, error)) *MockAtxServiceAtxCall { + c.Call = c.Call.Do(f) + return c +} + +// DoAndReturn rewrite *gomock.Call.DoAndReturn +func (c *MockAtxServiceAtxCall) DoAndReturn(f func(context.Context, types.ATXID) (*types.ActivationTx, error)) *MockAtxServiceAtxCall { + c.Call = c.Call.DoAndReturn(f) + return c +} + +// LastATX mocks base method. +func (m *MockAtxService) LastATX(ctx context.Context, nodeID types.NodeID) (*types.ActivationTx, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "LastATX", ctx, nodeID) + ret0, _ := ret[0].(*types.ActivationTx) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// LastATX indicates an expected call of LastATX. +func (mr *MockAtxServiceMockRecorder) LastATX(ctx, nodeID any) *MockAtxServiceLastATXCall { + mr.mock.ctrl.T.Helper() + call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "LastATX", reflect.TypeOf((*MockAtxService)(nil).LastATX), ctx, nodeID) + return &MockAtxServiceLastATXCall{Call: call} +} + +// MockAtxServiceLastATXCall wrap *gomock.Call +type MockAtxServiceLastATXCall struct { + *gomock.Call +} + +// Return rewrite *gomock.Call.Return +func (c *MockAtxServiceLastATXCall) Return(arg0 *types.ActivationTx, arg1 error) *MockAtxServiceLastATXCall { + c.Call = c.Call.Return(arg0, arg1) + return c +} + +// Do rewrite *gomock.Call.Do +func (c *MockAtxServiceLastATXCall) Do(f func(context.Context, types.NodeID) (*types.ActivationTx, error)) *MockAtxServiceLastATXCall { + c.Call = c.Call.Do(f) + return c +} + +// DoAndReturn rewrite *gomock.Call.DoAndReturn +func (c *MockAtxServiceLastATXCall) DoAndReturn(f func(context.Context, types.NodeID) (*types.ActivationTx, error)) *MockAtxServiceLastATXCall { + c.Call = c.Call.DoAndReturn(f) + return c +} + +// PositioningATX mocks base method. +func (m *MockAtxService) PositioningATX(ctx context.Context, maxPublish types.EpochID) (types.ATXID, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "PositioningATX", ctx, maxPublish) + ret0, _ := ret[0].(types.ATXID) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// PositioningATX indicates an expected call of PositioningATX. +func (mr *MockAtxServiceMockRecorder) PositioningATX(ctx, maxPublish any) *MockAtxServicePositioningATXCall { + mr.mock.ctrl.T.Helper() + call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PositioningATX", reflect.TypeOf((*MockAtxService)(nil).PositioningATX), ctx, maxPublish) + return &MockAtxServicePositioningATXCall{Call: call} +} + +// MockAtxServicePositioningATXCall wrap *gomock.Call +type MockAtxServicePositioningATXCall struct { + *gomock.Call +} + +// Return rewrite *gomock.Call.Return +func (c *MockAtxServicePositioningATXCall) Return(arg0 types.ATXID, arg1 error) *MockAtxServicePositioningATXCall { + c.Call = c.Call.Return(arg0, arg1) + return c +} + +// Do rewrite *gomock.Call.Do +func (c *MockAtxServicePositioningATXCall) Do(f func(context.Context, types.EpochID) (types.ATXID, error)) *MockAtxServicePositioningATXCall { + c.Call = c.Call.Do(f) + return c +} + +// DoAndReturn rewrite *gomock.Call.DoAndReturn +func (c *MockAtxServicePositioningATXCall) DoAndReturn(f func(context.Context, types.EpochID) (types.ATXID, error)) *MockAtxServicePositioningATXCall { + c.Call = c.Call.DoAndReturn(f) + return c +} + // MockpostSetupProvider is a mock of postSetupProvider interface. type MockpostSetupProvider struct { ctrl *gomock.Controller diff --git a/common/errors.go b/common/errors.go new file mode 100644 index 0000000000..4342b6fa07 --- /dev/null +++ b/common/errors.go @@ -0,0 +1,5 @@ +package common + +import "errors" + +var ErrNotFound = errors.New("not found") diff --git a/node/node.go b/node/node.go index 70ea73a48f..6013061a5f 100644 --- a/node/node.go +++ b/node/node.go @@ -1058,22 +1058,30 @@ func (app *App) initServices(ctx context.Context) error { GoldenATXID: goldenATXID, RegossipInterval: app.Config.RegossipAtxInterval, } - atxBuilder := activation.NewBuilder( - builderConfig, + + atxBuilderLog := app.addLogger(ATXBuilderLogger, lg).Zap() + atxService := activation.NewDBAtxService( app.db, + goldenATXID, app.atxsdata, + app.validator, + atxBuilderLog, + activation.WithPostValidityDelay(app.Config.PostValidDelay), + ) + atxBuilder := activation.NewBuilder( + builderConfig, app.localDB, + atxService, app.host, + app.validator, nipostBuilder, app.clock, newSyncer, - app.addLogger(ATXBuilderLogger, lg).Zap(), + atxBuilderLog, activation.WithContext(ctx), activation.WithPoetConfig(app.Config.POET), // TODO(dshulyak) makes no sense. how we ended using it? activation.WithPoetRetryInterval(app.Config.HARE3.PreroundDelay), - activation.WithValidator(app.validator), - activation.WithPostValidityDelay(app.Config.PostValidDelay), activation.WithPostStates(postStates), activation.WithPoets(poetClients...), activation.BuilderAtxVersions(app.Config.AtxVersions), diff --git a/sql/database.go b/sql/database.go index a647086f4a..17e9cdd8aa 100644 --- a/sql/database.go +++ b/sql/database.go @@ -18,6 +18,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" + "github.com/spacemeshos/go-spacemesh/common" "github.com/spacemeshos/go-spacemesh/common/types" ) @@ -27,7 +28,7 @@ var ( // ErrNoConnection is returned if pooled connection is not available. ErrNoConnection = errors.New("database: no free connection") // ErrNotFound is returned if requested record is not found. - ErrNotFound = errors.New("database: not found") + ErrNotFound = fmt.Errorf("database: %w", common.ErrNotFound) // ErrObjectExists is returned if database constraints didn't allow to insert an object. ErrObjectExists = errors.New("database: object exists") // ErrTooNew is returned if database version is newer than expected. diff --git a/sql/localsql/atxs/atxs.go b/sql/localsql/atxs/atxs.go new file mode 100644 index 0000000000..6c0d351d5a --- /dev/null +++ b/sql/localsql/atxs/atxs.go @@ -0,0 +1,37 @@ +package atxs + +import ( + "github.com/spacemeshos/go-spacemesh/common/types" + "github.com/spacemeshos/go-spacemesh/sql" +) + +func AddBlob(db sql.LocalDatabase, epoch types.EpochID, id types.ATXID, nodeID types.NodeID, blob []byte) error { + _, err := db.Exec("INSERT INTO atx_blobs (epoch, id, pubkey, atx) VALUES (?1, ?2, ?3, ?4)", + func(s *sql.Statement) { + s.BindInt64(1, int64(epoch)) + s.BindBytes(2, id[:]) + s.BindBytes(3, nodeID[:]) + s.BindBytes(4, blob) + }, nil) + return err +} + +func AtxBlob(db sql.LocalDatabase, epoch types.EpochID, nodeID types.NodeID) (id types.ATXID, blob []byte, err error) { + rows, err := db.Exec("select id, atx from atx_blobs where epoch = ?1 and pubkey = ?2", + func(s *sql.Statement) { + s.BindInt64(1, int64(epoch)) + s.BindBytes(2, nodeID[:]) + }, + func(s *sql.Statement) bool { + s.ColumnBytes(0, id[:]) + blob = make([]byte, s.ColumnLen(1)) + s.ColumnBytes(1, blob) + return false + }, + ) + if rows == 0 { + return id, blob, sql.ErrNotFound + } + + return id, blob, err +} diff --git a/sql/localsql/schema/migrations/0010_atxs.sql b/sql/localsql/schema/migrations/0010_atxs.sql new file mode 100644 index 0000000000..19e9798333 --- /dev/null +++ b/sql/localsql/schema/migrations/0010_atxs.sql @@ -0,0 +1,12 @@ +--- Table for storing blobs of published ATX for regossiping purposes. +CREATE TABLE atx_blobs +( + id CHAR(32), + pubkey CHAR(32) NOT NULL, + epoch INT NOT NULL, + atx BLOB, + version INTEGER +); + +CREATE UNIQUE INDEX atx_blobs_id ON atx_blobs (id); +CREATE UNIQUE INDEX atx_blobs_epoch_pubkey ON atx_blobs (epoch, pubkey); diff --git a/sql/localsql/schema/schema.sql b/sql/localsql/schema/schema.sql index 02c44d3ccb..2c9f4dd1c6 100755 --- a/sql/localsql/schema/schema.sql +++ b/sql/localsql/schema/schema.sql @@ -1,4 +1,14 @@ -PRAGMA user_version = 9; +PRAGMA user_version = 10; +CREATE TABLE atx_blobs +( + id CHAR(32), + pubkey CHAR(32) NOT NULL, + epoch INT NOT NULL, + atx BLOB, + version INTEGER +); +CREATE UNIQUE INDEX atx_blobs_epoch_pubkey ON atx_blobs (epoch, pubkey); +CREATE UNIQUE INDEX atx_blobs_id ON atx_blobs (id); CREATE TABLE atx_sync_requests ( epoch INT NOT NULL, From 946297eb4c45ec7c2ac0c023c4fd566115649078 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bartosz=20R=C3=B3=C5=BCa=C5=84ski?= Date: Thu, 26 Sep 2024 12:04:16 +0200 Subject: [PATCH 02/10] node-service API and basic implementation --- Makefile | 2 + api/grpcserver/config.go | 2 + api/node/client/client.gen.go | 583 +++++++++++++++++++++++++++++ api/node/client/client.go | 110 ++++++ api/node/client/client_e2e_test.go | 128 +++++++ api/node/client/generate.go | 3 + api/node/client/logger.go | 25 ++ api/node/client/oapi_gen_cfg.yaml | 8 + api/node/models/components.yaml | 44 +++ api/node/models/generate.go | 3 + api/node/models/models.gen.go | 24 ++ api/node/models/models.go | 56 +++ api/node/models/models_test.go | 82 ++++ api/node/models/oapi_gen_cfg.yaml | 8 + api/node/node_service.yaml | 103 +++++ api/node/server/generate.go | 3 + api/node/server/oapi_gen_cfg.yaml | 9 + api/node/server/server.gen.go | 544 +++++++++++++++++++++++++++ api/node/server/server.go | 159 ++++++++ config/config.go | 4 + go.mod | 2 + go.sum | 8 + node/node.go | 60 ++- 23 files changed, 1961 insertions(+), 9 deletions(-) create mode 100644 api/node/client/client.gen.go create mode 100644 api/node/client/client.go create mode 100644 api/node/client/client_e2e_test.go create mode 100644 api/node/client/generate.go create mode 100644 api/node/client/logger.go create mode 100644 api/node/client/oapi_gen_cfg.yaml create mode 100644 api/node/models/components.yaml create mode 100644 api/node/models/generate.go create mode 100644 api/node/models/models.gen.go create mode 100644 api/node/models/models.go create mode 100644 api/node/models/models_test.go create mode 100644 api/node/models/oapi_gen_cfg.yaml create mode 100644 api/node/node_service.yaml create mode 100644 api/node/server/generate.go create mode 100644 api/node/server/oapi_gen_cfg.yaml create mode 100644 api/node/server/server.gen.go create mode 100644 api/node/server/server.go diff --git a/Makefile b/Makefile index 4317805262..995636c993 100644 --- a/Makefile +++ b/Makefile @@ -6,6 +6,7 @@ GOLANGCI_LINT_VERSION := v1.59.0 GOTESTSUM_VERSION := v1.12.0 GOSCALE_VERSION := v1.2.0 MOCKGEN_VERSION := v0.4.0 +OAPI_CODEGEN_VERSION := v2.4.0 # Add an indicator to the branch name if dirty and use commithash if running in detached mode ifeq ($(BRANCH),HEAD) @@ -60,6 +61,7 @@ install: go install github.com/spacemeshos/go-scale/scalegen@$(GOSCALE_VERSION) go install go.uber.org/mock/mockgen@$(MOCKGEN_VERSION) go install gotest.tools/gotestsum@$(GOTESTSUM_VERSION) + go install github.com/oapi-codegen/oapi-codegen/v2/cmd/oapi-codegen@$(OAPI_CODEGEN_VERSION) .PHONY: install build: go-spacemesh get-profiler get-postrs-service diff --git a/api/grpcserver/config.go b/api/grpcserver/config.go index 564d9184f4..46c160a0d3 100644 --- a/api/grpcserver/config.go +++ b/api/grpcserver/config.go @@ -22,6 +22,8 @@ type Config struct { JSONListener string `mapstructure:"grpc-json-listener"` JSONCorsAllowedOrigins []string `mapstructure:"grpc-cors-allowed-origins"` + NodeServiceListener string `mapstructure:"node-service-listener"` + SmesherStreamInterval time.Duration `mapstructure:"smesherstreaminterval"` } diff --git a/api/node/client/client.gen.go b/api/node/client/client.gen.go new file mode 100644 index 0000000000..5d4f8d00e5 --- /dev/null +++ b/api/node/client/client.gen.go @@ -0,0 +1,583 @@ +// Package client provides primitives to interact with the openapi HTTP API. +// +// Code generated by github.com/oapi-codegen/oapi-codegen/v2 version v2.4.0 DO NOT EDIT. +package client + +import ( + "context" + "encoding/json" + "fmt" + "io" + "net/http" + "net/url" + "strings" + + "github.com/oapi-codegen/runtime" + externalRef0 "github.com/spacemeshos/go-spacemesh/api/node/models" +) + +// Defines values for PostPublishProtocolParamsProtocol. +const ( + Ax1 PostPublishProtocolParamsProtocol = "ax1" + B1 PostPublishProtocolParamsProtocol = "b1" + Bc1 PostPublishProtocolParamsProtocol = "bc1" + Bf1 PostPublishProtocolParamsProtocol = "bf1" + Bo1 PostPublishProtocolParamsProtocol = "bo1" + Bp1 PostPublishProtocolParamsProtocol = "bp1" + Bw1 PostPublishProtocolParamsProtocol = "bw1" + Mp1 PostPublishProtocolParamsProtocol = "mp1" + Pp1 PostPublishProtocolParamsProtocol = "pp1" + Tx1 PostPublishProtocolParamsProtocol = "tx1" +) + +// PostPublishProtocolParamsProtocol defines parameters for PostPublishProtocol. +type PostPublishProtocolParamsProtocol string + +// RequestEditorFn is the function signature for the RequestEditor callback function +type RequestEditorFn func(ctx context.Context, req *http.Request) error + +// Doer performs HTTP requests. +// +// The standard http.Client implements this interface. +type HttpRequestDoer interface { + Do(req *http.Request) (*http.Response, error) +} + +// Client which conforms to the OpenAPI3 specification for this service. +type Client struct { + // The endpoint of the server conforming to this interface, with scheme, + // https://api.deepmap.com for example. This can contain a path relative + // to the server, such as https://api.deepmap.com/dev-test, and all the + // paths in the swagger spec will be appended to the server. + Server string + + // Doer for performing requests, typically a *http.Client with any + // customized settings, such as certificate chains. + Client HttpRequestDoer + + // A list of callbacks for modifying requests which are generated before sending over + // the network. + RequestEditors []RequestEditorFn +} + +// ClientOption allows setting custom parameters during construction +type ClientOption func(*Client) error + +// Creates a new Client, with reasonable defaults +func NewClient(server string, opts ...ClientOption) (*Client, error) { + // create a client with sane default values + client := Client{ + Server: server, + } + // mutate client and add all optional params + for _, o := range opts { + if err := o(&client); err != nil { + return nil, err + } + } + // ensure the server URL always has a trailing slash + if !strings.HasSuffix(client.Server, "/") { + client.Server += "/" + } + // create httpClient, if not already present + if client.Client == nil { + client.Client = &http.Client{} + } + return &client, nil +} + +// WithHTTPClient allows overriding the default Doer, which is +// automatically created using http.Client. This is useful for tests. +func WithHTTPClient(doer HttpRequestDoer) ClientOption { + return func(c *Client) error { + c.Client = doer + return nil + } +} + +// WithRequestEditorFn allows setting up a callback function, which will be +// called right before sending the request. This can be used to mutate the request. +func WithRequestEditorFn(fn RequestEditorFn) ClientOption { + return func(c *Client) error { + c.RequestEditors = append(c.RequestEditors, fn) + return nil + } +} + +// The interface specification for the client above. +type ClientInterface interface { + // GetActivationAtxAtxId request + GetActivationAtxAtxId(ctx context.Context, atxId externalRef0.ATXID, reqEditors ...RequestEditorFn) (*http.Response, error) + + // GetActivationLastAtxNodeId request + GetActivationLastAtxNodeId(ctx context.Context, nodeId externalRef0.NodeID, reqEditors ...RequestEditorFn) (*http.Response, error) + + // GetActivationPositioningAtxPublishEpoch request + GetActivationPositioningAtxPublishEpoch(ctx context.Context, publishEpoch externalRef0.EpochID, reqEditors ...RequestEditorFn) (*http.Response, error) + + // PostPublishProtocolWithBody request with any body + PostPublishProtocolWithBody(ctx context.Context, protocol PostPublishProtocolParamsProtocol, contentType string, body io.Reader, reqEditors ...RequestEditorFn) (*http.Response, error) +} + +func (c *Client) GetActivationAtxAtxId(ctx context.Context, atxId externalRef0.ATXID, reqEditors ...RequestEditorFn) (*http.Response, error) { + req, err := NewGetActivationAtxAtxIdRequest(c.Server, atxId) + if err != nil { + return nil, err + } + req = req.WithContext(ctx) + if err := c.applyEditors(ctx, req, reqEditors); err != nil { + return nil, err + } + return c.Client.Do(req) +} + +func (c *Client) GetActivationLastAtxNodeId(ctx context.Context, nodeId externalRef0.NodeID, reqEditors ...RequestEditorFn) (*http.Response, error) { + req, err := NewGetActivationLastAtxNodeIdRequest(c.Server, nodeId) + if err != nil { + return nil, err + } + req = req.WithContext(ctx) + if err := c.applyEditors(ctx, req, reqEditors); err != nil { + return nil, err + } + return c.Client.Do(req) +} + +func (c *Client) GetActivationPositioningAtxPublishEpoch(ctx context.Context, publishEpoch externalRef0.EpochID, reqEditors ...RequestEditorFn) (*http.Response, error) { + req, err := NewGetActivationPositioningAtxPublishEpochRequest(c.Server, publishEpoch) + if err != nil { + return nil, err + } + req = req.WithContext(ctx) + if err := c.applyEditors(ctx, req, reqEditors); err != nil { + return nil, err + } + return c.Client.Do(req) +} + +func (c *Client) PostPublishProtocolWithBody(ctx context.Context, protocol PostPublishProtocolParamsProtocol, contentType string, body io.Reader, reqEditors ...RequestEditorFn) (*http.Response, error) { + req, err := NewPostPublishProtocolRequestWithBody(c.Server, protocol, contentType, body) + if err != nil { + return nil, err + } + req = req.WithContext(ctx) + if err := c.applyEditors(ctx, req, reqEditors); err != nil { + return nil, err + } + return c.Client.Do(req) +} + +// NewGetActivationAtxAtxIdRequest generates requests for GetActivationAtxAtxId +func NewGetActivationAtxAtxIdRequest(server string, atxId externalRef0.ATXID) (*http.Request, error) { + var err error + + var pathParam0 string + + pathParam0, err = runtime.StyleParamWithLocation("simple", false, "atx_id", runtime.ParamLocationPath, atxId) + if err != nil { + return nil, err + } + + serverURL, err := url.Parse(server) + if err != nil { + return nil, err + } + + operationPath := fmt.Sprintf("/activation/atx/%s", pathParam0) + if operationPath[0] == '/' { + operationPath = "." + operationPath + } + + queryURL, err := serverURL.Parse(operationPath) + if err != nil { + return nil, err + } + + req, err := http.NewRequest("GET", queryURL.String(), nil) + if err != nil { + return nil, err + } + + return req, nil +} + +// NewGetActivationLastAtxNodeIdRequest generates requests for GetActivationLastAtxNodeId +func NewGetActivationLastAtxNodeIdRequest(server string, nodeId externalRef0.NodeID) (*http.Request, error) { + var err error + + var pathParam0 string + + pathParam0, err = runtime.StyleParamWithLocation("simple", false, "node_id", runtime.ParamLocationPath, nodeId) + if err != nil { + return nil, err + } + + serverURL, err := url.Parse(server) + if err != nil { + return nil, err + } + + operationPath := fmt.Sprintf("/activation/last_atx/%s", pathParam0) + if operationPath[0] == '/' { + operationPath = "." + operationPath + } + + queryURL, err := serverURL.Parse(operationPath) + if err != nil { + return nil, err + } + + req, err := http.NewRequest("GET", queryURL.String(), nil) + if err != nil { + return nil, err + } + + return req, nil +} + +// NewGetActivationPositioningAtxPublishEpochRequest generates requests for GetActivationPositioningAtxPublishEpoch +func NewGetActivationPositioningAtxPublishEpochRequest(server string, publishEpoch externalRef0.EpochID) (*http.Request, error) { + var err error + + var pathParam0 string + + pathParam0, err = runtime.StyleParamWithLocation("simple", false, "publish_epoch", runtime.ParamLocationPath, publishEpoch) + if err != nil { + return nil, err + } + + serverURL, err := url.Parse(server) + if err != nil { + return nil, err + } + + operationPath := fmt.Sprintf("/activation/positioning_atx/%s", pathParam0) + if operationPath[0] == '/' { + operationPath = "." + operationPath + } + + queryURL, err := serverURL.Parse(operationPath) + if err != nil { + return nil, err + } + + req, err := http.NewRequest("GET", queryURL.String(), nil) + if err != nil { + return nil, err + } + + return req, nil +} + +// NewPostPublishProtocolRequestWithBody generates requests for PostPublishProtocol with any type of body +func NewPostPublishProtocolRequestWithBody(server string, protocol PostPublishProtocolParamsProtocol, contentType string, body io.Reader) (*http.Request, error) { + var err error + + var pathParam0 string + + pathParam0, err = runtime.StyleParamWithLocation("simple", false, "protocol", runtime.ParamLocationPath, protocol) + if err != nil { + return nil, err + } + + serverURL, err := url.Parse(server) + if err != nil { + return nil, err + } + + operationPath := fmt.Sprintf("/publish/%s", pathParam0) + if operationPath[0] == '/' { + operationPath = "." + operationPath + } + + queryURL, err := serverURL.Parse(operationPath) + if err != nil { + return nil, err + } + + req, err := http.NewRequest("POST", queryURL.String(), body) + if err != nil { + return nil, err + } + + req.Header.Add("Content-Type", contentType) + + return req, nil +} + +func (c *Client) applyEditors(ctx context.Context, req *http.Request, additionalEditors []RequestEditorFn) error { + for _, r := range c.RequestEditors { + if err := r(ctx, req); err != nil { + return err + } + } + for _, r := range additionalEditors { + if err := r(ctx, req); err != nil { + return err + } + } + return nil +} + +// ClientWithResponses builds on ClientInterface to offer response payloads +type ClientWithResponses struct { + ClientInterface +} + +// NewClientWithResponses creates a new ClientWithResponses, which wraps +// Client with return type handling +func NewClientWithResponses(server string, opts ...ClientOption) (*ClientWithResponses, error) { + client, err := NewClient(server, opts...) + if err != nil { + return nil, err + } + return &ClientWithResponses{client}, nil +} + +// WithBaseURL overrides the baseURL. +func WithBaseURL(baseURL string) ClientOption { + return func(c *Client) error { + newBaseURL, err := url.Parse(baseURL) + if err != nil { + return err + } + c.Server = newBaseURL.String() + return nil + } +} + +// ClientWithResponsesInterface is the interface specification for the client with responses above. +type ClientWithResponsesInterface interface { + // GetActivationAtxAtxIdWithResponse request + GetActivationAtxAtxIdWithResponse(ctx context.Context, atxId externalRef0.ATXID, reqEditors ...RequestEditorFn) (*GetActivationAtxAtxIdResponse, error) + + // GetActivationLastAtxNodeIdWithResponse request + GetActivationLastAtxNodeIdWithResponse(ctx context.Context, nodeId externalRef0.NodeID, reqEditors ...RequestEditorFn) (*GetActivationLastAtxNodeIdResponse, error) + + // GetActivationPositioningAtxPublishEpochWithResponse request + GetActivationPositioningAtxPublishEpochWithResponse(ctx context.Context, publishEpoch externalRef0.EpochID, reqEditors ...RequestEditorFn) (*GetActivationPositioningAtxPublishEpochResponse, error) + + // PostPublishProtocolWithBodyWithResponse request with any body + PostPublishProtocolWithBodyWithResponse(ctx context.Context, protocol PostPublishProtocolParamsProtocol, contentType string, body io.Reader, reqEditors ...RequestEditorFn) (*PostPublishProtocolResponse, error) +} + +type GetActivationAtxAtxIdResponse struct { + Body []byte + HTTPResponse *http.Response + JSON200 *externalRef0.ActivationTx +} + +// Status returns HTTPResponse.Status +func (r GetActivationAtxAtxIdResponse) Status() string { + if r.HTTPResponse != nil { + return r.HTTPResponse.Status + } + return http.StatusText(0) +} + +// StatusCode returns HTTPResponse.StatusCode +func (r GetActivationAtxAtxIdResponse) StatusCode() int { + if r.HTTPResponse != nil { + return r.HTTPResponse.StatusCode + } + return 0 +} + +type GetActivationLastAtxNodeIdResponse struct { + Body []byte + HTTPResponse *http.Response + JSON200 *externalRef0.ActivationTx +} + +// Status returns HTTPResponse.Status +func (r GetActivationLastAtxNodeIdResponse) Status() string { + if r.HTTPResponse != nil { + return r.HTTPResponse.Status + } + return http.StatusText(0) +} + +// StatusCode returns HTTPResponse.StatusCode +func (r GetActivationLastAtxNodeIdResponse) StatusCode() int { + if r.HTTPResponse != nil { + return r.HTTPResponse.StatusCode + } + return 0 +} + +type GetActivationPositioningAtxPublishEpochResponse struct { + Body []byte + HTTPResponse *http.Response + JSON200 *struct { + ID externalRef0.ATXID `json:"ID"` + } +} + +// Status returns HTTPResponse.Status +func (r GetActivationPositioningAtxPublishEpochResponse) Status() string { + if r.HTTPResponse != nil { + return r.HTTPResponse.Status + } + return http.StatusText(0) +} + +// StatusCode returns HTTPResponse.StatusCode +func (r GetActivationPositioningAtxPublishEpochResponse) StatusCode() int { + if r.HTTPResponse != nil { + return r.HTTPResponse.StatusCode + } + return 0 +} + +type PostPublishProtocolResponse struct { + Body []byte + HTTPResponse *http.Response +} + +// Status returns HTTPResponse.Status +func (r PostPublishProtocolResponse) Status() string { + if r.HTTPResponse != nil { + return r.HTTPResponse.Status + } + return http.StatusText(0) +} + +// StatusCode returns HTTPResponse.StatusCode +func (r PostPublishProtocolResponse) StatusCode() int { + if r.HTTPResponse != nil { + return r.HTTPResponse.StatusCode + } + return 0 +} + +// GetActivationAtxAtxIdWithResponse request returning *GetActivationAtxAtxIdResponse +func (c *ClientWithResponses) GetActivationAtxAtxIdWithResponse(ctx context.Context, atxId externalRef0.ATXID, reqEditors ...RequestEditorFn) (*GetActivationAtxAtxIdResponse, error) { + rsp, err := c.GetActivationAtxAtxId(ctx, atxId, reqEditors...) + if err != nil { + return nil, err + } + return ParseGetActivationAtxAtxIdResponse(rsp) +} + +// GetActivationLastAtxNodeIdWithResponse request returning *GetActivationLastAtxNodeIdResponse +func (c *ClientWithResponses) GetActivationLastAtxNodeIdWithResponse(ctx context.Context, nodeId externalRef0.NodeID, reqEditors ...RequestEditorFn) (*GetActivationLastAtxNodeIdResponse, error) { + rsp, err := c.GetActivationLastAtxNodeId(ctx, nodeId, reqEditors...) + if err != nil { + return nil, err + } + return ParseGetActivationLastAtxNodeIdResponse(rsp) +} + +// GetActivationPositioningAtxPublishEpochWithResponse request returning *GetActivationPositioningAtxPublishEpochResponse +func (c *ClientWithResponses) GetActivationPositioningAtxPublishEpochWithResponse(ctx context.Context, publishEpoch externalRef0.EpochID, reqEditors ...RequestEditorFn) (*GetActivationPositioningAtxPublishEpochResponse, error) { + rsp, err := c.GetActivationPositioningAtxPublishEpoch(ctx, publishEpoch, reqEditors...) + if err != nil { + return nil, err + } + return ParseGetActivationPositioningAtxPublishEpochResponse(rsp) +} + +// PostPublishProtocolWithBodyWithResponse request with arbitrary body returning *PostPublishProtocolResponse +func (c *ClientWithResponses) PostPublishProtocolWithBodyWithResponse(ctx context.Context, protocol PostPublishProtocolParamsProtocol, contentType string, body io.Reader, reqEditors ...RequestEditorFn) (*PostPublishProtocolResponse, error) { + rsp, err := c.PostPublishProtocolWithBody(ctx, protocol, contentType, body, reqEditors...) + if err != nil { + return nil, err + } + return ParsePostPublishProtocolResponse(rsp) +} + +// ParseGetActivationAtxAtxIdResponse parses an HTTP response from a GetActivationAtxAtxIdWithResponse call +func ParseGetActivationAtxAtxIdResponse(rsp *http.Response) (*GetActivationAtxAtxIdResponse, error) { + bodyBytes, err := io.ReadAll(rsp.Body) + defer func() { _ = rsp.Body.Close() }() + if err != nil { + return nil, err + } + + response := &GetActivationAtxAtxIdResponse{ + Body: bodyBytes, + HTTPResponse: rsp, + } + + switch { + case strings.Contains(rsp.Header.Get("Content-Type"), "json") && rsp.StatusCode == 200: + var dest externalRef0.ActivationTx + if err := json.Unmarshal(bodyBytes, &dest); err != nil { + return nil, err + } + response.JSON200 = &dest + + } + + return response, nil +} + +// ParseGetActivationLastAtxNodeIdResponse parses an HTTP response from a GetActivationLastAtxNodeIdWithResponse call +func ParseGetActivationLastAtxNodeIdResponse(rsp *http.Response) (*GetActivationLastAtxNodeIdResponse, error) { + bodyBytes, err := io.ReadAll(rsp.Body) + defer func() { _ = rsp.Body.Close() }() + if err != nil { + return nil, err + } + + response := &GetActivationLastAtxNodeIdResponse{ + Body: bodyBytes, + HTTPResponse: rsp, + } + + switch { + case strings.Contains(rsp.Header.Get("Content-Type"), "json") && rsp.StatusCode == 200: + var dest externalRef0.ActivationTx + if err := json.Unmarshal(bodyBytes, &dest); err != nil { + return nil, err + } + response.JSON200 = &dest + + } + + return response, nil +} + +// ParseGetActivationPositioningAtxPublishEpochResponse parses an HTTP response from a GetActivationPositioningAtxPublishEpochWithResponse call +func ParseGetActivationPositioningAtxPublishEpochResponse(rsp *http.Response) (*GetActivationPositioningAtxPublishEpochResponse, error) { + bodyBytes, err := io.ReadAll(rsp.Body) + defer func() { _ = rsp.Body.Close() }() + if err != nil { + return nil, err + } + + response := &GetActivationPositioningAtxPublishEpochResponse{ + Body: bodyBytes, + HTTPResponse: rsp, + } + + switch { + case strings.Contains(rsp.Header.Get("Content-Type"), "json") && rsp.StatusCode == 200: + var dest struct { + ID externalRef0.ATXID `json:"ID"` + } + if err := json.Unmarshal(bodyBytes, &dest); err != nil { + return nil, err + } + response.JSON200 = &dest + + } + + return response, nil +} + +// ParsePostPublishProtocolResponse parses an HTTP response from a PostPublishProtocolWithResponse call +func ParsePostPublishProtocolResponse(rsp *http.Response) (*PostPublishProtocolResponse, error) { + bodyBytes, err := io.ReadAll(rsp.Body) + defer func() { _ = rsp.Body.Close() }() + if err != nil { + return nil, err + } + + response := &PostPublishProtocolResponse{ + Body: bodyBytes, + HTTPResponse: rsp, + } + + return response, nil +} diff --git a/api/node/client/client.go b/api/node/client/client.go new file mode 100644 index 0000000000..354cb189d5 --- /dev/null +++ b/api/node/client/client.go @@ -0,0 +1,110 @@ +package client + +import ( + "bytes" + "context" + "encoding/hex" + "fmt" + "net/http" + "time" + + "github.com/hashicorp/go-retryablehttp" + "go.uber.org/zap" + + "github.com/spacemeshos/go-spacemesh/activation" + "github.com/spacemeshos/go-spacemesh/api/node/models" + "github.com/spacemeshos/go-spacemesh/common" + "github.com/spacemeshos/go-spacemesh/common/types" + "github.com/spacemeshos/go-spacemesh/p2p/pubsub" +) + +type nodeService struct { + client *ClientWithResponses + logger *zap.Logger +} + +var ( + _ activation.AtxService = (*nodeService)(nil) + _ pubsub.Publisher = (*nodeService)(nil) +) + +type Config struct { + RetryWaitMin time.Duration // Minimum time to wait + RetryWaitMax time.Duration // Maximum time to wait + RetryMax int // Maximum number of retries +} + +func NewNodeServiceClient(server string, logger *zap.Logger, cfg *Config) (*nodeService, error) { + retryableClient := retryablehttp.Client{ + Logger: &retryableHttpLogger{logger}, + RetryWaitMin: cfg.RetryWaitMin, + RetryWaitMax: cfg.RetryWaitMax, + RetryMax: cfg.RetryMax, + CheckRetry: retryablehttp.DefaultRetryPolicy, + Backoff: retryablehttp.DefaultBackoff, + } + client, err := NewClientWithResponses(server, WithHTTPClient(retryableClient.StandardClient())) + if err != nil { + return nil, err + } + return &nodeService{ + client: client, + logger: logger, + }, nil +} + +func (s *nodeService) Atx(ctx context.Context, id types.ATXID) (*types.ActivationTx, error) { + resp, err := s.client.GetActivationAtxAtxIdWithResponse(ctx, hex.EncodeToString(id.Bytes())) + if err != nil { + return nil, err + } + switch resp.StatusCode() { + case http.StatusOK: + case http.StatusNotFound: + return nil, common.ErrNotFound + default: + return nil, fmt.Errorf("unexpected status: %s", resp.Status()) + } + return models.ParseATX(resp.JSON200) +} + +func (s *nodeService) LastATX(ctx context.Context, nodeID types.NodeID) (*types.ActivationTx, error) { + resp, err := s.client.GetActivationLastAtxNodeIdWithResponse(ctx, hex.EncodeToString(nodeID.Bytes())) + if err != nil { + return nil, err + } + switch resp.StatusCode() { + case http.StatusOK: + case http.StatusNotFound: + return nil, common.ErrNotFound + default: + return nil, fmt.Errorf("unexpected status: %s", resp.Status()) + } + return models.ParseATX(resp.JSON200) +} + +func (s *nodeService) PositioningATX(ctx context.Context, maxPublish types.EpochID) (types.ATXID, error) { + resp, err := s.client.GetActivationPositioningAtxPublishEpochWithResponse(ctx, maxPublish.Uint32()) + if err != nil { + return types.ATXID{}, err + } + if resp.StatusCode() != http.StatusOK { + return types.ATXID{}, fmt.Errorf("unexpected status: %s", resp.Status()) + } + + return models.ParseATXID(resp.JSON200.ID) +} + +// Publish implements pubsub.Publisher. +func (s *nodeService) Publish(ctx context.Context, proto string, blob []byte) error { + buf := bytes.NewBuffer(blob) + protocol := PostPublishProtocolParamsProtocol(proto) + resp, err := s.client.PostPublishProtocolWithBody(ctx, protocol, "application/octet-stream", buf) + if err != nil { + return err + } + if resp.StatusCode != http.StatusOK { + return fmt.Errorf("unexpected status: %s", resp.Status) + } + return nil +} diff --git a/api/node/client/client_e2e_test.go b/api/node/client/client_e2e_test.go new file mode 100644 index 0000000000..a8bdd31bd7 --- /dev/null +++ b/api/node/client/client_e2e_test.go @@ -0,0 +1,128 @@ +package client_test + +import ( + "context" + "errors" + "net" + "net/http" + "testing" + + "github.com/stretchr/testify/require" + "go.uber.org/mock/gomock" + "go.uber.org/zap/zaptest" + + "github.com/spacemeshos/go-spacemesh/activation" + "github.com/spacemeshos/go-spacemesh/api/node/client" + "github.com/spacemeshos/go-spacemesh/api/node/server" + "github.com/spacemeshos/go-spacemesh/common" + "github.com/spacemeshos/go-spacemesh/common/types" +) + +const retries = 3 + +func setupE2E(t *testing.T) (activation.AtxService, *activation.MockAtxService) { + log := zaptest.NewLogger(t) + actServiceMock := activation.NewMockAtxService(gomock.NewController(t)) + activationServiceServer := server.NewServer(actServiceMock, nil, log.Named("server")) + + listener, err := net.Listen("tcp", "localhost:0") + require.NoError(t, err) + + server := &http.Server{ + Handler: activationServiceServer.IntoHandler(http.NewServeMux()), + } + + go server.Serve(listener) + t.Cleanup(func() { + server.Close() + }) + + cfg := &client.Config{ + RetryMax: retries, + } + svc, err := client.NewNodeServiceClient("http://"+listener.Addr().String(), log.Named("server"), cfg) + require.NoError(t, err) + return svc, actServiceMock +} + +func Test_ActivationService_Atx(t *testing.T) { + svc, mock := setupE2E(t) + + atxid := types.ATXID{1, 2, 3, 4} + + t.Run("not found", func(t *testing.T) { + mock.EXPECT().Atx(gomock.Any(), atxid).Return(nil, common.ErrNotFound) + _, err := svc.Atx(context.Background(), atxid) + require.ErrorIs(t, err, common.ErrNotFound) + }) + + t.Run("found", func(t *testing.T) { + atx := &types.ActivationTx{} + atx.SetID(atxid) + mock.EXPECT().Atx(gomock.Any(), atxid).Return(atx, nil) + gotAtx, err := svc.Atx(context.Background(), atxid) + require.NoError(t, err) + require.Equal(t, atx, gotAtx) + }) + + t.Run("backend errors", func(t *testing.T) { + mock.EXPECT(). + Atx(gomock.Any(), atxid). + Times(retries+1). + Return(nil, errors.New("ops")) + _, err := svc.Atx(context.Background(), atxid) + require.Error(t, err) + }) +} + +func Test_ActivationService_PositioningATX(t *testing.T) { + svc, mock := setupE2E(t) + + t.Run("found", func(t *testing.T) { + posAtx := types.RandomATXID() + mock.EXPECT().PositioningATX(gomock.Any(), types.EpochID(77)).Return(posAtx, nil) + gotAtx, err := svc.PositioningATX(context.Background(), 77) + require.NoError(t, err) + require.Equal(t, posAtx, gotAtx) + }) + + t.Run("backend errors", func(t *testing.T) { + mock.EXPECT(). + PositioningATX(gomock.Any(), types.EpochID(77)). + Times(retries+1). + Return(types.EmptyATXID, errors.New("ops")) + _, err := svc.PositioningATX(context.Background(), 77) + require.Error(t, err) + }) +} + +func Test_ActivationService_LastATX(t *testing.T) { + svc, mock := setupE2E(t) + + atxid := types.ATXID{1, 2, 3, 4} + nodeid := types.NodeID{5, 6, 7, 8} + + t.Run("not found", func(t *testing.T) { + mock.EXPECT().LastATX(gomock.Any(), nodeid).Return(nil, common.ErrNotFound) + _, err := svc.LastATX(context.Background(), nodeid) + require.ErrorIs(t, err, common.ErrNotFound) + }) + + t.Run("found", func(t *testing.T) { + atx := &types.ActivationTx{} + atx.SetID(atxid) + mock.EXPECT().LastATX(gomock.Any(), nodeid).Return(atx, nil) + gotAtx, err := svc.LastATX(context.Background(), nodeid) + require.NoError(t, err) + require.Equal(t, atx, gotAtx) + }) + + t.Run("backend errors", func(t *testing.T) { + mock.EXPECT(). + LastATX(gomock.Any(), nodeid). + Times(retries+1). + Return(nil, errors.New("ops")) + _, err := svc.LastATX(context.Background(), nodeid) + require.Error(t, err) + }) +} diff --git a/api/node/client/generate.go b/api/node/client/generate.go new file mode 100644 index 0000000000..4ab6eeba18 --- /dev/null +++ b/api/node/client/generate.go @@ -0,0 +1,3 @@ +package client + +//go:generate oapi-codegen -config oapi_gen_cfg.yaml ../node_service.yaml diff --git a/api/node/client/logger.go b/api/node/client/logger.go new file mode 100644 index 0000000000..a011ccb9e2 --- /dev/null +++ b/api/node/client/logger.go @@ -0,0 +1,25 @@ +package client + +import "go.uber.org/zap" + +// A wrapper around zap.Logger to make it compatible with +// retryablehttp.LeveledLogger interface. +type retryableHttpLogger struct { + inner *zap.Logger +} + +func (r retryableHttpLogger) Error(format string, args ...any) { + r.inner.Sugar().Errorw(format, args...) +} + +func (r retryableHttpLogger) Info(format string, args ...any) { + r.inner.Sugar().Infow(format, args...) +} + +func (r retryableHttpLogger) Warn(format string, args ...any) { + r.inner.Sugar().Warnw(format, args...) +} + +func (r retryableHttpLogger) Debug(format string, args ...any) { + r.inner.Sugar().Debugw(format, args...) +} diff --git a/api/node/client/oapi_gen_cfg.yaml b/api/node/client/oapi_gen_cfg.yaml new file mode 100644 index 0000000000..0610799729 --- /dev/null +++ b/api/node/client/oapi_gen_cfg.yaml @@ -0,0 +1,8 @@ +# yaml-language-server: $schema=https://raw.githubusercontent.com/oapi-codegen/oapi-codegen/HEAD/configuration-schema.json +package: client +output: client.gen.go +generate: + models: true + client: true +import-mapping: + models/components.yaml: github.com/spacemeshos/go-spacemesh/api/node/models diff --git a/api/node/models/components.yaml b/api/node/models/components.yaml new file mode 100644 index 0000000000..4e445f0e57 --- /dev/null +++ b/api/node/models/components.yaml @@ -0,0 +1,44 @@ +components: + schemas: + ActivationTx: + type: object + properties: + ID: + type: string + SmesherID: + type: string + PublishEpoch: + type: integer + format: uint32 + NumUnits: + type: integer + format: uint32 + TickCount: + type: integer + format: uint64 + Weight: + type: integer + format: uint64 + Sequence: + type: integer + format: uint64 + required: + - PublishEpoch + - NumUnits + - TickCount + - SmesherID + - ID + - Weight + NodeID: + type: string + format: hex + minLength: 64 + maxLength: 64 + ATXID: + type: string + format: hex + minLength: 64 + maxLength: 64 + EpochID: + type: integer + format: uint32 diff --git a/api/node/models/generate.go b/api/node/models/generate.go new file mode 100644 index 0000000000..6287682bd4 --- /dev/null +++ b/api/node/models/generate.go @@ -0,0 +1,3 @@ +package models + +//go:generate oapi-codegen -config oapi_gen_cfg.yaml components.yaml diff --git a/api/node/models/models.gen.go b/api/node/models/models.gen.go new file mode 100644 index 0000000000..ccae04b0c8 --- /dev/null +++ b/api/node/models/models.gen.go @@ -0,0 +1,24 @@ +// Package models provides primitives to interact with the openapi HTTP API. +// +// Code generated by github.com/oapi-codegen/oapi-codegen/v2 version v2.4.0 DO NOT EDIT. +package models + +// ATXID defines model for ATXID. +type ATXID = string + +// ActivationTx defines model for ActivationTx. +type ActivationTx struct { + ID string `json:"ID"` + NumUnits uint32 `json:"NumUnits"` + PublishEpoch uint32 `json:"PublishEpoch"` + Sequence *uint64 `json:"Sequence,omitempty"` + SmesherID string `json:"SmesherID"` + TickCount uint64 `json:"TickCount"` + Weight uint64 `json:"Weight"` +} + +// EpochID defines model for EpochID. +type EpochID = uint32 + +// NodeID defines model for NodeID. +type NodeID = string diff --git a/api/node/models/models.go b/api/node/models/models.go new file mode 100644 index 0000000000..a5033e3abf --- /dev/null +++ b/api/node/models/models.go @@ -0,0 +1,56 @@ +package models + +import ( + "encoding/hex" + "fmt" + + "github.com/spacemeshos/go-spacemesh/common/types" +) + +func ParseNodeID(hexID NodeID) (types.NodeID, error) { + if len(hexID) != 2*len(types.NodeID{}) { + return types.NodeID{}, fmt.Errorf("invalid node ID length: %d", len(hexID)) + } + id, err := hex.DecodeString(hexID) + if err != nil { + return types.NodeID{}, fmt.Errorf("decoding node ID (%s): %w", hexID, err) + } + return types.BytesToNodeID(id), nil +} + +func ParseATXID(hexID ATXID) (types.ATXID, error) { + if len(hexID) != 2*len(types.ATXID{}) { + return types.ATXID{}, fmt.Errorf("invalid atx ID length: %d", len(hexID)) + } + id, err := hex.DecodeString(hexID) + if err != nil { + return types.ATXID{}, fmt.Errorf("decoding atx ID (%s): %w", hexID, err) + } + return types.BytesToATXID(id), nil +} + +func ParseATX(atx *ActivationTx) (*types.ActivationTx, error) { + smesherID, err := ParseNodeID(atx.SmesherID) + if err != nil { + return nil, err + } + + atxID, err := ParseATXID(atx.ID) + if err != nil { + return nil, err + } + + result := &types.ActivationTx{ + NumUnits: atx.NumUnits, + PublishEpoch: types.EpochID(atx.PublishEpoch), + SmesherID: smesherID, + TickCount: atx.TickCount, + Weight: atx.Weight, + } + if atx.Sequence != nil { + result.Sequence = *atx.Sequence + } + result.SetID(atxID) + + return result, nil +} diff --git a/api/node/models/models_test.go b/api/node/models/models_test.go new file mode 100644 index 0000000000..e85b17c7ba --- /dev/null +++ b/api/node/models/models_test.go @@ -0,0 +1,82 @@ +package models + +import ( + "encoding/hex" + "testing" + + "github.com/stretchr/testify/require" + + "github.com/spacemeshos/go-spacemesh/common/types" +) + +func Test_parseAtx(t *testing.T) { + atxid := types.ATXID{1, 2, 3, 4, 5} + nodeID := types.NodeID{6, 7, 8, 9, 0} + + validAtx := ActivationTx{ + ID: hex.EncodeToString(atxid[:]), + NumUnits: 8, + PublishEpoch: 9, + SmesherID: hex.EncodeToString(nodeID[:]), + TickCount: 5, + Weight: 4, + } + t.Run("ok", func(t *testing.T) { + sequence := uint64(7) + atx := validAtx + atx.Sequence = &sequence + + result, err := ParseATX(&atx) + require.NoError(t, err) + + expected := types.ActivationTx{ + PublishEpoch: types.EpochID(atx.PublishEpoch), + Sequence: *atx.Sequence, + NumUnits: atx.NumUnits, + TickCount: atx.TickCount, + SmesherID: nodeID, + Weight: atx.Weight, + } + expected.SetID(atxid) + require.Equal(t, &expected, result) + }) + t.Run("ok, empty sequence", func(t *testing.T) { + result, err := ParseATX(&validAtx) + require.NoError(t, err) + + expected := types.ActivationTx{ + PublishEpoch: types.EpochID(validAtx.PublishEpoch), + Sequence: 0, + NumUnits: validAtx.NumUnits, + TickCount: validAtx.TickCount, + SmesherID: nodeID, + Weight: validAtx.Weight, + } + expected.SetID(atxid) + require.Equal(t, &expected, result) + }) + t.Run("invalid nodeID length", func(t *testing.T) { + atx := validAtx + atx.SmesherID = "CAFE" + _, err := ParseATX(&atx) + require.Error(t, err) + }) + t.Run("invalid nodeID format", func(t *testing.T) { + atx := validAtx + atx.SmesherID = "Z234567890123456789012345678901234567890123456789012345678901234" + _, err := ParseATX(&atx) + require.Error(t, err) + }) + t.Run("invalid atx ID length", func(t *testing.T) { + atx := validAtx + atx.ID = "CAFE" + _, err := ParseATX(&atx) + require.Error(t, err) + }) + t.Run("invalid atx ID format", func(t *testing.T) { + atx := validAtx + atx.ID = "Z234567890123456789012345678901234567890123456789012345678901234" + _, err := ParseATX(&atx) + require.Error(t, err) + }) +} diff --git a/api/node/models/oapi_gen_cfg.yaml b/api/node/models/oapi_gen_cfg.yaml new file mode 100644 index 0000000000..d5005f5db3 --- /dev/null +++ b/api/node/models/oapi_gen_cfg.yaml @@ -0,0 +1,8 @@ +# yaml-language-server: $schema=https://raw.githubusercontent.com/oapi-codegen/oapi-codegen/HEAD/configuration-schema.json +package: models +output: models.gen.go +generate: + models: true +output-options: + # to make sure that all types are generated + skip-prune: true diff --git a/api/node/node_service.yaml b/api/node/node_service.yaml new file mode 100644 index 0000000000..fba19088a3 --- /dev/null +++ b/api/node/node_service.yaml @@ -0,0 +1,103 @@ +openapi: 3.0.0 +info: + title: Node Service API + description: API of a Node Service + version: 1.0.0 + +paths: + /activation/positioning_atx/{publish_epoch}: + get: + summary: Get Positioning ATX ID with given maximum publish epoch + tags: + - "Activation" + parameters: + - in: path + name: publish_epoch + required: true + schema: + $ref: "models/components.yaml#/components/schemas/EpochID" + + responses: + "200": + description: ATX ID + content: + application/json: + schema: + type: object + properties: + ID: + $ref: "models/components.yaml#/components/schemas/ATXID" + required: + - ID + + /activation/last_atx/{node_id}: + get: + summary: Get last ATX by node ID + tags: + - "Activation" + parameters: + - in: path + name: node_id + required: true + schema: + $ref: "models/components.yaml#/components/schemas/NodeID" + responses: + "200": + description: ATX + content: + application/json: + schema: + $ref: "models/components.yaml#/components/schemas/ActivationTx" + "400": + description: Bad request + content: + plain/text: + schema: + type: string + + "404": + description: There is no ATX for this node + + /activation/atx/{atx_id}: + get: + summary: Get ATX by ID + tags: + - "Activation" + parameters: + - in: path + name: atx_id + required: true + schema: + $ref: "models/components.yaml#/components/schemas/ATXID" + responses: + "200": + description: ATX + content: + application/json: + schema: + $ref: "models/components.yaml#/components/schemas/ActivationTx" + "404": + description: ATX not found + + /publish/{protocol}: + post: + summary: Publish a blob in the given p2p protocol + tags: + - "Publish" + parameters: + - in: path + name: protocol + required: true + schema: + type: string + enum: [ax1, pp1, tx1, bc1, b1, bw1, bp1, bf1, bo1, mp1] + requestBody: + required: true + content: + application/octet-stream: + schema: + type: string + format: binary + responses: + "200": + description: Successfully published the blob diff --git a/api/node/server/generate.go b/api/node/server/generate.go new file mode 100644 index 0000000000..9d9e2d5300 --- /dev/null +++ b/api/node/server/generate.go @@ -0,0 +1,3 @@ +package server + +//go:generate oapi-codegen -config oapi_gen_cfg.yaml ../node_service.yaml diff --git a/api/node/server/oapi_gen_cfg.yaml b/api/node/server/oapi_gen_cfg.yaml new file mode 100644 index 0000000000..8a7f331cc5 --- /dev/null +++ b/api/node/server/oapi_gen_cfg.yaml @@ -0,0 +1,9 @@ +# yaml-language-server: $schema=https://raw.githubusercontent.com/oapi-codegen/oapi-codegen/HEAD/configuration-schema.json +package: server +generate: + std-http-server: true + models: true + strict-server: true +output: server.gen.go +import-mapping: + models/components.yaml: github.com/spacemeshos/go-spacemesh/api/node/models diff --git a/api/node/server/server.gen.go b/api/node/server/server.gen.go new file mode 100644 index 0000000000..95c4c2b2c3 --- /dev/null +++ b/api/node/server/server.gen.go @@ -0,0 +1,544 @@ +//go:build go1.22 + +// Package server provides primitives to interact with the openapi HTTP API. +// +// Code generated by github.com/oapi-codegen/oapi-codegen/v2 version v2.4.0 DO NOT EDIT. +package server + +import ( + "context" + "encoding/json" + "fmt" + "io" + "net/http" + + "github.com/oapi-codegen/runtime" + strictnethttp "github.com/oapi-codegen/runtime/strictmiddleware/nethttp" + externalRef0 "github.com/spacemeshos/go-spacemesh/api/node/models" +) + +// Defines values for PostPublishProtocolParamsProtocol. +const ( + Ax1 PostPublishProtocolParamsProtocol = "ax1" + B1 PostPublishProtocolParamsProtocol = "b1" + Bc1 PostPublishProtocolParamsProtocol = "bc1" + Bf1 PostPublishProtocolParamsProtocol = "bf1" + Bo1 PostPublishProtocolParamsProtocol = "bo1" + Bp1 PostPublishProtocolParamsProtocol = "bp1" + Bw1 PostPublishProtocolParamsProtocol = "bw1" + Mp1 PostPublishProtocolParamsProtocol = "mp1" + Pp1 PostPublishProtocolParamsProtocol = "pp1" + Tx1 PostPublishProtocolParamsProtocol = "tx1" +) + +// PostPublishProtocolParamsProtocol defines parameters for PostPublishProtocol. +type PostPublishProtocolParamsProtocol string + +// ServerInterface represents all server handlers. +type ServerInterface interface { + // Get ATX by ID + // (GET /activation/atx/{atx_id}) + GetActivationAtxAtxId(w http.ResponseWriter, r *http.Request, atxId externalRef0.ATXID) + // Get last ATX by node ID + // (GET /activation/last_atx/{node_id}) + GetActivationLastAtxNodeId(w http.ResponseWriter, r *http.Request, nodeId externalRef0.NodeID) + // Get Positioning ATX ID with given maximum publish epoch + // (GET /activation/positioning_atx/{publish_epoch}) + GetActivationPositioningAtxPublishEpoch(w http.ResponseWriter, r *http.Request, publishEpoch externalRef0.EpochID) + // Publish a blob in the given p2p protocol + // (POST /publish/{protocol}) + PostPublishProtocol(w http.ResponseWriter, r *http.Request, protocol PostPublishProtocolParamsProtocol) +} + +// ServerInterfaceWrapper converts contexts to parameters. +type ServerInterfaceWrapper struct { + Handler ServerInterface + HandlerMiddlewares []MiddlewareFunc + ErrorHandlerFunc func(w http.ResponseWriter, r *http.Request, err error) +} + +type MiddlewareFunc func(http.Handler) http.Handler + +// GetActivationAtxAtxId operation middleware +func (siw *ServerInterfaceWrapper) GetActivationAtxAtxId(w http.ResponseWriter, r *http.Request) { + + var err error + + // ------------- Path parameter "atx_id" ------------- + var atxId externalRef0.ATXID + + err = runtime.BindStyledParameterWithOptions("simple", "atx_id", r.PathValue("atx_id"), &atxId, runtime.BindStyledParameterOptions{ParamLocation: runtime.ParamLocationPath, Explode: false, Required: true}) + if err != nil { + siw.ErrorHandlerFunc(w, r, &InvalidParamFormatError{ParamName: "atx_id", Err: err}) + return + } + + handler := http.Handler(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + siw.Handler.GetActivationAtxAtxId(w, r, atxId) + })) + + for _, middleware := range siw.HandlerMiddlewares { + handler = middleware(handler) + } + + handler.ServeHTTP(w, r) +} + +// GetActivationLastAtxNodeId operation middleware +func (siw *ServerInterfaceWrapper) GetActivationLastAtxNodeId(w http.ResponseWriter, r *http.Request) { + + var err error + + // ------------- Path parameter "node_id" ------------- + var nodeId externalRef0.NodeID + + err = runtime.BindStyledParameterWithOptions("simple", "node_id", r.PathValue("node_id"), &nodeId, runtime.BindStyledParameterOptions{ParamLocation: runtime.ParamLocationPath, Explode: false, Required: true}) + if err != nil { + siw.ErrorHandlerFunc(w, r, &InvalidParamFormatError{ParamName: "node_id", Err: err}) + return + } + + handler := http.Handler(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + siw.Handler.GetActivationLastAtxNodeId(w, r, nodeId) + })) + + for _, middleware := range siw.HandlerMiddlewares { + handler = middleware(handler) + } + + handler.ServeHTTP(w, r) +} + +// GetActivationPositioningAtxPublishEpoch operation middleware +func (siw *ServerInterfaceWrapper) GetActivationPositioningAtxPublishEpoch(w http.ResponseWriter, r *http.Request) { + + var err error + + // ------------- Path parameter "publish_epoch" ------------- + var publishEpoch externalRef0.EpochID + + err = runtime.BindStyledParameterWithOptions("simple", "publish_epoch", r.PathValue("publish_epoch"), &publishEpoch, runtime.BindStyledParameterOptions{ParamLocation: runtime.ParamLocationPath, Explode: false, Required: true}) + if err != nil { + siw.ErrorHandlerFunc(w, r, &InvalidParamFormatError{ParamName: "publish_epoch", Err: err}) + return + } + + handler := http.Handler(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + siw.Handler.GetActivationPositioningAtxPublishEpoch(w, r, publishEpoch) + })) + + for _, middleware := range siw.HandlerMiddlewares { + handler = middleware(handler) + } + + handler.ServeHTTP(w, r) +} + +// PostPublishProtocol operation middleware +func (siw *ServerInterfaceWrapper) PostPublishProtocol(w http.ResponseWriter, r *http.Request) { + + var err error + + // ------------- Path parameter "protocol" ------------- + var protocol PostPublishProtocolParamsProtocol + + err = runtime.BindStyledParameterWithOptions("simple", "protocol", r.PathValue("protocol"), &protocol, runtime.BindStyledParameterOptions{ParamLocation: runtime.ParamLocationPath, Explode: false, Required: true}) + if err != nil { + siw.ErrorHandlerFunc(w, r, &InvalidParamFormatError{ParamName: "protocol", Err: err}) + return + } + + handler := http.Handler(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + siw.Handler.PostPublishProtocol(w, r, protocol) + })) + + for _, middleware := range siw.HandlerMiddlewares { + handler = middleware(handler) + } + + handler.ServeHTTP(w, r) +} + +type UnescapedCookieParamError struct { + ParamName string + Err error +} + +func (e *UnescapedCookieParamError) Error() string { + return fmt.Sprintf("error unescaping cookie parameter '%s'", e.ParamName) +} + +func (e *UnescapedCookieParamError) Unwrap() error { + return e.Err +} + +type UnmarshalingParamError struct { + ParamName string + Err error +} + +func (e *UnmarshalingParamError) Error() string { + return fmt.Sprintf("Error unmarshaling parameter %s as JSON: %s", e.ParamName, e.Err.Error()) +} + +func (e *UnmarshalingParamError) Unwrap() error { + return e.Err +} + +type RequiredParamError struct { + ParamName string +} + +func (e *RequiredParamError) Error() string { + return fmt.Sprintf("Query argument %s is required, but not found", e.ParamName) +} + +type RequiredHeaderError struct { + ParamName string + Err error +} + +func (e *RequiredHeaderError) Error() string { + return fmt.Sprintf("Header parameter %s is required, but not found", e.ParamName) +} + +func (e *RequiredHeaderError) Unwrap() error { + return e.Err +} + +type InvalidParamFormatError struct { + ParamName string + Err error +} + +func (e *InvalidParamFormatError) Error() string { + return fmt.Sprintf("Invalid format for parameter %s: %s", e.ParamName, e.Err.Error()) +} + +func (e *InvalidParamFormatError) Unwrap() error { + return e.Err +} + +type TooManyValuesForParamError struct { + ParamName string + Count int +} + +func (e *TooManyValuesForParamError) Error() string { + return fmt.Sprintf("Expected one value for %s, got %d", e.ParamName, e.Count) +} + +// Handler creates http.Handler with routing matching OpenAPI spec. +func Handler(si ServerInterface) http.Handler { + return HandlerWithOptions(si, StdHTTPServerOptions{}) +} + +// ServeMux is an abstraction of http.ServeMux. +type ServeMux interface { + HandleFunc(pattern string, handler func(http.ResponseWriter, *http.Request)) + ServeHTTP(w http.ResponseWriter, r *http.Request) +} + +type StdHTTPServerOptions struct { + BaseURL string + BaseRouter ServeMux + Middlewares []MiddlewareFunc + ErrorHandlerFunc func(w http.ResponseWriter, r *http.Request, err error) +} + +// HandlerFromMux creates http.Handler with routing matching OpenAPI spec based on the provided mux. +func HandlerFromMux(si ServerInterface, m ServeMux) http.Handler { + return HandlerWithOptions(si, StdHTTPServerOptions{ + BaseRouter: m, + }) +} + +func HandlerFromMuxWithBaseURL(si ServerInterface, m ServeMux, baseURL string) http.Handler { + return HandlerWithOptions(si, StdHTTPServerOptions{ + BaseURL: baseURL, + BaseRouter: m, + }) +} + +// HandlerWithOptions creates http.Handler with additional options +func HandlerWithOptions(si ServerInterface, options StdHTTPServerOptions) http.Handler { + m := options.BaseRouter + + if m == nil { + m = http.NewServeMux() + } + if options.ErrorHandlerFunc == nil { + options.ErrorHandlerFunc = func(w http.ResponseWriter, r *http.Request, err error) { + http.Error(w, err.Error(), http.StatusBadRequest) + } + } + + wrapper := ServerInterfaceWrapper{ + Handler: si, + HandlerMiddlewares: options.Middlewares, + ErrorHandlerFunc: options.ErrorHandlerFunc, + } + + m.HandleFunc("GET "+options.BaseURL+"/activation/atx/{atx_id}", wrapper.GetActivationAtxAtxId) + m.HandleFunc("GET "+options.BaseURL+"/activation/last_atx/{node_id}", wrapper.GetActivationLastAtxNodeId) + m.HandleFunc("GET "+options.BaseURL+"/activation/positioning_atx/{publish_epoch}", wrapper.GetActivationPositioningAtxPublishEpoch) + m.HandleFunc("POST "+options.BaseURL+"/publish/{protocol}", wrapper.PostPublishProtocol) + + return m +} + +type GetActivationAtxAtxIdRequestObject struct { + AtxId externalRef0.ATXID `json:"atx_id"` +} + +type GetActivationAtxAtxIdResponseObject interface { + VisitGetActivationAtxAtxIdResponse(w http.ResponseWriter) error +} + +type GetActivationAtxAtxId200JSONResponse externalRef0.ActivationTx + +func (response GetActivationAtxAtxId200JSONResponse) VisitGetActivationAtxAtxIdResponse(w http.ResponseWriter) error { + w.Header().Set("Content-Type", "application/json") + w.WriteHeader(200) + + return json.NewEncoder(w).Encode(response) +} + +type GetActivationAtxAtxId404Response struct { +} + +func (response GetActivationAtxAtxId404Response) VisitGetActivationAtxAtxIdResponse(w http.ResponseWriter) error { + w.WriteHeader(404) + return nil +} + +type GetActivationLastAtxNodeIdRequestObject struct { + NodeId externalRef0.NodeID `json:"node_id"` +} + +type GetActivationLastAtxNodeIdResponseObject interface { + VisitGetActivationLastAtxNodeIdResponse(w http.ResponseWriter) error +} + +type GetActivationLastAtxNodeId200JSONResponse externalRef0.ActivationTx + +func (response GetActivationLastAtxNodeId200JSONResponse) VisitGetActivationLastAtxNodeIdResponse(w http.ResponseWriter) error { + w.Header().Set("Content-Type", "application/json") + w.WriteHeader(200) + + return json.NewEncoder(w).Encode(response) +} + +type GetActivationLastAtxNodeId400PlaintextResponse struct { + Body io.Reader + ContentLength int64 +} + +func (response GetActivationLastAtxNodeId400PlaintextResponse) VisitGetActivationLastAtxNodeIdResponse(w http.ResponseWriter) error { + w.Header().Set("Content-Type", "plain/text") + if response.ContentLength != 0 { + w.Header().Set("Content-Length", fmt.Sprint(response.ContentLength)) + } + w.WriteHeader(400) + + if closer, ok := response.Body.(io.ReadCloser); ok { + defer closer.Close() + } + _, err := io.Copy(w, response.Body) + return err +} + +type GetActivationLastAtxNodeId404Response struct { +} + +func (response GetActivationLastAtxNodeId404Response) VisitGetActivationLastAtxNodeIdResponse(w http.ResponseWriter) error { + w.WriteHeader(404) + return nil +} + +type GetActivationPositioningAtxPublishEpochRequestObject struct { + PublishEpoch externalRef0.EpochID `json:"publish_epoch"` +} + +type GetActivationPositioningAtxPublishEpochResponseObject interface { + VisitGetActivationPositioningAtxPublishEpochResponse(w http.ResponseWriter) error +} + +type GetActivationPositioningAtxPublishEpoch200JSONResponse struct { + ID externalRef0.ATXID `json:"ID"` +} + +func (response GetActivationPositioningAtxPublishEpoch200JSONResponse) VisitGetActivationPositioningAtxPublishEpochResponse(w http.ResponseWriter) error { + w.Header().Set("Content-Type", "application/json") + w.WriteHeader(200) + + return json.NewEncoder(w).Encode(response) +} + +type PostPublishProtocolRequestObject struct { + Protocol PostPublishProtocolParamsProtocol `json:"protocol"` + Body io.Reader +} + +type PostPublishProtocolResponseObject interface { + VisitPostPublishProtocolResponse(w http.ResponseWriter) error +} + +type PostPublishProtocol200Response struct { +} + +func (response PostPublishProtocol200Response) VisitPostPublishProtocolResponse(w http.ResponseWriter) error { + w.WriteHeader(200) + return nil +} + +// StrictServerInterface represents all server handlers. +type StrictServerInterface interface { + // Get ATX by ID + // (GET /activation/atx/{atx_id}) + GetActivationAtxAtxId(ctx context.Context, request GetActivationAtxAtxIdRequestObject) (GetActivationAtxAtxIdResponseObject, error) + // Get last ATX by node ID + // (GET /activation/last_atx/{node_id}) + GetActivationLastAtxNodeId(ctx context.Context, request GetActivationLastAtxNodeIdRequestObject) (GetActivationLastAtxNodeIdResponseObject, error) + // Get Positioning ATX ID with given maximum publish epoch + // (GET /activation/positioning_atx/{publish_epoch}) + GetActivationPositioningAtxPublishEpoch(ctx context.Context, request GetActivationPositioningAtxPublishEpochRequestObject) (GetActivationPositioningAtxPublishEpochResponseObject, error) + // Publish a blob in the given p2p protocol + // (POST /publish/{protocol}) + PostPublishProtocol(ctx context.Context, request PostPublishProtocolRequestObject) (PostPublishProtocolResponseObject, error) +} + +type StrictHandlerFunc = strictnethttp.StrictHTTPHandlerFunc +type StrictMiddlewareFunc = strictnethttp.StrictHTTPMiddlewareFunc + +type StrictHTTPServerOptions struct { + RequestErrorHandlerFunc func(w http.ResponseWriter, r *http.Request, err error) + ResponseErrorHandlerFunc func(w http.ResponseWriter, r *http.Request, err error) +} + +func NewStrictHandler(ssi StrictServerInterface, middlewares []StrictMiddlewareFunc) ServerInterface { + return &strictHandler{ssi: ssi, middlewares: middlewares, options: StrictHTTPServerOptions{ + RequestErrorHandlerFunc: func(w http.ResponseWriter, r *http.Request, err error) { + http.Error(w, err.Error(), http.StatusBadRequest) + }, + ResponseErrorHandlerFunc: func(w http.ResponseWriter, r *http.Request, err error) { + http.Error(w, err.Error(), http.StatusInternalServerError) + }, + }} +} + +func NewStrictHandlerWithOptions(ssi StrictServerInterface, middlewares []StrictMiddlewareFunc, options StrictHTTPServerOptions) ServerInterface { + return &strictHandler{ssi: ssi, middlewares: middlewares, options: options} +} + +type strictHandler struct { + ssi StrictServerInterface + middlewares []StrictMiddlewareFunc + options StrictHTTPServerOptions +} + +// GetActivationAtxAtxId operation middleware +func (sh *strictHandler) GetActivationAtxAtxId(w http.ResponseWriter, r *http.Request, atxId externalRef0.ATXID) { + var request GetActivationAtxAtxIdRequestObject + + request.AtxId = atxId + + handler := func(ctx context.Context, w http.ResponseWriter, r *http.Request, request interface{}) (interface{}, error) { + return sh.ssi.GetActivationAtxAtxId(ctx, request.(GetActivationAtxAtxIdRequestObject)) + } + for _, middleware := range sh.middlewares { + handler = middleware(handler, "GetActivationAtxAtxId") + } + + response, err := handler(r.Context(), w, r, request) + + if err != nil { + sh.options.ResponseErrorHandlerFunc(w, r, err) + } else if validResponse, ok := response.(GetActivationAtxAtxIdResponseObject); ok { + if err := validResponse.VisitGetActivationAtxAtxIdResponse(w); err != nil { + sh.options.ResponseErrorHandlerFunc(w, r, err) + } + } else if response != nil { + sh.options.ResponseErrorHandlerFunc(w, r, fmt.Errorf("unexpected response type: %T", response)) + } +} + +// GetActivationLastAtxNodeId operation middleware +func (sh *strictHandler) GetActivationLastAtxNodeId(w http.ResponseWriter, r *http.Request, nodeId externalRef0.NodeID) { + var request GetActivationLastAtxNodeIdRequestObject + + request.NodeId = nodeId + + handler := func(ctx context.Context, w http.ResponseWriter, r *http.Request, request interface{}) (interface{}, error) { + return sh.ssi.GetActivationLastAtxNodeId(ctx, request.(GetActivationLastAtxNodeIdRequestObject)) + } + for _, middleware := range sh.middlewares { + handler = middleware(handler, "GetActivationLastAtxNodeId") + } + + response, err := handler(r.Context(), w, r, request) + + if err != nil { + sh.options.ResponseErrorHandlerFunc(w, r, err) + } else if validResponse, ok := response.(GetActivationLastAtxNodeIdResponseObject); ok { + if err := validResponse.VisitGetActivationLastAtxNodeIdResponse(w); err != nil { + sh.options.ResponseErrorHandlerFunc(w, r, err) + } + } else if response != nil { + sh.options.ResponseErrorHandlerFunc(w, r, fmt.Errorf("unexpected response type: %T", response)) + } +} + +// GetActivationPositioningAtxPublishEpoch operation middleware +func (sh *strictHandler) GetActivationPositioningAtxPublishEpoch(w http.ResponseWriter, r *http.Request, publishEpoch externalRef0.EpochID) { + var request GetActivationPositioningAtxPublishEpochRequestObject + + request.PublishEpoch = publishEpoch + + handler := func(ctx context.Context, w http.ResponseWriter, r *http.Request, request interface{}) (interface{}, error) { + return sh.ssi.GetActivationPositioningAtxPublishEpoch(ctx, request.(GetActivationPositioningAtxPublishEpochRequestObject)) + } + for _, middleware := range sh.middlewares { + handler = middleware(handler, "GetActivationPositioningAtxPublishEpoch") + } + + response, err := handler(r.Context(), w, r, request) + + if err != nil { + sh.options.ResponseErrorHandlerFunc(w, r, err) + } else if validResponse, ok := response.(GetActivationPositioningAtxPublishEpochResponseObject); ok { + if err := validResponse.VisitGetActivationPositioningAtxPublishEpochResponse(w); err != nil { + sh.options.ResponseErrorHandlerFunc(w, r, err) + } + } else if response != nil { + sh.options.ResponseErrorHandlerFunc(w, r, fmt.Errorf("unexpected response type: %T", response)) + } +} + +// PostPublishProtocol operation middleware +func (sh *strictHandler) PostPublishProtocol(w http.ResponseWriter, r *http.Request, protocol PostPublishProtocolParamsProtocol) { + var request PostPublishProtocolRequestObject + + request.Protocol = protocol + + request.Body = r.Body + + handler := func(ctx context.Context, w http.ResponseWriter, r *http.Request, request interface{}) (interface{}, error) { + return sh.ssi.PostPublishProtocol(ctx, request.(PostPublishProtocolRequestObject)) + } + for _, middleware := range sh.middlewares { + handler = middleware(handler, "PostPublishProtocol") + } + + response, err := handler(r.Context(), w, r, request) + + if err != nil { + sh.options.ResponseErrorHandlerFunc(w, r, err) + } else if validResponse, ok := response.(PostPublishProtocolResponseObject); ok { + if err := validResponse.VisitPostPublishProtocolResponse(w); err != nil { + sh.options.ResponseErrorHandlerFunc(w, r, err) + } + } else if response != nil { + sh.options.ResponseErrorHandlerFunc(w, r, fmt.Errorf("unexpected response type: %T", response)) + } +} diff --git a/api/node/server/server.go b/api/node/server/server.go new file mode 100644 index 0000000000..95b3db10fd --- /dev/null +++ b/api/node/server/server.go @@ -0,0 +1,159 @@ +package server + +import ( + "bytes" + "context" + "encoding/hex" + "errors" + "io" + "net/http" + + "github.com/google/uuid" + "github.com/oapi-codegen/runtime/strictmiddleware/nethttp" + "go.uber.org/zap" + + "github.com/spacemeshos/go-spacemesh/activation" + "github.com/spacemeshos/go-spacemesh/api/node/models" + "github.com/spacemeshos/go-spacemesh/common" + "github.com/spacemeshos/go-spacemesh/common/types" + "github.com/spacemeshos/go-spacemesh/p2p/pubsub" +) + +type Server struct { + activationService activation.AtxService + publisher pubsub.Publisher + logger *zap.Logger +} + +var _ StrictServerInterface = (*Server)(nil) + +func NewServer(activationService activation.AtxService, publisher pubsub.PubSub, logger *zap.Logger) *Server { + return &Server{ + activationService: activationService, + publisher: publisher, + logger: logger, + } +} + +func (s *Server) IntoHandler(mux *http.ServeMux) http.Handler { + loggingMid := func(f nethttp.StrictHTTPHandlerFunc, operationID string) nethttp.StrictHTTPHandlerFunc { + return func(ctx context.Context, w http.ResponseWriter, r *http.Request, req any) (any, error) { + uuid := uuid.New() + s.logger.Debug( + "request", + zap.Stringer("request_id", uuid), + zap.String("operation", operationID), + zap.Any("request", req), + ) + response, err := f(ctx, w, r, req) + s.logger.Debug( + "response", + zap.Stringer("request_id", uuid), + zap.String("operation", operationID), + zap.Any("response", response), + zap.Error(err), + ) + return response, err + } + } + return HandlerFromMux(NewStrictHandler(s, []StrictMiddlewareFunc{loggingMid}), mux) +} + +func (s *Server) Start(address string) error { + server := &http.Server{ + Handler: s.IntoHandler(http.NewServeMux()), + Addr: address, + } + return server.ListenAndServe() +} + +// GetActivationAtxAtxId implements StrictServerInterface. +func (s *Server) GetActivationAtxAtxId( + ctx context.Context, + request GetActivationAtxAtxIdRequestObject, +) (GetActivationAtxAtxIdResponseObject, error) { + idBytes, err := hex.DecodeString(request.AtxId) + if err != nil { + return nil, err + } + id := types.BytesToATXID(idBytes) + atx, err := s.activationService.Atx(ctx, id) + switch { + case errors.Is(err, common.ErrNotFound): + return GetActivationAtxAtxId404Response{}, nil + case err != nil: + return nil, err + } + + return GetActivationAtxAtxId200JSONResponse{ + ID: request.AtxId, + NumUnits: atx.NumUnits, + PublishEpoch: atx.PublishEpoch.Uint32(), + Sequence: &atx.Sequence, + SmesherID: hex.EncodeToString(atx.SmesherID.Bytes()), + TickCount: atx.TickCount, + Weight: atx.Weight, + }, nil +} + +// GetActivationLastAtxNodeId implements StrictServerInterface. +func (s *Server) GetActivationLastAtxNodeId( + ctx context.Context, + request GetActivationLastAtxNodeIdRequestObject, +) (GetActivationLastAtxNodeIdResponseObject, error) { + id, err := models.ParseNodeID(request.NodeId) + if err != nil { + msg := err.Error() + return GetActivationLastAtxNodeId400PlaintextResponse{ + Body: bytes.NewBuffer([]byte(msg)), + ContentLength: int64(len(msg)), + }, nil + } + + atxid, err := s.activationService.LastATX(ctx, id) + switch { + case errors.Is(err, common.ErrNotFound): + return GetActivationLastAtxNodeId404Response{}, nil + case err != nil: + return nil, err + } + + return GetActivationLastAtxNodeId200JSONResponse{ + ID: hex.EncodeToString(atxid.ID().Bytes()), + NumUnits: atxid.NumUnits, + PublishEpoch: atxid.PublishEpoch.Uint32(), + Sequence: &atxid.Sequence, + SmesherID: hex.EncodeToString(atxid.SmesherID.Bytes()), + TickCount: atxid.TickCount, + Weight: atxid.Weight, + }, nil +} + +// GetActivationPositioningAtxEpoch implements StrictServerInterface. +func (s *Server) GetActivationPositioningAtxPublishEpoch( + ctx context.Context, + request GetActivationPositioningAtxPublishEpochRequestObject, +) (GetActivationPositioningAtxPublishEpochResponseObject, error) { + id, err := s.activationService.PositioningATX(ctx, types.EpochID(request.PublishEpoch)) + if err != nil { + return nil, err + } + + return GetActivationPositioningAtxPublishEpoch200JSONResponse{ + ID: hex.EncodeToString(id.Bytes()), + }, nil +} + +// PostPublishProtocol implements StrictServerInterface. +func (s *Server) PostPublishProtocol( + ctx context.Context, + request PostPublishProtocolRequestObject, +) (PostPublishProtocolResponseObject, error) { + blob, err := io.ReadAll(request.Body) + if err != nil { + return nil, err + } + + s.publisher.Publish(ctx, string(request.Protocol), blob) + return PostPublishProtocol200Response{}, nil +} diff --git a/config/config.go b/config/config.go index 0a08120db9..633276d330 100644 --- a/config/config.go +++ b/config/config.go @@ -151,6 +151,10 @@ type BaseConfig struct { // Note: There is always one valid version at any given time. // ATX V1 starts with epoch 0 unless configured otherwise. AtxVersions activation.AtxVersions `mapstructure:"atx-versions"` + + // Address of the node serivce + // TODO: add better description. + NodeServiceAddress string `mapstructure:"node-service-address"` } type DatabaseQueryCacheSizes struct { diff --git a/go.mod b/go.mod index d4f923d927..054f228a95 100644 --- a/go.mod +++ b/go.mod @@ -30,6 +30,7 @@ require ( github.com/multiformats/go-multiaddr v0.13.0 github.com/multiformats/go-varint v0.0.7 github.com/natefinch/atomic v1.0.1 + github.com/oapi-codegen/runtime v1.1.1 github.com/oasisprotocol/curve25519-voi v0.0.0-20230904125328-1f23a7beb09a github.com/prometheus/client_golang v1.20.4 github.com/prometheus/client_model v0.6.1 @@ -78,6 +79,7 @@ require ( github.com/anacrolix/missinggo v1.2.1 // indirect github.com/anacrolix/missinggo/perf v1.0.0 // indirect github.com/anacrolix/sync v0.3.0 // indirect + github.com/apapsch/go-jsonmerge/v2 v2.0.0 // indirect github.com/benbjohnson/clock v1.3.5 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/c0mm4nd/go-ripemd v0.0.0-20200326052756-bd1759ad7d10 // indirect diff --git a/go.sum b/go.sum index bd9837e52c..c61ff9e3de 100644 --- a/go.sum +++ b/go.sum @@ -24,6 +24,7 @@ git.apache.org/thrift.git v0.0.0-20180902110319-2566ecd5d999/go.mod h1:fPE2ZNJGy github.com/ALTree/bigfloat v0.2.0 h1:AwNzawrpFuw55/YDVlcPw0F0cmmXrmngBHhVrvdXPvM= github.com/ALTree/bigfloat v0.2.0/go.mod h1:+NaH2gLeY6RPBPPQf4aRotPPStg+eXc8f9ZaE4vRfD4= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= +github.com/RaveNoX/go-jsoncommentstrip v1.0.0/go.mod h1:78ihd09MekBnJnxpICcwzCMzGrKSKYe4AqU6PDYYpjk= github.com/RoaringBitmap/roaring v0.4.7/go.mod h1:8khRDP4HmeXns4xIj9oGrKSz7XTQiJx2zgh7AcNke4w= github.com/alecthomas/units v0.0.0-20211218093645-b94a6e3cc137 h1:s6gZFSlWYmbqAuRjVTiNNhvNRfY2Wxp9nhfyel4rklc= github.com/alecthomas/units v0.0.0-20211218093645-b94a6e3cc137/go.mod h1:OMCwj8VM1Kc9e19TLln2VL61YJF0x1XFtfdL4JdbSyE= @@ -41,6 +42,8 @@ github.com/anacrolix/sync v0.3.0 h1:ZPjTrkqQWEfnYVGTQHh5qNjokWaXnjsyXTJSMsKY0TA= github.com/anacrolix/sync v0.3.0/go.mod h1:BbecHL6jDSExojhNtgTFSBcdGerzNc64tz3DCOj/I0g= github.com/anacrolix/tagflag v0.0.0-20180109131632-2146c8d41bf0/go.mod h1:1m2U/K6ZT+JZG0+bdMK6qauP49QT4wE5pmhJXOKKCHw= github.com/anmitsu/go-shlex v0.0.0-20161002113705-648efa622239/go.mod h1:2FmKhYUyUczH0OGQWaF5ceTx0UBShxjsH6f8oGKYe2c= +github.com/apapsch/go-jsonmerge/v2 v2.0.0 h1:axGnT1gRIfimI7gJifB699GoE/oq+F2MU7Dml6nw9rQ= +github.com/apapsch/go-jsonmerge/v2 v2.0.0/go.mod h1:lvDnEdqiQrp0O42VQGgmlKpxL1AP2+08jFMw88y4klk= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/benbjohnson/clock v1.3.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/benbjohnson/clock v1.3.5 h1:VvXlSJBzZpA/zum6Sj74hxwYI2DIxRWuNIoXAzHZz5o= @@ -48,6 +51,7 @@ github.com/benbjohnson/clock v1.3.5/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZx github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= +github.com/bmatcuk/doublestar v1.1.1/go.mod h1:UD6OnuiIn0yFxxA2le/rnRU1G4RaI4UvFv1sNto9p6w= github.com/bradfitz/go-smtpd v0.0.0-20170404230938-deb6d6237625/go.mod h1:HYsPBTaaSFSlLx/70C2HPIMNZpVV8+vt/A+FMnYP11g= github.com/bradfitz/iter v0.0.0-20140124041915-454541ec3da2/go.mod h1:PyRFw1Lt2wKX4ZVSQ2mk+PeDa1rxyObEDlApuIsUKuo= github.com/bradfitz/iter v0.0.0-20190303215204-33e6a9893b0c h1:FUUopH4brHNO2kJoNN3pV+OBEYmgraLT/KHZrMM69r0= @@ -318,6 +322,7 @@ github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1 github.com/jtolds/gls v4.2.1+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/jtolds/gls v4.20.0+incompatible h1:xdiiI2gbIgH/gLH7ADydsJ1uDOEzR8yvV7C0MuV77Wo= github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= +github.com/juju/gnuflag v0.0.0-20171113085948-2ce1bb71843d/go.mod h1:2PavIy+JPciBPrBUjwbNvtwB6RQlve+hkpll6QSNmOE= github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= @@ -448,6 +453,8 @@ github.com/nullstyle/go-xdr v0.0.0-20180726165426-f4c839f75077/go.mod h1:sZZi9x5 github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= github.com/nxadm/tail v1.4.11 h1:8feyoE3OzPrcshW5/MJ4sGESc5cqmGkGCWlco4l0bqY= github.com/nxadm/tail v1.4.11/go.mod h1:OTaG3NK980DZzxbRq6lEuzgU+mug70nY11sMd4JXXHc= +github.com/oapi-codegen/runtime v1.1.1 h1:EXLHh0DXIJnWhdRPN2w4MXAzFyE4CskzhNLUmtpMYro= +github.com/oapi-codegen/runtime v1.1.1/go.mod h1:SK9X900oXmPWilYR5/WKPzt3Kqxn/uS/+lbpREv+eCg= github.com/oasisprotocol/curve25519-voi v0.0.0-20230904125328-1f23a7beb09a h1:dlRvE5fWabOchtH7znfiFCcOvmIYgOeAS5ifBXBlh9Q= github.com/oasisprotocol/curve25519-voi v0.0.0-20230904125328-1f23a7beb09a/go.mod h1:hVoHR2EVESiICEMbg137etN/Lx+lSrHPTD39Z/uE+2s= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= @@ -631,6 +638,7 @@ github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= github.com/spf13/viper v1.19.0 h1:RWq5SEjt8o25SROyN3z2OrDB9l7RPd3lwTWU8EcEdcI= github.com/spf13/viper v1.19.0/go.mod h1:GQUN9bilAbhU/jgc1bKs99f/suXKeUMct8Adx5+Ntkg= +github.com/spkg/bom v0.0.0-20160624110644-59b7046e48ad/go.mod h1:qLr4V1qq6nMqFKkMo8ZTx3f+BZEkzsRUY10Xsm2mwU0= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= diff --git a/node/node.go b/node/node.go index 6013061a5f..b44a36bae8 100644 --- a/node/node.go +++ b/node/node.go @@ -38,6 +38,8 @@ import ( "github.com/spacemeshos/go-spacemesh/activation" "github.com/spacemeshos/go-spacemesh/api/grpcserver" "github.com/spacemeshos/go-spacemesh/api/grpcserver/v2alpha1" + nodeclient "github.com/spacemeshos/go-spacemesh/api/node/client" + nodeserver "github.com/spacemeshos/go-spacemesh/api/node/server" "github.com/spacemeshos/go-spacemesh/atxsdata" "github.com/spacemeshos/go-spacemesh/beacon" "github.com/spacemeshos/go-spacemesh/blocks" @@ -382,6 +384,7 @@ type App struct { grpcPostServer *grpcserver.Server grpcTLSServer *grpcserver.Server jsonAPIServer *grpcserver.JSONHTTPServer + nodeServiceServer *http.Server grpcServices map[grpcserver.Service]grpcserver.ServiceAPI pprofService *http.Server profilerService *pyroscope.Profiler @@ -1059,20 +1062,41 @@ func (app *App) initServices(ctx context.Context) error { RegossipInterval: app.Config.RegossipAtxInterval, } - atxBuilderLog := app.addLogger(ATXBuilderLogger, lg).Zap() - atxService := activation.NewDBAtxService( - app.db, - goldenATXID, - app.atxsdata, - app.validator, - atxBuilderLog, - activation.WithPostValidityDelay(app.Config.PostValidDelay), + var ( + atxBuilderLog = app.addLogger(ATXBuilderLogger, lg).Zap() + atxService activation.AtxService + atxPublisher pubsub.Publisher ) + if server := app.Config.BaseConfig.NodeServiceAddress; server != "" { + logger := app.log.Zap().Named("node-svc-client") + cfg := &nodeclient.Config{ + RetryWaitMin: time.Millisecond * 500, + RetryWaitMax: time.Second, + RetryMax: 10, + } + nodeServiceClient, err := nodeclient.NewNodeServiceClient(server, logger, cfg) + if err != nil { + return fmt.Errorf("creating node service client: %w", err) + } + atxService = nodeServiceClient + atxPublisher = nodeServiceClient + } else { + atxService = activation.NewDBAtxService( + app.db, + goldenATXID, + app.atxsdata, + app.validator, + atxBuilderLog, + activation.WithPostValidityDelay(app.Config.PostValidDelay), + ) + atxPublisher = app.host + } + atxBuilder := activation.NewBuilder( builderConfig, app.localDB, atxService, - app.host, + atxPublisher, app.validator, nipostBuilder, app.clock, @@ -1821,6 +1845,24 @@ func (app *App) startAPIServices(ctx context.Context) error { })), ) } + + if len(app.Config.API.NodeServiceListener) > 0 { + app.log.Zap().Info("starting node service", zap.String("address", app.Config.API.NodeServiceListener)) + lis, err := net.Listen("tcp", app.Config.API.NodeServiceListener) + if err != nil { + return err + } + golden := types.ATXID(app.Config.Genesis.GoldenATX()) + logger := app.log.Zap().Named("atx-service") + actSvc := activation.NewDBAtxService(app.db, golden, app.atxsdata, app.validator, logger) + server := nodeserver.NewServer(actSvc, app.host, logger) + + app.nodeServiceServer = &http.Server{ + Handler: server.IntoHandler(http.NewServeMux()), + } + app.eg.Go(func() error { return app.nodeServiceServer.Serve(lis) }) + } + return nil } From 60c21b815d88933935ed7713a4bb1f47e60e72dc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bartosz=20R=C3=B3=C5=BCa=C5=84ski?= Date: Thu, 26 Sep 2024 14:23:05 +0200 Subject: [PATCH 03/10] Pass stored PoET proofs to the node-service --- activation/interface.go | 4 ++ activation/mocks.go | 61 ++++++++++++++++++++ activation/poetdb.go | 47 ++++++++++----- api/node/client/client.gen.go | 93 ++++++++++++++++++++++++++++++ api/node/client/client.go | 34 ++++++++--- api/node/client/client_e2e_test.go | 48 +++++++++++---- api/node/node_service.yaml | 21 +++++++ api/node/server/mocks.go | 79 +++++++++++++++++++++++++ api/node/server/server.gen.go | 82 ++++++++++++++++++++++++++ api/node/server/server.go | 60 +++++++++++++++---- node/node.go | 36 +++++++----- 11 files changed, 508 insertions(+), 57 deletions(-) create mode 100644 api/node/server/mocks.go diff --git a/activation/interface.go b/activation/interface.go index 4e642d24e7..2225d3dda0 100644 --- a/activation/interface.go +++ b/activation/interface.go @@ -188,6 +188,10 @@ type certifierService interface { DeleteCertificate(id types.NodeID, pubkey []byte) error } +type PoetDbStorer interface { + StorePoetProof(ctx context.Context, proofMessage *types.PoetProofMessage) error +} + type poetDbAPI interface { Proof(types.PoetProofRef) (*types.PoetProof, *types.Hash32, error) ProofForRound(poetID []byte, roundID string) (*types.PoetProof, error) diff --git a/activation/mocks.go b/activation/mocks.go index bdeee5b2b8..2e68026b7e 100644 --- a/activation/mocks.go +++ b/activation/mocks.go @@ -2113,6 +2113,67 @@ func (c *MockcertifierServiceDeleteCertificateCall) DoAndReturn(f func(types.Nod return c } +// MockPoetDbStorer is a mock of PoetDbStorer interface. +type MockPoetDbStorer struct { + ctrl *gomock.Controller + recorder *MockPoetDbStorerMockRecorder +} + +// MockPoetDbStorerMockRecorder is the mock recorder for MockPoetDbStorer. +type MockPoetDbStorerMockRecorder struct { + mock *MockPoetDbStorer +} + +// NewMockPoetDbStorer creates a new mock instance. +func NewMockPoetDbStorer(ctrl *gomock.Controller) *MockPoetDbStorer { + mock := &MockPoetDbStorer{ctrl: ctrl} + mock.recorder = &MockPoetDbStorerMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockPoetDbStorer) EXPECT() *MockPoetDbStorerMockRecorder { + return m.recorder +} + +// StorePoetProof mocks base method. +func (m *MockPoetDbStorer) StorePoetProof(ctx context.Context, proofMessage *types.PoetProofMessage) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "StorePoetProof", ctx, proofMessage) + ret0, _ := ret[0].(error) + return ret0 +} + +// StorePoetProof indicates an expected call of StorePoetProof. +func (mr *MockPoetDbStorerMockRecorder) StorePoetProof(ctx, proofMessage any) *MockPoetDbStorerStorePoetProofCall { + mr.mock.ctrl.T.Helper() + call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StorePoetProof", reflect.TypeOf((*MockPoetDbStorer)(nil).StorePoetProof), ctx, proofMessage) + return &MockPoetDbStorerStorePoetProofCall{Call: call} +} + +// MockPoetDbStorerStorePoetProofCall wrap *gomock.Call +type MockPoetDbStorerStorePoetProofCall struct { + *gomock.Call +} + +// Return rewrite *gomock.Call.Return +func (c *MockPoetDbStorerStorePoetProofCall) Return(arg0 error) *MockPoetDbStorerStorePoetProofCall { + c.Call = c.Call.Return(arg0) + return c +} + +// Do rewrite *gomock.Call.Do +func (c *MockPoetDbStorerStorePoetProofCall) Do(f func(context.Context, *types.PoetProofMessage) error) *MockPoetDbStorerStorePoetProofCall { + c.Call = c.Call.Do(f) + return c +} + +// DoAndReturn rewrite *gomock.Call.DoAndReturn +func (c *MockPoetDbStorerStorePoetProofCall) DoAndReturn(f func(context.Context, *types.PoetProofMessage) error) *MockPoetDbStorerStorePoetProofCall { + c.Call = c.Call.DoAndReturn(f) + return c +} + // MockpoetDbAPI is a mock of poetDbAPI interface. type MockpoetDbAPI struct { ctrl *gomock.Controller diff --git a/activation/poetdb.go b/activation/poetdb.go index 3905564f89..e3f1780532 100644 --- a/activation/poetdb.go +++ b/activation/poetdb.go @@ -21,13 +21,26 @@ import ( // PoetDb is a database for PoET proofs. type PoetDb struct { - sqlDB sql.StateDatabase - logger *zap.Logger + sqlDB sql.StateDatabase + logger *zap.Logger + remoteStorer PoetDbStorer +} + +type PoetDbOption func(*PoetDb) + +func WithRemotePoetStorer(storer PoetDbStorer) PoetDbOption { + return func(pd *PoetDb) { + pd.remoteStorer = storer + } } // NewPoetDb returns a new PoET handler. -func NewPoetDb(db sql.StateDatabase, log *zap.Logger) *PoetDb { - return &PoetDb{sqlDB: db, logger: log} +func NewPoetDb(db sql.StateDatabase, log *zap.Logger, opts ...PoetDbOption) *PoetDb { + poetdb := &PoetDb{sqlDB: db, logger: log} + for _, opt := range opts { + opt(poetdb) + } + return poetdb } // HasProof returns true if the database contains a proof with the given reference, or false otherwise. @@ -57,6 +70,13 @@ func (db *PoetDb) ValidateAndStore(ctx context.Context, proofMessage *types.Poet return err } + if db.remoteStorer != nil { + err := db.remoteStorer.StorePoetProof(ctx, proofMessage) + if err != nil { + db.logger.Warn("failed to store the poet proof in remote store", zap.Error(err)) + } + } + return db.StoreProof(ctx, ref, proofMessage) } @@ -135,25 +155,26 @@ func (db *PoetDb) GetProofRef(poetID []byte, roundID string) (types.PoetProofRef } // GetProofMessage returns the originally received PoET proof message. -func (db *PoetDb) GetProofMessage(proofRef types.PoetProofRef) ([]byte, error) { +func (db *PoetDb) ProofMessage(proofRef types.PoetProofRef) (*types.PoetProofMessage, error) { proof, err := poets.Get(db.sqlDB, proofRef) if err != nil { - return proof, fmt.Errorf("get proof from store: %w", err) + return nil, fmt.Errorf("get proof from store: %w", err) + } + var proofMessage types.PoetProofMessage + if err := codec.Decode(proof, &proofMessage); err != nil { + return nil, fmt.Errorf("failed to unmarshal poet proof for ref %x: %w", proofRef, err) } - return proof, nil + return &proofMessage, nil } // Proof returns full proof. func (db *PoetDb) Proof(proofRef types.PoetProofRef) (*types.PoetProof, *types.Hash32, error) { - proofMessageBytes, err := db.GetProofMessage(proofRef) + proofMessage, err := db.ProofMessage(proofRef) if err != nil { - return nil, nil, fmt.Errorf("could not fetch poet proof for ref %x: %w", proofRef, err) - } - var proofMessage types.PoetProofMessage - if err := codec.Decode(proofMessageBytes, &proofMessage); err != nil { - return nil, nil, fmt.Errorf("failed to unmarshal poet proof for ref %x: %w", proofRef, err) + return nil, nil, err } + return &proofMessage.PoetProof, &proofMessage.Statement, nil } diff --git a/api/node/client/client.gen.go b/api/node/client/client.gen.go index 5d4f8d00e5..e528badc57 100644 --- a/api/node/client/client.gen.go +++ b/api/node/client/client.gen.go @@ -115,6 +115,9 @@ type ClientInterface interface { // GetActivationPositioningAtxPublishEpoch request GetActivationPositioningAtxPublishEpoch(ctx context.Context, publishEpoch externalRef0.EpochID, reqEditors ...RequestEditorFn) (*http.Response, error) + // PostPoetWithBody request with any body + PostPoetWithBody(ctx context.Context, contentType string, body io.Reader, reqEditors ...RequestEditorFn) (*http.Response, error) + // PostPublishProtocolWithBody request with any body PostPublishProtocolWithBody(ctx context.Context, protocol PostPublishProtocolParamsProtocol, contentType string, body io.Reader, reqEditors ...RequestEditorFn) (*http.Response, error) } @@ -155,6 +158,18 @@ func (c *Client) GetActivationPositioningAtxPublishEpoch(ctx context.Context, pu return c.Client.Do(req) } +func (c *Client) PostPoetWithBody(ctx context.Context, contentType string, body io.Reader, reqEditors ...RequestEditorFn) (*http.Response, error) { + req, err := NewPostPoetRequestWithBody(c.Server, contentType, body) + if err != nil { + return nil, err + } + req = req.WithContext(ctx) + if err := c.applyEditors(ctx, req, reqEditors); err != nil { + return nil, err + } + return c.Client.Do(req) +} + func (c *Client) PostPublishProtocolWithBody(ctx context.Context, protocol PostPublishProtocolParamsProtocol, contentType string, body io.Reader, reqEditors ...RequestEditorFn) (*http.Response, error) { req, err := NewPostPublishProtocolRequestWithBody(c.Server, protocol, contentType, body) if err != nil { @@ -269,6 +284,35 @@ func NewGetActivationPositioningAtxPublishEpochRequest(server string, publishEpo return req, nil } +// NewPostPoetRequestWithBody generates requests for PostPoet with any type of body +func NewPostPoetRequestWithBody(server string, contentType string, body io.Reader) (*http.Request, error) { + var err error + + serverURL, err := url.Parse(server) + if err != nil { + return nil, err + } + + operationPath := fmt.Sprintf("/poet") + if operationPath[0] == '/' { + operationPath = "." + operationPath + } + + queryURL, err := serverURL.Parse(operationPath) + if err != nil { + return nil, err + } + + req, err := http.NewRequest("POST", queryURL.String(), body) + if err != nil { + return nil, err + } + + req.Header.Add("Content-Type", contentType) + + return req, nil +} + // NewPostPublishProtocolRequestWithBody generates requests for PostPublishProtocol with any type of body func NewPostPublishProtocolRequestWithBody(server string, protocol PostPublishProtocolParamsProtocol, contentType string, body io.Reader) (*http.Request, error) { var err error @@ -357,6 +401,9 @@ type ClientWithResponsesInterface interface { // GetActivationPositioningAtxPublishEpochWithResponse request GetActivationPositioningAtxPublishEpochWithResponse(ctx context.Context, publishEpoch externalRef0.EpochID, reqEditors ...RequestEditorFn) (*GetActivationPositioningAtxPublishEpochResponse, error) + // PostPoetWithBodyWithResponse request with any body + PostPoetWithBodyWithResponse(ctx context.Context, contentType string, body io.Reader, reqEditors ...RequestEditorFn) (*PostPoetResponse, error) + // PostPublishProtocolWithBodyWithResponse request with any body PostPublishProtocolWithBodyWithResponse(ctx context.Context, protocol PostPublishProtocolParamsProtocol, contentType string, body io.Reader, reqEditors ...RequestEditorFn) (*PostPublishProtocolResponse, error) } @@ -429,6 +476,27 @@ func (r GetActivationPositioningAtxPublishEpochResponse) StatusCode() int { return 0 } +type PostPoetResponse struct { + Body []byte + HTTPResponse *http.Response +} + +// Status returns HTTPResponse.Status +func (r PostPoetResponse) Status() string { + if r.HTTPResponse != nil { + return r.HTTPResponse.Status + } + return http.StatusText(0) +} + +// StatusCode returns HTTPResponse.StatusCode +func (r PostPoetResponse) StatusCode() int { + if r.HTTPResponse != nil { + return r.HTTPResponse.StatusCode + } + return 0 +} + type PostPublishProtocolResponse struct { Body []byte HTTPResponse *http.Response @@ -477,6 +545,15 @@ func (c *ClientWithResponses) GetActivationPositioningAtxPublishEpochWithRespons return ParseGetActivationPositioningAtxPublishEpochResponse(rsp) } +// PostPoetWithBodyWithResponse request with arbitrary body returning *PostPoetResponse +func (c *ClientWithResponses) PostPoetWithBodyWithResponse(ctx context.Context, contentType string, body io.Reader, reqEditors ...RequestEditorFn) (*PostPoetResponse, error) { + rsp, err := c.PostPoetWithBody(ctx, contentType, body, reqEditors...) + if err != nil { + return nil, err + } + return ParsePostPoetResponse(rsp) +} + // PostPublishProtocolWithBodyWithResponse request with arbitrary body returning *PostPublishProtocolResponse func (c *ClientWithResponses) PostPublishProtocolWithBodyWithResponse(ctx context.Context, protocol PostPublishProtocolParamsProtocol, contentType string, body io.Reader, reqEditors ...RequestEditorFn) (*PostPublishProtocolResponse, error) { rsp, err := c.PostPublishProtocolWithBody(ctx, protocol, contentType, body, reqEditors...) @@ -566,6 +643,22 @@ func ParseGetActivationPositioningAtxPublishEpochResponse(rsp *http.Response) (* return response, nil } +// ParsePostPoetResponse parses an HTTP response from a PostPoetWithResponse call +func ParsePostPoetResponse(rsp *http.Response) (*PostPoetResponse, error) { + bodyBytes, err := io.ReadAll(rsp.Body) + defer func() { _ = rsp.Body.Close() }() + if err != nil { + return nil, err + } + + response := &PostPoetResponse{ + Body: bodyBytes, + HTTPResponse: rsp, + } + + return response, nil +} + // ParsePostPublishProtocolResponse parses an HTTP response from a PostPublishProtocolWithResponse call func ParsePostPublishProtocolResponse(rsp *http.Response) (*PostPublishProtocolResponse, error) { bodyBytes, err := io.ReadAll(rsp.Body) diff --git a/api/node/client/client.go b/api/node/client/client.go index 354cb189d5..dab49ef641 100644 --- a/api/node/client/client.go +++ b/api/node/client/client.go @@ -13,19 +13,21 @@ import ( "github.com/spacemeshos/go-spacemesh/activation" "github.com/spacemeshos/go-spacemesh/api/node/models" + "github.com/spacemeshos/go-spacemesh/codec" "github.com/spacemeshos/go-spacemesh/common" "github.com/spacemeshos/go-spacemesh/common/types" "github.com/spacemeshos/go-spacemesh/p2p/pubsub" ) -type nodeService struct { +type NodeService struct { client *ClientWithResponses logger *zap.Logger } var ( - _ activation.AtxService = (*nodeService)(nil) - _ pubsub.Publisher = (*nodeService)(nil) + _ activation.AtxService = (*NodeService)(nil) + _ activation.PoetDbStorer = (*NodeService)(nil) + _ pubsub.Publisher = (*NodeService)(nil) ) type Config struct { @@ -34,7 +36,7 @@ type Config struct { RetryMax int // Maximum number of retries } -func NewNodeServiceClient(server string, logger *zap.Logger, cfg *Config) (*nodeService, error) { +func NewNodeServiceClient(server string, logger *zap.Logger, cfg *Config) (*NodeService, error) { retryableClient := retryablehttp.Client{ Logger: &retryableHttpLogger{logger}, RetryWaitMin: cfg.RetryWaitMin, @@ -47,13 +49,13 @@ func NewNodeServiceClient(server string, logger *zap.Logger, cfg *Config) (*node if err != nil { return nil, err } - return &nodeService{ + return &NodeService{ client: client, logger: logger, }, nil } -func (s *nodeService) Atx(ctx context.Context, id types.ATXID) (*types.ActivationTx, error) { +func (s *NodeService) Atx(ctx context.Context, id types.ATXID) (*types.ActivationTx, error) { resp, err := s.client.GetActivationAtxAtxIdWithResponse(ctx, hex.EncodeToString(id.Bytes())) if err != nil { return nil, err @@ -68,7 +70,7 @@ func (s *nodeService) Atx(ctx context.Context, id types.ATXID) (*types.Activatio return models.ParseATX(resp.JSON200) } -func (s *nodeService) LastATX(ctx context.Context, nodeID types.NodeID) (*types.ActivationTx, error) { +func (s *NodeService) LastATX(ctx context.Context, nodeID types.NodeID) (*types.ActivationTx, error) { resp, err := s.client.GetActivationLastAtxNodeIdWithResponse(ctx, hex.EncodeToString(nodeID.Bytes())) if err != nil { return nil, err @@ -83,7 +85,7 @@ func (s *nodeService) LastATX(ctx context.Context, nodeID types.NodeID) (*types. return models.ParseATX(resp.JSON200) } -func (s *nodeService) PositioningATX(ctx context.Context, maxPublish types.EpochID) (types.ATXID, error) { +func (s *NodeService) PositioningATX(ctx context.Context, maxPublish types.EpochID) (types.ATXID, error) { resp, err := s.client.GetActivationPositioningAtxPublishEpochWithResponse(ctx, maxPublish.Uint32()) if err != nil { return types.ATXID{}, err @@ -96,7 +98,7 @@ func (s *nodeService) PositioningATX(ctx context.Context, maxPublish types.Epoch } // Publish implements pubsub.Publisher. -func (s *nodeService) Publish(ctx context.Context, proto string, blob []byte) error { +func (s *NodeService) Publish(ctx context.Context, proto string, blob []byte) error { buf := bytes.NewBuffer(blob) protocol := PostPublishProtocolParamsProtocol(proto) resp, err := s.client.PostPublishProtocolWithBody(ctx, protocol, "application/octet-stream", buf) @@ -108,3 +110,17 @@ func (s *nodeService) Publish(ctx context.Context, proto string, blob []byte) er } return nil } + +// StorePoetProof implements activation.PoetDbStorer. +func (s *NodeService) StorePoetProof(ctx context.Context, proof *types.PoetProofMessage) error { + blob := codec.MustEncode(proof) + buf := bytes.NewBuffer(blob) + resp, err := s.client.PostPoetWithBody(ctx, "application/octet-stream", buf) + if err != nil { + return err + } + if resp.StatusCode != http.StatusOK { + return fmt.Errorf("unexpected status: %s", resp.Status) + } + return nil +} diff --git a/api/node/client/client_e2e_test.go b/api/node/client/client_e2e_test.go index a8bdd31bd7..30f804fd5c 100644 --- a/api/node/client/client_e2e_test.go +++ b/api/node/client/client_e2e_test.go @@ -16,14 +16,28 @@ import ( "github.com/spacemeshos/go-spacemesh/api/node/server" "github.com/spacemeshos/go-spacemesh/common" "github.com/spacemeshos/go-spacemesh/common/types" + pubsub "github.com/spacemeshos/go-spacemesh/p2p/pubsub/mocks" ) const retries = 3 -func setupE2E(t *testing.T) (activation.AtxService, *activation.MockAtxService) { +type mocks struct { + atxService *activation.MockAtxService + poetDb *server.MockpoetDB + publisher *pubsub.MockPublisher +} + +func setupE2E(t *testing.T) (*client.NodeService, *mocks) { log := zaptest.NewLogger(t) - actServiceMock := activation.NewMockAtxService(gomock.NewController(t)) - activationServiceServer := server.NewServer(actServiceMock, nil, log.Named("server")) + + ctrl := gomock.NewController(t) + m := &mocks{ + atxService: activation.NewMockAtxService(ctrl), + poetDb: server.NewMockpoetDB(ctrl), + publisher: pubsub.NewMockPublisher(ctrl), + } + + activationServiceServer := server.NewServer(m.atxService, m.publisher, m.poetDb, log.Named("server")) listener, err := net.Listen("tcp", "localhost:0") require.NoError(t, err) @@ -42,7 +56,7 @@ func setupE2E(t *testing.T) (activation.AtxService, *activation.MockAtxService) } svc, err := client.NewNodeServiceClient("http://"+listener.Addr().String(), log.Named("server"), cfg) require.NoError(t, err) - return svc, actServiceMock + return svc, m } func Test_ActivationService_Atx(t *testing.T) { @@ -51,7 +65,7 @@ func Test_ActivationService_Atx(t *testing.T) { atxid := types.ATXID{1, 2, 3, 4} t.Run("not found", func(t *testing.T) { - mock.EXPECT().Atx(gomock.Any(), atxid).Return(nil, common.ErrNotFound) + mock.atxService.EXPECT().Atx(gomock.Any(), atxid).Return(nil, common.ErrNotFound) _, err := svc.Atx(context.Background(), atxid) require.ErrorIs(t, err, common.ErrNotFound) }) @@ -59,14 +73,14 @@ func Test_ActivationService_Atx(t *testing.T) { t.Run("found", func(t *testing.T) { atx := &types.ActivationTx{} atx.SetID(atxid) - mock.EXPECT().Atx(gomock.Any(), atxid).Return(atx, nil) + mock.atxService.EXPECT().Atx(gomock.Any(), atxid).Return(atx, nil) gotAtx, err := svc.Atx(context.Background(), atxid) require.NoError(t, err) require.Equal(t, atx, gotAtx) }) t.Run("backend errors", func(t *testing.T) { - mock.EXPECT(). + mock.atxService.EXPECT(). Atx(gomock.Any(), atxid). Times(retries+1). Return(nil, errors.New("ops")) @@ -80,14 +94,14 @@ func Test_ActivationService_PositioningATX(t *testing.T) { t.Run("found", func(t *testing.T) { posAtx := types.RandomATXID() - mock.EXPECT().PositioningATX(gomock.Any(), types.EpochID(77)).Return(posAtx, nil) + mock.atxService.EXPECT().PositioningATX(gomock.Any(), types.EpochID(77)).Return(posAtx, nil) gotAtx, err := svc.PositioningATX(context.Background(), 77) require.NoError(t, err) require.Equal(t, posAtx, gotAtx) }) t.Run("backend errors", func(t *testing.T) { - mock.EXPECT(). + mock.atxService.EXPECT(). PositioningATX(gomock.Any(), types.EpochID(77)). Times(retries+1). Return(types.EmptyATXID, errors.New("ops")) @@ -103,7 +117,7 @@ func Test_ActivationService_LastATX(t *testing.T) { nodeid := types.NodeID{5, 6, 7, 8} t.Run("not found", func(t *testing.T) { - mock.EXPECT().LastATX(gomock.Any(), nodeid).Return(nil, common.ErrNotFound) + mock.atxService.EXPECT().LastATX(gomock.Any(), nodeid).Return(nil, common.ErrNotFound) _, err := svc.LastATX(context.Background(), nodeid) require.ErrorIs(t, err, common.ErrNotFound) }) @@ -111,14 +125,14 @@ func Test_ActivationService_LastATX(t *testing.T) { t.Run("found", func(t *testing.T) { atx := &types.ActivationTx{} atx.SetID(atxid) - mock.EXPECT().LastATX(gomock.Any(), nodeid).Return(atx, nil) + mock.atxService.EXPECT().LastATX(gomock.Any(), nodeid).Return(atx, nil) gotAtx, err := svc.LastATX(context.Background(), nodeid) require.NoError(t, err) require.Equal(t, atx, gotAtx) }) t.Run("backend errors", func(t *testing.T) { - mock.EXPECT(). + mock.atxService.EXPECT(). LastATX(gomock.Any(), nodeid). Times(retries+1). Return(nil, errors.New("ops")) @@ -126,3 +140,13 @@ func Test_ActivationService_LastATX(t *testing.T) { require.Error(t, err) }) } + +func Test_StoringPoetProof(t *testing.T) { + svc, mocks := setupE2E(t) + + proof := types.PoetProofMessage{ + RoundID: "some round", + } + mocks.poetDb.EXPECT().ValidateAndStore(gomock.Any(), &proof) + svc.StorePoetProof(context.Background(), &proof) +} diff --git a/api/node/node_service.yaml b/api/node/node_service.yaml index fba19088a3..608a7ee398 100644 --- a/api/node/node_service.yaml +++ b/api/node/node_service.yaml @@ -101,3 +101,24 @@ paths: responses: "200": description: Successfully published the blob + /poet: + post: + summary: Store PoET proof + tags: + - "Poet" + requestBody: + required: true + content: + application/octet-stream: + schema: + type: string + format: binary + responses: + "200": + description: Successfully stored the proof + "400": + description: Bad request + content: + plain/text: + schema: + type: string diff --git a/api/node/server/mocks.go b/api/node/server/mocks.go new file mode 100644 index 0000000000..e6885b33ae --- /dev/null +++ b/api/node/server/mocks.go @@ -0,0 +1,79 @@ +// Code generated by MockGen. DO NOT EDIT. +// Source: server.go +// +// Generated by this command: +// +// mockgen -typed -package=server -destination=mocks.go -source=server.go +// + +// Package server is a generated GoMock package. +package server + +import ( + context "context" + reflect "reflect" + + types "github.com/spacemeshos/go-spacemesh/common/types" + gomock "go.uber.org/mock/gomock" +) + +// MockpoetDB is a mock of poetDB interface. +type MockpoetDB struct { + ctrl *gomock.Controller + recorder *MockpoetDBMockRecorder +} + +// MockpoetDBMockRecorder is the mock recorder for MockpoetDB. +type MockpoetDBMockRecorder struct { + mock *MockpoetDB +} + +// NewMockpoetDB creates a new mock instance. +func NewMockpoetDB(ctrl *gomock.Controller) *MockpoetDB { + mock := &MockpoetDB{ctrl: ctrl} + mock.recorder = &MockpoetDBMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockpoetDB) EXPECT() *MockpoetDBMockRecorder { + return m.recorder +} + +// ValidateAndStore mocks base method. +func (m *MockpoetDB) ValidateAndStore(ctx context.Context, proofMessage *types.PoetProofMessage) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ValidateAndStore", ctx, proofMessage) + ret0, _ := ret[0].(error) + return ret0 +} + +// ValidateAndStore indicates an expected call of ValidateAndStore. +func (mr *MockpoetDBMockRecorder) ValidateAndStore(ctx, proofMessage any) *MockpoetDBValidateAndStoreCall { + mr.mock.ctrl.T.Helper() + call := mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ValidateAndStore", reflect.TypeOf((*MockpoetDB)(nil).ValidateAndStore), ctx, proofMessage) + return &MockpoetDBValidateAndStoreCall{Call: call} +} + +// MockpoetDBValidateAndStoreCall wrap *gomock.Call +type MockpoetDBValidateAndStoreCall struct { + *gomock.Call +} + +// Return rewrite *gomock.Call.Return +func (c *MockpoetDBValidateAndStoreCall) Return(arg0 error) *MockpoetDBValidateAndStoreCall { + c.Call = c.Call.Return(arg0) + return c +} + +// Do rewrite *gomock.Call.Do +func (c *MockpoetDBValidateAndStoreCall) Do(f func(context.Context, *types.PoetProofMessage) error) *MockpoetDBValidateAndStoreCall { + c.Call = c.Call.Do(f) + return c +} + +// DoAndReturn rewrite *gomock.Call.DoAndReturn +func (c *MockpoetDBValidateAndStoreCall) DoAndReturn(f func(context.Context, *types.PoetProofMessage) error) *MockpoetDBValidateAndStoreCall { + c.Call = c.Call.DoAndReturn(f) + return c +} diff --git a/api/node/server/server.gen.go b/api/node/server/server.gen.go index 95c4c2b2c3..64ed4825d0 100644 --- a/api/node/server/server.gen.go +++ b/api/node/server/server.gen.go @@ -45,6 +45,9 @@ type ServerInterface interface { // Get Positioning ATX ID with given maximum publish epoch // (GET /activation/positioning_atx/{publish_epoch}) GetActivationPositioningAtxPublishEpoch(w http.ResponseWriter, r *http.Request, publishEpoch externalRef0.EpochID) + // Store PoET proof + // (POST /poet) + PostPoet(w http.ResponseWriter, r *http.Request) // Publish a blob in the given p2p protocol // (POST /publish/{protocol}) PostPublishProtocol(w http.ResponseWriter, r *http.Request, protocol PostPublishProtocolParamsProtocol) @@ -134,6 +137,20 @@ func (siw *ServerInterfaceWrapper) GetActivationPositioningAtxPublishEpoch(w htt handler.ServeHTTP(w, r) } +// PostPoet operation middleware +func (siw *ServerInterfaceWrapper) PostPoet(w http.ResponseWriter, r *http.Request) { + + handler := http.Handler(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + siw.Handler.PostPoet(w, r) + })) + + for _, middleware := range siw.HandlerMiddlewares { + handler = middleware(handler) + } + + handler.ServeHTTP(w, r) +} + // PostPublishProtocol operation middleware func (siw *ServerInterfaceWrapper) PostPublishProtocol(w http.ResponseWriter, r *http.Request) { @@ -282,6 +299,7 @@ func HandlerWithOptions(si ServerInterface, options StdHTTPServerOptions) http.H m.HandleFunc("GET "+options.BaseURL+"/activation/atx/{atx_id}", wrapper.GetActivationAtxAtxId) m.HandleFunc("GET "+options.BaseURL+"/activation/last_atx/{node_id}", wrapper.GetActivationLastAtxNodeId) m.HandleFunc("GET "+options.BaseURL+"/activation/positioning_atx/{publish_epoch}", wrapper.GetActivationPositioningAtxPublishEpoch) + m.HandleFunc("POST "+options.BaseURL+"/poet", wrapper.PostPoet) m.HandleFunc("POST "+options.BaseURL+"/publish/{protocol}", wrapper.PostPublishProtocol) return m @@ -375,6 +393,41 @@ func (response GetActivationPositioningAtxPublishEpoch200JSONResponse) VisitGetA return json.NewEncoder(w).Encode(response) } +type PostPoetRequestObject struct { + Body io.Reader +} + +type PostPoetResponseObject interface { + VisitPostPoetResponse(w http.ResponseWriter) error +} + +type PostPoet200Response struct { +} + +func (response PostPoet200Response) VisitPostPoetResponse(w http.ResponseWriter) error { + w.WriteHeader(200) + return nil +} + +type PostPoet400PlaintextResponse struct { + Body io.Reader + ContentLength int64 +} + +func (response PostPoet400PlaintextResponse) VisitPostPoetResponse(w http.ResponseWriter) error { + w.Header().Set("Content-Type", "plain/text") + if response.ContentLength != 0 { + w.Header().Set("Content-Length", fmt.Sprint(response.ContentLength)) + } + w.WriteHeader(400) + + if closer, ok := response.Body.(io.ReadCloser); ok { + defer closer.Close() + } + _, err := io.Copy(w, response.Body) + return err +} + type PostPublishProtocolRequestObject struct { Protocol PostPublishProtocolParamsProtocol `json:"protocol"` Body io.Reader @@ -403,6 +456,9 @@ type StrictServerInterface interface { // Get Positioning ATX ID with given maximum publish epoch // (GET /activation/positioning_atx/{publish_epoch}) GetActivationPositioningAtxPublishEpoch(ctx context.Context, request GetActivationPositioningAtxPublishEpochRequestObject) (GetActivationPositioningAtxPublishEpochResponseObject, error) + // Store PoET proof + // (POST /poet) + PostPoet(ctx context.Context, request PostPoetRequestObject) (PostPoetResponseObject, error) // Publish a blob in the given p2p protocol // (POST /publish/{protocol}) PostPublishProtocol(ctx context.Context, request PostPublishProtocolRequestObject) (PostPublishProtocolResponseObject, error) @@ -515,6 +571,32 @@ func (sh *strictHandler) GetActivationPositioningAtxPublishEpoch(w http.Response } } +// PostPoet operation middleware +func (sh *strictHandler) PostPoet(w http.ResponseWriter, r *http.Request) { + var request PostPoetRequestObject + + request.Body = r.Body + + handler := func(ctx context.Context, w http.ResponseWriter, r *http.Request, request interface{}) (interface{}, error) { + return sh.ssi.PostPoet(ctx, request.(PostPoetRequestObject)) + } + for _, middleware := range sh.middlewares { + handler = middleware(handler, "PostPoet") + } + + response, err := handler(r.Context(), w, r, request) + + if err != nil { + sh.options.ResponseErrorHandlerFunc(w, r, err) + } else if validResponse, ok := response.(PostPoetResponseObject); ok { + if err := validResponse.VisitPostPoetResponse(w); err != nil { + sh.options.ResponseErrorHandlerFunc(w, r, err) + } + } else if response != nil { + sh.options.ResponseErrorHandlerFunc(w, r, fmt.Errorf("unexpected response type: %T", response)) + } +} + // PostPublishProtocol operation middleware func (sh *strictHandler) PostPublishProtocol(w http.ResponseWriter, r *http.Request, protocol PostPublishProtocolParamsProtocol) { var request PostPublishProtocolRequestObject diff --git a/api/node/server/server.go b/api/node/server/server.go index 95b3db10fd..5a1c9589e8 100644 --- a/api/node/server/server.go +++ b/api/node/server/server.go @@ -14,24 +14,38 @@ import ( "github.com/spacemeshos/go-spacemesh/activation" "github.com/spacemeshos/go-spacemesh/api/node/models" + "github.com/spacemeshos/go-spacemesh/codec" "github.com/spacemeshos/go-spacemesh/common" "github.com/spacemeshos/go-spacemesh/common/types" "github.com/spacemeshos/go-spacemesh/p2p/pubsub" ) +//go:generate mockgen -typed -package=server -destination=mocks.go -source=server.go + +type poetDB interface { + ValidateAndStore(ctx context.Context, proofMessage *types.PoetProofMessage) error +} + type Server struct { - activationService activation.AtxService - publisher pubsub.Publisher - logger *zap.Logger + atxService activation.AtxService + publisher pubsub.Publisher + poetDB poetDB + logger *zap.Logger } var _ StrictServerInterface = (*Server)(nil) -func NewServer(activationService activation.AtxService, publisher pubsub.PubSub, logger *zap.Logger) *Server { +func NewServer( + atxService activation.AtxService, + publisher pubsub.Publisher, + poetDB poetDB, + logger *zap.Logger, +) *Server { return &Server{ - activationService: activationService, - publisher: publisher, - logger: logger, + atxService: atxService, + publisher: publisher, + poetDB: poetDB, + logger: logger, } } @@ -77,7 +91,7 @@ func (s *Server) GetActivationAtxAtxId( return nil, err } id := types.BytesToATXID(idBytes) - atx, err := s.activationService.Atx(ctx, id) + atx, err := s.atxService.Atx(ctx, id) switch { case errors.Is(err, common.ErrNotFound): return GetActivationAtxAtxId404Response{}, nil @@ -110,7 +124,7 @@ func (s *Server) GetActivationLastAtxNodeId( }, nil } - atxid, err := s.activationService.LastATX(ctx, id) + atxid, err := s.atxService.LastATX(ctx, id) switch { case errors.Is(err, common.ErrNotFound): return GetActivationLastAtxNodeId404Response{}, nil @@ -134,7 +148,7 @@ func (s *Server) GetActivationPositioningAtxPublishEpoch( ctx context.Context, request GetActivationPositioningAtxPublishEpochRequestObject, ) (GetActivationPositioningAtxPublishEpochResponseObject, error) { - id, err := s.activationService.PositioningATX(ctx, types.EpochID(request.PublishEpoch)) + id, err := s.atxService.PositioningATX(ctx, types.EpochID(request.PublishEpoch)) if err != nil { return nil, err } @@ -157,3 +171,29 @@ func (s *Server) PostPublishProtocol( s.publisher.Publish(ctx, string(request.Protocol), blob) return PostPublishProtocol200Response{}, nil } + +// PostPoet implements StrictServerInterface. +func (s *Server) PostPoet(ctx context.Context, request PostPoetRequestObject) (PostPoetResponseObject, error) { + var proof types.PoetProofMessage + blob, err := io.ReadAll(request.Body) + if err != nil { + return nil, err + } + + if err := codec.Decode(blob, &proof); err != nil { + msg := err.Error() + return PostPoet400PlaintextResponse{ + Body: bytes.NewBuffer([]byte(msg)), + ContentLength: int64(len(msg)), + }, nil + } + + if err := s.poetDB.ValidateAndStore(ctx, &proof); err != nil { + msg := err.Error() + return PostPoet400PlaintextResponse{ + Body: bytes.NewBuffer([]byte(msg)), + ContentLength: int64(len(msg)), + }, nil + } + return PostPoet200Response{}, nil +} diff --git a/node/node.go b/node/node.go index b44a36bae8..309c0b8a12 100644 --- a/node/node.go +++ b/node/node.go @@ -38,6 +38,7 @@ import ( "github.com/spacemeshos/go-spacemesh/activation" "github.com/spacemeshos/go-spacemesh/api/grpcserver" "github.com/spacemeshos/go-spacemesh/api/grpcserver/v2alpha1" + "github.com/spacemeshos/go-spacemesh/api/node/client" nodeclient "github.com/spacemeshos/go-spacemesh/api/node/client" nodeserver "github.com/spacemeshos/go-spacemesh/api/node/server" "github.com/spacemeshos/go-spacemesh/atxsdata" @@ -583,7 +584,26 @@ func (app *App) initServices(ctx context.Context) error { layersPerEpoch := types.GetLayersPerEpoch() lg := app.log - poetDb := activation.NewPoetDb(app.db, app.addLogger(PoetDbLogger, lg).Zap()) + var nodeServiceClient *client.NodeService + if server := app.Config.BaseConfig.NodeServiceAddress; server != "" { + logger := app.log.Zap().Named("node-svc-client") + cfg := &nodeclient.Config{ + RetryWaitMin: time.Millisecond * 500, + RetryWaitMax: time.Second, + RetryMax: 10, + } + var err error + nodeServiceClient, err = nodeclient.NewNodeServiceClient(server, logger, cfg) + if err != nil { + return fmt.Errorf("creating node service client: %w", err) + } + } + + var poetDbOpts []activation.PoetDbOption + if nodeServiceClient != nil { + poetDbOpts = append(poetDbOpts, activation.WithRemotePoetStorer(nodeServiceClient)) + } + poetDb := activation.NewPoetDb(app.db, app.addLogger(PoetDbLogger, lg).Zap(), poetDbOpts...) postStates := activation.NewPostStates(app.addLogger(PostLogger, lg).Zap()) opts := []activation.PostVerifierOpt{ activation.WithVerifyingOpts(app.Config.SMESHING.VerifyingOpts), @@ -1067,17 +1087,7 @@ func (app *App) initServices(ctx context.Context) error { atxService activation.AtxService atxPublisher pubsub.Publisher ) - if server := app.Config.BaseConfig.NodeServiceAddress; server != "" { - logger := app.log.Zap().Named("node-svc-client") - cfg := &nodeclient.Config{ - RetryWaitMin: time.Millisecond * 500, - RetryWaitMax: time.Second, - RetryMax: 10, - } - nodeServiceClient, err := nodeclient.NewNodeServiceClient(server, logger, cfg) - if err != nil { - return fmt.Errorf("creating node service client: %w", err) - } + if nodeServiceClient != nil { atxService = nodeServiceClient atxPublisher = nodeServiceClient } else { @@ -1855,7 +1865,7 @@ func (app *App) startAPIServices(ctx context.Context) error { golden := types.ATXID(app.Config.Genesis.GoldenATX()) logger := app.log.Zap().Named("atx-service") actSvc := activation.NewDBAtxService(app.db, golden, app.atxsdata, app.validator, logger) - server := nodeserver.NewServer(actSvc, app.host, logger) + server := nodeserver.NewServer(actSvc, app.host, app.poetDb, logger) app.nodeServiceServer = &http.Server{ Handler: server.IntoHandler(http.NewServeMux()), From 89e7730a6abc7e159dba42e0f63d1aa5a4eb308c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bartosz=20R=C3=B3=C5=BCa=C5=84ski?= Date: Thu, 26 Sep 2024 15:45:26 +0200 Subject: [PATCH 04/10] expose --node-service-address for configuring over CLI --- cmd/root.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/cmd/root.go b/cmd/root.go index 47645ef313..4a30afd183 100644 --- a/cmd/root.go +++ b/cmd/root.go @@ -84,6 +84,8 @@ func AddFlags(flagSet *pflag.FlagSet, cfg *config.Config) (configPath *string) { flagSet.BoolVar(&cfg.NoMainOverride, "no-main-override", cfg.NoMainOverride, "force 'nomain' builds to run on the mainnet") + flagSet.StringVar(&cfg.NodeServiceAddress, "node-service-address", "", "address of the Node Service to connect to") + /** ======================== P2P Flags ========================== **/ flagSet.Var(flags.NewAddressListValue(cfg.P2P.Listen, &cfg.P2P.Listen), From 9a093742179cd135c833b133790668ad4b6d8552 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bartosz=20R=C3=B3=C5=BCa=C5=84ski?= Date: Thu, 26 Sep 2024 15:53:09 +0200 Subject: [PATCH 05/10] docker-compose for executing the PoC --- .../config.standalone.client.json | 31 +++++++++++++++++++ .../config.standalone.node-service.json | 23 ++++++++++++++ activation_service_poc/docker-compose.yml | 24 ++++++++++++++ 3 files changed, 78 insertions(+) create mode 100644 activation_service_poc/config.standalone.client.json create mode 100644 activation_service_poc/config.standalone.node-service.json create mode 100644 activation_service_poc/docker-compose.yml diff --git a/activation_service_poc/config.standalone.client.json b/activation_service_poc/config.standalone.client.json new file mode 100644 index 0000000000..ce162cca95 --- /dev/null +++ b/activation_service_poc/config.standalone.client.json @@ -0,0 +1,31 @@ +{ + "preset": "standalone", + "api": { + "grpc-public-listener": "0.0.0.0:9082", + "grpc-private-listener": "0.0.0.0:9083" + }, + "genesis": { + "genesis-time": "2024-09-25T13:00:00.000Z" + }, + "logging": { + "trtl": "WARN", + "beacon": "ERROR", + "proposalBuilder": "ERROR", + "atxBuilder": "DEBUG" + }, + "main": { + "node-service-address": "http://0.0.0.0:9099", + "data-folder": "/tmp/spacemesh-client", + "filelock": "/tmp/spacemesh-client/node.lock", + "poet-servers": [ + { + "address": "http://127.0.0.1:10011" + } + ] + }, + "smeshing": { + "smeshing-opts": { + "smeshing-opts-datadir": "/tmp/spacemesh-client/post-data" + } + } +} diff --git a/activation_service_poc/config.standalone.node-service.json b/activation_service_poc/config.standalone.node-service.json new file mode 100644 index 0000000000..99d10328af --- /dev/null +++ b/activation_service_poc/config.standalone.node-service.json @@ -0,0 +1,23 @@ +{ + "preset": "standalone", + "api": { + "grpc-public-listener": "0.0.0.0:9092", + "grpc-private-listener": "0.0.0.0:9093", + "node-service-listener": "0.0.0.0:9099" + }, + "genesis": { + "genesis-time": "2024-09-25T13:00:00.000Z" + }, + "logging": { + "trtl": "WARN", + "beacon": "ERROR", + "proposalBuilder": "ERROR" + }, + "main": { + "data-folder": "/tmp/spacemesh-node-service", + "filelock": "/tmp/spacemesh-node-service/node.lock" + }, + "smeshing": { + "smeshing-start": false + } +} diff --git a/activation_service_poc/docker-compose.yml b/activation_service_poc/docker-compose.yml new file mode 100644 index 0000000000..edbb5b0e76 --- /dev/null +++ b/activation_service_poc/docker-compose.yml @@ -0,0 +1,24 @@ +version: '3' + +services: + activation-service: + image: spacemeshos/go-spacemesh-dev:activation-service-poc.0 + command: ["-c", "/config.json", "--node-service-address", "http://node-service:9099"] + volumes: + - /tmp/spacemesh-client:/tmp/spacemesh-client + - ./config.standalone.client.json:/config.json + networks: + - spacemesh-net + + node-service: + image: spacemeshos/go-spacemesh-dev:activation-service-poc.0 + command: ["-c", "/config.json"] + volumes: + - /tmp/spacemesh-node-service:/tmp/spacemesh-node-service + - ./config.standalone.node-service.json:/config.json + networks: + - spacemesh-net + +networks: + spacemesh-net: + driver: bridge From cbcb7daa0a0d0ad89d2c1e705a7f3db24ebe662f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bartosz=20R=C3=B3=C5=BCa=C5=84ski?= Date: Thu, 3 Oct 2024 13:57:56 +0200 Subject: [PATCH 06/10] review feedback --- activation/activation.go | 24 ++++++++------------ activation/atx_service_db.go | 9 +++++++- activation/validation.go | 9 ++++---- activation/validation_test.go | 2 +- node/node.go | 6 +++++ sql/localsql/schema/migrations/0010_atxs.sql | 3 +-- sql/localsql/schema/schema.sql | 3 +-- 7 files changed, 32 insertions(+), 24 deletions(-) diff --git a/activation/activation.go b/activation/activation.go index 934b3624f6..5769f600ae 100644 --- a/activation/activation.go +++ b/activation/activation.go @@ -113,14 +113,16 @@ type Builder struct { stop context.CancelFunc } +type foundPosAtx struct { + id types.ATXID + forPublish types.EpochID +} + type positioningAtxFinder struct { finding sync.Mutex - found *struct { - id types.ATXID - forPublish types.EpochID - } - golden types.ATXID - logger *zap.Logger + found *foundPosAtx + golden types.ATXID + logger *zap.Logger } type BuilderOption func(*Builder) @@ -926,19 +928,13 @@ func (f *positioningAtxFinder) find( id, err := atxs.PositioningATX(ctx, publish-1) if err != nil { logger.Warn("failed to get positioning ATX - falling back to golden", zap.Error(err)) - f.found = &struct { - id types.ATXID - forPublish types.EpochID - }{f.golden, publish} + f.found = &foundPosAtx{f.golden, publish} return f.golden } logger.Debug("found candidate positioning atx", log.ZShortStringer("id", id)) - f.found = &struct { - id types.ATXID - forPublish types.EpochID - }{id, publish} + f.found = &foundPosAtx{id, publish} return id } diff --git a/activation/atx_service_db.go b/activation/atx_service_db.go index 94e40b17ba..e84713aa6e 100644 --- a/activation/atx_service_db.go +++ b/activation/atx_service_db.go @@ -27,6 +27,7 @@ type dbAtxService struct { type dbAtxServiceConfig struct { // delay before PoST in ATX is considered valid (counting from the time it was received) postValidityDelay time.Duration + trusted []types.NodeID } type dbAtxServiceOption func(*dbAtxServiceConfig) @@ -37,6 +38,12 @@ func WithPostValidityDelay(delay time.Duration) dbAtxServiceOption { } } +func WithTrustedIDs(ids ...types.NodeID) dbAtxServiceOption { + return func(cfg *dbAtxServiceConfig) { + cfg.trusted = ids + } +} + func NewDBAtxService( db sql.Executor, golden types.ATXID, @@ -91,7 +98,7 @@ func (s *dbAtxService) PositioningATX(ctx context.Context, maxPublish types.Epoc s.golden, s.validator, s.logger, VerifyChainOpts.AssumeValidBefore(time.Now().Add(-s.cfg.postValidityDelay)), - // VerifyChainOpts.WithTrustedID(nodeID), + VerifyChainOpts.WithTrustedIDs(s.cfg.trusted...), VerifyChainOpts.WithLogger(s.logger), ) if err != nil { diff --git a/activation/validation.go b/activation/validation.go index d6d070d895..b31a3ec786 100644 --- a/activation/validation.go +++ b/activation/validation.go @@ -4,6 +4,7 @@ import ( "context" "errors" "fmt" + "slices" "time" "github.com/spacemeshos/merkle-tree" @@ -360,7 +361,7 @@ func (v *Validator) PositioningAtx( type verifyChainOpts struct { assumedValidTime time.Time - trustedNodeID types.NodeID + trustedNodeID []types.NodeID logger *zap.Logger } @@ -377,8 +378,8 @@ func (verifyChainOptsNs) AssumeValidBefore(val time.Time) VerifyChainOption { } } -// WithTrustedID configures the validator to assume that ATXs created by the given node ID are valid. -func (verifyChainOptsNs) WithTrustedID(val types.NodeID) VerifyChainOption { +// WithTrustedIDs configures the validator to assume that ATXs created by the given node IDs are valid. +func (verifyChainOptsNs) WithTrustedIDs(val ...types.NodeID) VerifyChainOption { return func(o *verifyChainOpts) { o.trustedNodeID = val } @@ -533,7 +534,7 @@ func (v *Validator) verifyChainWithOpts( zap.Time("valid_before", opts.assumedValidTime), ) return nil - case atx.SmesherID == opts.trustedNodeID: + case slices.Contains(opts.trustedNodeID, atx.SmesherID): log.Debug("not verifying ATX chain", zap.Stringer("atx_id", id), zap.String("reason", "trusted")) return nil } diff --git a/activation/validation_test.go b/activation/validation_test.go index 09c5d7c5ed..169a227007 100644 --- a/activation/validation_test.go +++ b/activation/validation_test.go @@ -543,7 +543,7 @@ func TestVerifyChainDeps(t *testing.T) { ctrl := gomock.NewController(t) v := NewMockPostVerifier(ctrl) validator := NewValidator(db, nil, DefaultPostConfig(), config.ScryptParams{}, v) - err = validator.VerifyChain(ctx, vAtx.ID(), goldenATXID, VerifyChainOpts.WithTrustedID(signer.NodeID())) + err = validator.VerifyChain(ctx, vAtx.ID(), goldenATXID, VerifyChainOpts.WithTrustedIDs(signer.NodeID())) require.NoError(t, err) }) diff --git a/node/node.go b/node/node.go index 2c43cad4e9..3543cafc95 100644 --- a/node/node.go +++ b/node/node.go @@ -1071,6 +1071,11 @@ func (app *App) initServices(ctx context.Context) error { } atxBuilderLog := app.addLogger(ATXBuilderLogger, lg).Zap() + trustedIDs := make([]types.NodeID, 0, len(app.signers)) + for _, sig := range app.signers { + trustedIDs = append(trustedIDs, sig.NodeID()) + } + atxService := activation.NewDBAtxService( app.db, goldenATXID, @@ -1078,6 +1083,7 @@ func (app *App) initServices(ctx context.Context) error { app.validator, atxBuilderLog, activation.WithPostValidityDelay(app.Config.PostValidDelay), + activation.WithTrustedIDs(trustedIDs...), ) atxBuilder := activation.NewBuilder( builderConfig, diff --git a/sql/localsql/schema/migrations/0010_atxs.sql b/sql/localsql/schema/migrations/0010_atxs.sql index 19e9798333..97ead34ce4 100644 --- a/sql/localsql/schema/migrations/0010_atxs.sql +++ b/sql/localsql/schema/migrations/0010_atxs.sql @@ -1,12 +1,11 @@ --- Table for storing blobs of published ATX for regossiping purposes. CREATE TABLE atx_blobs ( - id CHAR(32), + id CHAR(32) PRIMARY KEY, pubkey CHAR(32) NOT NULL, epoch INT NOT NULL, atx BLOB, version INTEGER ); -CREATE UNIQUE INDEX atx_blobs_id ON atx_blobs (id); CREATE UNIQUE INDEX atx_blobs_epoch_pubkey ON atx_blobs (epoch, pubkey); diff --git a/sql/localsql/schema/schema.sql b/sql/localsql/schema/schema.sql index 2c9f4dd1c6..5b9cb31601 100755 --- a/sql/localsql/schema/schema.sql +++ b/sql/localsql/schema/schema.sql @@ -1,14 +1,13 @@ PRAGMA user_version = 10; CREATE TABLE atx_blobs ( - id CHAR(32), + id CHAR(32) PRIMARY KEY, pubkey CHAR(32) NOT NULL, epoch INT NOT NULL, atx BLOB, version INTEGER ); CREATE UNIQUE INDEX atx_blobs_epoch_pubkey ON atx_blobs (epoch, pubkey); -CREATE UNIQUE INDEX atx_blobs_id ON atx_blobs (id); CREATE TABLE atx_sync_requests ( epoch INT NOT NULL, From e6a9f2ae49d600dd825446621d14aae331fd3d08 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bartosz=20R=C3=B3=C5=BCa=C5=84ski?= Date: Mon, 7 Oct 2024 11:49:22 +0200 Subject: [PATCH 07/10] review feedback: remove common error --- activation/activation.go | 16 ++++++---------- activation/activation_test.go | 2 +- activation/atx_service_db.go | 12 ++++++++++-- activation/interface.go | 9 +++++++++ common/errors.go | 5 ----- sql/database.go | 3 +-- 6 files changed, 27 insertions(+), 20 deletions(-) delete mode 100644 common/errors.go diff --git a/activation/activation.go b/activation/activation.go index 5769f600ae..69222bced6 100644 --- a/activation/activation.go +++ b/activation/activation.go @@ -19,7 +19,6 @@ import ( "github.com/spacemeshos/go-spacemesh/activation/metrics" "github.com/spacemeshos/go-spacemesh/activation/wire" "github.com/spacemeshos/go-spacemesh/codec" - "github.com/spacemeshos/go-spacemesh/common" "github.com/spacemeshos/go-spacemesh/common/types" "github.com/spacemeshos/go-spacemesh/events" "github.com/spacemeshos/go-spacemesh/log" @@ -31,10 +30,7 @@ import ( "github.com/spacemeshos/go-spacemesh/sql/localsql/nipost" ) -var ( - ErrNotFound = errors.New("not found") - errNilVrfNonce = errors.New("nil VRF nonce") -) +var errNilVrfNonce = errors.New("nil VRF nonce") // PoetConfig is the configuration to interact with the poet server. type PoetConfig struct { @@ -351,7 +347,7 @@ func (b *Builder) BuildInitialPost(ctx context.Context, nodeID types.NodeID) err case err == nil: b.logger.Info("load initial post from db") return nil - case errors.Is(err, common.ErrNotFound): + case errors.Is(err, sql.ErrNotFound): b.logger.Info("creating initial post") default: return fmt.Errorf("get initial post: %w", err) @@ -527,7 +523,7 @@ func (b *Builder) BuildNIPostChallenge(ctx context.Context, nodeID types.NodeID) switch { case err == nil: currentEpochId = max(currentEpochId, prevAtx.PublishEpoch) - case errors.Is(err, common.ErrNotFound): + case errors.Is(err, ErrNotFound): // no previous ATX case err != nil: return nil, fmt.Errorf("get last ATX: %w", err) @@ -576,7 +572,7 @@ func (b *Builder) BuildNIPostChallenge(ctx context.Context, nodeID types.NodeID) var challenge *types.NIPostChallenge switch { - case errors.Is(err, common.ErrNotFound): + case errors.Is(err, ErrNotFound): logger.Info("no previous ATX found, creating an initial nipost challenge") challenge, err = b.buildInitialNIPostChallenge(ctx, logger, nodeID, publishEpochId) @@ -614,7 +610,7 @@ func (b *Builder) getExistingChallenge( challenge, err := nipost.Challenge(b.localDB, nodeID) switch { - case errors.Is(err, common.ErrNotFound): + case errors.Is(err, sql.ErrNotFound): return nil, nil case err != nil: @@ -986,7 +982,7 @@ func (b *Builder) getPositioningAtx( func (b *Builder) Regossip(ctx context.Context, nodeID types.NodeID) error { epoch := b.layerClock.CurrentLayer().GetEpoch() id, blob, err := atxs.AtxBlob(b.localDB, epoch, nodeID) - if errors.Is(err, common.ErrNotFound) { + if errors.Is(err, sql.ErrNotFound) { return nil } else if err != nil { return err diff --git a/activation/activation_test.go b/activation/activation_test.go index 620e5afb5a..6d728a4e5c 100644 --- a/activation/activation_test.go +++ b/activation/activation_test.go @@ -894,7 +894,7 @@ func TestBuilder_PublishActivationTx_TargetsEpochBasedOnPosAtx(t *testing.T) { } posAtx.SetID(types.RandomATXID()) atxSvc.EXPECT().PositioningATX(gomock.Any(), gomock.Any()).Return(posAtx.ID(), nil) - atxSvc.EXPECT().LastATX(gomock.Any(), sig.NodeID()).Return(nil, sql.ErrNotFound).Times(2) + atxSvc.EXPECT().LastATX(gomock.Any(), sig.NodeID()).Return(nil, ErrNotFound).Times(2) // Act & Assert tab.msync.EXPECT().RegisterForATXSynced().DoAndReturn(closedChan).AnyTimes() diff --git a/activation/atx_service_db.go b/activation/atx_service_db.go index e84713aa6e..a2cf4fd5fb 100644 --- a/activation/atx_service_db.go +++ b/activation/atx_service_db.go @@ -2,6 +2,7 @@ package activation import ( "context" + "errors" "fmt" "time" @@ -71,12 +72,19 @@ func NewDBAtxService( } func (s *dbAtxService) Atx(_ context.Context, id types.ATXID) (*types.ActivationTx, error) { - return atxs.Get(s.db, id) + atx, err := atxs.Get(s.db, id) + if errors.Is(err, sql.ErrNotFound) { + return nil, ErrNotFound + } + return atx, err } func (s *dbAtxService) LastATX(ctx context.Context, id types.NodeID) (*types.ActivationTx, error) { atxid, err := atxs.GetLastIDByNodeID(s.db, id) - if err != nil { + switch { + case errors.Is(err, sql.ErrNotFound): + return nil, ErrNotFound + case err != nil: return nil, fmt.Errorf("getting last ATXID: %w", err) } return atxs.Get(s.db, atxid) diff --git a/activation/interface.go b/activation/interface.go index 092f3da7fc..c5c7c138a0 100644 --- a/activation/interface.go +++ b/activation/interface.go @@ -19,6 +19,8 @@ import ( //go:generate mockgen -typed -package=activation -destination=./mocks.go -source=./interface.go +var ErrNotFound = errors.New("not found") + type AtxReceiver interface { OnAtx(*types.ActivationTx) } @@ -110,9 +112,16 @@ type atxProvider interface { // AtxService provides ATXs needed by the ATX Builder. type AtxService interface { + // Get ATX with given ID + // + // Returns ErrNotFound if couldn't get the ATX. Atx(ctx context.Context, id types.ATXID) (*types.ActivationTx, error) + // Get the last ATX of the given identitity. + // + // Returns ErrNotFound if couldn't get the ATX. LastATX(ctx context.Context, nodeID types.NodeID) (*types.ActivationTx, error) // PositioningATX returns atx id with the highest tick height. + // // The maxPublish epoch is the maximum publish epoch of the returned ATX. PositioningATX(ctx context.Context, maxPublish types.EpochID) (types.ATXID, error) } diff --git a/common/errors.go b/common/errors.go deleted file mode 100644 index 4342b6fa07..0000000000 --- a/common/errors.go +++ /dev/null @@ -1,5 +0,0 @@ -package common - -import "errors" - -var ErrNotFound = errors.New("not found") diff --git a/sql/database.go b/sql/database.go index 17e9cdd8aa..a647086f4a 100644 --- a/sql/database.go +++ b/sql/database.go @@ -18,7 +18,6 @@ import ( "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" - "github.com/spacemeshos/go-spacemesh/common" "github.com/spacemeshos/go-spacemesh/common/types" ) @@ -28,7 +27,7 @@ var ( // ErrNoConnection is returned if pooled connection is not available. ErrNoConnection = errors.New("database: no free connection") // ErrNotFound is returned if requested record is not found. - ErrNotFound = fmt.Errorf("database: %w", common.ErrNotFound) + ErrNotFound = errors.New("database: not found") // ErrObjectExists is returned if database constraints didn't allow to insert an object. ErrObjectExists = errors.New("database: object exists") // ErrTooNew is returned if database version is newer than expected. From 378063b0b63958d87561bd339a5dcf2e5b2b5b58 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bartosz=20R=C3=B3=C5=BCa=C5=84ski?= Date: Mon, 14 Oct 2024 15:35:57 +0200 Subject: [PATCH 08/10] review feedback --- activation/activation.go | 6 ++-- activation/activation_multi_test.go | 2 +- activation/atx_service_db.go | 8 +---- activation/atx_service_db_test.go | 6 ++-- sql/localsql/{atxs => localatxs}/atxs.go | 2 +- sql/localsql/localatxs/atxs_test.go | 41 ++++++++++++++++++++++++ sql/localsql/schema/schema.sql | 2 +- 7 files changed, 50 insertions(+), 17 deletions(-) rename sql/localsql/{atxs => localatxs}/atxs.go (98%) create mode 100644 sql/localsql/localatxs/atxs_test.go diff --git a/activation/activation.go b/activation/activation.go index 69222bced6..2201c828bd 100644 --- a/activation/activation.go +++ b/activation/activation.go @@ -26,7 +26,7 @@ import ( "github.com/spacemeshos/go-spacemesh/p2p/pubsub" "github.com/spacemeshos/go-spacemesh/signing" "github.com/spacemeshos/go-spacemesh/sql" - "github.com/spacemeshos/go-spacemesh/sql/localsql/atxs" + "github.com/spacemeshos/go-spacemesh/sql/localsql/localatxs" "github.com/spacemeshos/go-spacemesh/sql/localsql/nipost" ) @@ -731,7 +731,7 @@ func (b *Builder) PublishActivationTx(ctx context.Context, sig *signing.EdSigner case <-b.layerClock.AwaitLayer(challenge.PublishEpoch.FirstLayer()): } - err = atxs.AddBlob(b.localDB, challenge.PublishEpoch, atx.ID(), sig.NodeID(), codec.MustEncode(atx)) + err = localatxs.AddBlob(b.localDB, challenge.PublishEpoch, atx.ID(), sig.NodeID(), codec.MustEncode(atx)) if err != nil { b.logger.Warn("failed to persist built ATX into the local DB - regossiping won't work", zap.Error(err)) } @@ -981,7 +981,7 @@ func (b *Builder) getPositioningAtx( func (b *Builder) Regossip(ctx context.Context, nodeID types.NodeID) error { epoch := b.layerClock.CurrentLayer().GetEpoch() - id, blob, err := atxs.AtxBlob(b.localDB, epoch, nodeID) + id, blob, err := localatxs.AtxBlob(b.localDB, epoch, nodeID) if errors.Is(err, sql.ErrNotFound) { return nil } else if err != nil { diff --git a/activation/activation_multi_test.go b/activation/activation_multi_test.go index 56736a7c51..064dc4a98d 100644 --- a/activation/activation_multi_test.go +++ b/activation/activation_multi_test.go @@ -18,7 +18,7 @@ import ( "github.com/spacemeshos/go-spacemesh/p2p/pubsub" "github.com/spacemeshos/go-spacemesh/signing" "github.com/spacemeshos/go-spacemesh/sql" - localatxs "github.com/spacemeshos/go-spacemesh/sql/localsql/atxs" + "github.com/spacemeshos/go-spacemesh/sql/localsql/localatxs" "github.com/spacemeshos/go-spacemesh/sql/localsql/nipost" ) diff --git a/activation/atx_service_db.go b/activation/atx_service_db.go index a2cf4fd5fb..26e11ad7ae 100644 --- a/activation/atx_service_db.go +++ b/activation/atx_service_db.go @@ -99,7 +99,7 @@ func (s *dbAtxService) PositioningATX(ctx context.Context, maxPublish types.Epoc // positioning ATX publish epoch must be lower than the publish epoch of built ATX positioningAtxPublished := min(latestPublished, maxPublish) - id, err := findFullyValidHighTickAtx( + return findFullyValidHighTickAtx( ctx, s.atxsdata, positioningAtxPublished, @@ -109,12 +109,6 @@ func (s *dbAtxService) PositioningATX(ctx context.Context, maxPublish types.Epoc VerifyChainOpts.WithTrustedIDs(s.cfg.trusted...), VerifyChainOpts.WithLogger(s.logger), ) - if err != nil { - s.logger.Info("search failed - using golden atx as positioning atx", zap.Error(err)) - id = s.golden - } - - return id, nil } func findFullyValidHighTickAtx( diff --git a/activation/atx_service_db_test.go b/activation/atx_service_db_test.go index f2e916a595..9ed32db65c 100644 --- a/activation/atx_service_db_test.go +++ b/activation/atx_service_db_test.go @@ -73,10 +73,8 @@ func TestGetPositioningAtxPicksAtxWithValidChain(t *testing.T) { require.NoError(t, err) require.Equal(t, vValidAtx.ID(), posAtxID) - // it returns the golden ATX if couldn't find a better one - posAtxID, err = atxSvc.PositioningATX(context.Background(), validAtx.PublishEpoch-1) - require.NoError(t, err) - require.Equal(t, atxSvc.golden, posAtxID) + _, err = atxSvc.PositioningATX(context.Background(), validAtx.PublishEpoch-1) + require.ErrorIs(t, err, ErrNotFound) } func TestFindFullyValidHighTickAtx(t *testing.T) { diff --git a/sql/localsql/atxs/atxs.go b/sql/localsql/localatxs/atxs.go similarity index 98% rename from sql/localsql/atxs/atxs.go rename to sql/localsql/localatxs/atxs.go index 6c0d351d5a..de9eaadee0 100644 --- a/sql/localsql/atxs/atxs.go +++ b/sql/localsql/localatxs/atxs.go @@ -1,4 +1,4 @@ -package atxs +package localatxs import ( "github.com/spacemeshos/go-spacemesh/common/types" diff --git a/sql/localsql/localatxs/atxs_test.go b/sql/localsql/localatxs/atxs_test.go new file mode 100644 index 0000000000..23e77d402e --- /dev/null +++ b/sql/localsql/localatxs/atxs_test.go @@ -0,0 +1,41 @@ +package localatxs_test + +import ( + "testing" + + "github.com/stretchr/testify/require" + + "github.com/spacemeshos/go-spacemesh/common/types" + "github.com/spacemeshos/go-spacemesh/sql" + "github.com/spacemeshos/go-spacemesh/sql/localsql" + "github.com/spacemeshos/go-spacemesh/sql/localsql/localatxs" +) + +func Test_Blobs(t *testing.T) { + t.Run("not found", func(t *testing.T) { + db := localsql.InMemoryTest(t) + _, _, err := localatxs.AtxBlob(db, types.EpochID(0), types.NodeID{}) + require.ErrorIs(t, err, sql.ErrNotFound) + }) + t.Run("found", func(t *testing.T) { + db := localsql.InMemoryTest(t) + epoch := types.EpochID(2) + atxid := types.RandomATXID() + nodeID := types.RandomNodeID() + blob := types.RandomBytes(10) + err := localatxs.AddBlob(db, epoch, atxid, nodeID, blob) + require.NoError(t, err) + gotID, gotBlob, err := localatxs.AtxBlob(db, epoch, nodeID) + require.NoError(t, err) + require.Equal(t, atxid, gotID) + require.Equal(t, blob, gotBlob) + + // different ID + _, _, err = localatxs.AtxBlob(db, epoch, types.RandomNodeID()) + require.ErrorIs(t, err, sql.ErrNotFound) + + // different epoch + _, _, err = localatxs.AtxBlob(db, types.EpochID(3), nodeID) + require.ErrorIs(t, err, sql.ErrNotFound) + }) +} diff --git a/sql/localsql/schema/schema.sql b/sql/localsql/schema/schema.sql index 0d920e9cf7..a3a7333b18 100755 --- a/sql/localsql/schema/schema.sql +++ b/sql/localsql/schema/schema.sql @@ -7,7 +7,6 @@ CREATE TABLE atx_blobs atx BLOB, version INTEGER ); -CREATE UNIQUE INDEX atx_blobs_epoch_pubkey ON atx_blobs (epoch, pubkey); CREATE TABLE atx_sync_requests ( epoch INT NOT NULL, @@ -89,4 +88,5 @@ CREATE TABLE prepared_activeset data BLOB NOT NULL, PRIMARY KEY (kind, epoch) ) WITHOUT ROWID; +CREATE UNIQUE INDEX atx_blobs_epoch_pubkey ON atx_blobs (epoch, pubkey); CREATE UNIQUE INDEX idx_poet_certificates ON poet_certificates (node_id, certifier_id); From fa5c8309d7008960e24bee84e221a28752847ba4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bartosz=20R=C3=B3=C5=BCa=C5=84ski?= Date: Thu, 17 Oct 2024 12:37:52 +0200 Subject: [PATCH 09/10] review feedback: use singleflight --- activation/activation.go | 48 ++++++++++++++++++----------------- activation/activation_test.go | 16 ++++++++++++ 2 files changed, 41 insertions(+), 23 deletions(-) diff --git a/activation/activation.go b/activation/activation.go index 2201c828bd..8f63ce4a1e 100644 --- a/activation/activation.go +++ b/activation/activation.go @@ -15,6 +15,7 @@ import ( "go.uber.org/zap/zapcore" "golang.org/x/exp/maps" "golang.org/x/sync/errgroup" + "golang.org/x/sync/singleflight" "github.com/spacemeshos/go-spacemesh/activation/metrics" "github.com/spacemeshos/go-spacemesh/activation/wire" @@ -115,9 +116,8 @@ type foundPosAtx struct { } type positioningAtxFinder struct { - finding sync.Mutex - found *foundPosAtx - golden types.ATXID + finding singleflight.Group + found foundPosAtx logger *zap.Logger } @@ -193,7 +193,6 @@ func NewBuilder( postStates: NewPostStates(log), versions: []atxVersion{{0, types.AtxV1}}, posAtxFinder: positioningAtxFinder{ - golden: conf.GoldenATXID, logger: log, }, } @@ -910,28 +909,26 @@ func (f *positioningAtxFinder) find( ctx context.Context, atxs AtxService, publish types.EpochID, -) types.ATXID { +) (types.ATXID, error) { logger := f.logger.With(zap.Uint32("publish epoch", publish.Uint32())) - f.finding.Lock() - defer f.finding.Unlock() - - if found := f.found; found != nil && found.forPublish == publish { - logger.Debug("using cached positioning atx", log.ZShortStringer("atx_id", found.id)) - return found.id - } + atx, err, _ := f.finding.Do(publish.String(), func() (any, error) { + if f.found.forPublish == publish { + logger.Debug("using cached positioning atx", log.ZShortStringer("atx_id", f.found.id)) + return f.found.id, nil + } - id, err := atxs.PositioningATX(ctx, publish-1) - if err != nil { - logger.Warn("failed to get positioning ATX - falling back to golden", zap.Error(err)) - f.found = &foundPosAtx{f.golden, publish} - return f.golden - } + id, err := atxs.PositioningATX(ctx, publish-1) + if err != nil { + return types.EmptyATXID, err + } - logger.Debug("found candidate positioning atx", log.ZShortStringer("id", id)) + logger.Debug("found candidate positioning atx", log.ZShortStringer("id", id)) + f.found = foundPosAtx{id, publish} + return id, nil + }) - f.found = &foundPosAtx{id, publish} - return id + return atx.(types.ATXID), err } // getPositioningAtx returns the positioning ATX. @@ -943,7 +940,11 @@ func (b *Builder) getPositioningAtx( publish types.EpochID, previous *types.ActivationTx, ) (types.ATXID, error) { - id := b.posAtxFinder.find(ctx, b.atxSvc, publish) + id, err := b.posAtxFinder.find(ctx, b.atxSvc, publish) + if err != nil { + b.logger.Warn("failed to find positioning ATX - falling back to golden", zap.Error(err)) + id = b.conf.GoldenATXID + } if previous == nil { b.logger.Info("selected positioning atx", @@ -963,7 +964,8 @@ func (b *Builder) getPositioningAtx( candidate, err := b.atxSvc.Atx(ctx, id) if err != nil { - return types.EmptyATXID, fmt.Errorf("get candidate pos ATX %s: %w", id.ShortString(), err) + b.logger.Warn("failed to get candidate pos ATX - falling back to previous", zap.Error(err)) + return previous.ID(), nil } if previous.TickHeight() >= candidate.TickHeight() { diff --git a/activation/activation_test.go b/activation/activation_test.go index 6d728a4e5c..746f884b93 100644 --- a/activation/activation_test.go +++ b/activation/activation_test.go @@ -1350,6 +1350,22 @@ func TestGetPositioningAtx(t *testing.T) { require.NoError(t, err) require.Equal(t, tab.goldenATXID, posATX) }) + t.Run("picks previous when querying candidate fails and previous is available", func(t *testing.T) { + t.Parallel() + atxSvc := NewMockAtxService(gomock.NewController(t)) + tab := newTestBuilder(t, 1) + tab.atxSvc = atxSvc + + atxID := types.RandomATXID() + atxSvc.EXPECT().PositioningATX(gomock.Any(), types.EpochID(98)).Return(atxID, nil) + atxSvc.EXPECT().Atx(context.Background(), atxID).Return(nil, errors.New("failed")) + + previous := types.ActivationTx{} + previous.SetID(types.RandomATXID()) + posATX, err := tab.getPositioningAtx(context.Background(), types.EmptyNodeID, 99, &previous) + require.NoError(t, err) + require.Equal(t, previous.ID(), posATX) + }) t.Run("picks golden if no ATXs", func(t *testing.T) { tab := newTestBuilder(t, 1) atx, err := tab.getPositioningAtx(context.Background(), types.EmptyNodeID, 99, nil) From 24956524b661c48543f8573b56d9fe0174055088 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bartosz=20R=C3=B3=C5=BCa=C5=84ski?= Date: Tue, 29 Oct 2024 16:07:16 +0100 Subject: [PATCH 10/10] regenerate code --- activation/mocks.go | 1 + 1 file changed, 1 insertion(+) diff --git a/activation/mocks.go b/activation/mocks.go index 4cd3041f9d..4e1fe49cef 100644 --- a/activation/mocks.go +++ b/activation/mocks.go @@ -1221,6 +1221,7 @@ func (c *MockatxProviderGetAtxCall) DoAndReturn(f func(types.ATXID) (*types.Acti type MockAtxService struct { ctrl *gomock.Controller recorder *MockAtxServiceMockRecorder + isgomock struct{} } // MockAtxServiceMockRecorder is the mock recorder for MockAtxService.