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.

351 lines
11 KiB

7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
lint: Enable Golint (#4212) * Fix many golint errors * Fix golint errors in the 'lite' package * Don't export Pool.store * Fix typo * Revert unwanted changes * Fix errors in counter package * Fix linter errors in kvstore package * Fix linter error in example package * Fix error in tests package * Fix linter errors in v2 package * Fix linter errors in consensus package * Fix linter errors in evidence package * Fix linter error in fail package * Fix linter errors in query package * Fix linter errors in core package * Fix linter errors in node package * Fix linter errors in mempool package * Fix linter error in conn package * Fix linter errors in pex package * Rename PEXReactor export to Reactor * Fix linter errors in trust package * Fix linter errors in upnp package * Fix linter errors in p2p package * Fix linter errors in proxy package * Fix linter errors in mock_test package * Fix linter error in client_test package * Fix linter errors in coretypes package * Fix linter errors in coregrpc package * Fix linter errors in rpcserver package * Fix linter errors in rpctypes package * Fix linter errors in rpctest package * Fix linter error in json2wal script * Fix linter error in wal2json script * Fix linter errors in kv package * Fix linter error in state package * Fix linter error in grpc_client * Fix linter errors in types package * Fix linter error in version package * Fix remaining errors * Address review comments * Fix broken tests * Reconcile package coregrpc * Fix golangci bot error * Fix new golint errors * Fix broken reference * Enable golint linter * minor changes to bring golint into line * fix failing test * fix pex reactor naming * address PR comments
5 years ago
7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
  1. package evidence
  2. import (
  3. "context"
  4. "fmt"
  5. "runtime/debug"
  6. "sync"
  7. "time"
  8. clist "github.com/tendermint/tendermint/internal/libs/clist"
  9. tmsync "github.com/tendermint/tendermint/internal/libs/sync"
  10. "github.com/tendermint/tendermint/internal/p2p"
  11. "github.com/tendermint/tendermint/libs/log"
  12. "github.com/tendermint/tendermint/libs/service"
  13. tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
  14. "github.com/tendermint/tendermint/types"
  15. )
  16. var _ service.Service = (*Reactor)(nil)
  17. const (
  18. EvidenceChannel = p2p.ChannelID(0x38)
  19. maxMsgSize = 1048576 // 1MB TODO make it configurable
  20. // broadcast all uncommitted evidence this often. This sets when the reactor
  21. // goes back to the start of the list and begins sending the evidence again.
  22. // Most evidence should be committed in the very next block that is why we wait
  23. // just over the block production rate before sending evidence again.
  24. broadcastEvidenceIntervalS = 10
  25. )
  26. // GetChannelDescriptor produces an instance of a descriptor for this
  27. // package's required channels.
  28. func GetChannelDescriptor() *p2p.ChannelDescriptor {
  29. return &p2p.ChannelDescriptor{
  30. ID: EvidenceChannel,
  31. MessageType: new(tmproto.EvidenceList),
  32. Priority: 6,
  33. RecvMessageCapacity: maxMsgSize,
  34. RecvBufferCapacity: 32,
  35. }
  36. }
  37. // Reactor handles evpool evidence broadcasting amongst peers.
  38. type Reactor struct {
  39. service.BaseService
  40. logger log.Logger
  41. evpool *Pool
  42. evidenceCh *p2p.Channel
  43. peerUpdates *p2p.PeerUpdates
  44. peerWG sync.WaitGroup
  45. mtx tmsync.Mutex
  46. peerRoutines map[types.NodeID]*tmsync.Closer
  47. }
  48. // NewReactor returns a reference to a new evidence reactor, which implements the
  49. // service.Service interface. It accepts a p2p Channel dedicated for handling
  50. // envelopes with EvidenceList messages.
  51. func NewReactor(
  52. logger log.Logger,
  53. evidenceCh *p2p.Channel,
  54. peerUpdates *p2p.PeerUpdates,
  55. evpool *Pool,
  56. ) *Reactor {
  57. r := &Reactor{
  58. logger: logger,
  59. evpool: evpool,
  60. evidenceCh: evidenceCh,
  61. peerUpdates: peerUpdates,
  62. peerRoutines: make(map[types.NodeID]*tmsync.Closer),
  63. }
  64. r.BaseService = *service.NewBaseService(logger, "Evidence", r)
  65. return r
  66. }
  67. // OnStart starts separate go routines for each p2p Channel and listens for
  68. // envelopes on each. In addition, it also listens for peer updates and handles
  69. // messages on that p2p channel accordingly. The caller must be sure to execute
  70. // OnStop to ensure the outbound p2p Channels are closed. No error is returned.
  71. func (r *Reactor) OnStart(ctx context.Context) error {
  72. go r.processEvidenceCh(ctx)
  73. go r.processPeerUpdates(ctx)
  74. return nil
  75. }
  76. // OnStop stops the reactor by signaling to all spawned goroutines to exit and
  77. // blocking until they all exit.
  78. func (r *Reactor) OnStop() {
  79. r.mtx.Lock()
  80. for _, c := range r.peerRoutines {
  81. c.Close()
  82. }
  83. r.mtx.Unlock()
  84. // Wait for all spawned peer evidence broadcasting goroutines to gracefully
  85. // exit.
  86. r.peerWG.Wait()
  87. // Close the evidence db
  88. r.evpool.Close()
  89. }
  90. // handleEvidenceMessage handles envelopes sent from peers on the EvidenceChannel.
  91. // It returns an error only if the Envelope.Message is unknown for this channel
  92. // or if the given evidence is invalid. This should never be called outside of
  93. // handleMessage.
  94. func (r *Reactor) handleEvidenceMessage(envelope p2p.Envelope) error {
  95. logger := r.logger.With("peer", envelope.From)
  96. switch msg := envelope.Message.(type) {
  97. case *tmproto.EvidenceList:
  98. // TODO: Refactor the Evidence type to not contain a list since we only ever
  99. // send and receive one piece of evidence at a time. Or potentially consider
  100. // batching evidence.
  101. //
  102. // see: https://github.com/tendermint/tendermint/issues/4729
  103. for i := 0; i < len(msg.Evidence); i++ {
  104. ev, err := types.EvidenceFromProto(&msg.Evidence[i])
  105. if err != nil {
  106. logger.Error("failed to convert evidence", "err", err)
  107. continue
  108. }
  109. if err := r.evpool.AddEvidence(ev); err != nil {
  110. // If we're given invalid evidence by the peer, notify the router that
  111. // we should remove this peer by returning an error.
  112. if _, ok := err.(*types.ErrInvalidEvidence); ok {
  113. return err
  114. }
  115. }
  116. }
  117. default:
  118. return fmt.Errorf("received unknown message: %T", msg)
  119. }
  120. return nil
  121. }
  122. // handleMessage handles an Envelope sent from a peer on a specific p2p Channel.
  123. // It will handle errors and any possible panics gracefully. A caller can handle
  124. // any error returned by sending a PeerError on the respective channel.
  125. func (r *Reactor) handleMessage(chID p2p.ChannelID, envelope p2p.Envelope) (err error) {
  126. defer func() {
  127. if e := recover(); e != nil {
  128. err = fmt.Errorf("panic in processing message: %v", e)
  129. r.logger.Error(
  130. "recovering from processing message panic",
  131. "err", err,
  132. "stack", string(debug.Stack()),
  133. )
  134. }
  135. }()
  136. r.logger.Debug("received message", "message", envelope.Message, "peer", envelope.From)
  137. switch chID {
  138. case EvidenceChannel:
  139. err = r.handleEvidenceMessage(envelope)
  140. default:
  141. err = fmt.Errorf("unknown channel ID (%d) for envelope (%v)", chID, envelope)
  142. }
  143. return err
  144. }
  145. // processEvidenceCh implements a blocking event loop where we listen for p2p
  146. // Envelope messages from the evidenceCh.
  147. func (r *Reactor) processEvidenceCh(ctx context.Context) {
  148. for {
  149. select {
  150. case <-ctx.Done():
  151. r.logger.Debug("stopped listening on evidence channel; closing...")
  152. return
  153. case envelope := <-r.evidenceCh.In:
  154. if err := r.handleMessage(r.evidenceCh.ID, envelope); err != nil {
  155. r.logger.Error("failed to process message", "ch_id", r.evidenceCh.ID, "envelope", envelope, "err", err)
  156. if serr := r.evidenceCh.SendError(ctx, p2p.PeerError{
  157. NodeID: envelope.From,
  158. Err: err,
  159. }); serr != nil {
  160. return
  161. }
  162. }
  163. }
  164. }
  165. }
  166. // processPeerUpdate processes a PeerUpdate. For new or live peers it will check
  167. // if an evidence broadcasting goroutine needs to be started. For down or
  168. // removed peers, it will check if an evidence broadcasting goroutine
  169. // exists and signal that it should exit.
  170. //
  171. // FIXME: The peer may be behind in which case it would simply ignore the
  172. // evidence and treat it as invalid. This would cause the peer to disconnect.
  173. // The peer may also receive the same piece of evidence multiple times if it
  174. // connects/disconnects frequently from the broadcasting peer(s).
  175. //
  176. // REF: https://github.com/tendermint/tendermint/issues/4727
  177. func (r *Reactor) processPeerUpdate(ctx context.Context, peerUpdate p2p.PeerUpdate) {
  178. r.logger.Debug("received peer update", "peer", peerUpdate.NodeID, "status", peerUpdate.Status)
  179. r.mtx.Lock()
  180. defer r.mtx.Unlock()
  181. switch peerUpdate.Status {
  182. case p2p.PeerStatusUp:
  183. // Do not allow starting new evidence broadcast loops after reactor shutdown
  184. // has been initiated. This can happen after we've manually closed all
  185. // peer broadcast loops, but the router still sends in-flight peer updates.
  186. if !r.IsRunning() {
  187. return
  188. }
  189. // Check if we've already started a goroutine for this peer, if not we create
  190. // a new done channel so we can explicitly close the goroutine if the peer
  191. // is later removed, we increment the waitgroup so the reactor can stop
  192. // safely, and finally start the goroutine to broadcast evidence to that peer.
  193. _, ok := r.peerRoutines[peerUpdate.NodeID]
  194. if !ok {
  195. closer := tmsync.NewCloser()
  196. r.peerRoutines[peerUpdate.NodeID] = closer
  197. r.peerWG.Add(1)
  198. go r.broadcastEvidenceLoop(ctx, peerUpdate.NodeID, closer)
  199. }
  200. case p2p.PeerStatusDown:
  201. // Check if we've started an evidence broadcasting goroutine for this peer.
  202. // If we have, we signal to terminate the goroutine via the channel's closure.
  203. // This will internally decrement the peer waitgroup and remove the peer
  204. // from the map of peer evidence broadcasting goroutines.
  205. closer, ok := r.peerRoutines[peerUpdate.NodeID]
  206. if ok {
  207. closer.Close()
  208. }
  209. }
  210. }
  211. // processPeerUpdates initiates a blocking process where we listen for and handle
  212. // PeerUpdate messages. When the reactor is stopped, we will catch the signal and
  213. // close the p2p PeerUpdatesCh gracefully.
  214. func (r *Reactor) processPeerUpdates(ctx context.Context) {
  215. for {
  216. select {
  217. case peerUpdate := <-r.peerUpdates.Updates():
  218. r.processPeerUpdate(ctx, peerUpdate)
  219. case <-ctx.Done():
  220. r.logger.Debug("stopped listening on peer updates channel; closing...")
  221. return
  222. }
  223. }
  224. }
  225. // broadcastEvidenceLoop starts a blocking process that continuously reads pieces
  226. // of evidence off of a linked-list and sends the evidence in a p2p Envelope to
  227. // the given peer by ID. This should be invoked in a goroutine per unique peer
  228. // ID via an appropriate PeerUpdate. The goroutine can be signaled to gracefully
  229. // exit by either explicitly closing the provided doneCh or by the reactor
  230. // signaling to stop.
  231. //
  232. // TODO: This should be refactored so that we do not blindly gossip evidence
  233. // that the peer has already received or may not be ready for.
  234. //
  235. // REF: https://github.com/tendermint/tendermint/issues/4727
  236. func (r *Reactor) broadcastEvidenceLoop(ctx context.Context, peerID types.NodeID, closer *tmsync.Closer) {
  237. var next *clist.CElement
  238. defer func() {
  239. r.mtx.Lock()
  240. delete(r.peerRoutines, peerID)
  241. r.mtx.Unlock()
  242. r.peerWG.Done()
  243. if e := recover(); e != nil {
  244. r.logger.Error(
  245. "recovering from broadcasting evidence loop",
  246. "err", e,
  247. "stack", string(debug.Stack()),
  248. )
  249. }
  250. }()
  251. for {
  252. // This happens because the CElement we were looking at got garbage
  253. // collected (removed). That is, .NextWaitChan() returned nil. So we can go
  254. // ahead and start from the beginning.
  255. if next == nil {
  256. select {
  257. case <-r.evpool.EvidenceWaitChan(): // wait until next evidence is available
  258. if next = r.evpool.EvidenceFront(); next == nil {
  259. continue
  260. }
  261. case <-ctx.Done():
  262. return
  263. case <-closer.Done():
  264. // The peer is marked for removal via a PeerUpdate as the doneCh was
  265. // explicitly closed to signal we should exit.
  266. return
  267. }
  268. }
  269. ev := next.Value.(types.Evidence)
  270. evProto, err := types.EvidenceToProto(ev)
  271. if err != nil {
  272. panic(fmt.Errorf("failed to convert evidence: %w", err))
  273. }
  274. // Send the evidence to the corresponding peer. Note, the peer may be behind
  275. // and thus would not be able to process the evidence correctly. Also, the
  276. // peer may receive this piece of evidence multiple times if it added and
  277. // removed frequently from the broadcasting peer.
  278. select {
  279. case <-ctx.Done():
  280. return
  281. case r.evidenceCh.Out <- p2p.Envelope{
  282. To: peerID,
  283. Message: &tmproto.EvidenceList{
  284. Evidence: []tmproto.Evidence{*evProto},
  285. },
  286. }:
  287. }
  288. r.logger.Debug("gossiped evidence to peer", "evidence", ev, "peer", peerID)
  289. select {
  290. case <-time.After(time.Second * broadcastEvidenceIntervalS):
  291. // start from the beginning after broadcastEvidenceIntervalS seconds
  292. next = nil
  293. case <-next.NextWaitChan():
  294. next = next.Next()
  295. case <-closer.Done():
  296. // The peer is marked for removal via a PeerUpdate as the doneCh was
  297. // explicitly closed to signal we should exit.
  298. return
  299. case <-ctx.Done():
  300. return
  301. }
  302. }
  303. }