You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.

154 lines
5.8 KiB

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>
5 years ago
new pubsub package comment out failing consensus tests for now rewrite rpc httpclient to use new pubsub package import pubsub as tmpubsub, query as tmquery make event IDs constants EventKey -> EventTypeKey rename EventsPubsub to PubSub mempool does not use pubsub rename eventsSub to pubsub new subscribe API fix channel size issues and consensus tests bugs refactor rpc client add missing discardFromChan method add mutex rename pubsub to eventBus remove IsRunning from WSRPCConnection interface (not needed) add a comment in broadcastNewRoundStepsAndVotes rename registerEventCallbacks to broadcastNewRoundStepsAndVotes See https://dave.cheney.net/2014/03/19/channel-axioms stop eventBuses after reactor tests remove unnecessary Unsubscribe return subscribe helper function move discardFromChan to where it is used subscribe now returns an err this gives us ability to refuse to subscribe if pubsub is at its max capacity. use context for control overflow cache queries handle err when subscribing in replay_test rename testClientID to testSubscriber extract var set channel buffer capacity to 1 in replay_file fix byzantine_test unsubscribe from single event, not all events refactor httpclient to return events to appropriate channels return failing testReplayCrashBeforeWriteVote test fix TestValidatorSetChanges refactor code a bit fix testReplayCrashBeforeWriteVote add comment fix TestValidatorSetChanges fixes from Bucky's review update comment [ci skip] test TxEventBuffer update changelog fix TestValidatorSetChanges (2nd attempt) only do wg.Done when no errors benchmark event bus create pubsub server inside NewEventBus only expose config params (later if needed) set buffer capacity to 0 so we are not testing cache new tx event format: key = "Tx" plus a tag {"tx.hash": XYZ} This should allow to subscribe to all transactions! or a specific one using a query: "tm.events.type = Tx and tx.hash = '013ABF99434...'" use TimeoutCommit instead of afterPublishEventNewBlockTimeout TimeoutCommit is the time a node waits after committing a block, before it goes into the next height. So it will finish everything from the last block, but then wait a bit. The idea is this gives it time to hear more votes from other validators, to strengthen the commit it includes in the next block. But it also gives it time to hear about new transactions. waitForBlockWithUpdatedVals rewrite WAL crash tests Task: test that we can recover from any WAL crash. Solution: the old tests were relying on event hub being run in the same thread (we were injecting the private validator's last signature). when considering a rewrite, we considered two possible solutions: write a "fuzzy" testing system where WAL is crashing upon receiving a new message, or inject failures and trigger them in tests using something like https://github.com/coreos/gofail. remove sleep no cs.Lock around wal.Save test different cases (empty block, non-empty block, ...) comments add comments test 4 cases: empty block, non-empty block, non-empty block with smaller part size, many blocks fixes as per Bucky's last review reset subscriptions on UnsubscribeAll use a simple counter to track message for which we panicked also, set a smaller part size for all test cases
7 years ago
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>
5 years ago
new pubsub package comment out failing consensus tests for now rewrite rpc httpclient to use new pubsub package import pubsub as tmpubsub, query as tmquery make event IDs constants EventKey -> EventTypeKey rename EventsPubsub to PubSub mempool does not use pubsub rename eventsSub to pubsub new subscribe API fix channel size issues and consensus tests bugs refactor rpc client add missing discardFromChan method add mutex rename pubsub to eventBus remove IsRunning from WSRPCConnection interface (not needed) add a comment in broadcastNewRoundStepsAndVotes rename registerEventCallbacks to broadcastNewRoundStepsAndVotes See https://dave.cheney.net/2014/03/19/channel-axioms stop eventBuses after reactor tests remove unnecessary Unsubscribe return subscribe helper function move discardFromChan to where it is used subscribe now returns an err this gives us ability to refuse to subscribe if pubsub is at its max capacity. use context for control overflow cache queries handle err when subscribing in replay_test rename testClientID to testSubscriber extract var set channel buffer capacity to 1 in replay_file fix byzantine_test unsubscribe from single event, not all events refactor httpclient to return events to appropriate channels return failing testReplayCrashBeforeWriteVote test fix TestValidatorSetChanges refactor code a bit fix testReplayCrashBeforeWriteVote add comment fix TestValidatorSetChanges fixes from Bucky's review update comment [ci skip] test TxEventBuffer update changelog fix TestValidatorSetChanges (2nd attempt) only do wg.Done when no errors benchmark event bus create pubsub server inside NewEventBus only expose config params (later if needed) set buffer capacity to 0 so we are not testing cache new tx event format: key = "Tx" plus a tag {"tx.hash": XYZ} This should allow to subscribe to all transactions! or a specific one using a query: "tm.events.type = Tx and tx.hash = '013ABF99434...'" use TimeoutCommit instead of afterPublishEventNewBlockTimeout TimeoutCommit is the time a node waits after committing a block, before it goes into the next height. So it will finish everything from the last block, but then wait a bit. The idea is this gives it time to hear more votes from other validators, to strengthen the commit it includes in the next block. But it also gives it time to hear about new transactions. waitForBlockWithUpdatedVals rewrite WAL crash tests Task: test that we can recover from any WAL crash. Solution: the old tests were relying on event hub being run in the same thread (we were injecting the private validator's last signature). when considering a rewrite, we considered two possible solutions: write a "fuzzy" testing system where WAL is crashing upon receiving a new message, or inject failures and trigger them in tests using something like https://github.com/coreos/gofail. remove sleep no cs.Lock around wal.Save test different cases (empty block, non-empty block, ...) comments add comments test 4 cases: empty block, non-empty block, non-empty block with smaller part size, many blocks fixes as per Bucky's last review reset subscriptions on UnsubscribeAll use a simple counter to track message for which we panicked also, set a smaller part size for all test cases
7 years ago
  1. package client
  2. /*
  3. The client package provides a general purpose interface (Client) for connecting
  4. to a tendermint node, as well as higher-level functionality.
  5. The main implementation for production code is client.HTTP, which
  6. connects via http to the jsonrpc interface of the tendermint node.
  7. For connecting to a node running in the same process (eg. when
  8. compiling the abci app in the same process), you can use the client.Local
  9. implementation.
  10. For mocking out server responses during testing to see behavior for
  11. arbitrary return values, use the mock package.
  12. In addition to the Client interface, which should be used externally
  13. for maximum flexibility and testability, and two implementations,
  14. this package also provides helper functions that work on any Client
  15. implementation.
  16. */
  17. import (
  18. "context"
  19. "github.com/tendermint/tendermint/libs/bytes"
  20. "github.com/tendermint/tendermint/libs/service"
  21. ctypes "github.com/tendermint/tendermint/rpc/core/types"
  22. "github.com/tendermint/tendermint/types"
  23. )
  24. //go:generate ../../scripts/mockery_generate.sh Client
  25. // Client wraps most important rpc calls a client would make if you want to
  26. // listen for events, test if it also implements events.EventSwitch.
  27. type Client interface {
  28. service.Service
  29. ABCIClient
  30. EventsClient
  31. HistoryClient
  32. NetworkClient
  33. SignClient
  34. StatusClient
  35. EvidenceClient
  36. MempoolClient
  37. }
  38. // ABCIClient groups together the functionality that principally affects the
  39. // ABCI app.
  40. //
  41. // In many cases this will be all we want, so we can accept an interface which
  42. // is easier to mock.
  43. type ABCIClient interface {
  44. // Reading from abci app
  45. ABCIInfo(context.Context) (*ctypes.ResultABCIInfo, error)
  46. ABCIQuery(ctx context.Context, path string, data bytes.HexBytes) (*ctypes.ResultABCIQuery, error)
  47. ABCIQueryWithOptions(ctx context.Context, path string, data bytes.HexBytes,
  48. opts ABCIQueryOptions) (*ctypes.ResultABCIQuery, error)
  49. // Writing to abci app
  50. BroadcastTxCommit(context.Context, types.Tx) (*ctypes.ResultBroadcastTxCommit, error)
  51. BroadcastTxAsync(context.Context, types.Tx) (*ctypes.ResultBroadcastTx, error)
  52. BroadcastTxSync(context.Context, types.Tx) (*ctypes.ResultBroadcastTx, error)
  53. }
  54. // SignClient groups together the functionality needed to get valid signatures
  55. // and prove anything about the chain.
  56. type SignClient interface {
  57. Block(ctx context.Context, height *int64) (*ctypes.ResultBlock, error)
  58. BlockByHash(ctx context.Context, hash []byte) (*ctypes.ResultBlock, error)
  59. BlockResults(ctx context.Context, height *int64) (*ctypes.ResultBlockResults, error)
  60. Commit(ctx context.Context, height *int64) (*ctypes.ResultCommit, error)
  61. Validators(ctx context.Context, height *int64, page, perPage *int) (*ctypes.ResultValidators, error)
  62. Tx(ctx context.Context, hash []byte, prove bool) (*ctypes.ResultTx, error)
  63. // TxSearch defines a method to search for a paginated set of transactions by
  64. // DeliverTx event search criteria.
  65. TxSearch(
  66. ctx context.Context,
  67. query string,
  68. prove bool,
  69. page, perPage *int,
  70. orderBy string,
  71. ) (*ctypes.ResultTxSearch, error)
  72. // BlockSearch defines a method to search for a paginated set of blocks by
  73. // BeginBlock and EndBlock event search criteria.
  74. BlockSearch(
  75. ctx context.Context,
  76. query string,
  77. page, perPage *int,
  78. orderBy string,
  79. ) (*ctypes.ResultBlockSearch, error)
  80. }
  81. // HistoryClient provides access to data from genesis to now in large chunks.
  82. type HistoryClient interface {
  83. Genesis(context.Context) (*ctypes.ResultGenesis, error)
  84. GenesisChunked(context.Context, uint) (*ctypes.ResultGenesisChunk, error)
  85. BlockchainInfo(ctx context.Context, minHeight, maxHeight int64) (*ctypes.ResultBlockchainInfo, error)
  86. }
  87. // StatusClient provides access to general chain info.
  88. type StatusClient interface {
  89. Status(context.Context) (*ctypes.ResultStatus, error)
  90. }
  91. // NetworkClient is general info about the network state. May not be needed
  92. // usually.
  93. type NetworkClient interface {
  94. NetInfo(context.Context) (*ctypes.ResultNetInfo, error)
  95. DumpConsensusState(context.Context) (*ctypes.ResultDumpConsensusState, error)
  96. ConsensusState(context.Context) (*ctypes.ResultConsensusState, error)
  97. ConsensusParams(ctx context.Context, height *int64) (*ctypes.ResultConsensusParams, error)
  98. Health(context.Context) (*ctypes.ResultHealth, error)
  99. }
  100. // EventsClient is reactive, you can subscribe to any message, given the proper
  101. // string. see tendermint/types/events.go
  102. type EventsClient interface {
  103. // Subscribe subscribes given subscriber to query. Returns a channel with
  104. // cap=1 onto which events are published. An error is returned if it fails to
  105. // subscribe. outCapacity can be used optionally to set capacity for the
  106. // channel. Channel is never closed to prevent accidental reads.
  107. //
  108. // ctx cannot be used to unsubscribe. To unsubscribe, use either Unsubscribe
  109. // or UnsubscribeAll.
  110. Subscribe(ctx context.Context, subscriber, query string, outCapacity ...int) (out <-chan ctypes.ResultEvent, err error)
  111. // Unsubscribe unsubscribes given subscriber from query.
  112. Unsubscribe(ctx context.Context, subscriber, query string) error
  113. // UnsubscribeAll unsubscribes given subscriber from all the queries.
  114. UnsubscribeAll(ctx context.Context, subscriber string) error
  115. }
  116. // MempoolClient shows us data about current mempool state.
  117. type MempoolClient interface {
  118. UnconfirmedTxs(ctx context.Context, limit *int) (*ctypes.ResultUnconfirmedTxs, error)
  119. NumUnconfirmedTxs(context.Context) (*ctypes.ResultUnconfirmedTxs, error)
  120. CheckTx(context.Context, types.Tx) (*ctypes.ResultCheckTx, error)
  121. }
  122. // EvidenceClient is used for submitting an evidence of the malicious
  123. // behavior.
  124. type EvidenceClient interface {
  125. BroadcastEvidence(context.Context, types.Evidence) (*ctypes.ResultBroadcastEvidence, error)
  126. }
  127. // RemoteClient is a Client, which can also return the remote network address.
  128. type RemoteClient interface {
  129. Client
  130. // Remote returns the remote network address in a string form.
  131. Remote() string
  132. }