You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
 
 
 
 
 
 

456 lines
14 KiB

package consensus
import (
"context"
"fmt"
"os"
"path"
"sync"
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
dbm "github.com/tendermint/tm-db"
abcicli "github.com/tendermint/tendermint/abci/client"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/evidence"
"github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/libs/service"
tmsync "github.com/tendermint/tendermint/libs/sync"
mempl "github.com/tendermint/tendermint/mempool"
"github.com/tendermint/tendermint/p2p"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/store"
"github.com/tendermint/tendermint/types"
)
//----------------------------------------------
// byzantine failures
// Byzantine node sends two different prevotes (nil and blockID) to the same validator
func TestByzantinePrevoteEquivocation(t *testing.T) {
const nValidators = 4
const byzantineNode = 0
testName := "consensus_byzantine_test"
tickerFunc := newMockTickerFunc(true)
appFunc := newCounter
genDoc, privVals := randGenesisDoc(nValidators, false, 30)
css := make([]*State, nValidators)
for i := 0; i < nValidators; i++ {
logger := consensusLogger().With("test", "byzantine", "validator", i)
stateDB := dbm.NewMemDB() // each state needs its own db
state, _ := sm.LoadStateFromDBOrGenesisDoc(stateDB, genDoc)
thisConfig := ResetConfig(fmt.Sprintf("%s_%d", testName, i))
defer os.RemoveAll(thisConfig.RootDir)
ensureDir(path.Dir(thisConfig.Consensus.WalFile()), 0700) // dir for wal
app := appFunc()
vals := types.TM2PB.ValidatorUpdates(state.Validators)
app.InitChain(abci.RequestInitChain{Validators: vals})
blockDB := dbm.NewMemDB()
blockStore := store.NewBlockStore(blockDB)
// one for mempool, one for consensus
mtx := new(tmsync.Mutex)
proxyAppConnMem := abcicli.NewLocalClient(mtx, app)
proxyAppConnCon := abcicli.NewLocalClient(mtx, app)
// Make Mempool
mempool := mempl.NewCListMempool(thisConfig.Mempool, proxyAppConnMem, 0)
mempool.SetLogger(log.TestingLogger().With("module", "mempool"))
if thisConfig.Consensus.WaitForTxs() {
mempool.EnableTxsAvailable()
}
// Make a full instance of the evidence pool
evidenceDB := dbm.NewMemDB()
evpool, err := evidence.NewPool(evidenceDB, evidence.NewEvidenceStateStore(stateDB), blockStore)
require.NoError(t, err)
evpool.SetLogger(logger.With("module", "evidence"))
// Make State
blockExec := sm.NewBlockExecutor(stateDB, log.TestingLogger(), proxyAppConnCon, mempool, evpool)
cs := NewState(thisConfig.Consensus, state, blockExec, blockStore, mempool, evpool)
cs.SetLogger(cs.Logger)
// set private validator
pv := privVals[i]
cs.SetPrivValidator(pv)
eventBus := types.NewEventBus()
eventBus.SetLogger(log.TestingLogger().With("module", "events"))
eventBus.Start()
cs.SetEventBus(eventBus)
cs.SetTimeoutTicker(tickerFunc())
cs.SetLogger(logger)
css[i] = cs
}
// initialize the reactors for each of the validators
reactors := make([]*Reactor, nValidators)
blocksSubs := make([]types.Subscription, 0)
eventBuses := make([]*types.EventBus, nValidators)
for i := 0; i < nValidators; i++ {
reactors[i] = NewReactor(css[i], true) // so we dont start the consensus states
reactors[i].SetLogger(css[i].Logger)
// eventBus is already started with the cs
eventBuses[i] = css[i].eventBus
reactors[i].SetEventBus(eventBuses[i])
blocksSub, err := eventBuses[i].Subscribe(context.Background(), testSubscriber, types.EventQueryNewBlock)
require.NoError(t, err)
blocksSubs = append(blocksSubs, blocksSub)
if css[i].state.LastBlockHeight == 0 { //simulate handle initChain in handshake
sm.SaveState(css[i].blockExec.DB(), css[i].state)
}
}
// make connected switches and start all reactors
p2p.MakeConnectedSwitches(config.P2P, nValidators, func(i int, s *p2p.Switch) *p2p.Switch {
s.AddReactor("CONSENSUS", reactors[i])
s.SetLogger(reactors[i].conS.Logger.With("module", "p2p"))
return s
}, p2p.Connect2Switches)
// create byzantine validator
bcs := css[byzantineNode]
// alter prevote so that the byzantine node double votes when height is 2
bcs.doPrevote = func(height int64, round int32) {
// allow first height to happen normally so that byzantine validator is no longer proposer
if height == 2 {
bcs.Logger.Info("Sending two votes")
prevote1, err := bcs.signVote(tmproto.PrevoteType, bcs.ProposalBlock.Hash(), bcs.ProposalBlockParts.Header())
require.NoError(t, err)
prevote2, err := bcs.signVote(tmproto.PrevoteType, nil, types.PartSetHeader{})
require.NoError(t, err)
peerList := reactors[byzantineNode].Switch.Peers().List()
bcs.Logger.Info("Getting peer list", "peers", peerList)
// send two votes to all peers (1st to one half, 2nd to another half)
for i, peer := range peerList {
if i < len(peerList)/2 {
bcs.Logger.Info("Signed and pushed vote", "vote", prevote1, "peer", peer)
peer.Send(VoteChannel, MustEncode(&VoteMessage{prevote1}))
} else {
bcs.Logger.Info("Signed and pushed vote", "vote", prevote2, "peer", peer)
peer.Send(VoteChannel, MustEncode(&VoteMessage{prevote2}))
}
}
} else {
bcs.Logger.Info("Behaving normally")
bcs.defaultDoPrevote(height, round)
}
}
// start the consensus reactors
for i := 0; i < nValidators; i++ {
s := reactors[i].conS.GetState()
reactors[i].SwitchToConsensus(s, false)
}
defer stopConsensusNet(log.TestingLogger(), reactors, eventBuses)
// Evidence should be submitted and committed at the third height but
// we will check the first five just in case
var evidence types.Evidence
for i := 0; i < 5; i++ {
msg := <-blocksSubs[nValidators-1].Out()
block := msg.Data().(types.EventDataNewBlock).Block
if len(block.Evidence.Evidence) > 0 {
evidence = block.Evidence.Evidence[0]
break
}
}
if assert.NotNil(t, evidence) {
ev, ok := evidence.(*types.DuplicateVoteEvidence)
assert.True(t, ok)
pubkey, _ := bcs.privValidator.GetPubKey()
assert.Equal(t, []byte(pubkey.Address()), ev.Address())
}
}
// 4 validators. 1 is byzantine. The other three are partitioned into A (1 val) and B (2 vals).
// byzantine validator sends conflicting proposals into A and B,
// and prevotes/precommits on both of them.
// B sees a commit, A doesn't.
// Heal partition and ensure A sees the commit
func TestByzantineConflictingProposalsWithPartition(t *testing.T) {
N := 4
logger := consensusLogger().With("test", "byzantine")
app := newCounter
css, cleanup := randConsensusNet(N, "consensus_byzantine_test", newMockTickerFunc(false), app)
defer cleanup()
// give the byzantine validator a normal ticker
ticker := NewTimeoutTicker()
ticker.SetLogger(css[0].Logger)
css[0].SetTimeoutTicker(ticker)
switches := make([]*p2p.Switch, N)
p2pLogger := logger.With("module", "p2p")
for i := 0; i < N; i++ {
switches[i] = p2p.MakeSwitch(
config.P2P,
i,
"foo", "1.0.0",
func(i int, sw *p2p.Switch) *p2p.Switch {
return sw
})
switches[i].SetLogger(p2pLogger.With("validator", i))
}
blocksSubs := make([]types.Subscription, N)
reactors := make([]p2p.Reactor, N)
for i := 0; i < N; i++ {
// enable txs so we can create different proposals
assertMempool(css[i].txNotifier).EnableTxsAvailable()
// make first val byzantine
if i == 0 {
// NOTE: Now, test validators are MockPV, which by default doesn't
// do any safety checks.
css[i].privValidator.(types.MockPV).DisableChecks()
css[i].decideProposal = func(j int32) func(int64, int32) {
return func(height int64, round int32) {
byzantineDecideProposalFunc(t, height, round, css[j], switches[j])
}
}(int32(i))
// We are setting the prevote function to do nothing because the prevoting
// and precommitting are done alongside the proposal.
css[i].doPrevote = func(height int64, round int32) {}
}
eventBus := css[i].eventBus
eventBus.SetLogger(logger.With("module", "events", "validator", i))
var err error
blocksSubs[i], err = eventBus.Subscribe(context.Background(), testSubscriber, types.EventQueryNewBlock)
require.NoError(t, err)
conR := NewReactor(css[i], true) // so we don't start the consensus states
conR.SetLogger(logger.With("validator", i))
conR.SetEventBus(eventBus)
var conRI p2p.Reactor = conR
// make first val byzantine
if i == 0 {
conRI = NewByzantineReactor(conR)
}
reactors[i] = conRI
sm.SaveState(css[i].blockExec.DB(), css[i].state) //for save height 1's validators info
}
defer func() {
for _, r := range reactors {
if rr, ok := r.(*ByzantineReactor); ok {
rr.reactor.Switch.Stop()
} else {
r.(*Reactor).Switch.Stop()
}
}
}()
p2p.MakeConnectedSwitches(config.P2P, N, func(i int, s *p2p.Switch) *p2p.Switch {
// ignore new switch s, we already made ours
switches[i].AddReactor("CONSENSUS", reactors[i])
return switches[i]
}, func(sws []*p2p.Switch, i, j int) {
// the network starts partitioned with globally active adversary
if i != 0 {
return
}
p2p.Connect2Switches(sws, i, j)
})
// start the non-byz state machines.
// note these must be started before the byz
for i := 1; i < N; i++ {
cr := reactors[i].(*Reactor)
cr.SwitchToConsensus(cr.conS.GetState(), false)
}
// start the byzantine state machine
byzR := reactors[0].(*ByzantineReactor)
s := byzR.reactor.conS.GetState()
byzR.reactor.SwitchToConsensus(s, false)
// byz proposer sends one block to peers[0]
// and the other block to peers[1] and peers[2].
// note peers and switches order don't match.
peers := switches[0].Peers().List()
// partition A
ind0 := getSwitchIndex(switches, peers[0])
// partition B
ind1 := getSwitchIndex(switches, peers[1])
ind2 := getSwitchIndex(switches, peers[2])
p2p.Connect2Switches(switches, ind1, ind2)
// wait for someone in the big partition (B) to make a block
<-blocksSubs[ind2].Out()
t.Log("A block has been committed. Healing partition")
p2p.Connect2Switches(switches, ind0, ind1)
p2p.Connect2Switches(switches, ind0, ind2)
// wait till everyone makes the first new block
// (one of them already has)
wg := new(sync.WaitGroup)
wg.Add(2)
for i := 1; i < N-1; i++ {
go func(j int) {
<-blocksSubs[j].Out()
wg.Done()
}(i)
}
done := make(chan struct{})
go func() {
wg.Wait()
close(done)
}()
tick := time.NewTicker(time.Second * 10)
select {
case <-done:
case <-tick.C:
for i, reactor := range reactors {
t.Log(fmt.Sprintf("Consensus Reactor %v", i))
t.Log(fmt.Sprintf("%v", reactor))
}
t.Fatalf("Timed out waiting for all validators to commit first block")
}
}
//-------------------------------
// byzantine consensus functions
func byzantineDecideProposalFunc(t *testing.T, height int64, round int32, cs *State, sw *p2p.Switch) {
// byzantine user should create two proposals and try to split the vote.
// Avoid sending on internalMsgQueue and running consensus state.
// Create a new proposal block from state/txs from the mempool.
block1, blockParts1 := cs.createProposalBlock()
polRound, propBlockID := cs.ValidRound, types.BlockID{Hash: block1.Hash(), PartSetHeader: blockParts1.Header()}
proposal1 := types.NewProposal(height, round, polRound, propBlockID)
p1 := proposal1.ToProto()
if err := cs.privValidator.SignProposal(cs.state.ChainID, p1); err != nil {
t.Error(err)
}
proposal1.Signature = p1.Signature
// some new transactions come in (this ensures that the proposals are different)
deliverTxsRange(cs, 0, 1)
// Create a new proposal block from state/txs from the mempool.
block2, blockParts2 := cs.createProposalBlock()
polRound, propBlockID = cs.ValidRound, types.BlockID{Hash: block2.Hash(), PartSetHeader: blockParts2.Header()}
proposal2 := types.NewProposal(height, round, polRound, propBlockID)
p2 := proposal2.ToProto()
if err := cs.privValidator.SignProposal(cs.state.ChainID, p2); err != nil {
t.Error(err)
}
proposal2.Signature = p2.Signature
block1Hash := block1.Hash()
block2Hash := block2.Hash()
// broadcast conflicting proposals/block parts to peers
peers := sw.Peers().List()
t.Logf("Byzantine: broadcasting conflicting proposals to %d peers", len(peers))
for i, peer := range peers {
if i < len(peers)/2 {
go sendProposalAndParts(height, round, cs, peer, proposal1, block1Hash, blockParts1)
} else {
go sendProposalAndParts(height, round, cs, peer, proposal2, block2Hash, blockParts2)
}
}
}
func sendProposalAndParts(
height int64,
round int32,
cs *State,
peer p2p.Peer,
proposal *types.Proposal,
blockHash []byte,
parts *types.PartSet,
) {
// proposal
msg := &ProposalMessage{Proposal: proposal}
peer.Send(DataChannel, MustEncode(msg))
// parts
for i := 0; i < int(parts.Total()); i++ {
part := parts.GetPart(i)
msg := &BlockPartMessage{
Height: height, // This tells peer that this part applies to us.
Round: round, // This tells peer that this part applies to us.
Part: part,
}
peer.Send(DataChannel, MustEncode(msg))
}
// votes
cs.mtx.Lock()
prevote, _ := cs.signVote(tmproto.PrevoteType, blockHash, parts.Header())
precommit, _ := cs.signVote(tmproto.PrecommitType, blockHash, parts.Header())
cs.mtx.Unlock()
peer.Send(VoteChannel, MustEncode(&VoteMessage{prevote}))
peer.Send(VoteChannel, MustEncode(&VoteMessage{precommit}))
}
//----------------------------------------
// byzantine consensus reactor
type ByzantineReactor struct {
service.Service
reactor *Reactor
}
func NewByzantineReactor(conR *Reactor) *ByzantineReactor {
return &ByzantineReactor{
Service: conR,
reactor: conR,
}
}
func (br *ByzantineReactor) SetSwitch(s *p2p.Switch) { br.reactor.SetSwitch(s) }
func (br *ByzantineReactor) GetChannels() []*p2p.ChannelDescriptor { return br.reactor.GetChannels() }
func (br *ByzantineReactor) AddPeer(peer p2p.Peer) {
if !br.reactor.IsRunning() {
return
}
// Create peerState for peer
peerState := NewPeerState(peer).SetLogger(br.reactor.Logger)
peer.Set(types.PeerStateKey, peerState)
// Send our state to peer.
// If we're syncing, broadcast a RoundStepMessage later upon SwitchToConsensus().
if !br.reactor.waitSync {
br.reactor.sendNewRoundStepMessage(peer)
}
}
func (br *ByzantineReactor) RemovePeer(peer p2p.Peer, reason interface{}) {
br.reactor.RemovePeer(peer, reason)
}
func (br *ByzantineReactor) Receive(chID byte, peer p2p.Peer, msgBytes []byte) {
br.reactor.Receive(chID, peer, msgBytes)
}
func (br *ByzantineReactor) InitPeer(peer p2p.Peer) p2p.Peer { return peer }