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.

419 lines
14 KiB

8 years ago
9 years ago
9 years ago
9 years ago
9 years ago
8 years ago
7 years ago
9 years ago
8 years ago
abci: localClient improvements & bugfixes & pubsub Unsubscribe issues (#2748) * use READ lock/unlock in ConsensusState#GetLastHeight Refs #2721 * do not use defers when there's no need * fix peer formatting (output its address instead of the pointer) ``` [54310]: E[11-02|11:59:39.851] Connection failed @ sendRoutine module=p2p peer=0xb78f00 conn=MConn{74.207.236.148:26656} err="pong timeout" ``` https://github.com/tendermint/tendermint/issues/2721#issuecomment-435326581 * panic if peer has no state https://github.com/tendermint/tendermint/issues/2721#issuecomment-435347165 It's confusing that sometimes we check if peer has a state, but most of the times we expect it to be there 1. https://github.com/tendermint/tendermint/blob/add79700b5fe84417538202b6c927c8cc5383672/mempool/reactor.go#L138 2. https://github.com/tendermint/tendermint/blob/add79700b5fe84417538202b6c927c8cc5383672/rpc/core/consensus.go#L196 (edited) I will change everything to always assume peer has a state and panic otherwise that should help identify issues earlier * abci/localclient: extend lock on app callback App callback should be protected by lock as well (note this was already done for InitChainAsync, why not for others???). Otherwise, when we execute the block, tx might come in and call the callback in the same time we're updating it in execBlockOnProxyApp => DATA RACE Fixes #2721 Consensus state is locked ``` goroutine 113333 [semacquire, 309 minutes]: sync.runtime_SemacquireMutex(0xc00180009c, 0xc0000c7e00) /usr/local/go/src/runtime/sema.go:71 +0x3d sync.(*RWMutex).RLock(0xc001800090) /usr/local/go/src/sync/rwmutex.go:50 +0x4e github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).GetRoundState(0xc001800000, 0x0) /root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:218 +0x46 github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusReactor).queryMaj23Routine(0xc0017def80, 0x11104a0, 0xc0072488f0, 0xc007248 9c0) /root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/reactor.go:735 +0x16d created by github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusReactor).AddPeer /root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/reactor.go:172 +0x236 ``` because localClient is locked ``` goroutine 1899 [semacquire, 309 minutes]: sync.runtime_SemacquireMutex(0xc00003363c, 0xc0000cb500) /usr/local/go/src/runtime/sema.go:71 +0x3d sync.(*Mutex).Lock(0xc000033638) /usr/local/go/src/sync/mutex.go:134 +0xff github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client.(*localClient).SetResponseCallback(0xc0001fb560, 0xc007868540) /root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client/local_client.go:32 +0x33 github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy.(*appConnConsensus).SetResponseCallback(0xc00002f750, 0xc007868540) /root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy/app_conn.go:57 +0x40 github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state.execBlockOnProxyApp(0x1104e20, 0xc002ca0ba0, 0x11092a0, 0xc00002f750, 0xc0001fe960, 0xc000bfc660, 0x110cfe0, 0xc000090330, 0xc9d12, 0xc000d9d5a0, ...) /root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state/execution.go:230 +0x1fd github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state.(*BlockExecutor).ApplyBlock(0xc002c2a230, 0x7, 0x0, 0xc000eae880, 0x6, 0xc002e52c60, 0x16, 0x1f927, 0xc9d12, 0xc000d9d5a0, ...) /root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state/execution.go:96 +0x142 github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).finalizeCommit(0xc001800000, 0x1f928) /root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1339 +0xa3e github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).tryFinalizeCommit(0xc001800000, 0x1f928) /root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1270 +0x451 github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).enterCommit.func1(0xc001800000, 0x0, 0x1f928) /root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1218 +0x90 github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).enterCommit(0xc001800000, 0x1f928, 0x0) /root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1247 +0x6b8 github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).addVote(0xc001800000, 0xc003d8dea0, 0xc000cf4cc0, 0x28, 0xf1, 0xc003bc7ad0, 0xc003bc7b10) /root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1659 +0xbad github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).tryAddVote(0xc001800000, 0xc003d8dea0, 0xc000cf4cc0, 0x28, 0xf1, 0xf1, 0xf1) /root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1517 +0x59 github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).handleMsg(0xc001800000, 0xd98200, 0xc0070dbed0, 0xc000cf4cc0, 0x28) /root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:660 +0x64b github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).receiveRoutine(0xc001800000, 0x0) /root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:617 +0x670 created by github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).OnStart /root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:311 +0x132 ``` tx comes in and CheckTx is executed right when we execute the block ``` goroutine 111044 [semacquire, 309 minutes]: sync.runtime_SemacquireMutex(0xc00003363c, 0x0) /usr/local/go/src/runtime/sema.go:71 +0x3d sync.(*Mutex).Lock(0xc000033638) /usr/local/go/src/sync/mutex.go:134 +0xff github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client.(*localClient).CheckTxAsync(0xc0001fb0e0, 0xc002d94500, 0x13f, 0x280, 0x0) /root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client/local_client.go:85 +0x47 github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy.(*appConnMempool).CheckTxAsync(0xc00002f720, 0xc002d94500, 0x13f, 0x280, 0x1) /root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy/app_conn.go:114 +0x51 github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/mempool.(*Mempool).CheckTx(0xc002d3a320, 0xc002d94500, 0x13f, 0x280, 0xc0072355f0, 0x0, 0x0) /root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/mempool/mempool.go:316 +0x17b github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/core.BroadcastTxSync(0xc002d94500, 0x13f, 0x280, 0x0, 0x0, 0x0) /root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/core/mempool.go:93 +0xb8 reflect.Value.call(0xd85560, 0x10326c0, 0x13, 0xec7b8b, 0x4, 0xc00663f180, 0x1, 0x1, 0xc00663f180, 0xc00663f188, ...) /usr/local/go/src/reflect/value.go:447 +0x449 reflect.Value.Call(0xd85560, 0x10326c0, 0x13, 0xc00663f180, 0x1, 0x1, 0x0, 0x0, 0xc005cc9344) /usr/local/go/src/reflect/value.go:308 +0xa4 github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server.makeHTTPHandler.func2(0x1102060, 0xc00663f100, 0xc0082d7900) /root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server/handlers.go:269 +0x188 net/http.HandlerFunc.ServeHTTP(0xc002c81f20, 0x1102060, 0xc00663f100, 0xc0082d7900) /usr/local/go/src/net/http/server.go:1964 +0x44 net/http.(*ServeMux).ServeHTTP(0xc002c81b60, 0x1102060, 0xc00663f100, 0xc0082d7900) /usr/local/go/src/net/http/server.go:2361 +0x127 github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server.maxBytesHandler.ServeHTTP(0x10f8a40, 0xc002c81b60, 0xf4240, 0x1102060, 0xc00663f100, 0xc0082d7900) /root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server/http_server.go:219 +0xcf github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server.RecoverAndLogHandler.func1(0x1103220, 0xc00121e620, 0xc0082d7900) /root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server/http_server.go:192 +0x394 net/http.HandlerFunc.ServeHTTP(0xc002c06ea0, 0x1103220, 0xc00121e620, 0xc0082d7900) /usr/local/go/src/net/http/server.go:1964 +0x44 net/http.serverHandler.ServeHTTP(0xc001a1aa90, 0x1103220, 0xc00121e620, 0xc0082d7900) /usr/local/go/src/net/http/server.go:2741 +0xab net/http.(*conn).serve(0xc00785a3c0, 0x11041a0, 0xc000f844c0) /usr/local/go/src/net/http/server.go:1847 +0x646 created by net/http.(*Server).Serve /usr/local/go/src/net/http/server.go:2851 +0x2f5 ``` * consensus: use read lock in Receive#VoteMessage * use defer to unlock mutex because application might panic * use defer in every method of the localClient * add a changelog entry * drain channels before Unsubscribe(All) Read https://github.com/tendermint/tendermint/blob/55362ed76630f3e1ebec159a598f6a9fb5892cb1/libs/pubsub/pubsub.go#L13 for the detailed explanation of the issue. We'll need to fix it someday. Make sure to keep an eye on https://github.com/tendermint/tendermint/blob/master/docs/architecture/adr-033-pubsub.md * retry instead of panic when peer has no state in reactors other than consensus in /dump_consensus_state RPC endpoint, skip a peer with no state * rpc/core/mempool: simplify error messages * rpc/core/mempool: use time.After instead of timer also, do not log DeliverTx result (to be consistent with other memthods) * unlock before calling the callback in reqRes#SetCallback
6 years ago
9 years ago
9 years ago
  1. package abcicli
  2. import (
  3. "fmt"
  4. "net"
  5. "sync"
  6. "time"
  7. "golang.org/x/net/context"
  8. "google.golang.org/grpc"
  9. "github.com/tendermint/tendermint/abci/types"
  10. tmnet "github.com/tendermint/tendermint/libs/net"
  11. "github.com/tendermint/tendermint/libs/service"
  12. tmsync "github.com/tendermint/tendermint/libs/sync"
  13. )
  14. var _ Client = (*grpcClient)(nil)
  15. // A stripped copy of the remoteClient that makes
  16. // synchronous calls using grpc
  17. type grpcClient struct {
  18. service.BaseService
  19. mustConnect bool
  20. client types.ABCIApplicationClient
  21. conn *grpc.ClientConn
  22. chReqRes chan *ReqRes // dispatches "async" responses to callbacks *in order*, needed by mempool
  23. mtx tmsync.Mutex
  24. addr string
  25. err error
  26. resCb func(*types.Request, *types.Response) // listens to all callbacks
  27. }
  28. func NewGRPCClient(addr string, mustConnect bool) Client {
  29. cli := &grpcClient{
  30. addr: addr,
  31. mustConnect: mustConnect,
  32. // Buffering the channel is needed to make calls appear asynchronous,
  33. // which is required when the caller makes multiple async calls before
  34. // processing callbacks (e.g. due to holding locks). 64 means that a
  35. // caller can make up to 64 async calls before a callback must be
  36. // processed (otherwise it deadlocks). It also means that we can make 64
  37. // gRPC calls while processing a slow callback at the channel head.
  38. chReqRes: make(chan *ReqRes, 64),
  39. }
  40. cli.BaseService = *service.NewBaseService(nil, "grpcClient", cli)
  41. return cli
  42. }
  43. func dialerFunc(ctx context.Context, addr string) (net.Conn, error) {
  44. return tmnet.Connect(addr)
  45. }
  46. func (cli *grpcClient) OnStart() error {
  47. if err := cli.BaseService.OnStart(); err != nil {
  48. return err
  49. }
  50. // This processes asynchronous request/response messages and dispatches
  51. // them to callbacks.
  52. go func() {
  53. // Use a separate function to use defer for mutex unlocks (this handles panics)
  54. callCb := func(reqres *ReqRes) {
  55. cli.mtx.Lock()
  56. defer cli.mtx.Unlock()
  57. reqres.SetDone()
  58. reqres.Done()
  59. // Notify client listener if set
  60. if cli.resCb != nil {
  61. cli.resCb(reqres.Request, reqres.Response)
  62. }
  63. // Notify reqRes listener if set
  64. if cb := reqres.GetCallback(); cb != nil {
  65. cb(reqres.Response)
  66. }
  67. }
  68. for reqres := range cli.chReqRes {
  69. if reqres != nil {
  70. callCb(reqres)
  71. } else {
  72. cli.Logger.Error("Received nil reqres")
  73. }
  74. }
  75. }()
  76. RETRY_LOOP:
  77. for {
  78. conn, err := grpc.Dial(cli.addr, grpc.WithInsecure(), grpc.WithContextDialer(dialerFunc))
  79. if err != nil {
  80. if cli.mustConnect {
  81. return err
  82. }
  83. cli.Logger.Error(fmt.Sprintf("abci.grpcClient failed to connect to %v. Retrying...\n", cli.addr), "err", err)
  84. time.Sleep(time.Second * dialRetryIntervalSeconds)
  85. continue RETRY_LOOP
  86. }
  87. cli.Logger.Info("Dialed server. Waiting for echo.", "addr", cli.addr)
  88. client := types.NewABCIApplicationClient(conn)
  89. cli.conn = conn
  90. ENSURE_CONNECTED:
  91. for {
  92. _, err := client.Echo(context.Background(), &types.RequestEcho{Message: "hello"}, grpc.WaitForReady(true))
  93. if err == nil {
  94. break ENSURE_CONNECTED
  95. }
  96. cli.Logger.Error("Echo failed", "err", err)
  97. time.Sleep(time.Second * echoRetryIntervalSeconds)
  98. }
  99. cli.client = client
  100. return nil
  101. }
  102. }
  103. func (cli *grpcClient) OnStop() {
  104. cli.BaseService.OnStop()
  105. if cli.conn != nil {
  106. cli.conn.Close()
  107. }
  108. close(cli.chReqRes)
  109. }
  110. func (cli *grpcClient) StopForError(err error) {
  111. if !cli.IsRunning() {
  112. return
  113. }
  114. cli.mtx.Lock()
  115. if cli.err == nil {
  116. cli.err = err
  117. }
  118. cli.mtx.Unlock()
  119. cli.Logger.Error(fmt.Sprintf("Stopping abci.grpcClient for error: %v", err.Error()))
  120. if err := cli.Stop(); err != nil {
  121. cli.Logger.Error("Error stopping abci.grpcClient", "err", err)
  122. }
  123. }
  124. func (cli *grpcClient) Error() error {
  125. cli.mtx.Lock()
  126. defer cli.mtx.Unlock()
  127. return cli.err
  128. }
  129. // Set listener for all responses
  130. // NOTE: callback may get internally generated flush responses.
  131. func (cli *grpcClient) SetResponseCallback(resCb Callback) {
  132. cli.mtx.Lock()
  133. cli.resCb = resCb
  134. cli.mtx.Unlock()
  135. }
  136. //----------------------------------------
  137. // GRPC calls are synchronous, but some callbacks expect to be called asynchronously
  138. // (eg. the mempool expects to be able to lock to remove bad txs from cache).
  139. // To accommodate, we finish each call in its own go-routine,
  140. // which is expensive, but easy - if you want something better, use the socket protocol!
  141. // maybe one day, if people really want it, we use grpc streams,
  142. // but hopefully not :D
  143. func (cli *grpcClient) EchoAsync(msg string) *ReqRes {
  144. req := types.ToRequestEcho(msg)
  145. res, err := cli.client.Echo(context.Background(), req.GetEcho(), grpc.WaitForReady(true))
  146. if err != nil {
  147. cli.StopForError(err)
  148. }
  149. return cli.finishAsyncCall(req, &types.Response{Value: &types.Response_Echo{Echo: res}})
  150. }
  151. func (cli *grpcClient) FlushAsync() *ReqRes {
  152. req := types.ToRequestFlush()
  153. res, err := cli.client.Flush(context.Background(), req.GetFlush(), grpc.WaitForReady(true))
  154. if err != nil {
  155. cli.StopForError(err)
  156. }
  157. return cli.finishAsyncCall(req, &types.Response{Value: &types.Response_Flush{Flush: res}})
  158. }
  159. func (cli *grpcClient) InfoAsync(params types.RequestInfo) *ReqRes {
  160. req := types.ToRequestInfo(params)
  161. res, err := cli.client.Info(context.Background(), req.GetInfo(), grpc.WaitForReady(true))
  162. if err != nil {
  163. cli.StopForError(err)
  164. }
  165. return cli.finishAsyncCall(req, &types.Response{Value: &types.Response_Info{Info: res}})
  166. }
  167. func (cli *grpcClient) SetOptionAsync(params types.RequestSetOption) *ReqRes {
  168. req := types.ToRequestSetOption(params)
  169. res, err := cli.client.SetOption(context.Background(), req.GetSetOption(), grpc.WaitForReady(true))
  170. if err != nil {
  171. cli.StopForError(err)
  172. }
  173. return cli.finishAsyncCall(req, &types.Response{Value: &types.Response_SetOption{SetOption: res}})
  174. }
  175. func (cli *grpcClient) DeliverTxAsync(params types.RequestDeliverTx) *ReqRes {
  176. req := types.ToRequestDeliverTx(params)
  177. res, err := cli.client.DeliverTx(context.Background(), req.GetDeliverTx(), grpc.WaitForReady(true))
  178. if err != nil {
  179. cli.StopForError(err)
  180. }
  181. return cli.finishAsyncCall(req, &types.Response{Value: &types.Response_DeliverTx{DeliverTx: res}})
  182. }
  183. func (cli *grpcClient) CheckTxAsync(params types.RequestCheckTx) *ReqRes {
  184. req := types.ToRequestCheckTx(params)
  185. res, err := cli.client.CheckTx(context.Background(), req.GetCheckTx(), grpc.WaitForReady(true))
  186. if err != nil {
  187. cli.StopForError(err)
  188. }
  189. return cli.finishAsyncCall(req, &types.Response{Value: &types.Response_CheckTx{CheckTx: res}})
  190. }
  191. func (cli *grpcClient) QueryAsync(params types.RequestQuery) *ReqRes {
  192. req := types.ToRequestQuery(params)
  193. res, err := cli.client.Query(context.Background(), req.GetQuery(), grpc.WaitForReady(true))
  194. if err != nil {
  195. cli.StopForError(err)
  196. }
  197. return cli.finishAsyncCall(req, &types.Response{Value: &types.Response_Query{Query: res}})
  198. }
  199. func (cli *grpcClient) CommitAsync() *ReqRes {
  200. req := types.ToRequestCommit()
  201. res, err := cli.client.Commit(context.Background(), req.GetCommit(), grpc.WaitForReady(true))
  202. if err != nil {
  203. cli.StopForError(err)
  204. }
  205. return cli.finishAsyncCall(req, &types.Response{Value: &types.Response_Commit{Commit: res}})
  206. }
  207. func (cli *grpcClient) InitChainAsync(params types.RequestInitChain) *ReqRes {
  208. req := types.ToRequestInitChain(params)
  209. res, err := cli.client.InitChain(context.Background(), req.GetInitChain(), grpc.WaitForReady(true))
  210. if err != nil {
  211. cli.StopForError(err)
  212. }
  213. return cli.finishAsyncCall(req, &types.Response{Value: &types.Response_InitChain{InitChain: res}})
  214. }
  215. func (cli *grpcClient) BeginBlockAsync(params types.RequestBeginBlock) *ReqRes {
  216. req := types.ToRequestBeginBlock(params)
  217. res, err := cli.client.BeginBlock(context.Background(), req.GetBeginBlock(), grpc.WaitForReady(true))
  218. if err != nil {
  219. cli.StopForError(err)
  220. }
  221. return cli.finishAsyncCall(req, &types.Response{Value: &types.Response_BeginBlock{BeginBlock: res}})
  222. }
  223. func (cli *grpcClient) EndBlockAsync(params types.RequestEndBlock) *ReqRes {
  224. req := types.ToRequestEndBlock(params)
  225. res, err := cli.client.EndBlock(context.Background(), req.GetEndBlock(), grpc.WaitForReady(true))
  226. if err != nil {
  227. cli.StopForError(err)
  228. }
  229. return cli.finishAsyncCall(req, &types.Response{Value: &types.Response_EndBlock{EndBlock: res}})
  230. }
  231. func (cli *grpcClient) ListSnapshotsAsync(params types.RequestListSnapshots) *ReqRes {
  232. req := types.ToRequestListSnapshots(params)
  233. res, err := cli.client.ListSnapshots(context.Background(), req.GetListSnapshots(), grpc.WaitForReady(true))
  234. if err != nil {
  235. cli.StopForError(err)
  236. }
  237. return cli.finishAsyncCall(req, &types.Response{Value: &types.Response_ListSnapshots{ListSnapshots: res}})
  238. }
  239. func (cli *grpcClient) OfferSnapshotAsync(params types.RequestOfferSnapshot) *ReqRes {
  240. req := types.ToRequestOfferSnapshot(params)
  241. res, err := cli.client.OfferSnapshot(context.Background(), req.GetOfferSnapshot(), grpc.WaitForReady(true))
  242. if err != nil {
  243. cli.StopForError(err)
  244. }
  245. return cli.finishAsyncCall(req, &types.Response{Value: &types.Response_OfferSnapshot{OfferSnapshot: res}})
  246. }
  247. func (cli *grpcClient) LoadSnapshotChunkAsync(params types.RequestLoadSnapshotChunk) *ReqRes {
  248. req := types.ToRequestLoadSnapshotChunk(params)
  249. res, err := cli.client.LoadSnapshotChunk(context.Background(), req.GetLoadSnapshotChunk(), grpc.WaitForReady(true))
  250. if err != nil {
  251. cli.StopForError(err)
  252. }
  253. return cli.finishAsyncCall(req, &types.Response{Value: &types.Response_LoadSnapshotChunk{LoadSnapshotChunk: res}})
  254. }
  255. func (cli *grpcClient) ApplySnapshotChunkAsync(params types.RequestApplySnapshotChunk) *ReqRes {
  256. req := types.ToRequestApplySnapshotChunk(params)
  257. res, err := cli.client.ApplySnapshotChunk(context.Background(), req.GetApplySnapshotChunk(), grpc.WaitForReady(true))
  258. if err != nil {
  259. cli.StopForError(err)
  260. }
  261. return cli.finishAsyncCall(req, &types.Response{Value: &types.Response_ApplySnapshotChunk{ApplySnapshotChunk: res}})
  262. }
  263. // finishAsyncCall creates a ReqRes for an async call, and immediately populates it
  264. // with the response. We don't complete it until it's been ordered via the channel.
  265. func (cli *grpcClient) finishAsyncCall(req *types.Request, res *types.Response) *ReqRes {
  266. reqres := NewReqRes(req)
  267. reqres.Response = res
  268. cli.chReqRes <- reqres // use channel for async responses, since they must be ordered
  269. return reqres
  270. }
  271. // finishSyncCall waits for an async call to complete. It is necessary to call all
  272. // sync calls asynchronously as well, to maintain call and response ordering via
  273. // the channel, and this method will wait until the async call completes.
  274. func (cli *grpcClient) finishSyncCall(reqres *ReqRes) *types.Response {
  275. // It's possible that the callback is called twice, since the callback can
  276. // be called immediately on SetCallback() in addition to after it has been
  277. // set. This is because completing the ReqRes happens in a separate critical
  278. // section from the one where the callback is called: there is a race where
  279. // SetCallback() is called between completing the ReqRes and dispatching the
  280. // callback.
  281. //
  282. // We also buffer the channel with 1 response, since SetCallback() will be
  283. // called synchronously if the reqres is already completed, in which case
  284. // it will block on sending to the channel since it hasn't gotten around to
  285. // receiving from it yet.
  286. //
  287. // ReqRes should really handle callback dispatch internally, to guarantee
  288. // that it's only called once and avoid the above race conditions.
  289. var once sync.Once
  290. ch := make(chan *types.Response, 1)
  291. reqres.SetCallback(func(res *types.Response) {
  292. once.Do(func() {
  293. ch <- res
  294. })
  295. })
  296. return <-ch
  297. }
  298. //----------------------------------------
  299. func (cli *grpcClient) FlushSync() error {
  300. return nil
  301. }
  302. func (cli *grpcClient) EchoSync(msg string) (*types.ResponseEcho, error) {
  303. reqres := cli.EchoAsync(msg)
  304. // StopForError should already have been called if error is set
  305. return cli.finishSyncCall(reqres).GetEcho(), cli.Error()
  306. }
  307. func (cli *grpcClient) InfoSync(req types.RequestInfo) (*types.ResponseInfo, error) {
  308. reqres := cli.InfoAsync(req)
  309. return cli.finishSyncCall(reqres).GetInfo(), cli.Error()
  310. }
  311. func (cli *grpcClient) SetOptionSync(req types.RequestSetOption) (*types.ResponseSetOption, error) {
  312. reqres := cli.SetOptionAsync(req)
  313. return reqres.Response.GetSetOption(), cli.Error()
  314. }
  315. func (cli *grpcClient) DeliverTxSync(params types.RequestDeliverTx) (*types.ResponseDeliverTx, error) {
  316. reqres := cli.DeliverTxAsync(params)
  317. return cli.finishSyncCall(reqres).GetDeliverTx(), cli.Error()
  318. }
  319. func (cli *grpcClient) CheckTxSync(params types.RequestCheckTx) (*types.ResponseCheckTx, error) {
  320. reqres := cli.CheckTxAsync(params)
  321. return cli.finishSyncCall(reqres).GetCheckTx(), cli.Error()
  322. }
  323. func (cli *grpcClient) QuerySync(req types.RequestQuery) (*types.ResponseQuery, error) {
  324. reqres := cli.QueryAsync(req)
  325. return cli.finishSyncCall(reqres).GetQuery(), cli.Error()
  326. }
  327. func (cli *grpcClient) CommitSync() (*types.ResponseCommit, error) {
  328. reqres := cli.CommitAsync()
  329. return cli.finishSyncCall(reqres).GetCommit(), cli.Error()
  330. }
  331. func (cli *grpcClient) InitChainSync(params types.RequestInitChain) (*types.ResponseInitChain, error) {
  332. reqres := cli.InitChainAsync(params)
  333. return cli.finishSyncCall(reqres).GetInitChain(), cli.Error()
  334. }
  335. func (cli *grpcClient) BeginBlockSync(params types.RequestBeginBlock) (*types.ResponseBeginBlock, error) {
  336. reqres := cli.BeginBlockAsync(params)
  337. return cli.finishSyncCall(reqres).GetBeginBlock(), cli.Error()
  338. }
  339. func (cli *grpcClient) EndBlockSync(params types.RequestEndBlock) (*types.ResponseEndBlock, error) {
  340. reqres := cli.EndBlockAsync(params)
  341. return cli.finishSyncCall(reqres).GetEndBlock(), cli.Error()
  342. }
  343. func (cli *grpcClient) ListSnapshotsSync(params types.RequestListSnapshots) (*types.ResponseListSnapshots, error) {
  344. reqres := cli.ListSnapshotsAsync(params)
  345. return cli.finishSyncCall(reqres).GetListSnapshots(), cli.Error()
  346. }
  347. func (cli *grpcClient) OfferSnapshotSync(params types.RequestOfferSnapshot) (*types.ResponseOfferSnapshot, error) {
  348. reqres := cli.OfferSnapshotAsync(params)
  349. return cli.finishSyncCall(reqres).GetOfferSnapshot(), cli.Error()
  350. }
  351. func (cli *grpcClient) LoadSnapshotChunkSync(
  352. params types.RequestLoadSnapshotChunk) (*types.ResponseLoadSnapshotChunk, error) {
  353. reqres := cli.LoadSnapshotChunkAsync(params)
  354. return cli.finishSyncCall(reqres).GetLoadSnapshotChunk(), cli.Error()
  355. }
  356. func (cli *grpcClient) ApplySnapshotChunkSync(
  357. params types.RequestApplySnapshotChunk) (*types.ResponseApplySnapshotChunk, error) {
  358. reqres := cli.ApplySnapshotChunkAsync(params)
  359. return cli.finishSyncCall(reqres).GetApplySnapshotChunk(), cli.Error()
  360. }