Browse Source

rpc: remove global environment (#6426)

pull/6428/head
Sam Kleinman 3 years ago
committed by GitHub
parent
commit
0b0914b3df
No known key found for this signature in database GPG Key ID: 4AEE18F83AFDEB23
21 changed files with 193 additions and 176 deletions
  1. +9
    -9
      node/node.go
  2. +31
    -33
      rpc/client/local/local.go
  3. +29
    -25
      rpc/client/mock/client.go
  4. +2
    -2
      rpc/core/abci.go
  5. +13
    -10
      rpc/core/blocks.go
  6. +2
    -2
      rpc/core/blocks_test.go
  7. +14
    -7
      rpc/core/consensus.go
  8. +1
    -1
      rpc/core/dev.go
  9. +3
    -14
      rpc/core/env.go
  10. +5
    -3
      rpc/core/env_test.go
  11. +3
    -3
      rpc/core/events.go
  12. +4
    -1
      rpc/core/evidence.go
  13. +1
    -1
      rpc/core/health.go
  14. +7
    -7
      rpc/core/mempool.go
  15. +8
    -5
      rpc/core/net.go
  16. +4
    -2
      rpc/core/net_test.go
  17. +46
    -42
      rpc/core/routes.go
  18. +3
    -3
      rpc/core/status.go
  19. +3
    -3
      rpc/core/tx.go
  20. +2
    -1
      rpc/grpc/api.go
  21. +3
    -2
      rpc/grpc/client_server.go

+ 9
- 9
node/node.go View File

@ -1522,7 +1522,7 @@ func (n *Node) OnStop() {
}
// ConfigureRPC makes sure RPC has all the objects it needs to operate.
func (n *Node) ConfigureRPC() error {
func (n *Node) ConfigureRPC() (*rpccore.Environment, error) {
rpcCoreEnv := rpccore.Environment{
ProxyAppQuery: n.proxyApp.Query(),
ProxyAppMempool: n.proxyApp.Mempool(),
@ -1548,24 +1548,24 @@ func (n *Node) ConfigureRPC() error {
if n.config.Mode == cfg.ModeValidator {
pubKey, err := n.privValidator.GetPubKey(context.TODO())
if pubKey == nil || err != nil {
return fmt.Errorf("can't get pubkey: %w", err)
return nil, fmt.Errorf("can't get pubkey: %w", err)
}
rpcCoreEnv.PubKey = pubKey
}
rpccore.SetEnvironment(&rpcCoreEnv)
return nil
return &rpcCoreEnv, nil
}
func (n *Node) startRPC() ([]net.Listener, error) {
err := n.ConfigureRPC()
env, err := n.ConfigureRPC()
if err != nil {
return nil, err
}
listenAddrs := strings.SplitAndTrimEmpty(n.config.RPC.ListenAddress, ",", " ")
routes := env.GetRoutes()
if n.config.RPC.Unsafe {
rpccore.AddUnsafeRoutes()
env.AddUnsafe(routes)
}
config := rpcserver.DefaultConfig()
@ -1585,7 +1585,7 @@ func (n *Node) startRPC() ([]net.Listener, error) {
mux := http.NewServeMux()
rpcLogger := n.Logger.With("module", "rpc-server")
wmLogger := rpcLogger.With("protocol", "websocket")
wm := rpcserver.NewWebsocketManager(rpccore.Routes,
wm := rpcserver.NewWebsocketManager(routes,
rpcserver.OnDisconnect(func(remoteAddr string) {
err := n.eventBus.UnsubscribeAll(context.Background(), remoteAddr)
if err != nil && err != tmpubsub.ErrSubscriptionNotFound {
@ -1596,7 +1596,7 @@ func (n *Node) startRPC() ([]net.Listener, error) {
)
wm.SetLogger(wmLogger)
mux.HandleFunc("/websocket", wm.WebsocketHandler)
rpcserver.RegisterRPCFuncs(mux, rpccore.Routes, rpcLogger)
rpcserver.RegisterRPCFuncs(mux, routes, rpcLogger)
listener, err := rpcserver.Listen(
listenAddr,
config,
@ -1662,7 +1662,7 @@ func (n *Node) startRPC() ([]net.Listener, error) {
return nil, err
}
go func() {
if err := grpccore.StartGRPCServer(listener); err != nil {
if err := grpccore.StartGRPCServer(env, listener); err != nil {
n.Logger.Error("Error starting gRPC server", "err", err)
}
}()


+ 31
- 33
rpc/client/local/local.go View File

@ -41,22 +41,20 @@ type Local struct {
*types.EventBus
Logger log.Logger
ctx *rpctypes.Context
env *core.Environment
}
// NewLocal configures a client that calls the Node directly.
//
// Note that given how rpc/core works with package singletons, that
// you can only have one node per process. So make sure test cases
// don't run in parallel, or try to simulate an entire network in
// one process...
func New(node *nm.Node) *Local {
if err := node.ConfigureRPC(); err != nil {
env, err := node.ConfigureRPC()
if err != nil {
node.Logger.Error("Error configuring RPC", "err", err)
}
return &Local{
EventBus: node.EventBus(),
Logger: log.NewNopLogger(),
ctx: &rpctypes.Context{},
env: env,
}
}
@ -68,11 +66,11 @@ func (c *Local) SetLogger(l log.Logger) {
}
func (c *Local) Status(ctx context.Context) (*ctypes.ResultStatus, error) {
return core.Status(c.ctx)
return c.env.Status(c.ctx)
}
func (c *Local) ABCIInfo(ctx context.Context) (*ctypes.ResultABCIInfo, error) {
return core.ABCIInfo(c.ctx)
return c.env.ABCIInfo(c.ctx)
}
func (c *Local) ABCIQuery(ctx context.Context, path string, data bytes.HexBytes) (*ctypes.ResultABCIQuery, error) {
@ -84,55 +82,55 @@ func (c *Local) ABCIQueryWithOptions(
path string,
data bytes.HexBytes,
opts rpcclient.ABCIQueryOptions) (*ctypes.ResultABCIQuery, error) {
return core.ABCIQuery(c.ctx, path, data, opts.Height, opts.Prove)
return c.env.ABCIQuery(c.ctx, path, data, opts.Height, opts.Prove)
}
func (c *Local) BroadcastTxCommit(ctx context.Context, tx types.Tx) (*ctypes.ResultBroadcastTxCommit, error) {
return core.BroadcastTxCommit(c.ctx, tx)
return c.env.BroadcastTxCommit(c.ctx, tx)
}
func (c *Local) BroadcastTxAsync(ctx context.Context, tx types.Tx) (*ctypes.ResultBroadcastTx, error) {
return core.BroadcastTxAsync(c.ctx, tx)
return c.env.BroadcastTxAsync(c.ctx, tx)
}
func (c *Local) BroadcastTxSync(ctx context.Context, tx types.Tx) (*ctypes.ResultBroadcastTx, error) {
return core.BroadcastTxSync(c.ctx, tx)
return c.env.BroadcastTxSync(c.ctx, tx)
}
func (c *Local) UnconfirmedTxs(ctx context.Context, limit *int) (*ctypes.ResultUnconfirmedTxs, error) {
return core.UnconfirmedTxs(c.ctx, limit)
return c.env.UnconfirmedTxs(c.ctx, limit)
}
func (c *Local) NumUnconfirmedTxs(ctx context.Context) (*ctypes.ResultUnconfirmedTxs, error) {
return core.NumUnconfirmedTxs(c.ctx)
return c.env.NumUnconfirmedTxs(c.ctx)
}
func (c *Local) CheckTx(ctx context.Context, tx types.Tx) (*ctypes.ResultCheckTx, error) {
return core.CheckTx(c.ctx, tx)
return c.env.CheckTx(c.ctx, tx)
}
func (c *Local) NetInfo(ctx context.Context) (*ctypes.ResultNetInfo, error) {
return core.NetInfo(c.ctx)
return c.env.NetInfo(c.ctx)
}
func (c *Local) DumpConsensusState(ctx context.Context) (*ctypes.ResultDumpConsensusState, error) {
return core.DumpConsensusState(c.ctx)
return c.env.DumpConsensusState(c.ctx)
}
func (c *Local) ConsensusState(ctx context.Context) (*ctypes.ResultConsensusState, error) {
return core.ConsensusState(c.ctx)
return c.env.GetConsensusState(c.ctx)
}
func (c *Local) ConsensusParams(ctx context.Context, height *int64) (*ctypes.ResultConsensusParams, error) {
return core.ConsensusParams(c.ctx, height)
return c.env.ConsensusParams(c.ctx, height)
}
func (c *Local) Health(ctx context.Context) (*ctypes.ResultHealth, error) {
return core.Health(c.ctx)
return c.env.Health(c.ctx)
}
func (c *Local) DialSeeds(ctx context.Context, seeds []string) (*ctypes.ResultDialSeeds, error) {
return core.UnsafeDialSeeds(c.ctx, seeds)
return c.env.UnsafeDialSeeds(c.ctx, seeds)
}
func (c *Local) DialPeers(
@ -142,39 +140,39 @@ func (c *Local) DialPeers(
unconditional,
private bool,
) (*ctypes.ResultDialPeers, error) {
return core.UnsafeDialPeers(c.ctx, peers, persistent, unconditional, private)
return c.env.UnsafeDialPeers(c.ctx, peers, persistent, unconditional, private)
}
func (c *Local) BlockchainInfo(ctx context.Context, minHeight, maxHeight int64) (*ctypes.ResultBlockchainInfo, error) {
return core.BlockchainInfo(c.ctx, minHeight, maxHeight)
return c.env.BlockchainInfo(c.ctx, minHeight, maxHeight)
}
func (c *Local) Genesis(ctx context.Context) (*ctypes.ResultGenesis, error) {
return core.Genesis(c.ctx)
return c.env.Genesis(c.ctx)
}
func (c *Local) Block(ctx context.Context, height *int64) (*ctypes.ResultBlock, error) {
return core.Block(c.ctx, height)
return c.env.Block(c.ctx, height)
}
func (c *Local) BlockByHash(ctx context.Context, hash []byte) (*ctypes.ResultBlock, error) {
return core.BlockByHash(c.ctx, hash)
return c.env.BlockByHash(c.ctx, hash)
}
func (c *Local) BlockResults(ctx context.Context, height *int64) (*ctypes.ResultBlockResults, error) {
return core.BlockResults(c.ctx, height)
return c.env.BlockResults(c.ctx, height)
}
func (c *Local) Commit(ctx context.Context, height *int64) (*ctypes.ResultCommit, error) {
return core.Commit(c.ctx, height)
return c.env.Commit(c.ctx, height)
}
func (c *Local) Validators(ctx context.Context, height *int64, page, perPage *int) (*ctypes.ResultValidators, error) {
return core.Validators(c.ctx, height, page, perPage)
return c.env.Validators(c.ctx, height, page, perPage)
}
func (c *Local) Tx(ctx context.Context, hash []byte, prove bool) (*ctypes.ResultTx, error) {
return core.Tx(c.ctx, hash, prove)
return c.env.Tx(c.ctx, hash, prove)
}
func (c *Local) TxSearch(
@ -185,7 +183,7 @@ func (c *Local) TxSearch(
perPage *int,
orderBy string,
) (*ctypes.ResultTxSearch, error) {
return core.TxSearch(c.ctx, query, prove, page, perPage, orderBy)
return c.env.TxSearch(c.ctx, query, prove, page, perPage, orderBy)
}
func (c *Local) BlockSearch(
@ -194,11 +192,11 @@ func (c *Local) BlockSearch(
page, perPage *int,
orderBy string,
) (*ctypes.ResultBlockSearch, error) {
return core.BlockSearch(c.ctx, query, page, perPage, orderBy)
return c.env.BlockSearch(c.ctx, query, page, perPage, orderBy)
}
func (c *Local) BroadcastEvidence(ctx context.Context, ev types.Evidence) (*ctypes.ResultBroadcastEvidence, error) {
return core.BroadcastEvidence(c.ctx, ev)
return c.env.BroadcastEvidence(c.ctx, ev)
}
func (c *Local) Subscribe(


+ 29
- 25
rpc/client/mock/client.go View File

@ -28,10 +28,6 @@ import (
)
// Client wraps arbitrary implementations of the various interfaces.
//
// We provide a few choices to mock out each one in this package.
// Nothing hidden here, so no New function, just construct it from
// some parts, and swap them out them during the tests.
type Client struct {
client.ABCIClient
client.SignClient
@ -41,6 +37,14 @@ type Client struct {
client.EvidenceClient
client.MempoolClient
service.Service
env *core.Environment
}
func New() Client {
return Client{
env: &core.Environment{},
}
}
var _ client.Client = Client{}
@ -81,11 +85,11 @@ func (c Call) GetResponse(args interface{}) (interface{}, error) {
}
func (c Client) Status(ctx context.Context) (*ctypes.ResultStatus, error) {
return core.Status(&rpctypes.Context{})
return c.env.Status(&rpctypes.Context{})
}
func (c Client) ABCIInfo(ctx context.Context) (*ctypes.ResultABCIInfo, error) {
return core.ABCIInfo(&rpctypes.Context{})
return c.env.ABCIInfo(&rpctypes.Context{})
}
func (c Client) ABCIQuery(ctx context.Context, path string, data bytes.HexBytes) (*ctypes.ResultABCIQuery, error) {
@ -97,47 +101,47 @@ func (c Client) ABCIQueryWithOptions(
path string,
data bytes.HexBytes,
opts client.ABCIQueryOptions) (*ctypes.ResultABCIQuery, error) {
return core.ABCIQuery(&rpctypes.Context{}, path, data, opts.Height, opts.Prove)
return c.env.ABCIQuery(&rpctypes.Context{}, path, data, opts.Height, opts.Prove)
}
func (c Client) BroadcastTxCommit(ctx context.Context, tx types.Tx) (*ctypes.ResultBroadcastTxCommit, error) {
return core.BroadcastTxCommit(&rpctypes.Context{}, tx)
return c.env.BroadcastTxCommit(&rpctypes.Context{}, tx)
}
func (c Client) BroadcastTxAsync(ctx context.Context, tx types.Tx) (*ctypes.ResultBroadcastTx, error) {
return core.BroadcastTxAsync(&rpctypes.Context{}, tx)
return c.env.BroadcastTxAsync(&rpctypes.Context{}, tx)
}
func (c Client) BroadcastTxSync(ctx context.Context, tx types.Tx) (*ctypes.ResultBroadcastTx, error) {
return core.BroadcastTxSync(&rpctypes.Context{}, tx)
return c.env.BroadcastTxSync(&rpctypes.Context{}, tx)
}
func (c Client) CheckTx(ctx context.Context, tx types.Tx) (*ctypes.ResultCheckTx, error) {
return core.CheckTx(&rpctypes.Context{}, tx)
return c.env.CheckTx(&rpctypes.Context{}, tx)
}
func (c Client) NetInfo(ctx context.Context) (*ctypes.ResultNetInfo, error) {
return core.NetInfo(&rpctypes.Context{})
return c.env.NetInfo(&rpctypes.Context{})
}
func (c Client) ConsensusState(ctx context.Context) (*ctypes.ResultConsensusState, error) {
return core.ConsensusState(&rpctypes.Context{})
return c.env.GetConsensusState(&rpctypes.Context{})
}
func (c Client) DumpConsensusState(ctx context.Context) (*ctypes.ResultDumpConsensusState, error) {
return core.DumpConsensusState(&rpctypes.Context{})
return c.env.DumpConsensusState(&rpctypes.Context{})
}
func (c Client) ConsensusParams(ctx context.Context, height *int64) (*ctypes.ResultConsensusParams, error) {
return core.ConsensusParams(&rpctypes.Context{}, height)
return c.env.ConsensusParams(&rpctypes.Context{}, height)
}
func (c Client) Health(ctx context.Context) (*ctypes.ResultHealth, error) {
return core.Health(&rpctypes.Context{})
return c.env.Health(&rpctypes.Context{})
}
func (c Client) DialSeeds(ctx context.Context, seeds []string) (*ctypes.ResultDialSeeds, error) {
return core.UnsafeDialSeeds(&rpctypes.Context{}, seeds)
return c.env.UnsafeDialSeeds(&rpctypes.Context{}, seeds)
}
func (c Client) DialPeers(
@ -147,33 +151,33 @@ func (c Client) DialPeers(
unconditional,
private bool,
) (*ctypes.ResultDialPeers, error) {
return core.UnsafeDialPeers(&rpctypes.Context{}, peers, persistent, unconditional, private)
return c.env.UnsafeDialPeers(&rpctypes.Context{}, peers, persistent, unconditional, private)
}
func (c Client) BlockchainInfo(ctx context.Context, minHeight, maxHeight int64) (*ctypes.ResultBlockchainInfo, error) {
return core.BlockchainInfo(&rpctypes.Context{}, minHeight, maxHeight)
return c.env.BlockchainInfo(&rpctypes.Context{}, minHeight, maxHeight)
}
func (c Client) Genesis(ctx context.Context) (*ctypes.ResultGenesis, error) {
return core.Genesis(&rpctypes.Context{})
return c.env.Genesis(&rpctypes.Context{})
}
func (c Client) Block(ctx context.Context, height *int64) (*ctypes.ResultBlock, error) {
return core.Block(&rpctypes.Context{}, height)
return c.env.Block(&rpctypes.Context{}, height)
}
func (c Client) BlockByHash(ctx context.Context, hash []byte) (*ctypes.ResultBlock, error) {
return core.BlockByHash(&rpctypes.Context{}, hash)
return c.env.BlockByHash(&rpctypes.Context{}, hash)
}
func (c Client) Commit(ctx context.Context, height *int64) (*ctypes.ResultCommit, error) {
return core.Commit(&rpctypes.Context{}, height)
return c.env.Commit(&rpctypes.Context{}, height)
}
func (c Client) Validators(ctx context.Context, height *int64, page, perPage *int) (*ctypes.ResultValidators, error) {
return core.Validators(&rpctypes.Context{}, height, page, perPage)
return c.env.Validators(&rpctypes.Context{}, height, page, perPage)
}
func (c Client) BroadcastEvidence(ctx context.Context, ev types.Evidence) (*ctypes.ResultBroadcastEvidence, error) {
return core.BroadcastEvidence(&rpctypes.Context{}, ev)
return c.env.BroadcastEvidence(&rpctypes.Context{}, ev)
}

+ 2
- 2
rpc/core/abci.go View File

@ -10,7 +10,7 @@ import (
// ABCIQuery queries the application for some information.
// More: https://docs.tendermint.com/master/rpc/#/ABCI/abci_query
func ABCIQuery(
func (env *Environment) ABCIQuery(
ctx *rpctypes.Context,
path string,
data bytes.HexBytes,
@ -32,7 +32,7 @@ func ABCIQuery(
// ABCIInfo gets some info about the application.
// More: https://docs.tendermint.com/master/rpc/#/ABCI/abci_info
func ABCIInfo(ctx *rpctypes.Context) (*ctypes.ResultABCIInfo, error) {
func (env *Environment) ABCIInfo(ctx *rpctypes.Context) (*ctypes.ResultABCIInfo, error) {
resInfo, err := env.ProxyAppQuery.InfoSync(ctx.Context(), proxy.RequestInfo)
if err != nil {
return nil, err


+ 13
- 10
rpc/core/blocks.go View File

@ -23,7 +23,10 @@ import (
// order (highest first).
//
// More: https://docs.tendermint.com/master/rpc/#/Info/blockchain
func BlockchainInfo(ctx *rpctypes.Context, minHeight, maxHeight int64) (*ctypes.ResultBlockchainInfo, error) {
func (env *Environment) BlockchainInfo(
ctx *rpctypes.Context,
minHeight, maxHeight int64) (*ctypes.ResultBlockchainInfo, error) {
const limit int64 = 20
var err error
@ -88,8 +91,8 @@ func filterMinMax(base, height, min, max, limit int64) (int64, int64, error) {
// Block gets block at a given height.
// If no height is provided, it will fetch the latest block.
// More: https://docs.tendermint.com/master/rpc/#/Info/block
func Block(ctx *rpctypes.Context, heightPtr *int64) (*ctypes.ResultBlock, error) {
height, err := getHeight(env.BlockStore.Height(), heightPtr)
func (env *Environment) Block(ctx *rpctypes.Context, heightPtr *int64) (*ctypes.ResultBlock, error) {
height, err := env.getHeight(env.BlockStore.Height(), heightPtr)
if err != nil {
return nil, err
}
@ -104,7 +107,7 @@ func Block(ctx *rpctypes.Context, heightPtr *int64) (*ctypes.ResultBlock, error)
// BlockByHash gets block by hash.
// More: https://docs.tendermint.com/master/rpc/#/Info/block_by_hash
func BlockByHash(ctx *rpctypes.Context, hash []byte) (*ctypes.ResultBlock, error) {
func (env *Environment) BlockByHash(ctx *rpctypes.Context, hash []byte) (*ctypes.ResultBlock, error) {
block := env.BlockStore.LoadBlockByHash(hash)
if block == nil {
return &ctypes.ResultBlock{BlockID: types.BlockID{}, Block: nil}, nil
@ -117,8 +120,8 @@ func BlockByHash(ctx *rpctypes.Context, hash []byte) (*ctypes.ResultBlock, error
// Commit gets block commit at a given height.
// If no height is provided, it will fetch the commit for the latest block.
// More: https://docs.tendermint.com/master/rpc/#/Info/commit
func Commit(ctx *rpctypes.Context, heightPtr *int64) (*ctypes.ResultCommit, error) {
height, err := getHeight(env.BlockStore.Height(), heightPtr)
func (env *Environment) Commit(ctx *rpctypes.Context, heightPtr *int64) (*ctypes.ResultCommit, error) {
height, err := env.getHeight(env.BlockStore.Height(), heightPtr)
if err != nil {
return nil, err
}
@ -148,8 +151,8 @@ func Commit(ctx *rpctypes.Context, heightPtr *int64) (*ctypes.ResultCommit, erro
// Thus response.results.deliver_tx[5] is the results of executing
// getBlock(h).Txs[5]
// More: https://docs.tendermint.com/master/rpc/#/Info/block_results
func BlockResults(ctx *rpctypes.Context, heightPtr *int64) (*ctypes.ResultBlockResults, error) {
height, err := getHeight(env.BlockStore.Height(), heightPtr)
func (env *Environment) BlockResults(ctx *rpctypes.Context, heightPtr *int64) (*ctypes.ResultBlockResults, error) {
height, err := env.getHeight(env.BlockStore.Height(), heightPtr)
if err != nil {
return nil, err
}
@ -171,7 +174,7 @@ func BlockResults(ctx *rpctypes.Context, heightPtr *int64) (*ctypes.ResultBlockR
// BlockSearch searches for a paginated set of blocks matching BeginBlock and
// EndBlock event search criteria.
func BlockSearch(
func (env *Environment) BlockSearch(
ctx *rpctypes.Context,
query string,
pagePtr, perPagePtr *int,
@ -207,7 +210,7 @@ func BlockSearch(
// paginate results
totalCount := len(results)
perPage := validatePerPage(perPagePtr)
perPage := env.validatePerPage(perPagePtr)
page, err := validatePage(pagePtr, perPage, totalCount)
if err != nil {


+ 2
- 2
rpc/core/blocks_test.go View File

@ -80,7 +80,7 @@ func TestBlockResults(t *testing.T) {
BeginBlock: &abci.ResponseBeginBlock{},
}
env = &Environment{}
env := &Environment{}
env.StateStore = sm.NewStore(dbm.NewMemDB())
err := env.StateStore.SaveABCIResponses(100, results)
require.NoError(t, err)
@ -105,7 +105,7 @@ func TestBlockResults(t *testing.T) {
}
for _, tc := range testCases {
res, err := BlockResults(&rpctypes.Context{}, &tc.height)
res, err := env.BlockResults(&rpctypes.Context{}, &tc.height)
if tc.wantErr {
assert.Error(t, err)
} else {


+ 14
- 7
rpc/core/consensus.go View File

@ -15,9 +15,13 @@ import (
// for the validators in the set as used in computing their Merkle root.
//
// More: https://docs.tendermint.com/master/rpc/#/Info/validators
func Validators(ctx *rpctypes.Context, heightPtr *int64, pagePtr, perPagePtr *int) (*ctypes.ResultValidators, error) {
func (env *Environment) Validators(
ctx *rpctypes.Context,
heightPtr *int64,
pagePtr, perPagePtr *int) (*ctypes.ResultValidators, error) {
// The latest validator that we know is the NextValidator of the last block.
height, err := getHeight(latestUncommittedHeight(), heightPtr)
height, err := env.getHeight(env.latestUncommittedHeight(), heightPtr)
if err != nil {
return nil, err
}
@ -28,7 +32,7 @@ func Validators(ctx *rpctypes.Context, heightPtr *int64, pagePtr, perPagePtr *in
}
totalCount := len(validators.Validators)
perPage := validatePerPage(perPagePtr)
perPage := env.validatePerPage(perPagePtr)
page, err := validatePage(pagePtr, perPage, totalCount)
if err != nil {
return nil, err
@ -48,7 +52,7 @@ func Validators(ctx *rpctypes.Context, heightPtr *int64, pagePtr, perPagePtr *in
// DumpConsensusState dumps consensus state.
// UNSTABLE
// More: https://docs.tendermint.com/master/rpc/#/Info/dump_consensus_state
func DumpConsensusState(ctx *rpctypes.Context) (*ctypes.ResultDumpConsensusState, error) {
func (env *Environment) DumpConsensusState(ctx *rpctypes.Context) (*ctypes.ResultDumpConsensusState, error) {
// Get Peer consensus states.
peers := env.P2PPeers.Peers().List()
peerStates := make([]ctypes.PeerStateInfo, len(peers))
@ -81,7 +85,7 @@ func DumpConsensusState(ctx *rpctypes.Context) (*ctypes.ResultDumpConsensusState
// ConsensusState returns a concise summary of the consensus state.
// UNSTABLE
// More: https://docs.tendermint.com/master/rpc/#/Info/consensus_state
func ConsensusState(ctx *rpctypes.Context) (*ctypes.ResultConsensusState, error) {
func (env *Environment) GetConsensusState(ctx *rpctypes.Context) (*ctypes.ResultConsensusState, error) {
// Get self round state.
bz, err := env.ConsensusState.GetRoundStateSimpleJSON()
return &ctypes.ResultConsensusState{RoundState: bz}, err
@ -90,10 +94,13 @@ func ConsensusState(ctx *rpctypes.Context) (*ctypes.ResultConsensusState, error)
// ConsensusParams gets the consensus parameters at the given block height.
// If no height is provided, it will fetch the latest consensus params.
// More: https://docs.tendermint.com/master/rpc/#/Info/consensus_params
func ConsensusParams(ctx *rpctypes.Context, heightPtr *int64) (*ctypes.ResultConsensusParams, error) {
func (env *Environment) ConsensusParams(
ctx *rpctypes.Context,
heightPtr *int64) (*ctypes.ResultConsensusParams, error) {
// The latest consensus params that we know is the consensus params after the
// last block.
height, err := getHeight(latestUncommittedHeight(), heightPtr)
height, err := env.getHeight(env.latestUncommittedHeight(), heightPtr)
if err != nil {
return nil, err
}


+ 1
- 1
rpc/core/dev.go View File

@ -6,7 +6,7 @@ import (
)
// UnsafeFlushMempool removes all transactions from the mempool.
func UnsafeFlushMempool(ctx *rpctypes.Context) (*ctypes.ResultUnsafeFlushMempool, error) {
func (env *Environment) UnsafeFlushMempool(ctx *rpctypes.Context) (*ctypes.ResultUnsafeFlushMempool, error) {
env.Mempool.Flush()
return &ctypes.ResultUnsafeFlushMempool{}, nil
}

+ 3
- 14
rpc/core/env.go View File

@ -27,17 +27,6 @@ const (
SubscribeTimeout = 5 * time.Second
)
var (
// set by Node
env *Environment
)
// SetEnvironment sets up the given Environment.
// It will race if multiple Node call SetEnvironment.
func SetEnvironment(e *Environment) {
env = e
}
//----------------------------------------------
// These interfaces are used by RPC and must be thread safe
@ -117,7 +106,7 @@ func validatePage(pagePtr *int, perPage, totalCount int) (int, error) {
return page, nil
}
func validatePerPage(perPagePtr *int) int {
func (env *Environment) validatePerPage(perPagePtr *int) int {
if perPagePtr == nil { // no per_page parameter
return defaultPerPage
}
@ -143,7 +132,7 @@ func validateSkipCount(page, perPage int) int {
}
// latestHeight can be either latest committed or uncommitted (+1) height.
func getHeight(latestHeight int64, heightPtr *int64) (int64, error) {
func (env *Environment) getHeight(latestHeight int64, heightPtr *int64) (int64, error) {
if heightPtr != nil {
height := *heightPtr
if height <= 0 {
@ -162,7 +151,7 @@ func getHeight(latestHeight int64, heightPtr *int64) (int64, error) {
return latestHeight, nil
}
func latestUncommittedHeight() int64 {
func (env *Environment) latestUncommittedHeight() int64 {
nodeIsSyncing := env.ConsensusReactor.WaitSync()
if nodeIsSyncing {
return env.BlockStore.Height()


+ 5
- 3
rpc/core/env_test.go View File

@ -70,19 +70,21 @@ func TestPaginationPerPage(t *testing.T) {
{maxPerPage + 1, maxPerPage},
}
env := &Environment{}
for _, c := range cases {
p := validatePerPage(&c.perPage)
p := env.validatePerPage(&c.perPage)
assert.Equal(t, c.newPerPage, p, fmt.Sprintf("%v", c))
}
// nil case
p := validatePerPage(nil)
p := env.validatePerPage(nil)
assert.Equal(t, defaultPerPage, p)
// test in unsafe mode
env.Config.Unsafe = true
perPage := 1000
p = validatePerPage(&perPage)
p = env.validatePerPage(&perPage)
assert.Equal(t, perPage, p)
env.Config.Unsafe = false
}

+ 3
- 3
rpc/core/events.go View File

@ -18,7 +18,7 @@ const (
// Subscribe for events via WebSocket.
// More: https://docs.tendermint.com/master/rpc/#/Websocket/subscribe
func Subscribe(ctx *rpctypes.Context, query string) (*ctypes.ResultSubscribe, error) {
func (env *Environment) Subscribe(ctx *rpctypes.Context, query string) (*ctypes.ResultSubscribe, error) {
addr := ctx.RemoteAddr()
if env.EventBus.NumClients() >= env.Config.MaxSubscriptionClients {
@ -85,7 +85,7 @@ func Subscribe(ctx *rpctypes.Context, query string) (*ctypes.ResultSubscribe, er
// Unsubscribe from events via WebSocket.
// More: https://docs.tendermint.com/master/rpc/#/Websocket/unsubscribe
func Unsubscribe(ctx *rpctypes.Context, query string) (*ctypes.ResultUnsubscribe, error) {
func (env *Environment) Unsubscribe(ctx *rpctypes.Context, query string) (*ctypes.ResultUnsubscribe, error) {
addr := ctx.RemoteAddr()
env.Logger.Info("Unsubscribe from query", "remote", addr, "query", query)
q, err := tmquery.New(query)
@ -101,7 +101,7 @@ func Unsubscribe(ctx *rpctypes.Context, query string) (*ctypes.ResultUnsubscribe
// UnsubscribeAll from all events via WebSocket.
// More: https://docs.tendermint.com/master/rpc/#/Websocket/unsubscribe_all
func UnsubscribeAll(ctx *rpctypes.Context) (*ctypes.ResultUnsubscribe, error) {
func (env *Environment) UnsubscribeAll(ctx *rpctypes.Context) (*ctypes.ResultUnsubscribe, error) {
addr := ctx.RemoteAddr()
env.Logger.Info("Unsubscribe from all", "remote", addr)
err := env.EventBus.UnsubscribeAll(context.Background(), addr)


+ 4
- 1
rpc/core/evidence.go View File

@ -10,7 +10,10 @@ import (
// BroadcastEvidence broadcasts evidence of the misbehavior.
// More: https://docs.tendermint.com/master/rpc/#/Evidence/broadcast_evidence
func BroadcastEvidence(ctx *rpctypes.Context, ev types.Evidence) (*ctypes.ResultBroadcastEvidence, error) {
func (env *Environment) BroadcastEvidence(
ctx *rpctypes.Context,
ev types.Evidence) (*ctypes.ResultBroadcastEvidence, error) {
if ev == nil {
return nil, fmt.Errorf("%w: no evidence was provided", ctypes.ErrInvalidRequest)
}


+ 1
- 1
rpc/core/health.go View File

@ -8,6 +8,6 @@ import (
// Health gets node health. Returns empty result (200 OK) on success, no
// response - in case of an error.
// More: https://docs.tendermint.com/master/rpc/#/Info/health
func Health(ctx *rpctypes.Context) (*ctypes.ResultHealth, error) {
func (env *Environment) Health(ctx *rpctypes.Context) (*ctypes.ResultHealth, error) {
return &ctypes.ResultHealth{}, nil
}

+ 7
- 7
rpc/core/mempool.go View File

@ -19,7 +19,7 @@ import (
// BroadcastTxAsync returns right away, with no response. Does not wait for
// CheckTx nor DeliverTx results.
// More: https://docs.tendermint.com/master/rpc/#/Tx/broadcast_tx_async
func BroadcastTxAsync(ctx *rpctypes.Context, tx types.Tx) (*ctypes.ResultBroadcastTx, error) {
func (env *Environment) BroadcastTxAsync(ctx *rpctypes.Context, tx types.Tx) (*ctypes.ResultBroadcastTx, error) {
err := env.Mempool.CheckTx(tx, nil, mempl.TxInfo{Context: ctx.Context()})
if err != nil {
@ -31,7 +31,7 @@ func BroadcastTxAsync(ctx *rpctypes.Context, tx types.Tx) (*ctypes.ResultBroadca
// BroadcastTxSync returns with the response from CheckTx. Does not wait for
// DeliverTx result.
// More: https://docs.tendermint.com/master/rpc/#/Tx/broadcast_tx_sync
func BroadcastTxSync(ctx *rpctypes.Context, tx types.Tx) (*ctypes.ResultBroadcastTx, error) {
func (env *Environment) BroadcastTxSync(ctx *rpctypes.Context, tx types.Tx) (*ctypes.ResultBroadcastTx, error) {
resCh := make(chan *abci.Response, 1)
err := env.Mempool.CheckTx(tx, func(res *abci.Response) {
resCh <- res
@ -52,7 +52,7 @@ func BroadcastTxSync(ctx *rpctypes.Context, tx types.Tx) (*ctypes.ResultBroadcas
// BroadcastTxCommit returns with the responses from CheckTx and DeliverTx.
// More: https://docs.tendermint.com/master/rpc/#/Tx/broadcast_tx_commit
func BroadcastTxCommit(ctx *rpctypes.Context, tx types.Tx) (*ctypes.ResultBroadcastTxCommit, error) {
func (env *Environment) BroadcastTxCommit(ctx *rpctypes.Context, tx types.Tx) (*ctypes.ResultBroadcastTxCommit, error) {
subscriber := ctx.RemoteAddr()
if env.EventBus.NumClients() >= env.Config.MaxSubscriptionClients {
@ -134,9 +134,9 @@ func BroadcastTxCommit(ctx *rpctypes.Context, tx types.Tx) (*ctypes.ResultBroadc
// UnconfirmedTxs gets unconfirmed transactions (maximum ?limit entries)
// including their number.
// More: https://docs.tendermint.com/master/rpc/#/Info/unconfirmed_txs
func UnconfirmedTxs(ctx *rpctypes.Context, limitPtr *int) (*ctypes.ResultUnconfirmedTxs, error) {
func (env *Environment) UnconfirmedTxs(ctx *rpctypes.Context, limitPtr *int) (*ctypes.ResultUnconfirmedTxs, error) {
// reuse per_page validator
limit := validatePerPage(limitPtr)
limit := env.validatePerPage(limitPtr)
txs := env.Mempool.ReapMaxTxs(limit)
return &ctypes.ResultUnconfirmedTxs{
@ -148,7 +148,7 @@ func UnconfirmedTxs(ctx *rpctypes.Context, limitPtr *int) (*ctypes.ResultUnconfi
// NumUnconfirmedTxs gets number of unconfirmed transactions.
// More: https://docs.tendermint.com/master/rpc/#/Info/num_unconfirmed_txs
func NumUnconfirmedTxs(ctx *rpctypes.Context) (*ctypes.ResultUnconfirmedTxs, error) {
func (env *Environment) NumUnconfirmedTxs(ctx *rpctypes.Context) (*ctypes.ResultUnconfirmedTxs, error) {
return &ctypes.ResultUnconfirmedTxs{
Count: env.Mempool.Size(),
Total: env.Mempool.Size(),
@ -158,7 +158,7 @@ func NumUnconfirmedTxs(ctx *rpctypes.Context) (*ctypes.ResultUnconfirmedTxs, err
// CheckTx checks the transaction without executing it. The transaction won't
// be added to the mempool either.
// More: https://docs.tendermint.com/master/rpc/#/Tx/check_tx
func CheckTx(ctx *rpctypes.Context, tx types.Tx) (*ctypes.ResultCheckTx, error) {
func (env *Environment) CheckTx(ctx *rpctypes.Context, tx types.Tx) (*ctypes.ResultCheckTx, error) {
res, err := env.ProxyAppMempool.CheckTxSync(ctx.Context(), abci.RequestCheckTx{Tx: tx})
if err != nil {
return nil, err


+ 8
- 5
rpc/core/net.go View File

@ -11,7 +11,7 @@ import (
// NetInfo returns network info.
// More: https://docs.tendermint.com/master/rpc/#/Info/net_info
func NetInfo(ctx *rpctypes.Context) (*ctypes.ResultNetInfo, error) {
func (env *Environment) NetInfo(ctx *rpctypes.Context) (*ctypes.ResultNetInfo, error) {
peersList := env.P2PPeers.Peers().List()
peers := make([]ctypes.Peer, 0, len(peersList))
for _, peer := range peersList {
@ -34,7 +34,7 @@ func NetInfo(ctx *rpctypes.Context) (*ctypes.ResultNetInfo, error) {
}
// UnsafeDialSeeds dials the given seeds (comma-separated id@IP:PORT).
func UnsafeDialSeeds(ctx *rpctypes.Context, seeds []string) (*ctypes.ResultDialSeeds, error) {
func (env *Environment) UnsafeDialSeeds(ctx *rpctypes.Context, seeds []string) (*ctypes.ResultDialSeeds, error) {
if len(seeds) == 0 {
return &ctypes.ResultDialSeeds{}, fmt.Errorf("%w: no seeds provided", ctypes.ErrInvalidRequest)
}
@ -47,8 +47,11 @@ func UnsafeDialSeeds(ctx *rpctypes.Context, seeds []string) (*ctypes.ResultDialS
// UnsafeDialPeers dials the given peers (comma-separated id@IP:PORT),
// optionally making them persistent.
func UnsafeDialPeers(ctx *rpctypes.Context, peers []string, persistent, unconditional, private bool) (
*ctypes.ResultDialPeers, error) {
func (env *Environment) UnsafeDialPeers(
ctx *rpctypes.Context,
peers []string,
persistent, unconditional, private bool) (*ctypes.ResultDialPeers, error) {
if len(peers) == 0 {
return &ctypes.ResultDialPeers{}, fmt.Errorf("%w: no peers provided", ctypes.ErrInvalidRequest)
}
@ -88,7 +91,7 @@ func UnsafeDialPeers(ctx *rpctypes.Context, peers []string, persistent, uncondit
// Genesis returns genesis file.
// More: https://docs.tendermint.com/master/rpc/#/Info/genesis
func Genesis(ctx *rpctypes.Context) (*ctypes.ResultGenesis, error) {
func (env *Environment) Genesis(ctx *rpctypes.Context) (*ctypes.ResultGenesis, error) {
return &ctypes.ResultGenesis{Genesis: env.GenDoc}, nil
}


+ 4
- 2
rpc/core/net_test.go View File

@ -23,6 +23,7 @@ func TestUnsafeDialSeeds(t *testing.T) {
}
})
env := &Environment{}
env.Logger = log.TestingLogger()
env.P2PPeers = sw
@ -36,7 +37,7 @@ func TestUnsafeDialSeeds(t *testing.T) {
}
for _, tc := range testCases {
res, err := UnsafeDialSeeds(&rpctypes.Context{}, tc.seeds)
res, err := env.UnsafeDialSeeds(&rpctypes.Context{}, tc.seeds)
if tc.isErr {
assert.Error(t, err)
} else {
@ -62,6 +63,7 @@ func TestUnsafeDialPeers(t *testing.T) {
}
})
env := &Environment{}
env.Logger = log.TestingLogger()
env.P2PPeers = sw
@ -76,7 +78,7 @@ func TestUnsafeDialPeers(t *testing.T) {
}
for _, tc := range testCases {
res, err := UnsafeDialPeers(&rpctypes.Context{}, tc.peers, tc.persistence, tc.unconditional, tc.private)
res, err := env.UnsafeDialPeers(&rpctypes.Context{}, tc.peers, tc.persistence, tc.unconditional, tc.private)
if tc.isErr {
assert.Error(t, err)
} else {


+ 46
- 42
rpc/core/routes.go View File

@ -6,51 +6,55 @@ import (
// TODO: better system than "unsafe" prefix
type RoutesMap map[string]*rpc.RPCFunc
// Routes is a map of available routes.
var Routes = map[string]*rpc.RPCFunc{
// subscribe/unsubscribe are reserved for websocket events.
"subscribe": rpc.NewWSRPCFunc(Subscribe, "query"),
"unsubscribe": rpc.NewWSRPCFunc(Unsubscribe, "query"),
"unsubscribe_all": rpc.NewWSRPCFunc(UnsubscribeAll, ""),
// info API
"health": rpc.NewRPCFunc(Health, "", false),
"status": rpc.NewRPCFunc(Status, "", false),
"net_info": rpc.NewRPCFunc(NetInfo, "", false),
"blockchain": rpc.NewRPCFunc(BlockchainInfo, "minHeight,maxHeight", true),
"genesis": rpc.NewRPCFunc(Genesis, "", true),
"block": rpc.NewRPCFunc(Block, "height", true),
"block_by_hash": rpc.NewRPCFunc(BlockByHash, "hash", true),
"block_results": rpc.NewRPCFunc(BlockResults, "height", true),
"commit": rpc.NewRPCFunc(Commit, "height", true),
"check_tx": rpc.NewRPCFunc(CheckTx, "tx", true),
"tx": rpc.NewRPCFunc(Tx, "hash,prove", true),
"tx_search": rpc.NewRPCFunc(TxSearch, "query,prove,page,per_page,order_by", false),
"block_search": rpc.NewRPCFunc(BlockSearch, "query,page,per_page,order_by", false),
"validators": rpc.NewRPCFunc(Validators, "height,page,per_page", true),
"dump_consensus_state": rpc.NewRPCFunc(DumpConsensusState, "", false),
"consensus_state": rpc.NewRPCFunc(ConsensusState, "", false),
"consensus_params": rpc.NewRPCFunc(ConsensusParams, "height", true),
"unconfirmed_txs": rpc.NewRPCFunc(UnconfirmedTxs, "limit", false),
"num_unconfirmed_txs": rpc.NewRPCFunc(NumUnconfirmedTxs, "", false),
// tx broadcast API
"broadcast_tx_commit": rpc.NewRPCFunc(BroadcastTxCommit, "tx", false),
"broadcast_tx_sync": rpc.NewRPCFunc(BroadcastTxSync, "tx", false),
"broadcast_tx_async": rpc.NewRPCFunc(BroadcastTxAsync, "tx", false),
// abci API
"abci_query": rpc.NewRPCFunc(ABCIQuery, "path,data,height,prove", false),
"abci_info": rpc.NewRPCFunc(ABCIInfo, "", true),
// evidence API
"broadcast_evidence": rpc.NewRPCFunc(BroadcastEvidence, "evidence", false),
func (env *Environment) GetRoutes() RoutesMap {
return RoutesMap{
// subscribe/unsubscribe are reserved for websocket events.
"subscribe": rpc.NewWSRPCFunc(env.Subscribe, "query"),
"unsubscribe": rpc.NewWSRPCFunc(env.Unsubscribe, "query"),
"unsubscribe_all": rpc.NewWSRPCFunc(env.UnsubscribeAll, ""),
// info API
"health": rpc.NewRPCFunc(env.Health, "", false),
"status": rpc.NewRPCFunc(env.Status, "", false),
"net_info": rpc.NewRPCFunc(env.NetInfo, "", false),
"blockchain": rpc.NewRPCFunc(env.BlockchainInfo, "minHeight,maxHeight", true),
"genesis": rpc.NewRPCFunc(env.Genesis, "", true),
"block": rpc.NewRPCFunc(env.Block, "height", true),
"block_by_hash": rpc.NewRPCFunc(env.BlockByHash, "hash", true),
"block_results": rpc.NewRPCFunc(env.BlockResults, "height", true),
"commit": rpc.NewRPCFunc(env.Commit, "height", true),
"check_tx": rpc.NewRPCFunc(env.CheckTx, "tx", true),
"tx": rpc.NewRPCFunc(env.Tx, "hash,prove", true),
"tx_search": rpc.NewRPCFunc(env.TxSearch, "query,prove,page,per_page,order_by", false),
"block_search": rpc.NewRPCFunc(env.BlockSearch, "query,page,per_page,order_by", false),
"validators": rpc.NewRPCFunc(env.Validators, "height,page,per_page", true),
"dump_consensus_state": rpc.NewRPCFunc(env.DumpConsensusState, "", false),
"consensus_state": rpc.NewRPCFunc(env.GetConsensusState, "", false),
"consensus_params": rpc.NewRPCFunc(env.ConsensusParams, "height", true),
"unconfirmed_txs": rpc.NewRPCFunc(env.UnconfirmedTxs, "limit", false),
"num_unconfirmed_txs": rpc.NewRPCFunc(env.NumUnconfirmedTxs, "", false),
// tx broadcast API
"broadcast_tx_commit": rpc.NewRPCFunc(env.BroadcastTxCommit, "tx", false),
"broadcast_tx_sync": rpc.NewRPCFunc(env.BroadcastTxSync, "tx", false),
"broadcast_tx_async": rpc.NewRPCFunc(env.BroadcastTxAsync, "tx", false),
// abci API
"abci_query": rpc.NewRPCFunc(env.ABCIQuery, "path,data,height,prove", false),
"abci_info": rpc.NewRPCFunc(env.ABCIInfo, "", true),
// evidence API
"broadcast_evidence": rpc.NewRPCFunc(env.BroadcastEvidence, "evidence", false),
}
}
// AddUnsafeRoutes adds unsafe routes.
func AddUnsafeRoutes() {
func (env *Environment) AddUnsafe(routes RoutesMap) {
// control API
Routes["dial_seeds"] = rpc.NewRPCFunc(UnsafeDialSeeds, "seeds", false)
Routes["dial_peers"] = rpc.NewRPCFunc(UnsafeDialPeers, "peers,persistent,unconditional,private", false)
Routes["unsafe_flush_mempool"] = rpc.NewRPCFunc(UnsafeFlushMempool, "", false)
routes["dial_seeds"] = rpc.NewRPCFunc(env.UnsafeDialSeeds, "seeds", false)
routes["dial_peers"] = rpc.NewRPCFunc(env.UnsafeDialPeers, "peers,persistent,unconditional,private", false)
routes["unsafe_flush_mempool"] = rpc.NewRPCFunc(env.UnsafeFlushMempool, "", false)
}

+ 3
- 3
rpc/core/status.go View File

@ -13,7 +13,7 @@ import (
// Status returns Tendermint status including node info, pubkey, latest block
// hash, app hash, block height and time.
// More: https://docs.tendermint.com/master/rpc/#/Info/status
func Status(ctx *rpctypes.Context) (*ctypes.ResultStatus, error) {
func (env *Environment) Status(ctx *rpctypes.Context) (*ctypes.ResultStatus, error) {
var (
earliestBlockHeight int64
earliestBlockHash tmbytes.HexBytes
@ -47,7 +47,7 @@ func Status(ctx *rpctypes.Context) (*ctypes.ResultStatus, error) {
// Return the very last voting power, not the voting power of this validator
// during the last block.
var votingPower int64
if val := validatorAtHeight(latestUncommittedHeight()); val != nil {
if val := env.validatorAtHeight(env.latestUncommittedHeight()); val != nil {
votingPower = val.VotingPower
}
validatorInfo := ctypes.ValidatorInfo{}
@ -77,7 +77,7 @@ func Status(ctx *rpctypes.Context) (*ctypes.ResultStatus, error) {
return result, nil
}
func validatorAtHeight(h int64) *types.Validator {
func (env *Environment) validatorAtHeight(h int64) *types.Validator {
valsWithH, err := env.StateStore.LoadValidators(h)
if err != nil {
return nil


+ 3
- 3
rpc/core/tx.go View File

@ -17,7 +17,7 @@ import (
// transaction is in the mempool, invalidated, or was not sent in the first
// place.
// More: https://docs.tendermint.com/master/rpc/#/Info/tx
func Tx(ctx *rpctypes.Context, hash []byte, prove bool) (*ctypes.ResultTx, error) {
func (env *Environment) Tx(ctx *rpctypes.Context, hash []byte, prove bool) (*ctypes.ResultTx, error) {
// if index is disabled, return error
if _, ok := env.TxIndexer.(*null.TxIndex); ok {
return nil, fmt.Errorf("transaction indexing is disabled")
@ -54,7 +54,7 @@ func Tx(ctx *rpctypes.Context, hash []byte, prove bool) (*ctypes.ResultTx, error
// TxSearch allows you to query for multiple transactions results. It returns a
// list of transactions (maximum ?per_page entries) and the total count.
// More: https://docs.tendermint.com/master/rpc/#/Info/tx_search
func TxSearch(
func (env *Environment) TxSearch(
ctx *rpctypes.Context,
query string,
prove bool,
@ -99,7 +99,7 @@ func TxSearch(
// paginate results
totalCount := len(results)
perPage := validatePerPage(perPagePtr)
perPage := env.validatePerPage(perPagePtr)
page, err := validatePage(pagePtr, perPage, totalCount)
if err != nil {


+ 2
- 1
rpc/grpc/api.go View File

@ -9,6 +9,7 @@ import (
)
type broadcastAPI struct {
env *core.Environment
}
func (bapi *broadcastAPI) Ping(ctx context.Context, req *RequestPing) (*ResponsePing, error) {
@ -19,7 +20,7 @@ func (bapi *broadcastAPI) Ping(ctx context.Context, req *RequestPing) (*Response
func (bapi *broadcastAPI) BroadcastTx(ctx context.Context, req *RequestBroadcastTx) (*ResponseBroadcastTx, error) {
// NOTE: there's no way to get client's remote address
// see https://stackoverflow.com/questions/33684570/session-and-remote-ip-address-in-grpc-go
res, err := core.BroadcastTxCommit(&rpctypes.Context{}, req.Tx)
res, err := bapi.env.BroadcastTxCommit(&rpctypes.Context{}, req.Tx)
if err != nil {
return nil, err
}


+ 3
- 2
rpc/grpc/client_server.go View File

@ -7,6 +7,7 @@ import (
"google.golang.org/grpc"
tmnet "github.com/tendermint/tendermint/libs/net"
"github.com/tendermint/tendermint/rpc/core"
)
// Config is an gRPC server configuration.
@ -17,9 +18,9 @@ type Config struct {
// StartGRPCServer starts a new gRPC BroadcastAPIServer using the given
// net.Listener.
// NOTE: This function blocks - you may want to call it in a go-routine.
func StartGRPCServer(ln net.Listener) error {
func StartGRPCServer(env *core.Environment, ln net.Listener) error {
grpcServer := grpc.NewServer()
RegisterBroadcastAPIServer(grpcServer, &broadcastAPI{})
RegisterBroadcastAPIServer(grpcServer, &broadcastAPI{env: env})
return grpcServer.Serve(ln)
}


Loading…
Cancel
Save