2014-08-10 16:35:08 -07:00
|
|
|
package consensus
|
|
|
|
|
|
|
|
import (
|
|
|
|
"fmt"
|
2015-04-16 17:46:27 -07:00
|
|
|
"reflect"
|
2014-08-10 16:35:08 -07:00
|
|
|
"sync"
|
|
|
|
"time"
|
|
|
|
|
new pubsub package
comment out failing consensus tests for now
rewrite rpc httpclient to use new pubsub package
import pubsub as tmpubsub, query as tmquery
make event IDs constants
EventKey -> EventTypeKey
rename EventsPubsub to PubSub
mempool does not use pubsub
rename eventsSub to pubsub
new subscribe API
fix channel size issues and consensus tests bugs
refactor rpc client
add missing discardFromChan method
add mutex
rename pubsub to eventBus
remove IsRunning from WSRPCConnection interface (not needed)
add a comment in broadcastNewRoundStepsAndVotes
rename registerEventCallbacks to broadcastNewRoundStepsAndVotes
See https://dave.cheney.net/2014/03/19/channel-axioms
stop eventBuses after reactor tests
remove unnecessary Unsubscribe
return subscribe helper function
move discardFromChan to where it is used
subscribe now returns an err
this gives us ability to refuse to subscribe if pubsub is at its max
capacity.
use context for control overflow
cache queries
handle err when subscribing in replay_test
rename testClientID to testSubscriber
extract var
set channel buffer capacity to 1 in replay_file
fix byzantine_test
unsubscribe from single event, not all events
refactor httpclient to return events to appropriate channels
return failing testReplayCrashBeforeWriteVote test
fix TestValidatorSetChanges
refactor code a bit
fix testReplayCrashBeforeWriteVote
add comment
fix TestValidatorSetChanges
fixes from Bucky's review
update comment [ci skip]
test TxEventBuffer
update changelog
fix TestValidatorSetChanges (2nd attempt)
only do wg.Done when no errors
benchmark event bus
create pubsub server inside NewEventBus
only expose config params (later if needed)
set buffer capacity to 0 so we are not testing cache
new tx event format: key = "Tx" plus a tag {"tx.hash": XYZ}
This should allow to subscribe to all transactions! or a specific one
using a query: "tm.events.type = Tx and tx.hash = '013ABF99434...'"
use TimeoutCommit instead of afterPublishEventNewBlockTimeout
TimeoutCommit is the time a node waits after committing a block, before
it goes into the next height. So it will finish everything from the last
block, but then wait a bit. The idea is this gives it time to hear more
votes from other validators, to strengthen the commit it includes in the
next block. But it also gives it time to hear about new transactions.
waitForBlockWithUpdatedVals
rewrite WAL crash tests
Task:
test that we can recover from any WAL crash.
Solution:
the old tests were relying on event hub being run in the same thread (we
were injecting the private validator's last signature).
when considering a rewrite, we considered two possible solutions: write
a "fuzzy" testing system where WAL is crashing upon receiving a new
message, or inject failures and trigger them in tests using something
like https://github.com/coreos/gofail.
remove sleep
no cs.Lock around wal.Save
test different cases (empty block, non-empty block, ...)
comments
add comments
test 4 cases: empty block, non-empty block, non-empty block with smaller part size, many blocks
fixes as per Bucky's last review
reset subscriptions on UnsubscribeAll
use a simple counter to track message for which we panicked
also, set a smaller part size for all test cases
2017-06-26 19:00:30 +04:00
|
|
|
"github.com/pkg/errors"
|
|
|
|
|
2019-01-28 14:13:17 +02:00
|
|
|
amino "github.com/tendermint/go-amino"
|
2017-10-10 12:39:21 +04:00
|
|
|
cstypes "github.com/tendermint/tendermint/consensus/types"
|
2018-07-09 13:01:23 +04:00
|
|
|
cmn "github.com/tendermint/tendermint/libs/common"
|
2018-05-16 11:03:11 +04:00
|
|
|
tmevents "github.com/tendermint/tendermint/libs/events"
|
2018-07-09 13:01:23 +04:00
|
|
|
"github.com/tendermint/tendermint/libs/log"
|
2017-04-28 17:57:06 -04:00
|
|
|
"github.com/tendermint/tendermint/p2p"
|
2015-04-01 17:30:16 -07:00
|
|
|
sm "github.com/tendermint/tendermint/state"
|
|
|
|
"github.com/tendermint/tendermint/types"
|
2018-09-01 01:33:51 +02:00
|
|
|
tmtime "github.com/tendermint/tendermint/types/time"
|
2014-08-10 16:35:08 -07:00
|
|
|
)
|
|
|
|
|
|
|
|
const (
|
2016-09-05 17:33:02 -07:00
|
|
|
StateChannel = byte(0x20)
|
|
|
|
DataChannel = byte(0x21)
|
|
|
|
VoteChannel = byte(0x22)
|
|
|
|
VoteSetBitsChannel = byte(0x23)
|
|
|
|
|
2018-04-09 15:14:33 +03:00
|
|
|
maxMsgSize = 1048576 // 1MB; NOTE/TODO: keep in sync with types.PartSet sizes.
|
2018-03-19 09:55:08 +03:00
|
|
|
|
|
|
|
blocksToContributeToBecomeGoodPeer = 10000
|
2018-09-21 20:36:48 +02:00
|
|
|
votesToContributeToBecomeGoodPeer = 10000
|
2014-08-10 16:35:08 -07:00
|
|
|
)
|
|
|
|
|
|
|
|
//-----------------------------------------------------------------------------
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// ConsensusReactor defines a reactor for the consensus service.
|
2014-09-14 15:37:32 -07:00
|
|
|
type ConsensusReactor struct {
|
2016-10-28 12:14:24 -07:00
|
|
|
p2p.BaseReactor // BaseService + p2p.Switch
|
2014-08-10 16:35:08 -07:00
|
|
|
|
2017-08-09 14:55:21 -04:00
|
|
|
conS *ConsensusState
|
|
|
|
|
|
|
|
mtx sync.RWMutex
|
2016-09-15 16:01:01 -07:00
|
|
|
fastSync bool
|
new pubsub package
comment out failing consensus tests for now
rewrite rpc httpclient to use new pubsub package
import pubsub as tmpubsub, query as tmquery
make event IDs constants
EventKey -> EventTypeKey
rename EventsPubsub to PubSub
mempool does not use pubsub
rename eventsSub to pubsub
new subscribe API
fix channel size issues and consensus tests bugs
refactor rpc client
add missing discardFromChan method
add mutex
rename pubsub to eventBus
remove IsRunning from WSRPCConnection interface (not needed)
add a comment in broadcastNewRoundStepsAndVotes
rename registerEventCallbacks to broadcastNewRoundStepsAndVotes
See https://dave.cheney.net/2014/03/19/channel-axioms
stop eventBuses after reactor tests
remove unnecessary Unsubscribe
return subscribe helper function
move discardFromChan to where it is used
subscribe now returns an err
this gives us ability to refuse to subscribe if pubsub is at its max
capacity.
use context for control overflow
cache queries
handle err when subscribing in replay_test
rename testClientID to testSubscriber
extract var
set channel buffer capacity to 1 in replay_file
fix byzantine_test
unsubscribe from single event, not all events
refactor httpclient to return events to appropriate channels
return failing testReplayCrashBeforeWriteVote test
fix TestValidatorSetChanges
refactor code a bit
fix testReplayCrashBeforeWriteVote
add comment
fix TestValidatorSetChanges
fixes from Bucky's review
update comment [ci skip]
test TxEventBuffer
update changelog
fix TestValidatorSetChanges (2nd attempt)
only do wg.Done when no errors
benchmark event bus
create pubsub server inside NewEventBus
only expose config params (later if needed)
set buffer capacity to 0 so we are not testing cache
new tx event format: key = "Tx" plus a tag {"tx.hash": XYZ}
This should allow to subscribe to all transactions! or a specific one
using a query: "tm.events.type = Tx and tx.hash = '013ABF99434...'"
use TimeoutCommit instead of afterPublishEventNewBlockTimeout
TimeoutCommit is the time a node waits after committing a block, before
it goes into the next height. So it will finish everything from the last
block, but then wait a bit. The idea is this gives it time to hear more
votes from other validators, to strengthen the commit it includes in the
next block. But it also gives it time to hear about new transactions.
waitForBlockWithUpdatedVals
rewrite WAL crash tests
Task:
test that we can recover from any WAL crash.
Solution:
the old tests were relying on event hub being run in the same thread (we
were injecting the private validator's last signature).
when considering a rewrite, we considered two possible solutions: write
a "fuzzy" testing system where WAL is crashing upon receiving a new
message, or inject failures and trigger them in tests using something
like https://github.com/coreos/gofail.
remove sleep
no cs.Lock around wal.Save
test different cases (empty block, non-empty block, ...)
comments
add comments
test 4 cases: empty block, non-empty block, non-empty block with smaller part size, many blocks
fixes as per Bucky's last review
reset subscriptions on UnsubscribeAll
use a simple counter to track message for which we panicked
also, set a smaller part size for all test cases
2017-06-26 19:00:30 +04:00
|
|
|
eventBus *types.EventBus
|
2018-09-25 04:14:38 -07:00
|
|
|
|
|
|
|
metrics *Metrics
|
2014-08-10 16:35:08 -07:00
|
|
|
}
|
|
|
|
|
2018-09-25 04:14:38 -07:00
|
|
|
type ReactorOption func(*ConsensusReactor)
|
|
|
|
|
2018-05-16 11:25:53 +04:00
|
|
|
// NewConsensusReactor returns a new ConsensusReactor with the given
|
|
|
|
// consensusState.
|
2018-09-25 04:14:38 -07:00
|
|
|
func NewConsensusReactor(consensusState *ConsensusState, fastSync bool, options ...ReactorOption) *ConsensusReactor {
|
2014-09-14 15:37:32 -07:00
|
|
|
conR := &ConsensusReactor{
|
2016-09-15 16:01:01 -07:00
|
|
|
conS: consensusState,
|
|
|
|
fastSync: fastSync,
|
2018-10-04 15:37:13 +02:00
|
|
|
metrics: NopMetrics(),
|
2014-08-10 16:35:08 -07:00
|
|
|
}
|
2018-09-25 04:14:38 -07:00
|
|
|
conR.updateFastSyncingMetric()
|
2017-05-02 11:53:32 +04:00
|
|
|
conR.BaseReactor = *p2p.NewBaseReactor("ConsensusReactor", conR)
|
2018-09-25 04:14:38 -07:00
|
|
|
|
|
|
|
for _, option := range options {
|
|
|
|
option(conR)
|
|
|
|
}
|
|
|
|
|
2014-09-14 15:37:32 -07:00
|
|
|
return conR
|
2014-08-10 16:35:08 -07:00
|
|
|
}
|
|
|
|
|
2018-05-16 11:25:53 +04:00
|
|
|
// OnStart implements BaseService by subscribing to events, which later will be
|
|
|
|
// broadcasted to other peers and starting state if we're not in fast sync.
|
2015-08-04 18:44:15 -07:00
|
|
|
func (conR *ConsensusReactor) OnStart() error {
|
2017-08-09 14:55:21 -04:00
|
|
|
conR.Logger.Info("ConsensusReactor ", "fastSync", conR.FastSync())
|
2015-12-13 19:30:15 -05:00
|
|
|
|
2018-09-21 20:36:48 +02:00
|
|
|
// start routine that computes peer statistics for evaluating peer quality
|
|
|
|
go conR.peerStatsRoutine()
|
|
|
|
|
2018-05-16 11:25:53 +04:00
|
|
|
conR.subscribeToBroadcastEvents()
|
|
|
|
|
2017-08-09 14:55:21 -04:00
|
|
|
if !conR.FastSync() {
|
2017-11-06 13:20:39 -05:00
|
|
|
err := conR.conS.Start()
|
2015-08-04 18:44:15 -07:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2014-09-14 15:37:32 -07:00
|
|
|
}
|
new pubsub package
comment out failing consensus tests for now
rewrite rpc httpclient to use new pubsub package
import pubsub as tmpubsub, query as tmquery
make event IDs constants
EventKey -> EventTypeKey
rename EventsPubsub to PubSub
mempool does not use pubsub
rename eventsSub to pubsub
new subscribe API
fix channel size issues and consensus tests bugs
refactor rpc client
add missing discardFromChan method
add mutex
rename pubsub to eventBus
remove IsRunning from WSRPCConnection interface (not needed)
add a comment in broadcastNewRoundStepsAndVotes
rename registerEventCallbacks to broadcastNewRoundStepsAndVotes
See https://dave.cheney.net/2014/03/19/channel-axioms
stop eventBuses after reactor tests
remove unnecessary Unsubscribe
return subscribe helper function
move discardFromChan to where it is used
subscribe now returns an err
this gives us ability to refuse to subscribe if pubsub is at its max
capacity.
use context for control overflow
cache queries
handle err when subscribing in replay_test
rename testClientID to testSubscriber
extract var
set channel buffer capacity to 1 in replay_file
fix byzantine_test
unsubscribe from single event, not all events
refactor httpclient to return events to appropriate channels
return failing testReplayCrashBeforeWriteVote test
fix TestValidatorSetChanges
refactor code a bit
fix testReplayCrashBeforeWriteVote
add comment
fix TestValidatorSetChanges
fixes from Bucky's review
update comment [ci skip]
test TxEventBuffer
update changelog
fix TestValidatorSetChanges (2nd attempt)
only do wg.Done when no errors
benchmark event bus
create pubsub server inside NewEventBus
only expose config params (later if needed)
set buffer capacity to 0 so we are not testing cache
new tx event format: key = "Tx" plus a tag {"tx.hash": XYZ}
This should allow to subscribe to all transactions! or a specific one
using a query: "tm.events.type = Tx and tx.hash = '013ABF99434...'"
use TimeoutCommit instead of afterPublishEventNewBlockTimeout
TimeoutCommit is the time a node waits after committing a block, before
it goes into the next height. So it will finish everything from the last
block, but then wait a bit. The idea is this gives it time to hear more
votes from other validators, to strengthen the commit it includes in the
next block. But it also gives it time to hear about new transactions.
waitForBlockWithUpdatedVals
rewrite WAL crash tests
Task:
test that we can recover from any WAL crash.
Solution:
the old tests were relying on event hub being run in the same thread (we
were injecting the private validator's last signature).
when considering a rewrite, we considered two possible solutions: write
a "fuzzy" testing system where WAL is crashing upon receiving a new
message, or inject failures and trigger them in tests using something
like https://github.com/coreos/gofail.
remove sleep
no cs.Lock around wal.Save
test different cases (empty block, non-empty block, ...)
comments
add comments
test 4 cases: empty block, non-empty block, non-empty block with smaller part size, many blocks
fixes as per Bucky's last review
reset subscriptions on UnsubscribeAll
use a simple counter to track message for which we panicked
also, set a smaller part size for all test cases
2017-06-26 19:00:30 +04:00
|
|
|
|
2015-08-04 18:44:15 -07:00
|
|
|
return nil
|
2014-09-07 02:21:25 -07:00
|
|
|
}
|
|
|
|
|
2018-05-16 11:25:53 +04:00
|
|
|
// OnStop implements BaseService by unsubscribing from events and stopping
|
|
|
|
// state.
|
2015-07-21 18:31:01 -07:00
|
|
|
func (conR *ConsensusReactor) OnStop() {
|
2018-05-16 11:25:53 +04:00
|
|
|
conR.unsubscribeFromBroadcastEvents()
|
2015-07-20 14:40:41 -07:00
|
|
|
conR.conS.Stop()
|
2018-07-14 12:53:54 +01:00
|
|
|
if !conR.FastSync() {
|
|
|
|
conR.conS.Wait()
|
|
|
|
}
|
2014-08-10 16:35:08 -07:00
|
|
|
}
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// SwitchToConsensus switches from fast_sync mode to consensus mode.
|
|
|
|
// It resets the state, turns off fast_sync, and starts the consensus state-machine
|
2017-12-27 22:09:48 -05:00
|
|
|
func (conR *ConsensusReactor) SwitchToConsensus(state sm.State, blocksSynced int) {
|
2017-05-02 11:53:32 +04:00
|
|
|
conR.Logger.Info("SwitchToConsensus")
|
2016-02-07 16:56:59 -08:00
|
|
|
conR.conS.reconstructLastCommit(state)
|
2015-07-20 14:40:41 -07:00
|
|
|
// NOTE: The line below causes broadcastNewRoundStepRoutine() to
|
|
|
|
// broadcast a NewRoundStepMessage.
|
2015-09-15 16:13:39 -04:00
|
|
|
conR.conS.updateToState(state)
|
2017-08-09 14:55:21 -04:00
|
|
|
|
|
|
|
conR.mtx.Lock()
|
2015-07-20 14:40:41 -07:00
|
|
|
conR.fastSync = false
|
2017-08-09 14:55:21 -04:00
|
|
|
conR.mtx.Unlock()
|
2018-09-25 04:14:38 -07:00
|
|
|
conR.metrics.FastSyncing.Set(0)
|
2017-08-09 14:55:21 -04:00
|
|
|
|
2017-10-26 18:29:23 -04:00
|
|
|
if blocksSynced > 0 {
|
|
|
|
// dont bother with the WAL if we fast synced
|
|
|
|
conR.conS.doWALCatchup = false
|
|
|
|
}
|
2017-11-06 13:20:39 -05:00
|
|
|
err := conR.conS.Start()
|
2017-09-21 10:56:42 -04:00
|
|
|
if err != nil {
|
2017-10-03 18:12:17 -04:00
|
|
|
conR.Logger.Error("Error starting conS", "err", err)
|
2018-05-16 10:28:58 +04:00
|
|
|
return
|
2017-09-21 10:56:42 -04:00
|
|
|
}
|
2014-10-30 03:32:09 -07:00
|
|
|
}
|
|
|
|
|
2017-07-09 20:35:48 +02:00
|
|
|
// GetChannels implements Reactor
|
2014-09-14 15:37:32 -07:00
|
|
|
func (conR *ConsensusReactor) GetChannels() []*p2p.ChannelDescriptor {
|
|
|
|
// TODO optimize
|
|
|
|
return []*p2p.ChannelDescriptor{
|
2017-09-05 16:37:20 -04:00
|
|
|
{
|
2018-04-06 13:46:40 -07:00
|
|
|
ID: StateChannel,
|
|
|
|
Priority: 5,
|
|
|
|
SendQueueCapacity: 100,
|
2018-04-09 15:14:33 +03:00
|
|
|
RecvMessageCapacity: maxMsgSize,
|
2014-09-14 15:37:32 -07:00
|
|
|
},
|
2017-09-05 16:37:20 -04:00
|
|
|
{
|
2018-04-06 13:46:40 -07:00
|
|
|
ID: DataChannel, // maybe split between gossiping current block and catchup stuff
|
|
|
|
Priority: 10, // once we gossip the whole block there's nothing left to send until next height or round
|
|
|
|
SendQueueCapacity: 100,
|
|
|
|
RecvBufferCapacity: 50 * 4096,
|
2018-04-09 15:14:33 +03:00
|
|
|
RecvMessageCapacity: maxMsgSize,
|
2014-09-14 15:37:32 -07:00
|
|
|
},
|
2017-09-05 16:37:20 -04:00
|
|
|
{
|
2018-04-06 13:46:40 -07:00
|
|
|
ID: VoteChannel,
|
|
|
|
Priority: 5,
|
|
|
|
SendQueueCapacity: 100,
|
|
|
|
RecvBufferCapacity: 100 * 100,
|
2018-04-09 15:14:33 +03:00
|
|
|
RecvMessageCapacity: maxMsgSize,
|
2014-09-14 15:37:32 -07:00
|
|
|
},
|
2017-09-05 16:37:20 -04:00
|
|
|
{
|
2018-04-06 13:46:40 -07:00
|
|
|
ID: VoteSetBitsChannel,
|
|
|
|
Priority: 1,
|
|
|
|
SendQueueCapacity: 2,
|
|
|
|
RecvBufferCapacity: 1024,
|
2018-04-09 15:14:33 +03:00
|
|
|
RecvMessageCapacity: maxMsgSize,
|
2016-09-05 17:33:02 -07:00
|
|
|
},
|
2014-08-10 16:35:08 -07:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-05-28 03:39:58 +09:00
|
|
|
// InitPeer implements Reactor by creating a state for the peer.
|
|
|
|
func (conR *ConsensusReactor) InitPeer(peer p2p.Peer) p2p.Peer {
|
|
|
|
peerState := NewPeerState(peer).SetLogger(conR.Logger)
|
|
|
|
peer.Set(types.PeerStateKey, peerState)
|
|
|
|
return peer
|
|
|
|
}
|
|
|
|
|
|
|
|
// AddPeer implements Reactor by spawning multiple gossiping goroutines for the
|
|
|
|
// peer.
|
2017-09-12 20:49:22 -04:00
|
|
|
func (conR *ConsensusReactor) AddPeer(peer p2p.Peer) {
|
2015-03-25 13:01:28 -07:00
|
|
|
if !conR.IsRunning() {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2019-05-28 03:39:58 +09:00
|
|
|
peerState, ok := peer.Get(types.PeerStateKey).(*PeerState)
|
|
|
|
if !ok {
|
|
|
|
panic(fmt.Sprintf("peer %v has no state", peer))
|
|
|
|
}
|
2016-09-05 17:33:02 -07:00
|
|
|
// Begin routines for this peer.
|
2014-09-14 15:37:32 -07:00
|
|
|
go conR.gossipDataRoutine(peer, peerState)
|
|
|
|
go conR.gossipVotesRoutine(peer, peerState)
|
2016-09-05 17:33:02 -07:00
|
|
|
go conR.queryMaj23Routine(peer, peerState)
|
2015-01-13 17:52:27 -08:00
|
|
|
|
|
|
|
// Send our state to peer.
|
2015-07-09 21:46:15 -07:00
|
|
|
// If we're fast_syncing, broadcast a RoundStepMessage later upon SwitchToConsensus().
|
2017-08-09 14:55:21 -04:00
|
|
|
if !conR.FastSync() {
|
2018-10-31 14:20:36 +01:00
|
|
|
conR.sendNewRoundStepMessage(peer)
|
2015-07-09 21:46:15 -07:00
|
|
|
}
|
2014-08-10 16:35:08 -07:00
|
|
|
}
|
|
|
|
|
2019-05-28 03:39:58 +09:00
|
|
|
// RemovePeer is a noop.
|
2017-09-12 20:49:22 -04:00
|
|
|
func (conR *ConsensusReactor) RemovePeer(peer p2p.Peer, reason interface{}) {
|
2015-03-25 13:01:28 -07:00
|
|
|
if !conR.IsRunning() {
|
|
|
|
return
|
|
|
|
}
|
2015-07-10 15:39:49 +00:00
|
|
|
// TODO
|
abci: localClient improvements & bugfixes & pubsub Unsubscribe issues (#2748)
* use READ lock/unlock in ConsensusState#GetLastHeight
Refs #2721
* do not use defers when there's no need
* fix peer formatting (output its address instead of the pointer)
```
[54310]: E[11-02|11:59:39.851] Connection failed @ sendRoutine module=p2p peer=0xb78f00 conn=MConn{74.207.236.148:26656} err="pong timeout"
```
https://github.com/tendermint/tendermint/issues/2721#issuecomment-435326581
* panic if peer has no state
https://github.com/tendermint/tendermint/issues/2721#issuecomment-435347165
It's confusing that sometimes we check if peer has a state, but most of
the times we expect it to be there
1. https://github.com/tendermint/tendermint/blob/add79700b5fe84417538202b6c927c8cc5383672/mempool/reactor.go#L138
2. https://github.com/tendermint/tendermint/blob/add79700b5fe84417538202b6c927c8cc5383672/rpc/core/consensus.go#L196 (edited)
I will change everything to always assume peer has a state and panic
otherwise
that should help identify issues earlier
* abci/localclient: extend lock on app callback
App callback should be protected by lock as well (note this was already
done for InitChainAsync, why not for others???). Otherwise, when we
execute the block, tx might come in and call the callback in the same
time we're updating it in execBlockOnProxyApp => DATA RACE
Fixes #2721
Consensus state is locked
```
goroutine 113333 [semacquire, 309 minutes]:
sync.runtime_SemacquireMutex(0xc00180009c, 0xc0000c7e00)
/usr/local/go/src/runtime/sema.go:71 +0x3d
sync.(*RWMutex).RLock(0xc001800090)
/usr/local/go/src/sync/rwmutex.go:50 +0x4e
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).GetRoundState(0xc001800000, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:218 +0x46
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusReactor).queryMaj23Routine(0xc0017def80, 0x11104a0, 0xc0072488f0, 0xc007248
9c0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/reactor.go:735 +0x16d
created by github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusReactor).AddPeer
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/reactor.go:172 +0x236
```
because localClient is locked
```
goroutine 1899 [semacquire, 309 minutes]:
sync.runtime_SemacquireMutex(0xc00003363c, 0xc0000cb500)
/usr/local/go/src/runtime/sema.go:71 +0x3d
sync.(*Mutex).Lock(0xc000033638)
/usr/local/go/src/sync/mutex.go:134 +0xff
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client.(*localClient).SetResponseCallback(0xc0001fb560, 0xc007868540)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client/local_client.go:32 +0x33
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy.(*appConnConsensus).SetResponseCallback(0xc00002f750, 0xc007868540)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy/app_conn.go:57 +0x40
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state.execBlockOnProxyApp(0x1104e20, 0xc002ca0ba0, 0x11092a0, 0xc00002f750, 0xc0001fe960, 0xc000bfc660, 0x110cfe0, 0xc000090330, 0xc9d12, 0xc000d9d5a0, ...)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state/execution.go:230 +0x1fd
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state.(*BlockExecutor).ApplyBlock(0xc002c2a230, 0x7, 0x0, 0xc000eae880, 0x6, 0xc002e52c60, 0x16, 0x1f927, 0xc9d12, 0xc000d9d5a0, ...)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state/execution.go:96 +0x142
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).finalizeCommit(0xc001800000, 0x1f928)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1339 +0xa3e
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).tryFinalizeCommit(0xc001800000, 0x1f928)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1270 +0x451
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).enterCommit.func1(0xc001800000, 0x0, 0x1f928)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1218 +0x90
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).enterCommit(0xc001800000, 0x1f928, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1247 +0x6b8
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).addVote(0xc001800000, 0xc003d8dea0, 0xc000cf4cc0, 0x28, 0xf1, 0xc003bc7ad0, 0xc003bc7b10)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1659 +0xbad
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).tryAddVote(0xc001800000, 0xc003d8dea0, 0xc000cf4cc0, 0x28, 0xf1, 0xf1, 0xf1)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1517 +0x59
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).handleMsg(0xc001800000, 0xd98200, 0xc0070dbed0, 0xc000cf4cc0, 0x28)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:660 +0x64b
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).receiveRoutine(0xc001800000, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:617 +0x670
created by github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).OnStart
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:311 +0x132
```
tx comes in and CheckTx is executed right when we execute the block
```
goroutine 111044 [semacquire, 309 minutes]:
sync.runtime_SemacquireMutex(0xc00003363c, 0x0)
/usr/local/go/src/runtime/sema.go:71 +0x3d
sync.(*Mutex).Lock(0xc000033638)
/usr/local/go/src/sync/mutex.go:134 +0xff
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client.(*localClient).CheckTxAsync(0xc0001fb0e0, 0xc002d94500, 0x13f, 0x280, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client/local_client.go:85 +0x47
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy.(*appConnMempool).CheckTxAsync(0xc00002f720, 0xc002d94500, 0x13f, 0x280, 0x1)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy/app_conn.go:114 +0x51
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/mempool.(*Mempool).CheckTx(0xc002d3a320, 0xc002d94500, 0x13f, 0x280, 0xc0072355f0, 0x0, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/mempool/mempool.go:316 +0x17b
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/core.BroadcastTxSync(0xc002d94500, 0x13f, 0x280, 0x0, 0x0, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/core/mempool.go:93 +0xb8
reflect.Value.call(0xd85560, 0x10326c0, 0x13, 0xec7b8b, 0x4, 0xc00663f180, 0x1, 0x1, 0xc00663f180, 0xc00663f188, ...)
/usr/local/go/src/reflect/value.go:447 +0x449
reflect.Value.Call(0xd85560, 0x10326c0, 0x13, 0xc00663f180, 0x1, 0x1, 0x0, 0x0, 0xc005cc9344)
/usr/local/go/src/reflect/value.go:308 +0xa4
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server.makeHTTPHandler.func2(0x1102060, 0xc00663f100, 0xc0082d7900)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server/handlers.go:269 +0x188
net/http.HandlerFunc.ServeHTTP(0xc002c81f20, 0x1102060, 0xc00663f100, 0xc0082d7900)
/usr/local/go/src/net/http/server.go:1964 +0x44
net/http.(*ServeMux).ServeHTTP(0xc002c81b60, 0x1102060, 0xc00663f100, 0xc0082d7900)
/usr/local/go/src/net/http/server.go:2361 +0x127
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server.maxBytesHandler.ServeHTTP(0x10f8a40, 0xc002c81b60, 0xf4240, 0x1102060, 0xc00663f100, 0xc0082d7900)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server/http_server.go:219 +0xcf
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server.RecoverAndLogHandler.func1(0x1103220, 0xc00121e620, 0xc0082d7900)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server/http_server.go:192 +0x394
net/http.HandlerFunc.ServeHTTP(0xc002c06ea0, 0x1103220, 0xc00121e620, 0xc0082d7900)
/usr/local/go/src/net/http/server.go:1964 +0x44
net/http.serverHandler.ServeHTTP(0xc001a1aa90, 0x1103220, 0xc00121e620, 0xc0082d7900)
/usr/local/go/src/net/http/server.go:2741 +0xab
net/http.(*conn).serve(0xc00785a3c0, 0x11041a0, 0xc000f844c0)
/usr/local/go/src/net/http/server.go:1847 +0x646
created by net/http.(*Server).Serve
/usr/local/go/src/net/http/server.go:2851 +0x2f5
```
* consensus: use read lock in Receive#VoteMessage
* use defer to unlock mutex because application might panic
* use defer in every method of the localClient
* add a changelog entry
* drain channels before Unsubscribe(All)
Read https://github.com/tendermint/tendermint/blob/55362ed76630f3e1ebec159a598f6a9fb5892cb1/libs/pubsub/pubsub.go#L13
for the detailed explanation of the issue.
We'll need to fix it someday. Make sure to keep an eye on
https://github.com/tendermint/tendermint/blob/master/docs/architecture/adr-033-pubsub.md
* retry instead of panic when peer has no state in reactors other than consensus
in /dump_consensus_state RPC endpoint, skip a peer with no state
* rpc/core/mempool: simplify error messages
* rpc/core/mempool: use time.After instead of timer
also, do not log DeliverTx result (to be consistent with other memthods)
* unlock before calling the callback in reqRes#SetCallback
2018-11-13 20:32:51 +04:00
|
|
|
// ps, ok := peer.Get(PeerStateKey).(*PeerState)
|
|
|
|
// if !ok {
|
|
|
|
// panic(fmt.Sprintf("Peer %v has no state", peer))
|
|
|
|
// }
|
|
|
|
// ps.Disconnect()
|
2014-08-10 16:35:08 -07:00
|
|
|
}
|
|
|
|
|
2017-07-09 20:35:48 +02:00
|
|
|
// Receive implements Reactor
|
2015-07-05 20:02:23 -07:00
|
|
|
// NOTE: We process these messages even when we're fast_syncing.
|
2015-12-10 11:41:18 -05:00
|
|
|
// Messages affect either a peer state or the consensus state.
|
|
|
|
// Peer state updates can happen in parallel, but processing of
|
2015-12-13 19:30:15 -05:00
|
|
|
// proposals, block parts, and votes are ordered by the receiveRoutine
|
2016-08-14 12:31:24 -04:00
|
|
|
// NOTE: blocks on consensus state for proposals, block parts, and votes
|
2017-09-12 20:49:22 -04:00
|
|
|
func (conR *ConsensusReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) {
|
2015-07-05 20:02:23 -07:00
|
|
|
if !conR.IsRunning() {
|
2017-05-02 11:53:32 +04:00
|
|
|
conR.Logger.Debug("Receive", "src", src, "chId", chID, "bytes", msgBytes)
|
2015-03-25 13:01:28 -07:00
|
|
|
return
|
|
|
|
}
|
2014-08-10 16:35:08 -07:00
|
|
|
|
2018-07-09 13:01:23 +04:00
|
|
|
msg, err := decodeMsg(msgBytes)
|
2014-12-29 15:14:54 -08:00
|
|
|
if err != nil {
|
2017-06-14 12:50:49 +04:00
|
|
|
conR.Logger.Error("Error decoding message", "src", src, "chId", chID, "msg", msg, "err", err, "bytes", msgBytes)
|
2018-03-04 13:42:45 +04:00
|
|
|
conR.Switch.StopPeerForError(src, err)
|
2014-12-29 15:14:54 -08:00
|
|
|
return
|
|
|
|
}
|
2018-11-01 07:07:18 +01:00
|
|
|
|
|
|
|
if err = msg.ValidateBasic(); err != nil {
|
|
|
|
conR.Logger.Error("Peer sent us invalid msg", "peer", src, "msg", msg, "err", err)
|
|
|
|
conR.Switch.StopPeerForError(src, err)
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2017-05-02 11:53:32 +04:00
|
|
|
conR.Logger.Debug("Receive", "src", src, "chId", chID, "msg", msg)
|
2014-10-25 14:27:53 -07:00
|
|
|
|
2015-12-05 14:58:12 -05:00
|
|
|
// Get peer states
|
abci: localClient improvements & bugfixes & pubsub Unsubscribe issues (#2748)
* use READ lock/unlock in ConsensusState#GetLastHeight
Refs #2721
* do not use defers when there's no need
* fix peer formatting (output its address instead of the pointer)
```
[54310]: E[11-02|11:59:39.851] Connection failed @ sendRoutine module=p2p peer=0xb78f00 conn=MConn{74.207.236.148:26656} err="pong timeout"
```
https://github.com/tendermint/tendermint/issues/2721#issuecomment-435326581
* panic if peer has no state
https://github.com/tendermint/tendermint/issues/2721#issuecomment-435347165
It's confusing that sometimes we check if peer has a state, but most of
the times we expect it to be there
1. https://github.com/tendermint/tendermint/blob/add79700b5fe84417538202b6c927c8cc5383672/mempool/reactor.go#L138
2. https://github.com/tendermint/tendermint/blob/add79700b5fe84417538202b6c927c8cc5383672/rpc/core/consensus.go#L196 (edited)
I will change everything to always assume peer has a state and panic
otherwise
that should help identify issues earlier
* abci/localclient: extend lock on app callback
App callback should be protected by lock as well (note this was already
done for InitChainAsync, why not for others???). Otherwise, when we
execute the block, tx might come in and call the callback in the same
time we're updating it in execBlockOnProxyApp => DATA RACE
Fixes #2721
Consensus state is locked
```
goroutine 113333 [semacquire, 309 minutes]:
sync.runtime_SemacquireMutex(0xc00180009c, 0xc0000c7e00)
/usr/local/go/src/runtime/sema.go:71 +0x3d
sync.(*RWMutex).RLock(0xc001800090)
/usr/local/go/src/sync/rwmutex.go:50 +0x4e
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).GetRoundState(0xc001800000, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:218 +0x46
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusReactor).queryMaj23Routine(0xc0017def80, 0x11104a0, 0xc0072488f0, 0xc007248
9c0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/reactor.go:735 +0x16d
created by github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusReactor).AddPeer
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/reactor.go:172 +0x236
```
because localClient is locked
```
goroutine 1899 [semacquire, 309 minutes]:
sync.runtime_SemacquireMutex(0xc00003363c, 0xc0000cb500)
/usr/local/go/src/runtime/sema.go:71 +0x3d
sync.(*Mutex).Lock(0xc000033638)
/usr/local/go/src/sync/mutex.go:134 +0xff
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client.(*localClient).SetResponseCallback(0xc0001fb560, 0xc007868540)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client/local_client.go:32 +0x33
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy.(*appConnConsensus).SetResponseCallback(0xc00002f750, 0xc007868540)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy/app_conn.go:57 +0x40
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state.execBlockOnProxyApp(0x1104e20, 0xc002ca0ba0, 0x11092a0, 0xc00002f750, 0xc0001fe960, 0xc000bfc660, 0x110cfe0, 0xc000090330, 0xc9d12, 0xc000d9d5a0, ...)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state/execution.go:230 +0x1fd
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state.(*BlockExecutor).ApplyBlock(0xc002c2a230, 0x7, 0x0, 0xc000eae880, 0x6, 0xc002e52c60, 0x16, 0x1f927, 0xc9d12, 0xc000d9d5a0, ...)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state/execution.go:96 +0x142
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).finalizeCommit(0xc001800000, 0x1f928)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1339 +0xa3e
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).tryFinalizeCommit(0xc001800000, 0x1f928)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1270 +0x451
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).enterCommit.func1(0xc001800000, 0x0, 0x1f928)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1218 +0x90
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).enterCommit(0xc001800000, 0x1f928, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1247 +0x6b8
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).addVote(0xc001800000, 0xc003d8dea0, 0xc000cf4cc0, 0x28, 0xf1, 0xc003bc7ad0, 0xc003bc7b10)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1659 +0xbad
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).tryAddVote(0xc001800000, 0xc003d8dea0, 0xc000cf4cc0, 0x28, 0xf1, 0xf1, 0xf1)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1517 +0x59
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).handleMsg(0xc001800000, 0xd98200, 0xc0070dbed0, 0xc000cf4cc0, 0x28)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:660 +0x64b
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).receiveRoutine(0xc001800000, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:617 +0x670
created by github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).OnStart
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:311 +0x132
```
tx comes in and CheckTx is executed right when we execute the block
```
goroutine 111044 [semacquire, 309 minutes]:
sync.runtime_SemacquireMutex(0xc00003363c, 0x0)
/usr/local/go/src/runtime/sema.go:71 +0x3d
sync.(*Mutex).Lock(0xc000033638)
/usr/local/go/src/sync/mutex.go:134 +0xff
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client.(*localClient).CheckTxAsync(0xc0001fb0e0, 0xc002d94500, 0x13f, 0x280, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client/local_client.go:85 +0x47
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy.(*appConnMempool).CheckTxAsync(0xc00002f720, 0xc002d94500, 0x13f, 0x280, 0x1)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy/app_conn.go:114 +0x51
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/mempool.(*Mempool).CheckTx(0xc002d3a320, 0xc002d94500, 0x13f, 0x280, 0xc0072355f0, 0x0, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/mempool/mempool.go:316 +0x17b
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/core.BroadcastTxSync(0xc002d94500, 0x13f, 0x280, 0x0, 0x0, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/core/mempool.go:93 +0xb8
reflect.Value.call(0xd85560, 0x10326c0, 0x13, 0xec7b8b, 0x4, 0xc00663f180, 0x1, 0x1, 0xc00663f180, 0xc00663f188, ...)
/usr/local/go/src/reflect/value.go:447 +0x449
reflect.Value.Call(0xd85560, 0x10326c0, 0x13, 0xc00663f180, 0x1, 0x1, 0x0, 0x0, 0xc005cc9344)
/usr/local/go/src/reflect/value.go:308 +0xa4
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server.makeHTTPHandler.func2(0x1102060, 0xc00663f100, 0xc0082d7900)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server/handlers.go:269 +0x188
net/http.HandlerFunc.ServeHTTP(0xc002c81f20, 0x1102060, 0xc00663f100, 0xc0082d7900)
/usr/local/go/src/net/http/server.go:1964 +0x44
net/http.(*ServeMux).ServeHTTP(0xc002c81b60, 0x1102060, 0xc00663f100, 0xc0082d7900)
/usr/local/go/src/net/http/server.go:2361 +0x127
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server.maxBytesHandler.ServeHTTP(0x10f8a40, 0xc002c81b60, 0xf4240, 0x1102060, 0xc00663f100, 0xc0082d7900)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server/http_server.go:219 +0xcf
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server.RecoverAndLogHandler.func1(0x1103220, 0xc00121e620, 0xc0082d7900)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server/http_server.go:192 +0x394
net/http.HandlerFunc.ServeHTTP(0xc002c06ea0, 0x1103220, 0xc00121e620, 0xc0082d7900)
/usr/local/go/src/net/http/server.go:1964 +0x44
net/http.serverHandler.ServeHTTP(0xc001a1aa90, 0x1103220, 0xc00121e620, 0xc0082d7900)
/usr/local/go/src/net/http/server.go:2741 +0xab
net/http.(*conn).serve(0xc00785a3c0, 0x11041a0, 0xc000f844c0)
/usr/local/go/src/net/http/server.go:1847 +0x646
created by net/http.(*Server).Serve
/usr/local/go/src/net/http/server.go:2851 +0x2f5
```
* consensus: use read lock in Receive#VoteMessage
* use defer to unlock mutex because application might panic
* use defer in every method of the localClient
* add a changelog entry
* drain channels before Unsubscribe(All)
Read https://github.com/tendermint/tendermint/blob/55362ed76630f3e1ebec159a598f6a9fb5892cb1/libs/pubsub/pubsub.go#L13
for the detailed explanation of the issue.
We'll need to fix it someday. Make sure to keep an eye on
https://github.com/tendermint/tendermint/blob/master/docs/architecture/adr-033-pubsub.md
* retry instead of panic when peer has no state in reactors other than consensus
in /dump_consensus_state RPC endpoint, skip a peer with no state
* rpc/core/mempool: simplify error messages
* rpc/core/mempool: use time.After instead of timer
also, do not log DeliverTx result (to be consistent with other memthods)
* unlock before calling the callback in reqRes#SetCallback
2018-11-13 20:32:51 +04:00
|
|
|
ps, ok := src.Get(types.PeerStateKey).(*PeerState)
|
|
|
|
if !ok {
|
|
|
|
panic(fmt.Sprintf("Peer %v has no state", src))
|
|
|
|
}
|
2015-12-05 14:58:12 -05:00
|
|
|
|
2015-08-18 10:51:55 -07:00
|
|
|
switch chID {
|
2015-03-24 12:00:27 -07:00
|
|
|
case StateChannel:
|
2015-07-13 16:00:01 -07:00
|
|
|
switch msg := msg.(type) {
|
2014-09-14 15:37:32 -07:00
|
|
|
case *NewRoundStepMessage:
|
2015-08-26 18:56:34 -04:00
|
|
|
ps.ApplyNewRoundStepMessage(msg)
|
2018-10-31 14:20:36 +01:00
|
|
|
case *NewValidBlockMessage:
|
|
|
|
ps.ApplyNewValidBlockMessage(msg)
|
2014-11-01 22:42:04 -07:00
|
|
|
case *HasVoteMessage:
|
|
|
|
ps.ApplyHasVoteMessage(msg)
|
2016-09-05 17:33:02 -07:00
|
|
|
case *VoteSetMaj23Message:
|
|
|
|
cs := conR.conS
|
|
|
|
cs.mtx.Lock()
|
|
|
|
height, votes := cs.Height, cs.Votes
|
|
|
|
cs.mtx.Unlock()
|
2016-09-16 09:20:07 -07:00
|
|
|
if height != msg.Height {
|
|
|
|
return
|
2016-09-05 17:33:02 -07:00
|
|
|
}
|
2016-09-16 09:20:07 -07:00
|
|
|
// Peer claims to have a maj23 for some BlockID at H,R,S,
|
2018-05-10 22:43:21 -07:00
|
|
|
err := votes.SetPeerMaj23(msg.Round, msg.Type, ps.peer.ID(), msg.BlockID)
|
2018-01-21 13:32:04 -05:00
|
|
|
if err != nil {
|
|
|
|
conR.Switch.StopPeerForError(src, err)
|
|
|
|
return
|
|
|
|
}
|
2016-09-16 09:20:07 -07:00
|
|
|
// Respond with a VoteSetBitsMessage showing which votes we have.
|
|
|
|
// (and consequently shows which we don't have)
|
2017-07-09 18:38:59 -04:00
|
|
|
var ourVotes *cmn.BitArray
|
2016-09-16 09:20:07 -07:00
|
|
|
switch msg.Type {
|
2018-10-13 01:21:46 +02:00
|
|
|
case types.PrevoteType:
|
2016-09-16 09:20:07 -07:00
|
|
|
ourVotes = votes.Prevotes(msg.Round).BitArrayByBlockID(msg.BlockID)
|
2018-10-13 01:21:46 +02:00
|
|
|
case types.PrecommitType:
|
2016-09-16 09:20:07 -07:00
|
|
|
ourVotes = votes.Precommits(msg.Round).BitArrayByBlockID(msg.BlockID)
|
|
|
|
default:
|
2018-11-01 07:07:18 +01:00
|
|
|
panic("Bad VoteSetBitsMessage field Type. Forgot to add a check in ValidateBasic?")
|
2016-09-16 09:20:07 -07:00
|
|
|
}
|
2018-04-05 07:05:45 -07:00
|
|
|
src.TrySend(VoteSetBitsChannel, cdc.MustMarshalBinaryBare(&VoteSetBitsMessage{
|
2016-09-16 09:20:07 -07:00
|
|
|
Height: msg.Height,
|
|
|
|
Round: msg.Round,
|
|
|
|
Type: msg.Type,
|
|
|
|
BlockID: msg.BlockID,
|
|
|
|
Votes: ourVotes,
|
2018-04-05 07:05:45 -07:00
|
|
|
}))
|
2014-09-14 15:37:32 -07:00
|
|
|
default:
|
2018-08-10 00:25:57 -05:00
|
|
|
conR.Logger.Error(fmt.Sprintf("Unknown message type %v", reflect.TypeOf(msg)))
|
2014-09-14 15:37:32 -07:00
|
|
|
}
|
2014-08-10 16:35:08 -07:00
|
|
|
|
2015-03-24 12:00:27 -07:00
|
|
|
case DataChannel:
|
2017-08-09 14:55:21 -04:00
|
|
|
if conR.FastSync() {
|
2017-05-12 23:07:53 +02:00
|
|
|
conR.Logger.Info("Ignoring message received during fastSync", "msg", msg)
|
2015-07-09 21:46:15 -07:00
|
|
|
return
|
|
|
|
}
|
2015-07-13 16:00:01 -07:00
|
|
|
switch msg := msg.(type) {
|
2015-04-12 18:44:06 -07:00
|
|
|
case *ProposalMessage:
|
|
|
|
ps.SetHasProposal(msg.Proposal)
|
2018-01-01 21:27:38 -05:00
|
|
|
conR.conS.peerMsgQueue <- msgInfo{msg, src.ID()}
|
2015-06-22 19:04:31 -07:00
|
|
|
case *ProposalPOLMessage:
|
|
|
|
ps.ApplyProposalPOLMessage(msg)
|
|
|
|
case *BlockPartMessage:
|
2015-12-10 11:09:25 -08:00
|
|
|
ps.SetHasProposalBlockPart(msg.Height, msg.Round, msg.Part.Index)
|
2018-10-10 09:27:43 -07:00
|
|
|
conR.metrics.BlockParts.With("peer_id", string(src.ID())).Add(1)
|
2018-01-01 21:27:38 -05:00
|
|
|
conR.conS.peerMsgQueue <- msgInfo{msg, src.ID()}
|
2014-09-14 15:37:32 -07:00
|
|
|
default:
|
2018-08-10 00:25:57 -05:00
|
|
|
conR.Logger.Error(fmt.Sprintf("Unknown message type %v", reflect.TypeOf(msg)))
|
2014-09-14 15:37:32 -07:00
|
|
|
}
|
2014-09-07 02:21:25 -07:00
|
|
|
|
2015-03-24 12:00:27 -07:00
|
|
|
case VoteChannel:
|
2017-08-09 14:55:21 -04:00
|
|
|
if conR.FastSync() {
|
2017-05-02 11:53:32 +04:00
|
|
|
conR.Logger.Info("Ignoring message received during fastSync", "msg", msg)
|
2015-07-09 21:46:15 -07:00
|
|
|
return
|
|
|
|
}
|
2015-07-13 16:00:01 -07:00
|
|
|
switch msg := msg.(type) {
|
2014-12-09 18:49:04 -08:00
|
|
|
case *VoteMessage:
|
2015-08-26 18:56:34 -04:00
|
|
|
cs := conR.conS
|
abci: localClient improvements & bugfixes & pubsub Unsubscribe issues (#2748)
* use READ lock/unlock in ConsensusState#GetLastHeight
Refs #2721
* do not use defers when there's no need
* fix peer formatting (output its address instead of the pointer)
```
[54310]: E[11-02|11:59:39.851] Connection failed @ sendRoutine module=p2p peer=0xb78f00 conn=MConn{74.207.236.148:26656} err="pong timeout"
```
https://github.com/tendermint/tendermint/issues/2721#issuecomment-435326581
* panic if peer has no state
https://github.com/tendermint/tendermint/issues/2721#issuecomment-435347165
It's confusing that sometimes we check if peer has a state, but most of
the times we expect it to be there
1. https://github.com/tendermint/tendermint/blob/add79700b5fe84417538202b6c927c8cc5383672/mempool/reactor.go#L138
2. https://github.com/tendermint/tendermint/blob/add79700b5fe84417538202b6c927c8cc5383672/rpc/core/consensus.go#L196 (edited)
I will change everything to always assume peer has a state and panic
otherwise
that should help identify issues earlier
* abci/localclient: extend lock on app callback
App callback should be protected by lock as well (note this was already
done for InitChainAsync, why not for others???). Otherwise, when we
execute the block, tx might come in and call the callback in the same
time we're updating it in execBlockOnProxyApp => DATA RACE
Fixes #2721
Consensus state is locked
```
goroutine 113333 [semacquire, 309 minutes]:
sync.runtime_SemacquireMutex(0xc00180009c, 0xc0000c7e00)
/usr/local/go/src/runtime/sema.go:71 +0x3d
sync.(*RWMutex).RLock(0xc001800090)
/usr/local/go/src/sync/rwmutex.go:50 +0x4e
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).GetRoundState(0xc001800000, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:218 +0x46
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusReactor).queryMaj23Routine(0xc0017def80, 0x11104a0, 0xc0072488f0, 0xc007248
9c0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/reactor.go:735 +0x16d
created by github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusReactor).AddPeer
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/reactor.go:172 +0x236
```
because localClient is locked
```
goroutine 1899 [semacquire, 309 minutes]:
sync.runtime_SemacquireMutex(0xc00003363c, 0xc0000cb500)
/usr/local/go/src/runtime/sema.go:71 +0x3d
sync.(*Mutex).Lock(0xc000033638)
/usr/local/go/src/sync/mutex.go:134 +0xff
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client.(*localClient).SetResponseCallback(0xc0001fb560, 0xc007868540)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client/local_client.go:32 +0x33
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy.(*appConnConsensus).SetResponseCallback(0xc00002f750, 0xc007868540)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy/app_conn.go:57 +0x40
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state.execBlockOnProxyApp(0x1104e20, 0xc002ca0ba0, 0x11092a0, 0xc00002f750, 0xc0001fe960, 0xc000bfc660, 0x110cfe0, 0xc000090330, 0xc9d12, 0xc000d9d5a0, ...)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state/execution.go:230 +0x1fd
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state.(*BlockExecutor).ApplyBlock(0xc002c2a230, 0x7, 0x0, 0xc000eae880, 0x6, 0xc002e52c60, 0x16, 0x1f927, 0xc9d12, 0xc000d9d5a0, ...)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state/execution.go:96 +0x142
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).finalizeCommit(0xc001800000, 0x1f928)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1339 +0xa3e
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).tryFinalizeCommit(0xc001800000, 0x1f928)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1270 +0x451
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).enterCommit.func1(0xc001800000, 0x0, 0x1f928)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1218 +0x90
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).enterCommit(0xc001800000, 0x1f928, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1247 +0x6b8
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).addVote(0xc001800000, 0xc003d8dea0, 0xc000cf4cc0, 0x28, 0xf1, 0xc003bc7ad0, 0xc003bc7b10)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1659 +0xbad
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).tryAddVote(0xc001800000, 0xc003d8dea0, 0xc000cf4cc0, 0x28, 0xf1, 0xf1, 0xf1)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1517 +0x59
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).handleMsg(0xc001800000, 0xd98200, 0xc0070dbed0, 0xc000cf4cc0, 0x28)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:660 +0x64b
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).receiveRoutine(0xc001800000, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:617 +0x670
created by github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).OnStart
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:311 +0x132
```
tx comes in and CheckTx is executed right when we execute the block
```
goroutine 111044 [semacquire, 309 minutes]:
sync.runtime_SemacquireMutex(0xc00003363c, 0x0)
/usr/local/go/src/runtime/sema.go:71 +0x3d
sync.(*Mutex).Lock(0xc000033638)
/usr/local/go/src/sync/mutex.go:134 +0xff
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client.(*localClient).CheckTxAsync(0xc0001fb0e0, 0xc002d94500, 0x13f, 0x280, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client/local_client.go:85 +0x47
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy.(*appConnMempool).CheckTxAsync(0xc00002f720, 0xc002d94500, 0x13f, 0x280, 0x1)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy/app_conn.go:114 +0x51
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/mempool.(*Mempool).CheckTx(0xc002d3a320, 0xc002d94500, 0x13f, 0x280, 0xc0072355f0, 0x0, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/mempool/mempool.go:316 +0x17b
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/core.BroadcastTxSync(0xc002d94500, 0x13f, 0x280, 0x0, 0x0, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/core/mempool.go:93 +0xb8
reflect.Value.call(0xd85560, 0x10326c0, 0x13, 0xec7b8b, 0x4, 0xc00663f180, 0x1, 0x1, 0xc00663f180, 0xc00663f188, ...)
/usr/local/go/src/reflect/value.go:447 +0x449
reflect.Value.Call(0xd85560, 0x10326c0, 0x13, 0xc00663f180, 0x1, 0x1, 0x0, 0x0, 0xc005cc9344)
/usr/local/go/src/reflect/value.go:308 +0xa4
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server.makeHTTPHandler.func2(0x1102060, 0xc00663f100, 0xc0082d7900)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server/handlers.go:269 +0x188
net/http.HandlerFunc.ServeHTTP(0xc002c81f20, 0x1102060, 0xc00663f100, 0xc0082d7900)
/usr/local/go/src/net/http/server.go:1964 +0x44
net/http.(*ServeMux).ServeHTTP(0xc002c81b60, 0x1102060, 0xc00663f100, 0xc0082d7900)
/usr/local/go/src/net/http/server.go:2361 +0x127
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server.maxBytesHandler.ServeHTTP(0x10f8a40, 0xc002c81b60, 0xf4240, 0x1102060, 0xc00663f100, 0xc0082d7900)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server/http_server.go:219 +0xcf
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server.RecoverAndLogHandler.func1(0x1103220, 0xc00121e620, 0xc0082d7900)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server/http_server.go:192 +0x394
net/http.HandlerFunc.ServeHTTP(0xc002c06ea0, 0x1103220, 0xc00121e620, 0xc0082d7900)
/usr/local/go/src/net/http/server.go:1964 +0x44
net/http.serverHandler.ServeHTTP(0xc001a1aa90, 0x1103220, 0xc00121e620, 0xc0082d7900)
/usr/local/go/src/net/http/server.go:2741 +0xab
net/http.(*conn).serve(0xc00785a3c0, 0x11041a0, 0xc000f844c0)
/usr/local/go/src/net/http/server.go:1847 +0x646
created by net/http.(*Server).Serve
/usr/local/go/src/net/http/server.go:2851 +0x2f5
```
* consensus: use read lock in Receive#VoteMessage
* use defer to unlock mutex because application might panic
* use defer in every method of the localClient
* add a changelog entry
* drain channels before Unsubscribe(All)
Read https://github.com/tendermint/tendermint/blob/55362ed76630f3e1ebec159a598f6a9fb5892cb1/libs/pubsub/pubsub.go#L13
for the detailed explanation of the issue.
We'll need to fix it someday. Make sure to keep an eye on
https://github.com/tendermint/tendermint/blob/master/docs/architecture/adr-033-pubsub.md
* retry instead of panic when peer has no state in reactors other than consensus
in /dump_consensus_state RPC endpoint, skip a peer with no state
* rpc/core/mempool: simplify error messages
* rpc/core/mempool: use time.After instead of timer
also, do not log DeliverTx result (to be consistent with other memthods)
* unlock before calling the callback in reqRes#SetCallback
2018-11-13 20:32:51 +04:00
|
|
|
cs.mtx.RLock()
|
2015-08-26 18:56:34 -04:00
|
|
|
height, valSize, lastCommitSize := cs.Height, cs.Validators.Size(), cs.LastCommit.Size()
|
abci: localClient improvements & bugfixes & pubsub Unsubscribe issues (#2748)
* use READ lock/unlock in ConsensusState#GetLastHeight
Refs #2721
* do not use defers when there's no need
* fix peer formatting (output its address instead of the pointer)
```
[54310]: E[11-02|11:59:39.851] Connection failed @ sendRoutine module=p2p peer=0xb78f00 conn=MConn{74.207.236.148:26656} err="pong timeout"
```
https://github.com/tendermint/tendermint/issues/2721#issuecomment-435326581
* panic if peer has no state
https://github.com/tendermint/tendermint/issues/2721#issuecomment-435347165
It's confusing that sometimes we check if peer has a state, but most of
the times we expect it to be there
1. https://github.com/tendermint/tendermint/blob/add79700b5fe84417538202b6c927c8cc5383672/mempool/reactor.go#L138
2. https://github.com/tendermint/tendermint/blob/add79700b5fe84417538202b6c927c8cc5383672/rpc/core/consensus.go#L196 (edited)
I will change everything to always assume peer has a state and panic
otherwise
that should help identify issues earlier
* abci/localclient: extend lock on app callback
App callback should be protected by lock as well (note this was already
done for InitChainAsync, why not for others???). Otherwise, when we
execute the block, tx might come in and call the callback in the same
time we're updating it in execBlockOnProxyApp => DATA RACE
Fixes #2721
Consensus state is locked
```
goroutine 113333 [semacquire, 309 minutes]:
sync.runtime_SemacquireMutex(0xc00180009c, 0xc0000c7e00)
/usr/local/go/src/runtime/sema.go:71 +0x3d
sync.(*RWMutex).RLock(0xc001800090)
/usr/local/go/src/sync/rwmutex.go:50 +0x4e
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).GetRoundState(0xc001800000, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:218 +0x46
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusReactor).queryMaj23Routine(0xc0017def80, 0x11104a0, 0xc0072488f0, 0xc007248
9c0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/reactor.go:735 +0x16d
created by github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusReactor).AddPeer
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/reactor.go:172 +0x236
```
because localClient is locked
```
goroutine 1899 [semacquire, 309 minutes]:
sync.runtime_SemacquireMutex(0xc00003363c, 0xc0000cb500)
/usr/local/go/src/runtime/sema.go:71 +0x3d
sync.(*Mutex).Lock(0xc000033638)
/usr/local/go/src/sync/mutex.go:134 +0xff
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client.(*localClient).SetResponseCallback(0xc0001fb560, 0xc007868540)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client/local_client.go:32 +0x33
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy.(*appConnConsensus).SetResponseCallback(0xc00002f750, 0xc007868540)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy/app_conn.go:57 +0x40
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state.execBlockOnProxyApp(0x1104e20, 0xc002ca0ba0, 0x11092a0, 0xc00002f750, 0xc0001fe960, 0xc000bfc660, 0x110cfe0, 0xc000090330, 0xc9d12, 0xc000d9d5a0, ...)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state/execution.go:230 +0x1fd
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state.(*BlockExecutor).ApplyBlock(0xc002c2a230, 0x7, 0x0, 0xc000eae880, 0x6, 0xc002e52c60, 0x16, 0x1f927, 0xc9d12, 0xc000d9d5a0, ...)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state/execution.go:96 +0x142
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).finalizeCommit(0xc001800000, 0x1f928)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1339 +0xa3e
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).tryFinalizeCommit(0xc001800000, 0x1f928)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1270 +0x451
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).enterCommit.func1(0xc001800000, 0x0, 0x1f928)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1218 +0x90
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).enterCommit(0xc001800000, 0x1f928, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1247 +0x6b8
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).addVote(0xc001800000, 0xc003d8dea0, 0xc000cf4cc0, 0x28, 0xf1, 0xc003bc7ad0, 0xc003bc7b10)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1659 +0xbad
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).tryAddVote(0xc001800000, 0xc003d8dea0, 0xc000cf4cc0, 0x28, 0xf1, 0xf1, 0xf1)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1517 +0x59
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).handleMsg(0xc001800000, 0xd98200, 0xc0070dbed0, 0xc000cf4cc0, 0x28)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:660 +0x64b
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).receiveRoutine(0xc001800000, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:617 +0x670
created by github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).OnStart
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:311 +0x132
```
tx comes in and CheckTx is executed right when we execute the block
```
goroutine 111044 [semacquire, 309 minutes]:
sync.runtime_SemacquireMutex(0xc00003363c, 0x0)
/usr/local/go/src/runtime/sema.go:71 +0x3d
sync.(*Mutex).Lock(0xc000033638)
/usr/local/go/src/sync/mutex.go:134 +0xff
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client.(*localClient).CheckTxAsync(0xc0001fb0e0, 0xc002d94500, 0x13f, 0x280, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client/local_client.go:85 +0x47
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy.(*appConnMempool).CheckTxAsync(0xc00002f720, 0xc002d94500, 0x13f, 0x280, 0x1)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy/app_conn.go:114 +0x51
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/mempool.(*Mempool).CheckTx(0xc002d3a320, 0xc002d94500, 0x13f, 0x280, 0xc0072355f0, 0x0, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/mempool/mempool.go:316 +0x17b
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/core.BroadcastTxSync(0xc002d94500, 0x13f, 0x280, 0x0, 0x0, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/core/mempool.go:93 +0xb8
reflect.Value.call(0xd85560, 0x10326c0, 0x13, 0xec7b8b, 0x4, 0xc00663f180, 0x1, 0x1, 0xc00663f180, 0xc00663f188, ...)
/usr/local/go/src/reflect/value.go:447 +0x449
reflect.Value.Call(0xd85560, 0x10326c0, 0x13, 0xc00663f180, 0x1, 0x1, 0x0, 0x0, 0xc005cc9344)
/usr/local/go/src/reflect/value.go:308 +0xa4
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server.makeHTTPHandler.func2(0x1102060, 0xc00663f100, 0xc0082d7900)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server/handlers.go:269 +0x188
net/http.HandlerFunc.ServeHTTP(0xc002c81f20, 0x1102060, 0xc00663f100, 0xc0082d7900)
/usr/local/go/src/net/http/server.go:1964 +0x44
net/http.(*ServeMux).ServeHTTP(0xc002c81b60, 0x1102060, 0xc00663f100, 0xc0082d7900)
/usr/local/go/src/net/http/server.go:2361 +0x127
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server.maxBytesHandler.ServeHTTP(0x10f8a40, 0xc002c81b60, 0xf4240, 0x1102060, 0xc00663f100, 0xc0082d7900)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server/http_server.go:219 +0xcf
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server.RecoverAndLogHandler.func1(0x1103220, 0xc00121e620, 0xc0082d7900)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server/http_server.go:192 +0x394
net/http.HandlerFunc.ServeHTTP(0xc002c06ea0, 0x1103220, 0xc00121e620, 0xc0082d7900)
/usr/local/go/src/net/http/server.go:1964 +0x44
net/http.serverHandler.ServeHTTP(0xc001a1aa90, 0x1103220, 0xc00121e620, 0xc0082d7900)
/usr/local/go/src/net/http/server.go:2741 +0xab
net/http.(*conn).serve(0xc00785a3c0, 0x11041a0, 0xc000f844c0)
/usr/local/go/src/net/http/server.go:1847 +0x646
created by net/http.(*Server).Serve
/usr/local/go/src/net/http/server.go:2851 +0x2f5
```
* consensus: use read lock in Receive#VoteMessage
* use defer to unlock mutex because application might panic
* use defer in every method of the localClient
* add a changelog entry
* drain channels before Unsubscribe(All)
Read https://github.com/tendermint/tendermint/blob/55362ed76630f3e1ebec159a598f6a9fb5892cb1/libs/pubsub/pubsub.go#L13
for the detailed explanation of the issue.
We'll need to fix it someday. Make sure to keep an eye on
https://github.com/tendermint/tendermint/blob/master/docs/architecture/adr-033-pubsub.md
* retry instead of panic when peer has no state in reactors other than consensus
in /dump_consensus_state RPC endpoint, skip a peer with no state
* rpc/core/mempool: simplify error messages
* rpc/core/mempool: use time.After instead of timer
also, do not log DeliverTx result (to be consistent with other memthods)
* unlock before calling the callback in reqRes#SetCallback
2018-11-13 20:32:51 +04:00
|
|
|
cs.mtx.RUnlock()
|
2015-08-26 18:56:34 -04:00
|
|
|
ps.EnsureVoteBitArrays(height, valSize)
|
|
|
|
ps.EnsureVoteBitArrays(height-1, lastCommitSize)
|
2016-07-01 17:47:31 -04:00
|
|
|
ps.SetHasVote(msg.Vote)
|
2015-08-12 14:00:23 -04:00
|
|
|
|
2018-01-01 21:27:38 -05:00
|
|
|
cs.peerMsgQueue <- msgInfo{msg, src.ID()}
|
2014-08-10 16:35:08 -07:00
|
|
|
|
|
|
|
default:
|
2015-08-26 18:56:34 -04:00
|
|
|
// don't punish (leave room for soft upgrades)
|
2018-08-10 00:25:57 -05:00
|
|
|
conR.Logger.Error(fmt.Sprintf("Unknown message type %v", reflect.TypeOf(msg)))
|
2014-08-10 16:35:08 -07:00
|
|
|
}
|
2016-09-05 17:33:02 -07:00
|
|
|
|
|
|
|
case VoteSetBitsChannel:
|
2017-08-09 14:55:21 -04:00
|
|
|
if conR.FastSync() {
|
2017-05-02 11:53:32 +04:00
|
|
|
conR.Logger.Info("Ignoring message received during fastSync", "msg", msg)
|
2016-09-05 17:33:02 -07:00
|
|
|
return
|
|
|
|
}
|
|
|
|
switch msg := msg.(type) {
|
|
|
|
case *VoteSetBitsMessage:
|
|
|
|
cs := conR.conS
|
|
|
|
cs.mtx.Lock()
|
|
|
|
height, votes := cs.Height, cs.Votes
|
|
|
|
cs.mtx.Unlock()
|
|
|
|
|
|
|
|
if height == msg.Height {
|
2017-07-09 18:38:59 -04:00
|
|
|
var ourVotes *cmn.BitArray
|
2016-09-05 17:33:02 -07:00
|
|
|
switch msg.Type {
|
2018-10-13 01:21:46 +02:00
|
|
|
case types.PrevoteType:
|
2016-09-05 17:33:02 -07:00
|
|
|
ourVotes = votes.Prevotes(msg.Round).BitArrayByBlockID(msg.BlockID)
|
2018-10-13 01:21:46 +02:00
|
|
|
case types.PrecommitType:
|
2016-09-05 17:33:02 -07:00
|
|
|
ourVotes = votes.Precommits(msg.Round).BitArrayByBlockID(msg.BlockID)
|
|
|
|
default:
|
2018-11-01 07:07:18 +01:00
|
|
|
panic("Bad VoteSetBitsMessage field Type. Forgot to add a check in ValidateBasic?")
|
2016-09-05 17:33:02 -07:00
|
|
|
}
|
|
|
|
ps.ApplyVoteSetBitsMessage(msg, ourVotes)
|
|
|
|
} else {
|
|
|
|
ps.ApplyVoteSetBitsMessage(msg, nil)
|
|
|
|
}
|
|
|
|
default:
|
|
|
|
// don't punish (leave room for soft upgrades)
|
2018-08-10 00:25:57 -05:00
|
|
|
conR.Logger.Error(fmt.Sprintf("Unknown message type %v", reflect.TypeOf(msg)))
|
2016-09-05 17:33:02 -07:00
|
|
|
}
|
|
|
|
|
2014-09-14 15:37:32 -07:00
|
|
|
default:
|
2018-08-10 00:25:57 -05:00
|
|
|
conR.Logger.Error(fmt.Sprintf("Unknown chId %X", chID))
|
2014-08-10 16:35:08 -07:00
|
|
|
}
|
|
|
|
|
2014-09-14 15:37:32 -07:00
|
|
|
if err != nil {
|
2017-06-14 12:50:49 +04:00
|
|
|
conR.Logger.Error("Error in Receive()", "err", err)
|
2014-09-14 15:37:32 -07:00
|
|
|
}
|
2014-08-10 16:35:08 -07:00
|
|
|
}
|
|
|
|
|
new pubsub package
comment out failing consensus tests for now
rewrite rpc httpclient to use new pubsub package
import pubsub as tmpubsub, query as tmquery
make event IDs constants
EventKey -> EventTypeKey
rename EventsPubsub to PubSub
mempool does not use pubsub
rename eventsSub to pubsub
new subscribe API
fix channel size issues and consensus tests bugs
refactor rpc client
add missing discardFromChan method
add mutex
rename pubsub to eventBus
remove IsRunning from WSRPCConnection interface (not needed)
add a comment in broadcastNewRoundStepsAndVotes
rename registerEventCallbacks to broadcastNewRoundStepsAndVotes
See https://dave.cheney.net/2014/03/19/channel-axioms
stop eventBuses after reactor tests
remove unnecessary Unsubscribe
return subscribe helper function
move discardFromChan to where it is used
subscribe now returns an err
this gives us ability to refuse to subscribe if pubsub is at its max
capacity.
use context for control overflow
cache queries
handle err when subscribing in replay_test
rename testClientID to testSubscriber
extract var
set channel buffer capacity to 1 in replay_file
fix byzantine_test
unsubscribe from single event, not all events
refactor httpclient to return events to appropriate channels
return failing testReplayCrashBeforeWriteVote test
fix TestValidatorSetChanges
refactor code a bit
fix testReplayCrashBeforeWriteVote
add comment
fix TestValidatorSetChanges
fixes from Bucky's review
update comment [ci skip]
test TxEventBuffer
update changelog
fix TestValidatorSetChanges (2nd attempt)
only do wg.Done when no errors
benchmark event bus
create pubsub server inside NewEventBus
only expose config params (later if needed)
set buffer capacity to 0 so we are not testing cache
new tx event format: key = "Tx" plus a tag {"tx.hash": XYZ}
This should allow to subscribe to all transactions! or a specific one
using a query: "tm.events.type = Tx and tx.hash = '013ABF99434...'"
use TimeoutCommit instead of afterPublishEventNewBlockTimeout
TimeoutCommit is the time a node waits after committing a block, before
it goes into the next height. So it will finish everything from the last
block, but then wait a bit. The idea is this gives it time to hear more
votes from other validators, to strengthen the commit it includes in the
next block. But it also gives it time to hear about new transactions.
waitForBlockWithUpdatedVals
rewrite WAL crash tests
Task:
test that we can recover from any WAL crash.
Solution:
the old tests were relying on event hub being run in the same thread (we
were injecting the private validator's last signature).
when considering a rewrite, we considered two possible solutions: write
a "fuzzy" testing system where WAL is crashing upon receiving a new
message, or inject failures and trigger them in tests using something
like https://github.com/coreos/gofail.
remove sleep
no cs.Lock around wal.Save
test different cases (empty block, non-empty block, ...)
comments
add comments
test 4 cases: empty block, non-empty block, non-empty block with smaller part size, many blocks
fixes as per Bucky's last review
reset subscriptions on UnsubscribeAll
use a simple counter to track message for which we panicked
also, set a smaller part size for all test cases
2017-06-26 19:00:30 +04:00
|
|
|
// SetEventBus sets event bus.
|
|
|
|
func (conR *ConsensusReactor) SetEventBus(b *types.EventBus) {
|
|
|
|
conR.eventBus = b
|
|
|
|
conR.conS.SetEventBus(b)
|
2015-12-13 19:30:15 -05:00
|
|
|
}
|
|
|
|
|
2017-08-09 23:51:09 -04:00
|
|
|
// FastSync returns whether the consensus reactor is in fast-sync mode.
|
2017-07-17 09:44:23 +03:00
|
|
|
func (conR *ConsensusReactor) FastSync() bool {
|
2017-08-09 14:55:21 -04:00
|
|
|
conR.mtx.RLock()
|
|
|
|
defer conR.mtx.RUnlock()
|
2017-07-17 09:44:23 +03:00
|
|
|
return conR.fastSync
|
|
|
|
}
|
|
|
|
|
2015-12-13 19:30:15 -05:00
|
|
|
//--------------------------------------
|
|
|
|
|
2018-11-28 14:52:35 +01:00
|
|
|
// subscribeToBroadcastEvents subscribes for new round steps and votes
|
|
|
|
// using internal pubsub defined on state to broadcast
|
2018-05-16 10:28:58 +04:00
|
|
|
// them to peers upon receiving.
|
|
|
|
func (conR *ConsensusReactor) subscribeToBroadcastEvents() {
|
|
|
|
const subscriber = "consensus-reactor"
|
|
|
|
conR.conS.evsw.AddListenerForEvent(subscriber, types.EventNewRoundStep,
|
|
|
|
func(data tmevents.EventData) {
|
2018-10-31 14:20:36 +01:00
|
|
|
conR.broadcastNewRoundStepMessage(data.(*cstypes.RoundState))
|
|
|
|
})
|
|
|
|
|
|
|
|
conR.conS.evsw.AddListenerForEvent(subscriber, types.EventValidBlock,
|
|
|
|
func(data tmevents.EventData) {
|
|
|
|
conR.broadcastNewValidBlockMessage(data.(*cstypes.RoundState))
|
2018-05-16 10:28:58 +04:00
|
|
|
})
|
|
|
|
|
|
|
|
conR.conS.evsw.AddListenerForEvent(subscriber, types.EventVote,
|
|
|
|
func(data tmevents.EventData) {
|
|
|
|
conR.broadcastHasVoteMessage(data.(*types.Vote))
|
|
|
|
})
|
|
|
|
|
2017-07-20 15:09:44 -04:00
|
|
|
}
|
|
|
|
|
2018-05-16 11:25:53 +04:00
|
|
|
func (conR *ConsensusReactor) unsubscribeFromBroadcastEvents() {
|
|
|
|
const subscriber = "consensus-reactor"
|
|
|
|
conR.conS.evsw.RemoveListener(subscriber)
|
|
|
|
}
|
|
|
|
|
2018-10-31 14:20:36 +01:00
|
|
|
func (conR *ConsensusReactor) broadcastNewRoundStepMessage(rs *cstypes.RoundState) {
|
|
|
|
nrsMsg := makeRoundStepMessage(rs)
|
|
|
|
conR.Switch.Broadcast(StateChannel, cdc.MustMarshalBinaryBare(nrsMsg))
|
|
|
|
}
|
|
|
|
|
|
|
|
func (conR *ConsensusReactor) broadcastNewValidBlockMessage(rs *cstypes.RoundState) {
|
|
|
|
csMsg := &NewValidBlockMessage{
|
|
|
|
Height: rs.Height,
|
|
|
|
Round: rs.Round,
|
|
|
|
BlockPartsHeader: rs.ProposalBlockParts.Header(),
|
|
|
|
BlockParts: rs.ProposalBlockParts.BitArray(),
|
|
|
|
IsCommit: rs.Step == cstypes.RoundStepCommit,
|
2015-12-13 19:30:15 -05:00
|
|
|
}
|
2018-10-31 14:20:36 +01:00
|
|
|
conR.Switch.Broadcast(StateChannel, cdc.MustMarshalBinaryBare(csMsg))
|
2015-12-13 19:30:15 -05:00
|
|
|
}
|
|
|
|
|
2015-06-22 19:04:31 -07:00
|
|
|
// Broadcasts HasVoteMessage to peers that care.
|
2016-07-01 17:47:31 -04:00
|
|
|
func (conR *ConsensusReactor) broadcastHasVoteMessage(vote *types.Vote) {
|
2015-06-22 19:04:31 -07:00
|
|
|
msg := &HasVoteMessage{
|
|
|
|
Height: vote.Height,
|
|
|
|
Round: vote.Round,
|
|
|
|
Type: vote.Type,
|
2016-07-01 17:47:31 -04:00
|
|
|
Index: vote.ValidatorIndex,
|
2015-06-22 19:04:31 -07:00
|
|
|
}
|
2018-04-05 07:05:45 -07:00
|
|
|
conR.Switch.Broadcast(StateChannel, cdc.MustMarshalBinaryBare(msg))
|
2015-06-22 19:04:31 -07:00
|
|
|
/*
|
|
|
|
// TODO: Make this broadcast more selective.
|
2015-07-20 14:40:41 -07:00
|
|
|
for _, peer := range conR.Switch.Peers().List() {
|
abci: localClient improvements & bugfixes & pubsub Unsubscribe issues (#2748)
* use READ lock/unlock in ConsensusState#GetLastHeight
Refs #2721
* do not use defers when there's no need
* fix peer formatting (output its address instead of the pointer)
```
[54310]: E[11-02|11:59:39.851] Connection failed @ sendRoutine module=p2p peer=0xb78f00 conn=MConn{74.207.236.148:26656} err="pong timeout"
```
https://github.com/tendermint/tendermint/issues/2721#issuecomment-435326581
* panic if peer has no state
https://github.com/tendermint/tendermint/issues/2721#issuecomment-435347165
It's confusing that sometimes we check if peer has a state, but most of
the times we expect it to be there
1. https://github.com/tendermint/tendermint/blob/add79700b5fe84417538202b6c927c8cc5383672/mempool/reactor.go#L138
2. https://github.com/tendermint/tendermint/blob/add79700b5fe84417538202b6c927c8cc5383672/rpc/core/consensus.go#L196 (edited)
I will change everything to always assume peer has a state and panic
otherwise
that should help identify issues earlier
* abci/localclient: extend lock on app callback
App callback should be protected by lock as well (note this was already
done for InitChainAsync, why not for others???). Otherwise, when we
execute the block, tx might come in and call the callback in the same
time we're updating it in execBlockOnProxyApp => DATA RACE
Fixes #2721
Consensus state is locked
```
goroutine 113333 [semacquire, 309 minutes]:
sync.runtime_SemacquireMutex(0xc00180009c, 0xc0000c7e00)
/usr/local/go/src/runtime/sema.go:71 +0x3d
sync.(*RWMutex).RLock(0xc001800090)
/usr/local/go/src/sync/rwmutex.go:50 +0x4e
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).GetRoundState(0xc001800000, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:218 +0x46
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusReactor).queryMaj23Routine(0xc0017def80, 0x11104a0, 0xc0072488f0, 0xc007248
9c0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/reactor.go:735 +0x16d
created by github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusReactor).AddPeer
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/reactor.go:172 +0x236
```
because localClient is locked
```
goroutine 1899 [semacquire, 309 minutes]:
sync.runtime_SemacquireMutex(0xc00003363c, 0xc0000cb500)
/usr/local/go/src/runtime/sema.go:71 +0x3d
sync.(*Mutex).Lock(0xc000033638)
/usr/local/go/src/sync/mutex.go:134 +0xff
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client.(*localClient).SetResponseCallback(0xc0001fb560, 0xc007868540)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client/local_client.go:32 +0x33
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy.(*appConnConsensus).SetResponseCallback(0xc00002f750, 0xc007868540)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy/app_conn.go:57 +0x40
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state.execBlockOnProxyApp(0x1104e20, 0xc002ca0ba0, 0x11092a0, 0xc00002f750, 0xc0001fe960, 0xc000bfc660, 0x110cfe0, 0xc000090330, 0xc9d12, 0xc000d9d5a0, ...)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state/execution.go:230 +0x1fd
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state.(*BlockExecutor).ApplyBlock(0xc002c2a230, 0x7, 0x0, 0xc000eae880, 0x6, 0xc002e52c60, 0x16, 0x1f927, 0xc9d12, 0xc000d9d5a0, ...)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state/execution.go:96 +0x142
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).finalizeCommit(0xc001800000, 0x1f928)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1339 +0xa3e
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).tryFinalizeCommit(0xc001800000, 0x1f928)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1270 +0x451
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).enterCommit.func1(0xc001800000, 0x0, 0x1f928)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1218 +0x90
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).enterCommit(0xc001800000, 0x1f928, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1247 +0x6b8
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).addVote(0xc001800000, 0xc003d8dea0, 0xc000cf4cc0, 0x28, 0xf1, 0xc003bc7ad0, 0xc003bc7b10)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1659 +0xbad
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).tryAddVote(0xc001800000, 0xc003d8dea0, 0xc000cf4cc0, 0x28, 0xf1, 0xf1, 0xf1)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1517 +0x59
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).handleMsg(0xc001800000, 0xd98200, 0xc0070dbed0, 0xc000cf4cc0, 0x28)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:660 +0x64b
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).receiveRoutine(0xc001800000, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:617 +0x670
created by github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).OnStart
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:311 +0x132
```
tx comes in and CheckTx is executed right when we execute the block
```
goroutine 111044 [semacquire, 309 minutes]:
sync.runtime_SemacquireMutex(0xc00003363c, 0x0)
/usr/local/go/src/runtime/sema.go:71 +0x3d
sync.(*Mutex).Lock(0xc000033638)
/usr/local/go/src/sync/mutex.go:134 +0xff
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client.(*localClient).CheckTxAsync(0xc0001fb0e0, 0xc002d94500, 0x13f, 0x280, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client/local_client.go:85 +0x47
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy.(*appConnMempool).CheckTxAsync(0xc00002f720, 0xc002d94500, 0x13f, 0x280, 0x1)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy/app_conn.go:114 +0x51
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/mempool.(*Mempool).CheckTx(0xc002d3a320, 0xc002d94500, 0x13f, 0x280, 0xc0072355f0, 0x0, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/mempool/mempool.go:316 +0x17b
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/core.BroadcastTxSync(0xc002d94500, 0x13f, 0x280, 0x0, 0x0, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/core/mempool.go:93 +0xb8
reflect.Value.call(0xd85560, 0x10326c0, 0x13, 0xec7b8b, 0x4, 0xc00663f180, 0x1, 0x1, 0xc00663f180, 0xc00663f188, ...)
/usr/local/go/src/reflect/value.go:447 +0x449
reflect.Value.Call(0xd85560, 0x10326c0, 0x13, 0xc00663f180, 0x1, 0x1, 0x0, 0x0, 0xc005cc9344)
/usr/local/go/src/reflect/value.go:308 +0xa4
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server.makeHTTPHandler.func2(0x1102060, 0xc00663f100, 0xc0082d7900)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server/handlers.go:269 +0x188
net/http.HandlerFunc.ServeHTTP(0xc002c81f20, 0x1102060, 0xc00663f100, 0xc0082d7900)
/usr/local/go/src/net/http/server.go:1964 +0x44
net/http.(*ServeMux).ServeHTTP(0xc002c81b60, 0x1102060, 0xc00663f100, 0xc0082d7900)
/usr/local/go/src/net/http/server.go:2361 +0x127
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server.maxBytesHandler.ServeHTTP(0x10f8a40, 0xc002c81b60, 0xf4240, 0x1102060, 0xc00663f100, 0xc0082d7900)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server/http_server.go:219 +0xcf
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server.RecoverAndLogHandler.func1(0x1103220, 0xc00121e620, 0xc0082d7900)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server/http_server.go:192 +0x394
net/http.HandlerFunc.ServeHTTP(0xc002c06ea0, 0x1103220, 0xc00121e620, 0xc0082d7900)
/usr/local/go/src/net/http/server.go:1964 +0x44
net/http.serverHandler.ServeHTTP(0xc001a1aa90, 0x1103220, 0xc00121e620, 0xc0082d7900)
/usr/local/go/src/net/http/server.go:2741 +0xab
net/http.(*conn).serve(0xc00785a3c0, 0x11041a0, 0xc000f844c0)
/usr/local/go/src/net/http/server.go:1847 +0x646
created by net/http.(*Server).Serve
/usr/local/go/src/net/http/server.go:2851 +0x2f5
```
* consensus: use read lock in Receive#VoteMessage
* use defer to unlock mutex because application might panic
* use defer in every method of the localClient
* add a changelog entry
* drain channels before Unsubscribe(All)
Read https://github.com/tendermint/tendermint/blob/55362ed76630f3e1ebec159a598f6a9fb5892cb1/libs/pubsub/pubsub.go#L13
for the detailed explanation of the issue.
We'll need to fix it someday. Make sure to keep an eye on
https://github.com/tendermint/tendermint/blob/master/docs/architecture/adr-033-pubsub.md
* retry instead of panic when peer has no state in reactors other than consensus
in /dump_consensus_state RPC endpoint, skip a peer with no state
* rpc/core/mempool: simplify error messages
* rpc/core/mempool: use time.After instead of timer
also, do not log DeliverTx result (to be consistent with other memthods)
* unlock before calling the callback in reqRes#SetCallback
2018-11-13 20:32:51 +04:00
|
|
|
ps, ok := peer.Get(PeerStateKey).(*PeerState)
|
|
|
|
if !ok {
|
|
|
|
panic(fmt.Sprintf("Peer %v has no state", peer))
|
|
|
|
}
|
2015-06-22 19:04:31 -07:00
|
|
|
prs := ps.GetRoundState()
|
|
|
|
if prs.Height == vote.Height {
|
|
|
|
// TODO: Also filter on round?
|
2015-12-21 14:48:44 -08:00
|
|
|
peer.TrySend(StateChannel, struct{ ConsensusMessage }{msg})
|
2015-06-22 19:04:31 -07:00
|
|
|
} else {
|
|
|
|
// Height doesn't match
|
|
|
|
// TODO: check a field, maybe CatchupCommitRound?
|
|
|
|
// TODO: But that requires changing the struct field comment.
|
|
|
|
}
|
|
|
|
}
|
|
|
|
*/
|
|
|
|
}
|
|
|
|
|
2018-10-31 14:20:36 +01:00
|
|
|
func makeRoundStepMessage(rs *cstypes.RoundState) (nrsMsg *NewRoundStepMessage) {
|
2015-01-13 17:52:27 -08:00
|
|
|
nrsMsg = &NewRoundStepMessage{
|
2019-01-28 14:13:17 +02:00
|
|
|
Height: rs.Height,
|
|
|
|
Round: rs.Round,
|
|
|
|
Step: rs.Step,
|
2017-05-29 23:11:40 -04:00
|
|
|
SecondsSinceStartTime: int(time.Since(rs.StartTime).Seconds()),
|
2015-06-19 15:30:10 -07:00
|
|
|
LastCommitRound: rs.LastCommit.Round(),
|
2015-01-13 17:52:27 -08:00
|
|
|
}
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2018-10-31 14:20:36 +01:00
|
|
|
func (conR *ConsensusReactor) sendNewRoundStepMessage(peer p2p.Peer) {
|
2015-01-13 17:52:27 -08:00
|
|
|
rs := conR.conS.GetRoundState()
|
2018-10-31 14:20:36 +01:00
|
|
|
nrsMsg := makeRoundStepMessage(rs)
|
|
|
|
peer.Send(StateChannel, cdc.MustMarshalBinaryBare(nrsMsg))
|
2015-01-13 17:52:27 -08:00
|
|
|
}
|
|
|
|
|
2017-09-12 20:49:22 -04:00
|
|
|
func (conR *ConsensusReactor) gossipDataRoutine(peer p2p.Peer, ps *PeerState) {
|
2017-05-02 11:53:32 +04:00
|
|
|
logger := conR.Logger.With("peer", peer)
|
2014-09-14 15:37:32 -07:00
|
|
|
|
2014-08-10 16:35:08 -07:00
|
|
|
OUTER_LOOP:
|
|
|
|
for {
|
2014-09-14 15:37:32 -07:00
|
|
|
// Manage disconnects from self or peer.
|
2015-03-25 13:01:28 -07:00
|
|
|
if !peer.IsRunning() || !conR.IsRunning() {
|
2017-05-02 11:53:32 +04:00
|
|
|
logger.Info("Stopping gossipDataRoutine for peer")
|
2014-09-14 15:37:32 -07:00
|
|
|
return
|
2014-08-10 16:35:08 -07:00
|
|
|
}
|
2014-09-14 15:37:32 -07:00
|
|
|
rs := conR.conS.GetRoundState()
|
|
|
|
prs := ps.GetRoundState()
|
2014-08-10 16:35:08 -07:00
|
|
|
|
2014-10-26 13:26:27 -07:00
|
|
|
// Send proposal Block parts?
|
2015-06-22 19:04:31 -07:00
|
|
|
if rs.ProposalBlockParts.HasHeader(prs.ProposalBlockPartsHeader) {
|
|
|
|
if index, ok := rs.ProposalBlockParts.BitArray().Sub(prs.ProposalBlockParts.Copy()).PickRandom(); ok {
|
2014-12-30 21:11:06 -08:00
|
|
|
part := rs.ProposalBlockParts.GetPart(index)
|
2015-06-22 19:04:31 -07:00
|
|
|
msg := &BlockPartMessage{
|
2015-06-25 14:05:18 -07:00
|
|
|
Height: rs.Height, // This tells peer that this part applies to us.
|
|
|
|
Round: rs.Round, // This tells peer that this part applies to us.
|
2014-12-30 21:11:06 -08:00
|
|
|
Part: part,
|
2014-10-21 18:30:03 -07:00
|
|
|
}
|
2017-07-07 16:58:16 -04:00
|
|
|
logger.Debug("Sending block part", "height", prs.Height, "round", prs.Round)
|
2018-04-05 07:05:45 -07:00
|
|
|
if peer.Send(DataChannel, cdc.MustMarshalBinaryBare(msg)) {
|
2017-01-17 20:58:27 +04:00
|
|
|
ps.SetHasProposalBlockPart(prs.Height, prs.Round, index)
|
|
|
|
}
|
2014-10-21 18:30:03 -07:00
|
|
|
continue OUTER_LOOP
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2014-12-30 21:11:06 -08:00
|
|
|
// If the peer is on a previous height, help catch up.
|
2015-06-25 12:52:16 -07:00
|
|
|
if (0 < prs.Height) && (prs.Height < rs.Height) {
|
2017-07-09 18:38:59 -04:00
|
|
|
heightLogger := logger.With("height", prs.Height)
|
2017-11-09 18:14:41 +00:00
|
|
|
|
|
|
|
// if we never received the commit message from the peer, the block parts wont be initialized
|
|
|
|
if prs.ProposalBlockParts == nil {
|
|
|
|
blockMeta := conR.conS.blockStore.LoadBlockMeta(prs.Height)
|
|
|
|
if blockMeta == nil {
|
2019-04-26 06:23:43 -04:00
|
|
|
panic(fmt.Sprintf("Failed to load block %d when blockStore is at %d",
|
2017-11-09 18:14:41 +00:00
|
|
|
prs.Height, conR.conS.blockStore.Height()))
|
|
|
|
}
|
|
|
|
ps.InitProposalBlockParts(blockMeta.BlockID.PartsHeader)
|
|
|
|
// continue the loop since prs is a copy and not effected by this initialization
|
|
|
|
continue OUTER_LOOP
|
|
|
|
}
|
2017-07-09 18:38:59 -04:00
|
|
|
conR.gossipDataForCatchup(heightLogger, rs, prs, ps, peer)
|
|
|
|
continue OUTER_LOOP
|
2014-12-30 21:11:06 -08:00
|
|
|
}
|
|
|
|
|
2015-01-08 22:07:23 -08:00
|
|
|
// If height and round don't match, sleep.
|
2015-06-25 14:05:18 -07:00
|
|
|
if (rs.Height != prs.Height) || (rs.Round != prs.Round) {
|
2017-05-02 11:53:32 +04:00
|
|
|
//logger.Info("Peer Height|Round mismatch, sleeping", "peerHeight", prs.Height, "peerRound", prs.Round, "peer", peer)
|
2018-09-26 14:04:44 +04:00
|
|
|
time.Sleep(conR.conS.config.PeerGossipSleepDuration)
|
2014-08-10 16:35:08 -07:00
|
|
|
continue OUTER_LOOP
|
|
|
|
}
|
|
|
|
|
2015-06-22 19:04:31 -07:00
|
|
|
// By here, height and round match.
|
2015-06-25 14:05:18 -07:00
|
|
|
// Proposal block parts were already matched and sent if any were wanted.
|
|
|
|
// (These can match on hash so the round doesn't matter)
|
|
|
|
// Now consider sending other things, like the Proposal itself.
|
2015-06-22 19:04:31 -07:00
|
|
|
|
|
|
|
// Send Proposal && ProposalPOL BitArray?
|
2014-09-14 15:37:32 -07:00
|
|
|
if rs.Proposal != nil && !prs.Proposal {
|
2016-08-20 15:08:26 -07:00
|
|
|
// Proposal: share the proposal metadata with peer.
|
2015-06-22 19:04:31 -07:00
|
|
|
{
|
|
|
|
msg := &ProposalMessage{Proposal: rs.Proposal}
|
2017-07-07 16:58:16 -04:00
|
|
|
logger.Debug("Sending proposal", "height", prs.Height, "round", prs.Round)
|
2018-04-05 07:05:45 -07:00
|
|
|
if peer.Send(DataChannel, cdc.MustMarshalBinaryBare(msg)) {
|
2018-10-31 14:20:36 +01:00
|
|
|
// NOTE[ZM]: A peer might have received different proposal msg so this Proposal msg will be rejected!
|
2017-01-17 20:58:27 +04:00
|
|
|
ps.SetHasProposal(rs.Proposal)
|
|
|
|
}
|
2015-06-22 19:04:31 -07:00
|
|
|
}
|
2016-08-20 15:08:26 -07:00
|
|
|
// ProposalPOL: lets peer know which POL votes we have so far.
|
2015-06-25 14:05:18 -07:00
|
|
|
// Peer must receive ProposalMessage first.
|
|
|
|
// rs.Proposal was validated, so rs.Proposal.POLRound <= rs.Round,
|
|
|
|
// so we definitely have rs.Votes.Prevotes(rs.Proposal.POLRound).
|
2015-06-22 19:04:31 -07:00
|
|
|
if 0 <= rs.Proposal.POLRound {
|
|
|
|
msg := &ProposalPOLMessage{
|
|
|
|
Height: rs.Height,
|
2015-06-25 20:28:34 -07:00
|
|
|
ProposalPOLRound: rs.Proposal.POLRound,
|
|
|
|
ProposalPOL: rs.Votes.Prevotes(rs.Proposal.POLRound).BitArray(),
|
2015-06-22 19:04:31 -07:00
|
|
|
}
|
2017-07-07 16:58:16 -04:00
|
|
|
logger.Debug("Sending POL", "height", prs.Height, "round", prs.Round)
|
2018-04-05 07:05:45 -07:00
|
|
|
peer.Send(DataChannel, cdc.MustMarshalBinaryBare(msg))
|
2015-06-22 19:04:31 -07:00
|
|
|
}
|
2014-09-14 15:37:32 -07:00
|
|
|
continue OUTER_LOOP
|
2014-08-10 16:35:08 -07:00
|
|
|
}
|
2014-09-03 20:41:57 -07:00
|
|
|
|
2014-09-14 15:37:32 -07:00
|
|
|
// Nothing to do. Sleep.
|
2018-09-26 14:04:44 +04:00
|
|
|
time.Sleep(conR.conS.config.PeerGossipSleepDuration)
|
2014-09-14 15:37:32 -07:00
|
|
|
continue OUTER_LOOP
|
2014-08-10 16:35:08 -07:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-10-10 12:39:21 +04:00
|
|
|
func (conR *ConsensusReactor) gossipDataForCatchup(logger log.Logger, rs *cstypes.RoundState,
|
|
|
|
prs *cstypes.PeerRoundState, ps *PeerState, peer p2p.Peer) {
|
2017-07-09 18:38:59 -04:00
|
|
|
|
|
|
|
if index, ok := prs.ProposalBlockParts.Not().PickRandom(); ok {
|
|
|
|
// Ensure that the peer's PartSetHeader is correct
|
|
|
|
blockMeta := conR.conS.blockStore.LoadBlockMeta(prs.Height)
|
|
|
|
if blockMeta == nil {
|
|
|
|
logger.Error("Failed to load block meta",
|
|
|
|
"ourHeight", rs.Height, "blockstoreHeight", conR.conS.blockStore.Height())
|
2018-09-26 14:04:44 +04:00
|
|
|
time.Sleep(conR.conS.config.PeerGossipSleepDuration)
|
2017-07-09 18:38:59 -04:00
|
|
|
return
|
|
|
|
} else if !blockMeta.BlockID.PartsHeader.Equals(prs.ProposalBlockPartsHeader) {
|
|
|
|
logger.Info("Peer ProposalBlockPartsHeader mismatch, sleeping",
|
|
|
|
"blockPartsHeader", blockMeta.BlockID.PartsHeader, "peerBlockPartsHeader", prs.ProposalBlockPartsHeader)
|
2018-09-26 14:04:44 +04:00
|
|
|
time.Sleep(conR.conS.config.PeerGossipSleepDuration)
|
2017-07-09 18:38:59 -04:00
|
|
|
return
|
|
|
|
}
|
|
|
|
// Load the part
|
|
|
|
part := conR.conS.blockStore.LoadBlockPart(prs.Height, index)
|
|
|
|
if part == nil {
|
|
|
|
logger.Error("Could not load part", "index", index,
|
|
|
|
"blockPartsHeader", blockMeta.BlockID.PartsHeader, "peerBlockPartsHeader", prs.ProposalBlockPartsHeader)
|
2018-09-26 14:04:44 +04:00
|
|
|
time.Sleep(conR.conS.config.PeerGossipSleepDuration)
|
2017-07-09 18:38:59 -04:00
|
|
|
return
|
|
|
|
}
|
|
|
|
// Send the part
|
|
|
|
msg := &BlockPartMessage{
|
|
|
|
Height: prs.Height, // Not our height, so it doesn't matter.
|
|
|
|
Round: prs.Round, // Not our height, so it doesn't matter.
|
|
|
|
Part: part,
|
|
|
|
}
|
2017-11-09 13:37:29 -05:00
|
|
|
logger.Debug("Sending block part for catchup", "round", prs.Round, "index", index)
|
2018-04-05 07:05:45 -07:00
|
|
|
if peer.Send(DataChannel, cdc.MustMarshalBinaryBare(msg)) {
|
2017-07-09 18:38:59 -04:00
|
|
|
ps.SetHasProposalBlockPart(prs.Height, prs.Round, index)
|
2017-11-09 18:14:41 +00:00
|
|
|
} else {
|
|
|
|
logger.Debug("Sending block part for catchup failed")
|
2017-07-09 18:38:59 -04:00
|
|
|
}
|
|
|
|
return
|
|
|
|
}
|
2018-04-02 10:21:17 +02:00
|
|
|
//logger.Info("No parts to send in catch-up, sleeping")
|
2018-09-26 14:04:44 +04:00
|
|
|
time.Sleep(conR.conS.config.PeerGossipSleepDuration)
|
2017-07-09 18:38:59 -04:00
|
|
|
}
|
|
|
|
|
2017-09-12 20:49:22 -04:00
|
|
|
func (conR *ConsensusReactor) gossipVotesRoutine(peer p2p.Peer, ps *PeerState) {
|
2017-05-02 11:53:32 +04:00
|
|
|
logger := conR.Logger.With("peer", peer)
|
2015-05-04 22:21:07 -07:00
|
|
|
|
|
|
|
// Simple hack to throttle logs upon sleep.
|
|
|
|
var sleeping = 0
|
|
|
|
|
2014-09-14 15:37:32 -07:00
|
|
|
OUTER_LOOP:
|
|
|
|
for {
|
|
|
|
// Manage disconnects from self or peer.
|
2015-03-25 13:01:28 -07:00
|
|
|
if !peer.IsRunning() || !conR.IsRunning() {
|
2017-05-02 11:53:32 +04:00
|
|
|
logger.Info("Stopping gossipVotesRoutine for peer")
|
2014-09-14 15:37:32 -07:00
|
|
|
return
|
2014-08-10 16:35:08 -07:00
|
|
|
}
|
2014-09-14 15:37:32 -07:00
|
|
|
rs := conR.conS.GetRoundState()
|
|
|
|
prs := ps.GetRoundState()
|
|
|
|
|
2015-05-04 22:21:07 -07:00
|
|
|
switch sleeping {
|
|
|
|
case 1: // First sleep
|
|
|
|
sleeping = 2
|
|
|
|
case 2: // No more sleep
|
|
|
|
sleeping = 0
|
|
|
|
}
|
|
|
|
|
2017-05-02 11:53:32 +04:00
|
|
|
//logger.Debug("gossipVotesRoutine", "rsHeight", rs.Height, "rsRound", rs.Round,
|
2015-12-09 11:54:08 -08:00
|
|
|
// "prsHeight", prs.Height, "prsRound", prs.Round, "prsStep", prs.Step)
|
2015-07-05 15:35:26 -07:00
|
|
|
|
2015-06-19 15:30:10 -07:00
|
|
|
// If height matches, then send LastCommit, Prevotes, Precommits.
|
2014-11-01 22:42:04 -07:00
|
|
|
if rs.Height == prs.Height {
|
2017-07-09 18:38:59 -04:00
|
|
|
heightLogger := logger.With("height", prs.Height)
|
|
|
|
if conR.gossipVotesForHeight(heightLogger, rs, prs, ps) {
|
|
|
|
continue OUTER_LOOP
|
2015-06-22 19:04:31 -07:00
|
|
|
}
|
2014-09-14 15:37:32 -07:00
|
|
|
}
|
|
|
|
|
2015-06-19 15:30:10 -07:00
|
|
|
// Special catchup logic.
|
|
|
|
// If peer is lagging by height 1, send LastCommit.
|
2015-07-05 15:35:26 -07:00
|
|
|
if prs.Height != 0 && rs.Height == prs.Height+1 {
|
2015-07-05 13:40:59 -07:00
|
|
|
if ps.PickSendVote(rs.LastCommit) {
|
2017-07-07 16:58:16 -04:00
|
|
|
logger.Debug("Picked rs.LastCommit to send", "height", prs.Height)
|
2015-07-05 13:40:59 -07:00
|
|
|
continue OUTER_LOOP
|
2014-11-01 22:42:04 -07:00
|
|
|
}
|
2014-09-14 15:37:32 -07:00
|
|
|
}
|
2014-08-10 16:35:08 -07:00
|
|
|
|
2015-06-19 15:30:10 -07:00
|
|
|
// Catchup logic
|
2016-04-02 09:10:16 -07:00
|
|
|
// If peer is lagging by more than 1, send Commit.
|
2015-07-05 20:26:02 -07:00
|
|
|
if prs.Height != 0 && rs.Height >= prs.Height+2 {
|
2016-04-02 09:10:16 -07:00
|
|
|
// Load the block commit for prs.Height,
|
2015-06-19 15:30:10 -07:00
|
|
|
// which contains precommit signatures for prs.Height.
|
2016-09-15 16:01:01 -07:00
|
|
|
commit := conR.conS.blockStore.LoadBlockCommit(prs.Height)
|
2016-04-02 09:10:16 -07:00
|
|
|
if ps.PickSendVote(commit) {
|
2017-07-07 16:58:16 -04:00
|
|
|
logger.Debug("Picked Catchup commit to send", "height", prs.Height)
|
2015-06-19 15:30:10 -07:00
|
|
|
continue OUTER_LOOP
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2015-05-04 22:21:07 -07:00
|
|
|
if sleeping == 0 {
|
|
|
|
// We sent nothing. Sleep...
|
|
|
|
sleeping = 1
|
2017-07-07 16:58:16 -04:00
|
|
|
logger.Debug("No votes to send, sleeping", "rs.Height", rs.Height, "prs.Height", prs.Height,
|
2015-06-22 19:04:31 -07:00
|
|
|
"localPV", rs.Votes.Prevotes(rs.Round).BitArray(), "peerPV", prs.Prevotes,
|
|
|
|
"localPC", rs.Votes.Precommits(rs.Round).BitArray(), "peerPC", prs.Precommits)
|
2015-05-04 22:21:07 -07:00
|
|
|
} else if sleeping == 2 {
|
|
|
|
// Continued sleep...
|
|
|
|
sleeping = 1
|
|
|
|
}
|
|
|
|
|
2018-09-26 14:04:44 +04:00
|
|
|
time.Sleep(conR.conS.config.PeerGossipSleepDuration)
|
2014-09-14 15:37:32 -07:00
|
|
|
continue OUTER_LOOP
|
2014-08-10 16:35:08 -07:00
|
|
|
}
|
2014-09-14 15:37:32 -07:00
|
|
|
}
|
2014-08-10 16:35:08 -07:00
|
|
|
|
2017-10-10 12:39:21 +04:00
|
|
|
func (conR *ConsensusReactor) gossipVotesForHeight(logger log.Logger, rs *cstypes.RoundState, prs *cstypes.PeerRoundState, ps *PeerState) bool {
|
2017-07-09 18:38:59 -04:00
|
|
|
|
|
|
|
// If there are lastCommits to send...
|
2017-10-10 12:39:21 +04:00
|
|
|
if prs.Step == cstypes.RoundStepNewHeight {
|
2017-07-09 18:38:59 -04:00
|
|
|
if ps.PickSendVote(rs.LastCommit) {
|
|
|
|
logger.Debug("Picked rs.LastCommit to send")
|
|
|
|
return true
|
|
|
|
}
|
|
|
|
}
|
2018-05-14 13:45:26 +02:00
|
|
|
// If there are POL prevotes to send...
|
|
|
|
if prs.Step <= cstypes.RoundStepPropose && prs.Round != -1 && prs.Round <= rs.Round && prs.ProposalPOLRound != -1 {
|
|
|
|
if polPrevotes := rs.Votes.Prevotes(prs.ProposalPOLRound); polPrevotes != nil {
|
|
|
|
if ps.PickSendVote(polPrevotes) {
|
|
|
|
logger.Debug("Picked rs.Prevotes(prs.ProposalPOLRound) to send",
|
|
|
|
"round", prs.ProposalPOLRound)
|
|
|
|
return true
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2018-05-12 16:55:08 -07:00
|
|
|
// If there are prevotes to send...
|
|
|
|
if prs.Step <= cstypes.RoundStepPrevoteWait && prs.Round != -1 && prs.Round <= rs.Round {
|
2017-07-09 18:38:59 -04:00
|
|
|
if ps.PickSendVote(rs.Votes.Prevotes(prs.Round)) {
|
|
|
|
logger.Debug("Picked rs.Prevotes(prs.Round) to send", "round", prs.Round)
|
|
|
|
return true
|
|
|
|
}
|
2018-05-12 16:55:08 -07:00
|
|
|
}
|
|
|
|
// If there are precommits to send...
|
|
|
|
if prs.Step <= cstypes.RoundStepPrecommitWait && prs.Round != -1 && prs.Round <= rs.Round {
|
2017-07-09 18:38:59 -04:00
|
|
|
if ps.PickSendVote(rs.Votes.Precommits(prs.Round)) {
|
|
|
|
logger.Debug("Picked rs.Precommits(prs.Round) to send", "round", prs.Round)
|
|
|
|
return true
|
|
|
|
}
|
|
|
|
}
|
2018-05-14 13:45:26 +02:00
|
|
|
// If there are prevotes to send...Needed because of validBlock mechanism
|
|
|
|
if prs.Round != -1 && prs.Round <= rs.Round {
|
|
|
|
if ps.PickSendVote(rs.Votes.Prevotes(prs.Round)) {
|
|
|
|
logger.Debug("Picked rs.Prevotes(prs.Round) to send", "round", prs.Round)
|
|
|
|
return true
|
|
|
|
}
|
|
|
|
}
|
2018-05-12 16:55:08 -07:00
|
|
|
// If there are POLPrevotes to send...
|
|
|
|
if prs.ProposalPOLRound != -1 {
|
|
|
|
if polPrevotes := rs.Votes.Prevotes(prs.ProposalPOLRound); polPrevotes != nil {
|
|
|
|
if ps.PickSendVote(polPrevotes) {
|
|
|
|
logger.Debug("Picked rs.Prevotes(prs.ProposalPOLRound) to send",
|
|
|
|
"round", prs.ProposalPOLRound)
|
|
|
|
return true
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2018-04-19 10:55:30 +02:00
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
return false
|
|
|
|
}
|
|
|
|
|
2017-01-11 08:57:10 -08:00
|
|
|
// NOTE: `queryMaj23Routine` has a simple crude design since it only comes
|
|
|
|
// into play for liveness when there's a signature DDoS attack happening.
|
2017-09-12 20:49:22 -04:00
|
|
|
func (conR *ConsensusReactor) queryMaj23Routine(peer p2p.Peer, ps *PeerState) {
|
2017-05-02 11:53:32 +04:00
|
|
|
logger := conR.Logger.With("peer", peer)
|
2016-09-05 17:33:02 -07:00
|
|
|
|
|
|
|
OUTER_LOOP:
|
|
|
|
for {
|
|
|
|
// Manage disconnects from self or peer.
|
|
|
|
if !peer.IsRunning() || !conR.IsRunning() {
|
2017-05-02 11:53:32 +04:00
|
|
|
logger.Info("Stopping queryMaj23Routine for peer")
|
2016-09-05 17:33:02 -07:00
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
// Maybe send Height/Round/Prevotes
|
|
|
|
{
|
|
|
|
rs := conR.conS.GetRoundState()
|
|
|
|
prs := ps.GetRoundState()
|
|
|
|
if rs.Height == prs.Height {
|
|
|
|
if maj23, ok := rs.Votes.Prevotes(prs.Round).TwoThirdsMajority(); ok {
|
2018-04-05 07:05:45 -07:00
|
|
|
peer.TrySend(StateChannel, cdc.MustMarshalBinaryBare(&VoteSetMaj23Message{
|
2016-09-05 17:33:02 -07:00
|
|
|
Height: prs.Height,
|
|
|
|
Round: prs.Round,
|
2018-10-13 01:21:46 +02:00
|
|
|
Type: types.PrevoteType,
|
2016-09-05 17:33:02 -07:00
|
|
|
BlockID: maj23,
|
2018-04-05 07:05:45 -07:00
|
|
|
}))
|
2018-09-26 14:04:44 +04:00
|
|
|
time.Sleep(conR.conS.config.PeerQueryMaj23SleepDuration)
|
2016-09-05 17:33:02 -07:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Maybe send Height/Round/Precommits
|
|
|
|
{
|
|
|
|
rs := conR.conS.GetRoundState()
|
|
|
|
prs := ps.GetRoundState()
|
|
|
|
if rs.Height == prs.Height {
|
|
|
|
if maj23, ok := rs.Votes.Precommits(prs.Round).TwoThirdsMajority(); ok {
|
2018-04-05 07:05:45 -07:00
|
|
|
peer.TrySend(StateChannel, cdc.MustMarshalBinaryBare(&VoteSetMaj23Message{
|
2016-09-05 17:33:02 -07:00
|
|
|
Height: prs.Height,
|
|
|
|
Round: prs.Round,
|
2018-10-13 01:21:46 +02:00
|
|
|
Type: types.PrecommitType,
|
2016-09-05 17:33:02 -07:00
|
|
|
BlockID: maj23,
|
2018-04-05 07:05:45 -07:00
|
|
|
}))
|
2018-09-26 14:04:44 +04:00
|
|
|
time.Sleep(conR.conS.config.PeerQueryMaj23SleepDuration)
|
2016-09-05 17:33:02 -07:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Maybe send Height/Round/ProposalPOL
|
|
|
|
{
|
|
|
|
rs := conR.conS.GetRoundState()
|
|
|
|
prs := ps.GetRoundState()
|
2016-09-13 16:24:31 -04:00
|
|
|
if rs.Height == prs.Height && prs.ProposalPOLRound >= 0 {
|
2016-09-05 17:33:02 -07:00
|
|
|
if maj23, ok := rs.Votes.Prevotes(prs.ProposalPOLRound).TwoThirdsMajority(); ok {
|
2018-04-05 07:05:45 -07:00
|
|
|
peer.TrySend(StateChannel, cdc.MustMarshalBinaryBare(&VoteSetMaj23Message{
|
2016-09-05 17:33:02 -07:00
|
|
|
Height: prs.Height,
|
|
|
|
Round: prs.ProposalPOLRound,
|
2018-10-13 01:21:46 +02:00
|
|
|
Type: types.PrevoteType,
|
2016-09-05 17:33:02 -07:00
|
|
|
BlockID: maj23,
|
2018-04-05 07:05:45 -07:00
|
|
|
}))
|
2018-09-26 14:04:44 +04:00
|
|
|
time.Sleep(conR.conS.config.PeerQueryMaj23SleepDuration)
|
2016-09-05 17:33:02 -07:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Little point sending LastCommitRound/LastCommit,
|
|
|
|
// These are fleeting and non-blocking.
|
|
|
|
|
|
|
|
// Maybe send Height/CatchupCommitRound/CatchupCommit.
|
|
|
|
{
|
|
|
|
prs := ps.GetRoundState()
|
2016-09-15 17:55:07 -07:00
|
|
|
if prs.CatchupCommitRound != -1 && 0 < prs.Height && prs.Height <= conR.conS.blockStore.Height() {
|
2016-11-16 16:47:31 -05:00
|
|
|
commit := conR.conS.LoadCommit(prs.Height)
|
2018-04-05 07:05:45 -07:00
|
|
|
peer.TrySend(StateChannel, cdc.MustMarshalBinaryBare(&VoteSetMaj23Message{
|
2016-09-05 17:33:02 -07:00
|
|
|
Height: prs.Height,
|
|
|
|
Round: commit.Round(),
|
2018-10-13 01:21:46 +02:00
|
|
|
Type: types.PrecommitType,
|
2016-09-05 17:33:02 -07:00
|
|
|
BlockID: commit.BlockID,
|
2018-04-05 07:05:45 -07:00
|
|
|
}))
|
2018-09-26 14:04:44 +04:00
|
|
|
time.Sleep(conR.conS.config.PeerQueryMaj23SleepDuration)
|
2016-09-05 17:33:02 -07:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-09-26 14:04:44 +04:00
|
|
|
time.Sleep(conR.conS.config.PeerQueryMaj23SleepDuration)
|
2016-09-15 16:01:01 -07:00
|
|
|
|
2016-09-05 17:33:02 -07:00
|
|
|
continue OUTER_LOOP
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-09-21 20:36:48 +02:00
|
|
|
func (conR *ConsensusReactor) peerStatsRoutine() {
|
|
|
|
for {
|
|
|
|
if !conR.IsRunning() {
|
|
|
|
conR.Logger.Info("Stopping peerStatsRoutine")
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
select {
|
|
|
|
case msg := <-conR.conS.statsMsgQueue:
|
|
|
|
// Get peer
|
|
|
|
peer := conR.Switch.Peers().Get(msg.PeerID)
|
|
|
|
if peer == nil {
|
|
|
|
conR.Logger.Debug("Attempt to update stats for non-existent peer",
|
|
|
|
"peer", msg.PeerID)
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
// Get peer state
|
abci: localClient improvements & bugfixes & pubsub Unsubscribe issues (#2748)
* use READ lock/unlock in ConsensusState#GetLastHeight
Refs #2721
* do not use defers when there's no need
* fix peer formatting (output its address instead of the pointer)
```
[54310]: E[11-02|11:59:39.851] Connection failed @ sendRoutine module=p2p peer=0xb78f00 conn=MConn{74.207.236.148:26656} err="pong timeout"
```
https://github.com/tendermint/tendermint/issues/2721#issuecomment-435326581
* panic if peer has no state
https://github.com/tendermint/tendermint/issues/2721#issuecomment-435347165
It's confusing that sometimes we check if peer has a state, but most of
the times we expect it to be there
1. https://github.com/tendermint/tendermint/blob/add79700b5fe84417538202b6c927c8cc5383672/mempool/reactor.go#L138
2. https://github.com/tendermint/tendermint/blob/add79700b5fe84417538202b6c927c8cc5383672/rpc/core/consensus.go#L196 (edited)
I will change everything to always assume peer has a state and panic
otherwise
that should help identify issues earlier
* abci/localclient: extend lock on app callback
App callback should be protected by lock as well (note this was already
done for InitChainAsync, why not for others???). Otherwise, when we
execute the block, tx might come in and call the callback in the same
time we're updating it in execBlockOnProxyApp => DATA RACE
Fixes #2721
Consensus state is locked
```
goroutine 113333 [semacquire, 309 minutes]:
sync.runtime_SemacquireMutex(0xc00180009c, 0xc0000c7e00)
/usr/local/go/src/runtime/sema.go:71 +0x3d
sync.(*RWMutex).RLock(0xc001800090)
/usr/local/go/src/sync/rwmutex.go:50 +0x4e
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).GetRoundState(0xc001800000, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:218 +0x46
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusReactor).queryMaj23Routine(0xc0017def80, 0x11104a0, 0xc0072488f0, 0xc007248
9c0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/reactor.go:735 +0x16d
created by github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusReactor).AddPeer
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/reactor.go:172 +0x236
```
because localClient is locked
```
goroutine 1899 [semacquire, 309 minutes]:
sync.runtime_SemacquireMutex(0xc00003363c, 0xc0000cb500)
/usr/local/go/src/runtime/sema.go:71 +0x3d
sync.(*Mutex).Lock(0xc000033638)
/usr/local/go/src/sync/mutex.go:134 +0xff
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client.(*localClient).SetResponseCallback(0xc0001fb560, 0xc007868540)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client/local_client.go:32 +0x33
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy.(*appConnConsensus).SetResponseCallback(0xc00002f750, 0xc007868540)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy/app_conn.go:57 +0x40
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state.execBlockOnProxyApp(0x1104e20, 0xc002ca0ba0, 0x11092a0, 0xc00002f750, 0xc0001fe960, 0xc000bfc660, 0x110cfe0, 0xc000090330, 0xc9d12, 0xc000d9d5a0, ...)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state/execution.go:230 +0x1fd
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state.(*BlockExecutor).ApplyBlock(0xc002c2a230, 0x7, 0x0, 0xc000eae880, 0x6, 0xc002e52c60, 0x16, 0x1f927, 0xc9d12, 0xc000d9d5a0, ...)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state/execution.go:96 +0x142
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).finalizeCommit(0xc001800000, 0x1f928)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1339 +0xa3e
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).tryFinalizeCommit(0xc001800000, 0x1f928)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1270 +0x451
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).enterCommit.func1(0xc001800000, 0x0, 0x1f928)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1218 +0x90
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).enterCommit(0xc001800000, 0x1f928, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1247 +0x6b8
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).addVote(0xc001800000, 0xc003d8dea0, 0xc000cf4cc0, 0x28, 0xf1, 0xc003bc7ad0, 0xc003bc7b10)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1659 +0xbad
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).tryAddVote(0xc001800000, 0xc003d8dea0, 0xc000cf4cc0, 0x28, 0xf1, 0xf1, 0xf1)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1517 +0x59
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).handleMsg(0xc001800000, 0xd98200, 0xc0070dbed0, 0xc000cf4cc0, 0x28)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:660 +0x64b
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).receiveRoutine(0xc001800000, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:617 +0x670
created by github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).OnStart
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:311 +0x132
```
tx comes in and CheckTx is executed right when we execute the block
```
goroutine 111044 [semacquire, 309 minutes]:
sync.runtime_SemacquireMutex(0xc00003363c, 0x0)
/usr/local/go/src/runtime/sema.go:71 +0x3d
sync.(*Mutex).Lock(0xc000033638)
/usr/local/go/src/sync/mutex.go:134 +0xff
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client.(*localClient).CheckTxAsync(0xc0001fb0e0, 0xc002d94500, 0x13f, 0x280, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client/local_client.go:85 +0x47
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy.(*appConnMempool).CheckTxAsync(0xc00002f720, 0xc002d94500, 0x13f, 0x280, 0x1)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy/app_conn.go:114 +0x51
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/mempool.(*Mempool).CheckTx(0xc002d3a320, 0xc002d94500, 0x13f, 0x280, 0xc0072355f0, 0x0, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/mempool/mempool.go:316 +0x17b
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/core.BroadcastTxSync(0xc002d94500, 0x13f, 0x280, 0x0, 0x0, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/core/mempool.go:93 +0xb8
reflect.Value.call(0xd85560, 0x10326c0, 0x13, 0xec7b8b, 0x4, 0xc00663f180, 0x1, 0x1, 0xc00663f180, 0xc00663f188, ...)
/usr/local/go/src/reflect/value.go:447 +0x449
reflect.Value.Call(0xd85560, 0x10326c0, 0x13, 0xc00663f180, 0x1, 0x1, 0x0, 0x0, 0xc005cc9344)
/usr/local/go/src/reflect/value.go:308 +0xa4
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server.makeHTTPHandler.func2(0x1102060, 0xc00663f100, 0xc0082d7900)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server/handlers.go:269 +0x188
net/http.HandlerFunc.ServeHTTP(0xc002c81f20, 0x1102060, 0xc00663f100, 0xc0082d7900)
/usr/local/go/src/net/http/server.go:1964 +0x44
net/http.(*ServeMux).ServeHTTP(0xc002c81b60, 0x1102060, 0xc00663f100, 0xc0082d7900)
/usr/local/go/src/net/http/server.go:2361 +0x127
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server.maxBytesHandler.ServeHTTP(0x10f8a40, 0xc002c81b60, 0xf4240, 0x1102060, 0xc00663f100, 0xc0082d7900)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server/http_server.go:219 +0xcf
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server.RecoverAndLogHandler.func1(0x1103220, 0xc00121e620, 0xc0082d7900)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server/http_server.go:192 +0x394
net/http.HandlerFunc.ServeHTTP(0xc002c06ea0, 0x1103220, 0xc00121e620, 0xc0082d7900)
/usr/local/go/src/net/http/server.go:1964 +0x44
net/http.serverHandler.ServeHTTP(0xc001a1aa90, 0x1103220, 0xc00121e620, 0xc0082d7900)
/usr/local/go/src/net/http/server.go:2741 +0xab
net/http.(*conn).serve(0xc00785a3c0, 0x11041a0, 0xc000f844c0)
/usr/local/go/src/net/http/server.go:1847 +0x646
created by net/http.(*Server).Serve
/usr/local/go/src/net/http/server.go:2851 +0x2f5
```
* consensus: use read lock in Receive#VoteMessage
* use defer to unlock mutex because application might panic
* use defer in every method of the localClient
* add a changelog entry
* drain channels before Unsubscribe(All)
Read https://github.com/tendermint/tendermint/blob/55362ed76630f3e1ebec159a598f6a9fb5892cb1/libs/pubsub/pubsub.go#L13
for the detailed explanation of the issue.
We'll need to fix it someday. Make sure to keep an eye on
https://github.com/tendermint/tendermint/blob/master/docs/architecture/adr-033-pubsub.md
* retry instead of panic when peer has no state in reactors other than consensus
in /dump_consensus_state RPC endpoint, skip a peer with no state
* rpc/core/mempool: simplify error messages
* rpc/core/mempool: use time.After instead of timer
also, do not log DeliverTx result (to be consistent with other memthods)
* unlock before calling the callback in reqRes#SetCallback
2018-11-13 20:32:51 +04:00
|
|
|
ps, ok := peer.Get(types.PeerStateKey).(*PeerState)
|
|
|
|
if !ok {
|
|
|
|
panic(fmt.Sprintf("Peer %v has no state", peer))
|
|
|
|
}
|
2018-09-21 20:36:48 +02:00
|
|
|
switch msg.Msg.(type) {
|
|
|
|
case *VoteMessage:
|
|
|
|
if numVotes := ps.RecordVote(); numVotes%votesToContributeToBecomeGoodPeer == 0 {
|
|
|
|
conR.Switch.MarkPeerAsGood(peer)
|
|
|
|
}
|
|
|
|
case *BlockPartMessage:
|
|
|
|
if numParts := ps.RecordBlockPart(); numParts%blocksToContributeToBecomeGoodPeer == 0 {
|
|
|
|
conR.Switch.MarkPeerAsGood(peer)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
case <-conR.conS.Quit():
|
|
|
|
return
|
|
|
|
|
|
|
|
case <-conR.Quit():
|
|
|
|
return
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// String returns a string representation of the ConsensusReactor.
|
|
|
|
// NOTE: For now, it is just a hard-coded string to avoid accessing unprotected shared variables.
|
|
|
|
// TODO: improve!
|
2016-09-15 16:01:01 -07:00
|
|
|
func (conR *ConsensusReactor) String() string {
|
2016-12-19 23:18:08 -05:00
|
|
|
// better not to access shared variables
|
2016-12-23 11:11:22 -05:00
|
|
|
return "ConsensusReactor" // conR.StringIndented("")
|
2016-09-15 16:01:01 -07:00
|
|
|
}
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// StringIndented returns an indented string representation of the ConsensusReactor
|
2016-09-15 16:01:01 -07:00
|
|
|
func (conR *ConsensusReactor) StringIndented(indent string) string {
|
|
|
|
s := "ConsensusReactor{\n"
|
|
|
|
s += indent + " " + conR.conS.StringIndented(indent+" ") + "\n"
|
|
|
|
for _, peer := range conR.Switch.Peers().List() {
|
abci: localClient improvements & bugfixes & pubsub Unsubscribe issues (#2748)
* use READ lock/unlock in ConsensusState#GetLastHeight
Refs #2721
* do not use defers when there's no need
* fix peer formatting (output its address instead of the pointer)
```
[54310]: E[11-02|11:59:39.851] Connection failed @ sendRoutine module=p2p peer=0xb78f00 conn=MConn{74.207.236.148:26656} err="pong timeout"
```
https://github.com/tendermint/tendermint/issues/2721#issuecomment-435326581
* panic if peer has no state
https://github.com/tendermint/tendermint/issues/2721#issuecomment-435347165
It's confusing that sometimes we check if peer has a state, but most of
the times we expect it to be there
1. https://github.com/tendermint/tendermint/blob/add79700b5fe84417538202b6c927c8cc5383672/mempool/reactor.go#L138
2. https://github.com/tendermint/tendermint/blob/add79700b5fe84417538202b6c927c8cc5383672/rpc/core/consensus.go#L196 (edited)
I will change everything to always assume peer has a state and panic
otherwise
that should help identify issues earlier
* abci/localclient: extend lock on app callback
App callback should be protected by lock as well (note this was already
done for InitChainAsync, why not for others???). Otherwise, when we
execute the block, tx might come in and call the callback in the same
time we're updating it in execBlockOnProxyApp => DATA RACE
Fixes #2721
Consensus state is locked
```
goroutine 113333 [semacquire, 309 minutes]:
sync.runtime_SemacquireMutex(0xc00180009c, 0xc0000c7e00)
/usr/local/go/src/runtime/sema.go:71 +0x3d
sync.(*RWMutex).RLock(0xc001800090)
/usr/local/go/src/sync/rwmutex.go:50 +0x4e
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).GetRoundState(0xc001800000, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:218 +0x46
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusReactor).queryMaj23Routine(0xc0017def80, 0x11104a0, 0xc0072488f0, 0xc007248
9c0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/reactor.go:735 +0x16d
created by github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusReactor).AddPeer
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/reactor.go:172 +0x236
```
because localClient is locked
```
goroutine 1899 [semacquire, 309 minutes]:
sync.runtime_SemacquireMutex(0xc00003363c, 0xc0000cb500)
/usr/local/go/src/runtime/sema.go:71 +0x3d
sync.(*Mutex).Lock(0xc000033638)
/usr/local/go/src/sync/mutex.go:134 +0xff
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client.(*localClient).SetResponseCallback(0xc0001fb560, 0xc007868540)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client/local_client.go:32 +0x33
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy.(*appConnConsensus).SetResponseCallback(0xc00002f750, 0xc007868540)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy/app_conn.go:57 +0x40
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state.execBlockOnProxyApp(0x1104e20, 0xc002ca0ba0, 0x11092a0, 0xc00002f750, 0xc0001fe960, 0xc000bfc660, 0x110cfe0, 0xc000090330, 0xc9d12, 0xc000d9d5a0, ...)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state/execution.go:230 +0x1fd
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state.(*BlockExecutor).ApplyBlock(0xc002c2a230, 0x7, 0x0, 0xc000eae880, 0x6, 0xc002e52c60, 0x16, 0x1f927, 0xc9d12, 0xc000d9d5a0, ...)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/state/execution.go:96 +0x142
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).finalizeCommit(0xc001800000, 0x1f928)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1339 +0xa3e
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).tryFinalizeCommit(0xc001800000, 0x1f928)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1270 +0x451
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).enterCommit.func1(0xc001800000, 0x0, 0x1f928)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1218 +0x90
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).enterCommit(0xc001800000, 0x1f928, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1247 +0x6b8
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).addVote(0xc001800000, 0xc003d8dea0, 0xc000cf4cc0, 0x28, 0xf1, 0xc003bc7ad0, 0xc003bc7b10)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1659 +0xbad
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).tryAddVote(0xc001800000, 0xc003d8dea0, 0xc000cf4cc0, 0x28, 0xf1, 0xf1, 0xf1)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:1517 +0x59
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).handleMsg(0xc001800000, 0xd98200, 0xc0070dbed0, 0xc000cf4cc0, 0x28)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:660 +0x64b
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).receiveRoutine(0xc001800000, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:617 +0x670
created by github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus.(*ConsensusState).OnStart
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/consensus/state.go:311 +0x132
```
tx comes in and CheckTx is executed right when we execute the block
```
goroutine 111044 [semacquire, 309 minutes]:
sync.runtime_SemacquireMutex(0xc00003363c, 0x0)
/usr/local/go/src/runtime/sema.go:71 +0x3d
sync.(*Mutex).Lock(0xc000033638)
/usr/local/go/src/sync/mutex.go:134 +0xff
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client.(*localClient).CheckTxAsync(0xc0001fb0e0, 0xc002d94500, 0x13f, 0x280, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/abci/client/local_client.go:85 +0x47
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy.(*appConnMempool).CheckTxAsync(0xc00002f720, 0xc002d94500, 0x13f, 0x280, 0x1)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/proxy/app_conn.go:114 +0x51
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/mempool.(*Mempool).CheckTx(0xc002d3a320, 0xc002d94500, 0x13f, 0x280, 0xc0072355f0, 0x0, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/mempool/mempool.go:316 +0x17b
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/core.BroadcastTxSync(0xc002d94500, 0x13f, 0x280, 0x0, 0x0, 0x0)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/core/mempool.go:93 +0xb8
reflect.Value.call(0xd85560, 0x10326c0, 0x13, 0xec7b8b, 0x4, 0xc00663f180, 0x1, 0x1, 0xc00663f180, 0xc00663f188, ...)
/usr/local/go/src/reflect/value.go:447 +0x449
reflect.Value.Call(0xd85560, 0x10326c0, 0x13, 0xc00663f180, 0x1, 0x1, 0x0, 0x0, 0xc005cc9344)
/usr/local/go/src/reflect/value.go:308 +0xa4
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server.makeHTTPHandler.func2(0x1102060, 0xc00663f100, 0xc0082d7900)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server/handlers.go:269 +0x188
net/http.HandlerFunc.ServeHTTP(0xc002c81f20, 0x1102060, 0xc00663f100, 0xc0082d7900)
/usr/local/go/src/net/http/server.go:1964 +0x44
net/http.(*ServeMux).ServeHTTP(0xc002c81b60, 0x1102060, 0xc00663f100, 0xc0082d7900)
/usr/local/go/src/net/http/server.go:2361 +0x127
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server.maxBytesHandler.ServeHTTP(0x10f8a40, 0xc002c81b60, 0xf4240, 0x1102060, 0xc00663f100, 0xc0082d7900)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server/http_server.go:219 +0xcf
github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server.RecoverAndLogHandler.func1(0x1103220, 0xc00121e620, 0xc0082d7900)
/root/go/src/github.com/MinterTeam/minter-go-node/vendor/github.com/tendermint/tendermint/rpc/lib/server/http_server.go:192 +0x394
net/http.HandlerFunc.ServeHTTP(0xc002c06ea0, 0x1103220, 0xc00121e620, 0xc0082d7900)
/usr/local/go/src/net/http/server.go:1964 +0x44
net/http.serverHandler.ServeHTTP(0xc001a1aa90, 0x1103220, 0xc00121e620, 0xc0082d7900)
/usr/local/go/src/net/http/server.go:2741 +0xab
net/http.(*conn).serve(0xc00785a3c0, 0x11041a0, 0xc000f844c0)
/usr/local/go/src/net/http/server.go:1847 +0x646
created by net/http.(*Server).Serve
/usr/local/go/src/net/http/server.go:2851 +0x2f5
```
* consensus: use read lock in Receive#VoteMessage
* use defer to unlock mutex because application might panic
* use defer in every method of the localClient
* add a changelog entry
* drain channels before Unsubscribe(All)
Read https://github.com/tendermint/tendermint/blob/55362ed76630f3e1ebec159a598f6a9fb5892cb1/libs/pubsub/pubsub.go#L13
for the detailed explanation of the issue.
We'll need to fix it someday. Make sure to keep an eye on
https://github.com/tendermint/tendermint/blob/master/docs/architecture/adr-033-pubsub.md
* retry instead of panic when peer has no state in reactors other than consensus
in /dump_consensus_state RPC endpoint, skip a peer with no state
* rpc/core/mempool: simplify error messages
* rpc/core/mempool: use time.After instead of timer
also, do not log DeliverTx result (to be consistent with other memthods)
* unlock before calling the callback in reqRes#SetCallback
2018-11-13 20:32:51 +04:00
|
|
|
ps, ok := peer.Get(types.PeerStateKey).(*PeerState)
|
|
|
|
if !ok {
|
|
|
|
panic(fmt.Sprintf("Peer %v has no state", peer))
|
|
|
|
}
|
2016-09-15 16:01:01 -07:00
|
|
|
s += indent + " " + ps.StringIndented(indent+" ") + "\n"
|
|
|
|
}
|
|
|
|
s += indent + "}"
|
|
|
|
return s
|
|
|
|
}
|
|
|
|
|
2018-09-25 04:14:38 -07:00
|
|
|
func (conR *ConsensusReactor) updateFastSyncingMetric() {
|
|
|
|
var fastSyncing float64
|
|
|
|
if conR.fastSync {
|
|
|
|
fastSyncing = 1
|
|
|
|
} else {
|
|
|
|
fastSyncing = 0
|
|
|
|
}
|
|
|
|
conR.metrics.FastSyncing.Set(fastSyncing)
|
|
|
|
}
|
|
|
|
|
|
|
|
// ReactorMetrics sets the metrics
|
|
|
|
func ReactorMetrics(metrics *Metrics) ReactorOption {
|
|
|
|
return func(conR *ConsensusReactor) { conR.metrics = metrics }
|
|
|
|
}
|
|
|
|
|
2014-08-10 16:35:08 -07:00
|
|
|
//-----------------------------------------------------------------------------
|
|
|
|
|
|
|
|
var (
|
|
|
|
ErrPeerStateHeightRegression = errors.New("Error peer state height regression")
|
|
|
|
ErrPeerStateInvalidStartTime = errors.New("Error peer state invalid startTime")
|
|
|
|
)
|
|
|
|
|
2018-04-10 11:15:16 +02:00
|
|
|
// PeerState contains the known state of a peer, including its connection and
|
|
|
|
// threadsafe access to its PeerRoundState.
|
2018-05-10 22:43:21 -07:00
|
|
|
// NOTE: THIS GETS DUMPED WITH rpc/core/consensus.go.
|
|
|
|
// Be mindful of what you Expose.
|
2014-08-10 16:35:08 -07:00
|
|
|
type PeerState struct {
|
2018-05-10 22:43:21 -07:00
|
|
|
peer p2p.Peer
|
2017-09-12 20:49:22 -04:00
|
|
|
logger log.Logger
|
2015-05-05 22:53:06 -07:00
|
|
|
|
2019-02-11 16:31:34 +04:00
|
|
|
mtx sync.Mutex // NOTE: Modify below using setters, never directly.
|
2018-05-12 15:39:30 -07:00
|
|
|
PRS cstypes.PeerRoundState `json:"round_state"` // Exposed.
|
|
|
|
Stats *peerStateStats `json:"stats"` // Exposed.
|
2018-03-05 16:51:52 +04:00
|
|
|
}
|
|
|
|
|
|
|
|
// peerStateStats holds internal statistics for a peer.
|
|
|
|
type peerStateStats struct {
|
2018-09-21 20:36:48 +02:00
|
|
|
Votes int `json:"votes"`
|
|
|
|
BlockParts int `json:"block_parts"`
|
2014-08-10 16:35:08 -07:00
|
|
|
}
|
|
|
|
|
2018-03-19 13:13:19 +03:00
|
|
|
func (pss peerStateStats) String() string {
|
2018-09-21 20:36:48 +02:00
|
|
|
return fmt.Sprintf("peerStateStats{votes: %d, blockParts: %d}",
|
|
|
|
pss.Votes, pss.BlockParts)
|
2018-03-19 13:13:19 +03:00
|
|
|
}
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// NewPeerState returns a new PeerState for the given Peer
|
2017-09-12 20:49:22 -04:00
|
|
|
func NewPeerState(peer p2p.Peer) *PeerState {
|
2015-09-10 01:29:49 -07:00
|
|
|
return &PeerState{
|
2018-05-10 22:43:21 -07:00
|
|
|
peer: peer,
|
2017-09-12 20:49:22 -04:00
|
|
|
logger: log.NewNopLogger(),
|
2018-05-12 15:39:30 -07:00
|
|
|
PRS: cstypes.PeerRoundState{
|
2015-09-10 01:29:49 -07:00
|
|
|
Round: -1,
|
|
|
|
ProposalPOLRound: -1,
|
|
|
|
LastCommitRound: -1,
|
|
|
|
CatchupCommitRound: -1,
|
|
|
|
},
|
2018-05-10 22:43:21 -07:00
|
|
|
Stats: &peerStateStats{},
|
2015-09-10 01:29:49 -07:00
|
|
|
}
|
2014-08-10 16:35:08 -07:00
|
|
|
}
|
|
|
|
|
2018-04-10 11:15:16 +02:00
|
|
|
// SetLogger allows to set a logger on the peer state. Returns the peer state
|
|
|
|
// itself.
|
2017-09-12 20:49:22 -04:00
|
|
|
func (ps *PeerState) SetLogger(logger log.Logger) *PeerState {
|
|
|
|
ps.logger = logger
|
|
|
|
return ps
|
|
|
|
}
|
|
|
|
|
2018-04-10 11:15:16 +02:00
|
|
|
// GetRoundState returns an shallow copy of the PeerRoundState.
|
2014-09-14 15:37:32 -07:00
|
|
|
// There's no point in mutating it since it won't change PeerState.
|
2017-10-10 12:39:21 +04:00
|
|
|
func (ps *PeerState) GetRoundState() *cstypes.PeerRoundState {
|
2014-09-07 02:21:25 -07:00
|
|
|
ps.mtx.Lock()
|
|
|
|
defer ps.mtx.Unlock()
|
2014-12-30 21:11:06 -08:00
|
|
|
|
2018-05-12 15:39:30 -07:00
|
|
|
prs := ps.PRS // copy
|
2014-09-14 15:37:32 -07:00
|
|
|
return &prs
|
2014-09-07 02:21:25 -07:00
|
|
|
}
|
|
|
|
|
2018-05-12 15:39:30 -07:00
|
|
|
// ToJSON returns a json of PeerState, marshalled using go-amino.
|
|
|
|
func (ps *PeerState) ToJSON() ([]byte, error) {
|
2018-04-10 11:15:16 +02:00
|
|
|
ps.mtx.Lock()
|
|
|
|
defer ps.mtx.Unlock()
|
|
|
|
|
2018-05-12 15:39:30 -07:00
|
|
|
return cdc.MarshalJSON(ps)
|
2018-04-10 11:15:16 +02:00
|
|
|
}
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// GetHeight returns an atomic snapshot of the PeerRoundState's height
|
2015-09-25 12:55:59 -04:00
|
|
|
// used by the mempool to ensure peers are caught up before broadcasting new txs
|
2017-12-01 19:04:53 -06:00
|
|
|
func (ps *PeerState) GetHeight() int64 {
|
2015-09-25 12:55:59 -04:00
|
|
|
ps.mtx.Lock()
|
|
|
|
defer ps.mtx.Unlock()
|
2018-05-12 15:39:30 -07:00
|
|
|
return ps.PRS.Height
|
2015-09-25 12:55:59 -04:00
|
|
|
}
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// SetHasProposal sets the given proposal as known for the peer.
|
2015-08-10 20:38:45 -07:00
|
|
|
func (ps *PeerState) SetHasProposal(proposal *types.Proposal) {
|
2014-09-07 02:21:25 -07:00
|
|
|
ps.mtx.Lock()
|
|
|
|
defer ps.mtx.Unlock()
|
|
|
|
|
2018-05-12 15:39:30 -07:00
|
|
|
if ps.PRS.Height != proposal.Height || ps.PRS.Round != proposal.Round {
|
2014-10-26 00:09:22 -07:00
|
|
|
return
|
|
|
|
}
|
2018-10-31 14:20:36 +01:00
|
|
|
|
2018-05-12 15:39:30 -07:00
|
|
|
if ps.PRS.Proposal {
|
2014-10-26 00:09:22 -07:00
|
|
|
return
|
2014-09-07 02:21:25 -07:00
|
|
|
}
|
2014-10-26 00:09:22 -07:00
|
|
|
|
2018-05-12 15:39:30 -07:00
|
|
|
ps.PRS.Proposal = true
|
2018-10-31 14:20:36 +01:00
|
|
|
|
|
|
|
// ps.PRS.ProposalBlockParts is set due to NewValidBlockMessage
|
|
|
|
if ps.PRS.ProposalBlockParts != nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2018-10-31 15:27:11 +01:00
|
|
|
ps.PRS.ProposalBlockPartsHeader = proposal.BlockID.PartsHeader
|
|
|
|
ps.PRS.ProposalBlockParts = cmn.NewBitArray(proposal.BlockID.PartsHeader.Total)
|
2018-05-12 15:39:30 -07:00
|
|
|
ps.PRS.ProposalPOLRound = proposal.POLRound
|
|
|
|
ps.PRS.ProposalPOL = nil // Nil until ProposalPOLMessage received.
|
2014-09-14 15:37:32 -07:00
|
|
|
}
|
|
|
|
|
2017-11-09 18:14:41 +00:00
|
|
|
// InitProposalBlockParts initializes the peer's proposal block parts header and bit array.
|
|
|
|
func (ps *PeerState) InitProposalBlockParts(partsHeader types.PartSetHeader) {
|
|
|
|
ps.mtx.Lock()
|
|
|
|
defer ps.mtx.Unlock()
|
|
|
|
|
2018-05-12 15:39:30 -07:00
|
|
|
if ps.PRS.ProposalBlockParts != nil {
|
2017-11-09 18:14:41 +00:00
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2018-05-12 15:39:30 -07:00
|
|
|
ps.PRS.ProposalBlockPartsHeader = partsHeader
|
|
|
|
ps.PRS.ProposalBlockParts = cmn.NewBitArray(partsHeader.Total)
|
2017-11-09 18:14:41 +00:00
|
|
|
}
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// SetHasProposalBlockPart sets the given block part index as known for the peer.
|
2017-12-01 19:04:53 -06:00
|
|
|
func (ps *PeerState) SetHasProposalBlockPart(height int64, round int, index int) {
|
2014-08-10 16:35:08 -07:00
|
|
|
ps.mtx.Lock()
|
|
|
|
defer ps.mtx.Unlock()
|
2014-09-14 15:37:32 -07:00
|
|
|
|
2018-05-12 15:39:30 -07:00
|
|
|
if ps.PRS.Height != height || ps.PRS.Round != round {
|
2014-10-26 00:09:22 -07:00
|
|
|
return
|
2014-08-10 16:35:08 -07:00
|
|
|
}
|
2014-10-26 00:09:22 -07:00
|
|
|
|
2018-05-12 15:39:30 -07:00
|
|
|
ps.PRS.ProposalBlockParts.SetIndex(index, true)
|
2014-10-26 00:09:22 -07:00
|
|
|
}
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// PickSendVote picks a vote and sends it to the peer.
|
2015-07-05 13:40:59 -07:00
|
|
|
// Returns true if vote was sent.
|
2017-07-09 18:38:59 -04:00
|
|
|
func (ps *PeerState) PickSendVote(votes types.VoteSetReader) bool {
|
2016-07-01 17:47:31 -04:00
|
|
|
if vote, ok := ps.PickVoteToSend(votes); ok {
|
|
|
|
msg := &VoteMessage{vote}
|
2017-12-15 23:08:05 -05:00
|
|
|
ps.logger.Debug("Sending vote message", "ps", ps, "vote", vote)
|
2018-11-11 13:57:08 +01:00
|
|
|
if ps.peer.Send(VoteChannel, cdc.MustMarshalBinaryBare(msg)) {
|
|
|
|
ps.SetHasVote(vote)
|
|
|
|
return true
|
|
|
|
}
|
|
|
|
return false
|
2015-07-05 13:40:59 -07:00
|
|
|
}
|
|
|
|
return false
|
|
|
|
}
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// PickVoteToSend picks a vote to send to the peer.
|
|
|
|
// Returns true if a vote was picked.
|
|
|
|
// NOTE: `votes` must be the correct Size() for the Height().
|
2016-07-01 17:47:31 -04:00
|
|
|
func (ps *PeerState) PickVoteToSend(votes types.VoteSetReader) (vote *types.Vote, ok bool) {
|
2015-07-05 13:40:59 -07:00
|
|
|
ps.mtx.Lock()
|
|
|
|
defer ps.mtx.Unlock()
|
|
|
|
|
|
|
|
if votes.Size() == 0 {
|
2016-07-01 17:47:31 -04:00
|
|
|
return nil, false
|
2015-07-05 13:40:59 -07:00
|
|
|
}
|
|
|
|
|
2018-10-13 01:21:46 +02:00
|
|
|
height, round, type_, size := votes.Height(), votes.Round(), types.SignedMsgType(votes.Type()), votes.Size()
|
2015-07-05 13:40:59 -07:00
|
|
|
|
|
|
|
// Lazily set data using 'votes'.
|
|
|
|
if votes.IsCommit() {
|
|
|
|
ps.ensureCatchupCommitRound(height, round, size)
|
|
|
|
}
|
|
|
|
ps.ensureVoteBitArrays(height, size)
|
|
|
|
|
|
|
|
psVotes := ps.getVoteBitArray(height, round, type_)
|
|
|
|
if psVotes == nil {
|
2016-07-01 17:47:31 -04:00
|
|
|
return nil, false // Not something worth sending
|
2015-07-05 13:40:59 -07:00
|
|
|
}
|
|
|
|
if index, ok := votes.BitArray().Sub(psVotes).PickRandom(); ok {
|
2016-07-01 17:47:31 -04:00
|
|
|
return votes.GetByIndex(index), true
|
2015-07-05 13:40:59 -07:00
|
|
|
}
|
2016-07-01 17:47:31 -04:00
|
|
|
return nil, false
|
2015-07-05 13:40:59 -07:00
|
|
|
}
|
|
|
|
|
2018-10-13 01:21:46 +02:00
|
|
|
func (ps *PeerState) getVoteBitArray(height int64, round int, type_ types.SignedMsgType) *cmn.BitArray {
|
2016-09-16 09:20:07 -07:00
|
|
|
if !types.IsVoteTypeValid(type_) {
|
2017-08-16 00:43:55 -04:00
|
|
|
return nil
|
2016-09-05 17:33:02 -07:00
|
|
|
}
|
|
|
|
|
2018-05-12 15:39:30 -07:00
|
|
|
if ps.PRS.Height == height {
|
|
|
|
if ps.PRS.Round == round {
|
2015-07-05 13:40:59 -07:00
|
|
|
switch type_ {
|
2018-10-13 01:21:46 +02:00
|
|
|
case types.PrevoteType:
|
2018-05-12 15:39:30 -07:00
|
|
|
return ps.PRS.Prevotes
|
2018-10-13 01:21:46 +02:00
|
|
|
case types.PrecommitType:
|
2018-05-12 15:39:30 -07:00
|
|
|
return ps.PRS.Precommits
|
2015-07-05 13:40:59 -07:00
|
|
|
}
|
|
|
|
}
|
2018-05-12 15:39:30 -07:00
|
|
|
if ps.PRS.CatchupCommitRound == round {
|
2015-07-05 13:40:59 -07:00
|
|
|
switch type_ {
|
2018-10-13 01:21:46 +02:00
|
|
|
case types.PrevoteType:
|
2015-07-05 13:40:59 -07:00
|
|
|
return nil
|
2018-10-13 01:21:46 +02:00
|
|
|
case types.PrecommitType:
|
2018-05-12 15:39:30 -07:00
|
|
|
return ps.PRS.CatchupCommit
|
2016-09-05 17:33:02 -07:00
|
|
|
}
|
|
|
|
}
|
2018-05-12 15:39:30 -07:00
|
|
|
if ps.PRS.ProposalPOLRound == round {
|
2016-09-05 17:33:02 -07:00
|
|
|
switch type_ {
|
2018-10-13 01:21:46 +02:00
|
|
|
case types.PrevoteType:
|
2018-05-12 15:39:30 -07:00
|
|
|
return ps.PRS.ProposalPOL
|
2018-10-13 01:21:46 +02:00
|
|
|
case types.PrecommitType:
|
2016-09-05 17:33:02 -07:00
|
|
|
return nil
|
2015-07-05 13:40:59 -07:00
|
|
|
}
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
2018-05-12 15:39:30 -07:00
|
|
|
if ps.PRS.Height == height+1 {
|
|
|
|
if ps.PRS.LastCommitRound == round {
|
2015-07-05 13:40:59 -07:00
|
|
|
switch type_ {
|
2018-10-13 01:21:46 +02:00
|
|
|
case types.PrevoteType:
|
2015-07-05 13:40:59 -07:00
|
|
|
return nil
|
2018-10-13 01:21:46 +02:00
|
|
|
case types.PrecommitType:
|
2018-05-12 15:39:30 -07:00
|
|
|
return ps.PRS.LastCommit
|
2015-07-05 13:40:59 -07:00
|
|
|
}
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2015-10-12 15:19:55 -07:00
|
|
|
// 'round': A round for which we have a +2/3 commit.
|
2017-12-01 19:04:53 -06:00
|
|
|
func (ps *PeerState) ensureCatchupCommitRound(height int64, round int, numValidators int) {
|
2018-05-12 15:39:30 -07:00
|
|
|
if ps.PRS.Height != height {
|
2015-07-05 13:40:59 -07:00
|
|
|
return
|
|
|
|
}
|
2015-10-12 15:19:55 -07:00
|
|
|
/*
|
|
|
|
NOTE: This is wrong, 'round' could change.
|
2016-04-02 09:10:16 -07:00
|
|
|
e.g. if orig round is not the same as block LastCommit round.
|
2015-10-12 15:19:55 -07:00
|
|
|
if ps.CatchupCommitRound != -1 && ps.CatchupCommitRound != round {
|
2019-04-26 06:23:43 -04:00
|
|
|
panic(fmt.Sprintf("Conflicting CatchupCommitRound. Height: %v, Orig: %v, New: %v", height, ps.CatchupCommitRound, round))
|
2015-10-12 15:19:55 -07:00
|
|
|
}
|
|
|
|
*/
|
2018-05-12 15:39:30 -07:00
|
|
|
if ps.PRS.CatchupCommitRound == round {
|
2015-07-05 13:40:59 -07:00
|
|
|
return // Nothing to do!
|
|
|
|
}
|
2018-05-12 15:39:30 -07:00
|
|
|
ps.PRS.CatchupCommitRound = round
|
|
|
|
if round == ps.PRS.Round {
|
|
|
|
ps.PRS.CatchupCommit = ps.PRS.Precommits
|
2015-07-05 13:40:59 -07:00
|
|
|
} else {
|
2018-05-12 15:39:30 -07:00
|
|
|
ps.PRS.CatchupCommit = cmn.NewBitArray(numValidators)
|
2015-07-05 13:40:59 -07:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-04-10 11:15:16 +02:00
|
|
|
// EnsureVoteBitArrays ensures the bit-arrays have been allocated for tracking
|
2017-07-09 18:38:59 -04:00
|
|
|
// what votes this peer has received.
|
2015-06-25 14:05:18 -07:00
|
|
|
// NOTE: It's important to make sure that numValidators actually matches
|
|
|
|
// what the node sees as the number of validators for height.
|
2017-12-01 19:04:53 -06:00
|
|
|
func (ps *PeerState) EnsureVoteBitArrays(height int64, numValidators int) {
|
2014-10-26 00:09:22 -07:00
|
|
|
ps.mtx.Lock()
|
|
|
|
defer ps.mtx.Unlock()
|
2015-07-05 13:40:59 -07:00
|
|
|
ps.ensureVoteBitArrays(height, numValidators)
|
|
|
|
}
|
2014-10-26 00:09:22 -07:00
|
|
|
|
2017-12-01 19:04:53 -06:00
|
|
|
func (ps *PeerState) ensureVoteBitArrays(height int64, numValidators int) {
|
2018-05-12 15:39:30 -07:00
|
|
|
if ps.PRS.Height == height {
|
|
|
|
if ps.PRS.Prevotes == nil {
|
|
|
|
ps.PRS.Prevotes = cmn.NewBitArray(numValidators)
|
2015-05-07 17:35:58 -07:00
|
|
|
}
|
2018-05-12 15:39:30 -07:00
|
|
|
if ps.PRS.Precommits == nil {
|
|
|
|
ps.PRS.Precommits = cmn.NewBitArray(numValidators)
|
2015-05-07 17:35:58 -07:00
|
|
|
}
|
2018-05-12 15:39:30 -07:00
|
|
|
if ps.PRS.CatchupCommit == nil {
|
|
|
|
ps.PRS.CatchupCommit = cmn.NewBitArray(numValidators)
|
2015-05-07 17:35:58 -07:00
|
|
|
}
|
2018-05-12 15:39:30 -07:00
|
|
|
if ps.PRS.ProposalPOL == nil {
|
|
|
|
ps.PRS.ProposalPOL = cmn.NewBitArray(numValidators)
|
2015-06-22 19:04:31 -07:00
|
|
|
}
|
2018-05-12 15:39:30 -07:00
|
|
|
} else if ps.PRS.Height == height+1 {
|
|
|
|
if ps.PRS.LastCommit == nil {
|
|
|
|
ps.PRS.LastCommit = cmn.NewBitArray(numValidators)
|
2015-05-07 17:35:58 -07:00
|
|
|
}
|
2014-09-08 16:18:50 -07:00
|
|
|
}
|
2014-08-10 16:35:08 -07:00
|
|
|
}
|
|
|
|
|
2018-09-21 20:36:48 +02:00
|
|
|
// RecordVote increments internal votes related statistics for this peer.
|
|
|
|
// It returns the total number of added votes.
|
|
|
|
func (ps *PeerState) RecordVote() int {
|
2018-03-05 16:51:52 +04:00
|
|
|
ps.mtx.Lock()
|
|
|
|
defer ps.mtx.Unlock()
|
|
|
|
|
2018-05-10 22:43:21 -07:00
|
|
|
ps.Stats.Votes++
|
2018-09-21 20:36:48 +02:00
|
|
|
|
2018-05-10 22:43:21 -07:00
|
|
|
return ps.Stats.Votes
|
2018-03-05 16:51:52 +04:00
|
|
|
}
|
|
|
|
|
2018-03-19 13:13:19 +03:00
|
|
|
// VotesSent returns the number of blocks for which peer has been sending us
|
|
|
|
// votes.
|
|
|
|
func (ps *PeerState) VotesSent() int {
|
2018-04-05 11:42:45 +02:00
|
|
|
ps.mtx.Lock()
|
|
|
|
defer ps.mtx.Unlock()
|
|
|
|
|
2018-05-10 22:43:21 -07:00
|
|
|
return ps.Stats.Votes
|
2018-03-19 13:13:19 +03:00
|
|
|
}
|
|
|
|
|
2018-09-21 20:36:48 +02:00
|
|
|
// RecordBlockPart increments internal block part related statistics for this peer.
|
|
|
|
// It returns the total number of added block parts.
|
|
|
|
func (ps *PeerState) RecordBlockPart() int {
|
2018-03-05 16:51:52 +04:00
|
|
|
ps.mtx.Lock()
|
|
|
|
defer ps.mtx.Unlock()
|
|
|
|
|
2018-05-10 22:43:21 -07:00
|
|
|
ps.Stats.BlockParts++
|
|
|
|
return ps.Stats.BlockParts
|
2018-03-05 16:51:52 +04:00
|
|
|
}
|
|
|
|
|
2018-09-21 20:36:48 +02:00
|
|
|
// BlockPartsSent returns the number of useful block parts the peer has sent us.
|
2018-03-19 13:13:19 +03:00
|
|
|
func (ps *PeerState) BlockPartsSent() int {
|
2018-04-05 11:42:45 +02:00
|
|
|
ps.mtx.Lock()
|
|
|
|
defer ps.mtx.Unlock()
|
|
|
|
|
2018-05-10 22:43:21 -07:00
|
|
|
return ps.Stats.BlockParts
|
2018-03-05 16:51:52 +04:00
|
|
|
}
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// SetHasVote sets the given vote as known by the peer
|
2016-07-01 17:47:31 -04:00
|
|
|
func (ps *PeerState) SetHasVote(vote *types.Vote) {
|
2014-08-10 16:35:08 -07:00
|
|
|
ps.mtx.Lock()
|
|
|
|
defer ps.mtx.Unlock()
|
2014-12-30 21:11:06 -08:00
|
|
|
|
2016-07-01 17:47:31 -04:00
|
|
|
ps.setHasVote(vote.Height, vote.Round, vote.Type, vote.ValidatorIndex)
|
2014-11-01 22:42:04 -07:00
|
|
|
}
|
2014-10-26 00:09:22 -07:00
|
|
|
|
2018-10-13 01:21:46 +02:00
|
|
|
func (ps *PeerState) setHasVote(height int64, round int, type_ types.SignedMsgType, index int) {
|
2018-08-10 00:25:57 -05:00
|
|
|
logger := ps.logger.With("peerH/R", fmt.Sprintf("%d/%d", ps.PRS.Height, ps.PRS.Round), "H/R", fmt.Sprintf("%d/%d", height, round))
|
2017-11-12 06:40:27 +00:00
|
|
|
logger.Debug("setHasVote", "type", type_, "index", index)
|
2014-10-26 00:09:22 -07:00
|
|
|
|
2016-09-05 17:33:02 -07:00
|
|
|
// NOTE: some may be nil BitArrays -> no side effects.
|
2017-08-16 00:43:55 -04:00
|
|
|
psVotes := ps.getVoteBitArray(height, round, type_)
|
|
|
|
if psVotes != nil {
|
|
|
|
psVotes.SetIndex(index, true)
|
|
|
|
}
|
2014-12-30 14:36:45 -08:00
|
|
|
}
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// ApplyNewRoundStepMessage updates the peer state for the new round.
|
2015-08-26 18:56:34 -04:00
|
|
|
func (ps *PeerState) ApplyNewRoundStepMessage(msg *NewRoundStepMessage) {
|
2014-08-10 16:35:08 -07:00
|
|
|
ps.mtx.Lock()
|
|
|
|
defer ps.mtx.Unlock()
|
2014-09-14 15:37:32 -07:00
|
|
|
|
2016-03-05 01:18:14 -05:00
|
|
|
// Ignore duplicates or decreases
|
2018-05-12 15:39:30 -07:00
|
|
|
if CompareHRS(msg.Height, msg.Round, msg.Step, ps.PRS.Height, ps.PRS.Round, ps.PRS.Step) <= 0 {
|
2015-07-05 19:05:07 -07:00
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2014-10-26 00:09:22 -07:00
|
|
|
// Just remember these values.
|
2018-05-12 15:39:30 -07:00
|
|
|
psHeight := ps.PRS.Height
|
|
|
|
psRound := ps.PRS.Round
|
|
|
|
psCatchupCommitRound := ps.PRS.CatchupCommitRound
|
|
|
|
psCatchupCommit := ps.PRS.CatchupCommit
|
2014-10-26 00:09:22 -07:00
|
|
|
|
2018-09-01 01:33:51 +02:00
|
|
|
startTime := tmtime.Now().Add(-1 * time.Duration(msg.SecondsSinceStartTime) * time.Second)
|
2018-05-12 15:39:30 -07:00
|
|
|
ps.PRS.Height = msg.Height
|
|
|
|
ps.PRS.Round = msg.Round
|
|
|
|
ps.PRS.Step = msg.Step
|
|
|
|
ps.PRS.StartTime = startTime
|
2014-10-26 00:09:22 -07:00
|
|
|
if psHeight != msg.Height || psRound != msg.Round {
|
2018-05-12 15:39:30 -07:00
|
|
|
ps.PRS.Proposal = false
|
|
|
|
ps.PRS.ProposalBlockPartsHeader = types.PartSetHeader{}
|
|
|
|
ps.PRS.ProposalBlockParts = nil
|
|
|
|
ps.PRS.ProposalPOLRound = -1
|
|
|
|
ps.PRS.ProposalPOL = nil
|
2014-10-26 00:09:22 -07:00
|
|
|
// We'll update the BitArray capacity later.
|
2018-05-12 15:39:30 -07:00
|
|
|
ps.PRS.Prevotes = nil
|
|
|
|
ps.PRS.Precommits = nil
|
2014-10-25 14:27:53 -07:00
|
|
|
}
|
2015-06-25 20:28:34 -07:00
|
|
|
if psHeight == msg.Height && psRound != msg.Round && msg.Round == psCatchupCommitRound {
|
2015-06-19 15:30:10 -07:00
|
|
|
// Peer caught up to CatchupCommitRound.
|
2015-06-25 14:05:18 -07:00
|
|
|
// Preserve psCatchupCommit!
|
|
|
|
// NOTE: We prefer to use prs.Precommits if
|
|
|
|
// pr.Round matches pr.CatchupCommitRound.
|
2018-05-12 15:39:30 -07:00
|
|
|
ps.PRS.Precommits = psCatchupCommit
|
2015-06-19 15:30:10 -07:00
|
|
|
}
|
2014-10-26 00:09:22 -07:00
|
|
|
if psHeight != msg.Height {
|
2015-06-19 15:30:10 -07:00
|
|
|
// Shift Precommits to LastCommit.
|
|
|
|
if psHeight+1 == msg.Height && psRound == msg.LastCommitRound {
|
2018-05-12 15:39:30 -07:00
|
|
|
ps.PRS.LastCommitRound = msg.LastCommitRound
|
|
|
|
ps.PRS.LastCommit = ps.PRS.Precommits
|
2014-11-01 22:42:04 -07:00
|
|
|
} else {
|
2018-05-12 15:39:30 -07:00
|
|
|
ps.PRS.LastCommitRound = msg.LastCommitRound
|
|
|
|
ps.PRS.LastCommit = nil
|
2014-11-01 22:42:04 -07:00
|
|
|
}
|
2014-10-26 00:09:22 -07:00
|
|
|
// We'll update the BitArray capacity later.
|
2018-05-12 15:39:30 -07:00
|
|
|
ps.PRS.CatchupCommitRound = -1
|
|
|
|
ps.PRS.CatchupCommit = nil
|
2014-09-14 15:37:32 -07:00
|
|
|
}
|
2014-08-10 16:35:08 -07:00
|
|
|
}
|
|
|
|
|
2018-10-31 14:20:36 +01:00
|
|
|
// ApplyNewValidBlockMessage updates the peer state for the new valid block.
|
|
|
|
func (ps *PeerState) ApplyNewValidBlockMessage(msg *NewValidBlockMessage) {
|
2014-11-01 04:04:58 -07:00
|
|
|
ps.mtx.Lock()
|
|
|
|
defer ps.mtx.Unlock()
|
|
|
|
|
2018-05-12 15:39:30 -07:00
|
|
|
if ps.PRS.Height != msg.Height {
|
2014-11-01 04:04:58 -07:00
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2018-10-31 14:20:36 +01:00
|
|
|
if ps.PRS.Round != msg.Round && !msg.IsCommit {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2018-05-12 15:39:30 -07:00
|
|
|
ps.PRS.ProposalBlockPartsHeader = msg.BlockPartsHeader
|
|
|
|
ps.PRS.ProposalBlockParts = msg.BlockParts
|
2014-11-01 04:04:58 -07:00
|
|
|
}
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// ApplyProposalPOLMessage updates the peer state for the new proposal POL.
|
2016-09-05 17:33:02 -07:00
|
|
|
func (ps *PeerState) ApplyProposalPOLMessage(msg *ProposalPOLMessage) {
|
2014-09-07 02:21:25 -07:00
|
|
|
ps.mtx.Lock()
|
|
|
|
defer ps.mtx.Unlock()
|
2014-10-26 00:09:22 -07:00
|
|
|
|
2018-05-12 15:39:30 -07:00
|
|
|
if ps.PRS.Height != msg.Height {
|
2014-10-26 00:09:22 -07:00
|
|
|
return
|
|
|
|
}
|
2018-05-12 15:39:30 -07:00
|
|
|
if ps.PRS.ProposalPOLRound != msg.ProposalPOLRound {
|
2016-09-05 17:33:02 -07:00
|
|
|
return
|
|
|
|
}
|
2014-10-26 00:09:22 -07:00
|
|
|
|
2018-05-12 15:39:30 -07:00
|
|
|
// TODO: Merge onto existing ps.PRS.ProposalPOL?
|
2016-09-05 17:33:02 -07:00
|
|
|
// We might have sent some prevotes in the meantime.
|
2018-05-12 15:39:30 -07:00
|
|
|
ps.PRS.ProposalPOL = msg.ProposalPOL
|
2014-09-07 02:21:25 -07:00
|
|
|
}
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// ApplyHasVoteMessage updates the peer state for the new vote.
|
2016-09-05 17:33:02 -07:00
|
|
|
func (ps *PeerState) ApplyHasVoteMessage(msg *HasVoteMessage) {
|
2015-06-22 19:04:31 -07:00
|
|
|
ps.mtx.Lock()
|
|
|
|
defer ps.mtx.Unlock()
|
|
|
|
|
2018-05-12 15:39:30 -07:00
|
|
|
if ps.PRS.Height != msg.Height {
|
2015-06-22 19:04:31 -07:00
|
|
|
return
|
|
|
|
}
|
2016-09-05 17:33:02 -07:00
|
|
|
|
|
|
|
ps.setHasVote(msg.Height, msg.Round, msg.Type, msg.Index)
|
|
|
|
}
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// ApplyVoteSetBitsMessage updates the peer state for the bit-array of votes
|
|
|
|
// it claims to have for the corresponding BlockID.
|
|
|
|
// `ourVotes` is a BitArray of votes we have for msg.BlockID
|
2016-09-05 17:33:02 -07:00
|
|
|
// NOTE: if ourVotes is nil (e.g. msg.Height < rs.Height),
|
|
|
|
// we conservatively overwrite ps's votes w/ msg.Votes.
|
2017-07-09 18:38:59 -04:00
|
|
|
func (ps *PeerState) ApplyVoteSetBitsMessage(msg *VoteSetBitsMessage, ourVotes *cmn.BitArray) {
|
2016-09-05 17:33:02 -07:00
|
|
|
ps.mtx.Lock()
|
|
|
|
defer ps.mtx.Unlock()
|
|
|
|
|
|
|
|
votes := ps.getVoteBitArray(msg.Height, msg.Round, msg.Type)
|
|
|
|
if votes != nil {
|
|
|
|
if ourVotes == nil {
|
|
|
|
votes.Update(msg.Votes)
|
|
|
|
} else {
|
|
|
|
otherVotes := votes.Sub(ourVotes)
|
|
|
|
hasVotes := otherVotes.Or(msg.Votes)
|
|
|
|
votes.Update(hasVotes)
|
|
|
|
}
|
|
|
|
}
|
2015-06-22 19:04:31 -07:00
|
|
|
}
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// String returns a string representation of the PeerState
|
2016-09-15 16:01:01 -07:00
|
|
|
func (ps *PeerState) String() string {
|
|
|
|
return ps.StringIndented("")
|
|
|
|
}
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// StringIndented returns a string representation of the PeerState
|
2016-09-15 16:01:01 -07:00
|
|
|
func (ps *PeerState) StringIndented(indent string) string {
|
2017-12-16 19:16:08 -05:00
|
|
|
ps.mtx.Lock()
|
|
|
|
defer ps.mtx.Unlock()
|
2016-09-15 16:01:01 -07:00
|
|
|
return fmt.Sprintf(`PeerState{
|
2018-05-10 22:43:21 -07:00
|
|
|
%s Key %v
|
|
|
|
%s RoundState %v
|
|
|
|
%s Stats %v
|
2016-09-15 16:01:01 -07:00
|
|
|
%s}`,
|
2018-05-10 22:43:21 -07:00
|
|
|
indent, ps.peer.ID(),
|
2018-05-12 15:39:30 -07:00
|
|
|
indent, ps.PRS.StringIndented(indent+" "),
|
2018-05-10 22:43:21 -07:00
|
|
|
indent, ps.Stats,
|
2016-09-15 16:01:01 -07:00
|
|
|
indent)
|
|
|
|
}
|
|
|
|
|
2014-08-10 16:35:08 -07:00
|
|
|
//-----------------------------------------------------------------------------
|
|
|
|
// Messages
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// ConsensusMessage is a message that can be sent and received on the ConsensusReactor
|
2018-11-01 07:07:18 +01:00
|
|
|
type ConsensusMessage interface {
|
|
|
|
ValidateBasic() error
|
|
|
|
}
|
2015-04-14 15:57:16 -07:00
|
|
|
|
2018-04-05 07:05:45 -07:00
|
|
|
func RegisterConsensusMessages(cdc *amino.Codec) {
|
|
|
|
cdc.RegisterInterface((*ConsensusMessage)(nil), nil)
|
|
|
|
cdc.RegisterConcrete(&NewRoundStepMessage{}, "tendermint/NewRoundStepMessage", nil)
|
2018-10-31 14:20:36 +01:00
|
|
|
cdc.RegisterConcrete(&NewValidBlockMessage{}, "tendermint/NewValidBlockMessage", nil)
|
2018-04-05 07:05:45 -07:00
|
|
|
cdc.RegisterConcrete(&ProposalMessage{}, "tendermint/Proposal", nil)
|
|
|
|
cdc.RegisterConcrete(&ProposalPOLMessage{}, "tendermint/ProposalPOL", nil)
|
|
|
|
cdc.RegisterConcrete(&BlockPartMessage{}, "tendermint/BlockPart", nil)
|
|
|
|
cdc.RegisterConcrete(&VoteMessage{}, "tendermint/Vote", nil)
|
|
|
|
cdc.RegisterConcrete(&HasVoteMessage{}, "tendermint/HasVote", nil)
|
|
|
|
cdc.RegisterConcrete(&VoteSetMaj23Message{}, "tendermint/VoteSetMaj23", nil)
|
|
|
|
cdc.RegisterConcrete(&VoteSetBitsMessage{}, "tendermint/VoteSetBits", nil)
|
|
|
|
}
|
2015-04-14 15:57:16 -07:00
|
|
|
|
2018-07-09 13:01:23 +04:00
|
|
|
func decodeMsg(bz []byte) (msg ConsensusMessage, err error) {
|
2018-04-09 15:14:33 +03:00
|
|
|
if len(bz) > maxMsgSize {
|
2018-07-09 13:01:23 +04:00
|
|
|
return msg, fmt.Errorf("Msg exceeds max size (%d > %d)", len(bz), maxMsgSize)
|
2018-04-09 15:14:33 +03:00
|
|
|
}
|
2018-04-05 07:05:45 -07:00
|
|
|
err = cdc.UnmarshalBinaryBare(bz, &msg)
|
2014-09-08 16:18:50 -07:00
|
|
|
return
|
2014-08-10 16:35:08 -07:00
|
|
|
}
|
|
|
|
|
|
|
|
//-------------------------------------
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// NewRoundStepMessage is sent for every step taken in the ConsensusState.
|
2015-06-19 15:30:10 -07:00
|
|
|
// For every height/round/step transition
|
2014-09-14 15:37:32 -07:00
|
|
|
type NewRoundStepMessage struct {
|
2017-12-01 19:04:53 -06:00
|
|
|
Height int64
|
2015-06-25 20:28:34 -07:00
|
|
|
Round int
|
2017-10-10 12:39:21 +04:00
|
|
|
Step cstypes.RoundStepType
|
2015-06-25 20:28:34 -07:00
|
|
|
SecondsSinceStartTime int
|
|
|
|
LastCommitRound int
|
2014-08-10 16:35:08 -07:00
|
|
|
}
|
|
|
|
|
2018-11-01 07:07:18 +01:00
|
|
|
// ValidateBasic performs basic validation.
|
|
|
|
func (m *NewRoundStepMessage) ValidateBasic() error {
|
|
|
|
if m.Height < 0 {
|
|
|
|
return errors.New("Negative Height")
|
|
|
|
}
|
|
|
|
if m.Round < 0 {
|
|
|
|
return errors.New("Negative Round")
|
|
|
|
}
|
|
|
|
if !m.Step.IsValid() {
|
|
|
|
return errors.New("Invalid Step")
|
|
|
|
}
|
|
|
|
|
|
|
|
// NOTE: SecondsSinceStartTime may be negative
|
|
|
|
|
|
|
|
if (m.Height == 1 && m.LastCommitRound != -1) ||
|
|
|
|
(m.Height > 1 && m.LastCommitRound < -1) { // TODO: #2737 LastCommitRound should always be >= 0 for heights > 1
|
|
|
|
return errors.New("Invalid LastCommitRound (for 1st block: -1, for others: >= 0)")
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// String returns a string representation.
|
2014-09-14 15:37:32 -07:00
|
|
|
func (m *NewRoundStepMessage) String() string {
|
2015-06-19 15:30:10 -07:00
|
|
|
return fmt.Sprintf("[NewRoundStep H:%v R:%v S:%v LCR:%v]",
|
|
|
|
m.Height, m.Round, m.Step, m.LastCommitRound)
|
2014-08-10 16:35:08 -07:00
|
|
|
}
|
|
|
|
|
|
|
|
//-------------------------------------
|
|
|
|
|
2018-10-31 15:59:01 +01:00
|
|
|
// NewValidBlockMessage is sent when a validator observes a valid block B in some round r,
|
|
|
|
//i.e., there is a Proposal for block B and 2/3+ prevotes for the block B in the round r.
|
|
|
|
// In case the block is also committed, then IsCommit flag is set to true.
|
2018-10-31 14:20:36 +01:00
|
|
|
type NewValidBlockMessage struct {
|
2017-12-01 19:04:53 -06:00
|
|
|
Height int64
|
2018-10-31 14:20:36 +01:00
|
|
|
Round int
|
2015-06-22 19:04:31 -07:00
|
|
|
BlockPartsHeader types.PartSetHeader
|
2017-07-09 18:38:59 -04:00
|
|
|
BlockParts *cmn.BitArray
|
2018-10-31 14:20:36 +01:00
|
|
|
IsCommit bool
|
2014-11-01 04:04:58 -07:00
|
|
|
}
|
|
|
|
|
2018-11-01 07:07:18 +01:00
|
|
|
// ValidateBasic performs basic validation.
|
|
|
|
func (m *NewValidBlockMessage) ValidateBasic() error {
|
|
|
|
if m.Height < 0 {
|
|
|
|
return errors.New("Negative Height")
|
|
|
|
}
|
|
|
|
if m.Round < 0 {
|
|
|
|
return errors.New("Negative Round")
|
|
|
|
}
|
|
|
|
if err := m.BlockPartsHeader.ValidateBasic(); err != nil {
|
|
|
|
return fmt.Errorf("Wrong BlockPartsHeader: %v", err)
|
|
|
|
}
|
|
|
|
if m.BlockParts.Size() != m.BlockPartsHeader.Total {
|
|
|
|
return fmt.Errorf("BlockParts bit array size %d not equal to BlockPartsHeader.Total %d",
|
|
|
|
m.BlockParts.Size(),
|
|
|
|
m.BlockPartsHeader.Total)
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// String returns a string representation.
|
2018-10-31 14:20:36 +01:00
|
|
|
func (m *NewValidBlockMessage) String() string {
|
|
|
|
return fmt.Sprintf("[ValidBlockMessage H:%v R:%v BP:%v BA:%v IsCommit:%v]",
|
|
|
|
m.Height, m.Round, m.BlockPartsHeader, m.BlockParts, m.IsCommit)
|
2014-11-01 04:04:58 -07:00
|
|
|
}
|
|
|
|
|
|
|
|
//-------------------------------------
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// ProposalMessage is sent when a new block is proposed.
|
2015-04-12 18:44:06 -07:00
|
|
|
type ProposalMessage struct {
|
2015-08-10 20:38:45 -07:00
|
|
|
Proposal *types.Proposal
|
2015-04-12 18:44:06 -07:00
|
|
|
}
|
|
|
|
|
2018-11-01 07:07:18 +01:00
|
|
|
// ValidateBasic performs basic validation.
|
|
|
|
func (m *ProposalMessage) ValidateBasic() error {
|
|
|
|
return m.Proposal.ValidateBasic()
|
|
|
|
}
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// String returns a string representation.
|
2015-04-12 18:44:06 -07:00
|
|
|
func (m *ProposalMessage) String() string {
|
|
|
|
return fmt.Sprintf("[Proposal %v]", m.Proposal)
|
|
|
|
}
|
|
|
|
|
|
|
|
//-------------------------------------
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// ProposalPOLMessage is sent when a previous proposal is re-proposed.
|
2015-06-22 19:04:31 -07:00
|
|
|
type ProposalPOLMessage struct {
|
2017-12-01 19:04:53 -06:00
|
|
|
Height int64
|
2015-06-25 20:28:34 -07:00
|
|
|
ProposalPOLRound int
|
2017-07-09 18:38:59 -04:00
|
|
|
ProposalPOL *cmn.BitArray
|
2015-06-22 19:04:31 -07:00
|
|
|
}
|
|
|
|
|
2018-11-01 07:07:18 +01:00
|
|
|
// ValidateBasic performs basic validation.
|
|
|
|
func (m *ProposalPOLMessage) ValidateBasic() error {
|
|
|
|
if m.Height < 0 {
|
|
|
|
return errors.New("Negative Height")
|
|
|
|
}
|
|
|
|
if m.ProposalPOLRound < 0 {
|
|
|
|
return errors.New("Negative ProposalPOLRound")
|
|
|
|
}
|
|
|
|
if m.ProposalPOL.Size() == 0 {
|
|
|
|
return errors.New("Empty ProposalPOL bit array")
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// String returns a string representation.
|
2015-06-22 19:04:31 -07:00
|
|
|
func (m *ProposalPOLMessage) String() string {
|
|
|
|
return fmt.Sprintf("[ProposalPOL H:%v POLR:%v POL:%v]", m.Height, m.ProposalPOLRound, m.ProposalPOL)
|
|
|
|
}
|
2014-09-14 15:37:32 -07:00
|
|
|
|
2015-06-22 19:04:31 -07:00
|
|
|
//-------------------------------------
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// BlockPartMessage is sent when gossipping a piece of the proposed block.
|
2015-06-22 19:04:31 -07:00
|
|
|
type BlockPartMessage struct {
|
2017-12-01 19:04:53 -06:00
|
|
|
Height int64
|
2015-06-25 20:28:34 -07:00
|
|
|
Round int
|
2015-03-22 19:00:08 -07:00
|
|
|
Part *types.Part
|
2014-08-10 16:35:08 -07:00
|
|
|
}
|
|
|
|
|
2018-11-01 07:07:18 +01:00
|
|
|
// ValidateBasic performs basic validation.
|
|
|
|
func (m *BlockPartMessage) ValidateBasic() error {
|
|
|
|
if m.Height < 0 {
|
|
|
|
return errors.New("Negative Height")
|
|
|
|
}
|
|
|
|
if m.Round < 0 {
|
|
|
|
return errors.New("Negative Round")
|
|
|
|
}
|
|
|
|
if err := m.Part.ValidateBasic(); err != nil {
|
|
|
|
return fmt.Errorf("Wrong Part: %v", err)
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// String returns a string representation.
|
2015-06-22 19:04:31 -07:00
|
|
|
func (m *BlockPartMessage) String() string {
|
2015-06-26 17:14:40 -07:00
|
|
|
return fmt.Sprintf("[BlockPart H:%v R:%v P:%v]", m.Height, m.Round, m.Part)
|
2014-08-10 16:35:08 -07:00
|
|
|
}
|
|
|
|
|
2014-12-09 18:49:04 -08:00
|
|
|
//-------------------------------------
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// VoteMessage is sent when voting for a proposal (or lack thereof).
|
2014-12-09 18:49:04 -08:00
|
|
|
type VoteMessage struct {
|
2016-07-01 17:47:31 -04:00
|
|
|
Vote *types.Vote
|
2014-08-10 16:35:08 -07:00
|
|
|
}
|
|
|
|
|
2018-11-01 07:07:18 +01:00
|
|
|
// ValidateBasic performs basic validation.
|
|
|
|
func (m *VoteMessage) ValidateBasic() error {
|
|
|
|
return m.Vote.ValidateBasic()
|
|
|
|
}
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// String returns a string representation.
|
2014-12-09 18:49:04 -08:00
|
|
|
func (m *VoteMessage) String() string {
|
2016-07-01 17:47:31 -04:00
|
|
|
return fmt.Sprintf("[Vote %v]", m.Vote)
|
2014-08-10 16:35:08 -07:00
|
|
|
}
|
2014-11-01 22:42:04 -07:00
|
|
|
|
|
|
|
//-------------------------------------
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// HasVoteMessage is sent to indicate that a particular vote has been received.
|
2014-11-01 22:42:04 -07:00
|
|
|
type HasVoteMessage struct {
|
2017-12-01 19:04:53 -06:00
|
|
|
Height int64
|
2015-06-25 20:28:34 -07:00
|
|
|
Round int
|
2018-10-13 01:21:46 +02:00
|
|
|
Type types.SignedMsgType
|
2015-06-25 20:28:34 -07:00
|
|
|
Index int
|
2014-11-01 22:42:04 -07:00
|
|
|
}
|
|
|
|
|
2018-11-01 07:07:18 +01:00
|
|
|
// ValidateBasic performs basic validation.
|
|
|
|
func (m *HasVoteMessage) ValidateBasic() error {
|
|
|
|
if m.Height < 0 {
|
|
|
|
return errors.New("Negative Height")
|
|
|
|
}
|
|
|
|
if m.Round < 0 {
|
|
|
|
return errors.New("Negative Round")
|
|
|
|
}
|
|
|
|
if !types.IsVoteTypeValid(m.Type) {
|
|
|
|
return errors.New("Invalid Type")
|
|
|
|
}
|
|
|
|
if m.Index < 0 {
|
|
|
|
return errors.New("Negative Index")
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// String returns a string representation.
|
2014-11-01 22:42:04 -07:00
|
|
|
func (m *HasVoteMessage) String() string {
|
2017-12-15 23:08:05 -05:00
|
|
|
return fmt.Sprintf("[HasVote VI:%v V:{%v/%02d/%v}]", m.Index, m.Height, m.Round, m.Type)
|
2014-11-01 22:42:04 -07:00
|
|
|
}
|
2016-09-05 17:33:02 -07:00
|
|
|
|
|
|
|
//-------------------------------------
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// VoteSetMaj23Message is sent to indicate that a given BlockID has seen +2/3 votes.
|
2016-09-05 17:33:02 -07:00
|
|
|
type VoteSetMaj23Message struct {
|
2017-12-01 19:04:53 -06:00
|
|
|
Height int64
|
2016-09-05 17:33:02 -07:00
|
|
|
Round int
|
2018-10-13 01:21:46 +02:00
|
|
|
Type types.SignedMsgType
|
2016-09-05 17:33:02 -07:00
|
|
|
BlockID types.BlockID
|
|
|
|
}
|
|
|
|
|
2018-11-01 07:07:18 +01:00
|
|
|
// ValidateBasic performs basic validation.
|
|
|
|
func (m *VoteSetMaj23Message) ValidateBasic() error {
|
|
|
|
if m.Height < 0 {
|
|
|
|
return errors.New("Negative Height")
|
|
|
|
}
|
|
|
|
if m.Round < 0 {
|
|
|
|
return errors.New("Negative Round")
|
|
|
|
}
|
|
|
|
if !types.IsVoteTypeValid(m.Type) {
|
|
|
|
return errors.New("Invalid Type")
|
|
|
|
}
|
|
|
|
if err := m.BlockID.ValidateBasic(); err != nil {
|
|
|
|
return fmt.Errorf("Wrong BlockID: %v", err)
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// String returns a string representation.
|
2016-09-05 17:33:02 -07:00
|
|
|
func (m *VoteSetMaj23Message) String() string {
|
|
|
|
return fmt.Sprintf("[VSM23 %v/%02d/%v %v]", m.Height, m.Round, m.Type, m.BlockID)
|
|
|
|
}
|
|
|
|
|
|
|
|
//-------------------------------------
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// VoteSetBitsMessage is sent to communicate the bit-array of votes seen for the BlockID.
|
2016-09-05 17:33:02 -07:00
|
|
|
type VoteSetBitsMessage struct {
|
2017-12-01 19:04:53 -06:00
|
|
|
Height int64
|
2016-09-05 17:33:02 -07:00
|
|
|
Round int
|
2018-10-13 01:21:46 +02:00
|
|
|
Type types.SignedMsgType
|
2016-09-05 17:33:02 -07:00
|
|
|
BlockID types.BlockID
|
2017-07-09 18:38:59 -04:00
|
|
|
Votes *cmn.BitArray
|
2016-09-05 17:33:02 -07:00
|
|
|
}
|
|
|
|
|
2018-11-01 07:07:18 +01:00
|
|
|
// ValidateBasic performs basic validation.
|
|
|
|
func (m *VoteSetBitsMessage) ValidateBasic() error {
|
|
|
|
if m.Height < 0 {
|
|
|
|
return errors.New("Negative Height")
|
|
|
|
}
|
|
|
|
if m.Round < 0 {
|
|
|
|
return errors.New("Negative Round")
|
|
|
|
}
|
|
|
|
if !types.IsVoteTypeValid(m.Type) {
|
|
|
|
return errors.New("Invalid Type")
|
|
|
|
}
|
|
|
|
if err := m.BlockID.ValidateBasic(); err != nil {
|
|
|
|
return fmt.Errorf("Wrong BlockID: %v", err)
|
|
|
|
}
|
|
|
|
// NOTE: Votes.Size() can be zero if the node does not have any
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2017-07-09 18:38:59 -04:00
|
|
|
// String returns a string representation.
|
2016-09-05 17:33:02 -07:00
|
|
|
func (m *VoteSetBitsMessage) String() string {
|
|
|
|
return fmt.Sprintf("[VSB %v/%02d/%v %v %v]", m.Height, m.Round, m.Type, m.BlockID, m.Votes)
|
|
|
|
}
|
2017-07-20 15:09:44 -04:00
|
|
|
|
|
|
|
//-------------------------------------
|