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.

290 lines
7.0 KiB

7 years ago
7 years ago
7 years ago
7 years ago
abci: Flush socket requests and responses immediately. (#6997) The main effect of this change is to flush the socket client and server message encoding buffers immediately once the message is fully and correctly encoded. This allows us to remove the timer and some other special cases, without changing the observed behaviour of the system. -- Background The socket protocol client and server each use a buffered writer to encode request and response messages onto the underlying connection. This reduces the possibility of a single message being split across multiple writes, but has the side-effect that a request may remain buffered for some time. The implementation worked around this by keeping a ticker that occasionally triggers a flush, and by flushing the writer in response to an explicit request baked into the client/server protocol (see also #6994). These workarounds are both unnecessary: Once a message has been dequeued for sending and fully encoded in wire format, there is no real use keeping all or part of it buffered locally. Moreover, using an asynchronous process to flush the buffer makes the round-trip performance of the request unpredictable. -- Benchmarks Code: https://play.golang.org/p/0ChUOxJOiHt I found no pre-existing performance benchmarks to justify the flush pattern, but a natural question is whether this will significantly harm client/server performance. To test this, I implemented a simple benchmark that transfers randomly-sized byte buffers from a no-op "client" to a no-op "server" over a Unix-domain socket, using a buffered writer, both with and without explicit flushes after each write. As the following data show, flushing every time (FLUSH=true) does reduce raw throughput, but not by a significant amount except for very small request sizes, where the transfer time is already trivial (1.9μs). Given that the client is calibrated for 1MiB transactions, the overhead is not meaningful. The percentage in each section is the speedup for flushing only when the buffer is full, relative to flushing every block. The benchmark uses the default buffer size (4096 bytes), which is the same value used by the socket client and server implementation: FLUSH NBLOCKS MAX AVG TOTAL ELAPSED TIME/BLOCK false 3957471 512 255 1011165416 2.00018873s 505ns true 1068568 512 255 273064368 2.000217051s 1.871µs (73%) false 536096 4096 2048 1098066401 2.000229108s 3.731µs true 477911 4096 2047 978746731 2.000177825s 4.185µs (10.8%) false 124595 16384 8181 1019340160 2.000235086s 16.053µs true 120995 16384 8179 989703064 2.000329349s 16.532µs (2.9%) false 2114 1048576 525693 1111316541 2.000479928s 946.3µs true 2083 1048576 526379 1096449173 2.001817137s 961.025µs (1.5%) Note also that the FLUSH=false baseline is actually faster than the production code, which flushes more often than is required by the buffer filling up. Moreover, the timer slows down the overall transaction rate of the client and server, indepenedent of how fast the socket transfer is, so the loss on a real workload is probably much less.
3 years ago
abci: Flush socket requests and responses immediately. (#6997) The main effect of this change is to flush the socket client and server message encoding buffers immediately once the message is fully and correctly encoded. This allows us to remove the timer and some other special cases, without changing the observed behaviour of the system. -- Background The socket protocol client and server each use a buffered writer to encode request and response messages onto the underlying connection. This reduces the possibility of a single message being split across multiple writes, but has the side-effect that a request may remain buffered for some time. The implementation worked around this by keeping a ticker that occasionally triggers a flush, and by flushing the writer in response to an explicit request baked into the client/server protocol (see also #6994). These workarounds are both unnecessary: Once a message has been dequeued for sending and fully encoded in wire format, there is no real use keeping all or part of it buffered locally. Moreover, using an asynchronous process to flush the buffer makes the round-trip performance of the request unpredictable. -- Benchmarks Code: https://play.golang.org/p/0ChUOxJOiHt I found no pre-existing performance benchmarks to justify the flush pattern, but a natural question is whether this will significantly harm client/server performance. To test this, I implemented a simple benchmark that transfers randomly-sized byte buffers from a no-op "client" to a no-op "server" over a Unix-domain socket, using a buffered writer, both with and without explicit flushes after each write. As the following data show, flushing every time (FLUSH=true) does reduce raw throughput, but not by a significant amount except for very small request sizes, where the transfer time is already trivial (1.9μs). Given that the client is calibrated for 1MiB transactions, the overhead is not meaningful. The percentage in each section is the speedup for flushing only when the buffer is full, relative to flushing every block. The benchmark uses the default buffer size (4096 bytes), which is the same value used by the socket client and server implementation: FLUSH NBLOCKS MAX AVG TOTAL ELAPSED TIME/BLOCK false 3957471 512 255 1011165416 2.00018873s 505ns true 1068568 512 255 273064368 2.000217051s 1.871µs (73%) false 536096 4096 2048 1098066401 2.000229108s 3.731µs true 477911 4096 2047 978746731 2.000177825s 4.185µs (10.8%) false 124595 16384 8181 1019340160 2.000235086s 16.053µs true 120995 16384 8179 989703064 2.000329349s 16.532µs (2.9%) false 2114 1048576 525693 1111316541 2.000479928s 946.3µs true 2083 1048576 526379 1096449173 2.001817137s 961.025µs (1.5%) Note also that the FLUSH=false baseline is actually faster than the production code, which flushes more often than is required by the buffer filling up. Moreover, the timer slows down the overall transaction rate of the client and server, indepenedent of how fast the socket transfer is, so the loss on a real workload is probably much less.
3 years ago
  1. package server
  2. import (
  3. "bufio"
  4. "context"
  5. "fmt"
  6. "io"
  7. "net"
  8. "runtime"
  9. "sync"
  10. "github.com/tendermint/tendermint/abci/types"
  11. "github.com/tendermint/tendermint/libs/log"
  12. tmnet "github.com/tendermint/tendermint/libs/net"
  13. "github.com/tendermint/tendermint/libs/service"
  14. )
  15. // var maxNumberConnections = 2
  16. type SocketServer struct {
  17. service.BaseService
  18. logger log.Logger
  19. proto string
  20. addr string
  21. listener net.Listener
  22. connsMtx sync.Mutex
  23. conns map[int]net.Conn
  24. nextConnID int
  25. appMtx sync.Mutex
  26. app types.Application
  27. }
  28. func NewSocketServer(logger log.Logger, protoAddr string, app types.Application) service.Service {
  29. proto, addr := tmnet.ProtocolAndAddress(protoAddr)
  30. s := &SocketServer{
  31. logger: logger,
  32. proto: proto,
  33. addr: addr,
  34. listener: nil,
  35. app: app,
  36. conns: make(map[int]net.Conn),
  37. }
  38. s.BaseService = *service.NewBaseService(logger, "ABCIServer", s)
  39. return s
  40. }
  41. func (s *SocketServer) OnStart(ctx context.Context) error {
  42. ln, err := net.Listen(s.proto, s.addr)
  43. if err != nil {
  44. return err
  45. }
  46. s.listener = ln
  47. go s.acceptConnectionsRoutine(ctx)
  48. return nil
  49. }
  50. func (s *SocketServer) OnStop() {
  51. if err := s.listener.Close(); err != nil {
  52. s.logger.Error("error closing listener", "err", err)
  53. }
  54. s.connsMtx.Lock()
  55. defer s.connsMtx.Unlock()
  56. for id, conn := range s.conns {
  57. delete(s.conns, id)
  58. if err := conn.Close(); err != nil {
  59. s.logger.Error("error closing connection", "id", id, "conn", conn, "err", err)
  60. }
  61. }
  62. }
  63. func (s *SocketServer) addConn(conn net.Conn) int {
  64. s.connsMtx.Lock()
  65. defer s.connsMtx.Unlock()
  66. connID := s.nextConnID
  67. s.nextConnID++
  68. s.conns[connID] = conn
  69. return connID
  70. }
  71. // deletes conn even if close errs
  72. func (s *SocketServer) rmConn(connID int) error {
  73. s.connsMtx.Lock()
  74. defer s.connsMtx.Unlock()
  75. conn, ok := s.conns[connID]
  76. if !ok {
  77. return fmt.Errorf("connection %d does not exist", connID)
  78. }
  79. delete(s.conns, connID)
  80. return conn.Close()
  81. }
  82. func (s *SocketServer) acceptConnectionsRoutine(ctx context.Context) {
  83. for {
  84. if ctx.Err() != nil {
  85. return
  86. }
  87. // Accept a connection
  88. s.logger.Info("Waiting for new connection...")
  89. conn, err := s.listener.Accept()
  90. if err != nil {
  91. if !s.IsRunning() {
  92. return // Ignore error from listener closing.
  93. }
  94. s.logger.Error("Failed to accept connection", "err", err)
  95. continue
  96. }
  97. s.logger.Info("Accepted a new connection")
  98. connID := s.addConn(conn)
  99. closeConn := make(chan error, 2) // Push to signal connection closed
  100. responses := make(chan *types.Response, 1000) // A channel to buffer responses
  101. // Read requests from conn and deal with them
  102. go s.handleRequests(ctx, closeConn, conn, responses)
  103. // Pull responses from 'responses' and write them to conn.
  104. go s.handleResponses(ctx, closeConn, conn, responses)
  105. // Wait until signal to close connection
  106. go s.waitForClose(ctx, closeConn, connID)
  107. }
  108. }
  109. func (s *SocketServer) waitForClose(ctx context.Context, closeConn chan error, connID int) {
  110. defer func() {
  111. // Close the connection
  112. if err := s.rmConn(connID); err != nil {
  113. s.logger.Error("error closing connection", "err", err)
  114. }
  115. }()
  116. select {
  117. case <-ctx.Done():
  118. return
  119. case err := <-closeConn:
  120. switch {
  121. case err == io.EOF:
  122. s.logger.Error("Connection was closed by client")
  123. case err != nil:
  124. s.logger.Error("Connection error", "err", err)
  125. default:
  126. // never happens
  127. s.logger.Error("Connection was closed")
  128. }
  129. }
  130. }
  131. // Read requests from conn and deal with them
  132. func (s *SocketServer) handleRequests(
  133. ctx context.Context,
  134. closeConn chan error,
  135. conn io.Reader,
  136. responses chan<- *types.Response,
  137. ) {
  138. var count int
  139. var bufReader = bufio.NewReader(conn)
  140. defer func() {
  141. // make sure to recover from any app-related panics to allow proper socket cleanup
  142. r := recover()
  143. if r != nil {
  144. const size = 64 << 10
  145. buf := make([]byte, size)
  146. buf = buf[:runtime.Stack(buf, false)]
  147. err := fmt.Errorf("recovered from panic: %v\n%s", r, buf)
  148. closeConn <- err
  149. s.appMtx.Unlock()
  150. }
  151. }()
  152. for {
  153. if ctx.Err() != nil {
  154. return
  155. }
  156. var req = &types.Request{}
  157. err := types.ReadMessage(bufReader, req)
  158. if err != nil {
  159. if err == io.EOF {
  160. closeConn <- err
  161. } else {
  162. closeConn <- fmt.Errorf("error reading message: %w", err)
  163. }
  164. return
  165. }
  166. s.appMtx.Lock()
  167. count++
  168. s.handleRequest(req, responses)
  169. s.appMtx.Unlock()
  170. }
  171. }
  172. func (s *SocketServer) handleRequest(req *types.Request, responses chan<- *types.Response) {
  173. switch r := req.Value.(type) {
  174. case *types.Request_Echo:
  175. responses <- types.ToResponseEcho(r.Echo.Message)
  176. case *types.Request_Flush:
  177. responses <- types.ToResponseFlush()
  178. case *types.Request_Info:
  179. res := s.app.Info(*r.Info)
  180. responses <- types.ToResponseInfo(res)
  181. case *types.Request_CheckTx:
  182. res := s.app.CheckTx(*r.CheckTx)
  183. responses <- types.ToResponseCheckTx(res)
  184. case *types.Request_Commit:
  185. res := s.app.Commit()
  186. responses <- types.ToResponseCommit(res)
  187. case *types.Request_Query:
  188. res := s.app.Query(*r.Query)
  189. responses <- types.ToResponseQuery(res)
  190. case *types.Request_InitChain:
  191. res := s.app.InitChain(*r.InitChain)
  192. responses <- types.ToResponseInitChain(res)
  193. case *types.Request_ListSnapshots:
  194. res := s.app.ListSnapshots(*r.ListSnapshots)
  195. responses <- types.ToResponseListSnapshots(res)
  196. case *types.Request_OfferSnapshot:
  197. res := s.app.OfferSnapshot(*r.OfferSnapshot)
  198. responses <- types.ToResponseOfferSnapshot(res)
  199. case *types.Request_PrepareProposal:
  200. res := s.app.PrepareProposal(*r.PrepareProposal)
  201. responses <- types.ToResponsePrepareProposal(res)
  202. case *types.Request_ProcessProposal:
  203. res := s.app.ProcessProposal(*r.ProcessProposal)
  204. responses <- types.ToResponseProcessProposal(res)
  205. case *types.Request_LoadSnapshotChunk:
  206. res := s.app.LoadSnapshotChunk(*r.LoadSnapshotChunk)
  207. responses <- types.ToResponseLoadSnapshotChunk(res)
  208. case *types.Request_ApplySnapshotChunk:
  209. res := s.app.ApplySnapshotChunk(*r.ApplySnapshotChunk)
  210. responses <- types.ToResponseApplySnapshotChunk(res)
  211. case *types.Request_ExtendVote:
  212. res := s.app.ExtendVote(*r.ExtendVote)
  213. responses <- types.ToResponseExtendVote(res)
  214. case *types.Request_VerifyVoteExtension:
  215. res := s.app.VerifyVoteExtension(*r.VerifyVoteExtension)
  216. responses <- types.ToResponseVerifyVoteExtension(res)
  217. case *types.Request_FinalizeBlock:
  218. res := s.app.FinalizeBlock(*r.FinalizeBlock)
  219. responses <- types.ToResponseFinalizeBlock(res)
  220. default:
  221. responses <- types.ToResponseException("Unknown request")
  222. }
  223. }
  224. // Pull responses from 'responses' and write them to conn.
  225. func (s *SocketServer) handleResponses(
  226. ctx context.Context,
  227. closeConn chan error,
  228. conn io.Writer,
  229. responses <-chan *types.Response,
  230. ) {
  231. bw := bufio.NewWriter(conn)
  232. for {
  233. select {
  234. case <-ctx.Done():
  235. return
  236. case res := <-responses:
  237. if err := types.WriteMessage(res, bw); err != nil {
  238. select {
  239. case <-ctx.Done():
  240. case closeConn <- fmt.Errorf("error writing message: %w", err):
  241. }
  242. return
  243. }
  244. if err := bw.Flush(); err != nil {
  245. select {
  246. case <-ctx.Done():
  247. case closeConn <- fmt.Errorf("error flushing write buffer: %w", err):
  248. }
  249. return
  250. }
  251. }
  252. }
  253. }