diff --git a/runtime/v2/app.go b/runtime/v2/app.go index 94d4d78ee592..59ede48711c2 100644 --- a/runtime/v2/app.go +++ b/runtime/v2/app.go @@ -47,7 +47,7 @@ type AppI[T transaction.Tx] interface { // done declaratively with an app config and the rest of it is done the old way. // See simapp/app_v2.go for an example of this setup. type App[T transaction.Tx] struct { - appmanager.AppManager[T] + *appmanager.AppManager[T] // app manager dependencies stf *stf.STF[T] @@ -127,6 +127,6 @@ func (a *App[T]) ExecuteGenesisTx(_ []byte) error { panic("App.ExecuteGenesisTx not supported in runtime/v2") } -func (a *App[T]) GetAppManager() appmanager.AppManager[T] { +func (a *App[T]) GetAppManager() *appmanager.AppManager[T] { return a.AppManager } diff --git a/runtime/v2/module.go b/runtime/v2/module.go index 772c94e6fdcc..ffc5632f9403 100644 --- a/runtime/v2/module.go +++ b/runtime/v2/module.go @@ -179,11 +179,10 @@ func ProvideModuleManager[T transaction.Tx]( } // ProvideEnvironment provides the environment for keeper modules, while maintaining backward compatibility and provide services directly as well. -func ProvideEnvironment[T transaction.Tx](logger log.Logger, config *runtimev2.Module, key depinject.ModuleKey) ( - appBuilder *AppBuilder[T], - env appmodulev2.Environment, - kvss store.KVStoreService, - mss store.MemoryStoreService, +func ProvideEnvironment[T transaction.Tx](logger log.Logger, config *runtimev2.Module, key depinject.ModuleKey, appBuilder *AppBuilder[T]) ( + appmodulev2.Environment, + store.KVStoreService, + store.MemoryStoreService, ) { var ( kvService store.KVStoreService = failingStoreService{} @@ -208,7 +207,7 @@ func ProvideEnvironment[T transaction.Tx](logger log.Logger, config *runtimev2.M memKvService = stf.NewMemoryStoreService([]byte(memStoreKey)) } - env = appmodulev2.Environment{ + env := appmodulev2.Environment{ Logger: logger, BranchService: stf.BranchService{}, EventService: stf.NewEventService(), @@ -221,7 +220,7 @@ func ProvideEnvironment[T transaction.Tx](logger log.Logger, config *runtimev2.M MemStoreService: memKvService, } - return appBuilder, env, kvService, memKvService + return env, kvService, memKvService } func registerStoreKey[T transaction.Tx](wrapper *AppBuilder[T], key string) { diff --git a/server/v2/appmanager/appmanager.go b/server/v2/appmanager/appmanager.go index 572514295dd0..0dcb2cec5939 100644 --- a/server/v2/appmanager/appmanager.go +++ b/server/v2/appmanager/appmanager.go @@ -12,8 +12,9 @@ import ( "cosmossdk.io/server/v2/appmanager/store" ) -// appManager is a coordinator for all things related to an application -type appManager[T transaction.Tx] struct { +// AppManager is a coordinator for all things related to an application +// TODO: add exportGenesis function +type AppManager[T transaction.Tx] struct { config Config db store.Store @@ -24,7 +25,7 @@ type appManager[T transaction.Tx] struct { stf StateTransitionFunction[T] } -func (a appManager[T]) InitGenesis( +func (a AppManager[T]) InitGenesis( ctx context.Context, blockRequest *appmanager.BlockRequest[T], initGenesisJSON []byte, @@ -68,7 +69,7 @@ func (a appManager[T]) InitGenesis( } // ExportGenesis exports the genesis state of the application. -func (a appManager[T]) ExportGenesis(ctx context.Context, version uint64) ([]byte, error) { +func (a AppManager[T]) ExportGenesis(ctx context.Context, version uint64) ([]byte, error) { bz, err := a.exportGenesis(ctx, version) if err != nil { return nil, fmt.Errorf("failed to export genesis state: %w", err) @@ -77,7 +78,7 @@ func (a appManager[T]) ExportGenesis(ctx context.Context, version uint64) ([]byt return bz, nil } -func (a appManager[T]) DeliverBlock( +func (a AppManager[T]) DeliverBlock( ctx context.Context, block *appmanager.BlockRequest[T], ) (*appmanager.BlockResponse, corestore.WriterMap, error) { @@ -101,7 +102,7 @@ func (a appManager[T]) DeliverBlock( // ValidateTx will validate the tx against the latest storage state. This means that // only the stateful validation will be run, not the execution portion of the tx. // If full execution is needed, Simulate must be used. -func (a appManager[T]) ValidateTx(ctx context.Context, tx T) (appmanager.TxResult, error) { +func (a AppManager[T]) ValidateTx(ctx context.Context, tx T) (appmanager.TxResult, error) { _, latestState, err := a.db.StateLatest() if err != nil { return appmanager.TxResult{}, err @@ -110,7 +111,7 @@ func (a appManager[T]) ValidateTx(ctx context.Context, tx T) (appmanager.TxResul } // Simulate runs validation and execution flow of a Tx. -func (a appManager[T]) Simulate(ctx context.Context, tx T) (appmanager.TxResult, corestore.WriterMap, error) { +func (a AppManager[T]) Simulate(ctx context.Context, tx T) (appmanager.TxResult, corestore.WriterMap, error) { _, state, err := a.db.StateLatest() if err != nil { return appmanager.TxResult{}, nil, err @@ -121,7 +122,7 @@ func (a appManager[T]) Simulate(ctx context.Context, tx T) (appmanager.TxResult, // Query queries the application at the provided version. // CONTRACT: Version must always be provided, if 0, get latest -func (a appManager[T]) Query(ctx context.Context, version uint64, request transaction.Msg) (transaction.Msg, error) { +func (a AppManager[T]) Query(ctx context.Context, version uint64, request transaction.Msg) (transaction.Msg, error) { // if version is provided attempt to do a height query. if version != 0 { queryState, err := a.db.StateAt(version) @@ -142,7 +143,7 @@ func (a appManager[T]) Query(ctx context.Context, version uint64, request transa // QueryWithState executes a query with the provided state. This allows to process a query // independently of the db state. For example, it can be used to process a query with temporary // and uncommitted state -func (a appManager[T]) QueryWithState( +func (a AppManager[T]) QueryWithState( ctx context.Context, state corestore.ReaderMap, request transaction.Msg, diff --git a/server/v2/appmanager/appmanager_builder.go b/server/v2/appmanager/appmanager_builder.go index 5c832c56baa8..7b952c92e982 100644 --- a/server/v2/appmanager/appmanager_builder.go +++ b/server/v2/appmanager/appmanager_builder.go @@ -26,8 +26,8 @@ type Builder[T transaction.Tx] struct { // Build creates a new instance of AppManager with the provided configuration and returns it. // It initializes the AppManager with the given database, export state, import state, initGenesis function, and state transition function. -func (b Builder[T]) Build() (AppManager[T], error) { - return &appManager[T]{ +func (b Builder[T]) Build() (*AppManager[T], error) { + return &AppManager[T]{ config: Config{ ValidateTxGasLimit: b.ValidateTxGasLimit, QueryGasLimit: b.QueryGasLimit, diff --git a/server/v2/appmanager/interfaces.go b/server/v2/appmanager/interfaces.go deleted file mode 100644 index d80b4148a131..000000000000 --- a/server/v2/appmanager/interfaces.go +++ /dev/null @@ -1,54 +0,0 @@ -package appmanager - -import ( - "context" - - appmanager "cosmossdk.io/core/app" - corestore "cosmossdk.io/core/store" - "cosmossdk.io/core/transaction" -) - -// AppManager is an interface that defines the methods for managing an application. -type AppManager[T transaction.Tx] interface { - // InitGenesis initializes the genesis state of the application. - // It takes a context, a block request, the genesis JSON, and a transaction decoder. - // It returns a block response, a writer map, and an error if any. - InitGenesis( - ctx context.Context, - blockRequest *appmanager.BlockRequest[T], - initGenesisJSON []byte, - txDecoder transaction.Codec[T], - ) (*appmanager.BlockResponse, corestore.WriterMap, error) - // ExportGenesis exports the genesis state of the application. - // It takes a context and a version number. - // It returns the genesis state as a byte slice and an error if any. - ExportGenesis(ctx context.Context, version uint64) ([]byte, error) - // DeliverBlock processes a block of transactions. - // It takes a context and a block request. - // It returns a block response, a writer map, and an error if any. - DeliverBlock( - ctx context.Context, - block *appmanager.BlockRequest[T], - ) (*appmanager.BlockResponse, corestore.WriterMap, error) - // ValidateTx validates a transaction against the latest storage state. - // It takes a context and a transaction. - // It returns a transaction result and an error if any. - ValidateTx(ctx context.Context, tx T) (appmanager.TxResult, error) - // Simulate runs validation and execution flow of a transaction. - // It takes a context and a transaction. - // It returns a transaction result, a writer map, and an error if any. - Simulate(ctx context.Context, tx T) (appmanager.TxResult, corestore.WriterMap, error) - // Query queries the application at the provided version. - // It takes a context, a version number, and a request message. - // It returns a response message and an error if any. - Query(ctx context.Context, version uint64, request transaction.Msg) (transaction.Msg, error) - // QueryWithState executes a query with the provided state. - // It takes a context, a state, and a request message. - // It returns a response message and an error if any. - QueryWithState( - ctx context.Context, - state corestore.ReaderMap, - request transaction.Msg, - ) (transaction.Msg, error) - MakeContext(context.Context, transaction.ExecMode) context.Context -} diff --git a/server/v2/cometbft/abci.go b/server/v2/cometbft/abci.go index 0321abeaa414..b1a0afeb038c 100644 --- a/server/v2/cometbft/abci.go +++ b/server/v2/cometbft/abci.go @@ -28,29 +28,10 @@ import ( consensustypes "cosmossdk.io/x/consensus/types" ) -var ( - _ abci.Application = (*consensus[transaction.Tx])(nil) - _ Consensus[transaction.Tx] = (*consensus[transaction.Tx])(nil) - _ abci.Application = (Consensus[transaction.Tx])(nil) -) - -// Consensus defines the interface for consensus operations in the CometBFT system. -type Consensus[T transaction.Tx] interface { - abci.Application - // GetStore returns the store used by the consensus. - GetStore() types.Store - // SetStreamingManager sets the streaming manager for the consensus. - SetStreamingManager(sm streaming.Manager) - // SetSnapshotManager sets the snapshot manager for the consensus. - SetSnapshotManager(sm *snapshots.Manager) - // RegisterExtensions registers snapshot extensions for the consensus. - RegisterExtensions(extensions ...snapshots.ExtensionSnapshotter) - // CheckTx checks the validity of a transaction. - CheckTx(ctx context.Context, req *abciproto.CheckTxRequest) (*abciproto.CheckTxResponse, error) -} +var _ abci.Application = (*Consensus[transaction.Tx])(nil) -type consensus[T transaction.Tx] struct { - app appmanager.AppManager[T] +type Consensus[T transaction.Tx] struct { + app *appmanager.AppManager[T] cfg Config store types.Store logger log.Logger @@ -73,14 +54,14 @@ type consensus[T transaction.Tx] struct { } func NewConsensus[T transaction.Tx]( - app appmanager.AppManager[T], + app *appmanager.AppManager[T], mp mempool.Mempool[T], store types.Store, cfg Config, txCodec transaction.Codec[T], logger log.Logger, -) *consensus[T] { - return &consensus[T]{ +) *Consensus[T] { + return &Consensus[T]{ mempool: mp, store: store, app: app, @@ -90,7 +71,7 @@ func NewConsensus[T transaction.Tx]( } } -func (c *consensus[T]) SetStreamingManager(sm streaming.Manager) { +func (c *Consensus[T]) SetStreamingManager(sm streaming.Manager) { c.streaming = sm } @@ -98,18 +79,13 @@ func (c *consensus[T]) SetStreamingManager(sm streaming.Manager) { // The snapshot manager is responsible for managing snapshots of the Consensus state. // It allows for creating, storing, and restoring snapshots of the Consensus state. // The provided snapshot manager will be used by the Consensus to handle snapshots. -func (c *consensus[T]) SetSnapshotManager(sm *snapshots.Manager) { +func (c *Consensus[T]) SetSnapshotManager(sm *snapshots.Manager) { c.snapshotManager = sm } -// GetStore returns the store used by the consensus. -func (c *consensus[T]) GetStore() types.Store { - return c.store -} - // RegisterExtensions registers the given extensions with the consensus module's snapshot manager. // It allows additional snapshotter implementations to be used for creating and restoring snapshots. -func (c *consensus[T]) RegisterExtensions(extensions ...snapshots.ExtensionSnapshotter) { +func (c *Consensus[T]) RegisterExtensions(extensions ...snapshots.ExtensionSnapshotter) { if err := c.snapshotManager.RegisterExtensions(extensions...); err != nil { panic(fmt.Errorf("failed to register snapshot extensions: %w", err)) } @@ -117,7 +93,7 @@ func (c *consensus[T]) RegisterExtensions(extensions ...snapshots.ExtensionSnaps // CheckTx implements types.Application. // It is called by cometbft to verify transaction validity -func (c *consensus[T]) CheckTx(ctx context.Context, req *abciproto.CheckTxRequest) (*abciproto.CheckTxResponse, error) { +func (c *Consensus[T]) CheckTx(ctx context.Context, req *abciproto.CheckTxRequest) (*abciproto.CheckTxResponse, error) { decodedTx, err := c.txCodec.Decode(req.Tx) if err != nil { return nil, err @@ -146,7 +122,7 @@ func (c *consensus[T]) CheckTx(ctx context.Context, req *abciproto.CheckTxReques } // Info implements types.Application. -func (c *consensus[T]) Info(ctx context.Context, _ *abciproto.InfoRequest) (*abciproto.InfoResponse, error) { +func (c *Consensus[T]) Info(ctx context.Context, _ *abciproto.InfoRequest) (*abciproto.InfoResponse, error) { version, _, err := c.store.StateLatest() if err != nil { return nil, err @@ -174,7 +150,7 @@ func (c *consensus[T]) Info(ctx context.Context, _ *abciproto.InfoRequest) (*abc // Query implements types.Application. // It is called by cometbft to query application state. -func (c *consensus[T]) Query(ctx context.Context, req *abciproto.QueryRequest) (*abciproto.QueryResponse, error) { +func (c *Consensus[T]) Query(ctx context.Context, req *abciproto.QueryRequest) (*abciproto.QueryResponse, error) { // follow the query path from here decodedMsg, err := c.txCodec.Decode(req.Data) protoMsg, ok := any(decodedMsg).(transaction.Msg) @@ -228,7 +204,7 @@ func (c *consensus[T]) Query(ctx context.Context, req *abciproto.QueryRequest) ( } // InitChain implements types.Application. -func (c *consensus[T]) InitChain(ctx context.Context, req *abciproto.InitChainRequest) (*abciproto.InitChainResponse, error) { +func (c *Consensus[T]) InitChain(ctx context.Context, req *abciproto.InitChainRequest) (*abciproto.InitChainResponse, error) { c.logger.Info("InitChain", "initialHeight", req.InitialHeight, "chainID", req.ChainId) // store chainID to be used later on in execution @@ -313,7 +289,7 @@ func (c *consensus[T]) InitChain(ctx context.Context, req *abciproto.InitChainRe // PrepareProposal implements types.Application. // It is called by cometbft to prepare a proposal block. -func (c *consensus[T]) PrepareProposal( +func (c *Consensus[T]) PrepareProposal( ctx context.Context, req *abciproto.PrepareProposalRequest, ) (resp *abciproto.PrepareProposalResponse, err error) { @@ -357,7 +333,7 @@ func (c *consensus[T]) PrepareProposal( // ProcessProposal implements types.Application. // It is called by cometbft to process/verify a proposal block. -func (c *consensus[T]) ProcessProposal( +func (c *Consensus[T]) ProcessProposal( ctx context.Context, req *abciproto.ProcessProposalRequest, ) (*abciproto.ProcessProposalResponse, error) { @@ -395,7 +371,7 @@ func (c *consensus[T]) ProcessProposal( // FinalizeBlock implements types.Application. // It is called by cometbft to finalize a block. -func (c *consensus[T]) FinalizeBlock( +func (c *Consensus[T]) FinalizeBlock( ctx context.Context, req *abciproto.FinalizeBlockRequest, ) (*abciproto.FinalizeBlockResponse, error) { @@ -515,7 +491,7 @@ func (c *consensus[T]) FinalizeBlock( // Commit implements types.Application. // It is called by cometbft to notify the application that a block was committed. -func (c *consensus[T]) Commit(ctx context.Context, _ *abciproto.CommitRequest) (*abciproto.CommitResponse, error) { +func (c *Consensus[T]) Commit(ctx context.Context, _ *abciproto.CommitRequest) (*abciproto.CommitResponse, error) { lastCommittedHeight := c.lastCommittedHeight.Load() c.snapshotManager.SnapshotIfApplicable(lastCommittedHeight) @@ -532,7 +508,7 @@ func (c *consensus[T]) Commit(ctx context.Context, _ *abciproto.CommitRequest) ( // Vote extensions // VerifyVoteExtension implements types.Application. -func (c *consensus[T]) VerifyVoteExtension( +func (c *Consensus[T]) VerifyVoteExtension( ctx context.Context, req *abciproto.VerifyVoteExtensionRequest, ) (*abciproto.VerifyVoteExtensionResponse, error) { @@ -569,7 +545,7 @@ func (c *consensus[T]) VerifyVoteExtension( } // ExtendVote implements types.Application. -func (c *consensus[T]) ExtendVote(ctx context.Context, req *abciproto.ExtendVoteRequest) (*abciproto.ExtendVoteResponse, error) { +func (c *Consensus[T]) ExtendVote(ctx context.Context, req *abciproto.ExtendVoteRequest) (*abciproto.ExtendVoteResponse, error) { // If vote extensions are not enabled, as a safety precaution, we return an // error. cp, err := c.GetConsensusParams(ctx) diff --git a/server/v2/cometbft/commands.go b/server/v2/cometbft/commands.go index 002d4c83dd50..a27a846c58ae 100644 --- a/server/v2/cometbft/commands.go +++ b/server/v2/cometbft/commands.go @@ -29,7 +29,7 @@ import ( ) func (s *CometBFTServer[AppT, T]) rpcClient(cmd *cobra.Command) (rpc.CometRPC, error) { - if s.Config.Standalone { + if s.config.Standalone { client, err := rpchttp.New(client.GetConfigFromCmd(cmd).RPC.ListenAddress) if err != nil { return nil, err @@ -395,7 +395,7 @@ func (s *CometBFTServer[AppT, T]) BootstrapStateCmd() *cobra.Command { return err } if height == 0 { - height, err = s.Consensus.GetStore().GetLatestVersion() + height, err = s.Consensus.store.GetLatestVersion() if err != nil { return err } diff --git a/server/v2/cometbft/query.go b/server/v2/cometbft/query.go index 1e32a06060da..d45c97cd078b 100644 --- a/server/v2/cometbft/query.go +++ b/server/v2/cometbft/query.go @@ -12,7 +12,7 @@ import ( cometerrors "cosmossdk.io/server/v2/cometbft/types/errors" ) -func (c *consensus[T]) handleQueryP2P(path []string) (*abci.QueryResponse, error) { +func (c *Consensus[T]) handleQueryP2P(path []string) (*abci.QueryResponse, error) { // "/p2p" prefix for p2p queries if len(path) < 4 { return nil, errorsmod.Wrap(cometerrors.ErrUnknownRequest, "path should be p2p filter ") @@ -41,7 +41,7 @@ func (c *consensus[T]) handleQueryP2P(path []string) (*abci.QueryResponse, error // simulates the transaction using the application, and returns the simulation result. // If the second element is 'version', it returns the version of the application. // If the second element is neither 'simulate' nor 'version', it returns an error indicating an unknown query. -func (c *consensus[T]) handlerQueryApp(ctx context.Context, path []string, req *abci.QueryRequest) (*abci.QueryResponse, error) { +func (c *Consensus[T]) handlerQueryApp(ctx context.Context, path []string, req *abci.QueryRequest) (*abci.QueryResponse, error) { if len(path) < 2 { return nil, errorsmod.Wrap( cometerrors.ErrUnknownRequest, @@ -83,7 +83,7 @@ func (c *consensus[T]) handlerQueryApp(ctx context.Context, path []string, req * return nil, errorsmod.Wrapf(cometerrors.ErrUnknownRequest, "unknown query: %s", path) } -func (c *consensus[T]) handleQueryStore(path []string, _ types.Store, req *abci.QueryRequest) (*abci.QueryResponse, error) { +func (c *Consensus[T]) handleQueryStore(path []string, _ types.Store, req *abci.QueryRequest) (*abci.QueryResponse, error) { req.Path = "/" + strings.Join(path[1:], "/") if req.Height <= 1 && req.Prove { return nil, errorsmod.Wrap( diff --git a/server/v2/cometbft/server.go b/server/v2/cometbft/server.go index 22df59cca84d..c13aa66a4bbc 100644 --- a/server/v2/cometbft/server.go +++ b/server/v2/cometbft/server.go @@ -39,44 +39,44 @@ var ( type CometBFTServer[AppT serverv2.AppI[T], T transaction.Tx] struct { Node *node.Node - Consensus Consensus[T] + Consensus *Consensus[T] initTxCodec transaction.Codec[T] - Logger log.Logger - Config Config - Options ServerOptions[T] + logger log.Logger + config Config + options ServerOptions[T] } func New[AppT serverv2.AppI[T], T transaction.Tx](txCodec transaction.Codec[T], options ServerOptions[T]) *CometBFTServer[AppT, T] { return &CometBFTServer[AppT, T]{ initTxCodec: txCodec, - Options: options, + options: options, } } func (s *CometBFTServer[AppT, T]) Init(appI AppT, v *viper.Viper, logger log.Logger) error { - s.Config = Config{CmtConfig: GetConfigFromViper(v), ConsensusAuthority: appI.GetConsensusAuthority()} - s.Logger = logger.With(log.ModuleKey, s.Name()) + s.config = Config{CmtConfig: GetConfigFromViper(v), ConsensusAuthority: appI.GetConsensusAuthority()} + s.logger = logger.With(log.ModuleKey, s.Name()) // create consensus store := appI.GetStore().(types.Store) - consensus := NewConsensus[T](appI.GetAppManager(), s.Options.Mempool, store, s.Config, s.initTxCodec, s.Logger) + consensus := NewConsensus[T](appI.GetAppManager(), s.options.Mempool, store, s.config, s.initTxCodec, s.logger) - consensus.prepareProposalHandler = s.Options.PrepareProposalHandler - consensus.processProposalHandler = s.Options.ProcessProposalHandler - consensus.verifyVoteExt = s.Options.VerifyVoteExtensionHandler - consensus.extendVote = s.Options.ExtendVoteHandler + consensus.prepareProposalHandler = s.options.PrepareProposalHandler + consensus.processProposalHandler = s.options.ProcessProposalHandler + consensus.verifyVoteExt = s.options.VerifyVoteExtensionHandler + consensus.extendVote = s.options.ExtendVoteHandler // TODO: set these; what is the appropriate presence of the Store interface here? var ss snapshots.StorageSnapshotter var sc snapshots.CommitSnapshotter - snapshotStore, err := GetSnapshotStore(s.Config.CmtConfig.RootDir) + snapshotStore, err := GetSnapshotStore(s.config.CmtConfig.RootDir) if err != nil { return err } - sm := snapshots.NewManager(snapshotStore, s.Options.SnapshotOptions, sc, ss, nil, s.Logger) + sm := snapshots.NewManager(snapshotStore, s.options.SnapshotOptions, sc, ss, nil, s.logger) consensus.SetSnapshotManager(sm) s.Consensus = consensus @@ -91,9 +91,9 @@ func (s *CometBFTServer[AppT, T]) Start(ctx context.Context) error { viper := ctx.Value(corectx.ViperContextKey).(*viper.Viper) cometConfig := GetConfigFromViper(viper) - wrappedLogger := cometlog.CometLoggerWrapper{Logger: s.Logger} - if s.Config.Standalone { - svr, err := abciserver.NewServer(s.Config.Addr, s.Config.Transport, s.Consensus) + wrappedLogger := cometlog.CometLoggerWrapper{Logger: s.logger} + if s.config.Standalone { + svr, err := abciserver.NewServer(s.config.Addr, s.config.Transport, s.Consensus) if err != nil { return fmt.Errorf("error creating listener: %w", err) } diff --git a/server/v2/cometbft/snapshots.go b/server/v2/cometbft/snapshots.go index e47312a469da..5534712864df 100644 --- a/server/v2/cometbft/snapshots.go +++ b/server/v2/cometbft/snapshots.go @@ -34,7 +34,7 @@ func GetSnapshotStore(rootDir string) (*snapshots.Store, error) { } // ApplySnapshotChunk implements types.Application. -func (c *consensus[T]) ApplySnapshotChunk(_ context.Context, req *abci.ApplySnapshotChunkRequest) (*abci.ApplySnapshotChunkResponse, error) { +func (c *Consensus[T]) ApplySnapshotChunk(_ context.Context, req *abci.ApplySnapshotChunkRequest) (*abci.ApplySnapshotChunkResponse, error) { if c.snapshotManager == nil { c.logger.Error("snapshot manager not configured") return &abci.ApplySnapshotChunkResponse{Result: abci.APPLY_SNAPSHOT_CHUNK_RESULT_ABORT}, nil @@ -65,7 +65,7 @@ func (c *consensus[T]) ApplySnapshotChunk(_ context.Context, req *abci.ApplySnap } // ListSnapshots implements types.Application. -func (c *consensus[T]) ListSnapshots(_ context.Context, ctx *abci.ListSnapshotsRequest) (*abci.ListSnapshotsResponse, error) { +func (c *Consensus[T]) ListSnapshots(_ context.Context, ctx *abci.ListSnapshotsRequest) (*abci.ListSnapshotsResponse, error) { if c.snapshotManager == nil { return nil, nil } @@ -91,7 +91,7 @@ func (c *consensus[T]) ListSnapshots(_ context.Context, ctx *abci.ListSnapshotsR } // LoadSnapshotChunk implements types.Application. -func (c *consensus[T]) LoadSnapshotChunk(_ context.Context, req *abci.LoadSnapshotChunkRequest) (*abci.LoadSnapshotChunkResponse, error) { +func (c *Consensus[T]) LoadSnapshotChunk(_ context.Context, req *abci.LoadSnapshotChunkRequest) (*abci.LoadSnapshotChunkResponse, error) { if c.snapshotManager == nil { return &abci.LoadSnapshotChunkResponse{}, nil } @@ -112,7 +112,7 @@ func (c *consensus[T]) LoadSnapshotChunk(_ context.Context, req *abci.LoadSnapsh } // OfferSnapshot implements types.Application. -func (c *consensus[T]) OfferSnapshot(_ context.Context, req *abci.OfferSnapshotRequest) (*abci.OfferSnapshotResponse, error) { +func (c *Consensus[T]) OfferSnapshot(_ context.Context, req *abci.OfferSnapshotRequest) (*abci.OfferSnapshotResponse, error) { if c.snapshotManager == nil { c.logger.Error("snapshot manager not configured") return &abci.OfferSnapshotResponse{Result: abci.OFFER_SNAPSHOT_RESULT_ABORT}, nil diff --git a/server/v2/cometbft/streaming.go b/server/v2/cometbft/streaming.go index 4d9e5bf90009..1ac9991b9ffa 100644 --- a/server/v2/cometbft/streaming.go +++ b/server/v2/cometbft/streaming.go @@ -10,7 +10,7 @@ import ( ) // streamDeliverBlockChanges will stream all the changes happened during deliver block. -func (c *consensus[T]) streamDeliverBlockChanges( +func (c *Consensus[T]) streamDeliverBlockChanges( ctx context.Context, height int64, txs [][]byte, diff --git a/server/v2/cometbft/utils.go b/server/v2/cometbft/utils.go index 806e2491fe0b..b302e90c78c3 100644 --- a/server/v2/cometbft/utils.go +++ b/server/v2/cometbft/utils.go @@ -22,7 +22,7 @@ import ( "cosmossdk.io/core/event" "cosmossdk.io/core/transaction" errorsmod "cosmossdk.io/errors" - consensustypes "cosmossdk.io/x/consensus/types" + consensus "cosmossdk.io/x/consensus/types" ) func queryResponse(res transaction.Msg, height int64) (*abci.QueryResponse, error) { @@ -250,7 +250,7 @@ func QueryResult(err error, debug bool) *abci.QueryResponse { } } -func (c *consensus[T]) validateFinalizeBlockHeight(req *abci.FinalizeBlockRequest) error { +func (c *Consensus[T]) validateFinalizeBlockHeight(req *abci.FinalizeBlockRequest) error { if req.Height < 1 { return fmt.Errorf("invalid height: %d", req.Height) } @@ -284,18 +284,18 @@ func (c *consensus[T]) validateFinalizeBlockHeight(req *abci.FinalizeBlockReques // GetConsensusParams makes a query to the consensus module in order to get the latest consensus // parameters from committed state -func (c *consensus[T]) GetConsensusParams(ctx context.Context) (*cmtproto.ConsensusParams, error) { +func (c *Consensus[T]) GetConsensusParams(ctx context.Context) (*cmtproto.ConsensusParams, error) { latestVersion, err := c.store.GetLatestVersion() if err != nil { return nil, err } - res, err := c.app.Query(ctx, latestVersion, &consensustypes.QueryParamsRequest{}) + res, err := c.app.Query(ctx, latestVersion, &consensus.QueryParamsRequest{}) if err != nil { return nil, err } - if r, ok := res.(*consensustypes.QueryParamsResponse); !ok { + if r, ok := res.(*consensus.QueryParamsResponse); !ok { return nil, fmt.Errorf("failed to query consensus params") } else { // convert our params to cometbft params @@ -325,7 +325,7 @@ func (c *consensus[T]) GetConsensusParams(ctx context.Context) (*cmtproto.Consen } } -func (c *consensus[T]) GetBlockRetentionHeight(cp *cmtproto.ConsensusParams, commitHeight int64) int64 { +func (c *Consensus[T]) GetBlockRetentionHeight(cp *cmtproto.ConsensusParams, commitHeight int64) int64 { // pruning is disabled if minRetainBlocks is zero if c.cfg.MinRetainBlocks == 0 { return 0 @@ -380,7 +380,7 @@ func (c *consensus[T]) GetBlockRetentionHeight(cp *cmtproto.ConsensusParams, com } // checkHalt checks if height or time exceeds halt-height or halt-time respectively. -func (c *consensus[T]) checkHalt(height int64, time time.Time) error { +func (c *Consensus[T]) checkHalt(height int64, time time.Time) error { var halt bool switch { case c.cfg.HaltHeight > 0 && uint64(height) > c.cfg.HaltHeight: diff --git a/server/v2/types.go b/server/v2/types.go index 517a2f775969..1a22abf8e16f 100644 --- a/server/v2/types.go +++ b/server/v2/types.go @@ -12,7 +12,7 @@ import ( type AppCreator[AppT AppI[T], T transaction.Tx] func(log.Logger, *viper.Viper) AppT type AppI[T transaction.Tx] interface { - GetAppManager() appmanager.AppManager[T] + GetAppManager() *appmanager.AppManager[T] GetConsensusAuthority() string InterfaceRegistry() coreapp.InterfaceRegistry GetStore() any