diff --git a/core/block_validator.go b/core/block_validator.go index 831209393..8dbd0f755 100644 --- a/core/block_validator.go +++ b/core/block_validator.go @@ -62,7 +62,7 @@ func (v *BlockValidator) ValidateBody(block *types.Block) error { if hash := types.CalcUncleHash(block.Uncles()); hash != header.UncleHash { return fmt.Errorf("uncle root hash mismatch: have %x, want %x", hash, header.UncleHash) } - if hash := types.DeriveSha(block.Transactions(), new(trie.Trie)); hash != header.TxHash { + if hash := types.DeriveSha(block.Transactions(), trie.NewStackTrie(nil)); hash != header.TxHash { return fmt.Errorf("transaction root hash mismatch: have %x, want %x", hash, header.TxHash) } if !v.bc.HasBlockAndState(block.ParentHash(), block.NumberU64()-1) { @@ -90,7 +90,7 @@ func (v *BlockValidator) ValidateState(block *types.Block, statedb *state.StateD return fmt.Errorf("invalid bloom (remote: %x local: %x)", header.Bloom, rbloom) } // Tre receipt Trie's root (R = (Tr [[H1, R1], ... [Hn, Rn]])) - receiptSha := types.DeriveSha(receipts, new(trie.Trie)) + receiptSha := types.DeriveSha(receipts, trie.NewStackTrie(nil)) if receiptSha != header.ReceiptHash { return fmt.Errorf("invalid receipt root hash (remote: %x local: %x)", header.ReceiptHash, receiptSha) } diff --git a/core/types/derive_sha.go b/core/types/derive_sha.go index 7d40c7f66..51b8506bc 100644 --- a/core/types/derive_sha.go +++ b/core/types/derive_sha.go @@ -23,7 +23,6 @@ import ( "github.com/ethereum/go-ethereum/rlp" ) -// DerivableList is the interface which can derive the hash. type DerivableList interface { Len() int GetRlp(i int) []byte @@ -39,7 +38,22 @@ type Hasher interface { func DeriveSha(list DerivableList, hasher Hasher) common.Hash { hasher.Reset() keybuf := new(bytes.Buffer) - for i := 0; i < list.Len(); i++ { + + // StackTrie requires values to be inserted in increasing + // hash order, which is not the order that `list` provides + // hashes in. This insertion sequence ensures that the + // order is correct. + for i := 1; i < list.Len() && i <= 0x7f; i++ { + keybuf.Reset() + rlp.Encode(keybuf, uint(i)) + hasher.Update(keybuf.Bytes(), list.GetRlp(i)) + } + if list.Len() > 0 { + keybuf.Reset() + rlp.Encode(keybuf, uint(0)) + hasher.Update(keybuf.Bytes(), list.GetRlp(0)) + } + for i := 0x80; i < list.Len(); i++ { keybuf.Reset() rlp.Encode(keybuf, uint(i)) hasher.Update(keybuf.Bytes(), list.GetRlp(i)) diff --git a/eth/downloader/queue.go b/eth/downloader/queue.go index 745f7c748..51a57f0d4 100644 --- a/eth/downloader/queue.go +++ b/eth/downloader/queue.go @@ -774,7 +774,7 @@ func (q *queue) DeliverBodies(id string, txLists [][]*types.Transaction, uncleLi q.lock.Lock() defer q.lock.Unlock() validate := func(index int, header *types.Header) error { - if types.DeriveSha(types.Transactions(txLists[index]), new(trie.Trie)) != header.TxHash { + if types.DeriveSha(types.Transactions(txLists[index]), trie.NewStackTrie(nil)) != header.TxHash { return errInvalidBody } if types.CalcUncleHash(uncleLists[index]) != header.UncleHash { @@ -799,7 +799,7 @@ func (q *queue) DeliverReceipts(id string, receiptList [][]*types.Receipt) (int, q.lock.Lock() defer q.lock.Unlock() validate := func(index int, header *types.Header) error { - if types.DeriveSha(types.Receipts(receiptList[index]), new(trie.Trie)) != header.ReceiptHash { + if types.DeriveSha(types.Receipts(receiptList[index]), trie.NewStackTrie(nil)) != header.ReceiptHash { return errInvalidReceipt } return nil diff --git a/eth/handler.go b/eth/handler.go index f5ec2c4fb..5b8998653 100644 --- a/eth/handler.go +++ b/eth/handler.go @@ -715,7 +715,7 @@ func (pm *ProtocolManager) handleMsg(p *peer) error { log.Warn("Propagated block has invalid uncles", "have", hash, "exp", request.Block.UncleHash()) break // TODO(karalabe): return error eventually, but wait a few releases } - if hash := types.DeriveSha(request.Block.Transactions(), new(trie.Trie)); hash != request.Block.TxHash() { + if hash := types.DeriveSha(request.Block.Transactions(), trie.NewStackTrie(nil)); hash != request.Block.TxHash() { log.Warn("Propagated block has invalid body", "have", hash, "exp", request.Block.TxHash()) break // TODO(karalabe): return error eventually, but wait a few releases } diff --git a/trie/database.go b/trie/database.go index fa8906b7a..c0c8870f8 100644 --- a/trie/database.go +++ b/trie/database.go @@ -99,6 +99,11 @@ type rawNode []byte func (n rawNode) cache() (hashNode, bool) { panic("this should never end up in a live trie") } func (n rawNode) fstring(ind string) string { panic("this should never end up in a live trie") } +func (n rawNode) EncodeRLP(w io.Writer) error { + _, err := w.Write([]byte(n)) + return err +} + // rawFullNode represents only the useful data content of a full node, with the // caches and flags stripped out to minimize its data storage. This type honors // the same RLP encoding as the original parent. @@ -199,7 +204,7 @@ func forGatherChildren(n node, onChild func(hash common.Hash)) { } case hashNode: onChild(common.BytesToHash(n)) - case valueNode, nil: + case valueNode, nil, rawNode: default: panic(fmt.Sprintf("unknown node type: %T", n)) } diff --git a/trie/encoding.go b/trie/encoding.go index 1955a3e66..8ee0022ef 100644 --- a/trie/encoding.go +++ b/trie/encoding.go @@ -51,6 +51,35 @@ func hexToCompact(hex []byte) []byte { return buf } +// hexToCompactInPlace places the compact key in input buffer, returning the length +// needed for the representation +func hexToCompactInPlace(hex []byte) int { + var ( + hexLen = len(hex) // length of the hex input + firstByte = byte(0) + ) + // Check if we have a terminator there + if hexLen > 0 && hex[hexLen-1] == 16 { + firstByte = 1 << 5 + hexLen-- // last part was the terminator, ignore that + } + var ( + binLen = hexLen/2 + 1 + ni = 0 // index in hex + bi = 1 // index in bin (compact) + ) + if hexLen&1 == 1 { + firstByte |= 1 << 4 // odd flag + firstByte |= hex[0] // first nibble is contained in the first byte + ni++ + } + for ; ni < hexLen; bi, ni = bi+1, ni+2 { + hex[bi] = hex[ni]<<4 | hex[ni+1] + } + hex[0] = firstByte + return binLen +} + func compactToHex(compact []byte) []byte { if len(compact) == 0 { return compact diff --git a/trie/encoding_test.go b/trie/encoding_test.go index 97d8da136..16393313f 100644 --- a/trie/encoding_test.go +++ b/trie/encoding_test.go @@ -18,6 +18,8 @@ package trie import ( "bytes" + "encoding/hex" + "math/rand" "testing" ) @@ -75,6 +77,40 @@ func TestHexKeybytes(t *testing.T) { } } +func TestHexToCompactInPlace(t *testing.T) { + for i, keyS := range []string{ + "00", + "060a040c0f000a090b040803010801010900080d090a0a0d0903000b10", + "10", + } { + hexBytes, _ := hex.DecodeString(keyS) + exp := hexToCompact(hexBytes) + sz := hexToCompactInPlace(hexBytes) + got := hexBytes[:sz] + if !bytes.Equal(exp, got) { + t.Fatalf("test %d: encoding err\ninp %v\ngot %x\nexp %x\n", i, keyS, got, exp) + } + } +} + +func TestHexToCompactInPlaceRandom(t *testing.T) { + for i := 0; i < 10000; i++ { + l := rand.Intn(128) + key := make([]byte, l) + rand.Read(key) + hexBytes := keybytesToHex(key) + hexOrig := []byte(string(hexBytes)) + exp := hexToCompact(hexBytes) + sz := hexToCompactInPlace(hexBytes) + got := hexBytes[:sz] + + if !bytes.Equal(exp, got) { + t.Fatalf("encoding err \ncpt %x\nhex %x\ngot %x\nexp %x\n", + key, hexOrig, got, exp) + } + } +} + func BenchmarkHexToCompact(b *testing.B) { testBytes := []byte{0, 15, 1, 12, 11, 8, 16 /*term*/} for i := 0; i < b.N; i++ { diff --git a/trie/stacktrie.go b/trie/stacktrie.go new file mode 100644 index 000000000..fc653101a --- /dev/null +++ b/trie/stacktrie.go @@ -0,0 +1,404 @@ +// Copyright 2020 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package trie + +import ( + "fmt" + "sync" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/ethdb" + "github.com/ethereum/go-ethereum/log" + "github.com/ethereum/go-ethereum/rlp" +) + +var stPool = sync.Pool{ + New: func() interface{} { + return NewStackTrie(nil) + }, +} + +func stackTrieFromPool(db ethdb.KeyValueStore) *StackTrie { + st := stPool.Get().(*StackTrie) + st.db = db + return st +} + +func returnToPool(st *StackTrie) { + st.Reset() + stPool.Put(st) +} + +// StackTrie is a trie implementation that expects keys to be inserted +// in order. Once it determines that a subtree will no longer be inserted +// into, it will hash it and free up the memory it uses. +type StackTrie struct { + nodeType uint8 // node type (as in branch, ext, leaf) + val []byte // value contained by this node if it's a leaf + key []byte // key chunk covered by this (full|ext) node + keyOffset int // offset of the key chunk inside a full key + children [16]*StackTrie // list of children (for fullnodes and exts) + + db ethdb.KeyValueStore // Pointer to the commit db, can be nil +} + +// NewStackTrie allocates and initializes an empty trie. +func NewStackTrie(db ethdb.KeyValueStore) *StackTrie { + return &StackTrie{ + nodeType: emptyNode, + db: db, + } +} + +func newLeaf(ko int, key, val []byte, db ethdb.KeyValueStore) *StackTrie { + st := stackTrieFromPool(db) + st.nodeType = leafNode + st.keyOffset = ko + st.key = append(st.key, key[ko:]...) + st.val = val + return st +} + +func newExt(ko int, key []byte, child *StackTrie, db ethdb.KeyValueStore) *StackTrie { + st := stackTrieFromPool(db) + st.nodeType = extNode + st.keyOffset = ko + st.key = append(st.key, key[ko:]...) + st.children[0] = child + return st +} + +// List all values that StackTrie#nodeType can hold +const ( + emptyNode = iota + branchNode + extNode + leafNode + hashedNode +) + +// TryUpdate inserts a (key, value) pair into the stack trie +func (st *StackTrie) TryUpdate(key, value []byte) error { + k := keybytesToHex(key) + if len(value) == 0 { + panic("deletion not supported") + } + st.insert(k[:len(k)-1], value) + return nil +} + +func (st *StackTrie) Update(key, value []byte) { + if err := st.TryUpdate(key, value); err != nil { + log.Error(fmt.Sprintf("Unhandled trie error: %v", err)) + } +} + +func (st *StackTrie) Reset() { + st.db = nil + st.key = st.key[:0] + st.val = st.val[:0] + for i := range st.children { + st.children[i] = nil + } + st.nodeType = emptyNode + st.keyOffset = 0 +} + +// Helper function that, given a full key, determines the index +// at which the chunk pointed by st.keyOffset is different from +// the same chunk in the full key. +func (st *StackTrie) getDiffIndex(key []byte) int { + diffindex := 0 + for ; diffindex < len(st.key) && st.key[diffindex] == key[st.keyOffset+diffindex]; diffindex++ { + } + return diffindex +} + +// Helper function to that inserts a (key, value) pair into +// the trie. +func (st *StackTrie) insert(key, value []byte) { + switch st.nodeType { + case branchNode: /* Branch */ + idx := int(key[st.keyOffset]) + // Unresolve elder siblings + for i := idx - 1; i >= 0; i-- { + if st.children[i] != nil { + if st.children[i].nodeType != hashedNode { + st.children[i].hash() + } + break + } + } + // Add new child + if st.children[idx] == nil { + st.children[idx] = stackTrieFromPool(st.db) + st.children[idx].keyOffset = st.keyOffset + 1 + } + st.children[idx].insert(key, value) + case extNode: /* Ext */ + // Compare both key chunks and see where they differ + diffidx := st.getDiffIndex(key) + + // Check if chunks are identical. If so, recurse into + // the child node. Otherwise, the key has to be split + // into 1) an optional common prefix, 2) the fullnode + // representing the two differing path, and 3) a leaf + // for each of the differentiated subtrees. + if diffidx == len(st.key) { + // Ext key and key segment are identical, recurse into + // the child node. + st.children[0].insert(key, value) + return + } + // Save the original part. Depending if the break is + // at the extension's last byte or not, create an + // intermediate extension or use the extension's child + // node directly. + var n *StackTrie + if diffidx < len(st.key)-1 { + n = newExt(diffidx+1, st.key, st.children[0], st.db) + } else { + // Break on the last byte, no need to insert + // an extension node: reuse the current node + n = st.children[0] + } + // Convert to hash + n.hash() + var p *StackTrie + if diffidx == 0 { + // the break is on the first byte, so + // the current node is converted into + // a branch node. + st.children[0] = nil + p = st + st.nodeType = branchNode + } else { + // the common prefix is at least one byte + // long, insert a new intermediate branch + // node. + st.children[0] = stackTrieFromPool(st.db) + st.children[0].nodeType = branchNode + st.children[0].keyOffset = st.keyOffset + diffidx + p = st.children[0] + } + // Create a leaf for the inserted part + o := newLeaf(st.keyOffset+diffidx+1, key, value, st.db) + + // Insert both child leaves where they belong: + origIdx := st.key[diffidx] + newIdx := key[diffidx+st.keyOffset] + p.children[origIdx] = n + p.children[newIdx] = o + st.key = st.key[:diffidx] + + case leafNode: /* Leaf */ + // Compare both key chunks and see where they differ + diffidx := st.getDiffIndex(key) + + // Overwriting a key isn't supported, which means that + // the current leaf is expected to be split into 1) an + // optional extension for the common prefix of these 2 + // keys, 2) a fullnode selecting the path on which the + // keys differ, and 3) one leaf for the differentiated + // component of each key. + if diffidx >= len(st.key) { + panic("Trying to insert into existing key") + } + + // Check if the split occurs at the first nibble of the + // chunk. In that case, no prefix extnode is necessary. + // Otherwise, create that + var p *StackTrie + if diffidx == 0 { + // Convert current leaf into a branch + st.nodeType = branchNode + p = st + st.children[0] = nil + } else { + // Convert current node into an ext, + // and insert a child branch node. + st.nodeType = extNode + st.children[0] = NewStackTrie(st.db) + st.children[0].nodeType = branchNode + st.children[0].keyOffset = st.keyOffset + diffidx + p = st.children[0] + } + + // Create the two child leaves: the one containing the + // original value and the one containing the new value + // The child leave will be hashed directly in order to + // free up some memory. + origIdx := st.key[diffidx] + p.children[origIdx] = newLeaf(diffidx+1, st.key, st.val, st.db) + p.children[origIdx].hash() + + newIdx := key[diffidx+st.keyOffset] + p.children[newIdx] = newLeaf(p.keyOffset+1, key, value, st.db) + + // Finally, cut off the key part that has been passed + // over to the children. + st.key = st.key[:diffidx] + st.val = nil + case emptyNode: /* Empty */ + st.nodeType = leafNode + st.key = key[st.keyOffset:] + st.val = value + case hashedNode: + panic("trying to insert into hash") + default: + panic("invalid type") + } +} + +// hash() hashes the node 'st' and converts it into 'hashedNode', if possible. +// Possible outcomes: +// 1. The rlp-encoded value was >= 32 bytes: +// - Then the 32-byte `hash` will be accessible in `st.val`. +// - And the 'st.type' will be 'hashedNode' +// 2. The rlp-encoded value was < 32 bytes +// - Then the <32 byte rlp-encoded value will be accessible in 'st.val'. +// - And the 'st.type' will be 'hashedNode' AGAIN +// +// This method will also: +// set 'st.type' to hashedNode +// clear 'st.key' +func (st *StackTrie) hash() { + /* Shortcut if node is already hashed */ + if st.nodeType == hashedNode { + return + } + // The 'hasher' is taken from a pool, but we don't actually + // claim an instance until all children are done with their hashing, + // and we actually need one + var h *hasher + + switch st.nodeType { + case branchNode: + var nodes [17]node + for i, child := range st.children { + if child == nil { + nodes[i] = nilValueNode + continue + } + child.hash() + if len(child.val) < 32 { + nodes[i] = rawNode(child.val) + } else { + nodes[i] = hashNode(child.val) + } + st.children[i] = nil // Reclaim mem from subtree + returnToPool(child) + } + nodes[16] = nilValueNode + h = newHasher(false) + defer returnHasherToPool(h) + h.tmp.Reset() + if err := rlp.Encode(&h.tmp, nodes); err != nil { + panic(err) + } + case extNode: + h = newHasher(false) + defer returnHasherToPool(h) + h.tmp.Reset() + st.children[0].hash() + // This is also possible: + //sz := hexToCompactInPlace(st.key) + //n := [][]byte{ + // st.key[:sz], + // st.children[0].val, + //} + n := [][]byte{ + hexToCompact(st.key), + st.children[0].val, + } + if err := rlp.Encode(&h.tmp, n); err != nil { + panic(err) + } + returnToPool(st.children[0]) + st.children[0] = nil // Reclaim mem from subtree + case leafNode: + h = newHasher(false) + defer returnHasherToPool(h) + h.tmp.Reset() + st.key = append(st.key, byte(16)) + sz := hexToCompactInPlace(st.key) + n := [][]byte{st.key[:sz], st.val} + if err := rlp.Encode(&h.tmp, n); err != nil { + panic(err) + } + case emptyNode: + st.val = st.val[:0] + st.val = append(st.val, emptyRoot[:]...) + st.key = st.key[:0] + st.nodeType = hashedNode + return + default: + panic("Invalid node type") + } + st.key = st.key[:0] + st.nodeType = hashedNode + if len(h.tmp) < 32 { + st.val = st.val[:0] + st.val = append(st.val, h.tmp...) + return + } + // Going to write the hash to the 'val'. Need to ensure it's properly sized first + // Typically, 'branchNode's will have no 'val', and require this allocation + if required := 32 - len(st.val); required > 0 { + buf := make([]byte, required) + st.val = append(st.val, buf...) + } + st.val = st.val[:32] + h.sha.Reset() + h.sha.Write(h.tmp) + h.sha.Read(st.val) + if st.db != nil { + // TODO! Is it safe to Put the slice here? + // Do all db implementations copy the value provided? + st.db.Put(st.val, h.tmp) + } +} + +// Hash returns the hash of the current node +func (st *StackTrie) Hash() (h common.Hash) { + st.hash() + if len(st.val) != 32 { + // If the node's RLP isn't 32 bytes long, the node will not + // be hashed, and instead contain the rlp-encoding of the + // node. For the top level node, we need to force the hashing. + ret := make([]byte, 32) + h := newHasher(false) + defer returnHasherToPool(h) + h.sha.Reset() + h.sha.Write(st.val) + h.sha.Read(ret) + return common.BytesToHash(ret) + } + return common.BytesToHash(st.val) +} + +// Commit will commit the current node to database db +func (st *StackTrie) Commit(db ethdb.KeyValueStore) common.Hash { + oldDb := st.db + st.db = db + defer func() { + st.db = oldDb + }() + st.hash() + h := common.BytesToHash(st.val) + return h +} diff --git a/trie/stacktrie_test.go b/trie/stacktrie_test.go new file mode 100644 index 000000000..26e3bade2 --- /dev/null +++ b/trie/stacktrie_test.go @@ -0,0 +1,242 @@ +package trie + +import ( + "bytes" + "fmt" + "math/big" + mrand "math/rand" + "testing" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/common/hexutil" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/crypto" + "github.com/ethereum/go-ethereum/ethdb/memorydb" +) + +func TestSizeBug(t *testing.T) { + st := NewStackTrie(nil) + nt, _ := New(common.Hash{}, NewDatabase(memorydb.New())) + + leaf := common.FromHex("290decd9548b62a8d60345a988386fc84ba6bc95484008f6362f93160ef3e563") + value := common.FromHex("94cf40d0d2b44f2b66e07cace1372ca42b73cf21a3") + + nt.TryUpdate(leaf, value) + st.TryUpdate(leaf, value) + + if nt.Hash() != st.Hash() { + t.Fatalf("error %x != %x", st.Hash(), nt.Hash()) + } +} + +func TestEmptyBug(t *testing.T) { + st := NewStackTrie(nil) + nt, _ := New(common.Hash{}, NewDatabase(memorydb.New())) + + //leaf := common.FromHex("290decd9548b62a8d60345a988386fc84ba6bc95484008f6362f93160ef3e563") + //value := common.FromHex("94cf40d0d2b44f2b66e07cace1372ca42b73cf21a3") + kvs := []struct { + K string + V string + }{ + {K: "405787fa12a823e0f2b7631cc41b3ba8828b3321ca811111fa75cd3aa3bb5ace", V: "9496f4ec2bf9dab484cac6be589e8417d84781be08"}, + {K: "40edb63a35fcf86c08022722aa3287cdd36440d671b4918131b2514795fefa9c", V: "01"}, + {K: "b10e2d527612073b26eecdfd717e6a320cf44b4afac2b0732d9fcbe2b7fa0cf6", V: "947a30f7736e48d6599356464ba4c150d8da0302ff"}, + {K: "c2575a0e9e593c00f959f8c92f12db2869c3395a3b0502d05e2516446f71f85b", V: "02"}, + } + + for _, kv := range kvs { + nt.TryUpdate(common.FromHex(kv.K), common.FromHex(kv.V)) + st.TryUpdate(common.FromHex(kv.K), common.FromHex(kv.V)) + } + + if nt.Hash() != st.Hash() { + t.Fatalf("error %x != %x", st.Hash(), nt.Hash()) + } +} + +func TestValLength56(t *testing.T) { + st := NewStackTrie(nil) + nt, _ := New(common.Hash{}, NewDatabase(memorydb.New())) + + //leaf := common.FromHex("290decd9548b62a8d60345a988386fc84ba6bc95484008f6362f93160ef3e563") + //value := common.FromHex("94cf40d0d2b44f2b66e07cace1372ca42b73cf21a3") + kvs := []struct { + K string + V string + }{ + {K: "405787fa12a823e0f2b7631cc41b3ba8828b3321ca811111fa75cd3aa3bb5ace", V: "1111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111111"}, + } + + for _, kv := range kvs { + nt.TryUpdate(common.FromHex(kv.K), common.FromHex(kv.V)) + st.TryUpdate(common.FromHex(kv.K), common.FromHex(kv.V)) + } + + if nt.Hash() != st.Hash() { + t.Fatalf("error %x != %x", st.Hash(), nt.Hash()) + } +} + +func genTxs(num uint64) (types.Transactions, error) { + key, err := crypto.HexToECDSA("deadbeefdeadbeefdeadbeefdeadbeefdeadbeefdeadbeefdeadbeefdeadbeef") + if err != nil { + return nil, err + } + var addr = crypto.PubkeyToAddress(key.PublicKey) + newTx := func(i uint64) (*types.Transaction, error) { + signer := types.NewEIP155Signer(big.NewInt(18)) + tx, err := types.SignTx(types.NewTransaction(i, addr, new(big.Int), 0, new(big.Int).SetUint64(10000000), nil), signer, key) + return tx, err + } + var txs types.Transactions + for i := uint64(0); i < num; i++ { + tx, err := newTx(i) + if err != nil { + return nil, err + } + txs = append(txs, tx) + } + return txs, nil +} + +func TestDeriveSha(t *testing.T) { + txs, err := genTxs(0) + if err != nil { + t.Fatal(err) + } + for len(txs) < 1000 { + exp := types.DeriveSha(txs, newEmpty()) + got := types.DeriveSha(txs, NewStackTrie(nil)) + if !bytes.Equal(got[:], exp[:]) { + t.Fatalf("%d txs: got %x exp %x", len(txs), got, exp) + } + newTxs, err := genTxs(uint64(len(txs) + 1)) + if err != nil { + t.Fatal(err) + } + txs = append(txs, newTxs...) + } +} + +func BenchmarkDeriveSha200(b *testing.B) { + txs, err := genTxs(200) + if err != nil { + b.Fatal(err) + } + var exp common.Hash + var got common.Hash + b.Run("std_trie", func(b *testing.B) { + b.ResetTimer() + b.ReportAllocs() + for i := 0; i < b.N; i++ { + exp = types.DeriveSha(txs, newEmpty()) + } + }) + + b.Run("stack_trie", func(b *testing.B) { + b.ResetTimer() + b.ReportAllocs() + for i := 0; i < b.N; i++ { + got = types.DeriveSha(txs, NewStackTrie(nil)) + } + }) + if got != exp { + b.Errorf("got %x exp %x", got, exp) + } +} + +type dummyDerivableList struct { + len int + seed int +} + +func newDummy(seed int) *dummyDerivableList { + d := &dummyDerivableList{} + src := mrand.NewSource(int64(seed)) + // don't use lists longer than 4K items + d.len = int(src.Int63() & 0x0FFF) + d.seed = seed + return d +} + +func (d *dummyDerivableList) Len() int { + return d.len +} + +func (d *dummyDerivableList) GetRlp(i int) []byte { + src := mrand.NewSource(int64(d.seed + i)) + // max item size 256, at least 1 byte per item + size := 1 + src.Int63()&0x00FF + data := make([]byte, size) + _, err := mrand.New(src).Read(data) + if err != nil { + panic(err) + } + return data +} + +func printList(l types.DerivableList) { + fmt.Printf("list length: %d\n", l.Len()) + fmt.Printf("{\n") + for i := 0; i < l.Len(); i++ { + v := l.GetRlp(i) + fmt.Printf("\"0x%x\",\n", v) + } + fmt.Printf("},\n") +} + +func TestFuzzDeriveSha(t *testing.T) { + // increase this for longer runs -- it's set to quite low for travis + rndSeed := mrand.Int() + for i := 0; i < 10; i++ { + seed := rndSeed + i + exp := types.DeriveSha(newDummy(i), newEmpty()) + got := types.DeriveSha(newDummy(i), NewStackTrie(nil)) + if !bytes.Equal(got[:], exp[:]) { + printList(newDummy(seed)) + t.Fatalf("seed %d: got %x exp %x", seed, got, exp) + } + } +} + +type flatList struct { + rlpvals []string +} + +func newFlatList(rlpvals []string) *flatList { + return &flatList{rlpvals} +} +func (f *flatList) Len() int { + return len(f.rlpvals) +} +func (f *flatList) GetRlp(i int) []byte { + return hexutil.MustDecode(f.rlpvals[i]) +} + +// TestDerivableList contains testcases found via fuzzing +func TestDerivableList(t *testing.T) { + type tcase []string + tcs := []tcase{ + { + "0xc041", + }, + { + "0xf04cf757812428b0763112efb33b6f4fad7deb445e", + "0xf04cf757812428b0763112efb33b6f4fad7deb445e", + }, + { + "0xca410605310cdc3bb8d4977ae4f0143df54a724ed873457e2272f39d66e0460e971d9d", + "0x6cd850eca0a7ac46bb1748d7b9cb88aa3bd21c57d852c28198ad8fa422c4595032e88a4494b4778b36b944fe47a52b8c5cd312910139dfcb4147ab8e972cc456bcb063f25dd78f54c4d34679e03142c42c662af52947d45bdb6e555751334ace76a5080ab5a0256a1d259855dfc5c0b8023b25befbb13fd3684f9f755cbd3d63544c78ee2001452dd54633a7593ade0b183891a0a4e9c7844e1254005fbe592b1b89149a502c24b6e1dca44c158aebedf01beae9c30cabe16a", + "0x14abd5c47c0be87b0454596baad2", + "0xca410605310cdc3bb8d4977ae4f0143df54a724ed873457e2272f39d66e0460e971d9d", + }, + } + for i, tc := range tcs[1:] { + exp := types.DeriveSha(newFlatList(tc), newEmpty()) + got := types.DeriveSha(newFlatList(tc), NewStackTrie(nil)) + if !bytes.Equal(got[:], exp[:]) { + t.Fatalf("case %d: got %x exp %x", i, got, exp) + } + } +}