refactor(mempool): refactor code according to comments
This commit is contained in:
parent
f328d1d6a1
commit
2aed7f6ecd
|
@ -1,6 +1,6 @@
|
|||
[raft]
|
||||
batch_timeout = "0.3s" # Block packaging time period.
|
||||
rebroadcast_timeout = "3m" # Node timeout tick to check if there exist txs to be rebroadcast
|
||||
check_interval = "3m" # Node check interval to check if there exist txs to be rebroadcast
|
||||
tick_timeout = "0.1s" # TickTimeout is the internal logical clock for the Node by a single tick, Election timeouts and heartbeat timeouts are in units of ticks.
|
||||
election_tick = 10 # ElectionTick is the number of Node.Tick invocations that must pass between elections.
|
||||
heartbeat_tick = 1 # HeartbeatTick is the number of Node.Tick invocations that must pass between heartbeats.
|
||||
|
|
|
@ -27,7 +27,7 @@ type SyncerConfig struct {
|
|||
|
||||
type RAFT struct {
|
||||
BatchTimeout time.Duration `mapstructure:"batch_timeout"`
|
||||
RebroadcastTimeout time.Duration `mapstructure:"rebroadcast_timeout"`
|
||||
CheckInterval time.Duration `mapstructure:"check_interval"`
|
||||
TickTimeout time.Duration `mapstructure:"tick_timeout"`
|
||||
ElectionTick int `mapstructure:"election_tick"`
|
||||
HeartbeatTick int `mapstructure:"heartbeat_tick"`
|
||||
|
|
|
@ -39,15 +39,15 @@ type Node struct {
|
|||
txCache *mempool.TxCache // cache the transactions received from api
|
||||
batchTimerMgr *BatchTimer
|
||||
|
||||
proposeC chan *raftproto.RequestBatch // proposed ready, input channel
|
||||
confChangeC <-chan raftpb.ConfChange // proposed cluster config changes
|
||||
commitC chan *pb.CommitEvent // the hash commit channel
|
||||
errorC chan<- error // errors from raft session
|
||||
tickTimeout time.Duration // tick timeout
|
||||
rebroadcastTimeout time.Duration // rebroadcast timeout
|
||||
msgC chan []byte // receive messages from remote peer
|
||||
stateC chan *mempool.ChainState // receive the executed block state
|
||||
rebroadcastTicker chan *raftproto.TxSlice // receive the executed block state
|
||||
proposeC chan *raftproto.RequestBatch // proposed ready, input channel
|
||||
confChangeC <-chan raftpb.ConfChange // proposed cluster config changes
|
||||
commitC chan *pb.CommitEvent // the hash commit channel
|
||||
errorC chan<- error // errors from raft session
|
||||
tickTimeout time.Duration // tick timeout
|
||||
checkInterval time.Duration // interval for rebroadcast
|
||||
msgC chan []byte // receive messages from remote peer
|
||||
stateC chan *mempool.ChainState // receive the executed block state
|
||||
rebroadcastTicker chan *raftproto.TxSlice // receive the executed block state
|
||||
|
||||
confState raftpb.ConfState // raft requires ConfState to be persisted within snapshot
|
||||
blockAppliedIndex sync.Map // mapping of block height and apply index in raft log
|
||||
|
@ -122,36 +122,36 @@ func NewNode(opts ...order.Option) (order.Order, error) {
|
|||
snapCount = DefaultSnapshotCount
|
||||
}
|
||||
|
||||
var rebroadcastTimeout time.Duration
|
||||
if raftConfig.RAFT.RebroadcastTimeout == 0 {
|
||||
rebroadcastTimeout = DefaultRebroadcastTimeout
|
||||
var checkInterval time.Duration
|
||||
if raftConfig.RAFT.CheckInterval == 0 {
|
||||
checkInterval = DefaultCheckInterval
|
||||
} else {
|
||||
rebroadcastTimeout = raftConfig.RAFT.RebroadcastTimeout
|
||||
checkInterval = raftConfig.RAFT.CheckInterval
|
||||
}
|
||||
|
||||
node := &Node{
|
||||
id: config.ID,
|
||||
lastExec: config.Applied,
|
||||
confChangeC: make(chan raftpb.ConfChange),
|
||||
commitC: make(chan *pb.CommitEvent, 1024),
|
||||
errorC: make(chan<- error),
|
||||
msgC: make(chan []byte),
|
||||
stateC: make(chan *mempool.ChainState),
|
||||
proposeC: make(chan *raftproto.RequestBatch),
|
||||
snapCount: snapCount,
|
||||
repoRoot: repoRoot,
|
||||
peerMgr: config.PeerMgr,
|
||||
txCache: txCache,
|
||||
batchTimerMgr: batchTimerMgr,
|
||||
peers: peers,
|
||||
logger: config.Logger,
|
||||
getChainMetaFunc: config.GetChainMetaFunc,
|
||||
storage: dbStorage,
|
||||
raftStorage: raftStorage,
|
||||
readyPool: readyPool,
|
||||
ctx: context.Background(),
|
||||
mempool: mempoolInst,
|
||||
rebroadcastTimeout: rebroadcastTimeout,
|
||||
id: config.ID,
|
||||
lastExec: config.Applied,
|
||||
confChangeC: make(chan raftpb.ConfChange),
|
||||
commitC: make(chan *pb.CommitEvent, 1024),
|
||||
errorC: make(chan<- error),
|
||||
msgC: make(chan []byte),
|
||||
stateC: make(chan *mempool.ChainState),
|
||||
proposeC: make(chan *raftproto.RequestBatch),
|
||||
snapCount: snapCount,
|
||||
repoRoot: repoRoot,
|
||||
peerMgr: config.PeerMgr,
|
||||
txCache: txCache,
|
||||
batchTimerMgr: batchTimerMgr,
|
||||
peers: peers,
|
||||
logger: config.Logger,
|
||||
getChainMetaFunc: config.GetChainMetaFunc,
|
||||
storage: dbStorage,
|
||||
raftStorage: raftStorage,
|
||||
readyPool: readyPool,
|
||||
ctx: context.Background(),
|
||||
mempool: mempoolInst,
|
||||
checkInterval: checkInterval,
|
||||
}
|
||||
node.raftStorage.SnapshotCatchUpEntries = node.snapCount
|
||||
|
||||
|
@ -260,7 +260,7 @@ func (n *Node) run() {
|
|||
n.snapshotIndex = snap.Metadata.Index
|
||||
n.appliedIndex = snap.Metadata.Index
|
||||
ticker := time.NewTicker(n.tickTimeout)
|
||||
rebroadcastTicker := time.NewTicker(n.rebroadcastTimeout)
|
||||
rebroadcastTicker := time.NewTicker(n.checkInterval)
|
||||
defer ticker.Stop()
|
||||
defer rebroadcastTicker.Stop()
|
||||
|
||||
|
@ -328,7 +328,7 @@ func (n *Node) run() {
|
|||
|
||||
case <-rebroadcastTicker.C:
|
||||
// check periodically if there are long-pending txs in mempool
|
||||
rebroadcastTxs := n.mempool.GetTimeoutTransactions(n.rebroadcastTimeout)
|
||||
rebroadcastTxs := n.mempool.GetTimeoutTransactions(n.checkInterval)
|
||||
for _, txSlice := range rebroadcastTxs {
|
||||
txSet := &raftproto.TxSlice{TxList: txSlice}
|
||||
data, err := txSet.Marshal()
|
||||
|
|
|
@ -14,9 +14,9 @@ import (
|
|||
)
|
||||
|
||||
const (
|
||||
DefaultBatchTick = 500 * time.Millisecond
|
||||
DefaultSnapshotCount = 1000
|
||||
DefaultRebroadcastTimeout = 3 * time.Minute
|
||||
DefaultBatchTick = 500 * time.Millisecond
|
||||
DefaultSnapshotCount = 1000
|
||||
DefaultCheckInterval = 3 * time.Minute
|
||||
)
|
||||
|
||||
func generateRaftPeers(config *order.Config) ([]raft.Peer, error) {
|
||||
|
|
|
@ -24,17 +24,6 @@ func (oik *orderedIndexKey) Less(than btree.Item) bool {
|
|||
return oik.nonce < other.nonce
|
||||
}
|
||||
|
||||
type sortedTtlKey struct {
|
||||
account string
|
||||
nonce uint64
|
||||
liveTime int64 // the latest live time of tx after it is received or rebroadcast
|
||||
}
|
||||
|
||||
func (stk *sortedTtlKey) Less(than btree.Item) bool {
|
||||
other := than.(*sortedTtlKey)
|
||||
return stk.liveTime < other.liveTime
|
||||
}
|
||||
|
||||
type sortedNonceKey struct {
|
||||
nonce uint64
|
||||
}
|
||||
|
@ -48,7 +37,7 @@ func (snk *sortedNonceKey) Less(item btree.Item) bool {
|
|||
type orderedTimeoutKey struct {
|
||||
account string
|
||||
nonce uint64
|
||||
timestamp int64 // the timestamp of tx when it is received
|
||||
timestamp int64 // the timestamp of index key created
|
||||
}
|
||||
|
||||
func (otk *orderedTimeoutKey) Less(than btree.Item) bool {
|
||||
|
|
|
@ -101,14 +101,10 @@ func (mpi *mempoolImpl) processDirtyAccount(dirtyAccounts map[string]bool) {
|
|||
readyTxs, nonReadyTxs, nextDemandNonce := list.filterReady(pendingNonce)
|
||||
mpi.txStore.nonceCache.setPendingNonce(account, nextDemandNonce)
|
||||
|
||||
// insert ready txs into priorityIndex and set ttlIndex for these txs.
|
||||
// insert ready txs into priorityIndex.
|
||||
for _, tx := range readyTxs {
|
||||
if !mpi.txStore.priorityIndex.data.Has(makeTimeoutKey(account, tx)) {
|
||||
mpi.txStore.priorityIndex.insertByTimeoutKey(account, tx)
|
||||
if list.items[tx.Nonce].local {
|
||||
// no need to rebroadcast tx from other nodes to reduce network overhead
|
||||
mpi.txStore.ttlIndex.insertByTtlKey(account, tx.Nonce, tx.Timestamp)
|
||||
}
|
||||
}
|
||||
}
|
||||
mpi.txStore.updateEarliestTimestamp()
|
||||
|
@ -281,67 +277,60 @@ func (mpi *mempoolImpl) processCommitTransactions(state *ChainState) {
|
|||
|
||||
func (mpi *mempoolImpl) GetTimeoutTransactions(rebroadcastDuration time.Duration) [][]*pb.Transaction {
|
||||
// all the tx whose live time is less than lowBoundTime should be rebroadcast
|
||||
mpi.logger.Debugf("------- Start gathering timeout txs, ttl index len is %d -----------", mpi.txStore.ttlIndex.index.Len())
|
||||
mpi.logger.Debugf("Start gathering timeout txs, ttl index len is %d", mpi.txStore.ttlIndex.index.Len())
|
||||
currentTime := time.Now().UnixNano()
|
||||
if currentTime < mpi.txStore.earliestTimestamp+rebroadcastDuration.Nanoseconds() {
|
||||
// if the latest incoming tx has not exceeded the timeout limit, then none will be timeout
|
||||
return [][]*pb.Transaction{}
|
||||
}
|
||||
|
||||
txList := make([]*pb.Transaction, 0)
|
||||
timeoutItems := make([]*sortedTtlKey, 0)
|
||||
timeoutItems := make([]*orderedTimeoutKey, 0)
|
||||
mpi.txStore.ttlIndex.index.Ascend(func(i btree.Item) bool {
|
||||
item := i.(*sortedTtlKey)
|
||||
if item.liveTime > math.MaxInt64 {
|
||||
item := i.(*orderedTimeoutKey)
|
||||
if item.timestamp > math.MaxInt64 {
|
||||
// TODO(tyx): if this tx has rebroadcast many times and exceeded a final limit,
|
||||
// it is expired and will be removed from mempool
|
||||
return true
|
||||
}
|
||||
// if this tx has not exceeded the rebroadcast duration, break iteration
|
||||
timeoutTime := item.liveTime + rebroadcastDuration.Nanoseconds()
|
||||
txMap, ok := mpi.txStore.allTxs[item.account]
|
||||
timeoutTime := item.timestamp + rebroadcastDuration.Nanoseconds()
|
||||
_, ok := mpi.txStore.allTxs[item.account]
|
||||
if !ok || currentTime < timeoutTime {
|
||||
return false
|
||||
}
|
||||
tx := txMap.items[item.nonce].tx
|
||||
txList = append(txList, tx)
|
||||
timeoutItems = append(timeoutItems, item)
|
||||
return true
|
||||
})
|
||||
if len(txList) == 0 {
|
||||
return [][]*pb.Transaction{}
|
||||
}
|
||||
for _, item := range timeoutItems {
|
||||
// update the liveTime of timeout txs
|
||||
item.liveTime = currentTime
|
||||
item.timestamp = currentTime
|
||||
mpi.txStore.ttlIndex.items[makeAccountNonceKey(item.account, item.nonce)] = currentTime
|
||||
}
|
||||
// shard txList into fixed size in case txList is too large to broadcast one time
|
||||
return shardTxList(txList, mpi.txSliceSize)
|
||||
return mpi.shardTxList(timeoutItems, mpi.txSliceSize)
|
||||
}
|
||||
|
||||
func shardTxList(txs []*pb.Transaction, batchLen uint64) [][]*pb.Transaction {
|
||||
func (mpi *mempoolImpl) shardTxList(timeoutItems []*orderedTimeoutKey, batchLen uint64) [][]*pb.Transaction {
|
||||
begin := uint64(0)
|
||||
end := uint64(len(txs)) - 1
|
||||
totalLen := uint64(len(txs))
|
||||
end := uint64(len(timeoutItems)) - 1
|
||||
totalLen := uint64(len(timeoutItems))
|
||||
|
||||
// shape txs to fixed size in case totalLen is too large
|
||||
// shape timeout txs to batch size in case totalLen is too large
|
||||
batchNums := totalLen / batchLen
|
||||
if totalLen%batchLen != 0 {
|
||||
batchNums++
|
||||
}
|
||||
shardedLists := make([][]*pb.Transaction, 0, batchNums)
|
||||
for i := uint64(0); i <= batchNums; i++ {
|
||||
for i := uint64(0); i < batchNums; i++ {
|
||||
actualLen := batchLen
|
||||
if end-begin+1 < batchLen {
|
||||
actualLen = end - begin + 1
|
||||
}
|
||||
if actualLen == 0 {
|
||||
continue
|
||||
}
|
||||
|
||||
shardedList := make([]*pb.Transaction, actualLen)
|
||||
for j := uint64(0); j < batchLen && begin <= end; j++ {
|
||||
shardedList[j] = txs[begin]
|
||||
txMap, _ := mpi.txStore.allTxs[timeoutItems[begin].account]
|
||||
shardedList[j] = txMap.items[timeoutItems[begin].nonce].tx
|
||||
begin++
|
||||
}
|
||||
shardedLists = append(shardedLists, shardedList)
|
||||
|
|
|
@ -65,6 +65,10 @@ func (txStore *transactionStore) insertTxs(txs map[string][]*pb.Transaction, isL
|
|||
}
|
||||
txList.items[tx.Nonce] = txItem
|
||||
txList.index.insertBySortedNonceKey(tx)
|
||||
if isLocal {
|
||||
// no need to rebroadcast tx from other nodes to reduce network overhead
|
||||
txStore.ttlIndex.insertByTtlKey(account, tx.Nonce, tx.Timestamp)
|
||||
}
|
||||
}
|
||||
dirtyAccounts[account] = true
|
||||
}
|
||||
|
@ -88,7 +92,7 @@ func (txStore *transactionStore) updateEarliestTimestamp() {
|
|||
earliestTime := int64(math.MaxInt64)
|
||||
latestItem := txStore.ttlIndex.index.Min()
|
||||
if latestItem != nil {
|
||||
earliestTime = latestItem.(*sortedTtlKey).liveTime
|
||||
earliestTime = latestItem.(*orderedTimeoutKey).timestamp
|
||||
}
|
||||
txStore.earliestTimestamp = earliestTime
|
||||
}
|
||||
|
@ -205,7 +209,7 @@ func newTxLiveTimeMap() *txLiveTimeMap {
|
|||
}
|
||||
|
||||
func (tlm *txLiveTimeMap) insertByTtlKey(account string, nonce uint64, liveTime int64) {
|
||||
tlm.index.ReplaceOrInsert(&sortedTtlKey{account, nonce, liveTime})
|
||||
tlm.index.ReplaceOrInsert(&orderedTimeoutKey{account, nonce, liveTime})
|
||||
tlm.items[makeAccountNonceKey(account, nonce)] = liveTime
|
||||
}
|
||||
|
||||
|
@ -216,7 +220,7 @@ func (tlm *txLiveTimeMap) removeByTtlKey(txs map[string][]*pb.Transaction) {
|
|||
if !ok {
|
||||
return
|
||||
}
|
||||
tlm.index.Delete(&sortedTtlKey{account, tx.Nonce, liveTime})
|
||||
tlm.index.Delete(&orderedTimeoutKey{account, tx.Nonce, liveTime})
|
||||
delete(tlm.items, makeAccountNonceKey(account, tx.Nonce))
|
||||
}
|
||||
}
|
||||
|
|
|
@ -10,6 +10,7 @@ import (
|
|||
peer "github.com/libp2p/go-libp2p-core/peer"
|
||||
pb "github.com/meshplus/bitxhub-model/pb"
|
||||
events "github.com/meshplus/bitxhub/internal/model/events"
|
||||
peermgr "github.com/meshplus/bitxhub/pkg/peermgr"
|
||||
network "github.com/meshplus/go-lightp2p"
|
||||
reflect "reflect"
|
||||
)
|
||||
|
@ -227,3 +228,69 @@ func (mr *MockPeerManagerMockRecorder) Disconnect(vpInfos interface{}) *gomock.C
|
|||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Disconnect", reflect.TypeOf((*MockPeerManager)(nil).Disconnect), vpInfos)
|
||||
}
|
||||
|
||||
// PierManager mocks base method
|
||||
func (m *MockPeerManager) PierManager() peermgr.PierManager {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "PierManager")
|
||||
ret0, _ := ret[0].(peermgr.PierManager)
|
||||
return ret0
|
||||
}
|
||||
|
||||
// PierManager indicates an expected call of PierManager
|
||||
func (mr *MockPeerManagerMockRecorder) PierManager() *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PierManager", reflect.TypeOf((*MockPeerManager)(nil).PierManager))
|
||||
}
|
||||
|
||||
// MockPierManager is a mock of PierManager interface
|
||||
type MockPierManager struct {
|
||||
ctrl *gomock.Controller
|
||||
recorder *MockPierManagerMockRecorder
|
||||
}
|
||||
|
||||
// MockPierManagerMockRecorder is the mock recorder for MockPierManager
|
||||
type MockPierManagerMockRecorder struct {
|
||||
mock *MockPierManager
|
||||
}
|
||||
|
||||
// NewMockPierManager creates a new mock instance
|
||||
func NewMockPierManager(ctrl *gomock.Controller) *MockPierManager {
|
||||
mock := &MockPierManager{ctrl: ctrl}
|
||||
mock.recorder = &MockPierManagerMockRecorder{mock}
|
||||
return mock
|
||||
}
|
||||
|
||||
// EXPECT returns an object that allows the caller to indicate expected use
|
||||
func (m *MockPierManager) EXPECT() *MockPierManagerMockRecorder {
|
||||
return m.recorder
|
||||
}
|
||||
|
||||
// Piers mocks base method
|
||||
func (m *MockPierManager) Piers() *peermgr.Piers {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "Piers")
|
||||
ret0, _ := ret[0].(*peermgr.Piers)
|
||||
return ret0
|
||||
}
|
||||
|
||||
// Piers indicates an expected call of Piers
|
||||
func (mr *MockPierManagerMockRecorder) Piers() *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Piers", reflect.TypeOf((*MockPierManager)(nil).Piers))
|
||||
}
|
||||
|
||||
// AskPierMaster mocks base method
|
||||
func (m *MockPierManager) AskPierMaster(arg0 string) (bool, error) {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "AskPierMaster", arg0)
|
||||
ret0, _ := ret[0].(bool)
|
||||
ret1, _ := ret[1].(error)
|
||||
return ret0, ret1
|
||||
}
|
||||
|
||||
// AskPierMaster indicates an expected call of AskPierMaster
|
||||
func (mr *MockPierManagerMockRecorder) AskPierMaster(arg0 interface{}) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AskPierMaster", reflect.TypeOf((*MockPierManager)(nil).AskPierMaster), arg0)
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue