diff --git a/cmd/hack/hack.go b/cmd/hack/hack.go index 3b62352cf9ea24abcebffbb0cc2aff1a7452a4b2..d6f899a6a66603a2061466259f0a5eb28697b0f5 100644 --- a/cmd/hack/hack.go +++ b/cmd/hack/hack.go @@ -11,6 +11,7 @@ import ( "fmt" "io" "io/ioutil" + "math" "math/big" "os" "os/signal" @@ -625,69 +626,278 @@ func mgrSchedule(chaindata string, block uint64) { db, err := ethdb.NewBoltDatabase(chaindata) check(err) - //bc, err := core.NewBlockChain(db, nil, params.MainnetChainConfig, ethash.NewFaker(), vm.Config{}, nil, nil) - //check(err) - //defer db.Close() - //tds, err := bc.GetTrieDbState() - //check(err) - //currentBlock := bc.CurrentBlock() - //currentBlockNr := currentBlock.NumberU64() - - loader := trie.NewSubTrieLoader(0) - tr := trie.New(common.Hash{}) - rs := trie.NewRetainList(0) - rs.AddHex([]byte{}) - subTries, err := loader.LoadSubTries(db, 0, rs, [][]byte{nil}, []int{0}, false) + bc, err := core.NewBlockChain(db, nil, params.MainnetChainConfig, ethash.NewFaker(), vm.Config{}, nil, nil) check(err) - - err = tr.HookSubTries(subTries, [][]byte{nil}) // hook up to the root + defer db.Close() + tds, err := bc.GetTrieDbState() check(err) + //t3 := time.Now() + //rl := trie.NewRetainList(3) + //rl.AddHex([]byte{}) + //loader := trie.NewFlatDbSubTrieLoader() + //tr := tds.Trie() + //err = loader.Reset(db, rl, rl, [][]byte{nil}, []int{0}, false) + //check(err) + //subTries, err := loader.LoadSubTries() + //check(err) + //fmt.Println("LoadSubTries: ", time.Since(t3)) + //t4 := time.Now() + //err = tr.HookSubTries(subTries, [][]byte{nil}) // hook up to the root + //check(err) + //fmt.Println("HookSubTries: ", time.Since(t4)) + //_, err = bc.ChainDb().(ethdb.DbWithPendingMutations).Commit() // apply IH changes + //check(err) - stateSize := tr.EstimateWitnessSize([]byte{}) - schedule := mgr.NewStateSchedule(stateSize, block, block+mgr.BlocksPerCycle+100) + //r1, _ := tds.PrefixByCumulativeWitnessSize([]byte{}, 46814683) + //r2, _ := tds.PrefixByCumulativeWitnessSize([]byte{}, 47126779) + //fmt.Printf("R: %x %x\n", r1, r2) - var buf bytes.Buffer + t5 := time.Now() + //var buf bytes.Buffer var witnessSizeAccumulator uint64 var witnessCount int64 var witnessEstimatedSizeAccumulator uint64 - //fmt.Printf("%s\n", schedule) - //fmt.Printf("stateSize: %d\n", stateSize) - for i := range schedule.Ticks { - tick := schedule.Ticks[i] - for j := range tick.StateSizeSlices { - ss := tick.StateSizeSlices[j] - stateSlice, err2 := mgr.StateSizeSlice2StateSlice(db, tr, ss) - if err2 != nil { - panic(err2) - } - retain := trie.NewRetainRange(common.CopyBytes(stateSlice.From), common.CopyBytes(stateSlice.To)) - //if tick.IsLastInCycle() { - // fmt.Printf("\nretain: %s\n", retain) - //} - witness, err2 := tr.ExtractWitness(false, retain) - if err2 != nil { - panic(err2) - } + schedule := mgr.NewSchedule(tds) + var toBlock = block + mgr.BlocksPerCycle + var ( + max float64 + min float64 = 1_000_000_000_000_000 + avg float64 + ) - buf.Reset() - _, err = witness.WriteTo(&buf) - if err != nil { - panic(err) + counters := []int{} + counters2 := []int{} + counters3 := []int{} + counters4 := []int{} + tx, _ := db.KV().Begin(false) + defer func() { + _ = tx.Rollback() + }() + for block <= toBlock { + tick, err2 := schedule.Tick(block) + if err2 != nil { + panic(err2) + } + counter := 0 + counter2 := 0 + counter3 := 0 + c := tx.Bucket(dbutils.CurrentStateBucket).Cursor() + for k, v := c.SeekTo(tick.From); k != nil; k, v = c.Next() { + if bytes.Compare(k, tick.To) > 0 && !bytes.HasPrefix(k, tick.To) { + break } + counter += len(k) + len(v) + counter2 += len(v) + counter3++ + } + check(err) - witnessCount++ - witnessSizeAccumulator += uint64(buf.Len()) + counters = append(counters, counter) + counters2 = append(counters2, counter2) + counters3 = append(counters3, counter3) + + //retain := trie.NewRetainRange(common.CopyBytes(tick.From), common.CopyBytes(tick.To)) + //dbPrefixes, fixedbits, hooks := tr.FindSubTriesToLoad(rl) + //err = loader.Reset(db, retain, retain, dbPrefixes, fixedbits, false) + //check(err) + //subTries, err2 := loader.LoadSubTries() + //check(err2) + //err = tr.HookSubTries(subTries, hooks) // hook up to the root + //check(err) + //witness, err2 := tr.ExtractWitness(false, retain) + //if err2 != nil { + // panic(err2) + //} + //buf.Reset() + //_, err = witness.WriteTo(&buf) + //if err != nil { + // panic(err) + //} + //counters4 = append(counters4, buf.Len()) + + min = math.Min(min, float64(counter)) + max = math.Max(max, float64(counter)) + if avg == 0 { + avg = float64(counter) + } else { + avg = (avg + float64(counter)) / 2 } + witnessEstimatedSizeAccumulator += tick.ToSize - tick.FromSize + block = tick.ToBlock + 1 + } + + fmt.Println("MGR Time of All Ticks: ", time.Since(t5)) + fmt.Println("Min: ", min, "Max: ", max, "Avg: ", avg) + + accs := map[int]int{} + for _, counter := range counters { + accs[counter/100_000]++ + } + fmt.Printf("Counter Aggs: %v\n", accs) + + accs2 := map[int]int{} + for _, counter := range counters2 { + accs2[counter/100_000]++ + } + fmt.Printf("Counter Aggs2: %v\n", accs2) + + accs3 := map[int]int{} + for _, counter := range counters3 { + accs3[counter/1_000]++ + } + fmt.Printf("Counter Aggs3: %v\n", accs3) + + accs4 := map[int]int{} + for _, counter := range counters4 { + accs4[counter/1_000]++ } + fmt.Printf("Counter Aggs4: %v\n", accs4) fmt.Printf("witnessCount: %s\n", humanize.Comma(witnessCount)) fmt.Printf("witnessSizeAccumulator: %s\n", humanize.Bytes(witnessSizeAccumulator)) fmt.Printf("witnessEstimatedSizeAccumulator: %s\n", humanize.Bytes(witnessEstimatedSizeAccumulator)) + + //defer func(t time.Time) { fmt.Println("hack.go:746", time.Since(t)) }(time.Now()) + //tr.EvictNode([]byte{}) + //_, err = bc.ChainDb().(ethdb.DbWithPendingMutations).Commit() + //check(err) +} + +func resetIH(chaindata string, fromScratch bool) { + db, err := ethdb.NewBoltDatabase(chaindata) + check(err) + if fromScratch { + err = db.KV().Update(func(tx *bolt.Tx) error { + _ = tx.DeleteBucket(dbutils.IntermediateTrieHashBucket) + _, _ = tx.CreateBucket(dbutils.IntermediateTrieHashBucket, false) + _ = tx.DeleteBucket(dbutils.IntermediateWitnessSizeBucket) + _, _ = tx.CreateBucket(dbutils.IntermediateWitnessSizeBucket, false) + return nil + }) + check(err) + } + + var before int + err = db.KV().View(func(tx *bolt.Tx) error { + before = tx.Bucket(dbutils.IntermediateTrieHashBucket).Stats().KeyN + return nil + }) + check(err) + bc, err := core.NewBlockChain(db, nil, params.MainnetChainConfig, ethash.NewFaker(), vm.Config{}, nil, nil) + check(err) + defer db.Close() + tds, err := bc.GetTrieDbState() + check(err) + loader := trie.NewFlatDbSubTrieLoader() + tr := tds.Trie() + for i := 0; i < 16; i++ { + for j := 0; j < 16; j++ { + prefix := []byte{uint8(i), uint8(j)} + t1 := time.Now() + err = db.KV().Update(func(tx *bolt.Tx) error { + b := tx.Bucket(dbutils.IntermediateTrieHashBucket) + b2 := tx.Bucket(dbutils.IntermediateWitnessSizeBucket) + c := b.Cursor() + c2 := b2.Cursor() + for k, _ := c.Seek(prefix); k != nil && bytes.HasPrefix(k, prefix); k, _ = c.Next() { + _ = b.Delete(k) + } + for k, _ := c2.Seek(prefix); k != nil && bytes.HasPrefix(k, prefix); k, _ = c2.Next() { + _ = b2.Delete(k) + } + + return nil + }) + check(err) + t1Duration := time.Since(t1) + rl := trie.NewRetainList(0) + rl.AddHex(prefix) + dbPrefixes, fixedbits, hooks := tr.FindSubTriesToLoad(rl) + t3 := time.Now() + rl2 := trie.NewRetainRange(prefix, prefix) + err = loader.Reset(db, rl2, rl2, dbPrefixes, fixedbits, false) + check(err) + subTries, err2 := loader.LoadSubTries() + check(err2) + t3Duration := time.Since(t3) + t4 := time.Now() + err = tr.HookSubTries(subTries, hooks) // hook up to the root + check(err) + t4Duration := time.Since(t4) + t5 := time.Now() + tr.EvictNode(prefix) + t5Duration := time.Since(t5) + t6 := time.Now() + _, err = bc.ChainDb().(ethdb.DbWithPendingMutations).Commit() + check(err) + t6Duration := time.Since(t6) + fmt.Printf("%x: Delete %s Load %s, Hook %s, Evict %s, Commit %s\n", i*16+j, t1Duration, t3Duration, t4Duration, t5Duration, t6Duration) + } + } + tr.EvictNode([]byte{}) + _, err = bc.ChainDb().(ethdb.DbWithPendingMutations).Commit() + check(err) + + var after int + err = db.KV().View(func(tx *bolt.Tx) error { + after = tx.Bucket(dbutils.IntermediateTrieHashBucket).Stats().KeyN + return nil + }) + check(err) + fmt.Printf("IH bucket changed from %d to %d records\n", before, after) + + accs := map[uint64]int{} + accs2 := map[uint64]int{} + err = db.AbstractKV().View(context.Background(), func(tx ethdb.Tx) error { + return tx.Bucket(dbutils.IntermediateWitnessSizeBucket).Cursor().Walk(func(k, v []byte) (bool, error) { + i := binary.BigEndian.Uint64(v) + accs[i/10]++ + if len(k) > 32 { + accs2[i/10]++ + } + return true, nil + }) + }) + check(err) + for i := range accs { + if accs[i] < 1000 { + delete(accs, i) + } + } + for i := range accs2 { + if accs2[i] < 300 { + delete(accs2, i) + } + } + + fmt.Printf("Agg IWS Stats1: %v\n", accs) + fmt.Printf("Agg IWS Stats2: %v\n", accs2) + + // now can delete some IH + //err = db.KV().Update(func(tx *bolt.Tx) error { + // defer func(t time.Time) { fmt.Println("Del", time.Since(t)) }(time.Now()) + // b := tx.Bucket(dbutils.IntermediateTrieHashBucket) + // b2 := tx.Bucket(dbutils.IntermediateWitnessSizeBucket) + // c := b.Cursor() + // c2 := b2.Cursor() + // for k, _ := c.First(); k != nil; k, _ = c.Next() { + // if len(k) < 2 { + // _ = b.Delete(k) + // } + // } + // for k, _ := c2.First(); k != nil; k, _ = c2.Next() { + // if len(k) < 2 { + // _ = b2.Delete(k) + // } + // } + // + // return nil + //}) } func execToBlock(chaindata string, block uint64, fromScratch bool) { - state.MaxTrieCacheSize = 32 + state.MaxTrieCacheSize = 100 * 1024 blockDb, err := ethdb.NewBoltDatabase(chaindata) check(err) bcb, err := core.NewBlockChain(blockDb, nil, params.MainnetChainConfig, ethash.NewFaker(), vm.Config{}, nil, nil) @@ -696,14 +906,14 @@ func execToBlock(chaindata string, block uint64, fromScratch bool) { if fromScratch { os.Remove("statedb") } - stateDb, err := ethdb.NewBoltDatabase("statedb") + stateDB, err := ethdb.NewBoltDatabase("statedb") check(err) - defer stateDb.Close() + defer stateDB.Close() - //_, _, _, err = core.SetupGenesisBlock(stateDb, core.DefaultGenesisBlock()) - _, _, _, err = core.SetupGenesisBlock(stateDb, nil, false /* history */) + //_, _, _, err = core.SetupGenesisBlock(stateDB, core.DefaultGenesisBlock()) + _, _, _, err = core.SetupGenesisBlock(stateDB, nil, false /* history */) check(err) - bc, err := core.NewBlockChain(stateDb, nil, params.MainnetChainConfig, ethash.NewFaker(), vm.Config{}, nil, nil) + bc, err := core.NewBlockChain(stateDB, nil, params.MainnetChainConfig, ethash.NewFaker(), vm.Config{}, nil, nil) check(err) tds, err := bc.GetTrieDbState() check(err) @@ -722,7 +932,7 @@ func execToBlock(chaindata string, block uint64, fromScratch bool) { for i := importedBn; i <= block; i++ { lastBlock = bcb.GetBlockByNumber(i) blocks = append(blocks, lastBlock) - if len(blocks) >= 1000 || i == block { + if len(blocks) >= 100 || i == block { _, err = bc.InsertChain(context.Background(), blocks) if err != nil { log.Error("Could not insert blocks (group)", "number", len(blocks), "error", err) @@ -1993,7 +2203,7 @@ func (r *Receiver) Receive( storageValue []byte, hash []byte, cutoff int, - witnessLen uint64, + witnessSize uint64, ) error { for r.currentIdx < len(r.unfurlList) { ks := r.unfurlList[r.currentIdx] @@ -2015,7 +2225,7 @@ func (r *Receiver) Receive( c = -1 } if c > 0 { - return r.defaultReceiver.Receive(itemType, accountKey, storageKeyPart1, storageKeyPart2, accountValue, storageValue, hash, cutoff, witnessLen) + return r.defaultReceiver.Receive(itemType, accountKey, storageKeyPart1, storageKeyPart2, accountValue, storageValue, hash, cutoff, witnessSize) } if len(k) > common.HashLength { v := r.storageMap[ks] @@ -2038,7 +2248,7 @@ func (r *Receiver) Receive( } } // We ran out of modifications, simply pass through - return r.defaultReceiver.Receive(itemType, accountKey, storageKeyPart1, storageKeyPart2, accountValue, storageValue, hash, cutoff, witnessLen) + return r.defaultReceiver.Receive(itemType, accountKey, storageKeyPart1, storageKeyPart2, accountValue, storageValue, hash, cutoff, witnessSize) } func (r *Receiver) Result() trie.SubTries { @@ -2125,7 +2335,7 @@ func testGetProof(chaindata string, block uint64, account common.Address) { sort.Strings(unfurlList) fmt.Printf("Account changesets: %d, storage changesets: %d, unfurlList: %d\n", accountCs, storageCs, len(unfurlList)) loader := trie.NewFlatDbSubTrieLoader() - if err = loader.Reset(db, unfurl, [][]byte{nil}, []int{0}, false); err != nil { + if err = loader.Reset(db, unfurl, unfurl, [][]byte{nil}, []int{0}, false); err != nil { panic(err) } r := &Receiver{defaultReceiver: trie.NewDefaultReceiver(), unfurlList: unfurlList, accountMap: accountMap, storageMap: storageMap} @@ -2248,7 +2458,7 @@ func main() { } //printBranches(uint64(*block)) if *action == "execToBlock" { - execToBlock(*chaindata, uint64(*block), true) + execToBlock(*chaindata, uint64(*block), false) } //extractTrie(*block) //repair() @@ -2302,6 +2512,9 @@ func main() { if *action == "mgrSchedule" { mgrSchedule(*chaindata, uint64(*block)) } + if *action == "resetIH" { + resetIH(*chaindata, false) + } if *action == "resetState" { resetState(*chaindata) } diff --git a/cmd/restapi/apis/intermediate_data_len_api.go b/cmd/restapi/apis/intermediate_data_len_api.go index 099796107bd412bd4a7ce9b86901fdd50b3c3357..3317fafb9a7e7744febd175fc04fc0cd78663b07 100644 --- a/cmd/restapi/apis/intermediate_data_len_api.go +++ b/cmd/restapi/apis/intermediate_data_len_api.go @@ -34,7 +34,7 @@ func findIntermediateDataLenByPrefix(prefixS string, remoteDB ethdb.KV) ([]*Inte var results []*IntermediateDataLenResponse prefix := common.FromHex(prefixS) if err := remoteDB.View(context.TODO(), func(tx ethdb.Tx) error { - interBucket := tx.Bucket(dbutils.IntermediateTrieWitnessLenBucket) + interBucket := tx.Bucket(dbutils.IntermediateWitnessSizeBucket) c := interBucket.Cursor().Prefix(prefix) for k, v, err := c.First(); k != nil || err != nil; k, v, err = c.Next() { diff --git a/common/dbutils/bucket.go b/common/dbutils/bucket.go index 3efba613690cacac920d28e34fdd2b4be095e70b..411a231ab680d181c88f0e3ba10e8205f462a3a2 100644 --- a/common/dbutils/bucket.go +++ b/common/dbutils/bucket.go @@ -80,7 +80,7 @@ var ( IntermediateTrieHashBucket = []byte("iTh") // some_prefix_of(hash_of_address_of_account) => estimated_number_of_witness_bytes - IntermediateTrieWitnessLenBucket = []byte("iTw") + IntermediateWitnessSizeBucket = []byte("iws") // DatabaseInfoBucket is used to store information about data layout. DatabaseInfoBucket = []byte("DBINFO") @@ -156,7 +156,7 @@ var Buckets = [][]byte{ AccountChangeSetBucket, StorageChangeSetBucket, IntermediateTrieHashBucket, - IntermediateTrieWitnessLenBucket, + IntermediateWitnessSizeBucket, DatabaseVerisionKey, HeadHeaderKey, HeadBlockKey, diff --git a/common/dbutils/composite_keys.go b/common/dbutils/composite_keys.go index cffc45b77585be66214a612f6b963c82869300c3..c466e501b7cd18c0a477f32e7905da333d9f8ec8 100644 --- a/common/dbutils/composite_keys.go +++ b/common/dbutils/composite_keys.go @@ -170,6 +170,25 @@ func ParseStoragePrefix(prefix []byte) (common.Hash, uint64) { return addrHash, inc } +func DecodeIncarnation(buf []byte) uint64 { + incarnation := binary.BigEndian.Uint64(buf) + return incarnation ^ ^uint64(0) +} + +func EncodeIncarnation(incarnation uint64, buf []byte) { + binary.BigEndian.PutUint64(buf, ^incarnation) +} + +func RemoveIncarnationFromKey(key []byte, out *[]byte) { + tmp := (*out)[:0] + if len(key) <= common.HashLength { + tmp = append(tmp, key...) + } else { + tmp = append(tmp, key[:common.HashLength]...) + tmp = append(tmp, key[common.HashLength+8:]...) + } + *out = tmp +} // Key + blockNum func CompositeKeySuffix(key []byte, timestamp uint64) (composite, encodedTS []byte) { @@ -179,4 +198,3 @@ func CompositeKeySuffix(key []byte, timestamp uint64) (composite, encodedTS []by copy(composite[len(key):], encodedTS) return composite, encodedTS } - diff --git a/common/dbutils/helper.go b/common/dbutils/helper.go index 069e2b933c9a4f06999cd1c16c31e402b3696cbb..d8110ac05a07e896349c7c51ff301be113e91416 100644 --- a/common/dbutils/helper.go +++ b/common/dbutils/helper.go @@ -1,6 +1,8 @@ package dbutils -import "bytes" +import ( + "bytes" +) // EncodeTimestamp has the property: if a < b, then Encoding(a) < Encoding(b) lexicographically func EncodeTimestamp(timestamp uint64) []byte { @@ -42,27 +44,36 @@ func ChangeSetByIndexBucket(b []byte) []byte { panic("wrong bucket") } -// Cmp - like bytes.Compare, but nil - means "bucket over" and has highest order. -func Cmp(k1, k2 []byte) int { - if k1 == nil && k2 == nil { - return 0 - } - if k1 == nil { - return 1 - } - if k2 == nil { - return -1 - } +// NextSubtree does []byte++. Returns false if overflow. +func NextSubtree(in []byte) ([]byte, bool) { + r := make([]byte, len(in)) + copy(r, in) + for i := len(r) - 1; i >= 0; i-- { + if r[i] != 255 { + r[i]++ + return r, true + } - return bytes.Compare(k1, k2) + r[i] = 0 + } + return nil, false } -// IsBefore - kind of bytes.Compare, but nil is the last key. And return -func IsBefore(k1, k2 []byte) (bool, []byte) { - switch Cmp(k1, k2) { - case -1, 0: - return true, k1 - default: - return false, k2 +func NextS(in []byte, out *[]byte) bool { + tmp := *out + if cap(tmp) < len(in) { + tmp = make([]byte, len(in)) + } + tmp = tmp[:len(in)] + copy(tmp, in) + for i := len(tmp) - 1; i >= 0; i-- { + if tmp[i] != 255 { + tmp[i]++ + *out = tmp + return true + } + tmp[i] = 0 } + *out = tmp + return false } diff --git a/core/state/database.go b/core/state/database.go index 0fd5a1899413e122af2bba5e46d7869b62d44f27..2586fdd022038a481086c8cdc15783a8b8235ccf 100644 --- a/core/state/database.go +++ b/core/state/database.go @@ -27,6 +27,7 @@ import ( "sort" "sync" "sync/atomic" + "time" "github.com/holiman/uint256" @@ -36,9 +37,15 @@ import ( "github.com/ledgerwatch/turbo-geth/core/types/accounts" "github.com/ledgerwatch/turbo-geth/ethdb" "github.com/ledgerwatch/turbo-geth/log" + "github.com/ledgerwatch/turbo-geth/metrics" "github.com/ledgerwatch/turbo-geth/trie" ) +var ( + cumulativeSearchTimer = metrics.NewRegisteredTimer("db/cumulativesearch", nil) + cumulativeSearchMeter = metrics.NewRegisteredMeter("db/cumulativesearch/seeks", nil) +) + var _ StateWriter = (*TrieStateWriter)(nil) // MaxTrieCacheSize is the trie cache size limit after which to evict trie nodes from memory. @@ -1421,3 +1428,198 @@ func (tds *TrieDbState) GetTrieHash() common.Hash { defer tds.tMu.Unlock() return tds.t.Hash() } + +func (tds *TrieDbState) TotalCumulativeWitnessSize() (uint64, error) { + tds.tMu.Lock() + defer tds.tMu.Unlock() + + var kv ethdb.KV + if hasBolt, ok := tds.db.(ethdb.HasAbstractKV); ok { + kv = hasBolt.AbstractKV() + } else { + return 0, fmt.Errorf("unexpected db type: %T", tds.db) + } + + var accumulator uint64 + _, _, _, err := CumulativeSearch(kv, dbutils.IntermediateWitnessSizeBucket, []byte{}, []byte{}, 0, func(k, v []byte) (itsTimeToVisitChild bool, err error) { + accumulator += binary.BigEndian.Uint64(v) + return false, nil + }) + if err != nil { + return 0, err + } + //fmt.Printf("Total Count: Seeks: %d, accSeek: %d\n", seeks, accSeeks) + + return accumulator, nil +} + +func (tds *TrieDbState) PrefixByCumulativeWitnessSize(from []byte, size uint64) (prefix []byte, err error) { + if size == 0 { + return from, nil + } + var kv ethdb.KV + if hasBolt, ok := tds.db.(ethdb.HasAbstractKV); ok { + kv = hasBolt.AbstractKV() + } else { + return nil, fmt.Errorf("unexpected db type: %T", tds.db) + } + + var accumulator uint64 + prefix, _, _, err = CumulativeSearch(kv, dbutils.IntermediateWitnessSizeBucket, from, []byte{}, 0, func(k, v []byte) (itsTimeToVisitChild bool, err error) { + prefixSize := binary.BigEndian.Uint64(v) + overflow := accumulator+prefixSize >= size + //fmt.Printf("Loop: %x %d<%d=%t\n", k, accumulator+prefixSize, size, overflow) + if !overflow { + accumulator += prefixSize + } + return overflow, nil + }) + if err != nil { + return nil, err + } + return prefix, nil +} + +type CumulativeSearchF func(k, v []byte) (itsTimeToVisitChild bool, err error) + +// CumulativeSearch - on each iteration jumps to next subtries (siblings) by default until not get command to go to child +func CumulativeSearch(kv ethdb.KV, bucket []byte, startKey []byte, parent []byte, fixedbits int, f CumulativeSearchF) (lastVisitedParent []byte, seeks, accSeeks int64, err error) { + const doCorrectIncarnation = true + defer cumulativeSearchTimer.UpdateSince(time.Now()) + defer cumulativeSearchMeter.Mark(seeks) + + var ok bool + a := accounts.NewAccount() + if err = kv.View(context.Background(), func(tx ethdb.Tx) error { + c := tx.Bucket(bucket).Cursor() + cs := tx.Bucket(dbutils.CurrentStateBucket).Cursor() + + var k, v, csK, csV []byte + // move 2 cursors until find correct incarnation or face shorter prefix + var skipIncorrectIncarnation = func() error { + if !doCorrectIncarnation || len(k) < 32 || len(parent) > 32 || bytes.Equal(csK, k[:32]) { + return nil + } + counter := 0 + + for !(bytes.Equal(csK, k[:32]) && a.Incarnation == dbutils.DecodeIncarnation(k[32:40])) { + counter++ + next := k[:32] + accSeeks++ + var doSkip bool + csK, csV, err = cs.SeekTo(next) + for ; len(csK) != 32; csK, csV, err = cs.SeekTo(next) { + counter++ + if err != nil { + return err + } + doSkip = true + accSeeks++ + next, ok = dbutils.NextSubtree(csK[:32]) + if !ok { + k = nil + return nil + } + } + + if doSkip { + k, v, err = c.SeekTo(next) + if err != nil { + return err + } + if !bytes.HasPrefix(k, parent) || k == nil { + k = nil + return nil + } + } + if len(k) < 32 { + return nil // skipped all incarnations + } + + err = a.DecodeForStorage(csV) + if err != nil { + return err + } + + if bytes.Equal(csK, k[:32]) && a.Incarnation != dbutils.DecodeIncarnation(k[32:40]) { + seeks++ + k, v, err = c.SeekTo(dbutils.GenerateStoragePrefix(k[:32], a.Incarnation)) + if err != nil { + return err + } + if !bytes.HasPrefix(k, parent) || k == nil { + k = nil + return nil + } + if len(k) < 32 { + return nil // skipped all incarnations + } + } + } + return nil + } + + fixedbytes, mask := ethdb.Bytesmask(fixedbits) + next := common.CopyBytes(startKey) + seeks++ + k, v, err = c.SeekTo(next) + if err != nil { + return err + } + + for k != nil && len(k) >= fixedbytes && (fixedbits == 0 || bytes.Equal(k[:fixedbytes-1], parent[:fixedbytes-1]) && (k[fixedbytes-1]&mask) == (parent[fixedbytes-1]&mask)) { + itsTimeToVisitChild, err2 := f(k, v) + if err2 != nil { + return err2 + } + if !itsTimeToVisitChild { // go to sibling + next, ok = dbutils.NextSubtree(k) + if !ok { + k = nil + continue + } + seeks++ + k, v, err = c.SeekTo(next) + if err != nil { + return err + } + if !bytes.HasPrefix(k, parent) { + k = nil + } + err = skipIncorrectIncarnation() + if err != nil { + return err + } + if k == nil { + return nil // do something to express that it's end of state + } + continue + } + + parent = common.CopyBytes(k) + fixedbits = len(parent) * 8 + fixedbytes, mask = ethdb.Bytesmask(fixedbits) + seeks++ + k, v, err = c.Next() // go to child + if err != nil { + return err + } + if !bytes.HasPrefix(k, parent) { + k = nil + } + err = skipIncorrectIncarnation() + if err != nil { + return err + } + if k == nil { + return nil // do something to express that it's end + } + continue + } + return nil + }); err != nil { + return parent, seeks, accSeeks, err + } + + return parent, seeks, accSeeks, nil +} diff --git a/core/state/intermediate_hashes.go b/core/state/intermediate_hashes.go index 2101fa42bbc3e03226d593c55a4b1c340e1734f9..6ff169d9cf77fba15fb6108d341fc1047fe7c87e 100644 --- a/core/state/intermediate_hashes.go +++ b/core/state/intermediate_hashes.go @@ -30,7 +30,7 @@ func NewIntermediateHashes(putter ethdb.Putter, deleter ethdb.Deleter) *Intermed return &IntermediateHashes{putter: putter, deleter: deleter} } -func (ih *IntermediateHashes) WillUnloadBranchNode(prefixAsNibbles []byte, nodeHash common.Hash, incarnation uint64, witnessLen uint64) { +func (ih *IntermediateHashes) WillUnloadBranchNode(prefixAsNibbles []byte, nodeHash common.Hash, incarnation uint64, witnessSize uint64) { // only put to bucket prefixes with even number of nibbles if len(prefixAsNibbles) == 0 || len(prefixAsNibbles)%2 == 1 { return @@ -49,14 +49,14 @@ func (ih *IntermediateHashes) WillUnloadBranchNode(prefixAsNibbles []byte, nodeH key = common.CopyBytes(buf.B) } - lenBytes := make([]byte, 8) - binary.BigEndian.PutUint64(lenBytes, witnessLen) - if err := ih.putter.Put(dbutils.IntermediateTrieHashBucket, key, common.CopyBytes(nodeHash[:])); err != nil { log.Warn("could not put intermediate trie hash", "err", err) } + if debug.IsTrackWitnessSizeEnabled() { - if err := ih.putter.Put(dbutils.IntermediateTrieWitnessLenBucket, key, lenBytes); err != nil { + lenBytes := make([]byte, 8) + binary.BigEndian.PutUint64(lenBytes, witnessSize) + if err := ih.putter.Put(dbutils.IntermediateWitnessSizeBucket, common.CopyBytes(key), lenBytes); err != nil { log.Warn("could not put intermediate trie data len", "err", err) } } @@ -84,9 +84,8 @@ func (ih *IntermediateHashes) BranchNodeLoaded(prefixAsNibbles []byte, incarnati log.Warn("could not delete intermediate trie hash", "err", err) } if debug.IsTrackWitnessSizeEnabled() { - if err := ih.deleter.Delete(dbutils.IntermediateTrieWitnessLenBucket, key); err != nil { + if err := ih.deleter.Delete(dbutils.IntermediateWitnessSizeBucket, key); err != nil { log.Warn("could not delete intermediate trie hash", "err", err) } } - } diff --git a/eth/backend.go b/eth/backend.go index 5db07c40836b90a9f080194f8350958adffa8a3d..c8f77e6672632e3bbc772daea196e43dcfd2f2c4 100644 --- a/eth/backend.go +++ b/eth/backend.go @@ -663,7 +663,6 @@ func setStorageModeIfNotExist(db ethdb.Database, sm StorageMode) error { var ( err error ) - err = setModeOnEmpty(db, dbutils.StorageModeHistory, sm.History) if err != nil { return err diff --git a/eth/downloader/modes.go b/eth/downloader/modes.go index 4fad43ab214e66a50c9301149f660f5c477f79cb..bf5362c822fa24062613f64f194de656445a7707 100644 --- a/eth/downloader/modes.go +++ b/eth/downloader/modes.go @@ -26,6 +26,7 @@ const ( FastSync // Quickly download the headers, full sync only at the chain head LightSync // Download only the headers and terminate afterwards StagedSync // Full sync but done in stages + MgrSync // MarryGoRound sync ) const ( @@ -33,6 +34,7 @@ const ( FastSyncName = "fast" LightSyncName = "light" StagedSyncName = "staged" + MgrSyncName = "mgr" ) func (mode SyncMode) IsValid() bool { @@ -50,6 +52,8 @@ func (mode SyncMode) String() string { return LightSyncName case StagedSync: return StagedSyncName + case MgrSync: + return MgrSyncName default: return "unknown" } @@ -65,6 +69,8 @@ func (mode SyncMode) MarshalText() ([]byte, error) { return []byte(LightSyncName), nil case StagedSync: return []byte(StagedSyncName), nil + case MgrSync: + return []byte(MgrSyncName), nil default: return nil, fmt.Errorf("unknown sync mode %d", mode) } @@ -80,9 +86,11 @@ func (mode *SyncMode) UnmarshalText(text []byte) error { *mode = LightSync case StagedSyncName: *mode = StagedSync + case MgrSyncName: + *mode = MgrSync default: - return fmt.Errorf(`unknown sync mode %q, want "%s", "%s", "%s" or "%s"`, - FullSyncName, FastSyncName, LightSyncName, StagedSyncName, text) + return fmt.Errorf(`unknown sync mode %q, want "%s", "%s", "%s" or "%s" or "%s"`, + FullSyncName, FastSyncName, LightSyncName, StagedSyncName, MgrSyncName, text) } return nil } diff --git a/eth/handler.go b/eth/handler.go index e7f8bd838e2f5e5d2d373afa78643ce1501de02b..d54359650b519866c19f53173f5ba030480e6333 100644 --- a/eth/handler.go +++ b/eth/handler.go @@ -33,11 +33,13 @@ import ( "github.com/ledgerwatch/turbo-geth/consensus" "github.com/ledgerwatch/turbo-geth/core" "github.com/ledgerwatch/turbo-geth/core/forkid" + "github.com/ledgerwatch/turbo-geth/core/state" "github.com/ledgerwatch/turbo-geth/core/types" "github.com/ledgerwatch/turbo-geth/core/vm" "github.com/ledgerwatch/turbo-geth/crypto" "github.com/ledgerwatch/turbo-geth/eth/downloader" "github.com/ledgerwatch/turbo-geth/eth/fetcher" + "github.com/ledgerwatch/turbo-geth/eth/mgr" "github.com/ledgerwatch/turbo-geth/ethdb" "github.com/ledgerwatch/turbo-geth/ethdb/remote/remotedbserver" "github.com/ledgerwatch/turbo-geth/event" @@ -104,8 +106,9 @@ type ProtocolManager struct { // Test fields or hooks broadcastTxAnnouncesOnly bool // Testing field, disable transaction propagation - mode downloader.SyncMode // Sync mode passed from the command line + mode downloader.SyncMode // Sync mode passed from the command line datadir string + mgr *mgrBroadcast } // NewProtocolManager returns a new Ethereum sub protocol manager. The Ethereum sub protocol manages peers capable @@ -156,19 +159,23 @@ func NewProtocolManager(config *params.ChainConfig, checkpoint *params.TrustedCh manager.checkpointHash = checkpoint.SectionHead } - initPm(manager, txpool, engine, blockchain, chaindb) + tds, err := blockchain.GetTrieDbState() + if err != nil { + return nil, err + } + initPm(manager, txpool, engine, blockchain, tds, chaindb) return manager, nil } -func (manager *ProtocolManager) SetDataDir(datadir string) { - manager.datadir = datadir - if manager.downloader != nil { - manager.downloader.SetDataDir(datadir) +func (pm *ProtocolManager) SetDataDir(datadir string) { + pm.datadir = datadir + if pm.downloader != nil { + pm.downloader.SetDataDir(datadir) } } -func initPm(manager *ProtocolManager, txpool txPool, engine consensus.Engine, blockchain *core.BlockChain, chaindb ethdb.Database) { +func initPm(manager *ProtocolManager, txpool txPool, engine consensus.Engine, blockchain *core.BlockChain, tds *state.TrieDbState, chaindb ethdb.Database) { sm, err := GetStorageModeFromDB(chaindb) if err != nil { log.Error("Get storage mode", "err", err) @@ -223,6 +230,7 @@ func initPm(manager *ProtocolManager, txpool txPool, engine consensus.Engine, bl manager.txFetcher = fetcher.NewTxFetcher(txpool.Has, txpool.AddRemotes, fetchTx) } manager.chainSync = newChainSyncer(manager) + manager.mgr = NewMgr(mgr.NewSchedule(tds), tds) } func (pm *ProtocolManager) makeDebugProtocol() p2p.Protocol { @@ -1000,13 +1008,19 @@ func (pm *ProtocolManager) handleDebugMsg(p *debugPeer) error { pm.blockchain.ChainDb().Close() blockchain, err := core.NewBlockChain(ethDb, nil, chainConfig, engine, vm.Config{}, nil, nil) if err != nil { - return fmt.Errorf("NewBlockChain: %w", err) + return fmt.Errorf("fail in NewBlockChain: %w", err) } pm.blockchain.Stop() pm.blockchain = blockchain pm.forkFilter = forkid.NewFilter(pm.blockchain) - initPm(pm, pm.txpool, pm.blockchain.Engine(), pm.blockchain, pm.blockchain.ChainDb()) + + tds, err := pm.blockchain.GetTrieDbState() + if err != nil { + return fmt.Errorf("fail in GetTrieDbState: %w", err) + } + initPm(pm, pm.txpool, pm.blockchain.Engine(), pm.blockchain, tds, pm.blockchain.ChainDb()) pm.quitSync = make(chan struct{}) + remotedbserver.StartDeprecated(ethDb.AbstractKV(), "") // hack to make UI work. But need to somehow re-create whole Node or Ethereum objects // hacks to speedup local sync diff --git a/eth/mgr.go b/eth/mgr.go index f80e2844ba71b74fc6e513e311cd28c2113a351e..c305f79d22b0d8b05fac631c21c4a4242829600b 100644 --- a/eth/mgr.go +++ b/eth/mgr.go @@ -1,6 +1,10 @@ package eth import ( + "io" + "sync" + + "github.com/ledgerwatch/turbo-geth/eth/mgr" "github.com/ledgerwatch/turbo-geth/p2p" ) @@ -34,3 +38,75 @@ func (p *mgrPeer) SendByteCode(id uint64, data [][]byte) error { msg := bytecodeMsg{ID: id, Code: data} return p2p.Send(p.rw, BytecodeCode, msg) } + +type nodeState interface { + GetBlockNr() uint64 +} + +type mgrBroadcast struct { + lock sync.RWMutex + peers map[string]*peer + schedule *mgr.Schedule + state nodeState +} + +func NewMgr(schedule *mgr.Schedule, nodeState nodeState) *mgrBroadcast { + return &mgrBroadcast{schedule: schedule, state: nodeState} +} + +func (m *mgrBroadcast) Start() { + for { + block := m.state.GetBlockNr() + tick, err := m.schedule.Tick(block) + if err != nil { + panic(err) + } + //witnessCache := map[string][]byte{} + for m.state.GetBlockNr() <= tick.ToBlock { + // Produce and Broadcast witness of slice + + //retain := trie.NewRetainRange(common.CopyBytes(slice.From), common.CopyBytes(slice.To)) + //if tick.IsLastInCycle() { + // fmt.Printf("\nretain: %s\n", retain) + //} + //witness, err2 := tds.Trie().ExtractWitness(false, retain) + //if err2 != nil { + // panic(err2) + //} + // + //buf.Reset() + //_, err = witness.WriteTo(&buf) + //if err != nil { + // panic(err) + //} + } + } +} + +func (m *mgrBroadcast) AddPeer(p *peer) { + m.lock.Lock() + defer m.lock.Unlock() + m.peers[p.id] = p +} + +func (m *mgrBroadcast) RemovePeer(id string) { + m.lock.Lock() + defer m.lock.Unlock() + delete(m.peers, id) +} + +func (m *mgrBroadcast) Peer(id string) *peer { + m.lock.RLock() + defer m.lock.RUnlock() + return m.peers[id] +} + +func (m *mgrBroadcast) Broadcast(witness io.Reader) { + m.lock.RLock() + defer m.lock.RUnlock() + for _, p := range m.peers { + if err := p.rw.WriteMsg(p2p.Msg{Code: MGRWitness, Size: 0, Payload: witness}); err != nil { + p.Log().Debug("MGR message sending failed", "err", err) + } + } +} diff --git a/eth/mgr/mgr.go b/eth/mgr/mgr.go index aa0c52ffde36f185b33be0a89e4929c3cbb76a3a..4ec8e48976eafb2db6276825e9a4ed56fe4a061f 100644 --- a/eth/mgr/mgr.go +++ b/eth/mgr/mgr.go @@ -3,9 +3,7 @@ package mgr import ( "fmt" - "github.com/ledgerwatch/turbo-geth/common" - "github.com/ledgerwatch/turbo-geth/ethdb" - "github.com/ledgerwatch/turbo-geth/trie" + "github.com/ledgerwatch/turbo-geth/common/dbutils" ) const ( @@ -16,56 +14,31 @@ const ( BytesPerWitness uint64 = 1024 * 1024 ) -type Schedule struct { - Ticks []Tick -} - type Tick struct { - Number uint64 - FromBlock uint64 - ToBlock uint64 - FromSize uint64 - ToSize uint64 - StateSizeSlices []StateSizeSlice -} - -type StateSizeSlice struct { - FromSize uint64 - ToSize uint64 -} - -type StateSlice struct { - From []byte - To []byte + Number uint64 + FromSize uint64 + ToSize uint64 + From []byte + To []byte + FromBlock uint64 + ToBlock uint64 } -func (s Schedule) String() string { return fmt.Sprintf("Schedule{Ticks:%s}", s.Ticks) } func (t Tick) String() string { - return fmt.Sprintf("Tick{%d,Blocks:%d-%d,Sizes:%d-%d,Slices:%d}", t.Number, t.FromBlock, t.ToBlock, t.FromSize, t.ToSize, t.StateSizeSlices) + return fmt.Sprintf("Tick{%d,Blocks:%d-%d,Sizes:%d-%d,Prefixes:%x-%x}", t.Number, t.FromBlock, t.ToBlock, t.FromSize, t.ToSize, t.From, t.To) } -func (ss StateSlice) String() string { return fmt.Sprintf("{%x-%x}", ss.From, ss.To) } func (t Tick) IsLastInCycle() bool { return t.Number == TicksPerCycle-1 } -func NewStateSchedule(stateSize, fromBlock, toBlock uint64) Schedule { - schedule := Schedule{} - - for fromBlock <= toBlock { - tick := NewTick(fromBlock, stateSize) - schedule.Ticks = append(schedule.Ticks, tick) - fromBlock = tick.ToBlock + 1 - } - - return schedule -} - -func NewTick(blockNr, stateSize uint64) Tick { +// NewTick constructor building Tick object and calculating all state-size-parameters +// not filling exact keys: from, to +func NewTick(blockNr, stateSize uint64, prevTick *Tick) *Tick { number := blockNr / BlocksPerTick % TicksPerCycle fromSize := number * stateSize / TicksPerCycle - tick := Tick{ + tick := &Tick{ Number: number, FromBlock: blockNr, ToBlock: blockNr - blockNr%BlocksPerTick + BlocksPerTick - 1, @@ -73,64 +46,53 @@ func NewTick(blockNr, stateSize uint64) Tick { ToSize: fromSize + stateSize/TicksPerCycle - 1, } - for i := uint64(0); ; i++ { - ss := StateSizeSlice{ - FromSize: tick.FromSize + i*BytesPerWitness, - ToSize: min(tick.FromSize+(i+1)*BytesPerWitness-1, tick.ToSize), - } - - tick.StateSizeSlices = append(tick.StateSizeSlices, ss) - if ss.ToSize >= tick.ToSize { - break - } + if tick.Number != 0 && prevTick != nil { + prevTick.FromSize = prevTick.ToSize + 1 } return tick } -func min(a, b uint64) uint64 { - if a < b { - return a - } - return b +type Schedule struct { + estimator WitnessEstimator + prevTick *Tick + stateSize uint64 } -// Temporary unoptimal implementation. Get existing short prefixes from trie, then resolve range, and give long prefixes from trie. -func StateSizeSlice2StateSlice(db ethdb.Database, tr *trie.Trie, in StateSizeSlice) (StateSlice, error) { - out := StateSlice{} +type WitnessEstimator interface { + TotalCumulativeWitnessSize() (uint64, error) + PrefixByCumulativeWitnessSize(from []byte, size uint64) (prefix []byte, err error) +} - out.From, _ = tr.PrefixByCumulativeWitnessSize(in.FromSize) - out.To, _ = tr.PrefixByCumulativeWitnessSize(in.ToSize) +func NewSchedule(estimator WitnessEstimator) *Schedule { + return &Schedule{estimator: estimator} +} - retain := trie.NewRetainRange(common.CopyBytes(out.From), common.CopyBytes(out.To)) - if err := _resolve(db, tr, retain); err != nil { - return out, err +// Tick - next step of MGR Schedule. Calculating range of keys of valid size +// +// Important: ticks are cycled. When `TicksPerCycle` reached - it starts from beginning of state. +// Means tick.FromSize > prevTick.ToSize - only when tick.Number != 0 +func (s *Schedule) Tick(block uint64) (*Tick, error) { + var err error + if s.prevTick == nil { + s.stateSize, err = s.estimator.TotalCumulativeWitnessSize() + if err != nil { + return nil, err + } } - var found bool - out.From, found = tr.PrefixByCumulativeWitnessSize(in.FromSize) - if !found { - panic(fmt.Sprintf("why? %x\n", out.From)) - } - out.To, found = tr.PrefixByCumulativeWitnessSize(in.ToSize) - if !found { - panic(fmt.Sprintf("why? %x\n", out.To)) + tick := NewTick(block, s.stateSize, s.prevTick) + var prevKey []byte + if tick.Number != 0 && s.prevTick != nil { + prevKey = s.prevTick.To } - - return out, nil -} - -func _resolve(db ethdb.Database, tr *trie.Trie, decider trie.RetainDecider) error { - loader := trie.NewSubTrieLoader(0) - dbPrefixes, fixedbits, hooks := tr.FindSubTriesToLoad(decider) - subTries, err := loader.LoadSubTries(db, 0, decider, dbPrefixes, fixedbits, false) - if err != nil { - return err + tick.From, _ = dbutils.NextSubtree(prevKey) + if tick.To, err = s.estimator.PrefixByCumulativeWitnessSize(tick.From, tick.ToSize-tick.FromSize); err != nil { + return tick, err } - //fmt.Printf("retain: %s\n", decider) - //fmt.Printf("dbPrefixes: %x, %d\n", dbPrefixes, len(subTries.Hashes)) - if err := tr.HookSubTries(subTries, hooks); err != nil { - return err + if tick.From == nil { + tick.From = []byte{} } - return nil + s.prevTick = tick + return tick, nil } diff --git a/eth/mgr/mgr_test.go b/eth/mgr/mgr_test.go index ca2225b0f8ba2b7b6fb4c75c8e8e63b71db51471..187504fd407a0222aeb2a0e47b385e82f88ed8dc 100644 --- a/eth/mgr/mgr_test.go +++ b/eth/mgr/mgr_test.go @@ -11,31 +11,16 @@ func TestScheduleProperties(t *testing.T) { require := require.New(t) stateSize := uint64(123456) block := uint64(11) + toBlock := block + mgr.BlocksPerCycle + 100 + var prevTick *mgr.Tick - var prevTick mgr.Tick - var prevSlice mgr.StateSizeSlice - - var sizeFromSlicesAccumulator uint64 var sizeFromTicksAccumulator uint64 - schedule := mgr.NewStateSchedule(stateSize, block, block+mgr.BlocksPerCycle+100) - for i := range schedule.Ticks { - tick := schedule.Ticks[i] - for j := range tick.StateSizeSlices { - ss := tick.StateSizeSlices[j] - sizeFromSlicesAccumulator += ss.ToSize - ss.FromSize - - // props - if prevTick.Number < tick.Number { // because cycles are... cycled - require.Less(ss.FromSize, ss.ToSize) - require.LessOrEqual(prevSlice.ToSize, ss.FromSize) - } + for block <= toBlock { + tick := mgr.NewTick(block, stateSize, prevTick) - prevSlice = ss - } sizeFromTicksAccumulator += tick.ToSize - tick.FromSize - // props - if prevTick.Number < tick.Number { + if prevTick != nil && prevTick.Number < tick.Number { require.LessOrEqual(block, tick.FromBlock) require.Less(tick.FromBlock, tick.ToBlock) require.Less(prevTick.ToBlock, tick.FromBlock) @@ -46,6 +31,6 @@ func TestScheduleProperties(t *testing.T) { } prevTick = tick + block = tick.ToBlock + 1 } - } diff --git a/ethdb/bolt_db.go b/ethdb/bolt_db.go index 0c20ffb7f4dc583309581daed6913f58e5a8d20f..9587315e391ca62b2ef50634e733e9c386504e04 100644 --- a/ethdb/bolt_db.go +++ b/ethdb/bolt_db.go @@ -26,6 +26,7 @@ import ( "github.com/ledgerwatch/bolt" "github.com/ledgerwatch/turbo-geth/common" "github.com/ledgerwatch/turbo-geth/common/dbutils" + "github.com/ledgerwatch/turbo-geth/common/debug" "github.com/ledgerwatch/turbo-geth/core/types/accounts" "github.com/ledgerwatch/turbo-geth/log" ) @@ -76,6 +77,9 @@ func NewBoltDatabase(file string) (*BoltDatabase, error) { }); err != nil { return nil, err } + if debug.IsTrackWitnessSizeEnabled() { + db.NoSync = true + } return &BoltDatabase{ db: db, diff --git a/internal/debug/flags.go b/internal/debug/flags.go index 001f8e022083455f79df183613fc950a639a0a02..7dfcb6a34c8ed12ef3eaf4cce68b2c2140e7d97b 100644 --- a/internal/debug/flags.go +++ b/internal/debug/flags.go @@ -276,23 +276,21 @@ func Setup(ctx *cli.Context) error { } // pprof server - if ctx.GlobalBool(pprofFlag.Name) { - listenHost := ctx.GlobalString(pprofAddrFlag.Name) - if ctx.GlobalIsSet(legacyPprofAddrFlag.Name) && !ctx.GlobalIsSet(pprofAddrFlag.Name) { - listenHost = ctx.GlobalString(legacyPprofAddrFlag.Name) - log.Warn("The flag --pprofaddr is deprecated and will be removed in the future, please use --pprof.addr") - } - - port := ctx.GlobalInt(pprofPortFlag.Name) - if ctx.GlobalIsSet(legacyPprofPortFlag.Name) && !ctx.GlobalIsSet(pprofPortFlag.Name) { - port = ctx.GlobalInt(legacyPprofPortFlag.Name) - log.Warn("The flag --pprofport is deprecated and will be removed in the future, please use --pprof.port") - Exit() - } + listenHost := ctx.GlobalString(pprofAddrFlag.Name) + if ctx.GlobalIsSet(legacyPprofAddrFlag.Name) && !ctx.GlobalIsSet(pprofAddrFlag.Name) { + listenHost = ctx.GlobalString(legacyPprofAddrFlag.Name) + log.Warn("The flag --pprofaddr is deprecated and will be removed in the future, please use --pprof.addr") + } - address := fmt.Sprintf("%s:%d", listenHost, port) - StartPProf(ctx.GlobalBool(pprofFlag.Name), metrics.Enabled, address) + port := ctx.GlobalInt(pprofPortFlag.Name) + if ctx.GlobalIsSet(legacyPprofPortFlag.Name) && !ctx.GlobalIsSet(pprofPortFlag.Name) { + port = ctx.GlobalInt(legacyPprofPortFlag.Name) + log.Warn("The flag --pprofport is deprecated and will be removed in the future, please use --pprof.port") + Exit() } + + address := fmt.Sprintf("%s:%d", listenHost, port) + StartPProf(ctx.GlobalBool(pprofFlag.Name), metrics.Enabled, address) return nil } diff --git a/internal/ethapi/get_proof.go b/internal/ethapi/get_proof.go index d29e21834c5ce6391bfd76f0aaf669aed026f264..38d10973ffd513a84f4e36390d417d87ea840476 100644 --- a/internal/ethapi/get_proof.go +++ b/internal/ethapi/get_proof.go @@ -129,7 +129,7 @@ func (s *PublicBlockChainAPI) GetProof(ctx context.Context, address common.Addre } sort.Strings(unfurlList) loader := trie.NewFlatDbSubTrieLoader() - if err = loader.Reset(db, unfurl, [][]byte{nil}, []int{0}, false); err != nil { + if err = loader.Reset(db, unfurl, unfurl, [][]byte{nil}, []int{0}, false); err != nil { return nil, err } r := &Receiver{defaultReceiver: trie.NewDefaultReceiver(), unfurlList: unfurlList, accountMap: accountMap, storageMap: storageMap} diff --git a/trie/flatdb_sub_trie_loader.go b/trie/flatdb_sub_trie_loader.go index dbda82d7e7507e6e3b427c3d115b31ec9452e999..dd54b418634f439f40b3ca571782347259d7e79e 100644 --- a/trie/flatdb_sub_trie_loader.go +++ b/trie/flatdb_sub_trie_loader.go @@ -31,7 +31,7 @@ type StreamReceiver interface { storageValue []byte, hash []byte, cutoff int, - witnessLen uint64, + witnessSize uint64, ) error Result() SubTries @@ -52,9 +52,9 @@ type FlatDbSubTrieLoader struct { ihK, ihV []byte minKeyAsNibbles bytes.Buffer - itemPresent bool - itemType StreamItem - getWitnessLen func(prefix []byte) uint64 + itemPresent bool + itemType StreamItem + getWitnessSize func(prefix []byte) uint64 // Storage item buffer storageKeyPart1 []byte @@ -66,7 +66,7 @@ type FlatDbSubTrieLoader struct { accountValue accounts.Account hashValue []byte streamCutoff int - witnessLen uint64 + witnessSize uint64 receiver StreamReceiver defaultReceiver *DefaultReceiver @@ -90,7 +90,7 @@ type DefaultReceiver struct { a accounts.Account leafData GenStructStepLeafData accData GenStructStepAccountData - witnessLen uint64 + witnessSize uint64 } func NewDefaultReceiver() *DefaultReceiver { @@ -105,8 +105,8 @@ func NewFlatDbSubTrieLoader() *FlatDbSubTrieLoader { } // Reset prepares the loader for reuse -func (fstl *FlatDbSubTrieLoader) Reset(db ethdb.Getter, rl RetainDecider, dbPrefixes [][]byte, fixedbits []int, trace bool) error { - fstl.defaultReceiver.Reset(rl, trace) +func (fstl *FlatDbSubTrieLoader) Reset(db ethdb.Getter, rl RetainDecider, receiverDecider RetainDecider, dbPrefixes [][]byte, fixedbits []int, trace bool) error { + fstl.defaultReceiver.Reset(receiverDecider, trace) fstl.receiver = fstl.defaultReceiver fstl.rangeIdx = 0 @@ -294,7 +294,8 @@ func (fstl *FlatDbSubTrieLoader) iteration(c, ih *bolt.Cursor, first bool) error } copy(fstl.accAddrHashWithInc[:], fstl.k) binary.BigEndian.PutUint64(fstl.accAddrHashWithInc[32:], ^fstl.accountValue.Incarnation) - // Now we know the correct incarnation of the account, and we can skip all irrelevant storage records + // Now we know the correct incarnation of the account, an + //d we can skip all irrelevant storage records // Since 0 incarnation if 0xfff...fff, and we do not expect any records like that, this automatically // skips over all storage items fstl.k, fstl.v = c.SeekTo(fstl.accAddrHashWithInc[:]) @@ -356,18 +357,18 @@ func (fstl *FlatDbSubTrieLoader) iteration(c, ih *bolt.Cursor, first bool) error } fstl.hashValue = fstl.ihV fstl.storageValue = nil - fstl.witnessLen = fstl.getWitnessLen(fstl.ihK) + fstl.witnessSize = fstl.getWitnessSize(fstl.ihK) } else { fstl.itemType = AHashStreamItem fstl.accountKey = fstl.ihK fstl.storageKeyPart1 = nil fstl.storageKeyPart2 = nil fstl.hashValue = fstl.ihV - fstl.witnessLen = fstl.getWitnessLen(fstl.ihK) + fstl.witnessSize = fstl.getWitnessSize(fstl.ihK) } // skip subtree - next, ok := nextSubtree(fstl.ihK) + next, ok := dbutils.NextSubtree(fstl.ihK) if !ok { // no siblings left if !retain { // last sub-tree was taken from IH, then nothing to look in the main bucket. Can stop. fstl.k = nil @@ -438,7 +439,7 @@ func (dr *DefaultReceiver) Receive(itemType StreamItem, storageValue []byte, hash []byte, cutoff int, - witnessLen uint64, + witnessSize uint64, ) error { switch itemType { case StorageStreamItem: @@ -448,7 +449,7 @@ func (dr *DefaultReceiver) Receive(itemType StreamItem, return err } } - dr.saveValueStorage(false, storageValue, hash, witnessLen) + dr.saveValueStorage(false, storageValue, hash, witnessSize) case SHashStreamItem: dr.advanceKeysStorage(storageKeyPart1, storageKeyPart2, false /* terminator */) if dr.currStorage.Len() > 0 { @@ -456,7 +457,7 @@ func (dr *DefaultReceiver) Receive(itemType StreamItem, return err } } - dr.saveValueStorage(true, storageValue, hash, witnessLen) + dr.saveValueStorage(true, storageValue, hash, witnessSize) case AccountStreamItem: dr.advanceKeysAccount(accountKey, true /* terminator */) if dr.curr.Len() > 0 && !dr.wasIH { @@ -485,7 +486,7 @@ func (dr *DefaultReceiver) Receive(itemType StreamItem, return err } } - if err := dr.saveValueAccount(false, accountValue, hash, witnessLen); err != nil { + if err := dr.saveValueAccount(false, accountValue, hash, witnessSize); err != nil { return err } case AHashStreamItem: @@ -516,7 +517,7 @@ func (dr *DefaultReceiver) Receive(itemType StreamItem, return err } } - if err := dr.saveValueAccount(true, accountValue, hash, witnessLen); err != nil { + if err := dr.saveValueAccount(true, accountValue, hash, witnessSize); err != nil { return err } case CutoffStreamItem: @@ -606,14 +607,14 @@ func (fstl *FlatDbSubTrieLoader) LoadSubTries() (SubTries, error) { if err := fstl.boltDB.View(func(tx *bolt.Tx) error { c := tx.Bucket(dbutils.CurrentStateBucket).Cursor() ih := tx.Bucket(dbutils.IntermediateTrieHashBucket).Cursor() - iwl := tx.Bucket(dbutils.IntermediateTrieWitnessLenBucket).Cursor() - fstl.getWitnessLen = func(prefix []byte) uint64 { + iwl := tx.Bucket(dbutils.IntermediateWitnessSizeBucket).Cursor() + fstl.getWitnessSize = func(prefix []byte) uint64 { if !debug.IsTrackWitnessSizeEnabled() { return 0 } k, v := iwl.SeekTo(prefix) if !bytes.Equal(k, prefix) { - panic(fmt.Sprintf("IH and DataLen buckets must have same keys set: %x, %x", k, prefix)) + panic(fmt.Sprintf("IH and WitnessSize buckets must have same keys set: %x, %x", k, prefix)) } return binary.BigEndian.Uint64(v) } @@ -627,7 +628,7 @@ func (fstl *FlatDbSubTrieLoader) LoadSubTries() (SubTries, error) { } } if fstl.itemPresent { - if err := fstl.receiver.Receive(fstl.itemType, fstl.accountKey, fstl.storageKeyPart1, fstl.storageKeyPart2, &fstl.accountValue, fstl.storageValue, fstl.hashValue, fstl.streamCutoff, fstl.witnessLen); err != nil { + if err := fstl.receiver.Receive(fstl.itemType, fstl.accountKey, fstl.storageKeyPart1, fstl.storageKeyPart2, &fstl.accountValue, fstl.storageValue, fstl.hashValue, fstl.streamCutoff, fstl.witnessSize); err != nil { return err } fstl.itemPresent = false @@ -718,7 +719,7 @@ func (dr *DefaultReceiver) genStructStorage() error { var data GenStructStepData if dr.wasIHStorage { dr.hashData.Hash = common.BytesToHash(dr.valueStorage.Bytes()) - dr.hashData.DataLen = dr.witnessLen + dr.hashData.WitnessSize = dr.witnessSize data = &dr.hashData } else { dr.leafData.Value = rlphacks.RlpSerializableBytes(dr.valueStorage.Bytes()) @@ -731,13 +732,13 @@ func (dr *DefaultReceiver) genStructStorage() error { return nil } -func (dr *DefaultReceiver) saveValueStorage(isIH bool, v, h []byte, witnessLen uint64) { +func (dr *DefaultReceiver) saveValueStorage(isIH bool, v, h []byte, witnessSize uint64) { // Remember the current value dr.wasIHStorage = isIH dr.valueStorage.Reset() if isIH { dr.valueStorage.Write(h) - dr.witnessLen = witnessLen + dr.witnessSize = witnessSize } else { dr.valueStorage.Write(v) } @@ -770,7 +771,7 @@ func (dr *DefaultReceiver) genStructAccount() error { var data GenStructStepData if dr.wasIH { copy(dr.hashData.Hash[:], dr.value.Bytes()) - dr.hashData.DataLen = dr.witnessLen + dr.hashData.WitnessSize = dr.witnessSize data = &dr.hashData } else { dr.accData.Balance.Set(&dr.a.Balance) @@ -795,12 +796,12 @@ func (dr *DefaultReceiver) genStructAccount() error { return nil } -func (dr *DefaultReceiver) saveValueAccount(isIH bool, v *accounts.Account, h []byte, witnessLen uint64) error { +func (dr *DefaultReceiver) saveValueAccount(isIH bool, v *accounts.Account, h []byte, witnessSize uint64) error { dr.wasIH = isIH if isIH { dr.value.Reset() dr.value.Write(h) - dr.witnessLen = witnessLen + dr.witnessSize = witnessSize return nil } dr.a.Copy(v) @@ -815,21 +816,6 @@ func (dr *DefaultReceiver) saveValueAccount(isIH bool, v *accounts.Account, h [] return nil } -// nextSubtree does []byte++. Returns false if overflow. -func nextSubtree(in []byte) ([]byte, bool) { - r := make([]byte, len(in)) - copy(r, in) - for i := len(r) - 1; i >= 0; i-- { - if r[i] != 255 { - r[i]++ - return r, true - } - - r[i] = 0 - } - return nil, false -} - func nextAccount(in, out []byte) bool { copy(out, in) for i := len(out) - 1; i >= 0; i-- { diff --git a/trie/flatdb_sub_trie_loader_test.go b/trie/flatdb_sub_trie_loader_test.go index be381b872c202033211093755eef63862c11c656..1245d010908e3c226c9b4f171f8f32211a6a1c9c 100644 --- a/trie/flatdb_sub_trie_loader_test.go +++ b/trie/flatdb_sub_trie_loader_test.go @@ -261,7 +261,7 @@ func TestApiDetails(t *testing.T) { putIDataLen := func(k string, v uint64) { lenBytes := make([]byte, 8) binary.BigEndian.PutUint64(lenBytes, v) - require.NoError(db.Put(dbutils.IntermediateTrieWitnessLenBucket, common.Hex2Bytes(k), lenBytes)) + require.NoError(db.Put(dbutils.IntermediateWitnessSizeBucket, common.Hex2Bytes(k), lenBytes)) } // Test attempt handle cases when: Trie root hash is same for Cached and non-Cached SubTrieLoaders @@ -333,8 +333,8 @@ func TestApiDetails(t *testing.T) { //fmt.Printf("%x\n", tr.root.(*fullNode).Children[0].(*fullNode).Children[0].reference()) //fmt.Printf("%x\n", tr.root.(*fullNode).Children[15].(*fullNode).Children[15].reference()) - //fmt.Printf("%d\n", tr.root.(*fullNode).Children[0].(*fullNode).Children[0].witnessLen()) - //fmt.Printf("%d\n", tr.root.(*fullNode).Children[15].(*fullNode).Children[15].witnessLen()) + //fmt.Printf("%d\n", tr.root.(*fullNode).Children[0].(*fullNode).Children[0].witnessSize()) + //fmt.Printf("%d\n", tr.root.(*fullNode).Children[15].(*fullNode).Children[15].witnessSize()) _, found := tr.GetAccount(hexf("000%061x", 0)) assert.False(found) // exists in DB but resolved, there is hashNode @@ -425,10 +425,10 @@ func TestApiDetails(t *testing.T) { err = tr.HookSubTries(subTries, hooks) // hook up to the root assert.NoError(err) - assert.Equal(uint64(2050), tr.root.witnessLen()) - assert.Equal(uint64(1024), tr.root.(*fullNode).Children[0].witnessLen()) - assert.Equal(uint64(254), tr.root.(*fullNode).Children[0].(*fullNode).Children[0].witnessLen()) - assert.Equal(uint64(256), tr.root.(*fullNode).Children[15].(*fullNode).Children[15].witnessLen()) + assert.Equal(uint64(2050), tr.root.witnessSize()) + assert.Equal(uint64(1024), tr.root.(*fullNode).Children[0].witnessSize()) + assert.Equal(uint64(254), tr.root.(*fullNode).Children[0].(*fullNode).Children[0].witnessSize()) + assert.Equal(uint64(256), tr.root.(*fullNode).Children[15].(*fullNode).Children[15].witnessSize()) witness, err := tr.ExtractWitness(false, rs) if err != nil { @@ -440,7 +440,7 @@ func TestApiDetails(t *testing.T) { if err != nil { panic(err) } - fmt.Printf("WitnessLen. Real: %d, Estimate: %d\n", buf.Len(), tr.root.witnessLen()) + fmt.Printf("WitnessSize. Real: %d, Estimate: %d\n", buf.Len(), tr.root.witnessSize()) } } diff --git a/trie/gen_struct_step.go b/trie/gen_struct_step.go index a02adbe2f38f31df723f1daeb1505a41c387bda7..906b4ba9f61095b621bbe2467295805deddb5cbb 100644 --- a/trie/gen_struct_step.go +++ b/trie/gen_struct_step.go @@ -67,8 +67,8 @@ type GenStructStepLeafData struct { func (GenStructStepLeafData) GenStructStepData() {} type GenStructStepHashData struct { - Hash common.Hash - DataLen uint64 + Hash common.Hash + WitnessSize uint64 } func (GenStructStepHashData) GenStructStepData() {} @@ -128,7 +128,7 @@ func GenStructStep( switch v := data.(type) { case *GenStructStepHashData: /* building a hash */ - if err := e.hash(v.Hash[:], v.DataLen); err != nil { + if err := e.hash(v.Hash[:], v.WitnessSize); err != nil { return nil, err } buildExtensions = true diff --git a/trie/hashbuilder.go b/trie/hashbuilder.go index 5d8effbc0136e3a9395e2bb2fc70762c8ef68ab0..02b2b22d0b18d45a646a2a67924c73b566c5755f 100644 --- a/trie/hashbuilder.go +++ b/trie/hashbuilder.go @@ -17,6 +17,8 @@ import ( ) const hashStackStride = common.HashLength + 1 // + 1 byte for RLP encoding +const CountWitnessSizeWithoutStructure = true + var EmptyCodeHash = crypto.Keccak256Hash(nil) // HashBuilder implements the interface `structInfoReceiver` and opcodes that the structural information of the trie @@ -80,7 +82,7 @@ func (hb *HashBuilder) leaf(length int, keyHex []byte, val rlphacks.RlpSerializa s.ref.len = hb.hashStack[len(hb.hashStack)-common.HashLength-1] - 0xc0 + 1 copy(s.ref.data[:], hb.hashStack[len(hb.hashStack)-common.HashLength-1:]) } - s.witnessLength = hb.dataLenStack[len(hb.dataLenStack)-1] + s.iws = hb.dataLenStack[len(hb.dataLenStack)-1] if hb.trace { fmt.Printf("Stack depth: %d, %d\n", len(hb.nodeStack), len(hb.dataLenStack)) @@ -123,7 +125,11 @@ func (hb *HashBuilder) leafHashWithKeyVal(key []byte, val rlphacks.RlpSerializab if err != nil { return err } - hb.dataLenStack = append(hb.dataLenStack, uint64(len(val.RawBytes()))+1+uint64(len(key))/2) // + node opcode + len(key)/2 + if CountWitnessSizeWithoutStructure { + hb.dataLenStack = append(hb.dataLenStack, uint64(len(val.RawBytes()))) + } else { + hb.dataLenStack = append(hb.dataLenStack, uint64(len(val.RawBytes()))+1+uint64(len(key))/2) // + node opcode + len(key)/2 + } hb.hashStack = append(hb.hashStack, hb.hashBuf[:]...) if len(hb.hashStack) > hashStackStride*len(hb.nodeStack) { @@ -213,7 +219,7 @@ func (hb *HashBuilder) accountLeaf(length int, keyHex []byte, balance *uint256.I root = hb.nodeStack[len(hb.nodeStack)-popped-1] l := hb.dataLenStack[len(hb.dataLenStack)-popped-1] if root == nil { - root = hashNode{hash: common.CopyBytes(hb.acc.Root[:]), witnessLength: l} + root = hashNode{hash: common.CopyBytes(hb.acc.Root[:]), iws: l} } } popped++ @@ -251,7 +257,7 @@ func (hb *HashBuilder) accountLeaf(length int, keyHex []byte, balance *uint256.I } copy(s.ref.data[:], hb.hashStack[len(hb.hashStack)-common.HashLength:]) s.ref.len = 32 - s.witnessLength = hb.dataLenStack[len(hb.dataLenStack)-1] + s.iws = hb.dataLenStack[len(hb.dataLenStack)-1] // Replace top of the stack hb.nodeStack[len(hb.nodeStack)-1] = s if hb.trace { @@ -327,8 +333,13 @@ func (hb *HashBuilder) accountLeafHashWithKey(key []byte, popped int) error { if err != nil { return err } - dataLen := 1 + uint64(hb.acc.EncodingLengthForStorage()) // + opcode + account data len - dataLen += 1 + uint64(len(key))/2 // + opcode + len(key)/2 + dataLen := uint64(0) + if CountWitnessSizeWithoutStructure { + dataLen = uint64(hb.acc.EncodingLengthForStorage()) + } else { + dataLen = 1 + uint64(hb.acc.EncodingLengthForStorage()) // + opcode + account data len + dataLen += 1 + uint64(len(key))/2 // + opcode + len(key)/2 + } if popped > 0 { hb.hashStack = hb.hashStack[:len(hb.hashStack)-popped*hashStackStride] @@ -357,7 +368,7 @@ func (hb *HashBuilder) extension(key []byte) error { case nil: branchHash := common.CopyBytes(hb.hashStack[len(hb.hashStack)-common.HashLength:]) dataLen := hb.dataLenStack[len(hb.dataLenStack)-1] - s = &shortNode{Key: common.CopyBytes(key), Val: hashNode{hash: branchHash, witnessLength: dataLen}} + s = &shortNode{Key: common.CopyBytes(key), Val: hashNode{hash: branchHash, iws: dataLen}} case *fullNode: s = &shortNode{Key: common.CopyBytes(key), Val: n} default: @@ -369,7 +380,7 @@ func (hb *HashBuilder) extension(key []byte) error { } copy(s.ref.data[:], hb.hashStack[len(hb.hashStack)-common.HashLength:]) s.ref.len = 32 - s.witnessLength = hb.dataLenStack[len(hb.dataLenStack)-1] + s.iws = hb.dataLenStack[len(hb.dataLenStack)-1] if hb.trace { fmt.Printf("Stack depth: %d, %d\n", len(hb.nodeStack), len(hb.dataLenStack)) } @@ -438,7 +449,11 @@ func (hb *HashBuilder) extensionHash(key []byte) error { return err } hb.hashStack[len(hb.hashStack)-hashStackStride] = 0x80 + common.HashLength - hb.dataLenStack[len(hb.dataLenStack)-1] = 1 + uint64(len(key))/2 + hb.dataLenStack[len(hb.dataLenStack)-1] // + opcode + len(key)/2 + childrenWitnessLen + if CountWitnessSizeWithoutStructure { + hb.dataLenStack[len(hb.dataLenStack)-1] = 0 + hb.dataLenStack[len(hb.dataLenStack)-1] // + opcode + len(key)/2 + childrenWitnessSize + } else { + hb.dataLenStack[len(hb.dataLenStack)-1] = 1 + uint64(len(key))/2 + hb.dataLenStack[len(hb.dataLenStack)-1] // + opcode + len(key)/2 + childrenWitnessSize + } if _, ok := hb.nodeStack[len(hb.nodeStack)-1].(*fullNode); ok { return fmt.Errorf("extensionHash cannot be emitted when a node is on top of the stack") } @@ -464,7 +479,7 @@ func (hb *HashBuilder) branch(set uint16) error { for digit := uint(0); digit < 16; digit++ { if ((uint16(1) << digit) & set) != 0 { if nodes[i] == nil { - f.Children[digit] = hashNode{hash: common.CopyBytes(hashes[hashStackStride*i+1 : hashStackStride*(i+1)]), witnessLength: dataLengths[i]} + f.Children[digit] = hashNode{hash: common.CopyBytes(hashes[hashStackStride*i+1 : hashStackStride*(i+1)]), iws: dataLengths[i]} } else { f.Children[digit] = nodes[i] } @@ -478,7 +493,7 @@ func (hb *HashBuilder) branch(set uint16) error { } copy(f.ref.data[:], hb.hashStack[len(hb.hashStack)-common.HashLength:]) f.ref.len = 32 - f.witnessLength = hb.dataLenStack[len(hb.dataLenStack)-1] + f.iws = hb.dataLenStack[len(hb.dataLenStack)-1] if hb.trace { fmt.Printf("Stack depth: %d, %d\n", len(hb.nodeStack), len(hb.dataLenStack)) } @@ -542,12 +557,21 @@ func (hb *HashBuilder) branchHash(set uint16) error { if _, err := hb.sha.Read(hb.hashStack[len(hb.hashStack)-common.HashLength:]); err != nil { return err } - dataLen := uint64(1 + 1) // fullNode: opcode + mask + childrenWitnessLen - for _, l := range hb.dataLenStack[len(hb.dataLenStack)-digits:] { - dataLen += l + if CountWitnessSizeWithoutStructure { + dataLen := uint64(0) + for _, l := range hb.dataLenStack[len(hb.dataLenStack)-digits:] { + dataLen += l + } + hb.dataLenStack[len(hb.dataLenStack)-digits] = dataLen + hb.dataLenStack = hb.dataLenStack[:len(hb.dataLenStack)-digits+1] + } else { + dataLen := uint64(1 + 1) // fullNode: opcode + mask + childrenWitnessSize + for _, l := range hb.dataLenStack[len(hb.dataLenStack)-digits:] { + dataLen += l + } + hb.dataLenStack[len(hb.dataLenStack)-digits] = dataLen + hb.dataLenStack = hb.dataLenStack[:len(hb.dataLenStack)-digits+1] } - hb.dataLenStack[len(hb.dataLenStack)-digits] = dataLen - hb.dataLenStack = hb.dataLenStack[:len(hb.dataLenStack)-digits+1] if hashStackStride*len(hb.nodeStack) > len(hb.hashStack) { hb.nodeStack = hb.nodeStack[:len(hb.nodeStack)-digits+1] @@ -584,7 +608,11 @@ func (hb *HashBuilder) code(code []byte) error { codeCopy := common.CopyBytes(code) n := codeNode(codeCopy) hb.nodeStack = append(hb.nodeStack, n) - hb.dataLenStack = append(hb.dataLenStack, 1+uint64(len(code))) // opcode + len(code) + if CountWitnessSizeWithoutStructure { + hb.dataLenStack = append(hb.dataLenStack, 0) + } else { + hb.dataLenStack = append(hb.dataLenStack, 0) + } hb.sha.Reset() if _, err := hb.sha.Write(codeCopy); err != nil { return err diff --git a/trie/node.go b/trie/node.go index bedccb1a5d8930ca702b2bccc96e1d21f87c5442..db3a07f22a19f55fee4e66925ee21a416836c2e3 100644 --- a/trie/node.go +++ b/trie/node.go @@ -36,34 +36,34 @@ type node interface { // if not empty, returns node's RLP or hash thereof reference() []byte - witnessLen() uint64 + witnessSize() uint64 } type ( // DESCRIBED: docs/programmers_guide/guide.md#hexary-radix-patricia-tree fullNode struct { - ref nodeRef - Children [17]node // Actual trie node data to encode/decode (needs custom encoder) - witnessLength uint64 // amount of bytes holded in DB by all storage/code under this prefix. equal to sum of children's witnessLength + ref nodeRef + Children [17]node // Actual trie node data to encode/decode (needs custom encoder) + iws uint64 // IntermediateWitnessSize - amount of bytes used in DB by all storage/code under this prefix + few bytes for Trie structure encoding. equal to sum of children's iws } // DESCRIBED: docs/programmers_guide/guide.md#hexary-radix-patricia-tree duoNode struct { - ref nodeRef - mask uint32 // Bitmask. The set bits indicate the child is not nil - child1 node - child2 node - witnessLength uint64 + ref nodeRef + mask uint32 // Bitmask. The set bits indicate the child is not nil + child1 node + child2 node + iws uint64 } // DESCRIBED: docs/programmers_guide/guide.md#hexary-radix-patricia-tree shortNode struct { - ref nodeRef - Key []byte // HEX encoding - Val node - witnessLength uint64 + ref nodeRef + Key []byte // HEX encoding + Val node + iws uint64 } hashNode struct { - hash []byte - witnessLength uint64 + hash []byte + iws uint64 } valueNode []byte @@ -83,11 +83,16 @@ type ( var nilValueNode = valueNode(nil) func NewShortNode(key []byte, value node) *shortNode { - return &shortNode{ - Key: key, - Val: value, - witnessLength: 1 + uint64(1)/2 + value.witnessLen(), //opcode + len(key)/2 + childrenWitnessLen + s := &shortNode{ + Key: key, + Val: value, + iws: 1 + uint64(1)/2 + value.witnessSize(), //opcode + len(key)/2 + childrenWitnessSize } + if CountWitnessSizeWithoutStructure { + s.iws = value.witnessSize() + } + + return s } func EncodeAsValue(data []byte) ([]byte, error) { @@ -238,50 +243,94 @@ func (n *duoNode) reference() []byte { return n.ref.data[0:n.ref.len] } func (n *shortNode) reference() []byte { return n.ref.data[0:n.ref.len] } func (an *accountNode) reference() []byte { return nil } -// WitnessLen calculation logic: +// WitnessSize calculation logic: // hashNode: represents full underlying witness // valueNode: opcode + len(storage) // codeNode: opcode + len(code) -// fullNode: opcode + mask + childrenWitnessLen -// duoNode: opcode + mask + childrenWitnessLen -// shortNode: opcode + len(key)/2 + childrenWitnessLen -// accountNode: opcode + account data len + codeWitnessLen + storageWitnessLen -func (n hashNode) witnessLen() uint64 { return n.witnessLength } -func (n valueNode) witnessLen() uint64 { return uint64(len(n)) + 1 } -func (n codeNode) witnessLen() uint64 { return uint64(len(n)) + 1 } -func (n *fullNode) witnessLen() uint64 { return n.witnessLength } -func (n *duoNode) witnessLen() uint64 { return n.witnessLength } -func (n *shortNode) witnessLen() uint64 { return n.witnessLength } -func (an *accountNode) witnessLen() uint64 { +// fullNode: opcode + mask + childrenWitnessSize +// duoNode: opcode + mask + childrenWitnessSize +// shortNode: opcode + len(key)/2 + childrenWitnessSize +// accountNode: opcode + account data len + storageWitnessSize - could not include codeSize because it's not available yet +func (n hashNode) witnessSize() uint64 { return n.iws } +func (n valueNode) witnessSize() uint64 { + if CountWitnessSizeWithoutStructure { + return uint64(len(n)) + } + return uint64(len(n)) + 1 +} +func (n codeNode) witnessSize() uint64 { + if CountWitnessSizeWithoutStructure { + //return uint64(len(n)) + return 0 + } + return uint64(len(n)) + 1 +} +func (n *fullNode) witnessSize() uint64 { return n.iws } +func (n *duoNode) witnessSize() uint64 { return n.iws } +func (n *shortNode) witnessSize() uint64 { return n.iws } +func (an *accountNode) witnessSize() uint64 { + if CountWitnessSizeWithoutStructure { + res := uint64(an.EncodingLengthForStorage()) + if an.storage != nil { + res += an.storage.witnessSize() + } + return res + } + res := 1 + uint64(an.EncodingLengthForStorage()) if an.storage != nil { - res += an.storage.witnessLen() - } - if an.code != nil { - res += an.code.witnessLen() + res += an.storage.witnessSize() } + //if an.code != nil { + // res += an.code.witnessSize() + //} return res } -func (n *fullNode) recalculateWitnessLen() { - n.witnessLength = 1 + 1 // opcode + mask + childrenWitnessLen +func (n *fullNode) recalculateWitnessSize() { + if CountWitnessSizeWithoutStructure { + n.iws = 0 + for j := range n.Children { + if n.Children[j] != nil { + n.iws += n.Children[j].witnessSize() + } + } + return + } + n.iws = 1 + 1 // opcode + mask + childrenWitnessSize for j := range n.Children { if n.Children[j] != nil { - n.witnessLength += n.Children[j].witnessLen() + n.iws += n.Children[j].witnessSize() } } } -func (n *duoNode) recalculateWitnessLen() { - n.witnessLength = 1 + 1 // opcode + mask + childrenWitnessLen +func (n *duoNode) recalculateWitnessSize() { + if CountWitnessSizeWithoutStructure { + n.iws = 0 + if n.child1 != nil { + n.iws += n.child1.witnessSize() + } + if n.child2 != nil { + n.iws += n.child2.witnessSize() + } + return + } + + n.iws = 1 + 1 // opcode + mask + childrenWitnessSize if n.child1 != nil { - n.witnessLength += n.child1.witnessLen() + n.iws += n.child1.witnessSize() } if n.child2 != nil { - n.witnessLength += n.child2.witnessLen() + n.iws += n.child2.witnessSize() } } -func (n *shortNode) recalculateWitnessLen() { - n.witnessLength = 1 + 1 + uint64(len(n.Key))/2 + n.Val.witnessLen() // opcode + len(key)/2 + childrenWitnessLen +func (n *shortNode) recalculateWitnessSize() { + if CountWitnessSizeWithoutStructure { + n.iws = n.Val.witnessSize() + return + } + + n.iws = 1 + 1 + uint64(len(n.Key))/2 + n.Val.witnessSize() // opcode + len(key)/2 + childrenWitnessSize } // Pretty printing. diff --git a/trie/retain_list.go b/trie/retain_list.go index 6dd95cc1959de4e616e2735e4ad2b1cb9d945c8d..aa260ba8ad3fc768eca1daa911f51eb3bb30e066 100644 --- a/trie/retain_list.go +++ b/trie/retain_list.go @@ -185,3 +185,67 @@ func (rr *RetainRange) String() string { return fmt.Sprintf("%x-%x", rr.from, rr.to) } +// RetainAll - returns true to any prefix +type RetainAll struct { + codeTouches map[common.Hash]struct{} + decider RetainDecider +} + +// NewRetainAll creates new NewRetainRange +// to=nil - means no upper bound +func NewRetainAll(decider RetainDecider) *RetainAll { + return &RetainAll{decider: decider, codeTouches: make(map[common.Hash]struct{})} +} + +func (rr *RetainAll) Retain(prefix []byte) (retain bool) { + return true +} + +// AddCodeTouch adds a new code touch into the resolve set +func (rr *RetainAll) AddCodeTouch(codeHash common.Hash) { + rr.codeTouches[codeHash] = struct{}{} +} + +func (rr *RetainAll) IsCodeTouched(codeHash common.Hash) bool { + _, ok := rr.codeTouches[codeHash] + return ok +} + +func (rr *RetainAll) String() string { + return "" +} + +// RetainLevels - returns true to any prefix shorter than `levels` +type RetainLevels struct { + codeTouches map[common.Hash]struct{} + decider RetainDecider + levels int +} + +// NewRetainLevels creates new NewRetainRange +// to=nil - means no upper bound +func NewRetainLevels(decider RetainDecider, levels int) *RetainLevels { + return &RetainLevels{decider: decider, codeTouches: make(map[common.Hash]struct{}), levels: levels} +} + +func (rr *RetainLevels) Retain(prefix []byte) (retain bool) { + if len(prefix) > rr.levels { + return rr.decider.Retain(prefix) + } + + return true +} + +// AddCodeTouch adds a new code touch into the resolve set +func (rr *RetainLevels) AddCodeTouch(codeHash common.Hash) { + rr.codeTouches[codeHash] = struct{}{} +} + +func (rr *RetainLevels) IsCodeTouched(codeHash common.Hash) bool { + _, ok := rr.codeTouches[codeHash] + return ok +} + +func (rr *RetainLevels) String() string { + return "" +} diff --git a/trie/stream.go b/trie/stream.go index 89143c659750021eb18af7816d4554d03c6559e0..04b1c867c4843cfd0491e9b75af84cfcfe16390c 100644 --- a/trie/stream.go +++ b/trie/stream.go @@ -160,7 +160,7 @@ func (it *Iterator) Next() (itemType StreamItem, hex1 []byte, aValue *accounts.A fmt.Printf("valueNode %x\n", hex) } it.top-- - return StorageStreamItem, hex, nil, nil, []byte(n), n.witnessLen() + return StorageStreamItem, hex, nil, nil, n, n.witnessSize() case *shortNode: if it.trace { fmt.Printf("shortNode %x\n", hex) @@ -170,12 +170,12 @@ func (it *Iterator) Next() (itemType StreamItem, hex1 []byte, aValue *accounts.A case hashNode: it.top-- if accounts { - return AHashStreamItem, hex, nil, v.hash, nil, v.witnessLength + return AHashStreamItem, hex, nil, v.hash, nil, v.witnessSize() } - return SHashStreamItem, hex, nil, v.hash, nil, v.witnessLength + return SHashStreamItem, hex, nil, v.hash, nil, v.witnessSize() case valueNode: it.top-- - return StorageStreamItem, hex, nil, nil, v, v.witnessLen() + return StorageStreamItem, hex, nil, nil, v, v.witnessSize() case *accountNode: if it.trace { fmt.Printf("accountNode %x\n", hex) @@ -190,7 +190,7 @@ func (it *Iterator) Next() (itemType StreamItem, hex1 []byte, aValue *accounts.A } else { it.top-- } - return AccountStreamItem, hex, &v.Account, nil, nil, v.witnessLen() + return AccountStreamItem, hex, &v.Account, nil, nil, v.witnessSize() } it.hex = hex it.nodeStack[l] = n.Val @@ -205,9 +205,9 @@ func (it *Iterator) Next() (itemType StreamItem, hex1 []byte, aValue *accounts.A if !goDeep { it.top-- if accounts { - return AHashStreamItem, hex, nil, n.reference(), nil, n.witnessLen() + return AHashStreamItem, hex, nil, n.reference(), nil, n.witnessSize() } - return SHashStreamItem, hex, nil, n.reference(), nil, n.witnessLen() + return SHashStreamItem, hex, nil, n.reference(), nil, n.witnessSize() } i1, i2 := n.childrenIdx() index := it.iStack[l] @@ -218,24 +218,24 @@ func (it *Iterator) Next() (itemType StreamItem, hex1 []byte, aValue *accounts.A switch v := n.child1.(type) { case hashNode: if accounts { - return AHashStreamItem, hex, nil, v.hash, nil, v.witnessLength + return AHashStreamItem, hex, nil, v.hash, nil, v.witnessSize() } - return SHashStreamItem, hex, nil, v.hash, nil, v.witnessLength + return SHashStreamItem, hex, nil, v.hash, nil, v.witnessSize() case *duoNode: childGoDeep = it.rl.Retain(hex) if !childGoDeep { if accounts { - return AHashStreamItem, hex, nil, v.reference(), nil, v.witnessLength + return AHashStreamItem, hex, nil, v.reference(), nil, v.witnessSize() } - return SHashStreamItem, hex, nil, v.reference(), nil, v.witnessLength + return SHashStreamItem, hex, nil, v.reference(), nil, v.witnessSize() } case *fullNode: childGoDeep = it.rl.Retain(hex) if !childGoDeep { if accounts { - return AHashStreamItem, hex, nil, v.reference(), nil, v.witnessLength + return AHashStreamItem, hex, nil, v.reference(), nil, v.witnessSize() } - return SHashStreamItem, hex, nil, v.reference(), nil, v.witnessLength + return SHashStreamItem, hex, nil, v.reference(), nil, v.witnessSize() } } it.hex = hex @@ -261,26 +261,26 @@ func (it *Iterator) Next() (itemType StreamItem, hex1 []byte, aValue *accounts.A case hashNode: it.top-- if accounts { - return AHashStreamItem, hex, nil, v.hash, nil, v.witnessLength + return AHashStreamItem, hex, nil, v.hash, nil, v.witnessSize() } - return SHashStreamItem, hex, nil, v.hash, nil, v.witnessLength + return SHashStreamItem, hex, nil, v.hash, nil, v.witnessSize() case *duoNode: childGoDeep = it.rl.Retain(hex) if !childGoDeep { it.top-- if accounts { - return AHashStreamItem, hex, nil, v.reference(), nil, v.witnessLength + return AHashStreamItem, hex, nil, v.reference(), nil, v.witnessSize() } - return SHashStreamItem, hex, nil, v.reference(), nil, v.witnessLength + return SHashStreamItem, hex, nil, v.reference(), nil, v.witnessSize() } case *fullNode: childGoDeep = it.rl.Retain(hex) if !childGoDeep { it.top-- if accounts { - return AHashStreamItem, hex, nil, v.reference(), nil, v.witnessLength + return AHashStreamItem, hex, nil, v.reference(), nil, v.witnessSize() } - return SHashStreamItem, hex, nil, v.reference(), nil, v.witnessLength + return SHashStreamItem, hex, nil, v.reference(), nil, v.witnessSize() } } it.hex = hex @@ -297,9 +297,9 @@ func (it *Iterator) Next() (itemType StreamItem, hex1 []byte, aValue *accounts.A if !goDeep { it.top-- if accounts { - return AHashStreamItem, hex, nil, n.reference(), nil, n.witnessLength + return AHashStreamItem, hex, nil, n.reference(), nil, n.witnessSize() } - return SHashStreamItem, hex, nil, n.reference(), nil, n.witnessLength + return SHashStreamItem, hex, nil, n.reference(), nil, n.witnessSize() } var i1, i2 int i1Found := false @@ -324,24 +324,24 @@ func (it *Iterator) Next() (itemType StreamItem, hex1 []byte, aValue *accounts.A switch v := n.Children[i1].(type) { case hashNode: if accounts { - return AHashStreamItem, hex, nil, v.hash, nil, v.witnessLength + return AHashStreamItem, hex, nil, v.hash, nil, v.witnessSize() } - return SHashStreamItem, hex, nil, v.hash, nil, v.witnessLength + return SHashStreamItem, hex, nil, v.hash, nil, v.witnessSize() case *duoNode: childGoDeep = it.rl.Retain(hex) if !childGoDeep { if accounts { - return AHashStreamItem, hex, nil, v.reference(), nil, v.witnessLength + return AHashStreamItem, hex, nil, v.reference(), nil, v.witnessSize() } - return SHashStreamItem, hex, nil, v.reference(), nil, v.witnessLength + return SHashStreamItem, hex, nil, v.reference(), nil, v.witnessSize() } case *fullNode: childGoDeep = it.rl.Retain(hex) if !childGoDeep { if accounts { - return AHashStreamItem, hex, nil, v.reference(), nil, v.witnessLength + return AHashStreamItem, hex, nil, v.reference(), nil, v.witnessSize() } - return SHashStreamItem, hex, nil, v.reference(), nil, v.witnessLength + return SHashStreamItem, hex, nil, v.reference(), nil, v.witnessSize() } } it.hex = hex @@ -367,26 +367,26 @@ func (it *Iterator) Next() (itemType StreamItem, hex1 []byte, aValue *accounts.A case hashNode: it.top-- if accounts { - return AHashStreamItem, hex, nil, v.hash, nil, v.witnessLength + return AHashStreamItem, hex, nil, v.hash, nil, v.witnessSize() } - return SHashStreamItem, hex, nil, v.hash, nil, v.witnessLength + return SHashStreamItem, hex, nil, v.hash, nil, v.witnessSize() case *duoNode: childGoDeep = it.rl.Retain(hex) if !childGoDeep { it.top-- if accounts { - return AHashStreamItem, hex, nil, v.reference(), nil, v.witnessLength + return AHashStreamItem, hex, nil, v.reference(), nil, v.witnessSize() } - return SHashStreamItem, hex, nil, v.reference(), nil, v.witnessLength + return SHashStreamItem, hex, nil, v.reference(), nil, v.witnessSize() } case *fullNode: childGoDeep = it.rl.Retain(hex) if !childGoDeep { it.top-- if accounts { - return AHashStreamItem, hex, nil, v.reference(), nil, v.witnessLength + return AHashStreamItem, hex, nil, v.reference(), nil, v.witnessSize() } - return SHashStreamItem, hex, nil, v.reference(), nil, v.witnessLength + return SHashStreamItem, hex, nil, v.reference(), nil, v.witnessSize() } } it.hex = hex @@ -410,16 +410,16 @@ func (it *Iterator) Next() (itemType StreamItem, hex1 []byte, aValue *accounts.A } else { it.top-- } - return AccountStreamItem, hex, &n.Account, nil, nil, n.witnessLen() + return AccountStreamItem, hex, &n.Account, nil, nil, n.witnessSize() case hashNode: if it.trace { fmt.Printf("hashNode %x\n", hex) } it.top-- if accounts { - return AHashStreamItem, hex, nil, n.hash, nil, n.witnessLength + return AHashStreamItem, hex, nil, n.hash, nil, n.witnessSize() } - return SHashStreamItem, hex, nil, n.hash, nil, n.witnessLength + return SHashStreamItem, hex, nil, n.hash, nil, n.witnessSize() default: panic(fmt.Errorf("unexpected node: %T", nd)) } @@ -428,21 +428,21 @@ func (it *Iterator) Next() (itemType StreamItem, hex1 []byte, aValue *accounts.A // StreamMergeIterator merges an Iterator and a Stream type StreamMergeIterator struct { - it *Iterator - s *Stream - trace bool - ki, ai, si int - offset int - hex []byte - itemType StreamItem - oldItemType StreamItem - oldHex []byte - oldHexCopy []byte - oldAVal *accounts.Account - oldHash []byte - oldHashCopy []byte - oldVal []byte - oldDataLen uint64 + it *Iterator + s *Stream + trace bool + ki, ai, si int + offset int + hex []byte + itemType StreamItem + oldItemType StreamItem + oldHex []byte + oldHexCopy []byte + oldAVal *accounts.Account + oldHash []byte + oldHashCopy []byte + oldVal []byte + oldWitnessSize uint64 } // NewStreamMergeIterator create a brand new StreamMergeIterator @@ -452,7 +452,7 @@ func NewStreamMergeIterator(it *Iterator, s *Stream, trace bool) *StreamMergeIte s: s, trace: trace, } - smi.oldItemType, smi.oldHex, smi.oldAVal, smi.oldHash, smi.oldVal, smi.oldDataLen = it.Next() + smi.oldItemType, smi.oldHex, smi.oldAVal, smi.oldHash, smi.oldVal, smi.oldWitnessSize = it.Next() return smi } @@ -465,11 +465,11 @@ func (smi *StreamMergeIterator) Reset(it *Iterator, s *Stream, trace bool) { smi.ai = 0 smi.si = 0 smi.offset = 0 - smi.oldItemType, smi.oldHex, smi.oldAVal, smi.oldHash, smi.oldVal, smi.oldDataLen = it.Next() + smi.oldItemType, smi.oldHex, smi.oldAVal, smi.oldHash, smi.oldVal, smi.oldWitnessSize = it.Next() } // Next returns the next item in the merged iterator -func (smi *StreamMergeIterator) Next() (itemType1 StreamItem, hex1 []byte, aValue *accounts.Account, aCode []byte, hash []byte, value []byte, dataLen uint64) { +func (smi *StreamMergeIterator) Next() (itemType1 StreamItem, hex1 []byte, aValue *accounts.Account, aCode []byte, hash []byte, value []byte, witnessSize uint64) { for { if smi.trace { fmt.Printf("smi.hex %x, smi.ki %d, len(smi.s.keySizes) %d, smi.oldItemType %d smi.oldHex %x\n", smi.hex, smi.ki, len(smi.s.keySizes), smi.oldItemType, smi.oldHex) @@ -498,7 +498,7 @@ func (smi *StreamMergeIterator) Next() (itemType1 StreamItem, hex1 []byte, aValu si := smi.si smi.si++ if len(smi.s.sValues[si]) > 0 { - return StorageStreamItem, hex, nil, nil, nil, smi.s.sValues[si], uint64(len(smi.s.sValues[si])) + return StorageStreamItem, hex, nil, nil, nil, smi.s.sValues[si], 0 } default: panic(fmt.Errorf("unexpected stream item type (oldHex == nil): %d", smi.itemType)) @@ -515,12 +515,13 @@ func (smi *StreamMergeIterator) Next() (itemType1 StreamItem, hex1 []byte, aValu } smi.oldHashCopy = append(smi.oldHashCopy, smi.oldHash...) oldVal := smi.oldVal - smi.oldItemType, smi.oldHex, smi.oldAVal, smi.oldHash, smi.oldVal, smi.oldDataLen = smi.it.Next() - return oldItemType, smi.oldHexCopy, oldAVal, nil, smi.oldHashCopy, oldVal, uint64(len(oldVal)) + oldWitnessSize := smi.oldWitnessSize + smi.oldItemType, smi.oldHex, smi.oldAVal, smi.oldHash, smi.oldVal, smi.oldWitnessSize = smi.it.Next() + return oldItemType, smi.oldHexCopy, oldAVal, nil, smi.oldHashCopy, oldVal, oldWitnessSize } else { // Special case - account gets deleted if smi.itemType == AccountStreamItem && smi.s.aValues[smi.ai] == nil && bytes.HasPrefix(smi.oldHex, hex) { - smi.oldItemType, smi.oldHex, smi.oldAVal, smi.oldHash, smi.oldVal, smi.oldDataLen = smi.it.Next() + smi.oldItemType, smi.oldHex, smi.oldAVal, smi.oldHash, smi.oldVal, smi.oldWitnessSize = smi.it.Next() } else { switch bytes.Compare(smi.oldHex, hex) { case -1: @@ -535,8 +536,9 @@ func (smi *StreamMergeIterator) Next() (itemType1 StreamItem, hex1 []byte, aValu } smi.oldHashCopy = append(smi.oldHashCopy, smi.oldHash...) oldVal := smi.oldVal - smi.oldItemType, smi.oldHex, smi.oldAVal, smi.oldHash, smi.oldVal, smi.oldDataLen = smi.it.Next() - return oldItemType, smi.oldHexCopy, oldAVal, nil, smi.oldHashCopy, oldVal, uint64(len(oldVal)) + oldWitnessSize := smi.oldWitnessSize + smi.oldItemType, smi.oldHex, smi.oldAVal, smi.oldHash, smi.oldVal, smi.oldWitnessSize = smi.it.Next() + return oldItemType, smi.oldHexCopy, oldAVal, nil, smi.oldHashCopy, oldVal, oldWitnessSize case 1: smi.hex = nil // To be consumed switch smi.itemType { @@ -550,14 +552,14 @@ func (smi *StreamMergeIterator) Next() (itemType1 StreamItem, hex1 []byte, aValu si := smi.si smi.si++ if len(smi.s.sValues[si]) > 0 { - return StorageStreamItem, hex, nil, nil, nil, smi.s.sValues[si], uint64(len(smi.s.sValues[si])) + return StorageStreamItem, hex, nil, nil, nil, smi.s.sValues[si], 0 } default: panic(fmt.Errorf("unexpected stream item type (oldHex == nil): %d", smi.itemType)) } case 0: smi.hex = nil // To be consumed - smi.oldItemType, smi.oldHex, smi.oldAVal, smi.oldHash, smi.oldVal, smi.oldDataLen = smi.it.Next() + smi.oldItemType, smi.oldHex, smi.oldAVal, smi.oldHash, smi.oldVal, smi.oldWitnessSize = smi.it.Next() switch smi.itemType { case AccountStreamItem: ai := smi.ai @@ -569,7 +571,7 @@ func (smi *StreamMergeIterator) Next() (itemType1 StreamItem, hex1 []byte, aValu si := smi.si smi.si++ if len(smi.s.sValues[si]) > 0 { - return StorageStreamItem, hex, nil, nil, nil, smi.s.sValues[si], uint64(len(smi.s.sValues[si])) + return StorageStreamItem, hex, nil, nil, nil, smi.s.sValues[si], 0 } default: panic(fmt.Errorf("unexpected stream item type (oldHex == nil): %d", smi.itemType)) @@ -592,7 +594,7 @@ func StreamHash(it *StreamMergeIterator, storagePrefixLen int, hb *HashBuilder, var hashRef []byte var hashRefStorage []byte var groups, sGroups []uint16 // Separate groups slices for storage items and for accounts - var dataLenAcc, dataLenStorage uint64 + var witnessSizeAcc, witnessSizeStorage uint64 var aRoot common.Hash var aEmptyRoot = true var isAccount bool @@ -606,10 +608,10 @@ func StreamHash(it *StreamMergeIterator, storagePrefixLen int, hb *HashBuilder, curr.Reset() currStorage.Reset() - makeData := func(fieldSet uint32, hashRef []byte, dataLen uint64) GenStructStepData { + makeData := func(fieldSet uint32, hashRef []byte, witnessSize uint64) GenStructStepData { if hashRef != nil { copy(hashData.Hash[:], hashRef) - hashData.DataLen = dataLen + hashData.WitnessSize = witnessSize return &hashData } else if !isAccount { leafData.Value = rlphacks.RlpSerializableBytes(value.Bytes()) @@ -621,7 +623,7 @@ func StreamHash(it *StreamMergeIterator, storagePrefixLen int, hb *HashBuilder, } retain := func(_ []byte) bool { return trace } - for newItemType, hex, aVal, aCode, hash, val, dataLen := it.Next(); newItemType != NoItem; newItemType, hex, aVal, aCode, hash, val, dataLen = it.Next() { + for newItemType, hex, aVal, aCode, hash, val, witnessSize := it.Next(); newItemType != NoItem; newItemType, hex, aVal, aCode, hash, val, witnessSize = it.Next() { if newItemType == AccountStreamItem || newItemType == AHashStreamItem { // If there was an open storage "sub-stream", close it and set the storage flag on if succStorage.Len() > 0 { @@ -631,7 +633,7 @@ func StreamHash(it *StreamMergeIterator, storagePrefixLen int, hb *HashBuilder, if currStorage.Len() > 0 { isAccount = false var err error - sGroups, err = GenStructStep(retain, currStorage.Bytes(), succStorage.Bytes(), hb, makeData(0, hashRefStorage, dataLenStorage), sGroups, trace) + sGroups, err = GenStructStep(retain, currStorage.Bytes(), succStorage.Bytes(), hb, makeData(0, hashRefStorage, witnessSizeStorage), sGroups, trace) if err != nil { return common.Hash{}, err } @@ -639,7 +641,7 @@ func StreamHash(it *StreamMergeIterator, storagePrefixLen int, hb *HashBuilder, fieldSet += AccountFieldStorageOnly } } else if itemType == AccountStreamItem && !aEmptyRoot { - if err := hb.hash(aRoot[:], dataLen); err != nil { + if err := hb.hash(aRoot[:], witnessSize); err != nil { return common.Hash{}, err } fieldSet += AccountFieldStorageOnly @@ -651,13 +653,13 @@ func StreamHash(it *StreamMergeIterator, storagePrefixLen int, hb *HashBuilder, if curr.Len() > 0 { isAccount = true var err error - groups, err = GenStructStep(retain, curr.Bytes(), succ.Bytes(), hb, makeData(fieldSet, hashRef, dataLenAcc), groups, trace) + groups, err = GenStructStep(retain, curr.Bytes(), succ.Bytes(), hb, makeData(fieldSet, hashRef, witnessSizeAcc), groups, trace) if err != nil { return common.Hash{}, err } } itemType = newItemType - dataLenAcc = dataLen + witnessSizeAcc = witnessSize switch itemType { case AccountStreamItem: var a *accounts.Account = aVal @@ -697,13 +699,13 @@ func StreamHash(it *StreamMergeIterator, storagePrefixLen int, hb *HashBuilder, if currStorage.Len() > 0 { isAccount = false var err error - sGroups, err = GenStructStep(retain, currStorage.Bytes(), succStorage.Bytes(), hb, makeData(0, hashRefStorage, dataLenStorage), sGroups, trace) + sGroups, err = GenStructStep(retain, currStorage.Bytes(), succStorage.Bytes(), hb, makeData(0, hashRefStorage, witnessSizeStorage), sGroups, trace) if err != nil { return common.Hash{}, err } } sItemType = newItemType - dataLenStorage = dataLen + witnessSizeStorage = witnessSize switch sItemType { case StorageStreamItem: value.Reset() @@ -723,7 +725,7 @@ func StreamHash(it *StreamMergeIterator, storagePrefixLen int, hb *HashBuilder, if currStorage.Len() > 0 { isAccount = false var err error - _, err = GenStructStep(retain, currStorage.Bytes(), succStorage.Bytes(), hb, makeData(0, hashRefStorage, dataLenStorage), sGroups, trace) + _, err = GenStructStep(retain, currStorage.Bytes(), succStorage.Bytes(), hb, makeData(0, hashRefStorage, witnessSizeStorage), sGroups, trace) if err != nil { return common.Hash{}, err } @@ -731,7 +733,7 @@ func StreamHash(it *StreamMergeIterator, storagePrefixLen int, hb *HashBuilder, fieldSet |= AccountFieldStorageOnly } } else if itemType == AccountStreamItem && !aEmptyRoot { - if err := hb.hash(aRoot[:], dataLenAcc); err != nil { + if err := hb.hash(aRoot[:], witnessSizeAcc); err != nil { return common.Hash{}, err } fieldSet |= AccountFieldStorageOnly @@ -742,7 +744,7 @@ func StreamHash(it *StreamMergeIterator, storagePrefixLen int, hb *HashBuilder, if curr.Len() > 0 { isAccount = true var err error - _, err = GenStructStep(retain, curr.Bytes(), succ.Bytes(), hb, makeData(fieldSet, hashRef, dataLenAcc), groups, trace) + _, err = GenStructStep(retain, curr.Bytes(), succ.Bytes(), hb, makeData(fieldSet, hashRef, witnessSizeAcc), groups, trace) if err != nil { return common.Hash{}, err } diff --git a/trie/sub_trie_loader.go b/trie/sub_trie_loader.go index 5bc4d20b23c3bb4fb28e7d234cfcaf91b5bca2a9..243953ba3e5809f83ef3553729a52b82adfa408f 100644 --- a/trie/sub_trie_loader.go +++ b/trie/sub_trie_loader.go @@ -61,7 +61,7 @@ func (stl *SubTrieLoader) LoadSubTries(db ethdb.Database, blockNr uint64, rl Ret func (stl *SubTrieLoader) LoadFromFlatDB(db ethdb.Getter, rl RetainDecider, dbPrefixes [][]byte, fixedbits []int, trace bool) (SubTries, error) { loader := NewFlatDbSubTrieLoader() - if err1 := loader.Reset(db, rl, dbPrefixes, fixedbits, trace); err1 != nil { + if err1 := loader.Reset(db, rl, rl, dbPrefixes, fixedbits, trace); err1 != nil { return SubTries{}, err1 } subTries, err := loader.LoadSubTries() diff --git a/trie/trie.go b/trie/trie.go index 7e31e31e1246993d666b280da9f6722f72069c7b..b2ba3f3ca03037a890c90df5031ca09ade1f76fc 100644 --- a/trie/trie.go +++ b/trie/trie.go @@ -579,7 +579,7 @@ func (t *Trie) insertRecursive(origNode node, key []byte, pos int, value node) ( t.evictNodeFromHashMap(n) n.Val = nn n.ref.len = 0 - n.recalculateWitnessLen() + n.recalculateWitnessSize() } newNode = n } else { @@ -606,7 +606,7 @@ func (t *Trie) insertRecursive(origNode node, key []byte, pos int, value node) ( branch.child2 = c1 } branch.mask = (1 << (n.Key[matchlen])) | (1 << (key[pos+matchlen])) - branch.recalculateWitnessLen() + branch.recalculateWitnessSize() // Replace this shortNode with the branch if it occurs at index 0. if matchlen == 0 { @@ -616,7 +616,7 @@ func (t *Trie) insertRecursive(origNode node, key []byte, pos int, value node) ( n.Key = common.CopyBytes(key[pos : pos+matchlen]) n.Val = branch n.ref.len = 0 - n.recalculateWitnessLen() + n.recalculateWitnessSize() newNode = n } t.observers.BranchNodeCreated(key[:pos+matchlen]) @@ -634,7 +634,7 @@ func (t *Trie) insertRecursive(origNode node, key []byte, pos int, value node) ( t.evictNodeFromHashMap(n) n.child1 = nn n.ref.len = 0 - n.recalculateWitnessLen() + n.recalculateWitnessSize() } newNode = n case i2: @@ -643,7 +643,7 @@ func (t *Trie) insertRecursive(origNode node, key []byte, pos int, value node) ( t.evictNodeFromHashMap(n) n.child2 = nn n.ref.len = 0 - n.recalculateWitnessLen() + n.recalculateWitnessSize() } newNode = n default: @@ -658,7 +658,7 @@ func (t *Trie) insertRecursive(origNode node, key []byte, pos int, value node) ( newnode.Children[i1] = n.child1 newnode.Children[i2] = n.child2 newnode.Children[key[pos]] = child - newnode.recalculateWitnessLen() + newnode.recalculateWitnessSize() updated = true // current node leaves the generation but newnode joins it newNode = newnode @@ -677,14 +677,14 @@ func (t *Trie) insertRecursive(origNode node, key []byte, pos int, value node) ( } updated = true n.ref.len = 0 - n.recalculateWitnessLen() + n.recalculateWitnessSize() } else { updated, nn = t.insertRecursive(child, key, pos+1, value) if updated { t.evictNodeFromHashMap(n) n.Children[key[pos]] = nn n.ref.len = 0 - n.recalculateWitnessLen() + n.recalculateWitnessSize() } } newNode = n @@ -1063,7 +1063,7 @@ func (t *Trie) deleteRecursive(origNode node, key []byte, keyStart int, preserve duo.child2 = n.Children[pos2] } duo.mask = (1 << uint(pos1)) | (uint32(1) << uint(pos2)) - duo.witnessLength = duo.child1.witnessLen() + duo.child2.witnessLen() + duo.iws = duo.child1.witnessSize() + duo.child2.witnessSize() newNode = duo } else if count > 2 { t.observers.BranchNodeTouched(key[:keyStart]) @@ -1233,15 +1233,10 @@ func (t *Trie) EvictNode(hex []byte) { return } copy(hn[:], nd.reference()) - hnode := hashNode{hash: hn[:], witnessLength: nd.witnessLen()} + hnode := hashNode{hash: hn[:], iws: nd.witnessSize()} t.observers.WillUnloadNode(hex, hn) - switch nd.(type) { - case *fullNode, *duoNode: - t.observers.WillUnloadBranchNode(hex, hn, incarnation, nd.witnessLen()) - default: - // nothing to do - } + t.notifyUnloadRecursive(hex, incarnation, nd) switch p := parent.(type) { case nil: @@ -1264,6 +1259,47 @@ func (t *Trie) EvictNode(hex []byte) { } } +func (t *Trie) notifyUnloadRecursive(hex []byte, incarnation uint64, nd node) { + switch n := nd.(type) { + case *shortNode: + hex = append(hex, n.Key...) + if hex[len(hex)-1] == 16 { + hex = hex[:len(hex)-1] + } + t.notifyUnloadRecursive(hex, incarnation, n.Val) + case *accountNode: + if n.storage == nil { + return + } + if _, ok := n.storage.(hashNode); ok { + return + } + t.notifyUnloadRecursive(hex, n.Incarnation, n.storage) + case *fullNode: + t.observers.WillUnloadBranchNode(hex, common.BytesToHash(n.reference()), incarnation, n.witnessSize()) + for i := range n.Children { + if n.Children[i] == nil { + continue + } + if _, ok := n.Children[i].(hashNode); ok { + continue + } + t.notifyUnloadRecursive(append(hex, uint8(i)), incarnation, n.Children[i]) + } + case *duoNode: + t.observers.WillUnloadBranchNode(hex, common.BytesToHash(n.reference()), incarnation, n.witnessSize()) + i1, i2 := n.childrenIdx() + if n.child1 != nil { + t.notifyUnloadRecursive(append(hex, i1), incarnation, n.child1) + } + if n.child2 != nil { + t.notifyUnloadRecursive(append(hex, i2), incarnation, n.child2) + } + default: + // nothing to do + } +} + func (t *Trie) TrieSize() int { return calcSubtreeSize(t.root) } @@ -1334,7 +1370,7 @@ func (t *Trie) HashMapSize() int { return len(t.hashMap) } -func (t *Trie) EstimateWitnessSize(key []byte) uint64 { +func (t *Trie) CumulativeWitnessSize(key []byte) uint64 { hex := keybytesToHex(key) if t.binary { hex = keyHexToBin(hex) @@ -1346,21 +1382,21 @@ func (t *Trie) EstimateWitnessSize(key []byte) uint64 { if !ok { return 0 } - return n.witnessLen() + return n.witnessSize() } -// PrefixByCumulativeWitnessSize returns minimal prefix with accumulated size >= than given one. -// Returns (nil, true) if size > root.witnessLen() +// PrefixByCumulativeWitnessSizeDeprecated returns minimal prefix with accumulated size >= than given one. +// Returns (nil, true) if size > root.witnessSize() // Returns (nil, false) if faced nil node in trie -func (t *Trie) PrefixByCumulativeWitnessSize(size uint64) (prefix []byte, found bool) { +func (t *Trie) PrefixByCumulativeWitnessSize(size uint64) (prefix []byte, incarnation uint64, accumulator uint64, found bool) { return prefixGreaterThanWitnessSize(t.root, size) } -func prefixGreaterThanWitnessSize(nd node, size uint64) (prefix []byte, found bool) { - var accumulator uint64 // increase it when go to siblings, don't touch it when go to child - - if nd.witnessLen() < size { - return prefix, true +// prefixGreaterThanWitnessSize - returns lexicographically minimal prefix of maximal length with accumulated witness size >= than given threshold. +// accumulator - increase it when go to siblings, don't touch it when go to child. Doesn't includes last node - means accumulator <= size. +func prefixGreaterThanWitnessSize(nd node, size uint64) (prefix []byte, incarnation uint64, accumulator uint64, found bool) { + if nd.witnessSize() < size { + return prefix, incarnation, accumulator, true } Loop: @@ -1370,74 +1406,76 @@ Loop: found = false break Loop case *shortNode: - accumulator += 1 + uint64(len(n.Key)/2) - //fmt.Printf("short node: %d %d\n", accumulator, accumulator+1+uint64(len(n.Key)/2)) prefix = append(prefix, n.Key...) if prefix[len(prefix)-1] == 0x10 { // remove terminator prefix = prefix[:len(prefix)-1] } nd = n.Val + //fmt.Printf("short node: %d\n", accumulator) case *duoNode: - accumulator += 2 + var acc2 uint64 i1, i2 := n.childrenIdx() - if accumulator+n.child1.witnessLen() >= size { + if accumulator+n.child1.witnessSize() >= size { prefix = append(prefix, i1) nd = n.child1 - //fmt.Printf("duo overflow1: %d %d, %x\n", accumulator, accumulator+nd.witnessLen(), prefix) + //fmt.Printf("duo overflow1: %d+%d=%d <= %d, %x\n", accumulator, n.child1.witnessSize(), accumulator+n.child1.witnessSize(), size, prefix) + continue + } + acc2 += n.child1.witnessSize() + + if accumulator+acc2+n.child2.witnessSize() >= size { + prefix = append(prefix, i2) + nd = n.child2 + //fmt.Printf("duo overflow2: %d+%d+%d=%d <= %d, %x\n", accumulator, acc2, n.child2.witnessSize(), accumulator+acc2+n.child2.witnessSize(), size, prefix) + accumulator += acc2 continue } - accumulator += n.child1.witnessLen() - prefix = append(prefix, i2) - nd = n.child2 //fmt.Printf("duo: %d, %x\n", accumulator, prefix) + break Loop case *fullNode: - accumulator += 2 + var acc2 uint64 + //fmt.Printf("full children %d\n", n.Children) for i := range n.Children { if n.Children[i] == nil { + //fmt.Printf("full nill child %d\n", i) continue } - - if accumulator+n.Children[i].witnessLen() >= size { + //fmt.Printf("full child %d\n", i) + if accumulator+acc2+n.Children[i].witnessSize() >= size { prefix = append(prefix, uint8(i)) nd = n.Children[i] - //fmt.Printf("full overflow: %d %d, %x\n", accumulator, accumulator+nd.witnessLen(), prefix) - break + //fmt.Printf("full overflow(%d): %d+%d+%d=%d <= %d, %x\n", i, accumulator, acc2, n.Children[i].witnessSize(), accumulator+acc2+n.Children[i].witnessSize(), size, prefix) + accumulator += acc2 + //fmt.Printf("full overflow(%d)++\n", accumulator) + continue Loop } - accumulator += n.Children[i].witnessLen() - //fmt.Printf("full: %d, %x\n", accumulator, prefix) + //fmt.Printf("full no overflow (%d): %d\n", i, n.Children[i].witnessSize()) + acc2 += n.Children[i].witnessSize() } + + //fmt.Printf("full: %d+%d=%d<=%d, %x\n", accumulator, acc2, accumulator+acc2, size, prefix) + break Loop case *accountNode: - witnessOverhead := 1 + uint64(n.EncodingLengthForStorage()) - if n.codeSize != -1 { - witnessOverhead += uint64(n.codeSize) - } - //fmt.Printf("accountNode: %d %d\n", accumulator, accumulator+witnessOverhead) - accumulator += witnessOverhead + //fmt.Printf("accountNode: %d, %x\n", accumulator, prefix) if n.storage == nil { found = true break Loop } nd = n.storage + incarnation = n.Incarnation case valueNode: - //fmt.Printf("valueNode: %d %d %x\n", accumulator, accumulator+n.witnessLen(), prefix) - //accumulator += nd.witnessLen() - found = true - break Loop - case codeNode: - //fmt.Printf("codeNode: %d %d %x\n", accumulator, accumulator+n.witnessLen(), prefix) - //accumulator += nd.witnessLen() + //fmt.Printf("valueNode: %d %d %x\n", accumulator, accumulator+n.witnessSize(), prefix) found = true break Loop case hashNode: + //fmt.Printf("hashNode: %x %d %d\n", prefix, accumulator, n.witnessSize()) found = false - //fmt.Printf("hashNode: %x\n", prefix) break Loop default: panic(fmt.Sprintf("Unknown node: %T", n)) } } - //fmt.Printf("Accumulator: %d %d\n", size, accumulator) - return prefix, found + return prefix, incarnation, accumulator, found } diff --git a/trie/trie_observers.go b/trie/trie_observers.go index f66eca9f484af97257a3ac30c639ad527cf4846e..3c39e29e6860b6d839b0758e1c9061fe312e7a37 100644 --- a/trie/trie_observers.go +++ b/trie/trie_observers.go @@ -12,7 +12,7 @@ type Observer interface { CodeNodeTouched(hex []byte) CodeNodeSizeChanged(hex []byte, newSize uint) - WillUnloadBranchNode(key []byte, nodeHash common.Hash, incarnation uint64, witnessLen uint64) + WillUnloadBranchNode(key []byte, nodeHash common.Hash, incarnation uint64, witnessSize uint64) WillUnloadNode(key []byte, nodeHash common.Hash) BranchNodeLoaded(prefixAsNibbles []byte, incarnation uint64) } @@ -99,9 +99,9 @@ func (mux *ObserverMux) WillUnloadNode(key []byte, nodeHash common.Hash) { } } -func (mux *ObserverMux) WillUnloadBranchNode(key []byte, nodeHash common.Hash, incarnation uint64, witnessLen uint64) { +func (mux *ObserverMux) WillUnloadBranchNode(key []byte, nodeHash common.Hash, incarnation uint64, witnessSize uint64) { for _, child := range mux.children { - child.WillUnloadBranchNode(key, nodeHash, incarnation, witnessLen) + child.WillUnloadBranchNode(key, nodeHash, incarnation, witnessSize) } } diff --git a/trie/trie_observers_test.go b/trie/trie_observers_test.go index a98f578bcf7c932bf1d13929e53ab6dc665b10a1..ed634305688520de12040fe3b529f75e6f6964a6 100644 --- a/trie/trie_observers_test.go +++ b/trie/trie_observers_test.go @@ -96,7 +96,7 @@ func (m *mockObserver) CodeNodeSizeChanged(hex []byte, newSize uint) { m.createdNodes[common.Bytes2Hex(hex)] = newSize } -func (m *mockObserver) WillUnloadBranchNode(hex []byte, hash common.Hash, incarnation uint64, witnessLen uint64) { +func (m *mockObserver) WillUnloadBranchNode(hex []byte, hash common.Hash, incarnation uint64, witnessSize uint64) { } func (m *mockObserver) WillUnloadNode(hex []byte, hash common.Hash) { diff --git a/trie/trie_transform.go b/trie/trie_transform.go index dc1a65cd4c6c7aaff32bbc5486b6a0046cc773ca..b97256297918777f475b115c4746b4c62b77f6d6 100644 --- a/trie/trie_transform.go +++ b/trie/trie_transform.go @@ -31,7 +31,7 @@ func transformSubTrie(nd node, hex []byte, newTrie *Trie, transformFunc keyTrans } transformSubTrie(n.storage, aHex, newTrie, transformFunc) case hashNode: - _, newTrie.root = newTrie.insert(newTrie.root, transformFunc(hex), hashNode{hash: common.CopyBytes(n.hash), witnessLength: n.witnessLength}) + _, newTrie.root = newTrie.insert(newTrie.root, transformFunc(hex), hashNode{hash: common.CopyBytes(n.hash), iws: n.iws}) return case *shortNode: var hexVal []byte diff --git a/trie/visual.go b/trie/visual.go index 1a3bd38fa8316092ef97c7314d12eab0f2132fe8..b80519e0770dca91d9ce9085e11b0c2351439c25 100644 --- a/trie/visual.go +++ b/trie/visual.go @@ -285,7 +285,7 @@ func fold(nd node, hexes [][]byte, h *hasher, isRoot bool) (bool, node) { if bytes.Equal(n.Key, hex) { var hn common.Hash h.hash(n, isRoot, hn[:]) - return true, hashNode{hash: hn[:], witnessLength: n.witnessLength} + return true, hashNode{hash: hn[:], iws: n.iws} } pLen := prefixLen(n.Key, hex) if pLen > 0 { @@ -298,7 +298,7 @@ func fold(nd node, hexes [][]byte, h *hasher, isRoot bool) (bool, node) { if folded { var hn common.Hash h.hash(n, isRoot, hn[:]) - return true, hashNode{hash: hn[:], witnessLength: n.witnessLength} + return true, hashNode{hash: hn[:], iws: n.iws} } return false, n } @@ -326,7 +326,7 @@ func fold(nd node, hexes [][]byte, h *hasher, isRoot bool) (bool, node) { if folded1 && folded2 { var hn common.Hash h.hash(n, isRoot, hn[:]) - return true, hashNode{hash: hn[:], witnessLength: n.witnessLength} + return true, hashNode{hash: hn[:], iws: n.iws} } return false, n case *fullNode: @@ -354,7 +354,7 @@ func fold(nd node, hexes [][]byte, h *hasher, isRoot bool) (bool, node) { if !unfolded { var hn common.Hash h.hash(n, isRoot, hn[:]) - return true, hashNode{hash: hn[:], witnessLength: n.witnessLength} + return true, hashNode{hash: hn[:], iws: n.iws} } return false, n } diff --git a/trie/witness_builder.go b/trie/witness_builder.go index b420621530b8b55e515e6c76265293640bf11096..961de4dfa66a237819723d720e7026312706d283 100644 --- a/trie/witness_builder.go +++ b/trie/witness_builder.go @@ -98,7 +98,7 @@ func (b *WitnessBuilder) makeHashNode(n node, force bool, hashNodeFunc HashNodeF if _, err := hashNodeFunc(n, force, hash[:]); err != nil { return hashNode{}, err } - return hashNode{hash: hash[:], witnessLength: n.witnessLen()}, nil + return hashNode{hash: hash[:], iws: n.witnessSize()}, nil } } @@ -154,7 +154,7 @@ func (b *WitnessBuilder) processAccountCode(n *accountNode, retainDec RetainDeci } if n.code == nil || (retainDec != nil && !retainDec.IsCodeTouched(n.CodeHash)) { - return b.addHashOp(hashNode{hash: n.CodeHash[:], witnessLength: uint64(n.codeSize)}) + return b.addHashOp(hashNode{hash: n.CodeHash[:], iws: uint64(n.codeSize)}) } return b.addCodeOp(n.code)