fixes for the IPA testnet

upgrade to latest go-verkle

update go-verkle to get more fixes

simplify code by removing all stateless references (#25)

fix verkle proof test by enforcing values alignment to 32 bytes

remove unneeded KZG tag

fix the stateless test

Move AccessWitness into StateDB (#27)

* move AccessWitness into StateDB

* set Accesses in TxContext constructor

* Ensures that a statedb is initialized with a witness

* copy AccessWitness in StateDB.Copy.  use copied state in miner worker.commit.

* remove redundant line

Co-authored-by: Guillaume Ballet <3272758+gballet@users.noreply.github.com>

Fix contract creation issue
This commit is contained in:
Guillaume Ballet 2021-11-04 20:45:43 +01:00
parent 719bf47354
commit 9f9c03a94c
12 changed files with 59 additions and 190 deletions

View File

@ -1983,7 +1983,7 @@ func TestGolangBindings(t *testing.T) {
t.Fatalf("failed to tidy Go module file: %v\n%s", err, out)
}
// Test the entire package and report any failures
cmd := exec.Command(gocmd, "test", "-tags=bignum_kilic", "-v", "-count", "1")
cmd := exec.Command(gocmd, "test", "-v", "-count", "1")
cmd.Dir = pkg
if out, err := cmd.CombinedOutput(); err != nil {
t.Fatalf("failed to run binding test: %v\n%s", err, out)

View File

@ -34,6 +34,7 @@ import (
"github.com/ethereum/go-ethereum/params"
"github.com/ethereum/go-ethereum/rlp"
"github.com/ethereum/go-ethereum/trie"
"github.com/ethereum/go-ethereum/trie/utils"
"golang.org/x/crypto/sha3"
)
@ -592,6 +593,7 @@ func (ethash *Ethash) Prepare(chain consensus.ChainHeaderReader, header *types.H
// Finalize implements consensus.Engine, accumulating the block and uncle rewards,
// setting the final state on the header
func (ethash *Ethash) Finalize(chain consensus.ChainHeaderReader, header *types.Header, state *state.StateDB, txs []*types.Transaction, uncles []*types.Header) {
// TODO(gballet) aw should be part of StateDB
// Accumulate any block and uncle rewards and commit the final state root
accumulateRewards(chain.Config(), state, header, uncles)
header.Root = state.IntermediateRoot(chain.Config().IsEIP158(header.Number))
@ -666,4 +668,6 @@ func accumulateRewards(config *params.ChainConfig, state *state.StateDB, header
reward.Add(reward, r)
}
state.AddBalance(header.Coinbase, reward)
coinbase := utils.GetTreeKeyBalance(header.Coinbase.Bytes())
state.Witness().TouchAddress(coinbase, state.GetBalance(header.Coinbase).Bytes())
}

View File

@ -1598,17 +1598,7 @@ func (bc *BlockChain) insertChain(chain types.Blocks, verifySeals, setHead bool)
receipts types.Receipts
logs []*types.Log
)
if len(block.Header().VerkleProof) == 0 {
receipts, logs, _, usedGas, err = bc.processor.Process(block, statedb, bc.vmConfig)
} else {
var leaves map[common.Hash]common.Hash
leaves, err = trie.DeserializeAndVerifyVerkleProof(block.Header().VerkleProof)
if err != nil {
return it.index, err
}
statedb.SetStateless(leaves)
receipts, logs, usedGas, err = bc.processor.ProcessStateless(block, statedb, bc.vmConfig, leaves)
}
receipts, logs, _, usedGas, err = bc.processor.Process(block, statedb, bc.vmConfig)
if err != nil {
bc.reportBlock(block, receipts, err)
atomic.StoreUint32(&followupInterrupt, 1)

View File

@ -44,7 +44,6 @@ type BlockGen struct {
txs []*types.Transaction
receipts []*types.Receipt
uncles []*types.Header
witness *types.AccessWitness
config *params.ChainConfig
engine consensus.Engine
@ -110,10 +109,10 @@ func (b *BlockGen) AddTxWithChain(bc *BlockChain, tx *types.Transaction) {
panic(err)
}
if accesses != nil {
if b.witness != nil {
b.witness.Merge(accesses)
if b.statedb.Witness() != nil {
b.statedb.Witness().Merge(accesses)
} else {
b.witness = accesses
b.statedb.SetWitness(accesses)
}
}
b.txs = append(b.txs, tx)
@ -300,7 +299,7 @@ func GenerateVerkleChain(config *params.ChainConfig, parent *types.Block, engine
blocks, receipts := make(types.Blocks, n), make([]types.Receipts, n)
chainreader := &fakeChainReader{config: config}
genblock := func(i int, parent *types.Block, statedb *state.StateDB) (*types.Block, types.Receipts) {
b := &BlockGen{i: i, chain: blocks, parent: parent, statedb: statedb, config: config, engine: engine, witness: types.NewAccessWitness()}
b := &BlockGen{i: i, chain: blocks, parent: parent, statedb: statedb, config: config, engine: engine}
b.header = makeHeader(chainreader, parent, statedb, b.engine)
// Mutate the state and block according to any hard-fork specs
@ -343,7 +342,8 @@ func GenerateVerkleChain(config *params.ChainConfig, parent *types.Block, engine
// before building the proof. Ultimately, node
// resolution can be done with a prefetcher or
// from GetCommitmentsAlongPath.
keys := b.witness.Keys()
keys := statedb.Witness().Keys()
for _, key := range keys {
out, err := vtr.TryGet(key)
if err != nil {
@ -354,7 +354,7 @@ func GenerateVerkleChain(config *params.ChainConfig, parent *types.Block, engine
}
}
vtr.Hash()
_, err := vtr.ProveAndSerialize(keys, b.witness.KeyVals())
_, err := vtr.ProveAndSerialize(keys, statedb.Witness().KeyVals())
//block.SetVerkleProof(p)
if err != nil {
panic(err)

View File

@ -102,8 +102,7 @@ type StateDB struct {
// Per-transaction access list
accessList *accessList
// Stateless locations for this block
stateless map[common.Hash]common.Hash
witness *types.AccessWitness
// Journal of state modifications. This is the backbone of
// Snapshot and RevertToSnapshot.
@ -149,6 +148,7 @@ func New(root common.Hash, db Database, snaps *snapshot.Tree) (*StateDB, error)
journal: newJournal(),
accessList: newAccessList(),
hasher: crypto.NewKeccakState(),
witness: types.NewAccessWitness(),
}
if sdb.snaps == nil && tr.IsVerkle() {
sdb.snaps, err = snapshot.New(db.TrieDB().DiskDB(), db.TrieDB(), 1, root, false, true, false, true)
@ -166,6 +166,14 @@ func New(root common.Hash, db Database, snaps *snapshot.Tree) (*StateDB, error)
return sdb, nil
}
func (s *StateDB) Witness() *types.AccessWitness {
return s.witness
}
func (s *StateDB) SetWitness(aw *types.AccessWitness) {
s.witness = aw
}
// 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.
@ -423,12 +431,6 @@ func (s *StateDB) SetCode(addr common.Address, code []byte) {
}
}
// SetStateless sets the vales recovered from the execution of a stateless
// block.
func (s *StateDB) SetStateless(leaves map[common.Hash]common.Hash) {
s.stateless = leaves
}
func (s *StateDB) SetState(addr common.Address, key, value common.Hash) {
stateObject := s.GetOrNewStateObject(addr)
if stateObject != nil {
@ -470,46 +472,6 @@ func (s *StateDB) Suicide(addr common.Address) bool {
// Setting, updating & deleting state object methods.
//
func (s *StateDB) updateStatelessStateObject(obj *stateObject) {
addr := obj.Address()
var (
ok bool
n common.Hash
v common.Hash
b common.Hash
cs common.Hash
ch common.Hash
)
versionKey := common.BytesToHash(trieUtils.GetTreeKeyVersion(addr[:]))
if v, ok = s.stateless[versionKey]; ok {
nonceKey := common.BytesToHash(trieUtils.GetTreeKeyNonce(addr[:]))
if n, ok = s.stateless[nonceKey]; ok {
balanceKey := common.BytesToHash(trieUtils.GetTreeKeyBalance(addr[:]))
if b, ok = s.stateless[balanceKey]; ok {
codeHashKey := common.BytesToHash(trieUtils.GetTreeKeyCodeKeccak(addr[:]))
if _, ok = s.stateless[codeHashKey]; ok {
v[0] = byte(0)
binary.BigEndian.PutUint64(n[:], obj.data.Nonce)
copy(ch[:], obj.data.CodeHash[:])
copy(b[:], obj.data.Balance.Bytes())
binary.BigEndian.PutUint64(cs[:], uint64(len(obj.code)))
// TODO(@gballet) stateless tree update
// i.e. perform a "delta" update on all
// commitments. go-verkle currently has
// no support for these.
}
}
}
}
if !ok {
s.setError(fmt.Errorf("updateStatelessStateObject (%x) missing", addr[:]))
}
}
// updateStateObject writes the given object to the trie.
func (s *StateDB) updateStateObject(obj *stateObject) {
// Track the amount of time wasted on updating the account from the trie
@ -519,12 +481,6 @@ func (s *StateDB) updateStateObject(obj *stateObject) {
// Encode the account and update the account trie
addr := obj.Address()
// bypass the snapshot and writing to tree if in stateless mode
if s.stateless != nil {
s.updateStatelessStateObject(obj)
return
}
if err := s.trie.TryUpdateAccount(addr[:], &obj.data); err != nil {
s.setError(fmt.Errorf("updateStateObject (%x) error: %w", addr[:], err))
}
@ -534,6 +490,16 @@ func (s *StateDB) updateStateObject(obj *stateObject) {
if err := s.trie.TryUpdate(trieUtils.GetTreeKeyCodeSize(addr[:]), cs); err != nil {
s.setError(fmt.Errorf("updateStateObject (%x) error: %w", addr[:], err))
}
if obj.dirtyCode {
if chunks, err := trie.ChunkifyCode(addr, obj.code); err == nil {
for i := range chunks {
s.trie.TryUpdate(trieUtils.GetTreeKeyCodeChunk(addr[:], uint256.NewInt(uint64(i))), chunks[i][:])
}
} else {
s.setError(err)
}
}
}
// If state snapshotting is active, cache the data til commit. Note, this
@ -545,21 +511,12 @@ func (s *StateDB) updateStateObject(obj *stateObject) {
}
}
func (s *StateDB) deleteStatelessStateObject(obj *stateObject) {
// unsupported
panic("not currently supported")
}
// deleteStateObject removes the given object from the state trie.
func (s *StateDB) deleteStateObject(obj *stateObject) {
// Track the amount of time wasted on deleting the account from the trie
if metrics.EnabledExpensive {
defer func(start time.Time) { s.AccountUpdates += time.Since(start) }(time.Now())
}
if s.stateless != nil {
s.deleteStatelessStateObject(obj)
return
}
// Delete the account from the trie
if !s.trie.IsVerkle() {
@ -586,48 +543,6 @@ func (s *StateDB) getStateObject(addr common.Address) *stateObject {
return nil
}
func (s *StateDB) getStatelessDeletedStateObject(addr common.Address) *stateObject {
// Check that it is present in the witness, if running
// in stateless execution mode.
chunk := trieUtils.GetTreeKeyNonce(addr[:])
nb, ok := s.stateless[common.BytesToHash(chunk)]
if !ok {
log.Error("Failed to decode state object", "addr", addr)
s.setError(fmt.Errorf("could not find nonce chunk in proof: %x", chunk))
// TODO(gballet) remove after debug, and check the issue is found
panic("inivalid chunk")
return nil
}
chunk = trieUtils.GetTreeKeyBalance(addr[:])
bb, ok := s.stateless[common.BytesToHash(chunk)]
if !ok {
log.Error("Failed to decode state object", "addr", addr)
s.setError(fmt.Errorf("could not find balance chunk in proof: %x", chunk))
// TODO(gballet) remove after debug, and check the issue is found
panic("inivalid chunk")
return nil
}
chunk = trieUtils.GetTreeKeyCodeKeccak(addr[:])
cb, ok := s.stateless[common.BytesToHash(chunk)]
if !ok {
// Assume that this is an externally-owned account, and that
// the code has not been accessed.
// TODO(gballet) write this down, just like deletions, so
// that an error can be triggered if trying to access the
// account code.
copy(cb[:], emptyCodeHash)
}
data := &types.StateAccount{
Nonce: binary.BigEndian.Uint64(nb[:8]),
Balance: big.NewInt(0).SetBytes(bb[:]),
CodeHash: cb[:],
}
// Insert into the live set
obj := newObject(s, addr, *data)
s.setStateObject(obj)
return obj
}
// getDeletedStateObject is similar to getStateObject, but instead of returning
// nil for a deleted state object, it returns the actual object with the deleted
// flag set. This is needed by the state journal to revert to the correct s-
@ -642,10 +557,6 @@ func (s *StateDB) getDeletedStateObject(addr common.Address) *stateObject {
data *types.StateAccount
err error
)
// if executing statelessly, bypass the snapshot and the db.
if s.stateless != nil {
return s.getStatelessDeletedStateObject(addr)
}
if s.snap != nil {
if metrics.EnabledExpensive {
defer func(start time.Time) { s.SnapshotAccountReads += time.Since(start) }(time.Now())
@ -802,6 +713,7 @@ func (s *StateDB) Copy() *StateDB {
preimages: make(map[common.Hash][]byte, len(s.preimages)),
journal: newJournal(),
hasher: crypto.NewKeccakState(),
witness: s.witness.Copy(),
}
// Copy the dirty states, logs, and preimages
for addr := range s.journal.dirties {
@ -852,13 +764,6 @@ func (s *StateDB) Copy() *StateDB {
// to not blow up if we ever decide copy it in the middle of a transaction
state.accessList = s.accessList.Copy()
if s.stateless != nil {
state.stateless = make(map[common.Hash]common.Hash, len(s.stateless))
for addr, value := range s.stateless {
state.stateless[addr] = value
}
}
// If there's a prefetcher running, make an inactive copy of it that can
// only access data but does not actively preload (since the user will not
// know that they need to explicitly terminate an active copy).
@ -1088,8 +993,8 @@ func (s *StateDB) Commit(deleteEmptyObjects bool) (common.Hash, error) {
if obj.code != nil && obj.dirtyCode {
if s.trie.IsVerkle() {
if chunks, err := trie.ChunkifyCode(addr, obj.code); err == nil {
for i, chunk := range chunks {
s.trie.TryUpdate(trieUtils.GetTreeKeyCodeChunk(addr[:], uint256.NewInt(uint64(i))), chunk[:])
for i := range chunks {
s.trie.TryUpdate(trieUtils.GetTreeKeyCodeChunk(addr[:], uint256.NewInt(uint64(i))), chunks[i][:])
}
} else {
s.setError(err)

View File

@ -94,43 +94,6 @@ func (p *StateProcessor) Process(block *types.Block, statedb *state.StateDB, cfg
return receipts, allLogs, accesses, *usedGas, nil
}
func (p *StateProcessor) ProcessStateless(block *types.Block, statedb *state.StateDB, cfg vm.Config, leaves map[common.Hash]common.Hash) (types.Receipts, []*types.Log, uint64, error) {
var (
receipts types.Receipts
usedGas = new(uint64)
header = block.Header()
blockHash = block.Hash()
blockNumber = block.Number()
allLogs []*types.Log
gp = new(GasPool).AddGas(block.GasLimit())
)
// 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)
}
blockContext := NewEVMBlockContext(header, p.bc, nil)
blockContext.StatelessAccesses = leaves
vmenv := vm.NewEVM(blockContext, vm.TxContext{}, statedb, p.config, cfg)
// Iterate over and process the individual transactions
for i, tx := range block.Transactions() {
msg, err := tx.AsMessage(types.MakeSigner(p.config, header.Number), header.BaseFee)
if err != nil {
return nil, nil, 0, fmt.Errorf("could not apply tx %d [%v]: %w", i, tx.Hash().Hex(), err)
}
statedb.Prepare(tx.Hash(), i)
receipt, _, err := applyTransaction(msg, p.config, p.bc, nil, gp, statedb, blockNumber, blockHash, tx, usedGas, vmenv)
if err != nil {
return nil, nil, 0, fmt.Errorf("could not apply tx %d [%v]: %w", i, tx.Hash().Hex(), err)
}
receipts = append(receipts, receipt)
allLogs = append(allLogs, receipt.Logs...)
}
// Finalize the block, applying any consensus engine specific extras (e.g. block rewards)
p.engine.Finalize(p.bc, header, statedb, block.Transactions(), block.Uncles())
return receipts, allLogs, *usedGas, nil
}
func applyTransaction(msg types.Message, config *params.ChainConfig, bc ChainContext, author *common.Address, gp *GasPool, statedb *state.StateDB, blockNumber *big.Int, blockHash common.Hash, tx *types.Transaction, usedGas *uint64, evm *vm.EVM) (*types.Receipt, *types.AccessWitness, error) {
// Create a new context to be used in the EVM environment.
txContext := NewEVMTxContext(msg)

View File

@ -17,7 +17,6 @@
package core
import (
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/state"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/core/vm"
@ -49,6 +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) (types.Receipts, []*types.Log, *types.AccessWitness, uint64, error)
ProcessStateless(block *types.Block, statedb *state.StateDB, cfg vm.Config, accesses map[common.Hash]common.Hash) (types.Receipts, []*types.Log, uint64, error)
}

View File

@ -129,3 +129,15 @@ func (aw *AccessWitness) Keys() [][]byte {
func (aw *AccessWitness) KeyVals() map[common.Hash][]byte {
return aw.Chunks
}
func (aw *AccessWitness) Copy() *AccessWitness {
naw := &AccessWitness{
Branches: make(map[[31]byte]struct{}),
Chunks: make(map[common.Hash][]byte),
Undefined: make(map[common.Hash]struct{}),
}
naw.Merge(aw)
return naw
}

View File

@ -78,8 +78,6 @@ type BlockContext struct {
Time *big.Int // Provides information for TIME
Difficulty *big.Int // Provides information for DIFFICULTY
BaseFee *big.Int // Provides information for BASEFEE
StatelessAccesses map[common.Hash]common.Hash
}
// TxContext provides the EVM with information about a transaction.

View File

@ -74,6 +74,9 @@ type StateDB interface {
AddPreimage(common.Hash, []byte)
ForEachStorage(common.Address, func(common.Hash, common.Hash) bool) error
Witness() *types.AccessWitness
SetWitness(*types.AccessWitness)
}
// CallContext provides a basic interface for the EVM calling conventions. The EVM

View File

@ -91,9 +91,6 @@ type environment struct {
header *types.Header
txs []*types.Transaction
receipts []*types.Receipt
// list of all locations touched by the transactions in this block
witness *types.AccessWitness
}
// task contains all information for consensus engine sealing and result submitting.
@ -699,7 +696,6 @@ func (w *worker) makeCurrent(parent *types.Block, header *types.Header) error {
family: mapset.NewSet(),
uncles: mapset.NewSet(),
header: header,
witness: types.NewAccessWitness(),
}
// when 08 is processed ancestors contain 07 (quick block)
for _, ancestor := range w.chain.GetBlocksFromHash(parent.Hash(), 7) {
@ -849,10 +845,10 @@ func (w *worker) commitTransactions(txs *types.TransactionsByPriceAndNonce, coin
w.current.state.Prepare(tx.Hash(), w.current.tcount)
logs, accs, err := w.commitTransaction(tx, coinbase)
if w.current.witness == nil {
w.current.witness = accs
if w.current.state.Witness() == nil {
w.current.state.SetWitness(accs)
} else {
w.current.witness.Merge(accs)
w.current.state.Witness().Merge(accs)
}
switch {
case errors.Is(err, core.ErrGasLimitReached):
@ -1051,7 +1047,7 @@ func (w *worker) commit(uncles []*types.Header, interval func(), update bool, st
if tr := s.GetTrie(); tr.IsVerkle() {
vtr := tr.(*trie.VerkleTrie)
// Generate the proof if we are using a verkle tree
p, err := vtr.ProveAndSerialize(w.current.witness.Keys(), w.current.witness.KeyVals())
p, err := vtr.ProveAndSerialize(s.Witness().Keys(), s.Witness().KeyVals())
w.current.header.VerkleProof = p
if err != nil {
return err

View File

@ -135,6 +135,7 @@ func (trie *VerkleTrie) Commit(onleaf LeafCallback) (common.Hash, int, error) {
}
}
// XXX onleaf hasn't been called
return trie.Hash(), commitCount, nil
}
@ -237,19 +238,19 @@ func ChunkifyCode(addr common.Address, code []byte) ([][32]byte, error) {
chunkCount++
}
chunks := make([][32]byte, chunkCount)
for i, chunk := range chunks {
for i := range chunks {
end := 31 * (i + 1)
if len(code) < end {
end = len(code)
}
copy(chunk[1:], code[31*i:end])
copy(chunks[i][1:], code[31*i:end])
for j := lastOffset; int(j) < len(code[31*i:end]); j++ {
if code[j] >= byte(PUSH1) && code[j] <= byte(PUSH32) {
j += code[j] - byte(PUSH1) + 1
lastOffset = (j + 1) % 31
}
}
chunk[0] = lastOffset
chunks[i][0] = lastOffset
}
return chunks, nil