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.

376 lines
12 KiB

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