limit number of /subscribe clients and queries per client (#3269)

* limit number of /subscribe clients and queries per client

Add the following config variables (under [rpc] section):
  * max_subscription_clients
  * max_subscriptions_per_client
  * timeout_broadcast_tx_commit

Fixes #2826

new HTTPClient interface for subscriptions

finalize HTTPClient events interface

remove EventSubscriber

fix data race

```
WARNING: DATA RACE
Read at 0x00c000a36060 by goroutine 129:
  github.com/tendermint/tendermint/rpc/client.(*Local).Subscribe.func1()
      /go/src/github.com/tendermint/tendermint/rpc/client/localclient.go:168 +0x1f0

Previous write at 0x00c000a36060 by goroutine 132:
  github.com/tendermint/tendermint/rpc/client.(*Local).Subscribe()
      /go/src/github.com/tendermint/tendermint/rpc/client/localclient.go:191 +0x4e0
  github.com/tendermint/tendermint/rpc/client.WaitForOneEvent()
      /go/src/github.com/tendermint/tendermint/rpc/client/helpers.go:64 +0x178
  github.com/tendermint/tendermint/rpc/client_test.TestTxEventsSentWithBroadcastTxSync.func1()
      /go/src/github.com/tendermint/tendermint/rpc/client/event_test.go:139 +0x298
  testing.tRunner()
      /usr/local/go/src/testing/testing.go:827 +0x162

Goroutine 129 (running) created at:
  github.com/tendermint/tendermint/rpc/client.(*Local).Subscribe()
      /go/src/github.com/tendermint/tendermint/rpc/client/localclient.go:164 +0x4b7
  github.com/tendermint/tendermint/rpc/client.WaitForOneEvent()
      /go/src/github.com/tendermint/tendermint/rpc/client/helpers.go:64 +0x178
  github.com/tendermint/tendermint/rpc/client_test.TestTxEventsSentWithBroadcastTxSync.func1()
      /go/src/github.com/tendermint/tendermint/rpc/client/event_test.go:139 +0x298
  testing.tRunner()
      /usr/local/go/src/testing/testing.go:827 +0x162

Goroutine 132 (running) created at:
  testing.(*T).Run()
      /usr/local/go/src/testing/testing.go:878 +0x659
  github.com/tendermint/tendermint/rpc/client_test.TestTxEventsSentWithBroadcastTxSync()
      /go/src/github.com/tendermint/tendermint/rpc/client/event_test.go:119 +0x186
  testing.tRunner()
      /usr/local/go/src/testing/testing.go:827 +0x162
==================
```

lite client works (tested manually)

godoc comments

httpclient: do not close the out channel

use TimeoutBroadcastTxCommit

no timeout for unsubscribe

but 1s Local (5s HTTP) timeout for resubscribe

format code

change Subscribe#out cap to 1

and replace config vars with RPCConfig

TimeoutBroadcastTxCommit can't be greater than rpcserver.WriteTimeout

rpc: Context as first parameter to all functions

reformat code

fixes after my own review

fixes after Ethan's review

add test stubs

fix config.toml

* fixes after manual testing

- rpc: do not recommend to use BroadcastTxCommit because it's slow and wastes
Tendermint resources (pubsub)
- rpc: better error in Subscribe and BroadcastTxCommit
- HTTPClient: do not resubscribe if err = ErrAlreadySubscribed

* fixes after Ismail's review

* Update rpc/grpc/grpc_test.go

Co-Authored-By: melekes <anton.kalyaev@gmail.com>
This commit is contained in:
Anton Kaliaev
2019-03-11 22:45:58 +04:00
committed by GitHub
parent 15f621141d
commit d741c7b478
36 changed files with 657 additions and 350 deletions

View File

@ -5,6 +5,7 @@ import (
cmn "github.com/tendermint/tendermint/libs/common"
"github.com/tendermint/tendermint/proxy"
ctypes "github.com/tendermint/tendermint/rpc/core/types"
rpctypes "github.com/tendermint/tendermint/rpc/lib/types"
)
// Query the application for some information.
@ -52,7 +53,7 @@ import (
// | data | []byte | false | true | Data |
// | height | int64 | 0 | false | Height (0 means latest) |
// | prove | bool | false | false | Includes proof if true |
func ABCIQuery(path string, data cmn.HexBytes, height int64, prove bool) (*ctypes.ResultABCIQuery, error) {
func ABCIQuery(ctx *rpctypes.Context, path string, data cmn.HexBytes, height int64, prove bool) (*ctypes.ResultABCIQuery, error) {
resQuery, err := proxyAppQuery.QuerySync(abci.RequestQuery{
Path: path,
Data: data,
@ -96,7 +97,7 @@ func ABCIQuery(path string, data cmn.HexBytes, height int64, prove bool) (*ctype
// "jsonrpc": "2.0"
// }
// ```
func ABCIInfo() (*ctypes.ResultABCIInfo, error) {
func ABCIInfo(ctx *rpctypes.Context) (*ctypes.ResultABCIInfo, error) {
resInfo, err := proxyAppQuery.InfoSync(proxy.RequestInfo)
if err != nil {
return nil, err

View File

@ -5,6 +5,7 @@ import (
cmn "github.com/tendermint/tendermint/libs/common"
ctypes "github.com/tendermint/tendermint/rpc/core/types"
rpctypes "github.com/tendermint/tendermint/rpc/lib/types"
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
)
@ -68,7 +69,7 @@ import (
// ```
//
// <aside class="notice">Returns at most 20 items.</aside>
func BlockchainInfo(minHeight, maxHeight int64) (*ctypes.ResultBlockchainInfo, error) {
func BlockchainInfo(ctx *rpctypes.Context, minHeight, maxHeight int64) (*ctypes.ResultBlockchainInfo, error) {
// maximum 20 block metas
const limit int64 = 20
@ -226,7 +227,7 @@ func filterMinMax(height, min, max, limit int64) (int64, int64, error) {
// "jsonrpc": "2.0"
// }
// ```
func Block(heightPtr *int64) (*ctypes.ResultBlock, error) {
func Block(ctx *rpctypes.Context, heightPtr *int64) (*ctypes.ResultBlock, error) {
storeHeight := blockStore.Height()
height, err := getHeight(storeHeight, heightPtr)
if err != nil {
@ -313,7 +314,7 @@ func Block(heightPtr *int64) (*ctypes.ResultBlock, error) {
// "jsonrpc": "2.0"
// }
// ```
func Commit(heightPtr *int64) (*ctypes.ResultCommit, error) {
func Commit(ctx *rpctypes.Context, heightPtr *int64) (*ctypes.ResultCommit, error) {
storeHeight := blockStore.Height()
height, err := getHeight(storeHeight, heightPtr)
if err != nil {
@ -372,7 +373,7 @@ func Commit(heightPtr *int64) (*ctypes.ResultCommit, error) {
// ]
// }
// ```
func BlockResults(heightPtr *int64) (*ctypes.ResultBlockResults, error) {
func BlockResults(ctx *rpctypes.Context, heightPtr *int64) (*ctypes.ResultBlockResults, error) {
storeHeight := blockStore.Height()
height, err := getHeight(storeHeight, heightPtr)
if err != nil {

View File

@ -3,6 +3,7 @@ package core
import (
cm "github.com/tendermint/tendermint/consensus"
ctypes "github.com/tendermint/tendermint/rpc/core/types"
rpctypes "github.com/tendermint/tendermint/rpc/lib/types"
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
)
@ -47,7 +48,7 @@ import (
// "jsonrpc": "2.0"
// }
// ```
func Validators(heightPtr *int64) (*ctypes.ResultValidators, error) {
func Validators(ctx *rpctypes.Context, heightPtr *int64) (*ctypes.ResultValidators, error) {
// The latest validator that we know is the
// NextValidator of the last block.
height := consensusState.GetState().LastBlockHeight + 1
@ -200,7 +201,7 @@ func Validators(heightPtr *int64) (*ctypes.ResultValidators, error) {
// }
// }
// ```
func DumpConsensusState() (*ctypes.ResultDumpConsensusState, error) {
func DumpConsensusState(ctx *rpctypes.Context) (*ctypes.ResultDumpConsensusState, error) {
// Get Peer consensus states.
peers := p2pPeers.Peers().List()
peerStates := make([]ctypes.PeerStateInfo, len(peers))
@ -277,7 +278,7 @@ func DumpConsensusState() (*ctypes.ResultDumpConsensusState, error) {
// }
//}
//```
func ConsensusState() (*ctypes.ResultConsensusState, error) {
func ConsensusState(ctx *rpctypes.Context) (*ctypes.ResultConsensusState, error) {
// Get self round state.
bz, err := consensusState.GetRoundStateSimpleJSON()
return &ctypes.ResultConsensusState{RoundState: bz}, err
@ -320,7 +321,7 @@ func ConsensusState() (*ctypes.ResultConsensusState, error) {
// }
// }
// ```
func ConsensusParams(heightPtr *int64) (*ctypes.ResultConsensusParams, error) {
func ConsensusParams(ctx *rpctypes.Context, heightPtr *int64) (*ctypes.ResultConsensusParams, error) {
height := consensusState.GetState().LastBlockHeight + 1
height, err := getHeight(height, heightPtr)
if err != nil {

View File

@ -5,16 +5,19 @@ import (
"runtime/pprof"
ctypes "github.com/tendermint/tendermint/rpc/core/types"
rpctypes "github.com/tendermint/tendermint/rpc/lib/types"
)
func UnsafeFlushMempool() (*ctypes.ResultUnsafeFlushMempool, error) {
// UnsafeFlushMempool removes all transactions from the mempool.
func UnsafeFlushMempool(ctx *rpctypes.Context) (*ctypes.ResultUnsafeFlushMempool, error) {
mempool.Flush()
return &ctypes.ResultUnsafeFlushMempool{}, nil
}
var profFile *os.File
func UnsafeStartCPUProfiler(filename string) (*ctypes.ResultUnsafeProfile, error) {
// UnsafeStartCPUProfiler starts a pprof profiler using the given filename.
func UnsafeStartCPUProfiler(ctx *rpctypes.Context, filename string) (*ctypes.ResultUnsafeProfile, error) {
var err error
profFile, err = os.Create(filename)
if err != nil {
@ -27,7 +30,8 @@ func UnsafeStartCPUProfiler(filename string) (*ctypes.ResultUnsafeProfile, error
return &ctypes.ResultUnsafeProfile{}, nil
}
func UnsafeStopCPUProfiler() (*ctypes.ResultUnsafeProfile, error) {
// UnsafeStopCPUProfiler stops the running pprof profiler.
func UnsafeStopCPUProfiler(ctx *rpctypes.Context) (*ctypes.ResultUnsafeProfile, error) {
pprof.StopCPUProfile()
if err := profFile.Close(); err != nil {
return nil, err
@ -35,7 +39,8 @@ func UnsafeStopCPUProfiler() (*ctypes.ResultUnsafeProfile, error) {
return &ctypes.ResultUnsafeProfile{}, nil
}
func UnsafeWriteHeapProfile(filename string) (*ctypes.ResultUnsafeProfile, error) {
// UnsafeWriteHeapProfile dumps a heap profile to the given filename.
func UnsafeWriteHeapProfile(ctx *rpctypes.Context, filename string) (*ctypes.ResultUnsafeProfile, error) {
memProfFile, err := os.Create(filename)
if err != nil {
return nil, err

View File

@ -6,10 +6,10 @@ import (
"github.com/pkg/errors"
tmpubsub "github.com/tendermint/tendermint/libs/pubsub"
tmquery "github.com/tendermint/tendermint/libs/pubsub/query"
ctypes "github.com/tendermint/tendermint/rpc/core/types"
rpctypes "github.com/tendermint/tendermint/rpc/lib/types"
tmtypes "github.com/tendermint/tendermint/types"
)
// Subscribe for events via WebSocket.
@ -90,8 +90,15 @@ import (
// | query | string | "" | true | Query |
//
// <aside class="notice">WebSocket only</aside>
func Subscribe(wsCtx rpctypes.WSRPCContext, query string) (*ctypes.ResultSubscribe, error) {
addr := wsCtx.GetRemoteAddr()
func Subscribe(ctx *rpctypes.Context, query string) (*ctypes.ResultSubscribe, error) {
addr := ctx.RemoteAddr()
if eventBus.NumClients() >= config.MaxSubscriptionClients {
return nil, fmt.Errorf("max_subscription_clients %d reached", config.MaxSubscriptionClients)
} else if eventBus.NumClientSubscriptions(addr) >= config.MaxSubscriptionsPerClient {
return nil, fmt.Errorf("max_subscriptions_per_client %d reached", config.MaxSubscriptionsPerClient)
}
logger.Info("Subscribe to query", "remote", addr, "query", query)
q, err := tmquery.New(query)
@ -99,9 +106,9 @@ func Subscribe(wsCtx rpctypes.WSRPCContext, query string) (*ctypes.ResultSubscri
return nil, errors.Wrap(err, "failed to parse query")
}
ctx, cancel := context.WithTimeout(context.Background(), subscribeTimeout)
subCtx, cancel := context.WithTimeout(context.Background(), subscribeTimeout)
defer cancel()
sub, err := eventBusFor(wsCtx).Subscribe(ctx, addr, q)
sub, err := eventBus.Subscribe(subCtx, addr, q)
if err != nil {
return nil, err
}
@ -111,18 +118,26 @@ func Subscribe(wsCtx rpctypes.WSRPCContext, query string) (*ctypes.ResultSubscri
select {
case msg := <-sub.Out():
resultEvent := &ctypes.ResultEvent{Query: query, Data: msg.Data(), Tags: msg.Tags()}
wsCtx.TryWriteRPCResponse(
ctx.WSConn.TryWriteRPCResponse(
rpctypes.NewRPCSuccessResponse(
wsCtx.Codec(),
rpctypes.JSONRPCStringID(fmt.Sprintf("%v#event", wsCtx.Request.ID)),
ctx.WSConn.Codec(),
rpctypes.JSONRPCStringID(fmt.Sprintf("%v#event", ctx.JSONReq.ID)),
resultEvent,
))
case <-sub.Cancelled():
wsCtx.TryWriteRPCResponse(
rpctypes.RPCServerError(rpctypes.JSONRPCStringID(
fmt.Sprintf("%v#event", wsCtx.Request.ID)),
fmt.Errorf("subscription was cancelled (reason: %v)", sub.Err()),
))
if sub.Err() != tmpubsub.ErrUnsubscribed {
var reason string
if sub.Err() == nil {
reason = "Tendermint exited"
} else {
reason = sub.Err().Error()
}
ctx.WSConn.TryWriteRPCResponse(
rpctypes.RPCServerError(rpctypes.JSONRPCStringID(
fmt.Sprintf("%v#event", ctx.JSONReq.ID)),
fmt.Errorf("subscription was cancelled (reason: %s)", reason),
))
}
return
}
}
@ -161,14 +176,14 @@ func Subscribe(wsCtx rpctypes.WSRPCContext, query string) (*ctypes.ResultSubscri
// | query | string | "" | true | Query |
//
// <aside class="notice">WebSocket only</aside>
func Unsubscribe(wsCtx rpctypes.WSRPCContext, query string) (*ctypes.ResultUnsubscribe, error) {
addr := wsCtx.GetRemoteAddr()
func Unsubscribe(ctx *rpctypes.Context, query string) (*ctypes.ResultUnsubscribe, error) {
addr := ctx.RemoteAddr()
logger.Info("Unsubscribe from query", "remote", addr, "query", query)
q, err := tmquery.New(query)
if err != nil {
return nil, errors.Wrap(err, "failed to parse query")
}
err = eventBusFor(wsCtx).Unsubscribe(context.Background(), addr, q)
err = eventBus.Unsubscribe(context.Background(), addr, q)
if err != nil {
return nil, err
}
@ -199,20 +214,12 @@ func Unsubscribe(wsCtx rpctypes.WSRPCContext, query string) (*ctypes.ResultUnsub
// ```
//
// <aside class="notice">WebSocket only</aside>
func UnsubscribeAll(wsCtx rpctypes.WSRPCContext) (*ctypes.ResultUnsubscribe, error) {
addr := wsCtx.GetRemoteAddr()
func UnsubscribeAll(ctx *rpctypes.Context) (*ctypes.ResultUnsubscribe, error) {
addr := ctx.RemoteAddr()
logger.Info("Unsubscribe from all", "remote", addr)
err := eventBusFor(wsCtx).UnsubscribeAll(context.Background(), addr)
err := eventBus.UnsubscribeAll(context.Background(), addr)
if err != nil {
return nil, err
}
return &ctypes.ResultUnsubscribe{}, nil
}
func eventBusFor(wsCtx rpctypes.WSRPCContext) tmtypes.EventBusSubscriber {
es := wsCtx.GetEventSubscriber()
if es == nil {
es = eventBus
}
return es
}

View File

@ -2,6 +2,7 @@ package core
import (
ctypes "github.com/tendermint/tendermint/rpc/core/types"
rpctypes "github.com/tendermint/tendermint/rpc/lib/types"
)
// Get node health. Returns empty result (200 OK) on success, no response - in
@ -31,6 +32,6 @@ import (
// "jsonrpc": "2.0"
// }
// ```
func Health() (*ctypes.ResultHealth, error) {
func Health(ctx *rpctypes.Context) (*ctypes.ResultHealth, error) {
return &ctypes.ResultHealth{}, nil
}

View File

@ -9,7 +9,7 @@ import (
abci "github.com/tendermint/tendermint/abci/types"
ctypes "github.com/tendermint/tendermint/rpc/core/types"
rpcserver "github.com/tendermint/tendermint/rpc/lib/server"
rpctypes "github.com/tendermint/tendermint/rpc/lib/types"
"github.com/tendermint/tendermint/types"
)
@ -59,7 +59,7 @@ import (
// | Parameter | Type | Default | Required | Description |
// |-----------+------+---------+----------+-----------------|
// | tx | Tx | nil | true | The transaction |
func BroadcastTxAsync(tx types.Tx) (*ctypes.ResultBroadcastTx, error) {
func BroadcastTxAsync(ctx *rpctypes.Context, tx types.Tx) (*ctypes.ResultBroadcastTx, error) {
err := mempool.CheckTx(tx, nil)
if err != nil {
return nil, err
@ -108,7 +108,7 @@ func BroadcastTxAsync(tx types.Tx) (*ctypes.ResultBroadcastTx, error) {
// | Parameter | Type | Default | Required | Description |
// |-----------+------+---------+----------+-----------------|
// | tx | Tx | nil | true | The transaction |
func BroadcastTxSync(tx types.Tx) (*ctypes.ResultBroadcastTx, error) {
func BroadcastTxSync(ctx *rpctypes.Context, tx types.Tx) (*ctypes.ResultBroadcastTx, error) {
resCh := make(chan *abci.Response, 1)
err := mempool.CheckTx(tx, func(res *abci.Response) {
resCh <- res
@ -128,6 +128,11 @@ func BroadcastTxSync(tx types.Tx) (*ctypes.ResultBroadcastTx, error) {
// Returns with the responses from CheckTx and DeliverTx.
//
// IMPORTANT: use only for testing and development. In production, use
// BroadcastTxSync or BroadcastTxAsync. You can subscribe for the transaction
// result using JSONRPC via a websocket. See
// https://tendermint.com/docs/app-dev/subscribing-to-events-via-websocket.html
//
// CONTRACT: only returns error if mempool.CheckTx() errs or if we timeout
// waiting for tx to commit.
//
@ -182,18 +187,26 @@ func BroadcastTxSync(tx types.Tx) (*ctypes.ResultBroadcastTx, error) {
// | Parameter | Type | Default | Required | Description |
// |-----------+------+---------+----------+-----------------|
// | tx | Tx | nil | true | The transaction |
func BroadcastTxCommit(tx types.Tx) (*ctypes.ResultBroadcastTxCommit, error) {
func BroadcastTxCommit(ctx *rpctypes.Context, tx types.Tx) (*ctypes.ResultBroadcastTxCommit, error) {
subscriber := ctx.RemoteAddr()
if eventBus.NumClients() >= config.MaxSubscriptionClients {
return nil, fmt.Errorf("max_subscription_clients %d reached", config.MaxSubscriptionClients)
} else if eventBus.NumClientSubscriptions(subscriber) >= config.MaxSubscriptionsPerClient {
return nil, fmt.Errorf("max_subscriptions_per_client %d reached", config.MaxSubscriptionsPerClient)
}
// Subscribe to tx being committed in block.
ctx, cancel := context.WithTimeout(context.Background(), subscribeTimeout)
subCtx, cancel := context.WithTimeout(context.Background(), subscribeTimeout)
defer cancel()
q := types.EventQueryTxFor(tx)
deliverTxSub, err := eventBus.Subscribe(ctx, "mempool", q)
deliverTxSub, err := eventBus.Subscribe(subCtx, subscriber, q)
if err != nil {
err = errors.Wrap(err, "failed to subscribe to tx")
logger.Error("Error on broadcast_tx_commit", "err", err)
return nil, err
}
defer eventBus.Unsubscribe(context.Background(), "mempool", q)
defer eventBus.Unsubscribe(context.Background(), subscriber, q)
// Broadcast tx and wait for CheckTx result
checkTxResCh := make(chan *abci.Response, 1)
@ -215,8 +228,6 @@ func BroadcastTxCommit(tx types.Tx) (*ctypes.ResultBroadcastTxCommit, error) {
}
// Wait for the tx to be included in a block or timeout.
// TODO: configurable?
var deliverTxTimeout = rpcserver.WriteTimeout / 2
select {
case msg := <-deliverTxSub.Out(): // The tx was included in a block.
deliverTxRes := msg.Data().(types.EventDataTx)
@ -227,14 +238,20 @@ func BroadcastTxCommit(tx types.Tx) (*ctypes.ResultBroadcastTxCommit, error) {
Height: deliverTxRes.Height,
}, nil
case <-deliverTxSub.Cancelled():
err = errors.New("deliverTxSub was cancelled. Did the Tendermint stop?")
var reason string
if deliverTxSub.Err() == nil {
reason = "Tendermint exited"
} else {
reason = deliverTxSub.Err().Error()
}
err = fmt.Errorf("deliverTxSub was cancelled (reason: %s)", reason)
logger.Error("Error on broadcastTxCommit", "err", err)
return &ctypes.ResultBroadcastTxCommit{
CheckTx: *checkTxRes,
DeliverTx: abci.ResponseDeliverTx{},
Hash: tx.Hash(),
}, err
case <-time.After(deliverTxTimeout):
case <-time.After(config.TimeoutBroadcastTxCommit):
err = errors.New("Timed out waiting for tx to be included in a block")
logger.Error("Error on broadcastTxCommit", "err", err)
return &ctypes.ResultBroadcastTxCommit{
@ -281,7 +298,8 @@ func BroadcastTxCommit(tx types.Tx) (*ctypes.ResultBroadcastTxCommit, error) {
// | Parameter | Type | Default | Required | Description |
// |-----------+------+---------+----------+--------------------------------------|
// | limit | int | 30 | false | Maximum number of entries (max: 100) |
func UnconfirmedTxs(limit int) (*ctypes.ResultUnconfirmedTxs, error) {
// ```
func UnconfirmedTxs(ctx *rpctypes.Context, limit int) (*ctypes.ResultUnconfirmedTxs, error) {
// reuse per_page validator
limit = validatePerPage(limit)
@ -323,7 +341,7 @@ func UnconfirmedTxs(limit int) (*ctypes.ResultUnconfirmedTxs, error) {
// }
// }
// ```
func NumUnconfirmedTxs() (*ctypes.ResultUnconfirmedTxs, error) {
func NumUnconfirmedTxs(ctx *rpctypes.Context) (*ctypes.ResultUnconfirmedTxs, error) {
return &ctypes.ResultUnconfirmedTxs{
Count: mempool.Size(),
Total: mempool.Size(),

View File

@ -7,6 +7,7 @@ import (
"github.com/tendermint/tendermint/p2p"
ctypes "github.com/tendermint/tendermint/rpc/core/types"
rpctypes "github.com/tendermint/tendermint/rpc/lib/types"
)
// Get network info.
@ -153,7 +154,7 @@ import (
// ...
// }
// ```
func NetInfo() (*ctypes.ResultNetInfo, error) {
func NetInfo(ctx *rpctypes.Context) (*ctypes.ResultNetInfo, error) {
out, in, _ := p2pPeers.NumPeers()
peers := make([]ctypes.Peer, 0, out+in)
for _, peer := range p2pPeers.Peers().List() {
@ -179,7 +180,7 @@ func NetInfo() (*ctypes.ResultNetInfo, error) {
}, nil
}
func UnsafeDialSeeds(seeds []string) (*ctypes.ResultDialSeeds, error) {
func UnsafeDialSeeds(ctx *rpctypes.Context, seeds []string) (*ctypes.ResultDialSeeds, error) {
if len(seeds) == 0 {
return &ctypes.ResultDialSeeds{}, errors.New("No seeds provided")
}
@ -192,7 +193,7 @@ func UnsafeDialSeeds(seeds []string) (*ctypes.ResultDialSeeds, error) {
return &ctypes.ResultDialSeeds{Log: "Dialing seeds in progress. See /net_info for details"}, nil
}
func UnsafeDialPeers(peers []string, persistent bool) (*ctypes.ResultDialPeers, error) {
func UnsafeDialPeers(ctx *rpctypes.Context, peers []string, persistent bool) (*ctypes.ResultDialPeers, error) {
if len(peers) == 0 {
return &ctypes.ResultDialPeers{}, errors.New("No peers provided")
}
@ -247,6 +248,6 @@ func UnsafeDialPeers(peers []string, persistent bool) (*ctypes.ResultDialPeers,
// "jsonrpc": "2.0"
// }
// ```
func Genesis() (*ctypes.ResultGenesis, error) {
func Genesis(ctx *rpctypes.Context) (*ctypes.ResultGenesis, error) {
return &ctypes.ResultGenesis{Genesis: genDoc}, nil
}

View File

@ -1,6 +1,7 @@
package core
import (
cfg "github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/consensus"
"github.com/tendermint/tendermint/crypto"
dbm "github.com/tendermint/tendermint/libs/db"
@ -71,6 +72,8 @@ var (
mempool *mempl.Mempool
logger log.Logger
config cfg.RPCConfig
)
func SetStateDB(db dbm.DB) {
@ -133,6 +136,11 @@ func SetEventBus(b *types.EventBus) {
eventBus = b
}
// SetConfig sets an RPCConfig.
func SetConfig(c cfg.RPCConfig) {
config = c
}
func validatePage(page, perPage, totalCount int) int {
if perPage < 1 {
return 1

View File

@ -7,6 +7,7 @@ import (
cmn "github.com/tendermint/tendermint/libs/common"
"github.com/tendermint/tendermint/p2p"
ctypes "github.com/tendermint/tendermint/rpc/core/types"
rpctypes "github.com/tendermint/tendermint/rpc/lib/types"
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
)
@ -70,7 +71,7 @@ import (
// }
// }
// ```
func Status() (*ctypes.ResultStatus, error) {
func Status(ctx *rpctypes.Context) (*ctypes.ResultStatus, error) {
var latestHeight int64
if consensusReactor.FastSync() {
latestHeight = blockStore.Height()

View File

@ -7,6 +7,7 @@ import (
tmquery "github.com/tendermint/tendermint/libs/pubsub/query"
ctypes "github.com/tendermint/tendermint/rpc/core/types"
rpctypes "github.com/tendermint/tendermint/rpc/lib/types"
"github.com/tendermint/tendermint/state/txindex/null"
"github.com/tendermint/tendermint/types"
)
@ -77,7 +78,7 @@ import (
// - `index`: `int` - index of the transaction
// - `height`: `int` - height of the block where this transaction was in
// - `hash`: `[]byte` - hash of the transaction
func Tx(hash []byte, prove bool) (*ctypes.ResultTx, error) {
func Tx(ctx *rpctypes.Context, hash []byte, prove bool) (*ctypes.ResultTx, error) {
// if index is disabled, return error
if _, ok := txIndexer.(*null.TxIndex); ok {
@ -183,7 +184,7 @@ func Tx(hash []byte, prove bool) (*ctypes.ResultTx, error) {
// - `index`: `int` - index of the transaction
// - `height`: `int` - height of the block where this transaction was in
// - `hash`: `[]byte` - hash of the transaction
func TxSearch(query string, prove bool, page, perPage int) (*ctypes.ResultTxSearch, error) {
func TxSearch(ctx *rpctypes.Context, query string, prove bool, page, perPage int) (*ctypes.ResultTxSearch, error) {
// if index is disabled, return error
if _, ok := txIndexer.(*null.TxIndex); ok {
return nil, fmt.Errorf("Transaction indexing is disabled")