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.

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