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.

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