Browse Source

Merge branch 'mempool' into develop

pull/148/merge
Jae Kwon 9 years ago
parent
commit
252a7ed7d6
11 changed files with 504 additions and 45 deletions
  1. +10
    -4
      consensus/reactor.go
  2. +5
    -1
      consensus/state.go
  3. +13
    -0
      mempool/config.go
  4. +57
    -21
      mempool/mempool.go
  5. +273
    -0
      mempool/mempool_test.go
  6. +132
    -17
      mempool/reactor.go
  7. +4
    -0
      p2p/peer.go
  8. +1
    -1
      rpc/core/consensus.go
  9. +2
    -1
      rpc/core/mempool.go
  10. +1
    -0
      rpc/core/types/responses.go
  11. +6
    -0
      types/keys.go

+ 10
- 4
consensus/reactor.go View File

@ -22,8 +22,6 @@ const (
DataChannel = byte(0x21)
VoteChannel = byte(0x22)
PeerStateKey = "ConsensusReactor.peerState"
peerGossipSleepDuration = 100 * time.Millisecond // Time to sleep if there's nothing to send.
)
@ -107,7 +105,7 @@ func (conR *ConsensusReactor) AddPeer(peer *p2p.Peer) {
// Create peerState for peer
peerState := NewPeerState(peer)
peer.Data.Set(PeerStateKey, peerState)
peer.Data.Set(types.PeerStateKey, peerState)
// Begin gossip routines for this peer.
go conR.gossipDataRoutine(peer, peerState)
@ -138,7 +136,7 @@ func (conR *ConsensusReactor) Receive(chID byte, peer *p2p.Peer, msgBytes []byte
}
// Get peer states
ps := peer.Data.Get(PeerStateKey).(*PeerState)
ps := peer.Data.Get(types.PeerStateKey).(*PeerState)
_, msg, err := DecodeMessage(msgBytes)
if err != nil {
log.Warn("Error decoding message", "channel", chID, "peer", peer, "msg", msg, "error", err, "bytes", msgBytes)
@ -588,6 +586,14 @@ func (ps *PeerState) GetRoundState() *PeerRoundState {
return &prs
}
// Returns an atomic snapshot of the PeerRoundState's height
// used by the mempool to ensure peers are caught up before broadcasting new txs
func (ps *PeerState) GetHeight() int {
ps.mtx.Lock()
defer ps.mtx.Unlock()
return ps.PeerRoundState.Height
}
func (ps *PeerState) SetHasProposal(proposal *types.Proposal) {
ps.mtx.Lock()
defer ps.mtx.Unlock()


+ 5
- 1
consensus/state.go View File

@ -657,6 +657,10 @@ func (cs *ConsensusState) createProposalBlock() (block *types.Block, blockParts
return
}
txs := cs.mempoolReactor.Mempool.GetProposalTxs()
MaxTxsPerBlock := 100 // TODO
if len(txs) > MaxTxsPerBlock {
txs = txs[:MaxTxsPerBlock]
}
block = &types.Block{
Header: &types.Header{
ChainID: cs.state.ChainID,
@ -1308,7 +1312,7 @@ func (cs *ConsensusState) saveBlock(block *types.Block, blockParts *types.PartSe
cs.stagedState.Save()
// Update mempool.
cs.mempoolReactor.Mempool.ResetForBlockAndState(block, cs.stagedState)
cs.mempoolReactor.ResetForBlockAndState(block, cs.stagedState)
// Fire off event
if cs.evsw != nil && cs.evc != nil {


+ 13
- 0
mempool/config.go View File

@ -0,0 +1,13 @@
package mempool
import (
cfg "github.com/tendermint/tendermint/config"
)
var config cfg.Config = nil
func init() {
cfg.OnConfig(func(newConfig cfg.Config) {
config = newConfig
})
}

+ 57
- 21
mempool/mempool.go View File

@ -19,7 +19,7 @@ type Mempool struct {
mtx sync.Mutex
state *sm.State
cache *sm.BlockCache
txs []types.Tx
txs []types.Tx // TODO: we need to add a map to facilitate replace-by-fee
}
func NewMempool(state *sm.State) *Mempool {
@ -37,6 +37,12 @@ func (mem *Mempool) GetCache() *sm.BlockCache {
return mem.cache
}
func (mem *Mempool) GetHeight() int {
mem.mtx.Lock()
defer mem.mtx.Unlock()
return mem.state.LastBlockHeight
}
// Apply tx to the state and remember it.
func (mem *Mempool) AddTx(tx types.Tx) (err error) {
mem.mtx.Lock()
@ -59,11 +65,23 @@ func (mem *Mempool) GetProposalTxs() []types.Tx {
return mem.txs
}
// We use this to inform peer routines of how the mempool has been updated
type ResetInfo struct {
Height int
Included []Range
Invalid []Range
}
type Range struct {
Start int
Length int
}
// "block" is the new block being committed.
// "state" is the result of state.AppendBlock("block").
// Txs that are present in "block" are discarded from mempool.
// Txs that have become invalid in the new "state" are also discarded.
func (mem *Mempool) ResetForBlockAndState(block *types.Block, state *sm.State) {
func (mem *Mempool) ResetForBlockAndState(block *types.Block, state *sm.State) ResetInfo {
mem.mtx.Lock()
defer mem.mtx.Unlock()
mem.state = state.Copy()
@ -75,33 +93,51 @@ func (mem *Mempool) ResetForBlockAndState(block *types.Block, state *sm.State) {
blockTxsMap[string(types.TxID(state.ChainID, tx))] = struct{}{}
}
// Next, filter all txs from mem.txs that are in blockTxsMap
txs := []types.Tx{}
for _, tx := range mem.txs {
// Now we filter all txs from mem.txs that are in blockTxsMap,
// and ExecTx on what remains. Only valid txs are kept.
// We track the ranges of txs included in the block and invalidated by it
// so we can tell peer routines
var ri = ResetInfo{Height: block.Height}
var validTxs []types.Tx
includedStart, invalidStart := -1, -1
for i, tx := range mem.txs {
txID := types.TxID(state.ChainID, tx)
if _, ok := blockTxsMap[string(txID)]; ok {
startRange(&includedStart, i) // start counting included txs
endRange(&invalidStart, i, &ri.Invalid) // stop counting invalid txs
log.Info("Filter out, already committed", "tx", tx, "txID", txID)
continue
} else {
log.Info("Filter in, still new", "tx", tx, "txID", txID)
txs = append(txs, tx)
}
}
// Next, filter all txs that aren't valid given new state.
validTxs := []types.Tx{}
for _, tx := range txs {
err := sm.ExecTx(mem.cache, tx, false, nil)
if err == nil {
log.Info("Filter in, valid", "tx", tx)
validTxs = append(validTxs, tx)
} else {
// tx is no longer valid.
log.Info("Filter out, no longer valid", "tx", tx, "error", err)
endRange(&includedStart, i, &ri.Included) // stop counting included txs
err := sm.ExecTx(mem.cache, tx, false, nil)
if err != nil {
startRange(&invalidStart, i) // start counting invalid txs
log.Info("Filter out, no longer valid", "tx", tx, "error", err)
} else {
endRange(&invalidStart, i, &ri.Invalid) // stop counting invalid txs
log.Info("Filter in, new, valid", "tx", tx, "txID", txID)
validTxs = append(validTxs, tx)
}
}
}
endRange(&includedStart, len(mem.txs)-1, &ri.Included) // stop counting included txs
endRange(&invalidStart, len(mem.txs)-1, &ri.Invalid) // stop counting invalid txs
// We're done!
log.Info("New txs", "txs", validTxs, "oldTxs", mem.txs)
mem.txs = validTxs
return ri
}
func startRange(start *int, i int) {
if *start < 0 {
*start = i
}
}
func endRange(start *int, i int, ranger *[]Range) {
if *start >= 0 {
length := i - *start
*ranger = append(*ranger, Range{*start, length})
*start = -1
}
}

+ 273
- 0
mempool/mempool_test.go View File

@ -0,0 +1,273 @@
package mempool
import (
"fmt"
"sync"
"testing"
"time"
acm "github.com/tendermint/tendermint/account"
_ "github.com/tendermint/tendermint/config/tendermint_test"
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
)
var someAddr = []byte("ABCDEFGHIJABCDEFGHIJ")
// number of txs
var nTxs = 100
// what the ResetInfo should look like after ResetForBlockAndState
var TestResetInfoData = ResetInfo{
Included: []Range{
Range{0, 5},
Range{10, 10},
Range{30, 5},
},
Invalid: []Range{
Range{5, 5},
Range{20, 8}, // let 28 and 29 be valid
Range{35, 64}, // let 99 be valid
},
}
// inverse of the ResetInfo
var notInvalidNotIncluded = map[int]struct{}{
28: struct{}{},
29: struct{}{},
99: struct{}{},
}
func newSendTx(t *testing.T, mempool *Mempool, from *acm.PrivAccount, to []byte, amt int64) types.Tx {
tx := types.NewSendTx()
tx.AddInput(mempool.GetCache(), from.PubKey, amt)
tx.AddOutput(to, amt)
tx.SignInput(config.GetString("chain_id"), 0, from)
if err := mempool.AddTx(tx); err != nil {
t.Fatal(err)
}
return tx
}
func addTxs(t *testing.T, mempool *Mempool, lastAcc *acm.PrivAccount, privAccs []*acm.PrivAccount) []types.Tx {
txs := make([]types.Tx, nTxs)
for i := 0; i < nTxs; i++ {
if _, ok := notInvalidNotIncluded[i]; ok {
txs[i] = newSendTx(t, mempool, lastAcc, someAddr, 10)
} else {
txs[i] = newSendTx(t, mempool, privAccs[i%len(privAccs)], privAccs[(i+1)%len(privAccs)].Address, 5)
}
}
return txs
}
func makeBlock(mempool *Mempool) *types.Block {
txs := mempool.GetProposalTxs()
var includedTxs []types.Tx
for _, rid := range TestResetInfoData.Included {
includedTxs = append(includedTxs, txs[rid.Start:rid.Start+rid.Length]...)
}
mempool.mtx.Lock()
state := mempool.state
state.LastBlockHeight += 1
mempool.mtx.Unlock()
return &types.Block{
Header: &types.Header{
ChainID: state.ChainID,
Height: state.LastBlockHeight,
NumTxs: len(includedTxs),
},
Data: &types.Data{
Txs: includedTxs,
},
}
}
// Add txs. Grab chunks to put in block. All the others become invalid because of nonce errors except those in notInvalidNotIncluded
func TestResetInfo(t *testing.T) {
amtPerAccount := int64(100000)
state, privAccs, _ := sm.RandGenesisState(6, false, amtPerAccount, 1, true, 100)
mempool := NewMempool(state)
lastAcc := privAccs[5] // we save him (his tx wont become invalid)
privAccs = privAccs[:5]
txs := addTxs(t, mempool, lastAcc, privAccs)
// its actually an invalid block since we're skipping nonces
// but all we care about is how the mempool responds after
block := makeBlock(mempool)
ri := mempool.ResetForBlockAndState(block, state)
if len(ri.Included) != len(TestResetInfoData.Included) {
t.Fatalf("invalid number of included ranges. Got %d, expected %d\n", len(ri.Included), len(TestResetInfoData.Included))
}
if len(ri.Invalid) != len(TestResetInfoData.Invalid) {
t.Fatalf("invalid number of invalid ranges. Got %d, expected %d\n", len(ri.Invalid), len(TestResetInfoData.Invalid))
}
for i, rid := range ri.Included {
inc := TestResetInfoData.Included[i]
if rid.Start != inc.Start {
t.Fatalf("Invalid start of range. Got %d, expected %d\n", inc.Start, rid.Start)
}
if rid.Length != inc.Length {
t.Fatalf("Invalid length of range. Got %d, expected %d\n", inc.Length, rid.Length)
}
}
txs = mempool.GetProposalTxs()
if len(txs) != len(notInvalidNotIncluded) {
t.Fatalf("Expected %d txs left in mempool. Got %d", len(notInvalidNotIncluded), len(txs))
}
}
//------------------------------------------------------------------------------------------
type TestPeer struct {
sync.Mutex
running bool
height int
t *testing.T
received int
txs map[string]int
timeoutFail int
done chan int
}
func newPeer(t *testing.T, state *sm.State) *TestPeer {
return &TestPeer{
running: true,
height: state.LastBlockHeight,
t: t,
txs: make(map[string]int),
done: make(chan int),
}
}
func (tp *TestPeer) IsRunning() bool {
tp.Lock()
defer tp.Unlock()
return tp.running
}
func (tp *TestPeer) SetRunning(running bool) {
tp.Lock()
defer tp.Unlock()
tp.running = running
}
func (tp *TestPeer) Send(chID byte, msg interface{}) bool {
if tp.timeoutFail > 0 {
time.Sleep(time.Second * time.Duration(tp.timeoutFail))
return false
}
tx := msg.(*TxMessage).Tx
id := types.TxID(config.GetString("chain_id"), tx)
if _, ok := tp.txs[string(id)]; ok {
tp.t.Fatal("received the same tx twice!")
}
tp.txs[string(id)] = tp.received
tp.received += 1
tp.done <- tp.received
return true
}
func (tp *TestPeer) Get(key string) interface{} {
return tp
}
func (tp *TestPeer) GetHeight() int {
return tp.height
}
func TestBroadcast(t *testing.T) {
state, privAccs, _ := sm.RandGenesisState(6, false, 10000, 1, true, 100)
mempool := NewMempool(state)
reactor := NewMempoolReactor(mempool)
reactor.Start()
lastAcc := privAccs[5] // we save him (his tx wont become invalid)
privAccs = privAccs[:5]
peer := newPeer(t, state)
newBlockChan := make(chan ResetInfo)
tickerChan := make(chan time.Time)
go reactor.broadcastTxRoutine(tickerChan, newBlockChan, peer)
// we don't broadcast any before updating
fmt.Println("dont broadcast any")
addTxs(t, mempool, lastAcc, privAccs)
block := makeBlock(mempool)
ri := mempool.ResetForBlockAndState(block, state)
newBlockChan <- ri
peer.height = ri.Height
tickerChan <- time.Now()
pullTxs(t, peer, len(mempool.txs)) // should have sent whatever txs are left (3)
toBroadcast := []int{1, 3, 7, 9, 11, 12, 18, 20, 21, 28, 29, 30, 31, 34, 35, 36, 50, 90, 99, 100}
for _, N := range toBroadcast {
peer = resetPeer(t, reactor, mempool, state, tickerChan, newBlockChan, peer)
// we broadcast N txs before updating
fmt.Println("broadcast", N)
addTxs(t, mempool, lastAcc, privAccs)
txsToSendPerCheck = N
tickerChan <- time.Now()
pullTxs(t, peer, txsToSendPerCheck) // should have sent N txs
block = makeBlock(mempool)
ri := mempool.ResetForBlockAndState(block, state)
newBlockChan <- ri
peer.height = ri.Height
txsToSendPerCheck = 100
tickerChan <- time.Now()
left := len(mempool.txs)
if N > 99 {
left -= 3
} else if N > 29 {
left -= 2
} else if N > 28 {
left -= 1
}
pullTxs(t, peer, left) // should have sent whatever txs are left that havent been sent
}
}
func pullTxs(t *testing.T, peer *TestPeer, N int) {
timer := time.NewTicker(time.Second * 2)
for i := 0; i < N; i++ {
select {
case <-peer.done:
case <-timer.C:
panic(fmt.Sprintf("invalid number of received messages. Got %d, expected %d\n", i, N))
}
}
if N == 0 {
select {
case <-peer.done:
t.Fatalf("should not have sent any more txs")
case <-timer.C:
}
}
}
func resetPeer(t *testing.T, reactor *MempoolReactor, mempool *Mempool, state *sm.State, tickerChan chan time.Time, newBlockChan chan ResetInfo, peer *TestPeer) *TestPeer {
// reset peer
mempool.txs = []types.Tx{}
mempool.state = state
mempool.cache = sm.NewBlockCache(state)
peer.SetRunning(false)
tickerChan <- time.Now()
peer = newPeer(t, state)
go reactor.broadcastTxRoutine(tickerChan, newBlockChan, peer)
return peer
}

+ 132
- 17
mempool/reactor.go View File

@ -2,18 +2,25 @@ package mempool
import (
"bytes"
"errors"
"fmt"
"reflect"
"time"
. "github.com/tendermint/tendermint/common"
"github.com/tendermint/tendermint/events"
"github.com/tendermint/tendermint/p2p"
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
"github.com/tendermint/tendermint/wire"
)
var (
MempoolChannel = byte(0x30)
checkExecutedTxsMilliseconds = 1 // check for new mempool txs to send to peer
txsToSendPerCheck = 64 // send up to this many txs from the mempool per check
newBlockChCapacity = 100 // queue to process this many ResetInfos per peer
)
// MempoolReactor handles mempool tx broadcasting amongst peers.
@ -44,11 +51,17 @@ func (memR *MempoolReactor) GetChannels() []*p2p.ChannelDescriptor {
}
// Implements Reactor
func (pexR *MempoolReactor) AddPeer(peer *p2p.Peer) {
func (memR *MempoolReactor) AddPeer(peer *p2p.Peer) {
// Each peer gets a go routine on which we broadcast transactions in the same order we applied them to our state.
newBlockChan := make(chan ResetInfo, newBlockChCapacity)
peer.Data.Set(types.PeerMempoolChKey, newBlockChan)
timer := time.NewTicker(time.Millisecond * time.Duration(checkExecutedTxsMilliseconds))
go memR.broadcastTxRoutine(timer.C, newBlockChan, peer)
}
// Implements Reactor
func (pexR *MempoolReactor) RemovePeer(peer *p2p.Peer, reason interface{}) {
func (memR *MempoolReactor) RemovePeer(peer *p2p.Peer, reason interface{}) {
// broadcast routine checks if peer is gone and returns
}
// Implements Reactor
@ -70,29 +83,131 @@ func (memR *MempoolReactor) Receive(chID byte, src *p2p.Peer, msgBytes []byte) {
} else {
log.Info("Added valid tx", "tx", msg.Tx)
}
// Share tx.
// We use a simple shotgun approach for now.
// TODO: improve efficiency
for _, peer := range memR.Switch.Peers().List() {
if peer.Key == src.Key {
// broadcasting happens from go routines per peer
default:
log.Warn(Fmt("Unknown message type %v", reflect.TypeOf(msg)))
}
}
// "block" is the new block being committed.
// "state" is the result of state.AppendBlock("block").
// Txs that are present in "block" are discarded from mempool.
// Txs that have become invalid in the new "state" are also discarded.
func (memR *MempoolReactor) ResetForBlockAndState(block *types.Block, state *sm.State) {
ri := memR.Mempool.ResetForBlockAndState(block, state)
for _, peer := range memR.Switch.Peers().List() {
peerMempoolCh := peer.Data.Get(types.PeerMempoolChKey).(chan ResetInfo)
select {
case peerMempoolCh <- ri:
default:
memR.Switch.StopPeerForError(peer, errors.New("Peer's mempool push channel full"))
}
}
}
// Just an alias for AddTx since broadcasting happens in peer routines
func (memR *MempoolReactor) BroadcastTx(tx types.Tx) error {
return memR.Mempool.AddTx(tx)
}
type PeerState interface {
GetHeight() int
}
type Peer interface {
IsRunning() bool
Send(byte, interface{}) bool
Get(string) interface{}
}
// send new mempool txs to peer, strictly in order we applied them to our state.
// new blocks take chunks out of the mempool, but we've already sent some txs to the peer.
// so we wait to hear that the peer has progressed to the new height, and then continue sending txs from where we left off
func (memR *MempoolReactor) broadcastTxRoutine(tickerChan <-chan time.Time, newBlockChan chan ResetInfo, peer Peer) {
currentHeight := memR.Mempool.GetHeight()
var nTxs, txsSent int
var txs []types.Tx
for {
select {
case <-tickerChan:
if !peer.IsRunning() {
return
}
// make sure the peer is up to date
peerState := peer.Get(types.PeerStateKey).(PeerState)
if peerState.GetHeight() < currentHeight {
continue
}
peer.TrySend(MempoolChannel, msg)
// check the mempool for new transactions
nTxs, txs = memR.getNewTxs(txsSent, currentHeight)
theseTxsSent := 0
start := time.Now()
TX_LOOP:
for _, tx := range txs {
// send tx to peer.
msg := &TxMessage{Tx: tx}
success := peer.Send(MempoolChannel, msg)
if !success {
break TX_LOOP
} else {
theseTxsSent += 1
}
}
if theseTxsSent > 0 {
txsSent += theseTxsSent
log.Info("Sent txs to peer", "ntxs", theseTxsSent, "took", time.Since(start), "total_sent", txsSent, "total_exec", nTxs)
}
case ri := <-newBlockChan:
currentHeight = ri.Height
// find out how many txs below what we've sent were included in a block and how many became invalid
included := tallyRangesUpTo(ri.Included, txsSent)
invalidated := tallyRangesUpTo(ri.Invalid, txsSent)
txsSent -= included + invalidated
}
}
}
default:
log.Warn(Fmt("Unknown message type %v", reflect.TypeOf(msg)))
// fetch new txs from the mempool
func (memR *MempoolReactor) getNewTxs(txsSent, height int) (nTxs int, txs []types.Tx) {
memR.Mempool.mtx.Lock()
defer memR.Mempool.mtx.Unlock()
// if the mempool got ahead of us just return empty txs
if memR.Mempool.state.LastBlockHeight != height {
return
}
nTxs = len(memR.Mempool.txs)
if txsSent < nTxs {
if nTxs > txsSent+txsToSendPerCheck {
txs = memR.Mempool.txs[txsSent : txsSent+txsToSendPerCheck]
} else {
txs = memR.Mempool.txs[txsSent:]
}
}
return
}
func (memR *MempoolReactor) BroadcastTx(tx types.Tx) error {
err := memR.Mempool.AddTx(tx)
if err != nil {
return err
// return the size of ranges less than upTo
func tallyRangesUpTo(ranger []Range, upTo int) int {
totalUpTo := 0
for _, r := range ranger {
if r.Start >= upTo {
break
}
if r.Start+r.Length >= upTo {
totalUpTo += upTo - r.Start
break
}
totalUpTo += r.Length
}
msg := &TxMessage{Tx: tx}
memR.Switch.Broadcast(MempoolChannel, msg)
return nil
return totalUpTo
}
// implements events.Eventable


+ 4
- 0
p2p/peer.go View File

@ -128,3 +128,7 @@ func (p *Peer) String() string {
func (p *Peer) Equals(other *Peer) bool {
return p.Key == other.Key
}
func (p *Peer) Get(key string) interface{} {
return p.Data.Get(key)
}

+ 1
- 1
rpc/core/consensus.go View File

@ -31,7 +31,7 @@ func DumpConsensusState() (*ctypes.ResultDumpConsensusState, error) {
peerRoundStates := []string{}
for _, peer := range p2pSwitch.Peers().List() {
// TODO: clean this up?
peerState := peer.Data.Get(cm.PeerStateKey).(*cm.PeerState)
peerState := peer.Data.Get(types.PeerStateKey).(*cm.PeerState)
peerRoundState := peerState.GetRoundState()
peerRoundStateStr := peer.Key + ":" + string(wire.JSONBytes(peerRoundState))
peerRoundStates = append(peerRoundStates, peerRoundStateStr)


+ 2
- 1
rpc/core/mempool.go View File

@ -30,5 +30,6 @@ func BroadcastTx(tx types.Tx) (*ctypes.ResultBroadcastTx, error) {
}
func ListUnconfirmedTxs() (*ctypes.ResultListUnconfirmedTxs, error) {
return &ctypes.ResultListUnconfirmedTxs{mempoolReactor.Mempool.GetProposalTxs()}, nil
txs := mempoolReactor.Mempool.GetProposalTxs()
return &ctypes.ResultListUnconfirmedTxs{len(txs), txs}, nil
}

+ 1
- 0
rpc/core/types/responses.go View File

@ -98,6 +98,7 @@ type Receipt struct {
}
type ResultListUnconfirmedTxs struct {
N int `json:"n_txs"`
Txs []types.Tx `json:"txs"`
}


+ 6
- 0
types/keys.go View File

@ -0,0 +1,6 @@
package types
var (
PeerStateKey = "ConsensusReactor.peerState"
PeerMempoolChKey = "MempoolReactor.peerMempoolCh"
)

Loading…
Cancel
Save