From 8e5d355b831ae74412e12a390bfdff01f3591ebe Mon Sep 17 00:00:00 2001 From: setunapo Date: Tue, 14 Dec 2021 17:27:53 +0800 Subject: [PATCH 1/8] Parallel: Kick off for BEP-130: Parallel Transaction Execution. Add a new interface StateProcessor.ProcessParallel(...), it is a copy of Process(...) right now. This patch is a placeholder, we will implement BEP-130 based on it. --- core/state_processor.go | 68 +++++++++++++++++++++++++++++++++++++++++ core/types.go | 3 ++ 2 files changed, 71 insertions(+) diff --git a/core/state_processor.go b/core/state_processor.go index 14fe9b4b92..9d7b1ac1f1 100644 --- a/core/state_processor.go +++ b/core/state_processor.go @@ -445,6 +445,74 @@ func (p *StateProcessor) Process(block *types.Block, statedb *state.StateDB, cfg return statedb, receipts, allLogs, *usedGas, nil } +func (p *StateProcessor) ProcessParallel(block *types.Block, statedb *state.StateDB, cfg vm.Config) (*state.StateDB, types.Receipts, []*types.Log, uint64, error) { + var ( + usedGas = new(uint64) + header = block.Header() + allLogs []*types.Log + gp = new(GasPool).AddGas(block.GasLimit()) + ) + signer := types.MakeSigner(p.bc.chainConfig, block.Number()) + statedb.TryPreload(block, signer) + var receipts = make([]*types.Receipt, 0) + // Mutate the block and state according to any hard-fork specs + if p.config.DAOForkSupport && p.config.DAOForkBlock != nil && p.config.DAOForkBlock.Cmp(block.Number()) == 0 { + misc.ApplyDAOHardFork(statedb) + } + // Handle upgrade build-in system contract code + systemcontracts.UpgradeBuildInSystemContract(p.config, block.Number(), statedb) + + blockContext := NewEVMBlockContext(header, p.bc, nil) + vmenv := vm.NewEVM(blockContext, vm.TxContext{}, statedb, p.config, cfg) + + txNum := len(block.Transactions()) + // Iterate over and process the individual transactions + posa, isPoSA := p.engine.(consensus.PoSA) + commonTxs := make([]*types.Transaction, 0, txNum) + + // initilise bloom processors + bloomProcessors := NewAsyncReceiptBloomGenerator(txNum) + statedb.MarkFullProcessed() + + // usually do have two tx, one for validator set contract, another for system reward contract. + systemTxs := make([]*types.Transaction, 0, 2) + for i, tx := range block.Transactions() { + if isPoSA { + if isSystemTx, err := posa.IsSystemTransaction(tx, block.Header()); err != nil { + return statedb, nil, nil, 0, err + } else if isSystemTx { + systemTxs = append(systemTxs, tx) + continue + } + } + + msg, err := tx.AsMessage(signer) + if err != nil { + return statedb, nil, nil, 0, err + } + statedb.Prepare(tx.Hash(), block.Hash(), i) + receipt, err := applyTransaction(msg, p.config, p.bc, nil, gp, statedb, header, tx, usedGas, vmenv, bloomProcessors) + if err != nil { + return statedb, nil, nil, 0, fmt.Errorf("could not apply tx %d [%v]: %w", i, tx.Hash().Hex(), err) + } + + commonTxs = append(commonTxs, tx) + receipts = append(receipts, receipt) + } + bloomProcessors.Close() + + // Finalize the block, applying any consensus engine specific extras (e.g. block rewards) + err := p.engine.Finalize(p.bc, header, statedb, &commonTxs, block.Uncles(), &receipts, &systemTxs, usedGas) + if err != nil { + return statedb, receipts, allLogs, *usedGas, err + } + for _, receipt := range receipts { + allLogs = append(allLogs, receipt.Logs...) + } + + return statedb, receipts, allLogs, *usedGas, nil +} + func applyTransaction(msg types.Message, config *params.ChainConfig, bc ChainContext, author *common.Address, gp *GasPool, statedb *state.StateDB, header *types.Header, tx *types.Transaction, usedGas *uint64, evm *vm.EVM, receiptProcessors ...ReceiptProcessor) (*types.Receipt, error) { // Create a new context to be used in the EVM environment. txContext := NewEVMTxContext(msg) diff --git a/core/types.go b/core/types.go index 5ed4817e68..c1e04a2b23 100644 --- a/core/types.go +++ b/core/types.go @@ -48,4 +48,7 @@ type Processor interface { // the transaction messages using the statedb and applying any rewards to both // the processor (coinbase) and any included uncles. Process(block *types.Block, statedb *state.StateDB, cfg vm.Config) (*state.StateDB, types.Receipts, []*types.Log, uint64, error) + + // ProcessParallel will implement BEP-130, run transactions concurrently. + ProcessParallel(block *types.Block, statedb *state.StateDB, cfg vm.Config) (*state.StateDB, types.Receipts, []*types.Log, uint64, error) } From 18e238f3415663f6b5825f0458e1d4dada01fd70 Mon Sep 17 00:00:00 2001 From: setunapo Date: Tue, 14 Dec 2021 19:32:07 +0800 Subject: [PATCH 2/8] Parallel: implement modules && workflow ** modules of init, slot executer and dispatcher BEP 130 parallel transaction execution will maintain a tx execution routine pool, a configured number of slot(routine) to execution transactions. Init is executed once on startup and will create the routine pool. Slot executer is the place to execute transactions. The dispacther is the module that will dispatch transaction to the right slot. ** workflow: Stage Apply, Conflict Detector, Slot, Gas... > two stages of applyTransaction For sequential execution, applyTransaction will do transaction execution and result finalization. > Conflict detector We will check the parallel execution result for each transaction. If there is a confliction, the result can not be committed, redo will be scheduled to update its StateDB and re-run For parallel execution, the execution result may not be reliable(conflict), use try-rerun policy, the transaction could be executed more than once to get the correct result. Once the result is confirm, we will finalize it to StateDB. Balance, KV, Account Create&Suicide... will be checked And conflict window is important for conflict check. > Slot StateDB Each slot will have a StateDB to execute transaction in slot. The world state changes are stored in this StateDB and merged to the main StateDB when transaction result is confirmed. SlotState.slotdbChan is the current execute TX's slotDB. And only dirty state object are allowed to merge back, otherwise, there is a race condition of merge outdated stateobject back. ** others gas pool, transaction gas, gas fee reward to system address evm instance, receipt CumulativeGasUsed & Log Index, contract creation, slot state, parallel routine safety: 1.only dispatcher can access main stateDB 2.slotDB will be created and merged to stateDB in dispatch goroutine. ** workflow 2: CopyForSlot, redesign dispatch, slot StateDB reuse & several bugfix > simplifiy statedb copy with CopyForSlot only copy dirtied state objects delete prefetcher ** redesign dispatch, slot StateDB reuse... > dispatch enhance remove atomic idle, curExec... replace by pendingExec for slot. >slot StateDB reuse It will try to reuse the latest merged slotDB in the same slot. If reuse failed(conflict), it will try to update to the latest world state and redo. The reuse SlotDB will the same BaseTxIndex, since its world state was sync when it was created based on that txIndex Conflict check can skip current slot now. it is more aggressive to reuse SlotDB for idle dispatch not only pending Txs but also the idle dispatched Txs try to reuse SlotDB now. ** others state change no needs to store value add "--parallel" startup options Parallel is not enabled by default. To enable it, just add a simple flag to geth: --parallel To config parallel execute parameter: --parallel.num 20 --parallel.queuesize 30 "--parallel.num" is the number of parallel slot to execute Tx, by default it is CPUNum-1 "--parallel.queuesize" is the maxpending queue size for each slot, by default it is 10 For example: ./build/bin/geth --parallel ./build/bin/geth --parallel --parallel.num 10 ./build/bin/geth --parallel --parallel.num 20 --parallel.queuesize 30 ** several BugFix 1.system address balance conflict We take system address as a special address, since each transaction will pay gas fee to it. Parallel execution reset its balance in slotDB, if a transaction try to access its balance, it will receive 0. If the contract needs the real system address balance, we will schedule a redo with real system address balance One transaction that accessed system address: https://bscscan.com/tx/0xcd69755be1d2f55af259441ff5ee2f312830b8539899e82488a21e85bc121a2a 2.fork caused by address state changed and read in same block 3.test case error 4.statedb.Copy should initialize parallel elements 5.do merge for snapshot --- cmd/geth/main.go | 3 + cmd/utils/flags.go | 24 ++ core/blockchain.go | 12 +- core/state/journal.go | 6 +- core/state/state_object.go | 10 + core/state/statedb.go | 524 ++++++++++++++++++++++++++++--- core/state_processor.go | 622 ++++++++++++++++++++++++++++++++++++- go.mod | 1 + metrics/exp/exp.go | 4 + 9 files changed, 1154 insertions(+), 52 deletions(-) diff --git a/cmd/geth/main.go b/cmd/geth/main.go index e0f29bce77..acea1fd10d 100644 --- a/cmd/geth/main.go +++ b/cmd/geth/main.go @@ -163,6 +163,9 @@ var ( utils.GpoMaxGasPriceFlag, utils.EWASMInterpreterFlag, utils.EVMInterpreterFlag, + utils.ParallelTxFlag, + utils.ParallelTxNumFlag, + utils.ParallelTxQueueSizeFlag, utils.MinerNotifyFullFlag, configFileFlag, utils.CatalystFlag, diff --git a/cmd/utils/flags.go b/cmd/utils/flags.go index 8f5141907f..36b90329d3 100644 --- a/cmd/utils/flags.go +++ b/cmd/utils/flags.go @@ -802,6 +802,20 @@ var ( Usage: "External EVM configuration (default = built-in interpreter)", Value: "", } + ParallelTxFlag = cli.BoolFlag{ + Name: "parallel", + Usage: "Enable the experimental parallel transaction execution mode (default = false)", + } + ParallelTxNumFlag = cli.IntFlag{ + Name: "parallel.num", + Usage: "Number of slot for transaction execution, only valid in parallel mode (default = CPUNum - 1)", + Value: core.ParallelExecNum, + } + ParallelTxQueueSizeFlag = cli.IntFlag{ + Name: "parallel.queuesize", + Usage: "Max number of Tx that can be queued to a slot, only valid in parallel mode (default = 10)", + Value: core.MaxPendingQueueSize, + } // Init network InitNetworkSize = cli.IntFlag{ @@ -1322,6 +1336,16 @@ func SetNodeConfig(ctx *cli.Context, cfg *node.Config) { if ctx.GlobalIsSet(InsecureUnlockAllowedFlag.Name) { cfg.InsecureUnlockAllowed = ctx.GlobalBool(InsecureUnlockAllowedFlag.Name) } + if ctx.GlobalIsSet(ParallelTxFlag.Name) { + core.ParallelTxMode = true + } + if ctx.GlobalIsSet(ParallelTxNumFlag.Name) { + core.ParallelExecNum = ctx.GlobalInt(ParallelTxNumFlag.Name) + } + if ctx.GlobalIsSet(ParallelTxQueueSizeFlag.Name) { + core.MaxPendingQueueSize = ctx.GlobalInt(ParallelTxQueueSizeFlag.Name) + } + } func setSmartCard(ctx *cli.Context, cfg *node.Config) { diff --git a/core/blockchain.go b/core/blockchain.go index 6c87ffc708..c3294ce8b8 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -79,6 +79,7 @@ var ( errInsertionInterrupted = errors.New("insertion is interrupted") errStateRootVerificationFailed = errors.New("state root verification failed") + ParallelTxMode = false // parallel transaction execution ) const ( @@ -2123,7 +2124,16 @@ func (bc *BlockChain) insertChain(chain types.Blocks, verifySeals bool) (int, er statedb.EnablePipeCommit() } statedb.SetExpectedStateRoot(block.Root()) - statedb, receipts, logs, usedGas, err := bc.processor.Process(block, statedb, bc.vmConfig) + + var receipts types.Receipts + var logs []*types.Log + var usedGas uint64 + if ParallelTxMode { + statedb, receipts, logs, usedGas, err = bc.processor.ProcessParallel(block, statedb, bc.vmConfig) + } else { + statedb, receipts, logs, usedGas, err = bc.processor.Process(block, statedb, bc.vmConfig) + } + atomic.StoreUint32(&followupInterrupt, 1) activeState = statedb if err != nil { diff --git a/core/state/journal.go b/core/state/journal.go index d86823c2ca..d727819375 100644 --- a/core/state/journal.go +++ b/core/state/journal.go @@ -142,7 +142,11 @@ type ( ) func (ch createObjectChange) revert(s *StateDB) { - delete(s.stateObjects, *ch.account) + if s.isSlotDB { + delete(s.dirtiedStateObjectsInSlot, *ch.account) + } else { + s.stateObjects.Delete(*ch.account) + } delete(s.stateObjectsDirty, *ch.account) } diff --git a/core/state/state_object.go b/core/state/state_object.go index 298f4305ba..c40b38a958 100644 --- a/core/state/state_object.go +++ b/core/state/state_object.go @@ -37,6 +37,8 @@ func (c Code) String() string { return string(c) //strings.Join(Disassemble(c), " ") } +type StorageKeys map[common.Hash]struct{} + type Storage map[common.Hash]common.Hash func (s Storage) String() (str string) { @@ -486,6 +488,14 @@ func (s *StateObject) deepCopy(db *StateDB) *StateObject { return stateObject } +func (s *StateObject) deepCopyForSlot(db *StateDB) *StateObject { + s.db = db + if s.trie != nil { + s.trie = db.db.CopyTrie(s.trie) + } + return s +} + // // Attribute accessors // diff --git a/core/state/statedb.go b/core/state/statedb.go index 5ea84f4032..8b0685dcfc 100644 --- a/core/state/statedb.go +++ b/core/state/statedb.go @@ -67,6 +67,24 @@ func (n *proofList) Delete(key []byte) error { panic("not supported") } +type StateKeys map[common.Hash]struct{} + +type StateObjectSyncMap struct { + sync.Map +} + +func (s *StateObjectSyncMap) LoadStateObject(addr common.Address) (*StateObject, bool) { + stateObject, ok := s.Load(addr) + if !ok { + return nil, ok + } + return stateObject.(*StateObject), ok +} + +func (s *StateObjectSyncMap) StoreStateObject(addr common.Address, stateObject *StateObject) { + s.Store(addr, stateObject) +} + // StateDB structs within the ethereum protocol are used to store anything // within the merkle trie. StateDBs take care of caching and storing // nested states. It's the general query interface to retrieve: @@ -97,10 +115,35 @@ type StateDB struct { snapStorage map[common.Address]map[string][]byte // This map holds 'live' objects, which will get modified while processing a state transition. - stateObjects map[common.Address]*StateObject + stateObjects *StateObjectSyncMap stateObjectsPending map[common.Address]struct{} // State objects finalized but not yet written to the trie stateObjectsDirty map[common.Address]struct{} // State objects modified in the current execution + // parallel start + isSlotDB bool + baseTxIndex int // slotDB is created base on this tx index. + SlotIndex int // debug purpose, will be removed + dirtiedStateObjectsInSlot map[common.Address]*StateObject + // for conflict check + balanceChangedInSlot map[common.Address]struct{} // the address's balance has been changed + balanceReadsInSlot map[common.Address]struct{} // the address's balance has been read and used. + codeReadInSlot map[common.Address]struct{} + codeChangeInSlot map[common.Address]struct{} + stateReadsInSlot map[common.Address]StateKeys + stateChangedInSlot map[common.Address]StateKeys // no need record value + // Actions such as SetCode, Suicide will change address's state. + // Later call like Exist(), Empty(), HasSuicided() depond on the address's state. + addrStateReadInSlot map[common.Address]struct{} + addrStateChangeInSlot map[common.Address]struct{} + stateObjectSuicided map[common.Address]struct{} + // Transaction will pay gas fee to system address. + // Parallel execution will clear system address's balance at first, in order to maintain transaction's + // gas fee value. Normal transaction will access system address twice, otherwise it means the transaction + // needs real system address's balance, the transaction will be marked redo with keepSystemAddressBalance = true + systemAddress common.Address + systemAddressCount int + keepSystemAddressBalance bool + // DB error. // State objects are used by the consensus core and VM which are // unable to deal with database-level errors. Any error that occurs @@ -147,18 +190,71 @@ func New(root common.Hash, db Database, snaps *snapshot.Tree) (*StateDB, error) return newStateDB(root, db, snaps) } +// With parallel, each execute slot would have its own stateDB. +// NewSlotDB creates a new slot stateDB base on the provided stateDB. +func NewSlotDB(db *StateDB, systemAddr common.Address, txIndex int, keepSystem bool) *StateDB { + slotDB := db.CopyForSlot() + slotDB.originalRoot = db.originalRoot + slotDB.baseTxIndex = txIndex + slotDB.systemAddress = systemAddr + slotDB.systemAddressCount = 0 + slotDB.keepSystemAddressBalance = keepSystem + + // clear the slotDB's validator's balance first + // for slotDB, systemAddr's value is the tx's gas fee + if !keepSystem { + slotDB.SetBalance(systemAddr, big.NewInt(0)) + } + + return slotDB +} + +// to avoid new slotDB for each Tx, slotDB should be valid and merged +func ReUseSlotDB(slotDB *StateDB, keepSystem bool) *StateDB { + if !keepSystem { + slotDB.SetBalance(slotDB.systemAddress, big.NewInt(0)) + } + slotDB.logs = make(map[common.Hash][]*types.Log, defaultNumOfSlots) + slotDB.logSize = 0 + slotDB.systemAddressCount = 0 + slotDB.keepSystemAddressBalance = keepSystem + slotDB.stateObjectSuicided = make(map[common.Address]struct{}, defaultNumOfSlots) + slotDB.codeReadInSlot = make(map[common.Address]struct{}, defaultNumOfSlots) + slotDB.codeChangeInSlot = make(map[common.Address]struct{}, defaultNumOfSlots) + slotDB.stateChangedInSlot = make(map[common.Address]StateKeys, defaultNumOfSlots) + slotDB.stateReadsInSlot = make(map[common.Address]StateKeys, defaultNumOfSlots) + slotDB.balanceChangedInSlot = make(map[common.Address]struct{}, defaultNumOfSlots) + slotDB.balanceReadsInSlot = make(map[common.Address]struct{}, defaultNumOfSlots) + slotDB.addrStateReadInSlot = make(map[common.Address]struct{}, defaultNumOfSlots) + slotDB.addrStateChangeInSlot = make(map[common.Address]struct{}, defaultNumOfSlots) + + slotDB.stateObjectsDirty = make(map[common.Address]struct{}, defaultNumOfSlots) + slotDB.stateObjectsPending = make(map[common.Address]struct{}, defaultNumOfSlots) + + return slotDB +} + func newStateDB(root common.Hash, db Database, snaps *snapshot.Tree) (*StateDB, error) { sdb := &StateDB{ - db: db, - originalRoot: root, - snaps: snaps, - stateObjects: make(map[common.Address]*StateObject, defaultNumOfSlots), - stateObjectsPending: make(map[common.Address]struct{}, defaultNumOfSlots), - stateObjectsDirty: make(map[common.Address]struct{}, defaultNumOfSlots), - logs: make(map[common.Hash][]*types.Log, defaultNumOfSlots), - preimages: make(map[common.Hash][]byte), - journal: newJournal(), - hasher: crypto.NewKeccakState(), + db: db, + originalRoot: root, + snaps: snaps, + stateObjects: &StateObjectSyncMap{}, + stateObjectSuicided: make(map[common.Address]struct{}, defaultNumOfSlots), + codeReadInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + codeChangeInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + stateChangedInSlot: make(map[common.Address]StateKeys, defaultNumOfSlots), + stateReadsInSlot: make(map[common.Address]StateKeys, defaultNumOfSlots), + balanceChangedInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + balanceReadsInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + addrStateReadInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + addrStateChangeInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + stateObjectsPending: make(map[common.Address]struct{}, defaultNumOfSlots), + stateObjectsDirty: make(map[common.Address]struct{}, defaultNumOfSlots), + logs: make(map[common.Hash][]*types.Log, defaultNumOfSlots), + preimages: make(map[common.Hash][]byte), + journal: newJournal(), + hasher: crypto.NewKeccakState(), } if sdb.snaps != nil { if sdb.snap = sdb.snaps.Snapshot(root); sdb.snap != nil { @@ -178,6 +274,119 @@ func newStateDB(root common.Hash, db Database, snaps *snapshot.Tree) (*StateDB, return sdb, nil } +func (s *StateDB) getStateObjectFromStateObjects(addr common.Address) (*StateObject, bool) { + if s.isSlotDB { + obj, ok := s.dirtiedStateObjectsInSlot[addr] + if ok { + return obj, ok + } + } + return s.stateObjects.LoadStateObject(addr) +} + +// MergeSlotDB is for Parallel TX, when the TX is finalized(dirty -> pending) +// A bit similar to StateDB.Copy(), +// mainly copy stateObjects, since slotDB has been finalized. +// return: objSuicided, stateChanges, balanceChanges, codeChanges +func (s *StateDB) MergeSlotDB(slotDb *StateDB, slotReceipt *types.Receipt) (map[common.Address]struct{}, map[common.Address]StateKeys, map[common.Address]struct{}, map[common.Address]struct{}, map[common.Address]struct{}) { + // receipt.Logs with unified log Index within a block + // align slotDB's logs Index to the block stateDB's logSize + for _, l := range slotReceipt.Logs { + l.Index += s.logSize + } + s.logSize += slotDb.logSize + + // before merge, do validator reward first: AddBalance to consensus.SystemAddress + // object of SystemAddress is take care specially + systemAddress := slotDb.systemAddress + if slotDb.keepSystemAddressBalance { + s.SetBalance(systemAddress, slotDb.GetBalance(systemAddress)) + } else { + s.AddBalance(systemAddress, slotDb.GetBalance(systemAddress)) + } + + // only merge dirty objects + for addr := range slotDb.stateObjectsDirty { + if _, exist := s.stateObjectsDirty[addr]; !exist { + s.stateObjectsDirty[addr] = struct{}{} + } + + if addr == systemAddress { + continue + } + + // stateObjects: KV, balance, nonce... + if obj, ok := slotDb.getStateObjectFromStateObjects(addr); ok { + s.stateObjects.StoreStateObject(addr, obj.deepCopyForSlot(s)) + } + } + + for addr := range slotDb.stateObjectsPending { + if _, exist := s.stateObjectsPending[addr]; !exist { + s.stateObjectsPending[addr] = struct{}{} + } + } + + for addr, obj := range slotDb.dirtiedStateObjectsInSlot { + if addr == systemAddress { + continue + } + + if _, exist := s.stateObjects.LoadStateObject(addr); !exist { + s.stateObjects.StoreStateObject(addr, obj.deepCopyForSlot(s)) + } + } + + // slotDb.logs: logs will be kept in receipts, no need to do merge + + // Fixed: preimages should be merged not overwrite + for hash, preimage := range slotDb.preimages { + s.preimages[hash] = preimage + } + // Fixed: accessList should be merged not overwrite + if s.accessList != nil { + s.accessList = slotDb.accessList.Copy() + } + if slotDb.snaps != nil { + for k, v := range slotDb.snapDestructs { + s.snapDestructs[k] = v + } + for k, v := range slotDb.snapAccounts { + s.snapAccounts[k] = v + } + for k, v := range slotDb.snapStorage { + temp := make(map[string][]byte) + for kk, vv := range v { + temp[kk] = vv + } + s.snapStorage[k] = temp + } + } + + objectSuicided := make(map[common.Address]struct{}, len(slotDb.stateObjectSuicided)) + for addr := range slotDb.stateObjectSuicided { + objectSuicided[addr] = struct{}{} + } + stateChanges := make(map[common.Address]StateKeys, len(slotDb.stateChangedInSlot)) // must be a deep copy, since + for addr, storage := range slotDb.stateChangedInSlot { + stateChanges[addr] = storage + } + balanceChanges := make(map[common.Address]struct{}, len(slotDb.balanceChangedInSlot)) // must be a deep copy, since + for addr := range slotDb.balanceChangedInSlot { + balanceChanges[addr] = struct{}{} + } + codeChanges := make(map[common.Address]struct{}, len(slotDb.codeChangeInSlot)) + for addr := range slotDb.codeChangeInSlot { + codeChanges[addr] = struct{}{} + } + addrStateChanges := make(map[common.Address]struct{}, len(slotDb.addrStateChangeInSlot)) + for addr := range slotDb.addrStateChangeInSlot { + addrStateChanges[addr] = struct{}{} + } + + return objectSuicided, stateChanges, balanceChanges, codeChanges, addrStateChanges +} + // StartPrefetcher initializes a new trie prefetcher to pull in nodes from the // state trie concurrently while the state is mutated so that when we reach the // commit phase, most of the needed data is already hot. @@ -323,6 +532,9 @@ func (s *StateDB) SubRefund(gas uint64) { // Exist reports whether the given account address exists in the state. // Notably this also returns true for suicided accounts. func (s *StateDB) Exist(addr common.Address) bool { + if s.isSlotDB { + s.addrStateReadInSlot[addr] = struct{}{} + } return s.getStateObject(addr) != nil } @@ -330,11 +542,20 @@ func (s *StateDB) Exist(addr common.Address) bool { // or empty according to the EIP161 specification (balance = nonce = code = 0) func (s *StateDB) Empty(addr common.Address) bool { so := s.getStateObject(addr) + if s.isSlotDB { + s.addrStateReadInSlot[addr] = struct{}{} + } return so == nil || so.empty() } // GetBalance retrieves the balance from the given address or 0 if object not found func (s *StateDB) GetBalance(addr common.Address) *big.Int { + if s.isSlotDB { + s.balanceReadsInSlot[addr] = struct{}{} + if addr == s.systemAddress { + s.systemAddressCount++ + } + } stateObject := s.getStateObject(addr) if stateObject != nil { return stateObject.Balance() @@ -361,7 +582,35 @@ func (s *StateDB) BlockHash() common.Hash { return s.bhash } +// BaseTxIndex returns the tx index that slot db based. +func (s *StateDB) BaseTxIndex() int { + return s.baseTxIndex +} + +func (s *StateDB) CodeReadInSlot() map[common.Address]struct{} { + return s.codeReadInSlot +} + +func (s *StateDB) AddressReadInSlot() map[common.Address]struct{} { + return s.addrStateReadInSlot +} + +func (s *StateDB) StateReadsInSlot() map[common.Address]StateKeys { + return s.stateReadsInSlot +} + +func (s *StateDB) BalanceReadsInSlot() map[common.Address]struct{} { + return s.balanceReadsInSlot +} +func (s *StateDB) SystemAddressRedo() bool { + return s.systemAddressCount > 2 +} + func (s *StateDB) GetCode(addr common.Address) []byte { + if s.isSlotDB { + s.codeReadInSlot[addr] = struct{}{} + } + stateObject := s.getStateObject(addr) if stateObject != nil { return stateObject.Code(s.db) @@ -370,6 +619,10 @@ func (s *StateDB) GetCode(addr common.Address) []byte { } func (s *StateDB) GetCodeSize(addr common.Address) int { + if s.isSlotDB { + s.codeReadInSlot[addr] = struct{}{} // code size is part of code + } + stateObject := s.getStateObject(addr) if stateObject != nil { return stateObject.CodeSize(s.db) @@ -378,6 +631,10 @@ func (s *StateDB) GetCodeSize(addr common.Address) int { } func (s *StateDB) GetCodeHash(addr common.Address) common.Hash { + if s.isSlotDB { + s.codeReadInSlot[addr] = struct{}{} // code hash is part of code + } + stateObject := s.getStateObject(addr) if stateObject == nil { return common.Hash{} @@ -387,6 +644,13 @@ func (s *StateDB) GetCodeHash(addr common.Address) common.Hash { // GetState retrieves a value from the given account's storage trie. func (s *StateDB) GetState(addr common.Address, hash common.Hash) common.Hash { + if s.isSlotDB { + if s.stateReadsInSlot[addr] == nil { + s.stateReadsInSlot[addr] = make(map[common.Hash]struct{}, defaultNumOfSlots) + } + s.stateReadsInSlot[addr][hash] = struct{}{} + } + stateObject := s.getStateObject(addr) if stateObject != nil { return stateObject.GetState(s.db, hash) @@ -459,6 +723,10 @@ func (s *StateDB) StorageTrie(addr common.Address) Trie { func (s *StateDB) HasSuicided(addr common.Address) bool { stateObject := s.getStateObject(addr) + + if s.isSlotDB { + s.addrStateReadInSlot[addr] = struct{}{} // address suicided. + } if stateObject != nil { return stateObject.suicided } @@ -471,45 +739,134 @@ func (s *StateDB) HasSuicided(addr common.Address) bool { // AddBalance adds amount to the account associated with addr. func (s *StateDB) AddBalance(addr common.Address, amount *big.Int) { + if s.isSlotDB { + // just in case other tx creates this account, we will miss this if we only add this account when found + s.balanceChangedInSlot[addr] = struct{}{} + s.balanceReadsInSlot[addr] = struct{}{} // add balance will perform a read operation first + if addr == s.systemAddress { + s.systemAddressCount++ + } + } + stateObject := s.GetOrNewStateObject(addr) if stateObject != nil { - stateObject.AddBalance(amount) + if s.isSlotDB { + if _, ok := s.dirtiedStateObjectsInSlot[addr]; !ok { + newStateObject := stateObject.deepCopy(s) + newStateObject.AddBalance(amount) + s.dirtiedStateObjectsInSlot[addr] = newStateObject + } else { + stateObject.AddBalance(amount) + } + } else { + stateObject.AddBalance(amount) + } } } // SubBalance subtracts amount from the account associated with addr. func (s *StateDB) SubBalance(addr common.Address, amount *big.Int) { + if s.isSlotDB { + // just in case other tx creates this account, we will miss this if we only add this account when found + s.balanceChangedInSlot[addr] = struct{}{} + s.balanceReadsInSlot[addr] = struct{}{} + if addr == s.systemAddress { + s.systemAddressCount++ + } + } + stateObject := s.GetOrNewStateObject(addr) if stateObject != nil { - stateObject.SubBalance(amount) + if s.isSlotDB { + if _, ok := s.dirtiedStateObjectsInSlot[addr]; !ok { + newStateObject := stateObject.deepCopy(s) + newStateObject.SubBalance(amount) + s.dirtiedStateObjectsInSlot[addr] = newStateObject + } else { + stateObject.SubBalance(amount) + } + } else { + stateObject.SubBalance(amount) + } } } func (s *StateDB) SetBalance(addr common.Address, amount *big.Int) { stateObject := s.GetOrNewStateObject(addr) if stateObject != nil { - stateObject.SetBalance(amount) + if s.isSlotDB { + if _, ok := s.dirtiedStateObjectsInSlot[addr]; !ok { + newStateObject := stateObject.deepCopy(s) + newStateObject.SetBalance(amount) + s.dirtiedStateObjectsInSlot[addr] = newStateObject + } else { + stateObject.SetBalance(amount) + } + s.balanceChangedInSlot[addr] = struct{}{} + if addr == s.systemAddress { + s.systemAddressCount++ + } + } else { + stateObject.SetBalance(amount) + } } } func (s *StateDB) SetNonce(addr common.Address, nonce uint64) { stateObject := s.GetOrNewStateObject(addr) if stateObject != nil { - stateObject.SetNonce(nonce) + if s.isSlotDB { + if _, ok := s.dirtiedStateObjectsInSlot[addr]; !ok { + newStateObject := stateObject.deepCopy(s) + newStateObject.SetNonce(nonce) + s.dirtiedStateObjectsInSlot[addr] = newStateObject + } else { + stateObject.SetNonce(nonce) + } + } else { + stateObject.SetNonce(nonce) + } } } func (s *StateDB) SetCode(addr common.Address, code []byte) { stateObject := s.GetOrNewStateObject(addr) if stateObject != nil { - stateObject.SetCode(crypto.Keccak256Hash(code), code) + if s.isSlotDB { + if _, ok := s.dirtiedStateObjectsInSlot[addr]; !ok { + newStateObject := stateObject.deepCopy(s) + newStateObject.SetCode(crypto.Keccak256Hash(code), code) + s.dirtiedStateObjectsInSlot[addr] = newStateObject + } else { + stateObject.SetCode(crypto.Keccak256Hash(code), code) + } + + s.codeChangeInSlot[addr] = struct{}{} + } else { + stateObject.SetCode(crypto.Keccak256Hash(code), code) + } } } func (s *StateDB) SetState(addr common.Address, key, value common.Hash) { stateObject := s.GetOrNewStateObject(addr) if stateObject != nil { - stateObject.SetState(s.db, key, value) + if s.isSlotDB { + if _, ok := s.dirtiedStateObjectsInSlot[addr]; !ok { + newStateObject := stateObject.deepCopy(s) + newStateObject.SetState(s.db, key, value) + s.dirtiedStateObjectsInSlot[addr] = newStateObject + } else { + stateObject.SetState(s.db, key, value) + } + + if s.stateChangedInSlot[addr] == nil { + s.stateChangedInSlot[addr] = make(StateKeys, defaultNumOfSlots) + } + s.stateChangedInSlot[addr][key] = struct{}{} + } else { + stateObject.SetState(s.db, key, value) + } } } @@ -529,9 +886,16 @@ func (s *StateDB) SetStorage(addr common.Address, storage map[common.Hash]common // getStateObject will return a non-nil account after Suicide. func (s *StateDB) Suicide(addr common.Address) bool { stateObject := s.getStateObject(addr) + // fixme: should add read stateobject record if stateObject == nil { + log.Warn("StateDB Suicide stateObject not found", "slot", s.SlotIndex, "addr", addr) return false } + if s.isSlotDB { + s.stateObjectSuicided[addr] = struct{}{} + s.addrStateChangeInSlot[addr] = struct{}{} // address suicided. + } + s.journal.append(suicideChange{ account: &addr, prev: stateObject.suicided, @@ -669,7 +1033,7 @@ func (s *StateDB) preloadStateObject(address []common.Address) []*StateObject { // destructed object instead of wiping all knowledge about the state object. func (s *StateDB) getDeletedStateObject(addr common.Address) *StateObject { // Prefer live objects if any is available - if obj := s.stateObjects[addr]; obj != nil { + if obj, _ := s.getStateObjectFromStateObjects(addr); obj != nil { return obj } // If no live objects are available, attempt to use snapshots @@ -734,7 +1098,11 @@ func (s *StateDB) getDeletedStateObject(addr common.Address) *StateObject { } func (s *StateDB) SetStateObject(object *StateObject) { - s.stateObjects[object.Address()] = object + if s.isSlotDB { + s.dirtiedStateObjectsInSlot[object.Address()] = object + } else { + s.stateObjects.StoreStateObject(object.Address(), object) + } } // GetOrNewStateObject retrieves a state object or create a new state object if nil. @@ -742,6 +1110,9 @@ func (s *StateDB) GetOrNewStateObject(addr common.Address) *StateObject { stateObject := s.getStateObject(addr) if stateObject == nil { stateObject, _ = s.createObject(addr) + if s.isSlotDB { + s.addrStateChangeInSlot[addr] = struct{}{} // address created. + } } return stateObject } @@ -786,18 +1157,21 @@ func (s *StateDB) CreateAccount(addr common.Address) { newObj, prev := s.createObject(addr) if prev != nil { newObj.setBalance(prev.data.Balance) + } else if s.isSlotDB { + s.addrStateChangeInSlot[addr] = struct{}{} // new account created } + } -func (db *StateDB) ForEachStorage(addr common.Address, cb func(key, value common.Hash) bool) error { - so := db.getStateObject(addr) +func (s *StateDB) ForEachStorage(addr common.Address, cb func(key, value common.Hash) bool) error { + so := s.getStateObject(addr) if so == nil { return nil } - it := trie.NewIterator(so.getTrie(db.db).NodeIterator(nil)) + it := trie.NewIterator(so.getTrie(s.db).NodeIterator(nil)) for it.Next() { - key := common.BytesToHash(db.trie.GetKey(it.Key)) + key := common.BytesToHash(s.trie.GetKey(it.Key)) if value, dirty := so.dirtyStorage[key]; dirty { if !cb(key, value) { return nil @@ -825,7 +1199,7 @@ func (s *StateDB) Copy() *StateDB { state := &StateDB{ db: s.db, trie: s.db.CopyTrie(s.trie), - stateObjects: make(map[common.Address]*StateObject, len(s.journal.dirties)), + stateObjects: &StateObjectSyncMap{}, stateObjectsPending: make(map[common.Address]struct{}, len(s.stateObjectsPending)), stateObjectsDirty: make(map[common.Address]struct{}, len(s.journal.dirties)), refund: s.refund, @@ -834,6 +1208,18 @@ func (s *StateDB) Copy() *StateDB { preimages: make(map[common.Hash][]byte, len(s.preimages)), journal: newJournal(), hasher: crypto.NewKeccakState(), + + isSlotDB: false, + stateObjectSuicided: make(map[common.Address]struct{}, defaultNumOfSlots), + codeReadInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + codeChangeInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + stateChangedInSlot: make(map[common.Address]StateKeys, defaultNumOfSlots), + stateReadsInSlot: make(map[common.Address]StateKeys, defaultNumOfSlots), + balanceChangedInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + balanceReadsInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + addrStateReadInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + addrStateChangeInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + dirtiedStateObjectsInSlot: make(map[common.Address]*StateObject, defaultNumOfSlots), } // Copy the dirty states, logs, and preimages for addr := range s.journal.dirties { @@ -841,11 +1227,11 @@ func (s *StateDB) Copy() *StateDB { // and in the Finalise-method, there is a case where an object is in the journal but not // in the stateObjects: OOG after touch on ripeMD prior to Byzantium. Thus, we need to check for // nil - if object, exist := s.stateObjects[addr]; exist { + if object, exist := s.getStateObjectFromStateObjects(addr); exist { // Even though the original object is dirty, we are not copying the journal, // so we need to make sure that anyside effect the journal would have caused // during a commit (or similar op) is already applied to the copy. - state.stateObjects[addr] = object.deepCopy(state) + state.stateObjects.StoreStateObject(addr, object.deepCopy(state)) state.stateObjectsDirty[addr] = struct{}{} // Mark the copy dirty to force internal (code/state) commits state.stateObjectsPending[addr] = struct{}{} // Mark the copy pending to force external (account) commits @@ -855,14 +1241,16 @@ func (s *StateDB) Copy() *StateDB { // loop above will be a no-op, since the copy's journal is empty. // Thus, here we iterate over stateObjects, to enable copies of copies for addr := range s.stateObjectsPending { - if _, exist := state.stateObjects[addr]; !exist { - state.stateObjects[addr] = s.stateObjects[addr].deepCopy(state) + if _, exist := state.getStateObjectFromStateObjects(addr); !exist { + object, _ := s.getStateObjectFromStateObjects(addr) + state.stateObjects.StoreStateObject(addr, object.deepCopy(state)) } state.stateObjectsPending[addr] = struct{}{} } for addr := range s.stateObjectsDirty { - if _, exist := state.stateObjects[addr]; !exist { - state.stateObjects[addr] = s.stateObjects[addr].deepCopy(state) + if _, exist := state.getStateObjectFromStateObjects(addr); !exist { + object, _ := s.getStateObjectFromStateObjects(addr) + state.stateObjects.StoreStateObject(addr, object.deepCopy(state)) } state.stateObjectsDirty[addr] = struct{}{} } @@ -920,6 +1308,69 @@ func (s *StateDB) Copy() *StateDB { return state } +func (s *StateDB) CopyForSlot() *StateDB { + // Copy all the basic fields, initialize the memory ones + state := &StateDB{ + db: s.db, + trie: s.db.CopyTrie(s.trie), + stateObjects: s.stateObjects, + stateObjectsPending: make(map[common.Address]struct{}, defaultNumOfSlots), + stateObjectsDirty: make(map[common.Address]struct{}, defaultNumOfSlots), + refund: s.refund, + logs: make(map[common.Hash][]*types.Log, defaultNumOfSlots), + logSize: 0, + preimages: make(map[common.Hash][]byte, len(s.preimages)), + journal: newJournal(), + hasher: crypto.NewKeccakState(), + snapDestructs: make(map[common.Address]struct{}), + snapAccounts: make(map[common.Address][]byte), + snapStorage: make(map[common.Address]map[string][]byte), + stateObjectSuicided: make(map[common.Address]struct{}, defaultNumOfSlots), + codeReadInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + codeChangeInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + stateChangedInSlot: make(map[common.Address]StateKeys, defaultNumOfSlots), + stateReadsInSlot: make(map[common.Address]StateKeys, defaultNumOfSlots), + balanceChangedInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + balanceReadsInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + addrStateReadInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + addrStateChangeInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + + isSlotDB: true, + dirtiedStateObjectsInSlot: make(map[common.Address]*StateObject, defaultNumOfSlots), + } + + for hash, preimage := range s.preimages { + state.preimages[hash] = preimage + } + + if s.snaps != nil { + // In order for the miner to be able to use and make additions + // to the snapshot tree, we need to copy that aswell. + // Otherwise, any block mined by ourselves will cause gaps in the tree, + // and force the miner to operate trie-backed only + state.snaps = s.snaps + state.snap = s.snap + // deep copy needed + state.snapDestructs = make(map[common.Address]struct{}) + for k, v := range s.snapDestructs { + state.snapDestructs[k] = v + } + state.snapAccounts = make(map[common.Address][]byte) + for k, v := range s.snapAccounts { + state.snapAccounts[k] = v + } + state.snapStorage = make(map[common.Address]map[string][]byte) + for k, v := range s.snapStorage { + temp := make(map[string][]byte) + for kk, vv := range v { + temp[kk] = vv + } + state.snapStorage[k] = temp + } + } + return state +} + // Snapshot returns an identifier for the current revision of the state. func (s *StateDB) Snapshot() int { id := s.nextRevisionId @@ -966,7 +1417,7 @@ func (s *StateDB) WaitPipeVerification() error { func (s *StateDB) Finalise(deleteEmptyObjects bool) { addressesToPrefetch := make([][]byte, 0, len(s.journal.dirties)) for addr := range s.journal.dirties { - obj, exist := s.stateObjects[addr] + obj, exist := s.getStateObjectFromStateObjects(addr) if !exist { // ripeMD is 'touched' at block 1714175, in tx 0x1237f737031e40bcde4a8b7e717b2d15e3ecadfe49bb1bbc71ee9deb09c6fcf2 // That tx goes out of gas, and although the notion of 'touched' does not exist there, the @@ -1047,11 +1498,10 @@ func (s *StateDB) AccountsIntermediateRoot() { // first, giving the account prefeches just a few more milliseconds of time // to pull useful data from disk. for addr := range s.stateObjectsPending { - if obj := s.stateObjects[addr]; !obj.deleted { + if obj, _ := s.getStateObjectFromStateObjects(addr); !obj.deleted { wg.Add(1) tasks <- func() { obj.updateRoot(s.db) - // If state snapshotting is active, cache the data til commit. Note, this // update mechanism is not symmetric to the deletion, because whereas it is // enough to track account updates at commit time, deletions need tracking @@ -1110,7 +1560,7 @@ func (s *StateDB) StateIntermediateRoot() common.Hash { } usedAddrs := make([][]byte, 0, len(s.stateObjectsPending)) for addr := range s.stateObjectsPending { - if obj := s.stateObjects[addr]; obj.deleted { + if obj, _ := s.getStateObjectFromStateObjects(addr); obj.deleted { s.deleteStateObject(obj) } else { s.updateStateObject(obj) @@ -1335,7 +1785,7 @@ func (s *StateDB) Commit(failPostCommitFunc func(), postCommitFuncs ...func() er if s.snap != nil { for addr := range s.stateObjectsDirty { - if obj := s.stateObjects[addr]; !obj.deleted { + if obj, _ := s.getStateObjectFromStateObjects(addr); !obj.deleted { if obj.code != nil && obj.dirtyCode { diffLayer.Codes = append(diffLayer.Codes, types.DiffCode{ Hash: common.BytesToHash(obj.CodeHash()), @@ -1347,7 +1797,7 @@ func (s *StateDB) Commit(failPostCommitFunc func(), postCommitFuncs ...func() er } for addr := range s.stateObjectsDirty { - if obj := s.stateObjects[addr]; !obj.deleted { + if obj, _ := s.getStateObjectFromStateObjects(addr); !obj.deleted { // Write any contract code associated with the state object tasks <- func() { // Write any storage changes in the state object to its storage trie @@ -1418,7 +1868,7 @@ func (s *StateDB) Commit(failPostCommitFunc func(), postCommitFuncs ...func() er func() error { codeWriter := s.db.TrieDB().DiskDB().NewBatch() for addr := range s.stateObjectsDirty { - if obj := s.stateObjects[addr]; !obj.deleted { + if obj, _ := s.getStateObjectFromStateObjects(addr); !obj.deleted { if obj.code != nil && obj.dirtyCode { rawdb.WriteCode(codeWriter, common.BytesToHash(obj.CodeHash()), obj.code) obj.dirtyCode = false diff --git a/core/state_processor.go b/core/state_processor.go index 9d7b1ac1f1..80ac42852c 100644 --- a/core/state_processor.go +++ b/core/state_processor.go @@ -22,6 +22,7 @@ import ( "fmt" "math/big" "math/rand" + "runtime" "sync" "time" @@ -46,8 +47,12 @@ const ( recentTime = 1024 * 3 recentDiffLayerTimeout = 5 farDiffLayerTimeout = 2 + reuseSlotDB = false // parallel slot's pending Txs will reuse the latest slotDB ) +var MaxPendingQueueSize = 10 // parallel slot's maximum number of pending Txs +var ParallelExecNum = runtime.NumCPU() - 1 // leave a CPU to dispatcher + // StateProcessor is a basic Processor, which takes care of transitioning // state from one point to another. // @@ -56,6 +61,12 @@ type StateProcessor struct { config *params.ChainConfig // Chain configuration options bc *BlockChain // Canonical block chain engine consensus.Engine // Consensus engine used for block rewards + + // add for parallel execute + paraInitialized bool // todo: should use atomic value + paraTxResultChan chan *ParallelTxResult // to notify dispatcher that a tx is done + slotState []*SlotState // idle, or pending messages + mergedTxIndex int // the latest finalized tx index } // NewStateProcessor initialises a new StateProcessor. @@ -402,7 +413,7 @@ func (p *StateProcessor) Process(block *types.Block, statedb *state.StateDB, cfg posa, isPoSA := p.engine.(consensus.PoSA) commonTxs := make([]*types.Transaction, 0, txNum) - // initilise bloom processors + // initialise bloom processors bloomProcessors := NewAsyncReceiptBloomGenerator(txNum) statedb.MarkFullProcessed() @@ -445,7 +456,485 @@ func (p *StateProcessor) Process(block *types.Block, statedb *state.StateDB, cfg return statedb, receipts, allLogs, *usedGas, nil } +type MergedTxInfo struct { + slotDB *state.StateDB // used for SlotDb reuse only, otherwise, it can be discarded + StateObjectSuicided map[common.Address]struct{} + StateChangeSet map[common.Address]state.StateKeys + BalanceChangeSet map[common.Address]struct{} + CodeChangeSet map[common.Address]struct{} + AddrStateChangeSet map[common.Address]struct{} + txIndex int +} + +type SlotState struct { + tailTxReq *ParallelTxRequest // tail pending Tx of the slot, should be accessed on dispatcher only. + pendingExec chan *ParallelTxRequest + // slot needs to keep the historical stateDB for conflict check + // each finalized DB should match a TX index + mergedTxInfo []MergedTxInfo + slotdbChan chan *state.StateDB // dispatch will create and send this slotDB to slot + // conflict check uses conflict window + // conflict check will check all state changes from (cfWindowStart + 1) to the previous Tx +} + +type ParallelTxResult struct { + redo bool // for redo, dispatch will wait new tx result + updateSlotDB bool // for redo and pendingExec, slot needs new slotDB, + reuseSlotDB bool // will try to reuse latest finalized slotDB + keepSystem bool // for redo, should keep system address's balance + txIndex int + slotIndex int // slot index + err error // to describe error message? + tx *types.Transaction + txReq *ParallelTxRequest + receipt *types.Receipt + slotDB *state.StateDB +} + +type ParallelTxRequest struct { + txIndex int + tx *types.Transaction + slotDB *state.StateDB + gp *GasPool + msg types.Message + block *types.Block + vmConfig vm.Config + bloomProcessors *AsyncReceiptBloomGenerator + usedGas *uint64 + waitTxChan chan int // "int" represents the tx index + curTxChan chan int // "int" represents the tx index +} + +// if any state in readDb is updated in writeDb, then it has state conflict +func (p *StateProcessor) hasStateConflict(readDb *state.StateDB, mergedInfo MergedTxInfo) bool { + // check KV change + reads := readDb.StateReadsInSlot() + writes := mergedInfo.StateChangeSet + if len(reads) != 0 && len(writes) != 0 { + for readAddr, readKeys := range reads { + if _, exist := mergedInfo.StateObjectSuicided[readAddr]; exist { + log.Debug("hasStateConflict read suicide object", "addr", readAddr) + return true + } + if writeKeys, ok := writes[readAddr]; ok { + // readAddr exist + for writeKey := range writeKeys { + // same addr and same key, mark conflicted + if _, ok := readKeys[writeKey]; ok { + log.Info("hasStateConflict state conflict", "addr", readAddr, "key", writeKey) + return true + } + } + } + } + } + // check balance change + balanceReads := readDb.BalanceReadsInSlot() + balanceWrite := mergedInfo.BalanceChangeSet + if len(balanceReads) != 0 && len(balanceWrite) != 0 { + for readAddr := range balanceReads { + if _, exist := mergedInfo.StateObjectSuicided[readAddr]; exist { + log.Debug("hasStateConflict read suicide balance", "addr", readAddr) + return true + } + if _, ok := balanceWrite[readAddr]; ok { + if readAddr == consensus.SystemAddress { + log.Info("hasStateConflict skip specical system address's balance check") + continue + } + log.Info("hasStateConflict balance conflict", "addr", readAddr) + return true + } + } + } + + // check code change + codeReads := readDb.CodeReadInSlot() + codeWrite := mergedInfo.CodeChangeSet + if len(codeReads) != 0 && len(codeWrite) != 0 { + for readAddr := range codeReads { + if _, exist := mergedInfo.StateObjectSuicided[readAddr]; exist { + log.Debug("hasStateConflict read suicide code", "addr", readAddr) + return true + } + if _, ok := codeWrite[readAddr]; ok { + log.Debug("hasStateConflict code conflict", "addr", readAddr) + return true + } + } + } + + // check address state change: create, suicide... + addrReads := readDb.AddressReadInSlot() + addrWrite := mergedInfo.AddrStateChangeSet + if len(addrReads) != 0 && len(addrWrite) != 0 { + for readAddr := range addrReads { + if _, ok := addrWrite[readAddr]; ok { + log.Info("hasStateConflict address state conflict", "addr", readAddr) + return true + } + } + } + + return false +} + +// for parallel execute, we put contracts of same address in a slot, +// since these txs probably would have conflicts +func (p *StateProcessor) queueToSameAddress(execMsg *ParallelTxRequest) bool { + txToAddr := execMsg.tx.To() + if txToAddr == nil { + return false + } + for i, slot := range p.slotState { + if slot.tailTxReq == nil { // this slot is idle + // log.Debug("queueToSameAddress skip idle slot.") + continue + } + + // To() == nil means contract creation, won't queue to such slot. + if slot.tailTxReq.tx.To() == nil { + // log.Debug("queueToSameAddress, slot's To address is nil", "slotIndex", i) + continue + } + // same to address, put it on slot's pending list. + if *txToAddr == *slot.tailTxReq.tx.To() { + select { + case slot.pendingExec <- execMsg: + slot.tailTxReq = execMsg + log.Debug("queueToSameAddress", "slotIndex", i, "txIndex", execMsg.txIndex) + return true + default: + log.Debug("queueToSameAddress but queue is full", "slotIndex", i, "txIndex", execMsg.txIndex) + return false + } + } + } + return false +} + +// if there is idle slot, dispatch the msg to the first idle slot +func (p *StateProcessor) dispatchToIdleSlot(statedb *state.StateDB, txReq *ParallelTxRequest) bool { + for i, slot := range p.slotState { + if slot.tailTxReq == nil { + // for idle slot, we have to create a SlotDB for it. + if len(slot.mergedTxInfo) == 0 { + txReq.slotDB = state.NewSlotDB(statedb, consensus.SystemAddress, p.mergedTxIndex, false) + } + log.Debug("dispatchToIdleSlot", "slotIndex", i, "txIndex", txReq.txIndex) + slot.tailTxReq = txReq + slot.pendingExec <- txReq + return true + } + } + return false +} + +// wait until the next Tx is executed and its result is merged to the main stateDB +func (p *StateProcessor) waitUntilNextTxDone(statedb *state.StateDB) *ParallelTxResult { + var result *ParallelTxResult + for { + result = <-p.paraTxResultChan + // slot may request new slotDB, if it think its slotDB is outdated + // such as: + // tx in pendingExec, previous tx in same queue is likely "damaged" the slotDB + // tx redo for confict + // tx stage 1 failed, nonce out of order... + if result.updateSlotDB { + // the target slot is waiting for new slotDB + slotState := p.slotState[result.slotIndex] + var slotDB *state.StateDB + if result.reuseSlotDB { + // for reuse, len(slotState.mergedTxInfo) must >= 1 + lastSlotDB := slotState.mergedTxInfo[len(slotState.mergedTxInfo)-1].slotDB + slotDB = state.ReUseSlotDB(lastSlotDB, result.keepSystem) + } else { + slotDB = state.NewSlotDB(statedb, consensus.SystemAddress, p.mergedTxIndex, result.keepSystem) + } + slotState.slotdbChan <- slotDB + continue + } + if result.redo { + // wait result of redo + continue + } + // ok, the tx result is valid and can be merged + break + } + resultSlotIndex := result.slotIndex + resultTxIndex := result.txIndex + resultSlotState := p.slotState[resultSlotIndex] + if resultSlotState.tailTxReq.txIndex == resultTxIndex { + log.Debug("ProcessParallel slot is idle", "slotIndex", resultSlotIndex) + resultSlotState.tailTxReq = nil + } + + // merge slotDB to parent stateDB + log.Info("ProcessParallel a tx is done, merge to block stateDB", + "resultSlotIndex", resultSlotIndex, "resultTxIndex", resultTxIndex) + objSuicided, stateChanges, balanceChanges, codeChanges, addrChanges := statedb.MergeSlotDB(result.slotDB, result.receipt) + // slot's mergedTxInfo is updated by dispatcher, while consumed by slot. + // It is safe, since write and read is in sequential, do write -> notify -> read + // it is not good, but work right now. + + resultSlotState.mergedTxInfo = append(resultSlotState.mergedTxInfo, MergedTxInfo{result.slotDB, objSuicided, stateChanges, balanceChanges, codeChanges, addrChanges, resultTxIndex}) + + if resultTxIndex != p.mergedTxIndex+1 { + log.Warn("ProcessParallel tx result out of order", "resultTxIndex", resultTxIndex, + "p.mergedTxIndex", p.mergedTxIndex) + panic("ProcessParallel tx result out of order") + } + p.mergedTxIndex = resultTxIndex + // notify the following Tx, it is merged, what if no wait or next tx is in same slot? + result.txReq.curTxChan <- resultTxIndex + return result +} + +func (p *StateProcessor) execInParallelSlot(slotIndex int, txReq *ParallelTxRequest) *ParallelTxResult { + txIndex := txReq.txIndex + tx := txReq.tx + slotDB := txReq.slotDB + slotDB.SlotIndex = slotIndex + gp := txReq.gp // goroutine unsafe + msg := txReq.msg + block := txReq.block + header := block.Header() + cfg := txReq.vmConfig + bloomProcessors := txReq.bloomProcessors + + blockContext := NewEVMBlockContext(header, p.bc, nil) // fixme: share blockContext within a block? + vmenv := vm.NewEVM(blockContext, vm.TxContext{}, slotDB, p.config, cfg) + + var receipt *types.Receipt + var result *ExecutionResult + var err error + var evm *vm.EVM + + // fixme: to optimize, reuse the slotDB + slotDB.Prepare(tx.Hash(), block.Hash(), txIndex) + log.Debug("execInParallelSlot enter", "slotIndex", slotIndex, "txIndex", txIndex, "slotDB.baseTxIndex", slotDB.BaseTxIndex()) + + slotGasLimit := gp.Gas() + gpSlot := new(GasPool).AddGas(slotGasLimit) // each slot would use its own gas pool, and will do gaslimit check later + evm, result, err = applyTransactionStageExecution(msg, gpSlot, slotDB, vmenv) + log.Debug("execInParallelSlot Stage Execution done", "slotIndex", slotIndex, "txIndex", txIndex, "slotDB.baseTxIndex", slotDB.BaseTxIndex()) + + // wait until the previous tx is finalized. + if txReq.waitTxChan != nil { + log.Info("execInParallelSlot wait previous Tx done", "my slotIndex", slotIndex, "txIndex", txIndex) + waitTxIndex := <-txReq.waitTxChan + if waitTxIndex != txIndex-1 { + log.Error("execInParallelSlot wait tx index mismatch", "expect", txIndex-1, "actual", waitTxIndex) + panic(fmt.Sprintf("wait tx index mismatch expect:%d, actual:%d", txIndex-1, waitTxIndex)) + } + } + + // in parallel, tx can run into trouble + // for example: err="nonce too high" + // in this case, we will do re-run. + if err != nil { + log.Debug("Stage Execution err", "slotIndex", slotIndex, "txIndex", txIndex, + "current slotDB.baseTxIndex", slotDB.BaseTxIndex(), "err", err) + redoResult := &ParallelTxResult{ + redo: true, + updateSlotDB: true, + reuseSlotDB: false, + txIndex: txIndex, + slotIndex: slotIndex, + tx: tx, + txReq: txReq, + receipt: receipt, + err: err, + } + p.paraTxResultChan <- redoResult + slotDB = <-p.slotState[slotIndex].slotdbChan + slotDB.SlotIndex = slotIndex + slotDB.Prepare(tx.Hash(), block.Hash(), txIndex) + // vmenv.Reset(vm.TxContext{}, slotDB) + log.Debug("Stage Execution get new slotdb to redo", "slotIndex", slotIndex, + "txIndex", txIndex, "new slotDB.baseTxIndex", slotDB.BaseTxIndex()) + slotGasLimit = gp.Gas() + gpSlot = new(GasPool).AddGas(slotGasLimit) + evm, result, err = applyTransactionStageExecution(msg, gpSlot, slotDB, vmenv) + if err != nil { + panic(fmt.Sprintf("Stage Execution redo, error %v", err)) + } + } + + // fixme: + // parallel mode can not precheck, + // precheck should be replace by postCheck when previous Tx is finalized + + // do conflict detect + hasConflict := false + systemAddrConflict := false + + log.Debug("execInParallelSlot Tx Stage1 done, do conflict check", "slotIndex", slotIndex, "txIndex", txIndex) + if slotDB.SystemAddressRedo() { + hasConflict = true + systemAddrConflict = true + } else { + for index := 0; index < ParallelExecNum; index++ { + // can skip current slot now, since slotDB is always after current slot's merged DB + // ** idle: all previous Txs are merged, it will create a new SlotDB + // ** queued: it will request updateSlotDB, dispatcher will create or reuse a SlotDB after previous Tx results are merged + if index == slotIndex { + continue + } + + // check all finalizedDb from current slot's + for _, mergedInfo := range p.slotState[index].mergedTxInfo { + if mergedInfo.txIndex <= slotDB.BaseTxIndex() { + // log.Info("skip finalized DB which is out of the conflict window", "finDb.txIndex", finDb.txIndex, "slotDB.baseTxIndex", slotDB.baseTxIndex) + continue + } + if p.hasStateConflict(slotDB, mergedInfo) { + log.Debug("execInParallelSlot Stage Execution conflict", "slotIndex", slotIndex, + "txIndex", txIndex, " conflict slot", index, "slotDB.baseTxIndex", slotDB.BaseTxIndex()) + hasConflict = true + break + } + } + if hasConflict { + break + } + } + } + + if hasConflict { + // re-run should not have conflict, since it has the latest world state. + redoResult := &ParallelTxResult{ + redo: true, + updateSlotDB: true, + reuseSlotDB: false, // for conflict, we do not reuse + keepSystem: systemAddrConflict, + txIndex: txIndex, + slotIndex: slotIndex, + tx: tx, + txReq: txReq, + receipt: receipt, + err: err, + } + p.paraTxResultChan <- redoResult + slotDB = <-p.slotState[slotIndex].slotdbChan + slotDB.SlotIndex = slotIndex + slotDB.Prepare(tx.Hash(), block.Hash(), txIndex) + // vmenv.Reset(vm.TxContext{}, slotDB) + slotGasLimit = gp.Gas() + gpSlot = new(GasPool).AddGas(slotGasLimit) + evm, result, err = applyTransactionStageExecution(msg, gpSlot, slotDB, vmenv) + if err != nil { + panic(fmt.Sprintf("Stage Execution conflict redo, error %v", err)) + } + } + + // goroutine unsafe operation will be handled from here for safety + gasConsumed := slotGasLimit - gpSlot.Gas() + if gasConsumed != result.UsedGas { + log.Error("execInParallelSlot gasConsumed != result.UsedGas mismatch", + "gasConsumed", gasConsumed, "result.UsedGas", result.UsedGas) + panic(fmt.Sprintf("gas consume mismatch, consumed:%d, result.UsedGas:%d", gasConsumed, result.UsedGas)) + } + + if err := gp.SubGas(gasConsumed); err != nil { + log.Error("gas limit reached", "gasConsumed", gasConsumed, "gp", gp.Gas()) + panic(fmt.Sprintf("gas limit reached, gasConsumed:%d, gp.Gas():%d", gasConsumed, gp.Gas())) + } + + log.Debug("execInParallelSlot ok to finalize this TX", + "slotIndex", slotIndex, "txIndex", txIndex, "result.UsedGas", result.UsedGas, "txReq.usedGas", *txReq.usedGas) + // ok, time to do finalize, stage2 should not be parallel + receipt, err = applyTransactionStageFinalization(evm, result, msg, p.config, slotDB, header, tx, txReq.usedGas, bloomProcessors) + + if result.Err != nil { + // if Tx is reverted, all its state change will be discarded + log.Debug("execInParallelSlot TX reverted?", "slotIndex", slotIndex, "txIndex", txIndex, "result.Err", result.Err) + } + return &ParallelTxResult{ + redo: false, + updateSlotDB: false, + txIndex: txIndex, + slotIndex: slotIndex, + tx: tx, + txReq: txReq, + receipt: receipt, + slotDB: slotDB, + err: err, + } +} + +func (p *StateProcessor) runSlotLoop(slotIndex int) { + curSlot := p.slotState[slotIndex] + for { + // log.Info("parallel slot waiting", "slotIndex:", slotIndex) + // wait for new TxReq + txReq := <-curSlot.pendingExec + // receive a dispatched message + log.Debug("SlotLoop received a new TxReq", "slotIndex:", slotIndex, "txIndex", txReq.txIndex) + + // SlotDB create rational: + // ** for a dispatched tx, + // the slot should be idle, it is better to create a new SlotDB, since new Tx is not related to previous Tx + // ** for a queued tx, + // the previous SlotDB could be reused, since it is likely can be used + // reuse could avoid NewSlotDB cost, which could be costable when StateDB is full of state object + // if the previous SlotDB is + if txReq.slotDB == nil { + // for queued Tx, txReq.slotDB is nil, reuse slot's latest merged SlotDB + result := &ParallelTxResult{ + redo: false, + updateSlotDB: true, + reuseSlotDB: reuseSlotDB, + slotIndex: slotIndex, + err: nil, + } + p.paraTxResultChan <- result + txReq.slotDB = <-curSlot.slotdbChan + } + result := p.execInParallelSlot(slotIndex, txReq) + log.Debug("SlotLoop the TxReq is done", "slotIndex:", slotIndex, "err", result.err) + p.paraTxResultChan <- result + } +} + +// clear slotState for each block. +func (p *StateProcessor) resetSlotState() { + p.mergedTxIndex = -1 + for _, slotState := range p.slotState { + slotState.tailTxReq = nil + slotState.mergedTxInfo = make([]MergedTxInfo, 0) + } +} + +func (p *StateProcessor) InitParallelOnce() { + // to create and start the execution slot goroutines + if p.paraInitialized { + return + } + + p.paraTxResultChan = make(chan *ParallelTxResult, ParallelExecNum) // fixme: use blocked chan? + p.slotState = make([]*SlotState, ParallelExecNum) + + wg := sync.WaitGroup{} // make sure all goroutines are created and started + for i := 0; i < ParallelExecNum; i++ { + p.slotState[i] = new(SlotState) + p.slotState[i].slotdbChan = make(chan *state.StateDB, 1) + p.slotState[i].pendingExec = make(chan *ParallelTxRequest, MaxPendingQueueSize) + + wg.Add(1) + // start the slot's goroutine + go func(slotIndex int) { + wg.Done() + p.runSlotLoop(slotIndex) // this loop will be permanent live + log.Error("runSlotLoop exit!", "slotIndex", slotIndex) + }(i) + } + wg.Wait() + p.paraInitialized = true +} + func (p *StateProcessor) ProcessParallel(block *types.Block, statedb *state.StateDB, cfg vm.Config) (*state.StateDB, types.Receipts, []*types.Log, uint64, error) { + p.InitParallelOnce() var ( usedGas = new(uint64) header = block.Header() @@ -462,20 +951,22 @@ func (p *StateProcessor) ProcessParallel(block *types.Block, statedb *state.Stat // Handle upgrade build-in system contract code systemcontracts.UpgradeBuildInSystemContract(p.config, block.Number(), statedb) - blockContext := NewEVMBlockContext(header, p.bc, nil) - vmenv := vm.NewEVM(blockContext, vm.TxContext{}, statedb, p.config, cfg) - txNum := len(block.Transactions()) + if txNum > 0 { + log.Info("ProcessParallel", "block num", block.Number(), "txNum", txNum) + p.resetSlotState() + } // Iterate over and process the individual transactions posa, isPoSA := p.engine.(consensus.PoSA) commonTxs := make([]*types.Transaction, 0, txNum) - // initilise bloom processors + // initialise bloom processors bloomProcessors := NewAsyncReceiptBloomGenerator(txNum) statedb.MarkFullProcessed() // usually do have two tx, one for validator set contract, another for system reward contract. systemTxs := make([]*types.Transaction, 0, 2) + var waitTxChan, curTxChan chan int for i, tx := range block.Transactions() { if isPoSA { if isSystemTx, err := posa.IsSystemTransaction(tx, block.Header()); err != nil { @@ -486,21 +977,75 @@ func (p *StateProcessor) ProcessParallel(block *types.Block, statedb *state.Stat } } - msg, err := tx.AsMessage(signer) + msg, err := tx.AsMessage(signer) // fixme: move it into slot. if err != nil { return statedb, nil, nil, 0, err } - statedb.Prepare(tx.Hash(), block.Hash(), i) - receipt, err := applyTransaction(msg, p.config, p.bc, nil, gp, statedb, header, tx, usedGas, vmenv, bloomProcessors) - if err != nil { - return statedb, nil, nil, 0, fmt.Errorf("could not apply tx %d [%v]: %w", i, tx.Hash().Hex(), err) + + // parallel start, wrap an exec message, which will be dispatched to a slot + waitTxChan = curTxChan // can be nil, if this is the tx of first batch, otherwise, it is previous Tx's wait channel + curTxChan = make(chan int, 1) + + txReq := &ParallelTxRequest{ + txIndex: i, + tx: tx, + slotDB: nil, + gp: gp, + msg: msg, + block: block, + vmConfig: cfg, + bloomProcessors: bloomProcessors, + usedGas: usedGas, + waitTxChan: waitTxChan, + curTxChan: curTxChan, } - commonTxs = append(commonTxs, tx) - receipts = append(receipts, receipt) + // fixme: to optimize the for { for {} } loop code style + for { + // if p.queueToSameAddress(txReq) { + // log.Info("ProcessParallel queue to same slot", "txIndex", txReq.txIndex) + // continue + // } + + // if idle slot available, just dispatch and process next tx. + if p.dispatchToIdleSlot(statedb, txReq) { + // log.Info("ProcessParallel dispatch to idle slot", "txIndex", txReq.txIndex) + break + } + log.Debug("ProcessParallel no slot avaiable, wait", "txIndex", txReq.txIndex) + // no idle slot, wait until a tx is executed and merged. + result := p.waitUntilNextTxDone(statedb) + + // update tx result + if result.err != nil { + log.Warn("ProcessParallel a failed tx", "resultSlotIndex", result.slotIndex, + "resultTxIndex", result.txIndex, "result.err", result.err) + return statedb, nil, nil, 0, fmt.Errorf("could not apply tx %d [%v]: %w", result.txIndex, result.tx.Hash().Hex(), result.err) + } + commonTxs = append(commonTxs, result.tx) + receipts = append(receipts, result.receipt) + } } - bloomProcessors.Close() + // wait until all tx request are done + for len(commonTxs)+len(systemTxs) < txNum { + result := p.waitUntilNextTxDone(statedb) + // update tx result + if result.err != nil { + log.Warn("ProcessParallel a failed tx", "resultSlotIndex", result.slotIndex, + "resultTxIndex", result.txIndex, "result.err", result.err) + return statedb, nil, nil, 0, fmt.Errorf("could not apply tx %d [%v]: %w", result.txIndex, result.tx.Hash().Hex(), result.err) + } + commonTxs = append(commonTxs, result.tx) + receipts = append(receipts, result.receipt) + } + + bloomProcessors.Close() + if txNum > 0 { + log.Info("ProcessParallel tx all done", "block", header.Number, "usedGas", *usedGas, + "len(commonTxs)", len(commonTxs), "len(receipts)", len(receipts), + "len(systemTxs)", len(systemTxs)) + } // Finalize the block, applying any consensus engine specific extras (e.g. block rewards) err := p.engine.Finalize(p.bc, header, statedb, &commonTxs, block.Uncles(), &receipts, &systemTxs, usedGas) if err != nil { @@ -560,6 +1105,57 @@ func applyTransaction(msg types.Message, config *params.ChainConfig, bc ChainCon return receipt, err } +func applyTransactionStageExecution(msg types.Message, gp *GasPool, statedb *state.StateDB, evm *vm.EVM) (*vm.EVM, *ExecutionResult, error) { + // Create a new context to be used in the EVM environment. + txContext := NewEVMTxContext(msg) + evm.Reset(txContext, statedb) + + // Apply the transaction to the current state (included in the env). + result, err := ApplyMessage(evm, msg, gp) + if err != nil { + return nil, nil, err + } + + return evm, result, err +} + +func applyTransactionStageFinalization(evm *vm.EVM, result *ExecutionResult, msg types.Message, config *params.ChainConfig, statedb *state.StateDB, header *types.Header, tx *types.Transaction, usedGas *uint64, receiptProcessors ...ReceiptProcessor) (*types.Receipt, error) { + // Update the state with pending changes. + var root []byte + if config.IsByzantium(header.Number) { + statedb.Finalise(true) + } else { + root = statedb.IntermediateRoot(config.IsEIP158(header.Number)).Bytes() + } + *usedGas += result.UsedGas + + // Create a new receipt for the transaction, storing the intermediate root and gas used + // by the tx. + receipt := &types.Receipt{Type: tx.Type(), PostState: root, CumulativeGasUsed: *usedGas} + if result.Failed() { + receipt.Status = types.ReceiptStatusFailed + } else { + receipt.Status = types.ReceiptStatusSuccessful + } + receipt.TxHash = tx.Hash() + receipt.GasUsed = result.UsedGas + + // If the transaction created a contract, store the creation address in the receipt. + if msg.To() == nil { + receipt.ContractAddress = crypto.CreateAddress(evm.TxContext.Origin, tx.Nonce()) + } + + // Set the receipt logs and create the bloom filter. + receipt.Logs = statedb.GetLogs(tx.Hash()) + receipt.BlockHash = statedb.BlockHash() + receipt.BlockNumber = header.Number + receipt.TransactionIndex = uint(statedb.TxIndex()) + for _, receiptProcessor := range receiptProcessors { + receiptProcessor.Apply(receipt) + } + return receipt, nil +} + // ApplyTransaction attempts to apply a transaction to the given state database // and uses the input parameters for its environment. It returns the receipt // for the transaction, gas used and an error if the transaction failed, diff --git a/go.mod b/go.mod index c58a45dfd2..41b8a28076 100644 --- a/go.mod +++ b/go.mod @@ -58,6 +58,7 @@ require ( github.com/oxtoacart/bpool v0.0.0-20190530202638-03653db5a59c github.com/panjf2000/ants/v2 v2.4.5 github.com/peterh/liner v1.1.1-0.20190123174540-a2c9a5303de7 + github.com/prometheus/client_golang v1.0.0 github.com/prometheus/tsdb v0.7.1 github.com/rcrowley/go-metrics v0.0.0-20190826022208-cac0b30c2563 // indirect github.com/rjeczalik/notify v0.9.1 diff --git a/metrics/exp/exp.go b/metrics/exp/exp.go index 3ebe8cc68a..563a55bf65 100644 --- a/metrics/exp/exp.go +++ b/metrics/exp/exp.go @@ -8,6 +8,8 @@ import ( "net/http" "sync" + "github.com/prometheus/client_golang/prometheus/promhttp" + "github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/metrics" "github.com/ethereum/go-ethereum/metrics/prometheus" @@ -44,6 +46,7 @@ func Exp(r metrics.Registry) { // http.HandleFunc("/debug/vars", e.expHandler) // haven't found an elegant way, so just use a different endpoint http.Handle("/debug/metrics", h) + http.Handle("/debug/metrics/go_prometheus", promhttp.Handler()) http.Handle("/debug/metrics/prometheus", prometheus.Handler(r)) } @@ -58,6 +61,7 @@ func ExpHandler(r metrics.Registry) http.Handler { func Setup(address string) { m := http.NewServeMux() m.Handle("/debug/metrics", ExpHandler(metrics.DefaultRegistry)) + m.Handle("/debug/metrics/go_prometheus", promhttp.Handler()) m.Handle("/debug/metrics/prometheus", prometheus.Handler(metrics.DefaultRegistry)) log.Info("Starting metrics server", "addr", fmt.Sprintf("http://%s/debug/metrics", address)) go func() { From 1ecaf4b92f6952819b40ac9177e225346f83cd3d Mon Sep 17 00:00:00 2001 From: setunapo Date: Tue, 15 Feb 2022 11:03:35 +0800 Subject: [PATCH 3/8] Parallel: more readable code & dispatch policy & Revert & UT ** move .Process() close to .ProcessParallel() ** InitParallelOnce & preExec & postExec for code maintenance ** MergedTxInfo -> SlotChangeList & debug conflict ratio ** use ParallelState to keep all parallel statedb states. ** enable queue to same slot ** discard state change of reverted transaction And debug log refine ** add ut for statedb --- core/blockchain.go | 3 + core/state/journal.go | 4 +- core/state/statedb.go | 402 ++++++++++++++++++++---------------- core/state/statedb_test.go | 375 +++++++++++++++++++++++++++++++++ core/state_processor.go | 411 ++++++++++++++++++------------------- core/types.go | 3 +- 6 files changed, 803 insertions(+), 395 deletions(-) diff --git a/core/blockchain.go b/core/blockchain.go index c3294ce8b8..49071fef9e 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -311,6 +311,9 @@ func NewBlockChain(db ethdb.Database, cacheConfig *CacheConfig, chainConfig *par bc.prefetcher = NewStatePrefetcher(chainConfig, bc, engine) bc.validator = NewBlockValidator(chainConfig, bc, engine) bc.processor = NewStateProcessor(chainConfig, bc, engine) + if ParallelTxMode { + bc.processor.InitParallelOnce() + } var err error bc.hc, err = NewHeaderChain(db, chainConfig, engine, bc.insertStopped) diff --git a/core/state/journal.go b/core/state/journal.go index d727819375..20ae6a269a 100644 --- a/core/state/journal.go +++ b/core/state/journal.go @@ -142,8 +142,8 @@ type ( ) func (ch createObjectChange) revert(s *StateDB) { - if s.isSlotDB { - delete(s.dirtiedStateObjectsInSlot, *ch.account) + if s.parallel.isSlotDB { + delete(s.parallel.dirtiedStateObjectsInSlot, *ch.account) } else { s.stateObjects.Delete(*ch.account) } diff --git a/core/state/statedb.go b/core/state/statedb.go index 8b0685dcfc..281bf9787c 100644 --- a/core/state/statedb.go +++ b/core/state/statedb.go @@ -85,6 +85,43 @@ func (s *StateObjectSyncMap) StoreStateObject(addr common.Address, stateObject * s.Store(addr, stateObject) } +// For parallel mode only, keep the change list for later conflict detect +type SlotChangeList struct { + SlotDB *StateDB // used for SlotDb reuse only, otherwise, it can be discarded + TxIndex int // the tx index of change list + StateObjectSuicided map[common.Address]struct{} + StateChangeSet map[common.Address]StateKeys + BalanceChangeSet map[common.Address]struct{} + CodeChangeSet map[common.Address]struct{} + AddrStateChangeSet map[common.Address]struct{} +} + +// For parallel mode only +type ParallelState struct { + isSlotDB bool + baseTxIndex int // slotDB is created base on this tx index. + dirtiedStateObjectsInSlot map[common.Address]*StateObject + // for conflict check + balanceChangedInSlot map[common.Address]struct{} // the address's balance has been changed + balanceReadsInSlot map[common.Address]struct{} // the address's balance has been read and used. + codeReadInSlot map[common.Address]struct{} + codeChangeInSlot map[common.Address]struct{} + stateReadsInSlot map[common.Address]StateKeys + stateChangedInSlot map[common.Address]StateKeys // no need record value + // Actions such as SetCode, Suicide will change address's state. + // Later call like Exist(), Empty(), HasSuicided() depond on the address's state. + addrStateReadInSlot map[common.Address]struct{} + addrStateChangeInSlot map[common.Address]struct{} + stateObjectSuicided map[common.Address]struct{} + // Transaction will pay gas fee to system address. + // Parallel execution will clear system address's balance at first, in order to maintain transaction's + // gas fee value. Normal transaction will access system address twice, otherwise it means the transaction + // needs real system address's balance, the transaction will be marked redo with keepSystemAddressBalance = true + systemAddress common.Address + systemAddressCount int + keepSystemAddressBalance bool +} + // StateDB structs within the ethereum protocol are used to store anything // within the merkle trie. StateDBs take care of caching and storing // nested states. It's the general query interface to retrieve: @@ -119,30 +156,7 @@ type StateDB struct { stateObjectsPending map[common.Address]struct{} // State objects finalized but not yet written to the trie stateObjectsDirty map[common.Address]struct{} // State objects modified in the current execution - // parallel start - isSlotDB bool - baseTxIndex int // slotDB is created base on this tx index. - SlotIndex int // debug purpose, will be removed - dirtiedStateObjectsInSlot map[common.Address]*StateObject - // for conflict check - balanceChangedInSlot map[common.Address]struct{} // the address's balance has been changed - balanceReadsInSlot map[common.Address]struct{} // the address's balance has been read and used. - codeReadInSlot map[common.Address]struct{} - codeChangeInSlot map[common.Address]struct{} - stateReadsInSlot map[common.Address]StateKeys - stateChangedInSlot map[common.Address]StateKeys // no need record value - // Actions such as SetCode, Suicide will change address's state. - // Later call like Exist(), Empty(), HasSuicided() depond on the address's state. - addrStateReadInSlot map[common.Address]struct{} - addrStateChangeInSlot map[common.Address]struct{} - stateObjectSuicided map[common.Address]struct{} - // Transaction will pay gas fee to system address. - // Parallel execution will clear system address's balance at first, in order to maintain transaction's - // gas fee value. Normal transaction will access system address twice, otherwise it means the transaction - // needs real system address's balance, the transaction will be marked redo with keepSystemAddressBalance = true - systemAddress common.Address - systemAddressCount int - keepSystemAddressBalance bool + parallel ParallelState // to keep all the parallel execution elements // DB error. // State objects are used by the consensus core and VM which are @@ -195,10 +209,10 @@ func New(root common.Hash, db Database, snaps *snapshot.Tree) (*StateDB, error) func NewSlotDB(db *StateDB, systemAddr common.Address, txIndex int, keepSystem bool) *StateDB { slotDB := db.CopyForSlot() slotDB.originalRoot = db.originalRoot - slotDB.baseTxIndex = txIndex - slotDB.systemAddress = systemAddr - slotDB.systemAddressCount = 0 - slotDB.keepSystemAddressBalance = keepSystem + slotDB.parallel.baseTxIndex = txIndex + slotDB.parallel.systemAddress = systemAddr + slotDB.parallel.systemAddressCount = 0 + slotDB.parallel.keepSystemAddressBalance = keepSystem // clear the slotDB's validator's balance first // for slotDB, systemAddr's value is the tx's gas fee @@ -212,21 +226,21 @@ func NewSlotDB(db *StateDB, systemAddr common.Address, txIndex int, keepSystem b // to avoid new slotDB for each Tx, slotDB should be valid and merged func ReUseSlotDB(slotDB *StateDB, keepSystem bool) *StateDB { if !keepSystem { - slotDB.SetBalance(slotDB.systemAddress, big.NewInt(0)) + slotDB.SetBalance(slotDB.parallel.systemAddress, big.NewInt(0)) } slotDB.logs = make(map[common.Hash][]*types.Log, defaultNumOfSlots) slotDB.logSize = 0 - slotDB.systemAddressCount = 0 - slotDB.keepSystemAddressBalance = keepSystem - slotDB.stateObjectSuicided = make(map[common.Address]struct{}, defaultNumOfSlots) - slotDB.codeReadInSlot = make(map[common.Address]struct{}, defaultNumOfSlots) - slotDB.codeChangeInSlot = make(map[common.Address]struct{}, defaultNumOfSlots) - slotDB.stateChangedInSlot = make(map[common.Address]StateKeys, defaultNumOfSlots) - slotDB.stateReadsInSlot = make(map[common.Address]StateKeys, defaultNumOfSlots) - slotDB.balanceChangedInSlot = make(map[common.Address]struct{}, defaultNumOfSlots) - slotDB.balanceReadsInSlot = make(map[common.Address]struct{}, defaultNumOfSlots) - slotDB.addrStateReadInSlot = make(map[common.Address]struct{}, defaultNumOfSlots) - slotDB.addrStateChangeInSlot = make(map[common.Address]struct{}, defaultNumOfSlots) + slotDB.parallel.systemAddressCount = 0 + slotDB.parallel.keepSystemAddressBalance = keepSystem + slotDB.parallel.stateObjectSuicided = make(map[common.Address]struct{}, defaultNumOfSlots) + slotDB.parallel.codeReadInSlot = make(map[common.Address]struct{}, defaultNumOfSlots) + slotDB.parallel.codeChangeInSlot = make(map[common.Address]struct{}, defaultNumOfSlots) + slotDB.parallel.stateChangedInSlot = make(map[common.Address]StateKeys, defaultNumOfSlots) + slotDB.parallel.stateReadsInSlot = make(map[common.Address]StateKeys, defaultNumOfSlots) + slotDB.parallel.balanceChangedInSlot = make(map[common.Address]struct{}, defaultNumOfSlots) + slotDB.parallel.balanceReadsInSlot = make(map[common.Address]struct{}, defaultNumOfSlots) + slotDB.parallel.addrStateReadInSlot = make(map[common.Address]struct{}, defaultNumOfSlots) + slotDB.parallel.addrStateChangeInSlot = make(map[common.Address]struct{}, defaultNumOfSlots) slotDB.stateObjectsDirty = make(map[common.Address]struct{}, defaultNumOfSlots) slotDB.stateObjectsPending = make(map[common.Address]struct{}, defaultNumOfSlots) @@ -235,11 +249,9 @@ func ReUseSlotDB(slotDB *StateDB, keepSystem bool) *StateDB { } func newStateDB(root common.Hash, db Database, snaps *snapshot.Tree) (*StateDB, error) { - sdb := &StateDB{ - db: db, - originalRoot: root, - snaps: snaps, - stateObjects: &StateObjectSyncMap{}, + + parallel := ParallelState{ + isSlotDB: false, stateObjectSuicided: make(map[common.Address]struct{}, defaultNumOfSlots), codeReadInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), codeChangeInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), @@ -249,12 +261,19 @@ func newStateDB(root common.Hash, db Database, snaps *snapshot.Tree) (*StateDB, balanceReadsInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), addrStateReadInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), addrStateChangeInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - stateObjectsPending: make(map[common.Address]struct{}, defaultNumOfSlots), - stateObjectsDirty: make(map[common.Address]struct{}, defaultNumOfSlots), - logs: make(map[common.Hash][]*types.Log, defaultNumOfSlots), - preimages: make(map[common.Hash][]byte), - journal: newJournal(), - hasher: crypto.NewKeccakState(), + } + sdb := &StateDB{ + db: db, + originalRoot: root, + snaps: snaps, + stateObjects: &StateObjectSyncMap{}, + parallel: parallel, + stateObjectsPending: make(map[common.Address]struct{}, defaultNumOfSlots), + stateObjectsDirty: make(map[common.Address]struct{}, defaultNumOfSlots), + logs: make(map[common.Hash][]*types.Log, defaultNumOfSlots), + preimages: make(map[common.Hash][]byte), + journal: newJournal(), + hasher: crypto.NewKeccakState(), } if sdb.snaps != nil { if sdb.snap = sdb.snaps.Snapshot(root); sdb.snap != nil { @@ -275,8 +294,8 @@ func newStateDB(root common.Hash, db Database, snaps *snapshot.Tree) (*StateDB, } func (s *StateDB) getStateObjectFromStateObjects(addr common.Address) (*StateObject, bool) { - if s.isSlotDB { - obj, ok := s.dirtiedStateObjectsInSlot[addr] + if s.parallel.isSlotDB { + obj, ok := s.parallel.dirtiedStateObjectsInSlot[addr] if ok { return obj, ok } @@ -284,11 +303,21 @@ func (s *StateDB) getStateObjectFromStateObjects(addr common.Address) (*StateObj return s.stateObjects.LoadStateObject(addr) } +// If the transaction execution is failed, keep its read list for conflict detect +// and discard its state changed, execept its own balance change. +func (s *StateDB) RevertSlotDB(from common.Address) { + s.parallel.stateObjectSuicided = make(map[common.Address]struct{}) + s.parallel.stateChangedInSlot = make(map[common.Address]StateKeys) + s.parallel.balanceChangedInSlot = make(map[common.Address]struct{}, 1) + s.parallel.balanceChangedInSlot[from] = struct{}{} + s.parallel.addrStateChangeInSlot = make(map[common.Address]struct{}) +} + // MergeSlotDB is for Parallel TX, when the TX is finalized(dirty -> pending) // A bit similar to StateDB.Copy(), // mainly copy stateObjects, since slotDB has been finalized. -// return: objSuicided, stateChanges, balanceChanges, codeChanges -func (s *StateDB) MergeSlotDB(slotDb *StateDB, slotReceipt *types.Receipt) (map[common.Address]struct{}, map[common.Address]StateKeys, map[common.Address]struct{}, map[common.Address]struct{}, map[common.Address]struct{}) { +// return and keep the slot's change list for later conflict detect. +func (s *StateDB) MergeSlotDB(slotDb *StateDB, slotReceipt *types.Receipt, txIndex int) SlotChangeList { // receipt.Logs with unified log Index within a block // align slotDB's logs Index to the block stateDB's logSize for _, l := range slotReceipt.Logs { @@ -298,8 +327,8 @@ func (s *StateDB) MergeSlotDB(slotDb *StateDB, slotReceipt *types.Receipt) (map[ // before merge, do validator reward first: AddBalance to consensus.SystemAddress // object of SystemAddress is take care specially - systemAddress := slotDb.systemAddress - if slotDb.keepSystemAddressBalance { + systemAddress := slotDb.parallel.systemAddress + if slotDb.parallel.keepSystemAddressBalance { s.SetBalance(systemAddress, slotDb.GetBalance(systemAddress)) } else { s.AddBalance(systemAddress, slotDb.GetBalance(systemAddress)) @@ -327,7 +356,7 @@ func (s *StateDB) MergeSlotDB(slotDb *StateDB, slotReceipt *types.Receipt) (map[ } } - for addr, obj := range slotDb.dirtiedStateObjectsInSlot { + for addr, obj := range slotDb.parallel.dirtiedStateObjectsInSlot { if addr == systemAddress { continue } @@ -363,28 +392,34 @@ func (s *StateDB) MergeSlotDB(slotDb *StateDB, slotReceipt *types.Receipt) (map[ } } - objectSuicided := make(map[common.Address]struct{}, len(slotDb.stateObjectSuicided)) - for addr := range slotDb.stateObjectSuicided { - objectSuicided[addr] = struct{}{} + // we have to create a new object to store change list for conflict detect, since + // StateDB could be reused and its elements could be overwritten + changeList := SlotChangeList{ + SlotDB: slotDb, + TxIndex: txIndex, + StateObjectSuicided: make(map[common.Address]struct{}, len(slotDb.parallel.stateObjectSuicided)), + StateChangeSet: make(map[common.Address]StateKeys, len(slotDb.parallel.stateChangedInSlot)), + BalanceChangeSet: make(map[common.Address]struct{}, len(slotDb.parallel.balanceChangedInSlot)), + CodeChangeSet: make(map[common.Address]struct{}, len(slotDb.parallel.codeChangeInSlot)), + AddrStateChangeSet: make(map[common.Address]struct{}, len(slotDb.parallel.addrStateChangeInSlot)), + } + for addr := range slotDb.parallel.stateObjectSuicided { + changeList.StateObjectSuicided[addr] = struct{}{} } - stateChanges := make(map[common.Address]StateKeys, len(slotDb.stateChangedInSlot)) // must be a deep copy, since - for addr, storage := range slotDb.stateChangedInSlot { - stateChanges[addr] = storage + for addr, storage := range slotDb.parallel.stateChangedInSlot { + changeList.StateChangeSet[addr] = storage } - balanceChanges := make(map[common.Address]struct{}, len(slotDb.balanceChangedInSlot)) // must be a deep copy, since - for addr := range slotDb.balanceChangedInSlot { - balanceChanges[addr] = struct{}{} + for addr := range slotDb.parallel.balanceChangedInSlot { + changeList.BalanceChangeSet[addr] = struct{}{} } - codeChanges := make(map[common.Address]struct{}, len(slotDb.codeChangeInSlot)) - for addr := range slotDb.codeChangeInSlot { - codeChanges[addr] = struct{}{} + for addr := range slotDb.parallel.codeChangeInSlot { + changeList.CodeChangeSet[addr] = struct{}{} } - addrStateChanges := make(map[common.Address]struct{}, len(slotDb.addrStateChangeInSlot)) - for addr := range slotDb.addrStateChangeInSlot { - addrStateChanges[addr] = struct{}{} + for addr := range slotDb.parallel.addrStateChangeInSlot { + changeList.AddrStateChangeSet[addr] = struct{}{} } - return objectSuicided, stateChanges, balanceChanges, codeChanges, addrStateChanges + return changeList } // StartPrefetcher initializes a new trie prefetcher to pull in nodes from the @@ -532,8 +567,8 @@ func (s *StateDB) SubRefund(gas uint64) { // Exist reports whether the given account address exists in the state. // Notably this also returns true for suicided accounts. func (s *StateDB) Exist(addr common.Address) bool { - if s.isSlotDB { - s.addrStateReadInSlot[addr] = struct{}{} + if s.parallel.isSlotDB { + s.parallel.addrStateReadInSlot[addr] = struct{}{} } return s.getStateObject(addr) != nil } @@ -542,18 +577,18 @@ func (s *StateDB) Exist(addr common.Address) bool { // or empty according to the EIP161 specification (balance = nonce = code = 0) func (s *StateDB) Empty(addr common.Address) bool { so := s.getStateObject(addr) - if s.isSlotDB { - s.addrStateReadInSlot[addr] = struct{}{} + if s.parallel.isSlotDB { + s.parallel.addrStateReadInSlot[addr] = struct{}{} } return so == nil || so.empty() } // GetBalance retrieves the balance from the given address or 0 if object not found func (s *StateDB) GetBalance(addr common.Address) *big.Int { - if s.isSlotDB { - s.balanceReadsInSlot[addr] = struct{}{} - if addr == s.systemAddress { - s.systemAddressCount++ + if s.parallel.isSlotDB { + s.parallel.balanceReadsInSlot[addr] = struct{}{} + if addr == s.parallel.systemAddress { + s.parallel.systemAddressCount++ } } stateObject := s.getStateObject(addr) @@ -584,31 +619,31 @@ func (s *StateDB) BlockHash() common.Hash { // BaseTxIndex returns the tx index that slot db based. func (s *StateDB) BaseTxIndex() int { - return s.baseTxIndex + return s.parallel.baseTxIndex } func (s *StateDB) CodeReadInSlot() map[common.Address]struct{} { - return s.codeReadInSlot + return s.parallel.codeReadInSlot } func (s *StateDB) AddressReadInSlot() map[common.Address]struct{} { - return s.addrStateReadInSlot + return s.parallel.addrStateReadInSlot } func (s *StateDB) StateReadsInSlot() map[common.Address]StateKeys { - return s.stateReadsInSlot + return s.parallel.stateReadsInSlot } func (s *StateDB) BalanceReadsInSlot() map[common.Address]struct{} { - return s.balanceReadsInSlot + return s.parallel.balanceReadsInSlot } func (s *StateDB) SystemAddressRedo() bool { - return s.systemAddressCount > 2 + return s.parallel.systemAddressCount > 2 } func (s *StateDB) GetCode(addr common.Address) []byte { - if s.isSlotDB { - s.codeReadInSlot[addr] = struct{}{} + if s.parallel.isSlotDB { + s.parallel.codeReadInSlot[addr] = struct{}{} } stateObject := s.getStateObject(addr) @@ -619,8 +654,8 @@ func (s *StateDB) GetCode(addr common.Address) []byte { } func (s *StateDB) GetCodeSize(addr common.Address) int { - if s.isSlotDB { - s.codeReadInSlot[addr] = struct{}{} // code size is part of code + if s.parallel.isSlotDB { + s.parallel.codeReadInSlot[addr] = struct{}{} // code size is part of code } stateObject := s.getStateObject(addr) @@ -631,8 +666,8 @@ func (s *StateDB) GetCodeSize(addr common.Address) int { } func (s *StateDB) GetCodeHash(addr common.Address) common.Hash { - if s.isSlotDB { - s.codeReadInSlot[addr] = struct{}{} // code hash is part of code + if s.parallel.isSlotDB { + s.parallel.codeReadInSlot[addr] = struct{}{} // code hash is part of code } stateObject := s.getStateObject(addr) @@ -644,11 +679,11 @@ func (s *StateDB) GetCodeHash(addr common.Address) common.Hash { // GetState retrieves a value from the given account's storage trie. func (s *StateDB) GetState(addr common.Address, hash common.Hash) common.Hash { - if s.isSlotDB { - if s.stateReadsInSlot[addr] == nil { - s.stateReadsInSlot[addr] = make(map[common.Hash]struct{}, defaultNumOfSlots) + if s.parallel.isSlotDB { + if s.parallel.stateReadsInSlot[addr] == nil { + s.parallel.stateReadsInSlot[addr] = make(map[common.Hash]struct{}, defaultNumOfSlots) } - s.stateReadsInSlot[addr][hash] = struct{}{} + s.parallel.stateReadsInSlot[addr][hash] = struct{}{} } stateObject := s.getStateObject(addr) @@ -724,8 +759,8 @@ func (s *StateDB) StorageTrie(addr common.Address) Trie { func (s *StateDB) HasSuicided(addr common.Address) bool { stateObject := s.getStateObject(addr) - if s.isSlotDB { - s.addrStateReadInSlot[addr] = struct{}{} // address suicided. + if s.parallel.isSlotDB { + s.parallel.addrStateReadInSlot[addr] = struct{}{} // address suicided. } if stateObject != nil { return stateObject.suicided @@ -739,22 +774,22 @@ func (s *StateDB) HasSuicided(addr common.Address) bool { // AddBalance adds amount to the account associated with addr. func (s *StateDB) AddBalance(addr common.Address, amount *big.Int) { - if s.isSlotDB { + if s.parallel.isSlotDB { // just in case other tx creates this account, we will miss this if we only add this account when found - s.balanceChangedInSlot[addr] = struct{}{} - s.balanceReadsInSlot[addr] = struct{}{} // add balance will perform a read operation first - if addr == s.systemAddress { - s.systemAddressCount++ + s.parallel.balanceChangedInSlot[addr] = struct{}{} + s.parallel.balanceReadsInSlot[addr] = struct{}{} // add balance will perform a read operation first + if addr == s.parallel.systemAddress { + s.parallel.systemAddressCount++ } } stateObject := s.GetOrNewStateObject(addr) if stateObject != nil { - if s.isSlotDB { - if _, ok := s.dirtiedStateObjectsInSlot[addr]; !ok { + if s.parallel.isSlotDB { + if _, ok := s.parallel.dirtiedStateObjectsInSlot[addr]; !ok { newStateObject := stateObject.deepCopy(s) newStateObject.AddBalance(amount) - s.dirtiedStateObjectsInSlot[addr] = newStateObject + s.parallel.dirtiedStateObjectsInSlot[addr] = newStateObject } else { stateObject.AddBalance(amount) } @@ -766,22 +801,22 @@ func (s *StateDB) AddBalance(addr common.Address, amount *big.Int) { // SubBalance subtracts amount from the account associated with addr. func (s *StateDB) SubBalance(addr common.Address, amount *big.Int) { - if s.isSlotDB { + if s.parallel.isSlotDB { // just in case other tx creates this account, we will miss this if we only add this account when found - s.balanceChangedInSlot[addr] = struct{}{} - s.balanceReadsInSlot[addr] = struct{}{} - if addr == s.systemAddress { - s.systemAddressCount++ + s.parallel.balanceChangedInSlot[addr] = struct{}{} + s.parallel.balanceReadsInSlot[addr] = struct{}{} + if addr == s.parallel.systemAddress { + s.parallel.systemAddressCount++ } } stateObject := s.GetOrNewStateObject(addr) if stateObject != nil { - if s.isSlotDB { - if _, ok := s.dirtiedStateObjectsInSlot[addr]; !ok { + if s.parallel.isSlotDB { + if _, ok := s.parallel.dirtiedStateObjectsInSlot[addr]; !ok { newStateObject := stateObject.deepCopy(s) newStateObject.SubBalance(amount) - s.dirtiedStateObjectsInSlot[addr] = newStateObject + s.parallel.dirtiedStateObjectsInSlot[addr] = newStateObject } else { stateObject.SubBalance(amount) } @@ -794,17 +829,17 @@ func (s *StateDB) SubBalance(addr common.Address, amount *big.Int) { func (s *StateDB) SetBalance(addr common.Address, amount *big.Int) { stateObject := s.GetOrNewStateObject(addr) if stateObject != nil { - if s.isSlotDB { - if _, ok := s.dirtiedStateObjectsInSlot[addr]; !ok { + if s.parallel.isSlotDB { + if _, ok := s.parallel.dirtiedStateObjectsInSlot[addr]; !ok { newStateObject := stateObject.deepCopy(s) newStateObject.SetBalance(amount) - s.dirtiedStateObjectsInSlot[addr] = newStateObject + s.parallel.dirtiedStateObjectsInSlot[addr] = newStateObject } else { stateObject.SetBalance(amount) } - s.balanceChangedInSlot[addr] = struct{}{} - if addr == s.systemAddress { - s.systemAddressCount++ + s.parallel.balanceChangedInSlot[addr] = struct{}{} + if addr == s.parallel.systemAddress { + s.parallel.systemAddressCount++ } } else { stateObject.SetBalance(amount) @@ -815,11 +850,11 @@ func (s *StateDB) SetBalance(addr common.Address, amount *big.Int) { func (s *StateDB) SetNonce(addr common.Address, nonce uint64) { stateObject := s.GetOrNewStateObject(addr) if stateObject != nil { - if s.isSlotDB { - if _, ok := s.dirtiedStateObjectsInSlot[addr]; !ok { + if s.parallel.isSlotDB { + if _, ok := s.parallel.dirtiedStateObjectsInSlot[addr]; !ok { newStateObject := stateObject.deepCopy(s) newStateObject.SetNonce(nonce) - s.dirtiedStateObjectsInSlot[addr] = newStateObject + s.parallel.dirtiedStateObjectsInSlot[addr] = newStateObject } else { stateObject.SetNonce(nonce) } @@ -832,16 +867,16 @@ func (s *StateDB) SetNonce(addr common.Address, nonce uint64) { func (s *StateDB) SetCode(addr common.Address, code []byte) { stateObject := s.GetOrNewStateObject(addr) if stateObject != nil { - if s.isSlotDB { - if _, ok := s.dirtiedStateObjectsInSlot[addr]; !ok { + if s.parallel.isSlotDB { + if _, ok := s.parallel.dirtiedStateObjectsInSlot[addr]; !ok { newStateObject := stateObject.deepCopy(s) newStateObject.SetCode(crypto.Keccak256Hash(code), code) - s.dirtiedStateObjectsInSlot[addr] = newStateObject + s.parallel.dirtiedStateObjectsInSlot[addr] = newStateObject } else { stateObject.SetCode(crypto.Keccak256Hash(code), code) } - s.codeChangeInSlot[addr] = struct{}{} + s.parallel.codeChangeInSlot[addr] = struct{}{} } else { stateObject.SetCode(crypto.Keccak256Hash(code), code) } @@ -851,19 +886,19 @@ func (s *StateDB) SetCode(addr common.Address, code []byte) { func (s *StateDB) SetState(addr common.Address, key, value common.Hash) { stateObject := s.GetOrNewStateObject(addr) if stateObject != nil { - if s.isSlotDB { - if _, ok := s.dirtiedStateObjectsInSlot[addr]; !ok { + if s.parallel.isSlotDB { + if _, ok := s.parallel.dirtiedStateObjectsInSlot[addr]; !ok { newStateObject := stateObject.deepCopy(s) newStateObject.SetState(s.db, key, value) - s.dirtiedStateObjectsInSlot[addr] = newStateObject + s.parallel.dirtiedStateObjectsInSlot[addr] = newStateObject } else { stateObject.SetState(s.db, key, value) } - if s.stateChangedInSlot[addr] == nil { - s.stateChangedInSlot[addr] = make(StateKeys, defaultNumOfSlots) + if s.parallel.stateChangedInSlot[addr] == nil { + s.parallel.stateChangedInSlot[addr] = make(StateKeys, defaultNumOfSlots) } - s.stateChangedInSlot[addr][key] = struct{}{} + s.parallel.stateChangedInSlot[addr][key] = struct{}{} } else { stateObject.SetState(s.db, key, value) } @@ -888,12 +923,11 @@ func (s *StateDB) Suicide(addr common.Address) bool { stateObject := s.getStateObject(addr) // fixme: should add read stateobject record if stateObject == nil { - log.Warn("StateDB Suicide stateObject not found", "slot", s.SlotIndex, "addr", addr) return false } - if s.isSlotDB { - s.stateObjectSuicided[addr] = struct{}{} - s.addrStateChangeInSlot[addr] = struct{}{} // address suicided. + if s.parallel.isSlotDB { + s.parallel.stateObjectSuicided[addr] = struct{}{} + s.parallel.addrStateChangeInSlot[addr] = struct{}{} // address suicided. } s.journal.append(suicideChange{ @@ -1098,8 +1132,8 @@ func (s *StateDB) getDeletedStateObject(addr common.Address) *StateObject { } func (s *StateDB) SetStateObject(object *StateObject) { - if s.isSlotDB { - s.dirtiedStateObjectsInSlot[object.Address()] = object + if s.parallel.isSlotDB { + s.parallel.dirtiedStateObjectsInSlot[object.Address()] = object } else { s.stateObjects.StoreStateObject(object.Address(), object) } @@ -1110,8 +1144,8 @@ func (s *StateDB) GetOrNewStateObject(addr common.Address) *StateObject { stateObject := s.getStateObject(addr) if stateObject == nil { stateObject, _ = s.createObject(addr) - if s.isSlotDB { - s.addrStateChangeInSlot[addr] = struct{}{} // address created. + if s.parallel.isSlotDB { + s.parallel.addrStateChangeInSlot[addr] = struct{}{} // address created. } } return stateObject @@ -1157,8 +1191,8 @@ func (s *StateDB) CreateAccount(addr common.Address) { newObj, prev := s.createObject(addr) if prev != nil { newObj.setBalance(prev.data.Balance) - } else if s.isSlotDB { - s.addrStateChangeInSlot[addr] = struct{}{} // new account created + } else if s.parallel.isSlotDB { + s.parallel.addrStateChangeInSlot[addr] = struct{}{} // new account created } } @@ -1196,6 +1230,20 @@ func (s *StateDB) ForEachStorage(addr common.Address, cb func(key, value common. // Snapshots of the copied state cannot be applied to the copy. func (s *StateDB) Copy() *StateDB { // Copy all the basic fields, initialize the memory ones + parallel := ParallelState{ + isSlotDB: false, + stateObjectSuicided: make(map[common.Address]struct{}, defaultNumOfSlots), + codeReadInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + codeChangeInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + stateChangedInSlot: make(map[common.Address]StateKeys, defaultNumOfSlots), + stateReadsInSlot: make(map[common.Address]StateKeys, defaultNumOfSlots), + balanceChangedInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + balanceReadsInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + addrStateReadInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + addrStateChangeInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + dirtiedStateObjectsInSlot: make(map[common.Address]*StateObject, defaultNumOfSlots), + } + state := &StateDB{ db: s.db, trie: s.db.CopyTrie(s.trie), @@ -1208,18 +1256,7 @@ func (s *StateDB) Copy() *StateDB { preimages: make(map[common.Hash][]byte, len(s.preimages)), journal: newJournal(), hasher: crypto.NewKeccakState(), - - isSlotDB: false, - stateObjectSuicided: make(map[common.Address]struct{}, defaultNumOfSlots), - codeReadInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - codeChangeInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - stateChangedInSlot: make(map[common.Address]StateKeys, defaultNumOfSlots), - stateReadsInSlot: make(map[common.Address]StateKeys, defaultNumOfSlots), - balanceChangedInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - balanceReadsInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - addrStateReadInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - addrStateChangeInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - dirtiedStateObjectsInSlot: make(map[common.Address]*StateObject, defaultNumOfSlots), + parallel: parallel, } // Copy the dirty states, logs, and preimages for addr := range s.journal.dirties { @@ -1310,34 +1347,37 @@ func (s *StateDB) Copy() *StateDB { func (s *StateDB) CopyForSlot() *StateDB { // Copy all the basic fields, initialize the memory ones - state := &StateDB{ - db: s.db, - trie: s.db.CopyTrie(s.trie), - stateObjects: s.stateObjects, - stateObjectsPending: make(map[common.Address]struct{}, defaultNumOfSlots), - stateObjectsDirty: make(map[common.Address]struct{}, defaultNumOfSlots), - refund: s.refund, - logs: make(map[common.Hash][]*types.Log, defaultNumOfSlots), - logSize: 0, - preimages: make(map[common.Hash][]byte, len(s.preimages)), - journal: newJournal(), - hasher: crypto.NewKeccakState(), - snapDestructs: make(map[common.Address]struct{}), - snapAccounts: make(map[common.Address][]byte), - snapStorage: make(map[common.Address]map[string][]byte), - stateObjectSuicided: make(map[common.Address]struct{}, defaultNumOfSlots), - codeReadInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - codeChangeInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - stateChangedInSlot: make(map[common.Address]StateKeys, defaultNumOfSlots), - stateReadsInSlot: make(map[common.Address]StateKeys, defaultNumOfSlots), - balanceChangedInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - balanceReadsInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - addrStateReadInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - addrStateChangeInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + parallel := ParallelState{ + stateObjectSuicided: make(map[common.Address]struct{}, defaultNumOfSlots), + codeReadInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + codeChangeInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + stateChangedInSlot: make(map[common.Address]StateKeys, defaultNumOfSlots), + stateReadsInSlot: make(map[common.Address]StateKeys, defaultNumOfSlots), + balanceChangedInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + balanceReadsInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + addrStateReadInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + addrStateChangeInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), isSlotDB: true, dirtiedStateObjectsInSlot: make(map[common.Address]*StateObject, defaultNumOfSlots), } + state := &StateDB{ + db: s.db, + trie: s.db.CopyTrie(s.trie), + stateObjects: s.stateObjects, + stateObjectsPending: make(map[common.Address]struct{}, defaultNumOfSlots), + stateObjectsDirty: make(map[common.Address]struct{}, defaultNumOfSlots), + refund: s.refund, + logs: make(map[common.Hash][]*types.Log, defaultNumOfSlots), + logSize: 0, + preimages: make(map[common.Hash][]byte, len(s.preimages)), + journal: newJournal(), + hasher: crypto.NewKeccakState(), + snapDestructs: make(map[common.Address]struct{}), + snapAccounts: make(map[common.Address][]byte), + snapStorage: make(map[common.Address]map[string][]byte), + parallel: parallel, + } for hash, preimage := range s.preimages { state.preimages[hash] = preimage diff --git a/core/state/statedb_test.go b/core/state/statedb_test.go index acbbf1cd2f..2fce92ae1d 100644 --- a/core/state/statedb_test.go +++ b/core/state/statedb_test.go @@ -19,6 +19,7 @@ package state import ( "bytes" "encoding/binary" + "encoding/hex" "fmt" "math" "math/big" @@ -34,6 +35,10 @@ import ( "github.com/ethereum/go-ethereum/core/types" ) +var ( + systemAddress = common.HexToAddress("0xffffFFFfFFffffffffffffffFfFFFfffFFFfFFfE") +) + // Tests that updating a state trie does not leak any database writes prior to // actually committing the state. func TestUpdateLeaks(t *testing.T) { @@ -932,3 +937,373 @@ func TestStateDBAccessList(t *testing.T) { t.Fatalf("expected empty, got %d", got) } } + +func TestSuicide(t *testing.T) { + // Create an initial state with a few accounts + memDb := rawdb.NewMemoryDatabase() + db := NewDatabase(memDb) + state, _ := New(common.Hash{}, db, nil) + slotDb := NewSlotDB(state, systemAddress, 0, false) + + addr := common.BytesToAddress([]byte("so")) + slotDb.SetBalance(addr, big.NewInt(1)) + + result := slotDb.Suicide(addr) + if !result { + t.Fatalf("expected account suicide, got %v", result) + } + + if _, ok := slotDb.parallel.stateObjectSuicided[addr]; !ok { + t.Fatalf("address should exist in stateObjectSuicided") + } + + if _, ok := slotDb.parallel.addrStateChangeInSlot[addr]; !ok { + t.Fatalf("address should exist in addrStateChangeInSlot") + } + + if _, ok := slotDb.parallel.dirtiedStateObjectsInSlot[addr]; !ok { + t.Fatalf("address should exist in dirtiedStateObjectsInSlot") + } + + hasSuicide := slotDb.HasSuicided(addr) + if !hasSuicide { + t.Fatalf("address should be suicided") + } + + if _, ok := slotDb.parallel.addrStateReadInSlot[addr]; !ok { + t.Fatalf("address should exist in addrStateReadInSlot") + } +} + +func TestSetAndGetState(t *testing.T) { + memDb := rawdb.NewMemoryDatabase() + db := NewDatabase(memDb) + state, _ := New(common.Hash{}, db, nil) + slotDb := NewSlotDB(state, systemAddress, 0, false) + + addr := common.BytesToAddress([]byte("so")) + state.SetBalance(addr, big.NewInt(1)) + + slotDb.SetState(addr, common.BytesToHash([]byte("test key")), common.BytesToHash([]byte("test store"))) + + if _, ok := slotDb.parallel.dirtiedStateObjectsInSlot[addr]; !ok { + t.Fatalf("address should exist in dirtiedStateObjectsInSlot") + } + + if _, ok := slotDb.parallel.stateChangedInSlot[addr]; !ok { + t.Fatalf("address should exist in stateChangedInSlot") + } + + oldValueRead := state.GetState(addr, common.BytesToHash([]byte("test key"))) + emptyHash := common.Hash{} + if oldValueRead != emptyHash { + t.Fatalf("value read in old state should be empty") + } + + valueRead := slotDb.GetState(addr, common.BytesToHash([]byte("test key"))) + if valueRead != common.BytesToHash([]byte("test store")) { + t.Fatalf("value read should be equal to the stored value") + } + + if _, ok := slotDb.parallel.stateReadsInSlot[addr]; !ok { + t.Fatalf("address should exist in stateReadsInSlot") + } +} + +func TestSetAndGetCode(t *testing.T) { + memDb := rawdb.NewMemoryDatabase() + db := NewDatabase(memDb) + state, _ := New(common.Hash{}, db, nil) + slotDb := NewSlotDB(state, systemAddress, 0, false) + + addr := common.BytesToAddress([]byte("so")) + state.SetBalance(addr, big.NewInt(1)) + + if _, ok := slotDb.parallel.dirtiedStateObjectsInSlot[addr]; ok { + t.Fatalf("address should not exist in dirtiedStateObjectsInSlot") + } + + slotDb.SetCode(addr, []byte("test code")) + + if _, ok := slotDb.parallel.dirtiedStateObjectsInSlot[addr]; !ok { + t.Fatalf("address should exist in dirtiedStateObjectsInSlot") + } + + if _, ok := slotDb.parallel.codeChangeInSlot[addr]; !ok { + t.Fatalf("address should exist in codeChangeInSlot") + } + + codeRead := slotDb.GetCode(addr) + if string(codeRead) != "test code" { + t.Fatalf("code read should be equal to the code stored") + } + + if _, ok := slotDb.parallel.codeReadInSlot[addr]; !ok { + t.Fatalf("address should exist in codeReadInSlot") + } +} + +func TestGetCodeSize(t *testing.T) { + memDb := rawdb.NewMemoryDatabase() + db := NewDatabase(memDb) + state, _ := New(common.Hash{}, db, nil) + slotDb := NewSlotDB(state, systemAddress, 0, false) + + addr := common.BytesToAddress([]byte("so")) + state.SetBalance(addr, big.NewInt(1)) + + slotDb.SetCode(addr, []byte("test code")) + + codeSize := slotDb.GetCodeSize(addr) + if codeSize != 9 { + t.Fatalf("code size should be 9") + } + + if _, ok := slotDb.parallel.codeReadInSlot[addr]; !ok { + t.Fatalf("address should exist in codeReadInSlot") + } +} + +func TestGetCodeHash(t *testing.T) { + memDb := rawdb.NewMemoryDatabase() + db := NewDatabase(memDb) + state, _ := New(common.Hash{}, db, nil) + slotDb := NewSlotDB(state, systemAddress, 0, false) + + addr := common.BytesToAddress([]byte("so")) + state.SetBalance(addr, big.NewInt(1)) + + slotDb.SetCode(addr, []byte("test code")) + + codeSize := slotDb.GetCodeHash(addr) + print(hex.EncodeToString(codeSize[:])) + if hex.EncodeToString(codeSize[:]) != "6e73fa02f7828b28608b078b007a4023fb40453c3e102b83828a3609a94d8cbb" { + t.Fatalf("code hash should be 6e73fa02f7828b28608b078b007a4023fb40453c3e102b83828a3609a94d8cbb") + } + if _, ok := slotDb.parallel.codeReadInSlot[addr]; !ok { + t.Fatalf("address should exist in codeReadInSlot") + } +} + +func TestSetNonce(t *testing.T) { + memDb := rawdb.NewMemoryDatabase() + db := NewDatabase(memDb) + state, _ := New(common.Hash{}, db, nil) + slotDb := NewSlotDB(state, systemAddress, 0, false) + + addr := common.BytesToAddress([]byte("so")) + state.SetBalance(addr, big.NewInt(1)) + state.SetNonce(addr, 1) + + slotDb.SetNonce(addr, 2) + + oldNonce := state.GetNonce(addr) + if oldNonce != 1 { + t.Fatalf("old nonce should be 1") + } + + newNonce := slotDb.GetNonce(addr) + if newNonce != 2 { + t.Fatalf("new nonce should be 2") + } + if _, ok := slotDb.parallel.dirtiedStateObjectsInSlot[addr]; !ok { + t.Fatalf("address should exist in dirtiedStateObjectsInSlot") + } +} + +func TestSetAndGetBalance(t *testing.T) { + memDb := rawdb.NewMemoryDatabase() + db := NewDatabase(memDb) + state, _ := New(common.Hash{}, db, nil) + slotDb := NewSlotDB(state, systemAddress, 0, true) + + addr := systemAddress + state.SetBalance(addr, big.NewInt(1)) + + slotDb.SetBalance(addr, big.NewInt(2)) + + oldBalance := state.GetBalance(addr) + if oldBalance.Int64() != 1 { + t.Fatalf("old balance should be 1") + } + + if _, ok := slotDb.parallel.dirtiedStateObjectsInSlot[addr]; !ok { + t.Fatalf("address should exist in dirtiedStateObjectsInSlot") + } + + if _, ok := slotDb.parallel.balanceChangedInSlot[addr]; !ok { + t.Fatalf("address should exist in balanceChangedInSlot") + } + + if slotDb.parallel.systemAddressCount != 1 { + t.Fatalf("systemAddressCount should be 1") + } + + newBalance := slotDb.GetBalance(addr) + if newBalance.Int64() != 2 { + t.Fatalf("new nonce should be 2") + } + + if _, ok := slotDb.parallel.balanceReadsInSlot[addr]; !ok { + t.Fatalf("address should exist in balanceReadsInSlot") + } + + if slotDb.parallel.systemAddressCount != 2 { + t.Fatalf("systemAddressCount should be 1") + } +} + +func TestSubBalance(t *testing.T) { + memDb := rawdb.NewMemoryDatabase() + db := NewDatabase(memDb) + state, _ := New(common.Hash{}, db, nil) + slotDb := NewSlotDB(state, systemAddress, 0, true) + + addr := systemAddress + state.SetBalance(addr, big.NewInt(2)) + + slotDb.SubBalance(addr, big.NewInt(1)) + + oldBalance := state.GetBalance(addr) + if oldBalance.Int64() != 2 { + t.Fatalf("old balance should be 1") + } + + if _, ok := slotDb.parallel.dirtiedStateObjectsInSlot[addr]; !ok { + t.Fatalf("address should exist in dirtiedStateObjectsInSlot") + } + + if _, ok := slotDb.parallel.balanceChangedInSlot[addr]; !ok { + t.Fatalf("address should exist in balanceChangedInSlot") + } + + if _, ok := slotDb.parallel.balanceReadsInSlot[addr]; !ok { + t.Fatalf("address should exist in balanceReadsInSlot") + } + + if slotDb.parallel.systemAddressCount != 1 { + t.Fatalf("systemAddressCount should be 1") + } + + newBalance := slotDb.GetBalance(addr) + if newBalance.Int64() != 1 { + t.Fatalf("new nonce should be 2") + } +} + +func TestAddBalance(t *testing.T) { + memDb := rawdb.NewMemoryDatabase() + db := NewDatabase(memDb) + state, _ := New(common.Hash{}, db, nil) + slotDb := NewSlotDB(state, systemAddress, 0, true) + + addr := systemAddress + state.SetBalance(addr, big.NewInt(2)) + + slotDb.AddBalance(addr, big.NewInt(1)) + + oldBalance := state.GetBalance(addr) + if oldBalance.Int64() != 2 { + t.Fatalf("old balance should be 1") + } + + if _, ok := slotDb.parallel.dirtiedStateObjectsInSlot[addr]; !ok { + t.Fatalf("address should exist in dirtiedStateObjectsInSlot") + } + + if _, ok := slotDb.parallel.balanceChangedInSlot[addr]; !ok { + t.Fatalf("address should exist in balanceChangedInSlot") + } + + if _, ok := slotDb.parallel.balanceReadsInSlot[addr]; !ok { + t.Fatalf("address should exist in balanceReadsInSlot") + } + + if slotDb.parallel.systemAddressCount != 1 { + t.Fatalf("systemAddressCount should be 1") + } + + newBalance := slotDb.GetBalance(addr) + if newBalance.Int64() != 3 { + t.Fatalf("new nonce should be 2") + } +} + +func TestEmpty(t *testing.T) { + memDb := rawdb.NewMemoryDatabase() + db := NewDatabase(memDb) + state, _ := New(common.Hash{}, db, nil) + slotDb := NewSlotDB(state, systemAddress, 0, true) + + addr := systemAddress + state.SetBalance(addr, big.NewInt(2)) + + empty := slotDb.Empty(addr) + if empty { + t.Fatalf("address should exist") + } + + if _, ok := slotDb.parallel.addrStateReadInSlot[addr]; !ok { + t.Fatalf("address should exist in addrStateReadInSlot") + } +} + +func TestExist(t *testing.T) { + memDb := rawdb.NewMemoryDatabase() + db := NewDatabase(memDb) + state, _ := New(common.Hash{}, db, nil) + slotDb := NewSlotDB(state, systemAddress, 0, true) + + addr := systemAddress + state.SetBalance(addr, big.NewInt(2)) + + exist := slotDb.Exist(addr) + if !exist { + t.Fatalf("address should exist") + } + + if _, ok := slotDb.parallel.addrStateReadInSlot[addr]; !ok { + t.Fatalf("address should exist in addrStateReadInSlot") + } +} + +func TestMergeSlotDB(t *testing.T) { + memDb := rawdb.NewMemoryDatabase() + db := NewDatabase(memDb) + state, _ := New(common.Hash{}, db, nil) + oldSlotDb := NewSlotDB(state, systemAddress, 0, true) + + newSlotDb := NewSlotDB(state, systemAddress, 0, true) + + addr := systemAddress + newSlotDb.SetBalance(addr, big.NewInt(2)) + newSlotDb.SetState(addr, common.BytesToHash([]byte("test key")), common.BytesToHash([]byte("test store"))) + newSlotDb.SetCode(addr, []byte("test code")) + newSlotDb.Suicide(addr) + + changeList := oldSlotDb.MergeSlotDB(newSlotDb, &types.Receipt{}, 0) + + if _, ok := changeList.StateObjectSuicided[addr]; !ok { + t.Fatalf("address should exist in StateObjectSuicided") + } + + if _, ok := changeList.StateObjectSuicided[addr]; !ok { + t.Fatalf("address should exist in StateObjectSuicided") + } + + if _, ok := changeList.StateChangeSet[addr]; !ok { + t.Fatalf("address should exist in StateChangeSet") + } + + if _, ok := changeList.BalanceChangeSet[addr]; !ok { + t.Fatalf("address should exist in StateChangeSet") + } + + if _, ok := changeList.CodeChangeSet[addr]; !ok { + t.Fatalf("address should exist in CodeChangeSet") + } + + if _, ok := changeList.AddrStateChangeSet[addr]; !ok { + t.Fatalf("address should exist in AddrStateChangeSet") + } +} diff --git a/core/state_processor.go b/core/state_processor.go index 80ac42852c..bf11f464d9 100644 --- a/core/state_processor.go +++ b/core/state_processor.go @@ -24,6 +24,7 @@ import ( "math/rand" "runtime" "sync" + "sync/atomic" "time" "github.com/ethereum/go-ethereum/common" @@ -63,10 +64,12 @@ type StateProcessor struct { engine consensus.Engine // Consensus engine used for block rewards // add for parallel execute - paraInitialized bool // todo: should use atomic value - paraTxResultChan chan *ParallelTxResult // to notify dispatcher that a tx is done - slotState []*SlotState // idle, or pending messages - mergedTxIndex int // the latest finalized tx index + paraInitialized int32 + paraTxResultChan chan *ParallelTxResult // to notify dispatcher that a tx is done + slotState []*SlotState // idle, or pending messages + mergedTxIndex int // the latest finalized tx index + debugErrorRedoNum int + debugConflictRedoNum int } // NewStateProcessor initialises a new StateProcessor. @@ -381,98 +384,11 @@ func (p *LightStateProcessor) LightProcess(diffLayer *types.DiffLayer, block *ty return diffLayer.Receipts, allLogs, gasUsed, nil } -// Process processes the state changes according to the Ethereum rules by running -// the transaction messages using the statedb and applying any rewards to both -// the processor (coinbase) and any included uncles. -// -// Process returns the receipts and logs accumulated during the process and -// returns the amount of gas that was used in the process. If any of the -// transactions failed to execute due to insufficient gas it will return an error. -func (p *StateProcessor) Process(block *types.Block, statedb *state.StateDB, cfg vm.Config) (*state.StateDB, types.Receipts, []*types.Log, uint64, error) { - var ( - usedGas = new(uint64) - header = block.Header() - allLogs []*types.Log - gp = new(GasPool).AddGas(block.GasLimit()) - ) - signer := types.MakeSigner(p.bc.chainConfig, block.Number()) - statedb.TryPreload(block, signer) - var receipts = make([]*types.Receipt, 0) - // Mutate the block and state according to any hard-fork specs - if p.config.DAOForkSupport && p.config.DAOForkBlock != nil && p.config.DAOForkBlock.Cmp(block.Number()) == 0 { - misc.ApplyDAOHardFork(statedb) - } - // Handle upgrade build-in system contract code - systemcontracts.UpgradeBuildInSystemContract(p.config, block.Number(), statedb) - - blockContext := NewEVMBlockContext(header, p.bc, nil) - vmenv := vm.NewEVM(blockContext, vm.TxContext{}, statedb, p.config, cfg) - - txNum := len(block.Transactions()) - // Iterate over and process the individual transactions - posa, isPoSA := p.engine.(consensus.PoSA) - commonTxs := make([]*types.Transaction, 0, txNum) - - // initialise bloom processors - bloomProcessors := NewAsyncReceiptBloomGenerator(txNum) - statedb.MarkFullProcessed() - - // usually do have two tx, one for validator set contract, another for system reward contract. - systemTxs := make([]*types.Transaction, 0, 2) - for i, tx := range block.Transactions() { - if isPoSA { - if isSystemTx, err := posa.IsSystemTransaction(tx, block.Header()); err != nil { - return statedb, nil, nil, 0, err - } else if isSystemTx { - systemTxs = append(systemTxs, tx) - continue - } - } - - msg, err := tx.AsMessage(signer) - if err != nil { - return statedb, nil, nil, 0, err - } - statedb.Prepare(tx.Hash(), block.Hash(), i) - receipt, err := applyTransaction(msg, p.config, p.bc, nil, gp, statedb, header, tx, usedGas, vmenv, bloomProcessors) - if err != nil { - return statedb, nil, nil, 0, fmt.Errorf("could not apply tx %d [%v]: %w", i, tx.Hash().Hex(), err) - } - - commonTxs = append(commonTxs, tx) - receipts = append(receipts, receipt) - } - bloomProcessors.Close() - - // Finalize the block, applying any consensus engine specific extras (e.g. block rewards) - err := p.engine.Finalize(p.bc, header, statedb, &commonTxs, block.Uncles(), &receipts, &systemTxs, usedGas) - if err != nil { - return statedb, receipts, allLogs, *usedGas, err - } - for _, receipt := range receipts { - allLogs = append(allLogs, receipt.Logs...) - } - - return statedb, receipts, allLogs, *usedGas, nil -} - -type MergedTxInfo struct { - slotDB *state.StateDB // used for SlotDb reuse only, otherwise, it can be discarded - StateObjectSuicided map[common.Address]struct{} - StateChangeSet map[common.Address]state.StateKeys - BalanceChangeSet map[common.Address]struct{} - CodeChangeSet map[common.Address]struct{} - AddrStateChangeSet map[common.Address]struct{} - txIndex int -} - type SlotState struct { - tailTxReq *ParallelTxRequest // tail pending Tx of the slot, should be accessed on dispatcher only. - pendingExec chan *ParallelTxRequest - // slot needs to keep the historical stateDB for conflict check - // each finalized DB should match a TX index - mergedTxInfo []MergedTxInfo - slotdbChan chan *state.StateDB // dispatch will create and send this slotDB to slot + tailTxReq *ParallelTxRequest // tail pending Tx of the slot, should be accessed on dispatcher only. + pendingExec chan *ParallelTxRequest + mergedChangeList []state.SlotChangeList + slotdbChan chan *state.StateDB // dispatch will create and send this slotDB to slot // conflict check uses conflict window // conflict check will check all state changes from (cfWindowStart + 1) to the previous Tx } @@ -505,15 +421,41 @@ type ParallelTxRequest struct { curTxChan chan int // "int" represents the tx index } -// if any state in readDb is updated in writeDb, then it has state conflict -func (p *StateProcessor) hasStateConflict(readDb *state.StateDB, mergedInfo MergedTxInfo) bool { +func (p *StateProcessor) InitParallelOnce() { + // to create and start the execution slot goroutines + if !atomic.CompareAndSwapInt32(&p.paraInitialized, 0, 1) { // not swapped means already initialized. + return + } + log.Info("Parallel execution mode is used and initialized", "Parallel Num", ParallelExecNum) + p.paraTxResultChan = make(chan *ParallelTxResult, ParallelExecNum) // fixme: use blocked chan? + p.slotState = make([]*SlotState, ParallelExecNum) + + wg := sync.WaitGroup{} // make sure all goroutines are created and started + for i := 0; i < ParallelExecNum; i++ { + p.slotState[i] = new(SlotState) + p.slotState[i].slotdbChan = make(chan *state.StateDB, 1) + p.slotState[i].pendingExec = make(chan *ParallelTxRequest, MaxPendingQueueSize) + + wg.Add(1) + // start the slot's goroutine + go func(slotIndex int) { + wg.Done() + p.runSlotLoop(slotIndex) // this loop will be permanent live + log.Error("runSlotLoop exit!", "Slot", slotIndex) + }(i) + } + wg.Wait() +} + +// if any state in readDb is updated in changeList, then it has state conflict +func (p *StateProcessor) hasStateConflict(readDb *state.StateDB, changeList state.SlotChangeList) bool { // check KV change reads := readDb.StateReadsInSlot() - writes := mergedInfo.StateChangeSet + writes := changeList.StateChangeSet if len(reads) != 0 && len(writes) != 0 { for readAddr, readKeys := range reads { - if _, exist := mergedInfo.StateObjectSuicided[readAddr]; exist { - log.Debug("hasStateConflict read suicide object", "addr", readAddr) + if _, exist := changeList.StateObjectSuicided[readAddr]; exist { + log.Debug("conflict: read suicide object", "addr", readAddr) return true } if writeKeys, ok := writes[readAddr]; ok { @@ -521,7 +463,7 @@ func (p *StateProcessor) hasStateConflict(readDb *state.StateDB, mergedInfo Merg for writeKey := range writeKeys { // same addr and same key, mark conflicted if _, ok := readKeys[writeKey]; ok { - log.Info("hasStateConflict state conflict", "addr", readAddr, "key", writeKey) + log.Debug("conflict: state conflict", "addr", readAddr, "key", writeKey) return true } } @@ -530,19 +472,19 @@ func (p *StateProcessor) hasStateConflict(readDb *state.StateDB, mergedInfo Merg } // check balance change balanceReads := readDb.BalanceReadsInSlot() - balanceWrite := mergedInfo.BalanceChangeSet + balanceWrite := changeList.BalanceChangeSet if len(balanceReads) != 0 && len(balanceWrite) != 0 { for readAddr := range balanceReads { - if _, exist := mergedInfo.StateObjectSuicided[readAddr]; exist { - log.Debug("hasStateConflict read suicide balance", "addr", readAddr) + if _, exist := changeList.StateObjectSuicided[readAddr]; exist { + log.Debug("conflict: read suicide balance", "addr", readAddr) return true } if _, ok := balanceWrite[readAddr]; ok { if readAddr == consensus.SystemAddress { - log.Info("hasStateConflict skip specical system address's balance check") + log.Debug("conflict: skip specical system address's balance check") continue } - log.Info("hasStateConflict balance conflict", "addr", readAddr) + log.Debug("conflict: balance conflict", "addr", readAddr) return true } } @@ -550,15 +492,15 @@ func (p *StateProcessor) hasStateConflict(readDb *state.StateDB, mergedInfo Merg // check code change codeReads := readDb.CodeReadInSlot() - codeWrite := mergedInfo.CodeChangeSet + codeWrite := changeList.CodeChangeSet if len(codeReads) != 0 && len(codeWrite) != 0 { for readAddr := range codeReads { - if _, exist := mergedInfo.StateObjectSuicided[readAddr]; exist { - log.Debug("hasStateConflict read suicide code", "addr", readAddr) + if _, exist := changeList.StateObjectSuicided[readAddr]; exist { + log.Debug("conflict: read suicide code", "addr", readAddr) return true } if _, ok := codeWrite[readAddr]; ok { - log.Debug("hasStateConflict code conflict", "addr", readAddr) + log.Debug("conflict: code conflict", "addr", readAddr) return true } } @@ -566,11 +508,11 @@ func (p *StateProcessor) hasStateConflict(readDb *state.StateDB, mergedInfo Merg // check address state change: create, suicide... addrReads := readDb.AddressReadInSlot() - addrWrite := mergedInfo.AddrStateChangeSet + addrWrite := changeList.AddrStateChangeSet if len(addrReads) != 0 && len(addrWrite) != 0 { for readAddr := range addrReads { if _, ok := addrWrite[readAddr]; ok { - log.Info("hasStateConflict address state conflict", "addr", readAddr) + log.Debug("conflict: address state conflict", "addr", readAddr) return true } } @@ -594,7 +536,7 @@ func (p *StateProcessor) queueToSameAddress(execMsg *ParallelTxRequest) bool { // To() == nil means contract creation, won't queue to such slot. if slot.tailTxReq.tx.To() == nil { - // log.Debug("queueToSameAddress, slot's To address is nil", "slotIndex", i) + // log.Debug("queueToSameAddress, slot's To address is nil", "Slot", i) continue } // same to address, put it on slot's pending list. @@ -602,10 +544,10 @@ func (p *StateProcessor) queueToSameAddress(execMsg *ParallelTxRequest) bool { select { case slot.pendingExec <- execMsg: slot.tailTxReq = execMsg - log.Debug("queueToSameAddress", "slotIndex", i, "txIndex", execMsg.txIndex) + log.Debug("queueToSameAddress", "Slot", i, "txIndex", execMsg.txIndex) return true default: - log.Debug("queueToSameAddress but queue is full", "slotIndex", i, "txIndex", execMsg.txIndex) + log.Debug("queueToSameAddress but queue is full", "Slot", i, "txIndex", execMsg.txIndex) return false } } @@ -618,10 +560,10 @@ func (p *StateProcessor) dispatchToIdleSlot(statedb *state.StateDB, txReq *Paral for i, slot := range p.slotState { if slot.tailTxReq == nil { // for idle slot, we have to create a SlotDB for it. - if len(slot.mergedTxInfo) == 0 { + if len(slot.mergedChangeList) == 0 { txReq.slotDB = state.NewSlotDB(statedb, consensus.SystemAddress, p.mergedTxIndex, false) } - log.Debug("dispatchToIdleSlot", "slotIndex", i, "txIndex", txReq.txIndex) + log.Debug("dispatchToIdleSlot", "Slot", i, "txIndex", txReq.txIndex) slot.tailTxReq = txReq slot.pendingExec <- txReq return true @@ -645,8 +587,8 @@ func (p *StateProcessor) waitUntilNextTxDone(statedb *state.StateDB) *ParallelTx slotState := p.slotState[result.slotIndex] var slotDB *state.StateDB if result.reuseSlotDB { - // for reuse, len(slotState.mergedTxInfo) must >= 1 - lastSlotDB := slotState.mergedTxInfo[len(slotState.mergedTxInfo)-1].slotDB + // for reuse, len(slotState.mergedChangeList) must >= 1 + lastSlotDB := slotState.mergedChangeList[len(slotState.mergedChangeList)-1].SlotDB slotDB = state.ReUseSlotDB(lastSlotDB, result.keepSystem) } else { slotDB = state.NewSlotDB(statedb, consensus.SystemAddress, p.mergedTxIndex, result.keepSystem) @@ -665,19 +607,15 @@ func (p *StateProcessor) waitUntilNextTxDone(statedb *state.StateDB) *ParallelTx resultTxIndex := result.txIndex resultSlotState := p.slotState[resultSlotIndex] if resultSlotState.tailTxReq.txIndex == resultTxIndex { - log.Debug("ProcessParallel slot is idle", "slotIndex", resultSlotIndex) + log.Debug("ProcessParallel slot is idle", "Slot", resultSlotIndex) resultSlotState.tailTxReq = nil } - // merge slotDB to parent stateDB - log.Info("ProcessParallel a tx is done, merge to block stateDB", - "resultSlotIndex", resultSlotIndex, "resultTxIndex", resultTxIndex) - objSuicided, stateChanges, balanceChanges, codeChanges, addrChanges := statedb.MergeSlotDB(result.slotDB, result.receipt) - // slot's mergedTxInfo is updated by dispatcher, while consumed by slot. + // Slot's mergedChangeList is produced by dispatcher, while consumed by slot. // It is safe, since write and read is in sequential, do write -> notify -> read - // it is not good, but work right now. - - resultSlotState.mergedTxInfo = append(resultSlotState.mergedTxInfo, MergedTxInfo{result.slotDB, objSuicided, stateChanges, balanceChanges, codeChanges, addrChanges, resultTxIndex}) + // It is not good, but work right now. + changeList := statedb.MergeSlotDB(result.slotDB, result.receipt, resultTxIndex) + resultSlotState.mergedChangeList = append(resultSlotState.mergedChangeList, changeList) if resultTxIndex != p.mergedTxIndex+1 { log.Warn("ProcessParallel tx result out of order", "resultTxIndex", resultTxIndex, @@ -685,7 +623,8 @@ func (p *StateProcessor) waitUntilNextTxDone(statedb *state.StateDB) *ParallelTx panic("ProcessParallel tx result out of order") } p.mergedTxIndex = resultTxIndex - // notify the following Tx, it is merged, what if no wait or next tx is in same slot? + // notify the following Tx, it is merged, + // fixme: what if no wait or next tx is in same slot? result.txReq.curTxChan <- resultTxIndex return result } @@ -694,7 +633,6 @@ func (p *StateProcessor) execInParallelSlot(slotIndex int, txReq *ParallelTxRequ txIndex := txReq.txIndex tx := txReq.tx slotDB := txReq.slotDB - slotDB.SlotIndex = slotIndex gp := txReq.gp // goroutine unsafe msg := txReq.msg block := txReq.block @@ -712,19 +650,19 @@ func (p *StateProcessor) execInParallelSlot(slotIndex int, txReq *ParallelTxRequ // fixme: to optimize, reuse the slotDB slotDB.Prepare(tx.Hash(), block.Hash(), txIndex) - log.Debug("execInParallelSlot enter", "slotIndex", slotIndex, "txIndex", txIndex, "slotDB.baseTxIndex", slotDB.BaseTxIndex()) + log.Debug("exec In Slot", "Slot", slotIndex, "txIndex", txIndex, "slotDB.baseTxIndex", slotDB.BaseTxIndex()) slotGasLimit := gp.Gas() gpSlot := new(GasPool).AddGas(slotGasLimit) // each slot would use its own gas pool, and will do gaslimit check later evm, result, err = applyTransactionStageExecution(msg, gpSlot, slotDB, vmenv) - log.Debug("execInParallelSlot Stage Execution done", "slotIndex", slotIndex, "txIndex", txIndex, "slotDB.baseTxIndex", slotDB.BaseTxIndex()) + log.Debug("Stage Execution done", "Slot", slotIndex, "txIndex", txIndex, "slotDB.baseTxIndex", slotDB.BaseTxIndex()) // wait until the previous tx is finalized. if txReq.waitTxChan != nil { - log.Info("execInParallelSlot wait previous Tx done", "my slotIndex", slotIndex, "txIndex", txIndex) + log.Debug("Stage wait previous Tx done", "Slot", slotIndex, "txIndex", txIndex) waitTxIndex := <-txReq.waitTxChan if waitTxIndex != txIndex-1 { - log.Error("execInParallelSlot wait tx index mismatch", "expect", txIndex-1, "actual", waitTxIndex) + log.Error("Stage wait tx index mismatch", "expect", txIndex-1, "actual", waitTxIndex) panic(fmt.Sprintf("wait tx index mismatch expect:%d, actual:%d", txIndex-1, waitTxIndex)) } } @@ -733,7 +671,8 @@ func (p *StateProcessor) execInParallelSlot(slotIndex int, txReq *ParallelTxRequ // for example: err="nonce too high" // in this case, we will do re-run. if err != nil { - log.Debug("Stage Execution err", "slotIndex", slotIndex, "txIndex", txIndex, + p.debugErrorRedoNum++ + log.Debug("Stage Execution err", "Slot", slotIndex, "txIndex", txIndex, "current slotDB.baseTxIndex", slotDB.BaseTxIndex(), "err", err) redoResult := &ParallelTxResult{ redo: true, @@ -748,10 +687,9 @@ func (p *StateProcessor) execInParallelSlot(slotIndex int, txReq *ParallelTxRequ } p.paraTxResultChan <- redoResult slotDB = <-p.slotState[slotIndex].slotdbChan - slotDB.SlotIndex = slotIndex slotDB.Prepare(tx.Hash(), block.Hash(), txIndex) // vmenv.Reset(vm.TxContext{}, slotDB) - log.Debug("Stage Execution get new slotdb to redo", "slotIndex", slotIndex, + log.Debug("Stage Execution get new slotdb to redo", "Slot", slotIndex, "txIndex", txIndex, "new slotDB.baseTxIndex", slotDB.BaseTxIndex()) slotGasLimit = gp.Gas() gpSlot = new(GasPool).AddGas(slotGasLimit) @@ -769,7 +707,7 @@ func (p *StateProcessor) execInParallelSlot(slotIndex int, txReq *ParallelTxRequ hasConflict := false systemAddrConflict := false - log.Debug("execInParallelSlot Tx Stage1 done, do conflict check", "slotIndex", slotIndex, "txIndex", txIndex) + log.Debug("Stage Execution done, do conflict check", "Slot", slotIndex, "txIndex", txIndex) if slotDB.SystemAddressRedo() { hasConflict = true systemAddrConflict = true @@ -783,13 +721,13 @@ func (p *StateProcessor) execInParallelSlot(slotIndex int, txReq *ParallelTxRequ } // check all finalizedDb from current slot's - for _, mergedInfo := range p.slotState[index].mergedTxInfo { - if mergedInfo.txIndex <= slotDB.BaseTxIndex() { - // log.Info("skip finalized DB which is out of the conflict window", "finDb.txIndex", finDb.txIndex, "slotDB.baseTxIndex", slotDB.baseTxIndex) + for _, changeList := range p.slotState[index].mergedChangeList { + if changeList.TxIndex <= slotDB.BaseTxIndex() { + // log.Debug("skip finalized DB which is out of the conflict window", "finDb.txIndex", finDb.txIndex, "slotDB.baseTxIndex", slotDB.baseTxIndex) continue } - if p.hasStateConflict(slotDB, mergedInfo) { - log.Debug("execInParallelSlot Stage Execution conflict", "slotIndex", slotIndex, + if p.hasStateConflict(slotDB, changeList) { + log.Debug("Stage Execution conflict", "Slot", slotIndex, "txIndex", txIndex, " conflict slot", index, "slotDB.baseTxIndex", slotDB.BaseTxIndex()) hasConflict = true break @@ -802,6 +740,7 @@ func (p *StateProcessor) execInParallelSlot(slotIndex int, txReq *ParallelTxRequ } if hasConflict { + p.debugConflictRedoNum++ // re-run should not have conflict, since it has the latest world state. redoResult := &ParallelTxResult{ redo: true, @@ -817,7 +756,6 @@ func (p *StateProcessor) execInParallelSlot(slotIndex int, txReq *ParallelTxRequ } p.paraTxResultChan <- redoResult slotDB = <-p.slotState[slotIndex].slotdbChan - slotDB.SlotIndex = slotIndex slotDB.Prepare(tx.Hash(), block.Hash(), txIndex) // vmenv.Reset(vm.TxContext{}, slotDB) slotGasLimit = gp.Gas() @@ -831,7 +769,7 @@ func (p *StateProcessor) execInParallelSlot(slotIndex int, txReq *ParallelTxRequ // goroutine unsafe operation will be handled from here for safety gasConsumed := slotGasLimit - gpSlot.Gas() if gasConsumed != result.UsedGas { - log.Error("execInParallelSlot gasConsumed != result.UsedGas mismatch", + log.Error("gasConsumed != result.UsedGas mismatch", "gasConsumed", gasConsumed, "result.UsedGas", result.UsedGas) panic(fmt.Sprintf("gas consume mismatch, consumed:%d, result.UsedGas:%d", gasConsumed, result.UsedGas)) } @@ -841,15 +779,17 @@ func (p *StateProcessor) execInParallelSlot(slotIndex int, txReq *ParallelTxRequ panic(fmt.Sprintf("gas limit reached, gasConsumed:%d, gp.Gas():%d", gasConsumed, gp.Gas())) } - log.Debug("execInParallelSlot ok to finalize this TX", - "slotIndex", slotIndex, "txIndex", txIndex, "result.UsedGas", result.UsedGas, "txReq.usedGas", *txReq.usedGas) + log.Debug("ok to finalize this TX", + "Slot", slotIndex, "txIndex", txIndex, "result.UsedGas", result.UsedGas, "txReq.usedGas", *txReq.usedGas) // ok, time to do finalize, stage2 should not be parallel receipt, err = applyTransactionStageFinalization(evm, result, msg, p.config, slotDB, header, tx, txReq.usedGas, bloomProcessors) - if result.Err != nil { + if result.Failed() { // if Tx is reverted, all its state change will be discarded - log.Debug("execInParallelSlot TX reverted?", "slotIndex", slotIndex, "txIndex", txIndex, "result.Err", result.Err) + log.Debug("TX reverted?", "Slot", slotIndex, "txIndex", txIndex, "result.Err", result.Err) + slotDB.RevertSlotDB(msg.From()) } + return &ParallelTxResult{ redo: false, updateSlotDB: false, @@ -866,11 +806,11 @@ func (p *StateProcessor) execInParallelSlot(slotIndex int, txReq *ParallelTxRequ func (p *StateProcessor) runSlotLoop(slotIndex int) { curSlot := p.slotState[slotIndex] for { - // log.Info("parallel slot waiting", "slotIndex:", slotIndex) + // log.Info("parallel slot waiting", "Slot", slotIndex) // wait for new TxReq txReq := <-curSlot.pendingExec // receive a dispatched message - log.Debug("SlotLoop received a new TxReq", "slotIndex:", slotIndex, "txIndex", txReq.txIndex) + log.Debug("SlotLoop received a new TxReq", "Slot", slotIndex, "txIndex", txReq.txIndex) // SlotDB create rational: // ** for a dispatched tx, @@ -892,80 +832,135 @@ func (p *StateProcessor) runSlotLoop(slotIndex int) { txReq.slotDB = <-curSlot.slotdbChan } result := p.execInParallelSlot(slotIndex, txReq) - log.Debug("SlotLoop the TxReq is done", "slotIndex:", slotIndex, "err", result.err) + log.Debug("SlotLoop the TxReq is done", "Slot", slotIndex, "err", result.err) p.paraTxResultChan <- result } } // clear slotState for each block. -func (p *StateProcessor) resetSlotState() { +func (p *StateProcessor) resetParallelState(txNum int) { + if txNum == 0 { + return + } p.mergedTxIndex = -1 + p.debugErrorRedoNum = 0 + p.debugConflictRedoNum = 0 + for _, slotState := range p.slotState { slotState.tailTxReq = nil - slotState.mergedTxInfo = make([]MergedTxInfo, 0) + slotState.mergedChangeList = make([]state.SlotChangeList, 0) } } -func (p *StateProcessor) InitParallelOnce() { - // to create and start the execution slot goroutines - if p.paraInitialized { - return +// Before transactions are executed, do shared preparation for Process() & ProcessParallel() +func (p *StateProcessor) preExecute(block *types.Block, statedb *state.StateDB, cfg vm.Config, parallel bool) (types.Signer, *vm.EVM, *AsyncReceiptBloomGenerator) { + signer := types.MakeSigner(p.bc.chainConfig, block.Number()) + statedb.TryPreload(block, signer) + // Mutate the block and state according to any hard-fork specs + if p.config.DAOForkSupport && p.config.DAOForkBlock != nil && p.config.DAOForkBlock.Cmp(block.Number()) == 0 { + misc.ApplyDAOHardFork(statedb) } + // Handle upgrade build-in system contract code + systemcontracts.UpgradeBuildInSystemContract(p.config, block.Number(), statedb) - p.paraTxResultChan = make(chan *ParallelTxResult, ParallelExecNum) // fixme: use blocked chan? - p.slotState = make([]*SlotState, ParallelExecNum) + blockContext := NewEVMBlockContext(block.Header(), p.bc, nil) + // with parallel mode, vmenv will be created inside of slot + var vmenv *vm.EVM + if !parallel { + vmenv = vm.NewEVM(blockContext, vm.TxContext{}, statedb, p.config, cfg) + } - wg := sync.WaitGroup{} // make sure all goroutines are created and started - for i := 0; i < ParallelExecNum; i++ { - p.slotState[i] = new(SlotState) - p.slotState[i].slotdbChan = make(chan *state.StateDB, 1) - p.slotState[i].pendingExec = make(chan *ParallelTxRequest, MaxPendingQueueSize) + // initialise bloom processors + bloomProcessors := NewAsyncReceiptBloomGenerator(len(block.Transactions())) + statedb.MarkFullProcessed() - wg.Add(1) - // start the slot's goroutine - go func(slotIndex int) { - wg.Done() - p.runSlotLoop(slotIndex) // this loop will be permanent live - log.Error("runSlotLoop exit!", "slotIndex", slotIndex) - }(i) + return signer, vmenv, bloomProcessors +} + +func (p *StateProcessor) postExecute(block *types.Block, statedb *state.StateDB, commonTxs *[]*types.Transaction, + receipts *[]*types.Receipt, systemTxs *[]*types.Transaction, usedGas *uint64, bloomProcessors *AsyncReceiptBloomGenerator) ([]*types.Log, error) { + var allLogs []*types.Log + + bloomProcessors.Close() + + // Finalize the block, applying any consensus engine specific extras (e.g. block rewards) + err := p.engine.Finalize(p.bc, block.Header(), statedb, commonTxs, block.Uncles(), receipts, systemTxs, usedGas) + if err != nil { + return allLogs, err } - wg.Wait() - p.paraInitialized = true + for _, receipt := range *receipts { + allLogs = append(allLogs, receipt.Logs...) + } + return allLogs, nil } -func (p *StateProcessor) ProcessParallel(block *types.Block, statedb *state.StateDB, cfg vm.Config) (*state.StateDB, types.Receipts, []*types.Log, uint64, error) { - p.InitParallelOnce() +// Process processes the state changes according to the Ethereum rules by running +// the transaction messages using the statedb and applying any rewards to both +// the processor (coinbase) and any included uncles. +// +// Process returns the receipts and logs accumulated during the process and +// returns the amount of gas that was used in the process. If any of the +// transactions failed to execute due to insufficient gas it will return an error. +func (p *StateProcessor) Process(block *types.Block, statedb *state.StateDB, cfg vm.Config) (*state.StateDB, types.Receipts, []*types.Log, uint64, error) { var ( usedGas = new(uint64) header = block.Header() - allLogs []*types.Log gp = new(GasPool).AddGas(block.GasLimit()) ) - signer := types.MakeSigner(p.bc.chainConfig, block.Number()) - statedb.TryPreload(block, signer) var receipts = make([]*types.Receipt, 0) - // Mutate the block and state according to any hard-fork specs - if p.config.DAOForkSupport && p.config.DAOForkBlock != nil && p.config.DAOForkBlock.Cmp(block.Number()) == 0 { - misc.ApplyDAOHardFork(statedb) + txNum := len(block.Transactions()) + commonTxs := make([]*types.Transaction, 0, txNum) + // Iterate over and process the individual transactions + posa, isPoSA := p.engine.(consensus.PoSA) + // usually do have two tx, one for validator set contract, another for system reward contract. + systemTxs := make([]*types.Transaction, 0, 2) + + signer, vmenv, bloomProcessors := p.preExecute(block, statedb, cfg, false) + for i, tx := range block.Transactions() { + if isPoSA { + if isSystemTx, err := posa.IsSystemTransaction(tx, block.Header()); err != nil { + return statedb, nil, nil, 0, err + } else if isSystemTx { + systemTxs = append(systemTxs, tx) + continue + } + } + + msg, err := tx.AsMessage(signer) + if err != nil { + return statedb, nil, nil, 0, err + } + statedb.Prepare(tx.Hash(), block.Hash(), i) + receipt, err := applyTransaction(msg, p.config, p.bc, nil, gp, statedb, header, tx, usedGas, vmenv, bloomProcessors) + if err != nil { + return statedb, nil, nil, 0, fmt.Errorf("could not apply tx %d [%v]: %w", i, tx.Hash().Hex(), err) + } + + commonTxs = append(commonTxs, tx) + receipts = append(receipts, receipt) } - // Handle upgrade build-in system contract code - systemcontracts.UpgradeBuildInSystemContract(p.config, block.Number(), statedb) + allLogs, err := p.postExecute(block, statedb, &commonTxs, &receipts, &systemTxs, usedGas, bloomProcessors) + return statedb, receipts, allLogs, *usedGas, err +} + +func (p *StateProcessor) ProcessParallel(block *types.Block, statedb *state.StateDB, cfg vm.Config) (*state.StateDB, types.Receipts, []*types.Log, uint64, error) { + var ( + usedGas = new(uint64) + header = block.Header() + gp = new(GasPool).AddGas(block.GasLimit()) + ) + var receipts = make([]*types.Receipt, 0) txNum := len(block.Transactions()) - if txNum > 0 { - log.Info("ProcessParallel", "block num", block.Number(), "txNum", txNum) - p.resetSlotState() - } + p.resetParallelState(txNum) + // Iterate over and process the individual transactions posa, isPoSA := p.engine.(consensus.PoSA) commonTxs := make([]*types.Transaction, 0, txNum) - - // initialise bloom processors - bloomProcessors := NewAsyncReceiptBloomGenerator(txNum) - statedb.MarkFullProcessed() - // usually do have two tx, one for validator set contract, another for system reward contract. systemTxs := make([]*types.Transaction, 0, 2) + + signer, _, bloomProcessors := p.preExecute(block, statedb, cfg, true) var waitTxChan, curTxChan chan int for i, tx := range block.Transactions() { if isPoSA { @@ -1002,10 +997,9 @@ func (p *StateProcessor) ProcessParallel(block *types.Block, statedb *state.Stat // fixme: to optimize the for { for {} } loop code style for { - // if p.queueToSameAddress(txReq) { - // log.Info("ProcessParallel queue to same slot", "txIndex", txReq.txIndex) - // continue - // } + if p.queueToSameAddress(txReq) { + break + } // if idle slot available, just dispatch and process next tx. if p.dispatchToIdleSlot(statedb, txReq) { @@ -1040,22 +1034,17 @@ func (p *StateProcessor) ProcessParallel(block *types.Block, statedb *state.Stat receipts = append(receipts, result.receipt) } - bloomProcessors.Close() - if txNum > 0 { + // len(commonTxs) could be 0, such as: https://bscscan.com/block/14580486 + if len(commonTxs) > 0 { log.Info("ProcessParallel tx all done", "block", header.Number, "usedGas", *usedGas, - "len(commonTxs)", len(commonTxs), "len(receipts)", len(receipts), - "len(systemTxs)", len(systemTxs)) + "txNum", txNum, + "len(commonTxs)", len(commonTxs), + "debugErrorRedoNum", p.debugErrorRedoNum, + "debugConflictRedoNum", p.debugConflictRedoNum, + "redo rate(%)", 100*(p.debugErrorRedoNum+p.debugConflictRedoNum)/len(commonTxs)) } - // Finalize the block, applying any consensus engine specific extras (e.g. block rewards) - err := p.engine.Finalize(p.bc, header, statedb, &commonTxs, block.Uncles(), &receipts, &systemTxs, usedGas) - if err != nil { - return statedb, receipts, allLogs, *usedGas, err - } - for _, receipt := range receipts { - allLogs = append(allLogs, receipt.Logs...) - } - - return statedb, receipts, allLogs, *usedGas, nil + allLogs, err := p.postExecute(block, statedb, &commonTxs, &receipts, &systemTxs, usedGas, bloomProcessors) + return statedb, receipts, allLogs, *usedGas, err } func applyTransaction(msg types.Message, config *params.ChainConfig, bc ChainContext, author *common.Address, gp *GasPool, statedb *state.StateDB, header *types.Header, tx *types.Transaction, usedGas *uint64, evm *vm.EVM, receiptProcessors ...ReceiptProcessor) (*types.Receipt, error) { diff --git a/core/types.go b/core/types.go index c1e04a2b23..0038ce916b 100644 --- a/core/types.go +++ b/core/types.go @@ -49,6 +49,7 @@ type Processor interface { // the processor (coinbase) and any included uncles. Process(block *types.Block, statedb *state.StateDB, cfg vm.Config) (*state.StateDB, types.Receipts, []*types.Log, uint64, error) - // ProcessParallel will implement BEP-130, run transactions concurrently. + // Implement BEP-130: Parallel Transaction Execution. + InitParallelOnce() ProcessParallel(block *types.Block, statedb *state.StateDB, cfg vm.Config) (*state.StateDB, types.Receipts, []*types.Log, uint64, error) } From a46a2141bfb7f3ab13276349654e315419fd25d5 Mon Sep 17 00:00:00 2001 From: setunapo Date: Thu, 17 Feb 2022 18:16:12 +0800 Subject: [PATCH 4/8] Parallel: dispatch, queueSize, slot DB prefetch, disable cache prefetch for parallel this patch has 3 changes: 1.change default queuesize to 20, since 10 could be not enough and will cause more conflicts 2.enable slot DB trie prefetch, use the prefetch of main state DB. 3.disable transaction cache prefetch when parallel is enabled since in parallel mode CPU resource could be limitted, and paralle has its own piped transaction execution 4.change dispatch policy ** queue based on from address ** queue based on to address, try next slot if current is full Since from address is used to make dispatch policy, the pending transactions in a slot could have several different To address, so we will compare the To address of every pending transactions. --- cmd/utils/flags.go | 4 +- core/blockchain.go | 13 +++--- core/state/statedb.go | 8 ++++ core/state_processor.go | 92 ++++++++++++++++++++++++++++------------- 4 files changed, 81 insertions(+), 36 deletions(-) diff --git a/cmd/utils/flags.go b/cmd/utils/flags.go index 36b90329d3..87affa96c4 100644 --- a/cmd/utils/flags.go +++ b/cmd/utils/flags.go @@ -808,12 +808,12 @@ var ( } ParallelTxNumFlag = cli.IntFlag{ Name: "parallel.num", - Usage: "Number of slot for transaction execution, only valid in parallel mode (default = CPUNum - 1)", + Usage: "Number of slot for transaction execution, only valid in parallel mode (default: CPUNum - 1)", Value: core.ParallelExecNum, } ParallelTxQueueSizeFlag = cli.IntFlag{ Name: "parallel.queuesize", - Usage: "Max number of Tx that can be queued to a slot, only valid in parallel mode (default = 10)", + Usage: "Max number of Tx that can be queued to a slot, only valid in parallel mode", Value: core.MaxPendingQueueSize, } diff --git a/core/blockchain.go b/core/blockchain.go index 49071fef9e..e2e7a0e3ad 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -2115,11 +2115,14 @@ func (bc *BlockChain) insertChain(chain types.Blocks, verifySeals bool) (int, er statedb.StartPrefetcher("chain") var followupInterrupt uint32 // For diff sync, it may fallback to full sync, so we still do prefetch - if len(block.Transactions()) >= prefetchTxNumber { - throwaway := statedb.Copy() - go func(start time.Time, followup *types.Block, throwaway *state.StateDB, interrupt *uint32) { - bc.prefetcher.Prefetch(followup, throwaway, bc.vmConfig, &followupInterrupt) - }(time.Now(), block, throwaway, &followupInterrupt) + // parallel mode has a pipeline, similar to this prefetch, to save CPU we disable this prefetch for parallel + if !ParallelTxMode { + if len(block.Transactions()) >= prefetchTxNumber { + throwaway := statedb.Copy() + go func(start time.Time, followup *types.Block, throwaway *state.StateDB, interrupt *uint32) { + bc.prefetcher.Prefetch(followup, throwaway, bc.vmConfig, &followupInterrupt) + }(time.Now(), block, throwaway, &followupInterrupt) + } } //Process block using the parent state as reference point substart := time.Now() diff --git a/core/state/statedb.go b/core/state/statedb.go index 281bf9787c..3c79511d89 100644 --- a/core/state/statedb.go +++ b/core/state/statedb.go @@ -426,6 +426,9 @@ func (s *StateDB) MergeSlotDB(slotDb *StateDB, slotReceipt *types.Receipt, txInd // state trie concurrently while the state is mutated so that when we reach the // commit phase, most of the needed data is already hot. func (s *StateDB) StartPrefetcher(namespace string) { + if s.parallel.isSlotDB { + log.Warn("StartPrefetcher should not be called by slot DB") + } s.prefetcherLock.Lock() defer s.prefetcherLock.Unlock() if s.prefetcher != nil { @@ -440,6 +443,9 @@ func (s *StateDB) StartPrefetcher(namespace string) { // StopPrefetcher terminates a running prefetcher and reports any leftover stats // from the gathered metrics. func (s *StateDB) StopPrefetcher() { + if s.parallel.isSlotDB { + log.Warn("StopPrefetcher should not be called by slot DB") + } s.prefetcherLock.Lock() defer s.prefetcherLock.Unlock() if s.prefetcher != nil { @@ -1407,6 +1413,8 @@ func (s *StateDB) CopyForSlot() *StateDB { } state.snapStorage[k] = temp } + // slot will shared main stateDB's prefetcher + state.prefetcher = s.prefetcher } return state } diff --git a/core/state_processor.go b/core/state_processor.go index bf11f464d9..4eddca4c20 100644 --- a/core/state_processor.go +++ b/core/state_processor.go @@ -51,7 +51,7 @@ const ( reuseSlotDB = false // parallel slot's pending Txs will reuse the latest slotDB ) -var MaxPendingQueueSize = 10 // parallel slot's maximum number of pending Txs +var MaxPendingQueueSize = 20 // parallel slot's maximum number of pending Txs var ParallelExecNum = runtime.NumCPU() - 1 // leave a CPU to dispatcher // StateProcessor is a basic Processor, which takes care of transitioning @@ -386,7 +386,8 @@ func (p *LightStateProcessor) LightProcess(diffLayer *types.DiffLayer, block *ty type SlotState struct { tailTxReq *ParallelTxRequest // tail pending Tx of the slot, should be accessed on dispatcher only. - pendingExec chan *ParallelTxRequest + pendingTxReqChan chan *ParallelTxRequest + pendingTxReqList []*ParallelTxRequest // maintained by dispatcher for dispatch policy mergedChangeList []state.SlotChangeList slotdbChan chan *state.StateDB // dispatch will create and send this slotDB to slot // conflict check uses conflict window @@ -395,7 +396,7 @@ type SlotState struct { type ParallelTxResult struct { redo bool // for redo, dispatch will wait new tx result - updateSlotDB bool // for redo and pendingExec, slot needs new slotDB, + updateSlotDB bool // for redo and pending tx quest, slot needs new slotDB, reuseSlotDB bool // will try to reuse latest finalized slotDB keepSystem bool // for redo, should keep system address's balance txIndex int @@ -434,7 +435,7 @@ func (p *StateProcessor) InitParallelOnce() { for i := 0; i < ParallelExecNum; i++ { p.slotState[i] = new(SlotState) p.slotState[i].slotdbChan = make(chan *state.StateDB, 1) - p.slotState[i].pendingExec = make(chan *ParallelTxRequest, MaxPendingQueueSize) + p.slotState[i].pendingTxReqChan = make(chan *ParallelTxRequest, MaxPendingQueueSize) wg.Add(1) // start the slot's goroutine @@ -523,32 +524,60 @@ func (p *StateProcessor) hasStateConflict(readDb *state.StateDB, changeList stat // for parallel execute, we put contracts of same address in a slot, // since these txs probably would have conflicts -func (p *StateProcessor) queueToSameAddress(execMsg *ParallelTxRequest) bool { - txToAddr := execMsg.tx.To() +func (p *StateProcessor) queueSameToAddress(txReq *ParallelTxRequest) bool { + txToAddr := txReq.tx.To() + // To() == nil means contract creation, no same To address if txToAddr == nil { return false } for i, slot := range p.slotState { if slot.tailTxReq == nil { // this slot is idle - // log.Debug("queueToSameAddress skip idle slot.") continue } + for _, pending := range slot.pendingTxReqList { + // To() == nil means contract creation, skip it. + if pending.tx.To() == nil { + continue + } + // same to address, put it on slot's pending list. + if *txToAddr == *pending.tx.To() { + select { + case slot.pendingTxReqChan <- txReq: + slot.tailTxReq = txReq + slot.pendingTxReqList = append(slot.pendingTxReqList, txReq) + log.Debug("queue same To address", "Slot", i, "txIndex", txReq.txIndex) + return true + default: + log.Debug("queue same To address, but queue is full", "Slot", i, "txIndex", txReq.txIndex) + break // try next slot + } + } + } + } + return false +} - // To() == nil means contract creation, won't queue to such slot. - if slot.tailTxReq.tx.To() == nil { - // log.Debug("queueToSameAddress, slot's To address is nil", "Slot", i) +// for parallel execute, we put contracts of same address in a slot, +// since these txs probably would have conflicts +func (p *StateProcessor) queueSameFromAddress(txReq *ParallelTxRequest) bool { + txFromAddr := txReq.msg.From() + for i, slot := range p.slotState { + if slot.tailTxReq == nil { // this slot is idle continue } - // same to address, put it on slot's pending list. - if *txToAddr == *slot.tailTxReq.tx.To() { - select { - case slot.pendingExec <- execMsg: - slot.tailTxReq = execMsg - log.Debug("queueToSameAddress", "Slot", i, "txIndex", execMsg.txIndex) - return true - default: - log.Debug("queueToSameAddress but queue is full", "Slot", i, "txIndex", execMsg.txIndex) - return false + for _, pending := range slot.pendingTxReqList { + // same from address, put it on slot's pending list. + if txFromAddr == pending.msg.From() { + select { + case slot.pendingTxReqChan <- txReq: + slot.tailTxReq = txReq + slot.pendingTxReqList = append(slot.pendingTxReqList, txReq) + log.Debug("queue same From address", "Slot", i, "txIndex", txReq.txIndex) + return true + default: + log.Debug("queue same From address, but queue is full", "Slot", i, "txIndex", txReq.txIndex) + break // try next slot + } } } } @@ -565,7 +594,8 @@ func (p *StateProcessor) dispatchToIdleSlot(statedb *state.StateDB, txReq *Paral } log.Debug("dispatchToIdleSlot", "Slot", i, "txIndex", txReq.txIndex) slot.tailTxReq = txReq - slot.pendingExec <- txReq + slot.pendingTxReqList = append(slot.pendingTxReqList, txReq) + slot.pendingTxReqChan <- txReq return true } } @@ -579,7 +609,7 @@ func (p *StateProcessor) waitUntilNextTxDone(statedb *state.StateDB) *ParallelTx result = <-p.paraTxResultChan // slot may request new slotDB, if it think its slotDB is outdated // such as: - // tx in pendingExec, previous tx in same queue is likely "damaged" the slotDB + // tx in pending tx request, previous tx in same queue is likely "damaged" the slotDB // tx redo for confict // tx stage 1 failed, nonce out of order... if result.updateSlotDB { @@ -606,6 +636,7 @@ func (p *StateProcessor) waitUntilNextTxDone(statedb *state.StateDB) *ParallelTx resultSlotIndex := result.slotIndex resultTxIndex := result.txIndex resultSlotState := p.slotState[resultSlotIndex] + resultSlotState.pendingTxReqList = resultSlotState.pendingTxReqList[1:] if resultSlotState.tailTxReq.txIndex == resultTxIndex { log.Debug("ProcessParallel slot is idle", "Slot", resultSlotIndex) resultSlotState.tailTxReq = nil @@ -808,7 +839,7 @@ func (p *StateProcessor) runSlotLoop(slotIndex int) { for { // log.Info("parallel slot waiting", "Slot", slotIndex) // wait for new TxReq - txReq := <-curSlot.pendingExec + txReq := <-curSlot.pendingTxReqChan // receive a dispatched message log.Debug("SlotLoop received a new TxReq", "Slot", slotIndex, "txIndex", txReq.txIndex) @@ -837,7 +868,7 @@ func (p *StateProcessor) runSlotLoop(slotIndex int) { } } -// clear slotState for each block. +// clear slot state for each block. func (p *StateProcessor) resetParallelState(txNum int) { if txNum == 0 { return @@ -846,9 +877,10 @@ func (p *StateProcessor) resetParallelState(txNum int) { p.debugErrorRedoNum = 0 p.debugConflictRedoNum = 0 - for _, slotState := range p.slotState { - slotState.tailTxReq = nil - slotState.mergedChangeList = make([]state.SlotChangeList, 0) + for _, slot := range p.slotState { + slot.tailTxReq = nil + slot.mergedChangeList = make([]state.SlotChangeList, 0) + slot.pendingTxReqList = make([]*ParallelTxRequest, 0) } } @@ -997,10 +1029,12 @@ func (p *StateProcessor) ProcessParallel(block *types.Block, statedb *state.Stat // fixme: to optimize the for { for {} } loop code style for { - if p.queueToSameAddress(txReq) { + if p.queueSameToAddress(txReq) { + break + } + if p.queueSameFromAddress(txReq) { break } - // if idle slot available, just dispatch and process next tx. if p.dispatchToIdleSlot(statedb, txReq) { // log.Info("ProcessParallel dispatch to idle slot", "txIndex", txReq.txIndex) From 3d052a0bf0b030882f3c6a9c8ba89724412509df Mon Sep 17 00:00:00 2001 From: lunarblock Date: Wed, 23 Feb 2022 15:08:32 +0800 Subject: [PATCH 5/8] Parallel: implement COW(Copy-On-Write) ** use sync map for the stateObjects in parallel ** others fix a SlotDB reuse bug & enable it delete unnecessary parallel initialize for none slot DB. --- core/blockchain.go | 2 + core/state/journal.go | 2 +- core/state/state_object.go | 5 +- core/state/statedb.go | 114 +++++++++++++++++++++++-------------- core/state/statedb_test.go | 24 ++++++++ core/state_processor.go | 10 ++-- 6 files changed, 107 insertions(+), 50 deletions(-) diff --git a/core/blockchain.go b/core/blockchain.go index e2e7a0e3ad..dcbecdbabe 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -2105,10 +2105,12 @@ func (bc *BlockChain) insertChain(chain types.Blocks, verifySeals bool) (int, er if parent == nil { parent = bc.GetHeader(block.ParentHash(), block.NumberU64()-1) } + statedb, err := state.New(parent.Root, bc.stateCache, bc.snaps) if err != nil { return it.index, err } + bc.updateHighestVerifiedHeader(block.Header()) // Enable prefetching to pull in trie node paths while processing transactions diff --git a/core/state/journal.go b/core/state/journal.go index 20ae6a269a..487e79a57d 100644 --- a/core/state/journal.go +++ b/core/state/journal.go @@ -145,7 +145,7 @@ func (ch createObjectChange) revert(s *StateDB) { if s.parallel.isSlotDB { delete(s.parallel.dirtiedStateObjectsInSlot, *ch.account) } else { - s.stateObjects.Delete(*ch.account) + s.deleteStateObjectFromStateDB(*ch.account) } delete(s.stateObjectsDirty, *ch.account) } diff --git a/core/state/state_object.go b/core/state/state_object.go index c40b38a958..7db94f6b53 100644 --- a/core/state/state_object.go +++ b/core/state/state_object.go @@ -488,11 +488,10 @@ func (s *StateObject) deepCopy(db *StateDB) *StateObject { return stateObject } +// fixme: this is ownership transfer, to be optimized by state object merge. +// we can leave the ownership to slot and it can be reused. func (s *StateObject) deepCopyForSlot(db *StateDB) *StateObject { s.db = db - if s.trie != nil { - s.trie = db.db.CopyTrie(s.trie) - } return s } diff --git a/core/state/statedb.go b/core/state/statedb.go index 3c79511d89..b735b16909 100644 --- a/core/state/statedb.go +++ b/core/state/statedb.go @@ -85,6 +85,34 @@ func (s *StateObjectSyncMap) StoreStateObject(addr common.Address, stateObject * s.Store(addr, stateObject) } +// loadStateObjectFromStateDB is the entry for loading state object from stateObjects in StateDB or stateObjects in parallel +func (s *StateDB) loadStateObjectFromStateDB(addr common.Address) (*StateObject, bool) { + if s.isParallel { + return s.parallel.stateObjects.LoadStateObject(addr) + } else { + obj, ok := s.stateObjects[addr] + return obj, ok + } +} + +// storeStateObjectToStateDB is the entry for storing state object to stateObjects in StateDB or stateObjects in parallel +func (s *StateDB) storeStateObjectToStateDB(addr common.Address, stateObject *StateObject) { + if s.isParallel { + s.parallel.stateObjects.Store(addr, stateObject) + } else { + s.stateObjects[addr] = stateObject + } +} + +// deleteStateObjectFromStateDB is the entry for deleting state object to stateObjects in StateDB or stateObjects in parallel +func (s *StateDB) deleteStateObjectFromStateDB(addr common.Address) { + if s.isParallel { + s.parallel.stateObjects.Delete(addr) + } else { + delete(s.stateObjects, addr) + } +} + // For parallel mode only, keep the change list for later conflict detect type SlotChangeList struct { SlotDB *StateDB // used for SlotDb reuse only, otherwise, it can be discarded @@ -98,7 +126,14 @@ type SlotChangeList struct { // For parallel mode only type ParallelState struct { - isSlotDB bool + isSlotDB bool // isSlotDB denotes StateDB is used in slot + + // stateObjects holds the state objects in the base slot db + // the reason for using stateObjects instead of stateObjects on the outside is + // we need a thread safe map to hold state objects since there are many slots will read + // state objects from this and in the same time we will change this when merging slot db to the base slot db + stateObjects *StateObjectSyncMap + baseTxIndex int // slotDB is created base on this tx index. dirtiedStateObjectsInSlot map[common.Address]*StateObject // for conflict check @@ -152,11 +187,12 @@ type StateDB struct { snapStorage map[common.Address]map[string][]byte // This map holds 'live' objects, which will get modified while processing a state transition. - stateObjects *StateObjectSyncMap + stateObjects map[common.Address]*StateObject stateObjectsPending map[common.Address]struct{} // State objects finalized but not yet written to the trie stateObjectsDirty map[common.Address]struct{} // State objects modified in the current execution - parallel ParallelState // to keep all the parallel execution elements + isParallel bool + parallel ParallelState // to keep all the parallel execution elements // DB error. // State objects are used by the consensus core and VM which are @@ -204,8 +240,8 @@ func New(root common.Hash, db Database, snaps *snapshot.Tree) (*StateDB, error) return newStateDB(root, db, snaps) } -// With parallel, each execute slot would have its own stateDB. // NewSlotDB creates a new slot stateDB base on the provided stateDB. +// With parallel, each execute slot would have its own stateDB. func NewSlotDB(db *StateDB, systemAddr common.Address, txIndex int, keepSystem bool) *StateDB { slotDB := db.CopyForSlot() slotDB.originalRoot = db.originalRoot @@ -225,9 +261,6 @@ func NewSlotDB(db *StateDB, systemAddr common.Address, txIndex int, keepSystem b // to avoid new slotDB for each Tx, slotDB should be valid and merged func ReUseSlotDB(slotDB *StateDB, keepSystem bool) *StateDB { - if !keepSystem { - slotDB.SetBalance(slotDB.parallel.systemAddress, big.NewInt(0)) - } slotDB.logs = make(map[common.Hash][]*types.Log, defaultNumOfSlots) slotDB.logSize = 0 slotDB.parallel.systemAddressCount = 0 @@ -242,31 +275,28 @@ func ReUseSlotDB(slotDB *StateDB, keepSystem bool) *StateDB { slotDB.parallel.addrStateReadInSlot = make(map[common.Address]struct{}, defaultNumOfSlots) slotDB.parallel.addrStateChangeInSlot = make(map[common.Address]struct{}, defaultNumOfSlots) + // Previous *StateObject in slot db has been transfered to dispatcher now. + // Slot could no longer use these *StateObject, do clear. + slotDB.parallel.dirtiedStateObjectsInSlot = make(map[common.Address]*StateObject, defaultNumOfSlots) + slotDB.stateObjectsDirty = make(map[common.Address]struct{}, defaultNumOfSlots) slotDB.stateObjectsPending = make(map[common.Address]struct{}, defaultNumOfSlots) + if !keepSystem { + slotDB.SetBalance(slotDB.parallel.systemAddress, big.NewInt(0)) + } return slotDB } func newStateDB(root common.Hash, db Database, snaps *snapshot.Tree) (*StateDB, error) { - parallel := ParallelState{ - isSlotDB: false, - stateObjectSuicided: make(map[common.Address]struct{}, defaultNumOfSlots), - codeReadInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - codeChangeInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - stateChangedInSlot: make(map[common.Address]StateKeys, defaultNumOfSlots), - stateReadsInSlot: make(map[common.Address]StateKeys, defaultNumOfSlots), - balanceChangedInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - balanceReadsInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - addrStateReadInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - addrStateChangeInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + isSlotDB: false, } sdb := &StateDB{ db: db, originalRoot: root, snaps: snaps, - stateObjects: &StateObjectSyncMap{}, + stateObjects: make(map[common.Address]*StateObject, defaultNumOfSlots), parallel: parallel, stateObjectsPending: make(map[common.Address]struct{}, defaultNumOfSlots), stateObjectsDirty: make(map[common.Address]struct{}, defaultNumOfSlots), @@ -300,7 +330,7 @@ func (s *StateDB) getStateObjectFromStateObjects(addr common.Address) (*StateObj return obj, ok } } - return s.stateObjects.LoadStateObject(addr) + return s.loadStateObjectFromStateDB(addr) } // If the transaction execution is failed, keep its read list for conflict detect @@ -313,6 +343,11 @@ func (s *StateDB) RevertSlotDB(from common.Address) { s.parallel.addrStateChangeInSlot = make(map[common.Address]struct{}) } +func (s *StateDB) PrepareForParallel() { + s.isParallel = true + s.parallel.stateObjects = &StateObjectSyncMap{} +} + // MergeSlotDB is for Parallel TX, when the TX is finalized(dirty -> pending) // A bit similar to StateDB.Copy(), // mainly copy stateObjects, since slotDB has been finalized. @@ -346,7 +381,7 @@ func (s *StateDB) MergeSlotDB(slotDb *StateDB, slotReceipt *types.Receipt, txInd // stateObjects: KV, balance, nonce... if obj, ok := slotDb.getStateObjectFromStateObjects(addr); ok { - s.stateObjects.StoreStateObject(addr, obj.deepCopyForSlot(s)) + s.storeStateObjectToStateDB(addr, obj.deepCopyForSlot(s)) } } @@ -361,8 +396,8 @@ func (s *StateDB) MergeSlotDB(slotDb *StateDB, slotReceipt *types.Receipt, txInd continue } - if _, exist := s.stateObjects.LoadStateObject(addr); !exist { - s.stateObjects.StoreStateObject(addr, obj.deepCopyForSlot(s)) + if _, exist := s.loadStateObjectFromStateDB(addr); !exist { + s.storeStateObjectToStateDB(addr, obj.deepCopyForSlot(s)) } } @@ -419,6 +454,8 @@ func (s *StateDB) MergeSlotDB(slotDb *StateDB, slotReceipt *types.Receipt, txInd changeList.AddrStateChangeSet[addr] = struct{}{} } + // the slot DB's is valid now, move baseTxIndex forward, since it could be reused. + slotDb.parallel.baseTxIndex = txIndex return changeList } @@ -1141,7 +1178,7 @@ func (s *StateDB) SetStateObject(object *StateObject) { if s.parallel.isSlotDB { s.parallel.dirtiedStateObjectsInSlot[object.Address()] = object } else { - s.stateObjects.StoreStateObject(object.Address(), object) + s.storeStateObjectToStateDB(object.Address(), object) } } @@ -1237,23 +1274,13 @@ func (s *StateDB) ForEachStorage(addr common.Address, cb func(key, value common. func (s *StateDB) Copy() *StateDB { // Copy all the basic fields, initialize the memory ones parallel := ParallelState{ - isSlotDB: false, - stateObjectSuicided: make(map[common.Address]struct{}, defaultNumOfSlots), - codeReadInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - codeChangeInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - stateChangedInSlot: make(map[common.Address]StateKeys, defaultNumOfSlots), - stateReadsInSlot: make(map[common.Address]StateKeys, defaultNumOfSlots), - balanceChangedInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - balanceReadsInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - addrStateReadInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - addrStateChangeInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - dirtiedStateObjectsInSlot: make(map[common.Address]*StateObject, defaultNumOfSlots), + isSlotDB: false, } state := &StateDB{ db: s.db, trie: s.db.CopyTrie(s.trie), - stateObjects: &StateObjectSyncMap{}, + stateObjects: make(map[common.Address]*StateObject, len(s.journal.dirties)), stateObjectsPending: make(map[common.Address]struct{}, len(s.stateObjectsPending)), stateObjectsDirty: make(map[common.Address]struct{}, len(s.journal.dirties)), refund: s.refund, @@ -1274,7 +1301,7 @@ func (s *StateDB) Copy() *StateDB { // Even though the original object is dirty, we are not copying the journal, // so we need to make sure that anyside effect the journal would have caused // during a commit (or similar op) is already applied to the copy. - state.stateObjects.StoreStateObject(addr, object.deepCopy(state)) + state.storeStateObjectToStateDB(addr, object.deepCopy(state)) state.stateObjectsDirty[addr] = struct{}{} // Mark the copy dirty to force internal (code/state) commits state.stateObjectsPending[addr] = struct{}{} // Mark the copy pending to force external (account) commits @@ -1286,14 +1313,14 @@ func (s *StateDB) Copy() *StateDB { for addr := range s.stateObjectsPending { if _, exist := state.getStateObjectFromStateObjects(addr); !exist { object, _ := s.getStateObjectFromStateObjects(addr) - state.stateObjects.StoreStateObject(addr, object.deepCopy(state)) + state.storeStateObjectToStateDB(addr, object.deepCopy(state)) } state.stateObjectsPending[addr] = struct{}{} } for addr := range s.stateObjectsDirty { if _, exist := state.getStateObjectFromStateObjects(addr); !exist { object, _ := s.getStateObjectFromStateObjects(addr) - state.stateObjects.StoreStateObject(addr, object.deepCopy(state)) + state.storeStateObjectToStateDB(addr, object.deepCopy(state)) } state.stateObjectsDirty[addr] = struct{}{} } @@ -1353,8 +1380,11 @@ func (s *StateDB) Copy() *StateDB { func (s *StateDB) CopyForSlot() *StateDB { // Copy all the basic fields, initialize the memory ones - parallel := ParallelState{ + isSlotDB: true, + // Share base slot db's stateObjects + // It is a SyncMap, only readable to slot, not writable + stateObjects: s.parallel.stateObjects, stateObjectSuicided: make(map[common.Address]struct{}, defaultNumOfSlots), codeReadInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), codeChangeInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), @@ -1364,13 +1394,12 @@ func (s *StateDB) CopyForSlot() *StateDB { balanceReadsInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), addrStateReadInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), addrStateChangeInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - isSlotDB: true, dirtiedStateObjectsInSlot: make(map[common.Address]*StateObject, defaultNumOfSlots), } state := &StateDB{ db: s.db, trie: s.db.CopyTrie(s.trie), - stateObjects: s.stateObjects, + stateObjects: make(map[common.Address]*StateObject, defaultNumOfSlots), stateObjectsPending: make(map[common.Address]struct{}, defaultNumOfSlots), stateObjectsDirty: make(map[common.Address]struct{}, defaultNumOfSlots), refund: s.refund, @@ -1382,6 +1411,7 @@ func (s *StateDB) CopyForSlot() *StateDB { snapDestructs: make(map[common.Address]struct{}), snapAccounts: make(map[common.Address][]byte), snapStorage: make(map[common.Address]map[string][]byte), + isParallel: true, parallel: parallel, } diff --git a/core/state/statedb_test.go b/core/state/statedb_test.go index 2fce92ae1d..2841381963 100644 --- a/core/state/statedb_test.go +++ b/core/state/statedb_test.go @@ -943,6 +943,8 @@ func TestSuicide(t *testing.T) { memDb := rawdb.NewMemoryDatabase() db := NewDatabase(memDb) state, _ := New(common.Hash{}, db, nil) + state.PrepareForParallel() + slotDb := NewSlotDB(state, systemAddress, 0, false) addr := common.BytesToAddress([]byte("so")) @@ -979,6 +981,8 @@ func TestSetAndGetState(t *testing.T) { memDb := rawdb.NewMemoryDatabase() db := NewDatabase(memDb) state, _ := New(common.Hash{}, db, nil) + state.PrepareForParallel() + slotDb := NewSlotDB(state, systemAddress, 0, false) addr := common.BytesToAddress([]byte("so")) @@ -1014,6 +1018,8 @@ func TestSetAndGetCode(t *testing.T) { memDb := rawdb.NewMemoryDatabase() db := NewDatabase(memDb) state, _ := New(common.Hash{}, db, nil) + state.PrepareForParallel() + slotDb := NewSlotDB(state, systemAddress, 0, false) addr := common.BytesToAddress([]byte("so")) @@ -1047,6 +1053,8 @@ func TestGetCodeSize(t *testing.T) { memDb := rawdb.NewMemoryDatabase() db := NewDatabase(memDb) state, _ := New(common.Hash{}, db, nil) + state.PrepareForParallel() + slotDb := NewSlotDB(state, systemAddress, 0, false) addr := common.BytesToAddress([]byte("so")) @@ -1068,6 +1076,8 @@ func TestGetCodeHash(t *testing.T) { memDb := rawdb.NewMemoryDatabase() db := NewDatabase(memDb) state, _ := New(common.Hash{}, db, nil) + state.PrepareForParallel() + slotDb := NewSlotDB(state, systemAddress, 0, false) addr := common.BytesToAddress([]byte("so")) @@ -1089,6 +1099,8 @@ func TestSetNonce(t *testing.T) { memDb := rawdb.NewMemoryDatabase() db := NewDatabase(memDb) state, _ := New(common.Hash{}, db, nil) + state.PrepareForParallel() + slotDb := NewSlotDB(state, systemAddress, 0, false) addr := common.BytesToAddress([]byte("so")) @@ -1115,6 +1127,8 @@ func TestSetAndGetBalance(t *testing.T) { memDb := rawdb.NewMemoryDatabase() db := NewDatabase(memDb) state, _ := New(common.Hash{}, db, nil) + state.PrepareForParallel() + slotDb := NewSlotDB(state, systemAddress, 0, true) addr := systemAddress @@ -1157,6 +1171,8 @@ func TestSubBalance(t *testing.T) { memDb := rawdb.NewMemoryDatabase() db := NewDatabase(memDb) state, _ := New(common.Hash{}, db, nil) + state.PrepareForParallel() + slotDb := NewSlotDB(state, systemAddress, 0, true) addr := systemAddress @@ -1195,6 +1211,8 @@ func TestAddBalance(t *testing.T) { memDb := rawdb.NewMemoryDatabase() db := NewDatabase(memDb) state, _ := New(common.Hash{}, db, nil) + state.PrepareForParallel() + slotDb := NewSlotDB(state, systemAddress, 0, true) addr := systemAddress @@ -1233,6 +1251,8 @@ func TestEmpty(t *testing.T) { memDb := rawdb.NewMemoryDatabase() db := NewDatabase(memDb) state, _ := New(common.Hash{}, db, nil) + state.PrepareForParallel() + slotDb := NewSlotDB(state, systemAddress, 0, true) addr := systemAddress @@ -1252,6 +1272,8 @@ func TestExist(t *testing.T) { memDb := rawdb.NewMemoryDatabase() db := NewDatabase(memDb) state, _ := New(common.Hash{}, db, nil) + state.PrepareForParallel() + slotDb := NewSlotDB(state, systemAddress, 0, true) addr := systemAddress @@ -1271,6 +1293,8 @@ func TestMergeSlotDB(t *testing.T) { memDb := rawdb.NewMemoryDatabase() db := NewDatabase(memDb) state, _ := New(common.Hash{}, db, nil) + state.PrepareForParallel() + oldSlotDb := NewSlotDB(state, systemAddress, 0, true) newSlotDb := NewSlotDB(state, systemAddress, 0, true) diff --git a/core/state_processor.go b/core/state_processor.go index 4eddca4c20..ee0d8d98ce 100644 --- a/core/state_processor.go +++ b/core/state_processor.go @@ -48,7 +48,7 @@ const ( recentTime = 1024 * 3 recentDiffLayerTimeout = 5 farDiffLayerTimeout = 2 - reuseSlotDB = false // parallel slot's pending Txs will reuse the latest slotDB + reuseSlotDB = true // reuse could save state object copy cost ) var MaxPendingQueueSize = 20 // parallel slot's maximum number of pending Txs @@ -588,8 +588,8 @@ func (p *StateProcessor) queueSameFromAddress(txReq *ParallelTxRequest) bool { func (p *StateProcessor) dispatchToIdleSlot(statedb *state.StateDB, txReq *ParallelTxRequest) bool { for i, slot := range p.slotState { if slot.tailTxReq == nil { - // for idle slot, we have to create a SlotDB for it. if len(slot.mergedChangeList) == 0 { + // first transaction of a slot, there is no usable SlotDB, have to create one for it. txReq.slotDB = state.NewSlotDB(statedb, consensus.SystemAddress, p.mergedTxIndex, false) } log.Debug("dispatchToIdleSlot", "Slot", i, "txIndex", txReq.txIndex) @@ -869,7 +869,7 @@ func (p *StateProcessor) runSlotLoop(slotIndex int) { } // clear slot state for each block. -func (p *StateProcessor) resetParallelState(txNum int) { +func (p *StateProcessor) resetParallelState(txNum int, statedb *state.StateDB) { if txNum == 0 { return } @@ -877,6 +877,8 @@ func (p *StateProcessor) resetParallelState(txNum int) { p.debugErrorRedoNum = 0 p.debugConflictRedoNum = 0 + statedb.PrepareForParallel() + for _, slot := range p.slotState { slot.tailTxReq = nil slot.mergedChangeList = make([]state.SlotChangeList, 0) @@ -984,7 +986,7 @@ func (p *StateProcessor) ProcessParallel(block *types.Block, statedb *state.Stat ) var receipts = make([]*types.Receipt, 0) txNum := len(block.Transactions()) - p.resetParallelState(txNum) + p.resetParallelState(txNum, statedb) // Iterate over and process the individual transactions posa, isPoSA := p.engine.(consensus.PoSA) From fb58ee11e3f35db45cb7b7615463ed799f1f960f Mon Sep 17 00:00:00 2001 From: setunapo Date: Mon, 28 Feb 2022 15:31:33 +0800 Subject: [PATCH 6/8] Parallel: several bugfixs: state merge, suicide fixup, conflict detect, prefetch, fork This is a complicated patch, to do some fixup ** fix MergeSlotDB Since copy-on-write is used, transaction will do StateObject deepCopy before it writes the state; All the dirty state changed will be recorded in this copied one first, the ownership will be transfered to main StateDB on merge. It has a potential race condition that the simple ownership transfer may discard other state changes by other concurrent transactions. When copy-on-write is used, we should do StateObject merge. ** fix Suicide Suicide has an address state read operation. And it also needs do copy-on-write, to avoid damage main StateDB's state object. ** fix conflict detect If state read is not zero, should do conflict detect with addr state change first. Do conflict detect even with current slot, if we use copy-on-write and slotDB reuse, same slot could has race conditon of conflict. ** disable prefetch on slotDB trie prefetch should be started on main DB on Merge ** Add/Sub zero balance, Set State These are void operation, optimized to reduce conflict rate. Simple test show, conflict rate dropped from ~25% -> 12% **fix a fork on block 15,338,563 It a nonce conflict caused by opcode: opCreate & opCreate2 Generally, the nonce is advanced by 1 for the transaction sender; But opCreate & opCreate2 will try to create a new contract, the caller will advance its nonce too. It makes the nonce conflict detect more complicated: as nonce is a fundamental part of an account, as long as it has been changed, we mark the address as StateChanged, any concurrent access to it will be considered as conflicted. --- core/state/state_object.go | 11 +- core/state/statedb.go | 260 +++++++++++++++++++++++++++++-------- core/state_processor.go | 95 ++++++++------ core/vm/evm.go | 1 + core/vm/interface.go | 1 + 5 files changed, 272 insertions(+), 96 deletions(-) diff --git a/core/state/state_object.go b/core/state/state_object.go index 7db94f6b53..a809d2a565 100644 --- a/core/state/state_object.go +++ b/core/state/state_object.go @@ -488,11 +488,12 @@ func (s *StateObject) deepCopy(db *StateDB) *StateObject { return stateObject } -// fixme: this is ownership transfer, to be optimized by state object merge. -// we can leave the ownership to slot and it can be reused. -func (s *StateObject) deepCopyForSlot(db *StateDB) *StateObject { - s.db = db - return s +func (s *StateObject) MergeSlotObject(db Database, dirtyObjs *StateObject, keys StateKeys) { + for key := range keys { + // better to do s.GetState(db, key) to load originStorage for this key? + // since originStorage was in dirtyObjs, but it works even originStorage miss the state object. + s.SetState(db, key, dirtyObjs.GetState(db, key)) + } } // diff --git a/core/state/statedb.go b/core/state/statedb.go index b735b16909..e37b28de38 100644 --- a/core/state/statedb.go +++ b/core/state/statedb.go @@ -122,6 +122,7 @@ type SlotChangeList struct { BalanceChangeSet map[common.Address]struct{} CodeChangeSet map[common.Address]struct{} AddrStateChangeSet map[common.Address]struct{} + NonceAdvancedSet map[common.Address]struct{} } // For parallel mode only @@ -148,6 +149,7 @@ type ParallelState struct { addrStateReadInSlot map[common.Address]struct{} addrStateChangeInSlot map[common.Address]struct{} stateObjectSuicided map[common.Address]struct{} + nonceAdvanced map[common.Address]struct{} // Transaction will pay gas fee to system address. // Parallel execution will clear system address's balance at first, in order to maintain transaction's // gas fee value. Normal transaction will access system address twice, otherwise it means the transaction @@ -242,10 +244,11 @@ func New(root common.Hash, db Database, snaps *snapshot.Tree) (*StateDB, error) // NewSlotDB creates a new slot stateDB base on the provided stateDB. // With parallel, each execute slot would have its own stateDB. -func NewSlotDB(db *StateDB, systemAddr common.Address, txIndex int, keepSystem bool) *StateDB { +func NewSlotDB(db *StateDB, systemAddr common.Address, baseTxIndex int, keepSystem bool) *StateDB { + log.Debug("NewSlotDB", "baseTxIndex", baseTxIndex) slotDB := db.CopyForSlot() slotDB.originalRoot = db.originalRoot - slotDB.parallel.baseTxIndex = txIndex + slotDB.parallel.baseTxIndex = baseTxIndex slotDB.parallel.systemAddress = systemAddr slotDB.parallel.systemAddressCount = 0 slotDB.parallel.keepSystemAddressBalance = keepSystem @@ -261,6 +264,9 @@ func NewSlotDB(db *StateDB, systemAddr common.Address, txIndex int, keepSystem b // to avoid new slotDB for each Tx, slotDB should be valid and merged func ReUseSlotDB(slotDB *StateDB, keepSystem bool) *StateDB { + log.Debug("ReUseSlotDB", "baseTxIndex", slotDB.parallel.baseTxIndex, + "keepSystem", keepSystem, "refund", slotDB.refund, + "len(journal.entries)", len(slotDB.journal.entries)) slotDB.logs = make(map[common.Hash][]*types.Log, defaultNumOfSlots) slotDB.logSize = 0 slotDB.parallel.systemAddressCount = 0 @@ -274,6 +280,7 @@ func ReUseSlotDB(slotDB *StateDB, keepSystem bool) *StateDB { slotDB.parallel.balanceReadsInSlot = make(map[common.Address]struct{}, defaultNumOfSlots) slotDB.parallel.addrStateReadInSlot = make(map[common.Address]struct{}, defaultNumOfSlots) slotDB.parallel.addrStateChangeInSlot = make(map[common.Address]struct{}, defaultNumOfSlots) + slotDB.parallel.nonceAdvanced = make(map[common.Address]struct{}, defaultNumOfSlots) // Previous *StateObject in slot db has been transfered to dispatcher now. // Slot could no longer use these *StateObject, do clear. @@ -282,6 +289,10 @@ func ReUseSlotDB(slotDB *StateDB, keepSystem bool) *StateDB { slotDB.stateObjectsDirty = make(map[common.Address]struct{}, defaultNumOfSlots) slotDB.stateObjectsPending = make(map[common.Address]struct{}, defaultNumOfSlots) + // slotDB.snapDestructs = make(map[common.Address]struct{}) + // slotDB.snapAccounts = make(map[common.Address][]byte) + // slotDB.snapStorage = make(map[common.Address]map[string][]byte) + if !keepSystem { slotDB.SetBalance(slotDB.parallel.systemAddress, big.NewInt(0)) } @@ -336,11 +347,14 @@ func (s *StateDB) getStateObjectFromStateObjects(addr common.Address) (*StateObj // If the transaction execution is failed, keep its read list for conflict detect // and discard its state changed, execept its own balance change. func (s *StateDB) RevertSlotDB(from common.Address) { + log.Debug("RevertSlotDB", "addr", from, "txIndex", s.txIndex) s.parallel.stateObjectSuicided = make(map[common.Address]struct{}) s.parallel.stateChangedInSlot = make(map[common.Address]StateKeys) s.parallel.balanceChangedInSlot = make(map[common.Address]struct{}, 1) s.parallel.balanceChangedInSlot[from] = struct{}{} s.parallel.addrStateChangeInSlot = make(map[common.Address]struct{}) + s.parallel.nonceAdvanced = make(map[common.Address]struct{}) + } func (s *StateDB) PrepareForParallel() { @@ -374,14 +388,95 @@ func (s *StateDB) MergeSlotDB(slotDb *StateDB, slotReceipt *types.Receipt, txInd if _, exist := s.stateObjectsDirty[addr]; !exist { s.stateObjectsDirty[addr] = struct{}{} } - + // system address is EOA account, it should have no storage change if addr == systemAddress { continue } // stateObjects: KV, balance, nonce... - if obj, ok := slotDb.getStateObjectFromStateObjects(addr); ok { - s.storeStateObjectToStateDB(addr, obj.deepCopyForSlot(s)) + dirtyObj, ok := slotDb.getStateObjectFromStateObjects(addr) + if !ok { + panic(fmt.Sprintf("MergeSlotDB dirty object not exist! (txIndex: %d, addr: %s)", slotDb.txIndex, addr.String())) + } + mainObj, exist := s.loadStateObjectFromStateDB(addr) + + log.Debug("MergeSlotDB", "txIndex", slotDb.txIndex, "addr", addr, + "exist", exist, "dirtyObj.deleted", dirtyObj.deleted) + if !exist { + // addr not exist on main DB, do ownership transfer + dirtyObj.db = s + dirtyObj.finalise(true) // prefetch on dispatcher + s.storeStateObjectToStateDB(addr, dirtyObj) + delete(slotDb.parallel.dirtiedStateObjectsInSlot, addr) // transfer ownership + } else { + // addr already in main DB, do merge: balance, KV, code, State(create, suicide) + // can not do copy or ownership transfer directly, since dirtyObj could have outdated + // data(may be update within the conflict window) + + // This is debug log for add balance with 0, can be removed + if dirtyObj.deleted && dirtyObj.empty() && !mainObj.empty() { + if _, exist := slotDb.parallel.balanceChangedInSlot[addr]; !exist { + // add(0) could trigger empty delete + // note: what if add/sub with 0 result, it is also a fake delete? + // it is ok, since none-zero add will produce a read record, should conflict + log.Warn("MergeSlotDB empty deleted", "txIndex", slotDb.txIndex, "addr", addr) + } + } + + // Do deepCopy a temporary *StateObject for safety, + // since slot could read the address, dispatch should avoid overwrite the StateObject directly + // otherwise, it could crash for: concurrent map iteration and map write + var newMainObj *StateObject + if _, suicided := slotDb.parallel.stateObjectSuicided[addr]; suicided { + if !dirtyObj.suicided { + // debug purpose, can be removed + log.Warn("MergeSlotDB suicide and recreated", "txIndex", slotDb.txIndex, "addr", addr) + } + } + + if dirtyObj.deleted { + log.Debug("MergeSlotDB state object merge: Suicide") + if !dirtyObj.suicided && !dirtyObj.empty() { + // none suicide object, should be empty delete + log.Error("MergeSlotDB none suicide deleted, should be empty", "txIndex", slotDb.txIndex, "addr", addr) + } + // suicided object will keep its worldstate(useless), and will be deleted from trie on block commit + newMainObj = dirtyObj.deepCopy(s) + delete(s.snapAccounts, addr) + delete(s.snapStorage, addr) + } else if _, created := slotDb.parallel.addrStateChangeInSlot[addr]; created { + log.Debug("MergeSlotDB state object merge: addr state change") + // there are 2 kinds of object creation: + // 1.createObject: AddBalance,SetState to an unexist or emptyDeleted address. + // 2.CreateAccount: like DAO the fork, regenerate a account carry its balance without KV + // can not merge, do ownership transafer + dirtyObj.db = s + newMainObj = dirtyObj + delete(slotDb.parallel.dirtiedStateObjectsInSlot, addr) // transfer ownership + } else { + newMainObj = mainObj.deepCopy(s) + // do merge: balance, KV, code... + if _, balanced := slotDb.parallel.balanceChangedInSlot[addr]; balanced { + log.Debug("MergeSlotDB state object merge: state merge: balance", + "newMainObj.Balance()", newMainObj.Balance(), + "dirtyObj.Balance()", dirtyObj.Balance()) + newMainObj.SetBalance(dirtyObj.Balance()) + } + if _, coded := slotDb.parallel.codeChangeInSlot[addr]; coded { + log.Debug("MergeSlotDB state object merge: state merge: code") + newMainObj.code = dirtyObj.code + newMainObj.data.CodeHash = dirtyObj.data.CodeHash + newMainObj.dirtyCode = true + } + if keys, stated := slotDb.parallel.stateChangedInSlot[addr]; stated { + newMainObj.MergeSlotObject(s.db, dirtyObj, keys) + } + // dirtyObj.Nonce() should not be less than newMainObj + newMainObj.setNonce(dirtyObj.Nonce()) + } + newMainObj.finalise(true) // prefetch on dispatcher + // update the object + s.storeStateObjectToStateDB(addr, newMainObj) } } @@ -391,30 +486,17 @@ func (s *StateDB) MergeSlotDB(slotDb *StateDB, slotReceipt *types.Receipt, txInd } } - for addr, obj := range slotDb.parallel.dirtiedStateObjectsInSlot { - if addr == systemAddress { - continue - } - - if _, exist := s.loadStateObjectFromStateDB(addr); !exist { - s.storeStateObjectToStateDB(addr, obj.deepCopyForSlot(s)) - } - } - // slotDb.logs: logs will be kept in receipts, no need to do merge - // Fixed: preimages should be merged not overwrite for hash, preimage := range slotDb.preimages { s.preimages[hash] = preimage } - // Fixed: accessList should be merged not overwrite if s.accessList != nil { + // fixme: accessList is not enabled yet, should use merged rather than overwrite s.accessList = slotDb.accessList.Copy() } + if slotDb.snaps != nil { - for k, v := range slotDb.snapDestructs { - s.snapDestructs[k] = v - } for k, v := range slotDb.snapAccounts { s.snapAccounts[k] = v } @@ -425,6 +507,18 @@ func (s *StateDB) MergeSlotDB(slotDb *StateDB, slotReceipt *types.Receipt, txInd } s.snapStorage[k] = temp } + for k := range slotDb.snapDestructs { + obj, _ := s.loadStateObjectFromStateDB(k) + if !obj.suicided && !obj.empty() { + // There could be a race condition for parallel transaction execution + // One add balance 0 to an empty address, it will delete it(delete empty is enabled,). + // While another concurrent transaction could add a none-zero balance to it, make it not empty + // Right now, we mark a read operation to avoid this race condition + // We hope to remove this read record, since it could reduce conflict rate. + log.Warn("MergeSlotDB state object to be deleted is not empty, ", "addr", k) + } + s.snapDestructs[k] = struct{}{} + } } // we have to create a new object to store change list for conflict detect, since @@ -437,6 +531,7 @@ func (s *StateDB) MergeSlotDB(slotDb *StateDB, slotReceipt *types.Receipt, txInd BalanceChangeSet: make(map[common.Address]struct{}, len(slotDb.parallel.balanceChangedInSlot)), CodeChangeSet: make(map[common.Address]struct{}, len(slotDb.parallel.codeChangeInSlot)), AddrStateChangeSet: make(map[common.Address]struct{}, len(slotDb.parallel.addrStateChangeInSlot)), + NonceAdvancedSet: make(map[common.Address]struct{}, len(slotDb.parallel.nonceAdvanced)), } for addr := range slotDb.parallel.stateObjectSuicided { changeList.StateObjectSuicided[addr] = struct{}{} @@ -453,6 +548,9 @@ func (s *StateDB) MergeSlotDB(slotDb *StateDB, slotReceipt *types.Receipt, txInd for addr := range slotDb.parallel.addrStateChangeInSlot { changeList.AddrStateChangeSet[addr] = struct{}{} } + for addr := range slotDb.parallel.nonceAdvanced { + changeList.NonceAdvancedSet[addr] = struct{}{} + } // the slot DB's is valid now, move baseTxIndex forward, since it could be reused. slotDb.parallel.baseTxIndex = txIndex @@ -610,9 +708,6 @@ func (s *StateDB) SubRefund(gas uint64) { // Exist reports whether the given account address exists in the state. // Notably this also returns true for suicided accounts. func (s *StateDB) Exist(addr common.Address) bool { - if s.parallel.isSlotDB { - s.parallel.addrStateReadInSlot[addr] = struct{}{} - } return s.getStateObject(addr) != nil } @@ -620,9 +715,6 @@ func (s *StateDB) Exist(addr common.Address) bool { // or empty according to the EIP161 specification (balance = nonce = code = 0) func (s *StateDB) Empty(addr common.Address) bool { so := s.getStateObject(addr) - if s.parallel.isSlotDB { - s.parallel.addrStateReadInSlot[addr] = struct{}{} - } return so == nil || so.empty() } @@ -646,7 +738,6 @@ func (s *StateDB) GetNonce(addr common.Address) uint64 { if stateObject != nil { return stateObject.Nonce() } - return 0 } @@ -775,6 +866,13 @@ func (s *StateDB) GetStorageProofByHash(a common.Address, key common.Hash) ([][] // GetCommittedState retrieves a value from the given account's committed storage trie. func (s *StateDB) GetCommittedState(addr common.Address, hash common.Hash) common.Hash { + if s.parallel.isSlotDB { + if s.parallel.stateReadsInSlot[addr] == nil { + s.parallel.stateReadsInSlot[addr] = make(map[common.Hash]struct{}, defaultNumOfSlots) + } + s.parallel.stateReadsInSlot[addr][hash] = struct{}{} + } + stateObject := s.getStateObject(addr) if stateObject != nil { return stateObject.GetCommittedState(s.db, hash) @@ -801,10 +899,6 @@ func (s *StateDB) StorageTrie(addr common.Address) Trie { func (s *StateDB) HasSuicided(addr common.Address) bool { stateObject := s.getStateObject(addr) - - if s.parallel.isSlotDB { - s.parallel.addrStateReadInSlot[addr] = struct{}{} // address suicided. - } if stateObject != nil { return stateObject.suicided } @@ -818,9 +912,13 @@ func (s *StateDB) HasSuicided(addr common.Address) bool { // AddBalance adds amount to the account associated with addr. func (s *StateDB) AddBalance(addr common.Address, amount *big.Int) { if s.parallel.isSlotDB { - // just in case other tx creates this account, we will miss this if we only add this account when found - s.parallel.balanceChangedInSlot[addr] = struct{}{} - s.parallel.balanceReadsInSlot[addr] = struct{}{} // add balance will perform a read operation first + // just in case other tx creates this account, + // we will miss this if we only add this account when found + if amount.Sign() != 0 { + s.parallel.balanceChangedInSlot[addr] = struct{}{} + } + // add balance will perform a read operation first, empty object will be deleted + s.parallel.balanceReadsInSlot[addr] = struct{}{} if addr == s.parallel.systemAddress { s.parallel.systemAddressCount++ } @@ -845,9 +943,13 @@ func (s *StateDB) AddBalance(addr common.Address, amount *big.Int) { // SubBalance subtracts amount from the account associated with addr. func (s *StateDB) SubBalance(addr common.Address, amount *big.Int) { if s.parallel.isSlotDB { - // just in case other tx creates this account, we will miss this if we only add this account when found - s.parallel.balanceChangedInSlot[addr] = struct{}{} - s.parallel.balanceReadsInSlot[addr] = struct{}{} + // just in case other tx creates this account, + // we will miss this if we only add this account when found + if amount.Sign() != 0 { + s.parallel.balanceChangedInSlot[addr] = struct{}{} + // unlike add, sub 0 balance will not touch empty object + s.parallel.balanceReadsInSlot[addr] = struct{}{} + } if addr == s.parallel.systemAddress { s.parallel.systemAddressCount++ } @@ -890,6 +992,18 @@ func (s *StateDB) SetBalance(addr common.Address, amount *big.Int) { } } +// Generally sender's nonce will be increased by 1 for each transaction +// But if the contract tries to create a new contract, its nonce will be advanced +// for each opCreate or opCreate2. Nonce is key to transaction execution, once it is +// changed for contract created, the concurrent transaction will be marked invalid if +// they accessed the address. +func (s *StateDB) NonceChanged(addr common.Address) { + if s.parallel.isSlotDB { + log.Debug("NonceChanged", "txIndex", s.txIndex, "addr", addr) + s.parallel.nonceAdvanced[addr] = struct{}{} + } +} + func (s *StateDB) SetNonce(addr common.Address, nonce uint64) { stateObject := s.GetOrNewStateObject(addr) if stateObject != nil { @@ -930,6 +1044,15 @@ func (s *StateDB) SetState(addr common.Address, key, value common.Hash) { stateObject := s.GetOrNewStateObject(addr) if stateObject != nil { if s.parallel.isSlotDB { + if s.parallel.baseTxIndex+1 == s.txIndex { + // we check if state is unchanged + // only when current transaction is the next transaction to be committed + if stateObject.GetState(s.db, key) == value { + log.Debug("Skip set same state", "baseTxIndex", s.parallel.baseTxIndex, + "txIndex", s.txIndex) + return + } + } if _, ok := s.parallel.dirtiedStateObjectsInSlot[addr]; !ok { newStateObject := stateObject.deepCopy(s) newStateObject.SetState(s.db, key, value) @@ -964,22 +1087,37 @@ func (s *StateDB) SetStorage(addr common.Address, storage map[common.Hash]common // getStateObject will return a non-nil account after Suicide. func (s *StateDB) Suicide(addr common.Address) bool { stateObject := s.getStateObject(addr) - // fixme: should add read stateobject record + + if s.parallel.isSlotDB { + s.parallel.addrStateReadInSlot[addr] = struct{}{} + } if stateObject == nil { return false } - if s.parallel.isSlotDB { - s.parallel.stateObjectSuicided[addr] = struct{}{} - s.parallel.addrStateChangeInSlot[addr] = struct{}{} // address suicided. - } s.journal.append(suicideChange{ account: &addr, prev: stateObject.suicided, prevbalance: new(big.Int).Set(stateObject.Balance()), }) - stateObject.markSuicided() - stateObject.data.Balance = new(big.Int) + + if s.parallel.isSlotDB { + s.parallel.stateObjectSuicided[addr] = struct{}{} + s.parallel.addrStateChangeInSlot[addr] = struct{}{} + if _, ok := s.parallel.dirtiedStateObjectsInSlot[addr]; !ok { + // do copy-on-write for suicide "write" + newStateObject := stateObject.deepCopy(s) + newStateObject.markSuicided() + newStateObject.data.Balance = new(big.Int) + s.parallel.dirtiedStateObjectsInSlot[addr] = newStateObject + } else { + stateObject.markSuicided() + stateObject.data.Balance = new(big.Int) + } + } else { + stateObject.markSuicided() + stateObject.data.Balance = new(big.Int) + } return true } @@ -1026,6 +1164,10 @@ func (s *StateDB) deleteStateObject(obj *StateObject) { // the object is not found or was deleted in this execution context. If you need // to differentiate between non-existent/just-deleted, use getDeletedStateObject. func (s *StateDB) getStateObject(addr common.Address) *StateObject { + if s.parallel.isSlotDB { + s.parallel.addrStateReadInSlot[addr] = struct{}{} + } + if obj := s.getDeletedStateObject(addr); obj != nil && !obj.deleted { return obj } @@ -1187,9 +1329,6 @@ func (s *StateDB) GetOrNewStateObject(addr common.Address) *StateObject { stateObject := s.getStateObject(addr) if stateObject == nil { stateObject, _ = s.createObject(addr) - if s.parallel.isSlotDB { - s.parallel.addrStateChangeInSlot[addr] = struct{}{} // address created. - } } return stateObject } @@ -1197,12 +1336,19 @@ func (s *StateDB) GetOrNewStateObject(addr common.Address) *StateObject { // createObject creates a new state object. If there is an existing account with // the given address, it is overwritten and returned as the second return value. func (s *StateDB) createObject(addr common.Address) (newobj, prev *StateObject) { + if s.parallel.isSlotDB { + s.parallel.addrStateReadInSlot[addr] = struct{}{} // fixme: may not necessary + s.parallel.addrStateChangeInSlot[addr] = struct{}{} // address created. + } + prev = s.getDeletedStateObject(addr) // Note, prev might have been deleted, we need that! var prevdestruct bool if s.snap != nil && prev != nil { _, prevdestruct = s.snapDestructs[prev.address] if !prevdestruct { + // createObject for deleted object is ok, + // it will destroy the previous trie node and update with the new object on block commit s.snapDestructs[prev.address] = struct{}{} } } @@ -1233,11 +1379,12 @@ func (s *StateDB) createObject(addr common.Address) (newobj, prev *StateObject) func (s *StateDB) CreateAccount(addr common.Address) { newObj, prev := s.createObject(addr) if prev != nil { - newObj.setBalance(prev.data.Balance) - } else if s.parallel.isSlotDB { - s.parallel.addrStateChangeInSlot[addr] = struct{}{} // new account created + newObj.setBalance(prev.data.Balance) // this read + } + if s.parallel.isSlotDB { + s.parallel.balanceReadsInSlot[addr] = struct{}{} + s.parallel.dirtiedStateObjectsInSlot[addr] = newObj } - } func (s *StateDB) ForEachStorage(addr common.Address, cb func(key, value common.Hash) bool) error { @@ -1381,7 +1528,6 @@ func (s *StateDB) Copy() *StateDB { func (s *StateDB) CopyForSlot() *StateDB { // Copy all the basic fields, initialize the memory ones parallel := ParallelState{ - isSlotDB: true, // Share base slot db's stateObjects // It is a SyncMap, only readable to slot, not writable stateObjects: s.parallel.stateObjects, @@ -1394,6 +1540,8 @@ func (s *StateDB) CopyForSlot() *StateDB { balanceReadsInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), addrStateReadInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), addrStateChangeInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + nonceAdvanced: make(map[common.Address]struct{}, defaultNumOfSlots), + isSlotDB: true, dirtiedStateObjectsInSlot: make(map[common.Address]*StateObject, defaultNumOfSlots), } state := &StateDB{ @@ -1443,8 +1591,9 @@ func (s *StateDB) CopyForSlot() *StateDB { } state.snapStorage[k] = temp } - // slot will shared main stateDB's prefetcher - state.prefetcher = s.prefetcher + // trie prefetch should be done by dispacther on StateObject Merge, + // disable it in parallel slot + // state.prefetcher = s.prefetcher } return state } @@ -1506,6 +1655,9 @@ func (s *StateDB) Finalise(deleteEmptyObjects bool) { continue } if obj.suicided || (deleteEmptyObjects && obj.empty()) { + if s.parallel.isSlotDB { + s.parallel.addrStateChangeInSlot[addr] = struct{}{} // empty an StateObject is a state change + } obj.deleted = true // If state snapshotting is active, also mark the destruction there. diff --git a/core/state_processor.go b/core/state_processor.go index ee0d8d98ce..e6ff9f92bd 100644 --- a/core/state_processor.go +++ b/core/state_processor.go @@ -453,19 +453,21 @@ func (p *StateProcessor) hasStateConflict(readDb *state.StateDB, changeList stat // check KV change reads := readDb.StateReadsInSlot() writes := changeList.StateChangeSet - if len(reads) != 0 && len(writes) != 0 { + if len(reads) != 0 { for readAddr, readKeys := range reads { - if _, exist := changeList.StateObjectSuicided[readAddr]; exist { - log.Debug("conflict: read suicide object", "addr", readAddr) + if _, exist := changeList.AddrStateChangeSet[readAddr]; exist { + log.Debug("conflict: read addr changed state", "addr", readAddr) return true } - if writeKeys, ok := writes[readAddr]; ok { - // readAddr exist - for writeKey := range writeKeys { - // same addr and same key, mark conflicted - if _, ok := readKeys[writeKey]; ok { - log.Debug("conflict: state conflict", "addr", readAddr, "key", writeKey) - return true + if len(writes) != 0 { + if writeKeys, ok := writes[readAddr]; ok { + // readAddr exist + for writeKey := range writeKeys { + // same addr and same key, mark conflicted + if _, ok := readKeys[writeKey]; ok { + log.Debug("conflict: state conflict", "addr", readAddr, "key", writeKey) + return true + } } } } @@ -474,19 +476,22 @@ func (p *StateProcessor) hasStateConflict(readDb *state.StateDB, changeList stat // check balance change balanceReads := readDb.BalanceReadsInSlot() balanceWrite := changeList.BalanceChangeSet - if len(balanceReads) != 0 && len(balanceWrite) != 0 { + if len(balanceReads) != 0 { for readAddr := range balanceReads { - if _, exist := changeList.StateObjectSuicided[readAddr]; exist { - log.Debug("conflict: read suicide balance", "addr", readAddr) + if _, exist := changeList.AddrStateChangeSet[readAddr]; exist { + // txIndex = 0, would create StateObject for SystemAddress + log.Debug("conflict: read addr changed balance", "addr", readAddr) return true } - if _, ok := balanceWrite[readAddr]; ok { - if readAddr == consensus.SystemAddress { - log.Debug("conflict: skip specical system address's balance check") - continue + if len(balanceWrite) != 0 { + if _, ok := balanceWrite[readAddr]; ok { + if readAddr == consensus.SystemAddress { + // log.Debug("conflict: skip specical system address's balance check") + continue + } + log.Debug("conflict: balance conflict", "addr", readAddr) + return true } - log.Debug("conflict: balance conflict", "addr", readAddr) - return true } } } @@ -494,15 +499,17 @@ func (p *StateProcessor) hasStateConflict(readDb *state.StateDB, changeList stat // check code change codeReads := readDb.CodeReadInSlot() codeWrite := changeList.CodeChangeSet - if len(codeReads) != 0 && len(codeWrite) != 0 { + if len(codeReads) != 0 { for readAddr := range codeReads { - if _, exist := changeList.StateObjectSuicided[readAddr]; exist { - log.Debug("conflict: read suicide code", "addr", readAddr) + if _, exist := changeList.AddrStateChangeSet[readAddr]; exist { + log.Debug("conflict: read addr changed code", "addr", readAddr) return true } - if _, ok := codeWrite[readAddr]; ok { - log.Debug("conflict: code conflict", "addr", readAddr) - return true + if len(codeWrite) != 0 { + if _, ok := codeWrite[readAddr]; ok { + log.Debug("conflict: code conflict", "addr", readAddr) + return true + } } } } @@ -510,11 +517,22 @@ func (p *StateProcessor) hasStateConflict(readDb *state.StateDB, changeList stat // check address state change: create, suicide... addrReads := readDb.AddressReadInSlot() addrWrite := changeList.AddrStateChangeSet - if len(addrReads) != 0 && len(addrWrite) != 0 { - for readAddr := range addrReads { - if _, ok := addrWrite[readAddr]; ok { - log.Debug("conflict: address state conflict", "addr", readAddr) - return true + nonceWrite := changeList.NonceAdvancedSet + if len(addrReads) != 0 { + if len(addrWrite) != 0 { + for readAddr := range addrReads { + if _, ok := addrWrite[readAddr]; ok { + log.Debug("conflict: address state conflict", "addr", readAddr) + return true + } + } + } + if len(nonceWrite) != 0 { + for readAddr := range addrReads { + if _, ok := nonceWrite[readAddr]; ok { + log.Debug("conflict: address nonce conflict", "addr", readAddr) + return true + } } } } @@ -747,9 +765,11 @@ func (p *StateProcessor) execInParallelSlot(slotIndex int, txReq *ParallelTxRequ // can skip current slot now, since slotDB is always after current slot's merged DB // ** idle: all previous Txs are merged, it will create a new SlotDB // ** queued: it will request updateSlotDB, dispatcher will create or reuse a SlotDB after previous Tx results are merged - if index == slotIndex { - continue - } + + // with copy-on-write, can not skip current slot + // if index == slotIndex { + // continue + // } // check all finalizedDb from current slot's for _, changeList := range p.slotState[index].mergedChangeList { @@ -759,7 +779,8 @@ func (p *StateProcessor) execInParallelSlot(slotIndex int, txReq *ParallelTxRequ } if p.hasStateConflict(slotDB, changeList) { log.Debug("Stage Execution conflict", "Slot", slotIndex, - "txIndex", txIndex, " conflict slot", index, "slotDB.baseTxIndex", slotDB.BaseTxIndex()) + "txIndex", txIndex, " conflict slot", index, "slotDB.baseTxIndex", slotDB.BaseTxIndex(), + "conflict txIndex", changeList.TxIndex) hasConflict = true break } @@ -1075,9 +1096,9 @@ func (p *StateProcessor) ProcessParallel(block *types.Block, statedb *state.Stat log.Info("ProcessParallel tx all done", "block", header.Number, "usedGas", *usedGas, "txNum", txNum, "len(commonTxs)", len(commonTxs), - "debugErrorRedoNum", p.debugErrorRedoNum, - "debugConflictRedoNum", p.debugConflictRedoNum, - "redo rate(%)", 100*(p.debugErrorRedoNum+p.debugConflictRedoNum)/len(commonTxs)) + "errorNum", p.debugErrorRedoNum, + "conflictNum", p.debugConflictRedoNum, + "redoRate(%)", 100*(p.debugErrorRedoNum+p.debugConflictRedoNum)/len(commonTxs)) } allLogs, err := p.postExecute(block, statedb, &commonTxs, &receipts, &systemTxs, usedGas, bloomProcessors) return statedb, receipts, allLogs, *usedGas, err diff --git a/core/vm/evm.go b/core/vm/evm.go index 53e2e8797b..c7c8e0596c 100644 --- a/core/vm/evm.go +++ b/core/vm/evm.go @@ -475,6 +475,7 @@ func (evm *EVM) create(caller ContractRef, codeAndHash *codeAndHash, gas uint64, } nonce := evm.StateDB.GetNonce(caller.Address()) evm.StateDB.SetNonce(caller.Address(), nonce+1) + evm.StateDB.NonceChanged(caller.Address()) // We add this to the access list _before_ taking a snapshot. Even if the creation fails, // the access-list change should not be rolled back if evm.chainRules.IsBerlin { diff --git a/core/vm/interface.go b/core/vm/interface.go index ad9b05d666..c3d99aaa76 100644 --- a/core/vm/interface.go +++ b/core/vm/interface.go @@ -31,6 +31,7 @@ type StateDB interface { AddBalance(common.Address, *big.Int) GetBalance(common.Address) *big.Int + NonceChanged(common.Address) GetNonce(common.Address) uint64 SetNonce(common.Address, uint64) From 0c8e5bbc5391f9ab73d102f4904097cec41ab83b Mon Sep 17 00:00:00 2001 From: setunapo Date: Thu, 3 Mar 2022 19:45:40 +0800 Subject: [PATCH 7/8] Parallel: conflict optimize, remove SlotDB reuse, trie prefetch ** optimize conflict for AddBalance(0) Add balance with 0 did nothing, but it will do an empty() check, and add a touch event. Add on transaction finalize, the touch event will check if the StateObject is empty, do empty delete if it is. This patch is to take the empty check as a state check, if the addr state has not been changed(create, suicide, empty delete), then empty check is reliable. ** optimize conflict for system address ** some code improvement & lint fixup & refactor for params ** remove reuse SlotDB Reuse SlotDB was added to reduce copy of StateObject, in order to mitigate the Go GC problem. And COW(Copy-On-Write) is used to address the GC problem too. With COW enabled, reuse can be removed as it has limitted benefits now and add more complexity. ** fix trie prefetch on dispatcher Trie prefetch will be scheduled on object finalize. With parallel, we should schedule trie prefetch on dispatcher, since the TriePrefetcher is not safe for concurrent access and it is created & stopped on dispatcher routine. But object.finalize on slot cleared its dirtyStorage, which broken the later trie prefetch on dispatcher when do MergeSlotDB. --- core/state/statedb.go | 289 +++++++++++++++---------------------- core/state/statedb_test.go | 68 ++++----- core/state_processor.go | 134 +++++++---------- 3 files changed, 203 insertions(+), 288 deletions(-) diff --git a/core/state/statedb.go b/core/state/statedb.go index e37b28de38..b4e60811ee 100644 --- a/core/state/statedb.go +++ b/core/state/statedb.go @@ -115,14 +115,13 @@ func (s *StateDB) deleteStateObjectFromStateDB(addr common.Address) { // For parallel mode only, keep the change list for later conflict detect type SlotChangeList struct { - SlotDB *StateDB // used for SlotDb reuse only, otherwise, it can be discarded - TxIndex int // the tx index of change list + TxIndex int // the tx index of change list StateObjectSuicided map[common.Address]struct{} StateChangeSet map[common.Address]StateKeys BalanceChangeSet map[common.Address]struct{} CodeChangeSet map[common.Address]struct{} AddrStateChangeSet map[common.Address]struct{} - NonceAdvancedSet map[common.Address]struct{} + NonceChangeSet map[common.Address]struct{} } // For parallel mode only @@ -138,24 +137,24 @@ type ParallelState struct { baseTxIndex int // slotDB is created base on this tx index. dirtiedStateObjectsInSlot map[common.Address]*StateObject // for conflict check - balanceChangedInSlot map[common.Address]struct{} // the address's balance has been changed + balanceChangesInSlot map[common.Address]struct{} // the address's balance has been changed balanceReadsInSlot map[common.Address]struct{} // the address's balance has been read and used. - codeReadInSlot map[common.Address]struct{} - codeChangeInSlot map[common.Address]struct{} + codeReadsInSlot map[common.Address]struct{} + codeChangesInSlot map[common.Address]struct{} stateReadsInSlot map[common.Address]StateKeys - stateChangedInSlot map[common.Address]StateKeys // no need record value + stateChangesInSlot map[common.Address]StateKeys // no need record value // Actions such as SetCode, Suicide will change address's state. // Later call like Exist(), Empty(), HasSuicided() depond on the address's state. - addrStateReadInSlot map[common.Address]struct{} - addrStateChangeInSlot map[common.Address]struct{} - stateObjectSuicided map[common.Address]struct{} - nonceAdvanced map[common.Address]struct{} + addrStateReadsInSlot map[common.Address]struct{} + addrStateChangesInSlot map[common.Address]struct{} + stateObjectsSuicidedInSlot map[common.Address]struct{} + nonceChangesInSlot map[common.Address]struct{} // Transaction will pay gas fee to system address. // Parallel execution will clear system address's balance at first, in order to maintain transaction's // gas fee value. Normal transaction will access system address twice, otherwise it means the transaction // needs real system address's balance, the transaction will be marked redo with keepSystemAddressBalance = true systemAddress common.Address - systemAddressCount int + systemAddressOpsCount int keepSystemAddressBalance bool } @@ -250,7 +249,7 @@ func NewSlotDB(db *StateDB, systemAddr common.Address, baseTxIndex int, keepSyst slotDB.originalRoot = db.originalRoot slotDB.parallel.baseTxIndex = baseTxIndex slotDB.parallel.systemAddress = systemAddr - slotDB.parallel.systemAddressCount = 0 + slotDB.parallel.systemAddressOpsCount = 0 slotDB.parallel.keepSystemAddressBalance = keepSystem // clear the slotDB's validator's balance first @@ -262,53 +261,13 @@ func NewSlotDB(db *StateDB, systemAddr common.Address, baseTxIndex int, keepSyst return slotDB } -// to avoid new slotDB for each Tx, slotDB should be valid and merged -func ReUseSlotDB(slotDB *StateDB, keepSystem bool) *StateDB { - log.Debug("ReUseSlotDB", "baseTxIndex", slotDB.parallel.baseTxIndex, - "keepSystem", keepSystem, "refund", slotDB.refund, - "len(journal.entries)", len(slotDB.journal.entries)) - slotDB.logs = make(map[common.Hash][]*types.Log, defaultNumOfSlots) - slotDB.logSize = 0 - slotDB.parallel.systemAddressCount = 0 - slotDB.parallel.keepSystemAddressBalance = keepSystem - slotDB.parallel.stateObjectSuicided = make(map[common.Address]struct{}, defaultNumOfSlots) - slotDB.parallel.codeReadInSlot = make(map[common.Address]struct{}, defaultNumOfSlots) - slotDB.parallel.codeChangeInSlot = make(map[common.Address]struct{}, defaultNumOfSlots) - slotDB.parallel.stateChangedInSlot = make(map[common.Address]StateKeys, defaultNumOfSlots) - slotDB.parallel.stateReadsInSlot = make(map[common.Address]StateKeys, defaultNumOfSlots) - slotDB.parallel.balanceChangedInSlot = make(map[common.Address]struct{}, defaultNumOfSlots) - slotDB.parallel.balanceReadsInSlot = make(map[common.Address]struct{}, defaultNumOfSlots) - slotDB.parallel.addrStateReadInSlot = make(map[common.Address]struct{}, defaultNumOfSlots) - slotDB.parallel.addrStateChangeInSlot = make(map[common.Address]struct{}, defaultNumOfSlots) - slotDB.parallel.nonceAdvanced = make(map[common.Address]struct{}, defaultNumOfSlots) - - // Previous *StateObject in slot db has been transfered to dispatcher now. - // Slot could no longer use these *StateObject, do clear. - slotDB.parallel.dirtiedStateObjectsInSlot = make(map[common.Address]*StateObject, defaultNumOfSlots) - - slotDB.stateObjectsDirty = make(map[common.Address]struct{}, defaultNumOfSlots) - slotDB.stateObjectsPending = make(map[common.Address]struct{}, defaultNumOfSlots) - - // slotDB.snapDestructs = make(map[common.Address]struct{}) - // slotDB.snapAccounts = make(map[common.Address][]byte) - // slotDB.snapStorage = make(map[common.Address]map[string][]byte) - - if !keepSystem { - slotDB.SetBalance(slotDB.parallel.systemAddress, big.NewInt(0)) - } - return slotDB -} - func newStateDB(root common.Hash, db Database, snaps *snapshot.Tree) (*StateDB, error) { - parallel := ParallelState{ - isSlotDB: false, - } sdb := &StateDB{ db: db, originalRoot: root, snaps: snaps, stateObjects: make(map[common.Address]*StateObject, defaultNumOfSlots), - parallel: parallel, + parallel: ParallelState{}, stateObjectsPending: make(map[common.Address]struct{}, defaultNumOfSlots), stateObjectsDirty: make(map[common.Address]struct{}, defaultNumOfSlots), logs: make(map[common.Hash][]*types.Log, defaultNumOfSlots), @@ -348,15 +307,15 @@ func (s *StateDB) getStateObjectFromStateObjects(addr common.Address) (*StateObj // and discard its state changed, execept its own balance change. func (s *StateDB) RevertSlotDB(from common.Address) { log.Debug("RevertSlotDB", "addr", from, "txIndex", s.txIndex) - s.parallel.stateObjectSuicided = make(map[common.Address]struct{}) - s.parallel.stateChangedInSlot = make(map[common.Address]StateKeys) - s.parallel.balanceChangedInSlot = make(map[common.Address]struct{}, 1) - s.parallel.balanceChangedInSlot[from] = struct{}{} - s.parallel.addrStateChangeInSlot = make(map[common.Address]struct{}) - s.parallel.nonceAdvanced = make(map[common.Address]struct{}) - + s.parallel.stateObjectsSuicidedInSlot = make(map[common.Address]struct{}) + s.parallel.stateChangesInSlot = make(map[common.Address]StateKeys) + s.parallel.balanceChangesInSlot = make(map[common.Address]struct{}, 1) + s.parallel.balanceChangesInSlot[from] = struct{}{} + s.parallel.addrStateChangesInSlot = make(map[common.Address]struct{}) + s.parallel.nonceChangesInSlot = make(map[common.Address]struct{}) } +// PrepareForParallel prepares for state db to be used in parallel process. func (s *StateDB) PrepareForParallel() { s.isParallel = true s.parallel.stateObjects = &StateObjectSyncMap{} @@ -384,6 +343,7 @@ func (s *StateDB) MergeSlotDB(slotDb *StateDB, slotReceipt *types.Receipt, txInd } // only merge dirty objects + addressesToPrefetch := make([][]byte, 0, len(slotDb.stateObjectsDirty)) for addr := range slotDb.stateObjectsDirty { if _, exist := s.stateObjectsDirty[addr]; !exist { s.stateObjectsDirty[addr] = struct{}{} @@ -405,7 +365,7 @@ func (s *StateDB) MergeSlotDB(slotDb *StateDB, slotReceipt *types.Receipt, txInd if !exist { // addr not exist on main DB, do ownership transfer dirtyObj.db = s - dirtyObj.finalise(true) // prefetch on dispatcher + dirtyObj.finalise(true) // true: prefetch on dispatcher s.storeStateObjectToStateDB(addr, dirtyObj) delete(slotDb.parallel.dirtiedStateObjectsInSlot, addr) // transfer ownership } else { @@ -413,71 +373,59 @@ func (s *StateDB) MergeSlotDB(slotDb *StateDB, slotReceipt *types.Receipt, txInd // can not do copy or ownership transfer directly, since dirtyObj could have outdated // data(may be update within the conflict window) - // This is debug log for add balance with 0, can be removed - if dirtyObj.deleted && dirtyObj.empty() && !mainObj.empty() { - if _, exist := slotDb.parallel.balanceChangedInSlot[addr]; !exist { - // add(0) could trigger empty delete - // note: what if add/sub with 0 result, it is also a fake delete? - // it is ok, since none-zero add will produce a read record, should conflict - log.Warn("MergeSlotDB empty deleted", "txIndex", slotDb.txIndex, "addr", addr) - } - } - // Do deepCopy a temporary *StateObject for safety, // since slot could read the address, dispatch should avoid overwrite the StateObject directly // otherwise, it could crash for: concurrent map iteration and map write var newMainObj *StateObject - if _, suicided := slotDb.parallel.stateObjectSuicided[addr]; suicided { - if !dirtyObj.suicided { - // debug purpose, can be removed - log.Warn("MergeSlotDB suicide and recreated", "txIndex", slotDb.txIndex, "addr", addr) - } - } - - if dirtyObj.deleted { - log.Debug("MergeSlotDB state object merge: Suicide") - if !dirtyObj.suicided && !dirtyObj.empty() { - // none suicide object, should be empty delete - log.Error("MergeSlotDB none suicide deleted, should be empty", "txIndex", slotDb.txIndex, "addr", addr) - } - // suicided object will keep its worldstate(useless), and will be deleted from trie on block commit - newMainObj = dirtyObj.deepCopy(s) - delete(s.snapAccounts, addr) - delete(s.snapStorage, addr) - } else if _, created := slotDb.parallel.addrStateChangeInSlot[addr]; created { + if _, created := slotDb.parallel.addrStateChangesInSlot[addr]; created { + // there are 3 kinds of state change: + // 1.Suicide + // 2.Empty Delete + // 3.createObject + // a.AddBalance,SetState to an unexist or deleted(suicide, empty delete) address. + // b.CreateAccount: like DAO the fork, regenerate a account carry its balance without KV + // For these state change, do ownership transafer for efficiency: log.Debug("MergeSlotDB state object merge: addr state change") - // there are 2 kinds of object creation: - // 1.createObject: AddBalance,SetState to an unexist or emptyDeleted address. - // 2.CreateAccount: like DAO the fork, regenerate a account carry its balance without KV - // can not merge, do ownership transafer dirtyObj.db = s newMainObj = dirtyObj delete(slotDb.parallel.dirtiedStateObjectsInSlot, addr) // transfer ownership + if dirtyObj.deleted { + // remove the addr from snapAccounts&snapStorage only when object is deleted. + // "deleted" is not equal to "snapDestructs", since createObject() will add an addr for + // snapDestructs to destroy previous object, while it will keep the addr in snapAccounts & snapAccounts + delete(s.snapAccounts, addr) + delete(s.snapStorage, addr) + } } else { - newMainObj = mainObj.deepCopy(s) // do merge: balance, KV, code... - if _, balanced := slotDb.parallel.balanceChangedInSlot[addr]; balanced { + newMainObj = mainObj.deepCopy(s) + if _, balanced := slotDb.parallel.balanceChangesInSlot[addr]; balanced { log.Debug("MergeSlotDB state object merge: state merge: balance", "newMainObj.Balance()", newMainObj.Balance(), "dirtyObj.Balance()", dirtyObj.Balance()) newMainObj.SetBalance(dirtyObj.Balance()) } - if _, coded := slotDb.parallel.codeChangeInSlot[addr]; coded { + if _, coded := slotDb.parallel.codeChangesInSlot[addr]; coded { log.Debug("MergeSlotDB state object merge: state merge: code") newMainObj.code = dirtyObj.code newMainObj.data.CodeHash = dirtyObj.data.CodeHash newMainObj.dirtyCode = true } - if keys, stated := slotDb.parallel.stateChangedInSlot[addr]; stated { + if keys, stated := slotDb.parallel.stateChangesInSlot[addr]; stated { newMainObj.MergeSlotObject(s.db, dirtyObj, keys) } // dirtyObj.Nonce() should not be less than newMainObj newMainObj.setNonce(dirtyObj.Nonce()) } - newMainObj.finalise(true) // prefetch on dispatcher + newMainObj.finalise(true) // true: prefetch on dispatcher // update the object s.storeStateObjectToStateDB(addr, newMainObj) } + addressesToPrefetch = append(addressesToPrefetch, common.CopyBytes(addr[:])) // Copy needed for closure + } + + if s.prefetcher != nil && len(addressesToPrefetch) > 0 { + s.prefetcher.prefetch(s.originalRoot, addressesToPrefetch, emptyAddr) // prefetch for trie node of account } for addr := range slotDb.stateObjectsPending { @@ -497,6 +445,13 @@ func (s *StateDB) MergeSlotDB(slotDb *StateDB, slotReceipt *types.Receipt, txInd } if slotDb.snaps != nil { + for k := range slotDb.snapDestructs { + // There could be a race condition for parallel transaction execution + // One add balance 0 to an empty address, it will delete it(delete empty is enabled,). + // While another concurrent transaction could add a none-zero balance to it, make it not empty + // We fixed it by add a addr state read record for add balance 0 + s.snapDestructs[k] = struct{}{} + } for k, v := range slotDb.snapAccounts { s.snapAccounts[k] = v } @@ -507,49 +462,36 @@ func (s *StateDB) MergeSlotDB(slotDb *StateDB, slotReceipt *types.Receipt, txInd } s.snapStorage[k] = temp } - for k := range slotDb.snapDestructs { - obj, _ := s.loadStateObjectFromStateDB(k) - if !obj.suicided && !obj.empty() { - // There could be a race condition for parallel transaction execution - // One add balance 0 to an empty address, it will delete it(delete empty is enabled,). - // While another concurrent transaction could add a none-zero balance to it, make it not empty - // Right now, we mark a read operation to avoid this race condition - // We hope to remove this read record, since it could reduce conflict rate. - log.Warn("MergeSlotDB state object to be deleted is not empty, ", "addr", k) - } - s.snapDestructs[k] = struct{}{} - } } // we have to create a new object to store change list for conflict detect, since // StateDB could be reused and its elements could be overwritten changeList := SlotChangeList{ - SlotDB: slotDb, TxIndex: txIndex, - StateObjectSuicided: make(map[common.Address]struct{}, len(slotDb.parallel.stateObjectSuicided)), - StateChangeSet: make(map[common.Address]StateKeys, len(slotDb.parallel.stateChangedInSlot)), - BalanceChangeSet: make(map[common.Address]struct{}, len(slotDb.parallel.balanceChangedInSlot)), - CodeChangeSet: make(map[common.Address]struct{}, len(slotDb.parallel.codeChangeInSlot)), - AddrStateChangeSet: make(map[common.Address]struct{}, len(slotDb.parallel.addrStateChangeInSlot)), - NonceAdvancedSet: make(map[common.Address]struct{}, len(slotDb.parallel.nonceAdvanced)), - } - for addr := range slotDb.parallel.stateObjectSuicided { + StateObjectSuicided: make(map[common.Address]struct{}, len(slotDb.parallel.stateObjectsSuicidedInSlot)), + StateChangeSet: make(map[common.Address]StateKeys, len(slotDb.parallel.stateChangesInSlot)), + BalanceChangeSet: make(map[common.Address]struct{}, len(slotDb.parallel.balanceChangesInSlot)), + CodeChangeSet: make(map[common.Address]struct{}, len(slotDb.parallel.codeChangesInSlot)), + AddrStateChangeSet: make(map[common.Address]struct{}, len(slotDb.parallel.addrStateChangesInSlot)), + NonceChangeSet: make(map[common.Address]struct{}, len(slotDb.parallel.nonceChangesInSlot)), + } + for addr := range slotDb.parallel.stateObjectsSuicidedInSlot { changeList.StateObjectSuicided[addr] = struct{}{} } - for addr, storage := range slotDb.parallel.stateChangedInSlot { + for addr, storage := range slotDb.parallel.stateChangesInSlot { changeList.StateChangeSet[addr] = storage } - for addr := range slotDb.parallel.balanceChangedInSlot { + for addr := range slotDb.parallel.balanceChangesInSlot { changeList.BalanceChangeSet[addr] = struct{}{} } - for addr := range slotDb.parallel.codeChangeInSlot { + for addr := range slotDb.parallel.codeChangesInSlot { changeList.CodeChangeSet[addr] = struct{}{} } - for addr := range slotDb.parallel.addrStateChangeInSlot { + for addr := range slotDb.parallel.addrStateChangesInSlot { changeList.AddrStateChangeSet[addr] = struct{}{} } - for addr := range slotDb.parallel.nonceAdvanced { - changeList.NonceAdvancedSet[addr] = struct{}{} + for addr := range slotDb.parallel.nonceChangesInSlot { + changeList.NonceChangeSet[addr] = struct{}{} } // the slot DB's is valid now, move baseTxIndex forward, since it could be reused. @@ -723,7 +665,7 @@ func (s *StateDB) GetBalance(addr common.Address) *big.Int { if s.parallel.isSlotDB { s.parallel.balanceReadsInSlot[addr] = struct{}{} if addr == s.parallel.systemAddress { - s.parallel.systemAddressCount++ + s.parallel.systemAddressOpsCount++ } } stateObject := s.getStateObject(addr) @@ -756,12 +698,12 @@ func (s *StateDB) BaseTxIndex() int { return s.parallel.baseTxIndex } -func (s *StateDB) CodeReadInSlot() map[common.Address]struct{} { - return s.parallel.codeReadInSlot +func (s *StateDB) CodeReadsInSlot() map[common.Address]struct{} { + return s.parallel.codeReadsInSlot } -func (s *StateDB) AddressReadInSlot() map[common.Address]struct{} { - return s.parallel.addrStateReadInSlot +func (s *StateDB) AddressReadsInSlot() map[common.Address]struct{} { + return s.parallel.addrStateReadsInSlot } func (s *StateDB) StateReadsInSlot() map[common.Address]StateKeys { @@ -772,12 +714,12 @@ func (s *StateDB) BalanceReadsInSlot() map[common.Address]struct{} { return s.parallel.balanceReadsInSlot } func (s *StateDB) SystemAddressRedo() bool { - return s.parallel.systemAddressCount > 2 + return s.parallel.systemAddressOpsCount > 2 } func (s *StateDB) GetCode(addr common.Address) []byte { if s.parallel.isSlotDB { - s.parallel.codeReadInSlot[addr] = struct{}{} + s.parallel.codeReadsInSlot[addr] = struct{}{} } stateObject := s.getStateObject(addr) @@ -789,7 +731,7 @@ func (s *StateDB) GetCode(addr common.Address) []byte { func (s *StateDB) GetCodeSize(addr common.Address) int { if s.parallel.isSlotDB { - s.parallel.codeReadInSlot[addr] = struct{}{} // code size is part of code + s.parallel.codeReadsInSlot[addr] = struct{}{} // code size is part of code } stateObject := s.getStateObject(addr) @@ -801,7 +743,7 @@ func (s *StateDB) GetCodeSize(addr common.Address) int { func (s *StateDB) GetCodeHash(addr common.Address) common.Hash { if s.parallel.isSlotDB { - s.parallel.codeReadInSlot[addr] = struct{}{} // code hash is part of code + s.parallel.codeReadsInSlot[addr] = struct{}{} // code hash is part of code } stateObject := s.getStateObject(addr) @@ -915,12 +857,16 @@ func (s *StateDB) AddBalance(addr common.Address, amount *big.Int) { // just in case other tx creates this account, // we will miss this if we only add this account when found if amount.Sign() != 0 { - s.parallel.balanceChangedInSlot[addr] = struct{}{} + s.parallel.balanceChangesInSlot[addr] = struct{}{} + // add balance will perform a read operation first, empty object will be deleted + s.parallel.balanceReadsInSlot[addr] = struct{}{} + } else { + // if amount == 0, no balance change, but there is still an empty check. + // take this empty check as addr state read(create, suicide, empty delete) + s.parallel.addrStateReadsInSlot[addr] = struct{}{} } - // add balance will perform a read operation first, empty object will be deleted - s.parallel.balanceReadsInSlot[addr] = struct{}{} if addr == s.parallel.systemAddress { - s.parallel.systemAddressCount++ + s.parallel.systemAddressOpsCount++ } } @@ -946,12 +892,12 @@ func (s *StateDB) SubBalance(addr common.Address, amount *big.Int) { // just in case other tx creates this account, // we will miss this if we only add this account when found if amount.Sign() != 0 { - s.parallel.balanceChangedInSlot[addr] = struct{}{} + s.parallel.balanceChangesInSlot[addr] = struct{}{} // unlike add, sub 0 balance will not touch empty object s.parallel.balanceReadsInSlot[addr] = struct{}{} } if addr == s.parallel.systemAddress { - s.parallel.systemAddressCount++ + s.parallel.systemAddressOpsCount++ } } @@ -982,9 +928,9 @@ func (s *StateDB) SetBalance(addr common.Address, amount *big.Int) { } else { stateObject.SetBalance(amount) } - s.parallel.balanceChangedInSlot[addr] = struct{}{} + s.parallel.balanceChangesInSlot[addr] = struct{}{} if addr == s.parallel.systemAddress { - s.parallel.systemAddressCount++ + s.parallel.systemAddressOpsCount++ } } else { stateObject.SetBalance(amount) @@ -1000,7 +946,7 @@ func (s *StateDB) SetBalance(addr common.Address, amount *big.Int) { func (s *StateDB) NonceChanged(addr common.Address) { if s.parallel.isSlotDB { log.Debug("NonceChanged", "txIndex", s.txIndex, "addr", addr) - s.parallel.nonceAdvanced[addr] = struct{}{} + s.parallel.nonceChangesInSlot[addr] = struct{}{} } } @@ -1033,7 +979,7 @@ func (s *StateDB) SetCode(addr common.Address, code []byte) { stateObject.SetCode(crypto.Keccak256Hash(code), code) } - s.parallel.codeChangeInSlot[addr] = struct{}{} + s.parallel.codeChangesInSlot[addr] = struct{}{} } else { stateObject.SetCode(crypto.Keccak256Hash(code), code) } @@ -1061,10 +1007,10 @@ func (s *StateDB) SetState(addr common.Address, key, value common.Hash) { stateObject.SetState(s.db, key, value) } - if s.parallel.stateChangedInSlot[addr] == nil { - s.parallel.stateChangedInSlot[addr] = make(StateKeys, defaultNumOfSlots) + if s.parallel.stateChangesInSlot[addr] == nil { + s.parallel.stateChangesInSlot[addr] = make(StateKeys, defaultNumOfSlots) } - s.parallel.stateChangedInSlot[addr][key] = struct{}{} + s.parallel.stateChangesInSlot[addr][key] = struct{}{} } else { stateObject.SetState(s.db, key, value) } @@ -1089,7 +1035,7 @@ func (s *StateDB) Suicide(addr common.Address) bool { stateObject := s.getStateObject(addr) if s.parallel.isSlotDB { - s.parallel.addrStateReadInSlot[addr] = struct{}{} + s.parallel.addrStateReadsInSlot[addr] = struct{}{} } if stateObject == nil { return false @@ -1102,8 +1048,8 @@ func (s *StateDB) Suicide(addr common.Address) bool { }) if s.parallel.isSlotDB { - s.parallel.stateObjectSuicided[addr] = struct{}{} - s.parallel.addrStateChangeInSlot[addr] = struct{}{} + s.parallel.stateObjectsSuicidedInSlot[addr] = struct{}{} + s.parallel.addrStateChangesInSlot[addr] = struct{}{} if _, ok := s.parallel.dirtiedStateObjectsInSlot[addr]; !ok { // do copy-on-write for suicide "write" newStateObject := stateObject.deepCopy(s) @@ -1165,7 +1111,7 @@ func (s *StateDB) deleteStateObject(obj *StateObject) { // to differentiate between non-existent/just-deleted, use getDeletedStateObject. func (s *StateDB) getStateObject(addr common.Address) *StateObject { if s.parallel.isSlotDB { - s.parallel.addrStateReadInSlot[addr] = struct{}{} + s.parallel.addrStateReadsInSlot[addr] = struct{}{} } if obj := s.getDeletedStateObject(addr); obj != nil && !obj.deleted { @@ -1337,8 +1283,8 @@ func (s *StateDB) GetOrNewStateObject(addr common.Address) *StateObject { // the given address, it is overwritten and returned as the second return value. func (s *StateDB) createObject(addr common.Address) (newobj, prev *StateObject) { if s.parallel.isSlotDB { - s.parallel.addrStateReadInSlot[addr] = struct{}{} // fixme: may not necessary - s.parallel.addrStateChangeInSlot[addr] = struct{}{} // address created. + s.parallel.addrStateReadsInSlot[addr] = struct{}{} // fixme: may not necessary + s.parallel.addrStateChangesInSlot[addr] = struct{}{} // address created. } prev = s.getDeletedStateObject(addr) // Note, prev might have been deleted, we need that! @@ -1530,19 +1476,19 @@ func (s *StateDB) CopyForSlot() *StateDB { parallel := ParallelState{ // Share base slot db's stateObjects // It is a SyncMap, only readable to slot, not writable - stateObjects: s.parallel.stateObjects, - stateObjectSuicided: make(map[common.Address]struct{}, defaultNumOfSlots), - codeReadInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - codeChangeInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - stateChangedInSlot: make(map[common.Address]StateKeys, defaultNumOfSlots), - stateReadsInSlot: make(map[common.Address]StateKeys, defaultNumOfSlots), - balanceChangedInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - balanceReadsInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - addrStateReadInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - addrStateChangeInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - nonceAdvanced: make(map[common.Address]struct{}, defaultNumOfSlots), - isSlotDB: true, - dirtiedStateObjectsInSlot: make(map[common.Address]*StateObject, defaultNumOfSlots), + stateObjects: s.parallel.stateObjects, + stateObjectsSuicidedInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + codeReadsInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + codeChangesInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + stateChangesInSlot: make(map[common.Address]StateKeys, defaultNumOfSlots), + stateReadsInSlot: make(map[common.Address]StateKeys, defaultNumOfSlots), + balanceChangesInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + balanceReadsInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + addrStateReadsInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + addrStateChangesInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + nonceChangesInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + isSlotDB: true, + dirtiedStateObjectsInSlot: make(map[common.Address]*StateObject, defaultNumOfSlots), } state := &StateDB{ db: s.db, @@ -1656,7 +1602,7 @@ func (s *StateDB) Finalise(deleteEmptyObjects bool) { } if obj.suicided || (deleteEmptyObjects && obj.empty()) { if s.parallel.isSlotDB { - s.parallel.addrStateChangeInSlot[addr] = struct{}{} // empty an StateObject is a state change + s.parallel.addrStateChangesInSlot[addr] = struct{}{} // empty an StateObject is a state change } obj.deleted = true @@ -1670,7 +1616,12 @@ func (s *StateDB) Finalise(deleteEmptyObjects bool) { delete(s.snapStorage, obj.address) // Clear out any previously updated storage data (may be recreated via a ressurrect) } } else { - obj.finalise(true) // Prefetch slots in the background + // 1.none parallel mode, we do obj.finalise(true) as normal + // 2.with parallel mode, we do obj.finalise(true) on dispatcher, not on slot routine + // obj.finalise(true) will clear its dirtyStorage, will make prefetch broken. + if !s.isParallel || !s.parallel.isSlotDB { + obj.finalise(true) // Prefetch slots in the background + } } if _, exist := s.stateObjectsPending[addr]; !exist { s.stateObjectsPending[addr] = struct{}{} diff --git a/core/state/statedb_test.go b/core/state/statedb_test.go index 2841381963..5554375560 100644 --- a/core/state/statedb_test.go +++ b/core/state/statedb_test.go @@ -955,12 +955,12 @@ func TestSuicide(t *testing.T) { t.Fatalf("expected account suicide, got %v", result) } - if _, ok := slotDb.parallel.stateObjectSuicided[addr]; !ok { - t.Fatalf("address should exist in stateObjectSuicided") + if _, ok := slotDb.parallel.stateObjectsSuicidedInSlot[addr]; !ok { + t.Fatalf("address should exist in stateObjectsSuicidedInSlot") } - if _, ok := slotDb.parallel.addrStateChangeInSlot[addr]; !ok { - t.Fatalf("address should exist in addrStateChangeInSlot") + if _, ok := slotDb.parallel.addrStateChangesInSlot[addr]; !ok { + t.Fatalf("address should exist in addrStateChangesInSlot") } if _, ok := slotDb.parallel.dirtiedStateObjectsInSlot[addr]; !ok { @@ -972,8 +972,8 @@ func TestSuicide(t *testing.T) { t.Fatalf("address should be suicided") } - if _, ok := slotDb.parallel.addrStateReadInSlot[addr]; !ok { - t.Fatalf("address should exist in addrStateReadInSlot") + if _, ok := slotDb.parallel.addrStateReadsInSlot[addr]; !ok { + t.Fatalf("address should exist in addrStateReadsInSlot") } } @@ -994,8 +994,8 @@ func TestSetAndGetState(t *testing.T) { t.Fatalf("address should exist in dirtiedStateObjectsInSlot") } - if _, ok := slotDb.parallel.stateChangedInSlot[addr]; !ok { - t.Fatalf("address should exist in stateChangedInSlot") + if _, ok := slotDb.parallel.stateChangesInSlot[addr]; !ok { + t.Fatalf("address should exist in stateChangesInSlot") } oldValueRead := state.GetState(addr, common.BytesToHash([]byte("test key"))) @@ -1035,8 +1035,8 @@ func TestSetAndGetCode(t *testing.T) { t.Fatalf("address should exist in dirtiedStateObjectsInSlot") } - if _, ok := slotDb.parallel.codeChangeInSlot[addr]; !ok { - t.Fatalf("address should exist in codeChangeInSlot") + if _, ok := slotDb.parallel.codeChangesInSlot[addr]; !ok { + t.Fatalf("address should exist in codeChangesInSlot") } codeRead := slotDb.GetCode(addr) @@ -1044,8 +1044,8 @@ func TestSetAndGetCode(t *testing.T) { t.Fatalf("code read should be equal to the code stored") } - if _, ok := slotDb.parallel.codeReadInSlot[addr]; !ok { - t.Fatalf("address should exist in codeReadInSlot") + if _, ok := slotDb.parallel.codeReadsInSlot[addr]; !ok { + t.Fatalf("address should exist in codeReadsInSlot") } } @@ -1067,8 +1067,8 @@ func TestGetCodeSize(t *testing.T) { t.Fatalf("code size should be 9") } - if _, ok := slotDb.parallel.codeReadInSlot[addr]; !ok { - t.Fatalf("address should exist in codeReadInSlot") + if _, ok := slotDb.parallel.codeReadsInSlot[addr]; !ok { + t.Fatalf("address should exist in codeReadsInSlot") } } @@ -1090,8 +1090,8 @@ func TestGetCodeHash(t *testing.T) { if hex.EncodeToString(codeSize[:]) != "6e73fa02f7828b28608b078b007a4023fb40453c3e102b83828a3609a94d8cbb" { t.Fatalf("code hash should be 6e73fa02f7828b28608b078b007a4023fb40453c3e102b83828a3609a94d8cbb") } - if _, ok := slotDb.parallel.codeReadInSlot[addr]; !ok { - t.Fatalf("address should exist in codeReadInSlot") + if _, ok := slotDb.parallel.codeReadsInSlot[addr]; !ok { + t.Fatalf("address should exist in codeReadsInSlot") } } @@ -1145,12 +1145,12 @@ func TestSetAndGetBalance(t *testing.T) { t.Fatalf("address should exist in dirtiedStateObjectsInSlot") } - if _, ok := slotDb.parallel.balanceChangedInSlot[addr]; !ok { - t.Fatalf("address should exist in balanceChangedInSlot") + if _, ok := slotDb.parallel.balanceChangesInSlot[addr]; !ok { + t.Fatalf("address should exist in balanceChangesInSlot") } - if slotDb.parallel.systemAddressCount != 1 { - t.Fatalf("systemAddressCount should be 1") + if slotDb.parallel.systemAddressOpsCount != 1 { + t.Fatalf("systemAddressOpsCount should be 1") } newBalance := slotDb.GetBalance(addr) @@ -1162,8 +1162,8 @@ func TestSetAndGetBalance(t *testing.T) { t.Fatalf("address should exist in balanceReadsInSlot") } - if slotDb.parallel.systemAddressCount != 2 { - t.Fatalf("systemAddressCount should be 1") + if slotDb.parallel.systemAddressOpsCount != 2 { + t.Fatalf("systemAddressOpsCount should be 1") } } @@ -1189,16 +1189,16 @@ func TestSubBalance(t *testing.T) { t.Fatalf("address should exist in dirtiedStateObjectsInSlot") } - if _, ok := slotDb.parallel.balanceChangedInSlot[addr]; !ok { - t.Fatalf("address should exist in balanceChangedInSlot") + if _, ok := slotDb.parallel.balanceChangesInSlot[addr]; !ok { + t.Fatalf("address should exist in balanceChangesInSlot") } if _, ok := slotDb.parallel.balanceReadsInSlot[addr]; !ok { t.Fatalf("address should exist in balanceReadsInSlot") } - if slotDb.parallel.systemAddressCount != 1 { - t.Fatalf("systemAddressCount should be 1") + if slotDb.parallel.systemAddressOpsCount != 1 { + t.Fatalf("systemAddressOpsCount should be 1") } newBalance := slotDb.GetBalance(addr) @@ -1229,16 +1229,16 @@ func TestAddBalance(t *testing.T) { t.Fatalf("address should exist in dirtiedStateObjectsInSlot") } - if _, ok := slotDb.parallel.balanceChangedInSlot[addr]; !ok { - t.Fatalf("address should exist in balanceChangedInSlot") + if _, ok := slotDb.parallel.balanceChangesInSlot[addr]; !ok { + t.Fatalf("address should exist in balanceChangesInSlot") } if _, ok := slotDb.parallel.balanceReadsInSlot[addr]; !ok { t.Fatalf("address should exist in balanceReadsInSlot") } - if slotDb.parallel.systemAddressCount != 1 { - t.Fatalf("systemAddressCount should be 1") + if slotDb.parallel.systemAddressOpsCount != 1 { + t.Fatalf("systemAddressOpsCount should be 1") } newBalance := slotDb.GetBalance(addr) @@ -1263,8 +1263,8 @@ func TestEmpty(t *testing.T) { t.Fatalf("address should exist") } - if _, ok := slotDb.parallel.addrStateReadInSlot[addr]; !ok { - t.Fatalf("address should exist in addrStateReadInSlot") + if _, ok := slotDb.parallel.addrStateReadsInSlot[addr]; !ok { + t.Fatalf("address should exist in addrStateReadsInSlot") } } @@ -1284,8 +1284,8 @@ func TestExist(t *testing.T) { t.Fatalf("address should exist") } - if _, ok := slotDb.parallel.addrStateReadInSlot[addr]; !ok { - t.Fatalf("address should exist in addrStateReadInSlot") + if _, ok := slotDb.parallel.addrStateReadsInSlot[addr]; !ok { + t.Fatalf("address should exist in addrStateReadsInSlot") } } diff --git a/core/state_processor.go b/core/state_processor.go index e6ff9f92bd..14cf549f65 100644 --- a/core/state_processor.go +++ b/core/state_processor.go @@ -48,7 +48,6 @@ const ( recentTime = 1024 * 3 recentDiffLayerTimeout = 5 farDiffLayerTimeout = 2 - reuseSlotDB = true // reuse could save state object copy cost ) var MaxPendingQueueSize = 20 // parallel slot's maximum number of pending Txs @@ -397,7 +396,6 @@ type SlotState struct { type ParallelTxResult struct { redo bool // for redo, dispatch will wait new tx result updateSlotDB bool // for redo and pending tx quest, slot needs new slotDB, - reuseSlotDB bool // will try to reuse latest finalized slotDB keepSystem bool // for redo, should keep system address's balance txIndex int slotIndex int // slot index @@ -453,22 +451,18 @@ func (p *StateProcessor) hasStateConflict(readDb *state.StateDB, changeList stat // check KV change reads := readDb.StateReadsInSlot() writes := changeList.StateChangeSet - if len(reads) != 0 { - for readAddr, readKeys := range reads { - if _, exist := changeList.AddrStateChangeSet[readAddr]; exist { - log.Debug("conflict: read addr changed state", "addr", readAddr) - return true - } - if len(writes) != 0 { - if writeKeys, ok := writes[readAddr]; ok { - // readAddr exist - for writeKey := range writeKeys { - // same addr and same key, mark conflicted - if _, ok := readKeys[writeKey]; ok { - log.Debug("conflict: state conflict", "addr", readAddr, "key", writeKey) - return true - } - } + for readAddr, readKeys := range reads { + if _, exist := changeList.AddrStateChangeSet[readAddr]; exist { + log.Debug("conflict: read addr changed state", "addr", readAddr) + return true + } + if writeKeys, ok := writes[readAddr]; ok { + // readAddr exist + for writeKey := range writeKeys { + // same addr and same key, mark conflicted + if _, ok := readKeys[writeKey]; ok { + log.Debug("conflict: state conflict", "addr", readAddr, "key", writeKey) + return true } } } @@ -476,64 +470,53 @@ func (p *StateProcessor) hasStateConflict(readDb *state.StateDB, changeList stat // check balance change balanceReads := readDb.BalanceReadsInSlot() balanceWrite := changeList.BalanceChangeSet - if len(balanceReads) != 0 { - for readAddr := range balanceReads { - if _, exist := changeList.AddrStateChangeSet[readAddr]; exist { - // txIndex = 0, would create StateObject for SystemAddress + for readAddr := range balanceReads { + if _, exist := changeList.AddrStateChangeSet[readAddr]; exist { + // SystemAddress is special, SystemAddressRedo() is prepared for it. + // Since txIndex = 0 will create StateObject for SystemAddress, skip its state change check + if readAddr != consensus.SystemAddress { log.Debug("conflict: read addr changed balance", "addr", readAddr) return true } - if len(balanceWrite) != 0 { - if _, ok := balanceWrite[readAddr]; ok { - if readAddr == consensus.SystemAddress { - // log.Debug("conflict: skip specical system address's balance check") - continue - } - log.Debug("conflict: balance conflict", "addr", readAddr) - return true - } + } + if _, ok := balanceWrite[readAddr]; ok { + if readAddr != consensus.SystemAddress { + log.Debug("conflict: balance conflict", "addr", readAddr) + return true } } } // check code change - codeReads := readDb.CodeReadInSlot() + codeReads := readDb.CodeReadsInSlot() codeWrite := changeList.CodeChangeSet - if len(codeReads) != 0 { - for readAddr := range codeReads { - if _, exist := changeList.AddrStateChangeSet[readAddr]; exist { - log.Debug("conflict: read addr changed code", "addr", readAddr) - return true - } - if len(codeWrite) != 0 { - if _, ok := codeWrite[readAddr]; ok { - log.Debug("conflict: code conflict", "addr", readAddr) - return true - } - } + for readAddr := range codeReads { + if _, exist := changeList.AddrStateChangeSet[readAddr]; exist { + log.Debug("conflict: read addr changed code", "addr", readAddr) + return true + } + if _, ok := codeWrite[readAddr]; ok { + log.Debug("conflict: code conflict", "addr", readAddr) + return true } } // check address state change: create, suicide... - addrReads := readDb.AddressReadInSlot() + addrReads := readDb.AddressReadsInSlot() addrWrite := changeList.AddrStateChangeSet - nonceWrite := changeList.NonceAdvancedSet - if len(addrReads) != 0 { - if len(addrWrite) != 0 { - for readAddr := range addrReads { - if _, ok := addrWrite[readAddr]; ok { - log.Debug("conflict: address state conflict", "addr", readAddr) - return true - } + nonceWrite := changeList.NonceChangeSet + for readAddr := range addrReads { + if _, ok := addrWrite[readAddr]; ok { + // SystemAddress is special, SystemAddressRedo() is prepared for it. + // Since txIndex = 0 will create StateObject for SystemAddress, skip its state change check + if readAddr != consensus.SystemAddress { + log.Debug("conflict: address state conflict", "addr", readAddr) + return true } } - if len(nonceWrite) != 0 { - for readAddr := range addrReads { - if _, ok := nonceWrite[readAddr]; ok { - log.Debug("conflict: address nonce conflict", "addr", readAddr) - return true - } - } + if _, ok := nonceWrite[readAddr]; ok { + log.Debug("conflict: address nonce conflict", "addr", readAddr) + return true } } @@ -633,14 +616,7 @@ func (p *StateProcessor) waitUntilNextTxDone(statedb *state.StateDB) *ParallelTx if result.updateSlotDB { // the target slot is waiting for new slotDB slotState := p.slotState[result.slotIndex] - var slotDB *state.StateDB - if result.reuseSlotDB { - // for reuse, len(slotState.mergedChangeList) must >= 1 - lastSlotDB := slotState.mergedChangeList[len(slotState.mergedChangeList)-1].SlotDB - slotDB = state.ReUseSlotDB(lastSlotDB, result.keepSystem) - } else { - slotDB = state.NewSlotDB(statedb, consensus.SystemAddress, p.mergedTxIndex, result.keepSystem) - } + slotDB := state.NewSlotDB(statedb, consensus.SystemAddress, p.mergedTxIndex, result.keepSystem) slotState.slotdbChan <- slotDB continue } @@ -697,7 +673,6 @@ func (p *StateProcessor) execInParallelSlot(slotIndex int, txReq *ParallelTxRequ var err error var evm *vm.EVM - // fixme: to optimize, reuse the slotDB slotDB.Prepare(tx.Hash(), block.Hash(), txIndex) log.Debug("exec In Slot", "Slot", slotIndex, "txIndex", txIndex, "slotDB.baseTxIndex", slotDB.BaseTxIndex()) @@ -726,7 +701,6 @@ func (p *StateProcessor) execInParallelSlot(slotIndex int, txReq *ParallelTxRequ redoResult := &ParallelTxResult{ redo: true, updateSlotDB: true, - reuseSlotDB: false, txIndex: txIndex, slotIndex: slotIndex, tx: tx, @@ -762,14 +736,9 @@ func (p *StateProcessor) execInParallelSlot(slotIndex int, txReq *ParallelTxRequ systemAddrConflict = true } else { for index := 0; index < ParallelExecNum; index++ { - // can skip current slot now, since slotDB is always after current slot's merged DB - // ** idle: all previous Txs are merged, it will create a new SlotDB - // ** queued: it will request updateSlotDB, dispatcher will create or reuse a SlotDB after previous Tx results are merged - - // with copy-on-write, can not skip current slot - // if index == slotIndex { - // continue - // } + if index == slotIndex { + continue + } // check all finalizedDb from current slot's for _, changeList := range p.slotState[index].mergedChangeList { @@ -797,7 +766,6 @@ func (p *StateProcessor) execInParallelSlot(slotIndex int, txReq *ParallelTxRequ redoResult := &ParallelTxResult{ redo: true, updateSlotDB: true, - reuseSlotDB: false, // for conflict, we do not reuse keepSystem: systemAddrConflict, txIndex: txIndex, slotIndex: slotIndex, @@ -868,15 +836,11 @@ func (p *StateProcessor) runSlotLoop(slotIndex int) { // ** for a dispatched tx, // the slot should be idle, it is better to create a new SlotDB, since new Tx is not related to previous Tx // ** for a queued tx, - // the previous SlotDB could be reused, since it is likely can be used - // reuse could avoid NewSlotDB cost, which could be costable when StateDB is full of state object - // if the previous SlotDB is + // it is better to create a new SlotDB, since COW is used. if txReq.slotDB == nil { - // for queued Tx, txReq.slotDB is nil, reuse slot's latest merged SlotDB result := &ParallelTxResult{ redo: false, updateSlotDB: true, - reuseSlotDB: reuseSlotDB, slotIndex: slotIndex, err: nil, } @@ -1063,7 +1027,7 @@ func (p *StateProcessor) ProcessParallel(block *types.Block, statedb *state.Stat // log.Info("ProcessParallel dispatch to idle slot", "txIndex", txReq.txIndex) break } - log.Debug("ProcessParallel no slot avaiable, wait", "txIndex", txReq.txIndex) + log.Debug("ProcessParallel no slot available, wait", "txIndex", txReq.txIndex) // no idle slot, wait until a tx is executed and merged. result := p.waitUntilNextTxDone(statedb) From a630f1626c19010d2fed0d3d8fb6e190914dd88e Mon Sep 17 00:00:00 2001 From: setunapo Date: Thu, 10 Mar 2022 16:54:42 +0800 Subject: [PATCH 8/8] Parallel: handle fixup & code review & enhancement No fundamental change, some improvements, include: ** Add a new type ParallelStateProcessor; ** move Parallel Config to BlockChain ** more precious ParallelNum set ** Add EnableParallelProcessor() ** remove panic() ** remove useless: redo flag, ** change waitChan from `chan int` to `chan struct {}` and communicate by close() ** dispatch policy: queue `from` ahead of `to` ** pre-allocate allLogs ** disable parallel processor is snapshot is not enabled ** others: rename... --- cmd/utils/flags.go | 46 ++-- core/blockchain.go | 45 ++-- core/state/journal.go | 2 +- core/state/state_object.go | 2 - core/state/statedb.go | 283 ++++++++++-------------- core/state_processor.go | 438 +++++++++++++++++-------------------- core/types.go | 4 - eth/backend.go | 2 + eth/ethconfig/config.go | 5 +- 9 files changed, 384 insertions(+), 443 deletions(-) diff --git a/cmd/utils/flags.go b/cmd/utils/flags.go index 87affa96c4..b73d17e5f8 100644 --- a/cmd/utils/flags.go +++ b/cmd/utils/flags.go @@ -26,6 +26,7 @@ import ( "math/big" "os" "path/filepath" + "runtime" godebug "runtime/debug" "strconv" "strings" @@ -804,17 +805,16 @@ var ( } ParallelTxFlag = cli.BoolFlag{ Name: "parallel", - Usage: "Enable the experimental parallel transaction execution mode (default = false)", + Usage: "Enable the experimental parallel transaction execution mode, only valid in full sync mode (default = false)", } ParallelTxNumFlag = cli.IntFlag{ Name: "parallel.num", - Usage: "Number of slot for transaction execution, only valid in parallel mode (default: CPUNum - 1)", - Value: core.ParallelExecNum, + Usage: "Number of slot for transaction execution, only valid in parallel mode (runtime calculated, no fixed default value)", } ParallelTxQueueSizeFlag = cli.IntFlag{ Name: "parallel.queuesize", - Usage: "Max number of Tx that can be queued to a slot, only valid in parallel mode", - Value: core.MaxPendingQueueSize, + Usage: "Max number of Tx that can be queued to a slot, only valid in parallel mode (advanced option)", + Value: 20, } // Init network @@ -1336,16 +1336,6 @@ func SetNodeConfig(ctx *cli.Context, cfg *node.Config) { if ctx.GlobalIsSet(InsecureUnlockAllowedFlag.Name) { cfg.InsecureUnlockAllowed = ctx.GlobalBool(InsecureUnlockAllowedFlag.Name) } - if ctx.GlobalIsSet(ParallelTxFlag.Name) { - core.ParallelTxMode = true - } - if ctx.GlobalIsSet(ParallelTxNumFlag.Name) { - core.ParallelExecNum = ctx.GlobalInt(ParallelTxNumFlag.Name) - } - if ctx.GlobalIsSet(ParallelTxQueueSizeFlag.Name) { - core.MaxPendingQueueSize = ctx.GlobalInt(ParallelTxQueueSizeFlag.Name) - } - } func setSmartCard(ctx *cli.Context, cfg *node.Config) { @@ -1666,6 +1656,32 @@ func SetEthConfig(ctx *cli.Context, stack *node.Node, cfg *ethconfig.Config) { if ctx.GlobalIsSet(RangeLimitFlag.Name) { cfg.RangeLimit = ctx.GlobalBool(RangeLimitFlag.Name) } + if ctx.GlobalIsSet(ParallelTxFlag.Name) { + cfg.ParallelTxMode = ctx.GlobalBool(ParallelTxFlag.Name) + // The best prallel num will be tuned later, we do a simple parallel num set here + numCpu := runtime.NumCPU() + var parallelNum int + if ctx.GlobalIsSet(ParallelTxNumFlag.Name) { + // first of all, we use "--parallel.num", but "--parallel.num 0" is not allowed + parallelNum = ctx.GlobalInt(ParallelTxNumFlag.Name) + if parallelNum < 1 { + parallelNum = 1 + } + } else if numCpu == 1 { + parallelNum = 1 // single CPU core + } else if numCpu < 10 { + parallelNum = numCpu - 1 + } else { + parallelNum = 8 // we found concurrency 8 is slightly better than 15 + } + cfg.ParallelTxNum = parallelNum + // set up queue size, it is an advanced option + if ctx.GlobalIsSet(ParallelTxQueueSizeFlag.Name) { + cfg.ParallelTxQueueSize = ctx.GlobalInt(ParallelTxQueueSizeFlag.Name) + } else { + cfg.ParallelTxQueueSize = 20 // default queue size, will be optimized + } + } // Read the value from the flag no matter if it's set or not. cfg.Preimages = ctx.GlobalBool(CachePreimagesFlag.Name) if cfg.NoPruning && !cfg.Preimages { diff --git a/core/blockchain.go b/core/blockchain.go index dcbecdbabe..96c607a0ef 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -79,7 +79,6 @@ var ( errInsertionInterrupted = errors.New("insertion is interrupted") errStateRootVerificationFailed = errors.New("state root verification failed") - ParallelTxMode = false // parallel transaction execution ) const ( @@ -241,12 +240,13 @@ type BlockChain struct { running int32 // 0 if chain is running, 1 when stopped procInterrupt int32 // interrupt signaler for block processing - engine consensus.Engine - prefetcher Prefetcher - validator Validator // Block and state validator interface - processor Processor // Block transaction processor interface - vmConfig vm.Config - pipeCommit bool + engine consensus.Engine + prefetcher Prefetcher + validator Validator // Block and state validator interface + processor Processor // Block transaction processor interface + vmConfig vm.Config + pipeCommit bool + parallelExecution bool shouldPreserve func(*types.Block) bool // Function used to determine whether should preserve the given block. terminateInsert func(common.Hash, uint64) bool // Testing hook used to terminate ancient receipt chain insertion. @@ -311,9 +311,6 @@ func NewBlockChain(db ethdb.Database, cacheConfig *CacheConfig, chainConfig *par bc.prefetcher = NewStatePrefetcher(chainConfig, bc, engine) bc.validator = NewBlockValidator(chainConfig, bc, engine) bc.processor = NewStateProcessor(chainConfig, bc, engine) - if ParallelTxMode { - bc.processor.InitParallelOnce() - } var err error bc.hc, err = NewHeaderChain(db, chainConfig, engine, bc.insertStopped) @@ -2105,12 +2102,10 @@ func (bc *BlockChain) insertChain(chain types.Blocks, verifySeals bool) (int, er if parent == nil { parent = bc.GetHeader(block.ParentHash(), block.NumberU64()-1) } - statedb, err := state.New(parent.Root, bc.stateCache, bc.snaps) if err != nil { return it.index, err } - bc.updateHighestVerifiedHeader(block.Header()) // Enable prefetching to pull in trie node paths while processing transactions @@ -2118,7 +2113,7 @@ func (bc *BlockChain) insertChain(chain types.Blocks, verifySeals bool) (int, er var followupInterrupt uint32 // For diff sync, it may fallback to full sync, so we still do prefetch // parallel mode has a pipeline, similar to this prefetch, to save CPU we disable this prefetch for parallel - if !ParallelTxMode { + if !bc.parallelExecution { if len(block.Transactions()) >= prefetchTxNumber { throwaway := statedb.Copy() go func(start time.Time, followup *types.Block, throwaway *state.StateDB, interrupt *uint32) { @@ -2132,16 +2127,7 @@ func (bc *BlockChain) insertChain(chain types.Blocks, verifySeals bool) (int, er statedb.EnablePipeCommit() } statedb.SetExpectedStateRoot(block.Root()) - - var receipts types.Receipts - var logs []*types.Log - var usedGas uint64 - if ParallelTxMode { - statedb, receipts, logs, usedGas, err = bc.processor.ProcessParallel(block, statedb, bc.vmConfig) - } else { - statedb, receipts, logs, usedGas, err = bc.processor.Process(block, statedb, bc.vmConfig) - } - + statedb, receipts, logs, usedGas, err := bc.processor.Process(block, statedb, bc.vmConfig) atomic.StoreUint32(&followupInterrupt, 1) activeState = statedb if err != nil { @@ -3118,3 +3104,16 @@ func EnablePersistDiff(limit uint64) BlockChainOption { return chain } } + +func EnableParallelProcessor(parallelNum int, queueSize int) BlockChainOption { + return func(chain *BlockChain) *BlockChain { + if chain.snaps == nil { + // disable parallel processor if snapshot is not enabled to avoid concurrent issue for SecureTrie + log.Info("parallel processor is not enabled since snapshot is not enabled") + return chain + } + chain.parallelExecution = true + chain.processor = NewParallelStateProcessor(chain.Config(), chain, chain.engine, parallelNum, queueSize) + return chain + } +} diff --git a/core/state/journal.go b/core/state/journal.go index 487e79a57d..b3a2956f75 100644 --- a/core/state/journal.go +++ b/core/state/journal.go @@ -145,7 +145,7 @@ func (ch createObjectChange) revert(s *StateDB) { if s.parallel.isSlotDB { delete(s.parallel.dirtiedStateObjectsInSlot, *ch.account) } else { - s.deleteStateObjectFromStateDB(*ch.account) + s.deleteStateObj(*ch.account) } delete(s.stateObjectsDirty, *ch.account) } diff --git a/core/state/state_object.go b/core/state/state_object.go index a809d2a565..36adf786d6 100644 --- a/core/state/state_object.go +++ b/core/state/state_object.go @@ -37,8 +37,6 @@ func (c Code) String() string { return string(c) //strings.Join(Disassemble(c), " ") } -type StorageKeys map[common.Hash]struct{} - type Storage map[common.Hash]common.Hash func (s Storage) String() (str string) { diff --git a/core/state/statedb.go b/core/state/statedb.go index b4e60811ee..3a4297ea2f 100644 --- a/core/state/statedb.go +++ b/core/state/statedb.go @@ -85,18 +85,17 @@ func (s *StateObjectSyncMap) StoreStateObject(addr common.Address, stateObject * s.Store(addr, stateObject) } -// loadStateObjectFromStateDB is the entry for loading state object from stateObjects in StateDB or stateObjects in parallel -func (s *StateDB) loadStateObjectFromStateDB(addr common.Address) (*StateObject, bool) { +// loadStateObj is the entry for loading state object from stateObjects in StateDB or stateObjects in parallel +func (s *StateDB) loadStateObj(addr common.Address) (*StateObject, bool) { if s.isParallel { return s.parallel.stateObjects.LoadStateObject(addr) - } else { - obj, ok := s.stateObjects[addr] - return obj, ok } + obj, ok := s.stateObjects[addr] + return obj, ok } -// storeStateObjectToStateDB is the entry for storing state object to stateObjects in StateDB or stateObjects in parallel -func (s *StateDB) storeStateObjectToStateDB(addr common.Address, stateObject *StateObject) { +// storeStateObj is the entry for storing state object to stateObjects in StateDB or stateObjects in parallel +func (s *StateDB) storeStateObj(addr common.Address, stateObject *StateObject) { if s.isParallel { s.parallel.stateObjects.Store(addr, stateObject) } else { @@ -104,8 +103,8 @@ func (s *StateDB) storeStateObjectToStateDB(addr common.Address, stateObject *St } } -// deleteStateObjectFromStateDB is the entry for deleting state object to stateObjects in StateDB or stateObjects in parallel -func (s *StateDB) deleteStateObjectFromStateDB(addr common.Address) { +// deleteStateObj is the entry for deleting state object to stateObjects in StateDB or stateObjects in parallel +func (s *StateDB) deleteStateObj(addr common.Address) { if s.isParallel { s.parallel.stateObjects.Delete(addr) } else { @@ -115,7 +114,7 @@ func (s *StateDB) deleteStateObjectFromStateDB(addr common.Address) { // For parallel mode only, keep the change list for later conflict detect type SlotChangeList struct { - TxIndex int // the tx index of change list + TxIndex int StateObjectSuicided map[common.Address]struct{} StateChangeSet map[common.Address]StateKeys BalanceChangeSet map[common.Address]struct{} @@ -131,7 +130,8 @@ type ParallelState struct { // stateObjects holds the state objects in the base slot db // the reason for using stateObjects instead of stateObjects on the outside is // we need a thread safe map to hold state objects since there are many slots will read - // state objects from this and in the same time we will change this when merging slot db to the base slot db + // state objects from it; + // And we will merge all the changes made by the concurrent slot into it. stateObjects *StateObjectSyncMap baseTxIndex int // slotDB is created base on this tx index. @@ -144,7 +144,7 @@ type ParallelState struct { stateReadsInSlot map[common.Address]StateKeys stateChangesInSlot map[common.Address]StateKeys // no need record value // Actions such as SetCode, Suicide will change address's state. - // Later call like Exist(), Empty(), HasSuicided() depond on the address's state. + // Later call like Exist(), Empty(), HasSuicided() depend on the address's state. addrStateReadsInSlot map[common.Address]struct{} addrStateChangesInSlot map[common.Address]struct{} stateObjectsSuicidedInSlot map[common.Address]struct{} @@ -241,10 +241,9 @@ func New(root common.Hash, db Database, snaps *snapshot.Tree) (*StateDB, error) return newStateDB(root, db, snaps) } -// NewSlotDB creates a new slot stateDB base on the provided stateDB. -// With parallel, each execute slot would have its own stateDB. +// NewSlotDB creates a new State DB based on the provided StateDB. +// With parallel, each execution slot would have its own StateDB. func NewSlotDB(db *StateDB, systemAddr common.Address, baseTxIndex int, keepSystem bool) *StateDB { - log.Debug("NewSlotDB", "baseTxIndex", baseTxIndex) slotDB := db.CopyForSlot() slotDB.originalRoot = db.originalRoot slotDB.parallel.baseTxIndex = baseTxIndex @@ -252,8 +251,12 @@ func NewSlotDB(db *StateDB, systemAddr common.Address, baseTxIndex int, keepSyst slotDB.parallel.systemAddressOpsCount = 0 slotDB.parallel.keepSystemAddressBalance = keepSystem - // clear the slotDB's validator's balance first - // for slotDB, systemAddr's value is the tx's gas fee + // All transactions will pay gas fee to the systemAddr at the end, this address is + // deemed to conflict, we handle it specially, clear it now and set it back to the main + // StateDB later; + // But there are transactions that will try to read systemAddr's balance, such as: + // https://bscscan.com/tx/0xcd69755be1d2f55af259441ff5ee2f312830b8539899e82488a21e85bc121a2a. + // It will trigger transaction redo and keepSystem will be marked as true. if !keepSystem { slotDB.SetBalance(systemAddr, big.NewInt(0)) } @@ -300,13 +303,12 @@ func (s *StateDB) getStateObjectFromStateObjects(addr common.Address) (*StateObj return obj, ok } } - return s.loadStateObjectFromStateDB(addr) + return s.loadStateObj(addr) } -// If the transaction execution is failed, keep its read list for conflict detect -// and discard its state changed, execept its own balance change. +// RevertSlotDB keep its read list for conflict detect and discard its state changes except its own balance change, +// if the transaction execution is reverted, func (s *StateDB) RevertSlotDB(from common.Address) { - log.Debug("RevertSlotDB", "addr", from, "txIndex", s.txIndex) s.parallel.stateObjectsSuicidedInSlot = make(map[common.Address]struct{}) s.parallel.stateChangesInSlot = make(map[common.Address]StateKeys) s.parallel.balanceChangesInSlot = make(map[common.Address]struct{}, 1) @@ -315,26 +317,25 @@ func (s *StateDB) RevertSlotDB(from common.Address) { s.parallel.nonceChangesInSlot = make(map[common.Address]struct{}) } -// PrepareForParallel prepares for state db to be used in parallel process. +// PrepareForParallel prepares for state db to be used in parallel execution mode. func (s *StateDB) PrepareForParallel() { s.isParallel = true s.parallel.stateObjects = &StateObjectSyncMap{} } -// MergeSlotDB is for Parallel TX, when the TX is finalized(dirty -> pending) -// A bit similar to StateDB.Copy(), -// mainly copy stateObjects, since slotDB has been finalized. -// return and keep the slot's change list for later conflict detect. +// MergeSlotDB is for Parallel execution mode, when the transaction has been +// finalized(dirty -> pending) on execution slot, the execution results should be +// merged back to the main StateDB. +// And it will return and keep the slot's change list for later conflict detect. func (s *StateDB) MergeSlotDB(slotDb *StateDB, slotReceipt *types.Receipt, txIndex int) SlotChangeList { - // receipt.Logs with unified log Index within a block - // align slotDB's logs Index to the block stateDB's logSize + // receipt.Logs use unified log index within a block + // align slotDB's log index to the block stateDB's logSize for _, l := range slotReceipt.Logs { l.Index += s.logSize } s.logSize += slotDb.logSize - // before merge, do validator reward first: AddBalance to consensus.SystemAddress - // object of SystemAddress is take care specially + // before merge, pay the gas fee first: AddBalance to consensus.SystemAddress systemAddress := slotDb.parallel.systemAddress if slotDb.parallel.keepSystemAddressBalance { s.SetBalance(systemAddress, slotDb.GetBalance(systemAddress)) @@ -356,26 +357,21 @@ func (s *StateDB) MergeSlotDB(slotDb *StateDB, slotReceipt *types.Receipt, txInd // stateObjects: KV, balance, nonce... dirtyObj, ok := slotDb.getStateObjectFromStateObjects(addr) if !ok { - panic(fmt.Sprintf("MergeSlotDB dirty object not exist! (txIndex: %d, addr: %s)", slotDb.txIndex, addr.String())) + log.Error("parallel merge, but dirty object not exist!", "txIndex:", slotDb.txIndex, "addr", addr) + continue } - mainObj, exist := s.loadStateObjectFromStateDB(addr) - - log.Debug("MergeSlotDB", "txIndex", slotDb.txIndex, "addr", addr, - "exist", exist, "dirtyObj.deleted", dirtyObj.deleted) + mainObj, exist := s.loadStateObj(addr) if !exist { // addr not exist on main DB, do ownership transfer dirtyObj.db = s dirtyObj.finalise(true) // true: prefetch on dispatcher - s.storeStateObjectToStateDB(addr, dirtyObj) + s.storeStateObj(addr, dirtyObj) delete(slotDb.parallel.dirtiedStateObjectsInSlot, addr) // transfer ownership } else { // addr already in main DB, do merge: balance, KV, code, State(create, suicide) // can not do copy or ownership transfer directly, since dirtyObj could have outdated // data(may be update within the conflict window) - // Do deepCopy a temporary *StateObject for safety, - // since slot could read the address, dispatch should avoid overwrite the StateObject directly - // otherwise, it could crash for: concurrent map iteration and map write var newMainObj *StateObject if _, created := slotDb.parallel.addrStateChangesInSlot[addr]; created { // there are 3 kinds of state change: @@ -397,21 +393,24 @@ func (s *StateDB) MergeSlotDB(slotDb *StateDB, slotReceipt *types.Receipt, txInd delete(s.snapStorage, addr) } } else { - // do merge: balance, KV, code... + // deepCopy a temporary *StateObject for safety, since slot could read the address, + // dispatch should avoid overwrite the StateObject directly otherwise, it could + // crash for: concurrent map iteration and map write newMainObj = mainObj.deepCopy(s) if _, balanced := slotDb.parallel.balanceChangesInSlot[addr]; balanced { - log.Debug("MergeSlotDB state object merge: state merge: balance", + log.Debug("merge state object: Balance", "newMainObj.Balance()", newMainObj.Balance(), "dirtyObj.Balance()", dirtyObj.Balance()) newMainObj.SetBalance(dirtyObj.Balance()) } if _, coded := slotDb.parallel.codeChangesInSlot[addr]; coded { - log.Debug("MergeSlotDB state object merge: state merge: code") + log.Debug("merge state object: Code") newMainObj.code = dirtyObj.code newMainObj.data.CodeHash = dirtyObj.data.CodeHash newMainObj.dirtyCode = true } if keys, stated := slotDb.parallel.stateChangesInSlot[addr]; stated { + log.Debug("merge state object: KV") newMainObj.MergeSlotObject(s.db, dirtyObj, keys) } // dirtyObj.Nonce() should not be less than newMainObj @@ -419,7 +418,7 @@ func (s *StateDB) MergeSlotDB(slotDb *StateDB, slotReceipt *types.Receipt, txInd } newMainObj.finalise(true) // true: prefetch on dispatcher // update the object - s.storeStateObjectToStateDB(addr, newMainObj) + s.storeStateObj(addr, newMainObj) } addressesToPrefetch = append(addressesToPrefetch, common.CopyBytes(addr[:])) // Copy needed for closure } @@ -440,62 +439,44 @@ func (s *StateDB) MergeSlotDB(slotDb *StateDB, slotReceipt *types.Receipt, txInd s.preimages[hash] = preimage } if s.accessList != nil { - // fixme: accessList is not enabled yet, should use merged rather than overwrite + // fixme: accessList is not enabled yet, but it should use merge rather than overwrite Copy s.accessList = slotDb.accessList.Copy() } if slotDb.snaps != nil { for k := range slotDb.snapDestructs { // There could be a race condition for parallel transaction execution - // One add balance 0 to an empty address, it will delete it(delete empty is enabled,). + // One transaction add balance 0 to an empty address, will delete it(delete empty is enabled). // While another concurrent transaction could add a none-zero balance to it, make it not empty // We fixed it by add a addr state read record for add balance 0 s.snapDestructs[k] = struct{}{} } - for k, v := range slotDb.snapAccounts { - s.snapAccounts[k] = v - } - for k, v := range slotDb.snapStorage { - temp := make(map[string][]byte) - for kk, vv := range v { - temp[kk] = vv - } - s.snapStorage[k] = temp - } + + // slotDb.snapAccounts should be empty, comment out and to be deleted later + // for k, v := range slotDb.snapAccounts { + // s.snapAccounts[k] = v + // } + // slotDb.snapStorage should be empty, comment out and to be deleted later + // for k, v := range slotDb.snapStorage { + // temp := make(map[string][]byte) + // for kk, vv := range v { + // temp[kk] = vv + // } + // s.snapStorage[k] = temp + // } } - // we have to create a new object to store change list for conflict detect, since - // StateDB could be reused and its elements could be overwritten + // to create a new object to store change list for conflict detect, + // since slot db reuse is disabled, we do not need to do copy. changeList := SlotChangeList{ TxIndex: txIndex, - StateObjectSuicided: make(map[common.Address]struct{}, len(slotDb.parallel.stateObjectsSuicidedInSlot)), - StateChangeSet: make(map[common.Address]StateKeys, len(slotDb.parallel.stateChangesInSlot)), - BalanceChangeSet: make(map[common.Address]struct{}, len(slotDb.parallel.balanceChangesInSlot)), - CodeChangeSet: make(map[common.Address]struct{}, len(slotDb.parallel.codeChangesInSlot)), - AddrStateChangeSet: make(map[common.Address]struct{}, len(slotDb.parallel.addrStateChangesInSlot)), - NonceChangeSet: make(map[common.Address]struct{}, len(slotDb.parallel.nonceChangesInSlot)), + StateObjectSuicided: slotDb.parallel.stateObjectsSuicidedInSlot, + StateChangeSet: slotDb.parallel.stateChangesInSlot, + BalanceChangeSet: slotDb.parallel.balanceChangesInSlot, + CodeChangeSet: slotDb.parallel.codeChangesInSlot, + AddrStateChangeSet: slotDb.parallel.addrStateChangesInSlot, + NonceChangeSet: slotDb.parallel.nonceChangesInSlot, } - for addr := range slotDb.parallel.stateObjectsSuicidedInSlot { - changeList.StateObjectSuicided[addr] = struct{}{} - } - for addr, storage := range slotDb.parallel.stateChangesInSlot { - changeList.StateChangeSet[addr] = storage - } - for addr := range slotDb.parallel.balanceChangesInSlot { - changeList.BalanceChangeSet[addr] = struct{}{} - } - for addr := range slotDb.parallel.codeChangesInSlot { - changeList.CodeChangeSet[addr] = struct{}{} - } - for addr := range slotDb.parallel.addrStateChangesInSlot { - changeList.AddrStateChangeSet[addr] = struct{}{} - } - for addr := range slotDb.parallel.nonceChangesInSlot { - changeList.NonceChangeSet[addr] = struct{}{} - } - - // the slot DB's is valid now, move baseTxIndex forward, since it could be reused. - slotDb.parallel.baseTxIndex = txIndex return changeList } @@ -503,9 +484,6 @@ func (s *StateDB) MergeSlotDB(slotDb *StateDB, slotReceipt *types.Receipt, txInd // state trie concurrently while the state is mutated so that when we reach the // commit phase, most of the needed data is already hot. func (s *StateDB) StartPrefetcher(namespace string) { - if s.parallel.isSlotDB { - log.Warn("StartPrefetcher should not be called by slot DB") - } s.prefetcherLock.Lock() defer s.prefetcherLock.Unlock() if s.prefetcher != nil { @@ -520,9 +498,6 @@ func (s *StateDB) StartPrefetcher(namespace string) { // StopPrefetcher terminates a running prefetcher and reports any leftover stats // from the gathered metrics. func (s *StateDB) StopPrefetcher() { - if s.parallel.isSlotDB { - log.Warn("StopPrefetcher should not be called by slot DB") - } s.prefetcherLock.Lock() defer s.prefetcherLock.Unlock() if s.prefetcher != nil { @@ -713,6 +688,12 @@ func (s *StateDB) StateReadsInSlot() map[common.Address]StateKeys { func (s *StateDB) BalanceReadsInSlot() map[common.Address]struct{} { return s.parallel.balanceReadsInSlot } + +// For most of the transactions, systemAddressOpsCount should be 2: +// one for SetBalance(0) on NewSlotDB() +// the other is for AddBalance(GasFee) at the end. +// (systemAddressOpsCount > 2) means the transaction tries to access systemAddress, in +// this case, we should redo and keep its balance on NewSlotDB() func (s *StateDB) SystemAddressRedo() bool { return s.parallel.systemAddressOpsCount > 2 } @@ -854,11 +835,9 @@ func (s *StateDB) HasSuicided(addr common.Address) bool { // AddBalance adds amount to the account associated with addr. func (s *StateDB) AddBalance(addr common.Address, amount *big.Int) { if s.parallel.isSlotDB { - // just in case other tx creates this account, - // we will miss this if we only add this account when found if amount.Sign() != 0 { s.parallel.balanceChangesInSlot[addr] = struct{}{} - // add balance will perform a read operation first, empty object will be deleted + // add balance will perform a read operation first s.parallel.balanceReadsInSlot[addr] = struct{}{} } else { // if amount == 0, no balance change, but there is still an empty check. @@ -877,20 +856,16 @@ func (s *StateDB) AddBalance(addr common.Address, amount *big.Int) { newStateObject := stateObject.deepCopy(s) newStateObject.AddBalance(amount) s.parallel.dirtiedStateObjectsInSlot[addr] = newStateObject - } else { - stateObject.AddBalance(amount) + return } - } else { - stateObject.AddBalance(amount) } + stateObject.AddBalance(amount) } } // SubBalance subtracts amount from the account associated with addr. func (s *StateDB) SubBalance(addr common.Address, amount *big.Int) { if s.parallel.isSlotDB { - // just in case other tx creates this account, - // we will miss this if we only add this account when found if amount.Sign() != 0 { s.parallel.balanceChangesInSlot[addr] = struct{}{} // unlike add, sub 0 balance will not touch empty object @@ -908,12 +883,10 @@ func (s *StateDB) SubBalance(addr common.Address, amount *big.Int) { newStateObject := stateObject.deepCopy(s) newStateObject.SubBalance(amount) s.parallel.dirtiedStateObjectsInSlot[addr] = newStateObject - } else { - stateObject.SubBalance(amount) + return } - } else { - stateObject.SubBalance(amount) } + stateObject.SubBalance(amount) } } @@ -921,20 +894,19 @@ func (s *StateDB) SetBalance(addr common.Address, amount *big.Int) { stateObject := s.GetOrNewStateObject(addr) if stateObject != nil { if s.parallel.isSlotDB { + s.parallel.balanceChangesInSlot[addr] = struct{}{} + if addr == s.parallel.systemAddress { + s.parallel.systemAddressOpsCount++ + } + if _, ok := s.parallel.dirtiedStateObjectsInSlot[addr]; !ok { newStateObject := stateObject.deepCopy(s) newStateObject.SetBalance(amount) s.parallel.dirtiedStateObjectsInSlot[addr] = newStateObject - } else { - stateObject.SetBalance(amount) + return } - s.parallel.balanceChangesInSlot[addr] = struct{}{} - if addr == s.parallel.systemAddress { - s.parallel.systemAddressOpsCount++ - } - } else { - stateObject.SetBalance(amount) } + stateObject.SetBalance(amount) } } @@ -958,12 +930,10 @@ func (s *StateDB) SetNonce(addr common.Address, nonce uint64) { newStateObject := stateObject.deepCopy(s) newStateObject.SetNonce(nonce) s.parallel.dirtiedStateObjectsInSlot[addr] = newStateObject - } else { - stateObject.SetNonce(nonce) + return } - } else { - stateObject.SetNonce(nonce) } + stateObject.SetNonce(nonce) } } @@ -971,18 +941,16 @@ func (s *StateDB) SetCode(addr common.Address, code []byte) { stateObject := s.GetOrNewStateObject(addr) if stateObject != nil { if s.parallel.isSlotDB { + s.parallel.codeChangesInSlot[addr] = struct{}{} + if _, ok := s.parallel.dirtiedStateObjectsInSlot[addr]; !ok { newStateObject := stateObject.deepCopy(s) newStateObject.SetCode(crypto.Keccak256Hash(code), code) s.parallel.dirtiedStateObjectsInSlot[addr] = newStateObject - } else { - stateObject.SetCode(crypto.Keccak256Hash(code), code) + return } - - s.parallel.codeChangesInSlot[addr] = struct{}{} - } else { - stateObject.SetCode(crypto.Keccak256Hash(code), code) } + stateObject.SetCode(crypto.Keccak256Hash(code), code) } } @@ -999,21 +967,20 @@ func (s *StateDB) SetState(addr common.Address, key, value common.Hash) { return } } - if _, ok := s.parallel.dirtiedStateObjectsInSlot[addr]; !ok { - newStateObject := stateObject.deepCopy(s) - newStateObject.SetState(s.db, key, value) - s.parallel.dirtiedStateObjectsInSlot[addr] = newStateObject - } else { - stateObject.SetState(s.db, key, value) - } if s.parallel.stateChangesInSlot[addr] == nil { s.parallel.stateChangesInSlot[addr] = make(StateKeys, defaultNumOfSlots) } s.parallel.stateChangesInSlot[addr][key] = struct{}{} - } else { - stateObject.SetState(s.db, key, value) + + if _, ok := s.parallel.dirtiedStateObjectsInSlot[addr]; !ok { + newStateObject := stateObject.deepCopy(s) + newStateObject.SetState(s.db, key, value) + s.parallel.dirtiedStateObjectsInSlot[addr] = newStateObject + return + } } + stateObject.SetState(s.db, key, value) } } @@ -1033,10 +1000,6 @@ func (s *StateDB) SetStorage(addr common.Address, storage map[common.Hash]common // getStateObject will return a non-nil account after Suicide. func (s *StateDB) Suicide(addr common.Address) bool { stateObject := s.getStateObject(addr) - - if s.parallel.isSlotDB { - s.parallel.addrStateReadsInSlot[addr] = struct{}{} - } if stateObject == nil { return false } @@ -1056,15 +1019,12 @@ func (s *StateDB) Suicide(addr common.Address) bool { newStateObject.markSuicided() newStateObject.data.Balance = new(big.Int) s.parallel.dirtiedStateObjectsInSlot[addr] = newStateObject - } else { - stateObject.markSuicided() - stateObject.data.Balance = new(big.Int) + return true } - } else { - stateObject.markSuicided() - stateObject.data.Balance = new(big.Int) } + stateObject.markSuicided() + stateObject.data.Balance = new(big.Int) return true } @@ -1266,7 +1226,7 @@ func (s *StateDB) SetStateObject(object *StateObject) { if s.parallel.isSlotDB { s.parallel.dirtiedStateObjectsInSlot[object.Address()] = object } else { - s.storeStateObjectToStateDB(object.Address(), object) + s.storeStateObj(object.Address(), object) } } @@ -1283,8 +1243,8 @@ func (s *StateDB) GetOrNewStateObject(addr common.Address) *StateObject { // the given address, it is overwritten and returned as the second return value. func (s *StateDB) createObject(addr common.Address) (newobj, prev *StateObject) { if s.parallel.isSlotDB { - s.parallel.addrStateReadsInSlot[addr] = struct{}{} // fixme: may not necessary - s.parallel.addrStateChangesInSlot[addr] = struct{}{} // address created. + s.parallel.addrStateReadsInSlot[addr] = struct{}{} // will try to get the previous object. + s.parallel.addrStateChangesInSlot[addr] = struct{}{} } prev = s.getDeletedStateObject(addr) // Note, prev might have been deleted, we need that! @@ -1293,8 +1253,8 @@ func (s *StateDB) createObject(addr common.Address) (newobj, prev *StateObject) if s.snap != nil && prev != nil { _, prevdestruct = s.snapDestructs[prev.address] if !prevdestruct { - // createObject for deleted object is ok, - // it will destroy the previous trie node and update with the new object on block commit + // createObject for deleted object will destroy the previous trie node first + // and update the trie tree with the new object on block commit. s.snapDestructs[prev.address] = struct{}{} } } @@ -1325,10 +1285,10 @@ func (s *StateDB) createObject(addr common.Address) (newobj, prev *StateObject) func (s *StateDB) CreateAccount(addr common.Address) { newObj, prev := s.createObject(addr) if prev != nil { - newObj.setBalance(prev.data.Balance) // this read + newObj.setBalance(prev.data.Balance) } if s.parallel.isSlotDB { - s.parallel.balanceReadsInSlot[addr] = struct{}{} + s.parallel.balanceReadsInSlot[addr] = struct{}{} // read the balance of previous object s.parallel.dirtiedStateObjectsInSlot[addr] = newObj } } @@ -1366,10 +1326,6 @@ func (s *StateDB) ForEachStorage(addr common.Address, cb func(key, value common. // Snapshots of the copied state cannot be applied to the copy. func (s *StateDB) Copy() *StateDB { // Copy all the basic fields, initialize the memory ones - parallel := ParallelState{ - isSlotDB: false, - } - state := &StateDB{ db: s.db, trie: s.db.CopyTrie(s.trie), @@ -1382,7 +1338,7 @@ func (s *StateDB) Copy() *StateDB { preimages: make(map[common.Hash][]byte, len(s.preimages)), journal: newJournal(), hasher: crypto.NewKeccakState(), - parallel: parallel, + parallel: ParallelState{}, } // Copy the dirty states, logs, and preimages for addr := range s.journal.dirties { @@ -1394,7 +1350,7 @@ func (s *StateDB) Copy() *StateDB { // Even though the original object is dirty, we are not copying the journal, // so we need to make sure that anyside effect the journal would have caused // during a commit (or similar op) is already applied to the copy. - state.storeStateObjectToStateDB(addr, object.deepCopy(state)) + state.storeStateObj(addr, object.deepCopy(state)) state.stateObjectsDirty[addr] = struct{}{} // Mark the copy dirty to force internal (code/state) commits state.stateObjectsPending[addr] = struct{}{} // Mark the copy pending to force external (account) commits @@ -1406,14 +1362,14 @@ func (s *StateDB) Copy() *StateDB { for addr := range s.stateObjectsPending { if _, exist := state.getStateObjectFromStateObjects(addr); !exist { object, _ := s.getStateObjectFromStateObjects(addr) - state.storeStateObjectToStateDB(addr, object.deepCopy(state)) + state.storeStateObj(addr, object.deepCopy(state)) } state.stateObjectsPending[addr] = struct{}{} } for addr := range s.stateObjectsDirty { if _, exist := state.getStateObjectFromStateObjects(addr); !exist { object, _ := s.getStateObjectFromStateObjects(addr) - state.storeStateObjectToStateDB(addr, object.deepCopy(state)) + state.storeStateObj(addr, object.deepCopy(state)) } state.stateObjectsDirty[addr] = struct{}{} } @@ -1471,32 +1427,32 @@ func (s *StateDB) Copy() *StateDB { return state } +// Copy all the basic fields, initialize the memory ones func (s *StateDB) CopyForSlot() *StateDB { - // Copy all the basic fields, initialize the memory ones parallel := ParallelState{ - // Share base slot db's stateObjects + // use base(dispatcher) slot db's stateObjects. // It is a SyncMap, only readable to slot, not writable stateObjects: s.parallel.stateObjects, - stateObjectsSuicidedInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + stateObjectsSuicidedInSlot: make(map[common.Address]struct{}, 10), codeReadsInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - codeChangesInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + codeChangesInSlot: make(map[common.Address]struct{}, 10), stateChangesInSlot: make(map[common.Address]StateKeys, defaultNumOfSlots), stateReadsInSlot: make(map[common.Address]StateKeys, defaultNumOfSlots), balanceChangesInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), balanceReadsInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), addrStateReadsInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - addrStateChangesInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), - nonceChangesInSlot: make(map[common.Address]struct{}, defaultNumOfSlots), + addrStateChangesInSlot: make(map[common.Address]struct{}, 10), + nonceChangesInSlot: make(map[common.Address]struct{}, 10), isSlotDB: true, dirtiedStateObjectsInSlot: make(map[common.Address]*StateObject, defaultNumOfSlots), } state := &StateDB{ db: s.db, trie: s.db.CopyTrie(s.trie), - stateObjects: make(map[common.Address]*StateObject, defaultNumOfSlots), + stateObjects: make(map[common.Address]*StateObject), // replaced by parallel.stateObjects in parallel mode stateObjectsPending: make(map[common.Address]struct{}, defaultNumOfSlots), stateObjectsDirty: make(map[common.Address]struct{}, defaultNumOfSlots), - refund: s.refund, + refund: s.refund, // should be 0 logs: make(map[common.Hash][]*types.Log, defaultNumOfSlots), logSize: 0, preimages: make(map[common.Hash][]byte, len(s.preimages)), @@ -1525,6 +1481,7 @@ func (s *StateDB) CopyForSlot() *StateDB { for k, v := range s.snapDestructs { state.snapDestructs[k] = v } + // state.snapAccounts = make(map[common.Address][]byte) for k, v := range s.snapAccounts { state.snapAccounts[k] = v diff --git a/core/state_processor.go b/core/state_processor.go index 14cf549f65..38fe88ef99 100644 --- a/core/state_processor.go +++ b/core/state_processor.go @@ -24,7 +24,6 @@ import ( "math/rand" "runtime" "sync" - "sync/atomic" "time" "github.com/ethereum/go-ethereum/common" @@ -50,9 +49,6 @@ const ( farDiffLayerTimeout = 2 ) -var MaxPendingQueueSize = 20 // parallel slot's maximum number of pending Txs -var ParallelExecNum = runtime.NumCPU() - 1 // leave a CPU to dispatcher - // StateProcessor is a basic Processor, which takes care of transitioning // state from one point to another. // @@ -61,17 +57,8 @@ type StateProcessor struct { config *params.ChainConfig // Chain configuration options bc *BlockChain // Canonical block chain engine consensus.Engine // Consensus engine used for block rewards - - // add for parallel execute - paraInitialized int32 - paraTxResultChan chan *ParallelTxResult // to notify dispatcher that a tx is done - slotState []*SlotState // idle, or pending messages - mergedTxIndex int // the latest finalized tx index - debugErrorRedoNum int - debugConflictRedoNum int } -// NewStateProcessor initialises a new StateProcessor. func NewStateProcessor(config *params.ChainConfig, bc *BlockChain, engine consensus.Engine) *StateProcessor { return &StateProcessor{ config: config, @@ -80,6 +67,28 @@ func NewStateProcessor(config *params.ChainConfig, bc *BlockChain, engine consen } } +// add for parallel executions +type ParallelStateProcessor struct { + StateProcessor + parallelNum int // leave a CPU to dispatcher + queueSize int // parallel slot's maximum number of pending Txs + txResultChan chan *ParallelTxResult // to notify dispatcher that a tx is done + slotState []*SlotState // idle, or pending messages + mergedTxIndex int // the latest finalized tx index + debugErrorRedoNum int + debugConflictRedoNum int +} + +func NewParallelStateProcessor(config *params.ChainConfig, bc *BlockChain, engine consensus.Engine, parallelNum int, queueSize int) *ParallelStateProcessor { + processor := &ParallelStateProcessor{ + StateProcessor: *NewStateProcessor(config, bc, engine), + parallelNum: parallelNum, + queueSize: queueSize, + } + processor.init() + return processor +} + type LightStateProcessor struct { check int64 StateProcessor @@ -389,65 +398,55 @@ type SlotState struct { pendingTxReqList []*ParallelTxRequest // maintained by dispatcher for dispatch policy mergedChangeList []state.SlotChangeList slotdbChan chan *state.StateDB // dispatch will create and send this slotDB to slot - // conflict check uses conflict window - // conflict check will check all state changes from (cfWindowStart + 1) to the previous Tx } type ParallelTxResult struct { - redo bool // for redo, dispatch will wait new tx result - updateSlotDB bool // for redo and pending tx quest, slot needs new slotDB, - keepSystem bool // for redo, should keep system address's balance - txIndex int + updateSlotDB bool // for redo and pending tx quest, slot needs new slotDB, + keepSystem bool // for redo, should keep system address's balance slotIndex int // slot index err error // to describe error message? - tx *types.Transaction txReq *ParallelTxRequest receipt *types.Receipt - slotDB *state.StateDB + slotDB *state.StateDB // if updated, it is not equal to txReq.slotDB } type ParallelTxRequest struct { - txIndex int - tx *types.Transaction - slotDB *state.StateDB - gp *GasPool - msg types.Message - block *types.Block - vmConfig vm.Config - bloomProcessors *AsyncReceiptBloomGenerator - usedGas *uint64 - waitTxChan chan int // "int" represents the tx index - curTxChan chan int // "int" represents the tx index + txIndex int + tx *types.Transaction + slotDB *state.StateDB + gasLimit uint64 + msg types.Message + block *types.Block + vmConfig vm.Config + bloomProcessor *AsyncReceiptBloomGenerator + usedGas *uint64 + waitTxChan chan struct{} + curTxChan chan struct{} } -func (p *StateProcessor) InitParallelOnce() { - // to create and start the execution slot goroutines - if !atomic.CompareAndSwapInt32(&p.paraInitialized, 0, 1) { // not swapped means already initialized. - return - } - log.Info("Parallel execution mode is used and initialized", "Parallel Num", ParallelExecNum) - p.paraTxResultChan = make(chan *ParallelTxResult, ParallelExecNum) // fixme: use blocked chan? - p.slotState = make([]*SlotState, ParallelExecNum) - - wg := sync.WaitGroup{} // make sure all goroutines are created and started - for i := 0; i < ParallelExecNum; i++ { - p.slotState[i] = new(SlotState) - p.slotState[i].slotdbChan = make(chan *state.StateDB, 1) - p.slotState[i].pendingTxReqChan = make(chan *ParallelTxRequest, MaxPendingQueueSize) - - wg.Add(1) +// to create and start the execution slot goroutines +func (p *ParallelStateProcessor) init() { + log.Info("Parallel execution mode is enabled", "Parallel Num", p.parallelNum, + "CPUNum", runtime.NumCPU(), + "QueueSize", p.queueSize) + p.txResultChan = make(chan *ParallelTxResult, p.parallelNum) + p.slotState = make([]*SlotState, p.parallelNum) + + for i := 0; i < p.parallelNum; i++ { + p.slotState[i] = &SlotState{ + slotdbChan: make(chan *state.StateDB, 1), + pendingTxReqChan: make(chan *ParallelTxRequest, p.queueSize), + } // start the slot's goroutine go func(slotIndex int) { - wg.Done() p.runSlotLoop(slotIndex) // this loop will be permanent live - log.Error("runSlotLoop exit!", "Slot", slotIndex) }(i) } - wg.Wait() } -// if any state in readDb is updated in changeList, then it has state conflict -func (p *StateProcessor) hasStateConflict(readDb *state.StateDB, changeList state.SlotChangeList) bool { +// conflict check uses conflict window, it will check all state changes from (cfWindowStart + 1) +// to the previous Tx, if any state in readDb is updated in changeList, then it is conflicted +func (p *ParallelStateProcessor) hasStateConflict(readDb *state.StateDB, changeList state.SlotChangeList) bool { // check KV change reads := readDb.StateReadsInSlot() writes := changeList.StateChangeSet @@ -525,7 +524,7 @@ func (p *StateProcessor) hasStateConflict(readDb *state.StateDB, changeList stat // for parallel execute, we put contracts of same address in a slot, // since these txs probably would have conflicts -func (p *StateProcessor) queueSameToAddress(txReq *ParallelTxRequest) bool { +func (p *ParallelStateProcessor) queueSameToAddress(txReq *ParallelTxRequest) bool { txToAddr := txReq.tx.To() // To() == nil means contract creation, no same To address if txToAddr == nil { @@ -560,7 +559,7 @@ func (p *StateProcessor) queueSameToAddress(txReq *ParallelTxRequest) bool { // for parallel execute, we put contracts of same address in a slot, // since these txs probably would have conflicts -func (p *StateProcessor) queueSameFromAddress(txReq *ParallelTxRequest) bool { +func (p *ParallelStateProcessor) queueSameFromAddress(txReq *ParallelTxRequest) bool { txFromAddr := txReq.msg.From() for i, slot := range p.slotState { if slot.tailTxReq == nil { // this slot is idle @@ -586,7 +585,7 @@ func (p *StateProcessor) queueSameFromAddress(txReq *ParallelTxRequest) bool { } // if there is idle slot, dispatch the msg to the first idle slot -func (p *StateProcessor) dispatchToIdleSlot(statedb *state.StateDB, txReq *ParallelTxRequest) bool { +func (p *ParallelStateProcessor) dispatchToIdleSlot(statedb *state.StateDB, txReq *ParallelTxRequest) bool { for i, slot := range p.slotState { if slot.tailTxReq == nil { if len(slot.mergedChangeList) == 0 { @@ -604,14 +603,14 @@ func (p *StateProcessor) dispatchToIdleSlot(statedb *state.StateDB, txReq *Paral } // wait until the next Tx is executed and its result is merged to the main stateDB -func (p *StateProcessor) waitUntilNextTxDone(statedb *state.StateDB) *ParallelTxResult { +func (p *ParallelStateProcessor) waitUntilNextTxDone(statedb *state.StateDB, gp *GasPool) *ParallelTxResult { var result *ParallelTxResult for { - result = <-p.paraTxResultChan - // slot may request new slotDB, if it think its slotDB is outdated + result = <-p.txResultChan + // slot may request new slotDB, if slotDB is outdated // such as: // tx in pending tx request, previous tx in same queue is likely "damaged" the slotDB - // tx redo for confict + // tx redo for conflict // tx stage 1 failed, nonce out of order... if result.updateSlotDB { // the target slot is waiting for new slotDB @@ -620,15 +619,17 @@ func (p *StateProcessor) waitUntilNextTxDone(statedb *state.StateDB) *ParallelTx slotState.slotdbChan <- slotDB continue } - if result.redo { - // wait result of redo - continue - } // ok, the tx result is valid and can be merged break } + + if err := gp.SubGas(result.receipt.GasUsed); err != nil { + log.Error("gas limit reached", "block", result.txReq.block.Number(), + "txIndex", result.txReq.txIndex, "GasUsed", result.receipt.GasUsed, "gp.Gas", gp.Gas()) + } + resultSlotIndex := result.slotIndex - resultTxIndex := result.txIndex + resultTxIndex := result.txReq.txIndex resultSlotState := p.slotState[resultSlotIndex] resultSlotState.pendingTxReqList = resultSlotState.pendingTxReqList[1:] if resultSlotState.tailTxReq.txIndex == resultTxIndex { @@ -643,29 +644,28 @@ func (p *StateProcessor) waitUntilNextTxDone(statedb *state.StateDB) *ParallelTx resultSlotState.mergedChangeList = append(resultSlotState.mergedChangeList, changeList) if resultTxIndex != p.mergedTxIndex+1 { - log.Warn("ProcessParallel tx result out of order", "resultTxIndex", resultTxIndex, + log.Error("ProcessParallel tx result out of order", "resultTxIndex", resultTxIndex, "p.mergedTxIndex", p.mergedTxIndex) - panic("ProcessParallel tx result out of order") } p.mergedTxIndex = resultTxIndex // notify the following Tx, it is merged, - // fixme: what if no wait or next tx is in same slot? - result.txReq.curTxChan <- resultTxIndex + // todo(optimize): if next tx is in same slot, it do not need to wait; save this channel cost. + close(result.txReq.curTxChan) return result } -func (p *StateProcessor) execInParallelSlot(slotIndex int, txReq *ParallelTxRequest) *ParallelTxResult { +func (p *ParallelStateProcessor) execInSlot(slotIndex int, txReq *ParallelTxRequest) *ParallelTxResult { txIndex := txReq.txIndex tx := txReq.tx slotDB := txReq.slotDB - gp := txReq.gp // goroutine unsafe + slotGasLimit := txReq.gasLimit // not accurate, but it is ok for block import. msg := txReq.msg block := txReq.block header := block.Header() cfg := txReq.vmConfig - bloomProcessors := txReq.bloomProcessors + bloomProcessor := txReq.bloomProcessor - blockContext := NewEVMBlockContext(header, p.bc, nil) // fixme: share blockContext within a block? + blockContext := NewEVMBlockContext(header, p.bc, nil) // can share blockContext within a block for efficiency vmenv := vm.NewEVM(blockContext, vm.TxContext{}, slotDB, p.config, cfg) var receipt *types.Receipt @@ -676,7 +676,6 @@ func (p *StateProcessor) execInParallelSlot(slotIndex int, txReq *ParallelTxRequ slotDB.Prepare(tx.Hash(), block.Hash(), txIndex) log.Debug("exec In Slot", "Slot", slotIndex, "txIndex", txIndex, "slotDB.baseTxIndex", slotDB.BaseTxIndex()) - slotGasLimit := gp.Gas() gpSlot := new(GasPool).AddGas(slotGasLimit) // each slot would use its own gas pool, and will do gaslimit check later evm, result, err = applyTransactionStageExecution(msg, gpSlot, slotDB, vmenv) log.Debug("Stage Execution done", "Slot", slotIndex, "txIndex", txIndex, "slotDB.baseTxIndex", slotDB.BaseTxIndex()) @@ -684,58 +683,43 @@ func (p *StateProcessor) execInParallelSlot(slotIndex int, txReq *ParallelTxRequ // wait until the previous tx is finalized. if txReq.waitTxChan != nil { log.Debug("Stage wait previous Tx done", "Slot", slotIndex, "txIndex", txIndex) - waitTxIndex := <-txReq.waitTxChan - if waitTxIndex != txIndex-1 { - log.Error("Stage wait tx index mismatch", "expect", txIndex-1, "actual", waitTxIndex) - panic(fmt.Sprintf("wait tx index mismatch expect:%d, actual:%d", txIndex-1, waitTxIndex)) - } + <-txReq.waitTxChan // close the channel } - // in parallel, tx can run into trouble - // for example: err="nonce too high" - // in this case, we will do re-run. + // in parallel mode, tx can run into trouble, for example: err="nonce too high" + // in these cases, we will wait and re-run. if err != nil { p.debugErrorRedoNum++ log.Debug("Stage Execution err", "Slot", slotIndex, "txIndex", txIndex, "current slotDB.baseTxIndex", slotDB.BaseTxIndex(), "err", err) redoResult := &ParallelTxResult{ - redo: true, updateSlotDB: true, - txIndex: txIndex, slotIndex: slotIndex, - tx: tx, txReq: txReq, receipt: receipt, err: err, } - p.paraTxResultChan <- redoResult + p.txResultChan <- redoResult slotDB = <-p.slotState[slotIndex].slotdbChan slotDB.Prepare(tx.Hash(), block.Hash(), txIndex) - // vmenv.Reset(vm.TxContext{}, slotDB) log.Debug("Stage Execution get new slotdb to redo", "Slot", slotIndex, "txIndex", txIndex, "new slotDB.baseTxIndex", slotDB.BaseTxIndex()) - slotGasLimit = gp.Gas() gpSlot = new(GasPool).AddGas(slotGasLimit) evm, result, err = applyTransactionStageExecution(msg, gpSlot, slotDB, vmenv) if err != nil { - panic(fmt.Sprintf("Stage Execution redo, error %v", err)) + log.Error("Stage Execution redo, error", err) } } - // fixme: - // parallel mode can not precheck, - // precheck should be replace by postCheck when previous Tx is finalized - // do conflict detect hasConflict := false systemAddrConflict := false - log.Debug("Stage Execution done, do conflict check", "Slot", slotIndex, "txIndex", txIndex) if slotDB.SystemAddressRedo() { hasConflict = true systemAddrConflict = true } else { - for index := 0; index < ParallelExecNum; index++ { + for index := 0; index < p.parallelNum; index++ { if index == slotIndex { continue } @@ -743,7 +727,6 @@ func (p *StateProcessor) execInParallelSlot(slotIndex int, txReq *ParallelTxRequ // check all finalizedDb from current slot's for _, changeList := range p.slotState[index].mergedChangeList { if changeList.TxIndex <= slotDB.BaseTxIndex() { - // log.Debug("skip finalized DB which is out of the conflict window", "finDb.txIndex", finDb.txIndex, "slotDB.baseTxIndex", slotDB.baseTxIndex) continue } if p.hasStateConflict(slotDB, changeList) { @@ -764,25 +747,20 @@ func (p *StateProcessor) execInParallelSlot(slotIndex int, txReq *ParallelTxRequ p.debugConflictRedoNum++ // re-run should not have conflict, since it has the latest world state. redoResult := &ParallelTxResult{ - redo: true, updateSlotDB: true, keepSystem: systemAddrConflict, - txIndex: txIndex, slotIndex: slotIndex, - tx: tx, txReq: txReq, receipt: receipt, err: err, } - p.paraTxResultChan <- redoResult + p.txResultChan <- redoResult slotDB = <-p.slotState[slotIndex].slotdbChan slotDB.Prepare(tx.Hash(), block.Hash(), txIndex) - // vmenv.Reset(vm.TxContext{}, slotDB) - slotGasLimit = gp.Gas() gpSlot = new(GasPool).AddGas(slotGasLimit) evm, result, err = applyTransactionStageExecution(msg, gpSlot, slotDB, vmenv) if err != nil { - panic(fmt.Sprintf("Stage Execution conflict redo, error %v", err)) + log.Error("Stage Execution conflict redo, error", err) } } @@ -791,18 +769,12 @@ func (p *StateProcessor) execInParallelSlot(slotIndex int, txReq *ParallelTxRequ if gasConsumed != result.UsedGas { log.Error("gasConsumed != result.UsedGas mismatch", "gasConsumed", gasConsumed, "result.UsedGas", result.UsedGas) - panic(fmt.Sprintf("gas consume mismatch, consumed:%d, result.UsedGas:%d", gasConsumed, result.UsedGas)) - } - - if err := gp.SubGas(gasConsumed); err != nil { - log.Error("gas limit reached", "gasConsumed", gasConsumed, "gp", gp.Gas()) - panic(fmt.Sprintf("gas limit reached, gasConsumed:%d, gp.Gas():%d", gasConsumed, gp.Gas())) } log.Debug("ok to finalize this TX", "Slot", slotIndex, "txIndex", txIndex, "result.UsedGas", result.UsedGas, "txReq.usedGas", *txReq.usedGas) // ok, time to do finalize, stage2 should not be parallel - receipt, err = applyTransactionStageFinalization(evm, result, msg, p.config, slotDB, header, tx, txReq.usedGas, bloomProcessors) + receipt, err = applyTransactionStageFinalization(evm, result, msg, p.config, slotDB, header, tx, txReq.usedGas, bloomProcessor) if result.Failed() { // if Tx is reverted, all its state change will be discarded @@ -811,11 +783,8 @@ func (p *StateProcessor) execInParallelSlot(slotIndex int, txReq *ParallelTxRequ } return &ParallelTxResult{ - redo: false, updateSlotDB: false, - txIndex: txIndex, slotIndex: slotIndex, - tx: tx, txReq: txReq, receipt: receipt, slotDB: slotDB, @@ -823,10 +792,9 @@ func (p *StateProcessor) execInParallelSlot(slotIndex int, txReq *ParallelTxRequ } } -func (p *StateProcessor) runSlotLoop(slotIndex int) { +func (p *ParallelStateProcessor) runSlotLoop(slotIndex int) { curSlot := p.slotState[slotIndex] for { - // log.Info("parallel slot waiting", "Slot", slotIndex) // wait for new TxReq txReq := <-curSlot.pendingTxReqChan // receive a dispatched message @@ -839,22 +807,21 @@ func (p *StateProcessor) runSlotLoop(slotIndex int) { // it is better to create a new SlotDB, since COW is used. if txReq.slotDB == nil { result := &ParallelTxResult{ - redo: false, updateSlotDB: true, slotIndex: slotIndex, err: nil, } - p.paraTxResultChan <- result + p.txResultChan <- result txReq.slotDB = <-curSlot.slotdbChan } - result := p.execInParallelSlot(slotIndex, txReq) + result := p.execInSlot(slotIndex, txReq) log.Debug("SlotLoop the TxReq is done", "Slot", slotIndex, "err", result.err) - p.paraTxResultChan <- result + p.txResultChan <- result } } // clear slot state for each block. -func (p *StateProcessor) resetParallelState(txNum int, statedb *state.StateDB) { +func (p *ParallelStateProcessor) resetState(txNum int, statedb *state.StateDB) { if txNum == 0 { return } @@ -871,6 +838,114 @@ func (p *StateProcessor) resetParallelState(txNum int, statedb *state.StateDB) { } } +// Implement BEP-130: Parallel Transaction Execution. +func (p *ParallelStateProcessor) Process(block *types.Block, statedb *state.StateDB, cfg vm.Config) (*state.StateDB, types.Receipts, []*types.Log, uint64, error) { + var ( + usedGas = new(uint64) + header = block.Header() + gp = new(GasPool).AddGas(block.GasLimit()) + ) + var receipts = make([]*types.Receipt, 0) + txNum := len(block.Transactions()) + p.resetState(txNum, statedb) + + // Iterate over and process the individual transactions + posa, isPoSA := p.engine.(consensus.PoSA) + commonTxs := make([]*types.Transaction, 0, txNum) + // usually do have two tx, one for validator set contract, another for system reward contract. + systemTxs := make([]*types.Transaction, 0, 2) + + signer, _, bloomProcessor := p.preExecute(block, statedb, cfg, true) + var waitTxChan, curTxChan chan struct{} + for i, tx := range block.Transactions() { + if isPoSA { + if isSystemTx, err := posa.IsSystemTransaction(tx, block.Header()); err != nil { + return statedb, nil, nil, 0, err + } else if isSystemTx { + systemTxs = append(systemTxs, tx) + continue + } + } + + // can be moved it into slot for efficiency, but signer is not concurrent safe + msg, err := tx.AsMessage(signer) + if err != nil { + return statedb, nil, nil, 0, err + } + + // parallel start, wrap an exec message, which will be dispatched to a slot + waitTxChan = curTxChan // can be nil, if this is the tx of first batch, otherwise, it is previous Tx's wait channel + curTxChan = make(chan struct{}, 1) + + txReq := &ParallelTxRequest{ + txIndex: i, + tx: tx, + slotDB: nil, + gasLimit: gp.Gas(), + msg: msg, + block: block, + vmConfig: cfg, + bloomProcessor: bloomProcessor, + usedGas: usedGas, + waitTxChan: waitTxChan, + curTxChan: curTxChan, + } + + // to optimize the for { for {} } loop code style? it is ok right now. + for { + if p.queueSameFromAddress(txReq) { + break + } + + if p.queueSameToAddress(txReq) { + break + } + // if idle slot available, just dispatch and process next tx. + if p.dispatchToIdleSlot(statedb, txReq) { + break + } + log.Debug("ProcessParallel no slot available, wait", "txIndex", txReq.txIndex) + // no idle slot, wait until a tx is executed and merged. + result := p.waitUntilNextTxDone(statedb, gp) + + // update tx result + if result.err != nil { + log.Warn("ProcessParallel a failed tx", "resultSlotIndex", result.slotIndex, + "resultTxIndex", result.txReq.txIndex, "result.err", result.err) + return statedb, nil, nil, 0, fmt.Errorf("could not apply tx %d [%v]: %w", result.txReq.txIndex, result.txReq.tx.Hash().Hex(), result.err) + } + + commonTxs = append(commonTxs, result.txReq.tx) + receipts = append(receipts, result.receipt) + } + } + + // wait until all tx request are done + for len(commonTxs)+len(systemTxs) < txNum { + result := p.waitUntilNextTxDone(statedb, gp) + // update tx result + if result.err != nil { + log.Warn("ProcessParallel a failed tx", "resultSlotIndex", result.slotIndex, + "resultTxIndex", result.txReq.txIndex, "result.err", result.err) + return statedb, nil, nil, 0, fmt.Errorf("could not apply tx %d [%v]: %w", result.txReq.txIndex, result.txReq.tx.Hash().Hex(), result.err) + } + commonTxs = append(commonTxs, result.txReq.tx) + receipts = append(receipts, result.receipt) + } + + // len(commonTxs) could be 0, such as: https://bscscan.com/block/14580486 + if len(commonTxs) > 0 { + log.Info("ProcessParallel tx all done", "block", header.Number, "usedGas", *usedGas, + "txNum", txNum, + "len(commonTxs)", len(commonTxs), + "errorNum", p.debugErrorRedoNum, + "conflictNum", p.debugConflictRedoNum, + "redoRate(%)", 100*(p.debugErrorRedoNum+p.debugConflictRedoNum)/len(commonTxs)) + } + allLogs, err := p.postExecute(block, statedb, &commonTxs, &receipts, &systemTxs, usedGas, bloomProcessor) + return statedb, receipts, allLogs, *usedGas, err +} + // Before transactions are executed, do shared preparation for Process() & ProcessParallel() func (p *StateProcessor) preExecute(block *types.Block, statedb *state.StateDB, cfg vm.Config, parallel bool) (types.Signer, *vm.EVM, *AsyncReceiptBloomGenerator) { signer := types.MakeSigner(p.bc.chainConfig, block.Number()) @@ -882,25 +957,25 @@ func (p *StateProcessor) preExecute(block *types.Block, statedb *state.StateDB, // Handle upgrade build-in system contract code systemcontracts.UpgradeBuildInSystemContract(p.config, block.Number(), statedb) - blockContext := NewEVMBlockContext(block.Header(), p.bc, nil) // with parallel mode, vmenv will be created inside of slot var vmenv *vm.EVM if !parallel { + blockContext := NewEVMBlockContext(block.Header(), p.bc, nil) vmenv = vm.NewEVM(blockContext, vm.TxContext{}, statedb, p.config, cfg) } // initialise bloom processors - bloomProcessors := NewAsyncReceiptBloomGenerator(len(block.Transactions())) + bloomProcessor := NewAsyncReceiptBloomGenerator(len(block.Transactions())) statedb.MarkFullProcessed() - return signer, vmenv, bloomProcessors + return signer, vmenv, bloomProcessor } func (p *StateProcessor) postExecute(block *types.Block, statedb *state.StateDB, commonTxs *[]*types.Transaction, - receipts *[]*types.Receipt, systemTxs *[]*types.Transaction, usedGas *uint64, bloomProcessors *AsyncReceiptBloomGenerator) ([]*types.Log, error) { - var allLogs []*types.Log + receipts *[]*types.Receipt, systemTxs *[]*types.Transaction, usedGas *uint64, bloomProcessor *AsyncReceiptBloomGenerator) ([]*types.Log, error) { + allLogs := make([]*types.Log, 0, len(*receipts)) - bloomProcessors.Close() + bloomProcessor.Close() // Finalize the block, applying any consensus engine specific extras (e.g. block rewards) err := p.engine.Finalize(p.bc, block.Header(), statedb, commonTxs, block.Uncles(), receipts, systemTxs, usedGas) @@ -934,7 +1009,7 @@ func (p *StateProcessor) Process(block *types.Block, statedb *state.StateDB, cfg // usually do have two tx, one for validator set contract, another for system reward contract. systemTxs := make([]*types.Transaction, 0, 2) - signer, vmenv, bloomProcessors := p.preExecute(block, statedb, cfg, false) + signer, vmenv, bloomProcessor := p.preExecute(block, statedb, cfg, false) for i, tx := range block.Transactions() { if isPoSA { if isSystemTx, err := posa.IsSystemTransaction(tx, block.Header()); err != nil { @@ -950,7 +1025,7 @@ func (p *StateProcessor) Process(block *types.Block, statedb *state.StateDB, cfg return statedb, nil, nil, 0, err } statedb.Prepare(tx.Hash(), block.Hash(), i) - receipt, err := applyTransaction(msg, p.config, p.bc, nil, gp, statedb, header, tx, usedGas, vmenv, bloomProcessors) + receipt, err := applyTransaction(msg, p.config, p.bc, nil, gp, statedb, header, tx, usedGas, vmenv, bloomProcessor) if err != nil { return statedb, nil, nil, 0, fmt.Errorf("could not apply tx %d [%v]: %w", i, tx.Hash().Hex(), err) } @@ -959,112 +1034,7 @@ func (p *StateProcessor) Process(block *types.Block, statedb *state.StateDB, cfg receipts = append(receipts, receipt) } - allLogs, err := p.postExecute(block, statedb, &commonTxs, &receipts, &systemTxs, usedGas, bloomProcessors) - return statedb, receipts, allLogs, *usedGas, err -} - -func (p *StateProcessor) ProcessParallel(block *types.Block, statedb *state.StateDB, cfg vm.Config) (*state.StateDB, types.Receipts, []*types.Log, uint64, error) { - var ( - usedGas = new(uint64) - header = block.Header() - gp = new(GasPool).AddGas(block.GasLimit()) - ) - var receipts = make([]*types.Receipt, 0) - txNum := len(block.Transactions()) - p.resetParallelState(txNum, statedb) - - // Iterate over and process the individual transactions - posa, isPoSA := p.engine.(consensus.PoSA) - commonTxs := make([]*types.Transaction, 0, txNum) - // usually do have two tx, one for validator set contract, another for system reward contract. - systemTxs := make([]*types.Transaction, 0, 2) - - signer, _, bloomProcessors := p.preExecute(block, statedb, cfg, true) - var waitTxChan, curTxChan chan int - for i, tx := range block.Transactions() { - if isPoSA { - if isSystemTx, err := posa.IsSystemTransaction(tx, block.Header()); err != nil { - return statedb, nil, nil, 0, err - } else if isSystemTx { - systemTxs = append(systemTxs, tx) - continue - } - } - - msg, err := tx.AsMessage(signer) // fixme: move it into slot. - if err != nil { - return statedb, nil, nil, 0, err - } - - // parallel start, wrap an exec message, which will be dispatched to a slot - waitTxChan = curTxChan // can be nil, if this is the tx of first batch, otherwise, it is previous Tx's wait channel - curTxChan = make(chan int, 1) - - txReq := &ParallelTxRequest{ - txIndex: i, - tx: tx, - slotDB: nil, - gp: gp, - msg: msg, - block: block, - vmConfig: cfg, - bloomProcessors: bloomProcessors, - usedGas: usedGas, - waitTxChan: waitTxChan, - curTxChan: curTxChan, - } - - // fixme: to optimize the for { for {} } loop code style - for { - if p.queueSameToAddress(txReq) { - break - } - if p.queueSameFromAddress(txReq) { - break - } - // if idle slot available, just dispatch and process next tx. - if p.dispatchToIdleSlot(statedb, txReq) { - // log.Info("ProcessParallel dispatch to idle slot", "txIndex", txReq.txIndex) - break - } - log.Debug("ProcessParallel no slot available, wait", "txIndex", txReq.txIndex) - // no idle slot, wait until a tx is executed and merged. - result := p.waitUntilNextTxDone(statedb) - - // update tx result - if result.err != nil { - log.Warn("ProcessParallel a failed tx", "resultSlotIndex", result.slotIndex, - "resultTxIndex", result.txIndex, "result.err", result.err) - return statedb, nil, nil, 0, fmt.Errorf("could not apply tx %d [%v]: %w", result.txIndex, result.tx.Hash().Hex(), result.err) - } - commonTxs = append(commonTxs, result.tx) - receipts = append(receipts, result.receipt) - } - } - - // wait until all tx request are done - for len(commonTxs)+len(systemTxs) < txNum { - result := p.waitUntilNextTxDone(statedb) - // update tx result - if result.err != nil { - log.Warn("ProcessParallel a failed tx", "resultSlotIndex", result.slotIndex, - "resultTxIndex", result.txIndex, "result.err", result.err) - return statedb, nil, nil, 0, fmt.Errorf("could not apply tx %d [%v]: %w", result.txIndex, result.tx.Hash().Hex(), result.err) - } - commonTxs = append(commonTxs, result.tx) - receipts = append(receipts, result.receipt) - } - - // len(commonTxs) could be 0, such as: https://bscscan.com/block/14580486 - if len(commonTxs) > 0 { - log.Info("ProcessParallel tx all done", "block", header.Number, "usedGas", *usedGas, - "txNum", txNum, - "len(commonTxs)", len(commonTxs), - "errorNum", p.debugErrorRedoNum, - "conflictNum", p.debugConflictRedoNum, - "redoRate(%)", 100*(p.debugErrorRedoNum+p.debugConflictRedoNum)/len(commonTxs)) - } - allLogs, err := p.postExecute(block, statedb, &commonTxs, &receipts, &systemTxs, usedGas, bloomProcessors) + allLogs, err := p.postExecute(block, statedb, &commonTxs, &receipts, &systemTxs, usedGas, bloomProcessor) return statedb, receipts, allLogs, *usedGas, err } diff --git a/core/types.go b/core/types.go index 0038ce916b..5ed4817e68 100644 --- a/core/types.go +++ b/core/types.go @@ -48,8 +48,4 @@ type Processor interface { // the transaction messages using the statedb and applying any rewards to both // the processor (coinbase) and any included uncles. Process(block *types.Block, statedb *state.StateDB, cfg vm.Config) (*state.StateDB, types.Receipts, []*types.Log, uint64, error) - - // Implement BEP-130: Parallel Transaction Execution. - InitParallelOnce() - ProcessParallel(block *types.Block, statedb *state.StateDB, cfg vm.Config) (*state.StateDB, types.Receipts, []*types.Log, uint64, error) } diff --git a/eth/backend.go b/eth/backend.go index ab93006437..8551b57eaa 100644 --- a/eth/backend.go +++ b/eth/backend.go @@ -202,6 +202,8 @@ func New(stack *node.Node, config *ethconfig.Config) (*Ethereum, error) { bcOps := make([]core.BlockChainOption, 0) if config.DiffSync { bcOps = append(bcOps, core.EnableLightProcessor) + } else if config.ParallelTxMode { + bcOps = append(bcOps, core.EnableParallelProcessor(config.ParallelTxNum, config.ParallelTxQueueSize)) } if config.PipeCommit { bcOps = append(bcOps, core.EnablePipelineCommit) diff --git a/eth/ethconfig/config.go b/eth/ethconfig/config.go index 09baad1e1c..94998acd6c 100644 --- a/eth/ethconfig/config.go +++ b/eth/ethconfig/config.go @@ -135,10 +135,13 @@ type Config struct { NoPruning bool // Whether to disable pruning and flush everything to disk DirectBroadcast bool - DisableSnapProtocol bool //Whether disable snap protocol + DisableSnapProtocol bool // Whether disable snap protocol DiffSync bool // Whether support diff sync PipeCommit bool RangeLimit bool + ParallelTxMode bool // Whether to execute transaction in parallel mode when do full sync + ParallelTxNum int // Number of slot for transaction execution + ParallelTxQueueSize int // Max number of Tx that can be queued to a slot TxLookupLimit uint64 `toml:",omitempty"` // The maximum number of blocks from head whose tx indices are reserved.