mirror of
https://gitlab.com/pulsechaincom/erigon-pulse.git
synced 2024-12-27 22:28:21 +00:00
implement bin_patricia_hashed trie (#430)
* commitment: implemented semi-working bin patricia trie * commitment: added initialize function to select commitment implementation * deleted reference implementation of binary trie * added branch merge function selection in accordance with current commitment type * smarter branch prefix convolution to reduce disk usage * implemented DELETE update * commitment/bin-trie: fixed merge processing and storage encoding * added changed hex to bin patricia trie * fixed trie variant select * allocate if bufPos larger than buf size * added tracing code * Fix lint * Skip test Co-authored-by: Alexey Sharp <alexeysharp@Alexeys-iMac.local>
This commit is contained in:
parent
c18fea3a8c
commit
abd93fe9c9
@ -22,6 +22,7 @@ import (
|
||||
"container/heap"
|
||||
"context"
|
||||
"encoding/binary"
|
||||
"encoding/hex"
|
||||
"errors"
|
||||
"fmt"
|
||||
"hash"
|
||||
@ -41,6 +42,10 @@ import (
|
||||
|
||||
"github.com/RoaringBitmap/roaring/roaring64"
|
||||
"github.com/google/btree"
|
||||
"github.com/ledgerwatch/log/v3"
|
||||
"github.com/spaolacci/murmur3"
|
||||
"golang.org/x/crypto/sha3"
|
||||
|
||||
"github.com/ledgerwatch/erigon-lib/commitment"
|
||||
"github.com/ledgerwatch/erigon-lib/common"
|
||||
"github.com/ledgerwatch/erigon-lib/common/length"
|
||||
@ -48,9 +53,6 @@ import (
|
||||
"github.com/ledgerwatch/erigon-lib/kv"
|
||||
"github.com/ledgerwatch/erigon-lib/recsplit"
|
||||
"github.com/ledgerwatch/erigon-lib/recsplit/eliasfano32"
|
||||
"github.com/ledgerwatch/log/v3"
|
||||
"github.com/spaolacci/murmur3"
|
||||
"golang.org/x/crypto/sha3"
|
||||
)
|
||||
|
||||
// Aggregator of multiple state files to support state reader and state writer
|
||||
@ -175,7 +177,7 @@ type Aggregator struct {
|
||||
changesBtree *btree.BTree // btree of ChangesItem
|
||||
trace bool // Turns on tracing for specific accounts and locations
|
||||
tracedKeys map[string]struct{} // Set of keys being traced during aggregations
|
||||
hph *commitment.HexPatriciaHashed
|
||||
hph commitment.Trie //*commitment.HexPatriciaHashed
|
||||
keccak hash.Hash
|
||||
changesets bool // Whether to generate changesets (off by default)
|
||||
commitments bool // Whether to calculate commitments
|
||||
@ -616,12 +618,12 @@ func buildIndex(d *compress.Decompressor, idxPath, tmpDir string, count int) (*r
|
||||
// aggregate gathers changes from the changefiles into a B-tree, and "removes" them from the database
|
||||
// This function is time-critical because it needs to be run in the same go-routine (thread) as the general
|
||||
// execution (due to read-write tx). After that, we can optimistically execute the rest in the background
|
||||
func (c *Changes) aggregate(blockFrom, blockTo uint64, prefixLen int, tx kv.RwTx, table string, commitments bool) (*btree.BTree, error) {
|
||||
func (c *Changes) aggregate(blockFrom, blockTo uint64, prefixLen int, tx kv.RwTx, table string, commitMerger commitmentMerger) (*btree.BTree, error) {
|
||||
if err := c.openFiles(blockTo, false /* write */); err != nil {
|
||||
return nil, fmt.Errorf("open files: %w", err)
|
||||
}
|
||||
bt := btree.New(32)
|
||||
err := c.aggregateToBtree(bt, prefixLen, commitments)
|
||||
err := c.aggregateToBtree(bt, prefixLen, commitMerger)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("aggregateToBtree: %w", err)
|
||||
}
|
||||
@ -642,6 +644,7 @@ func (c *Changes) aggregate(blockFrom, blockTo uint64, prefixLen int, tx kv.RwTx
|
||||
e = fmt.Errorf("record not found in db for %s key %x", table, dbPrefix)
|
||||
return false
|
||||
}
|
||||
|
||||
prevNum := binary.BigEndian.Uint32(prevV[:4])
|
||||
if prevNum < item.count {
|
||||
e = fmt.Errorf("record count too low for %s key %s count %d, subtracting %d", table, dbPrefix, prevNum, item.count)
|
||||
@ -655,6 +658,7 @@ func (c *Changes) aggregate(blockFrom, blockTo uint64, prefixLen int, tx kv.RwTx
|
||||
v := make([]byte, len(prevV))
|
||||
binary.BigEndian.PutUint32(v[:4], prevNum-item.count)
|
||||
copy(v[4:], prevV[4:])
|
||||
|
||||
if e = tx.Put(table, dbPrefix, v); e != nil {
|
||||
return false
|
||||
}
|
||||
@ -826,7 +830,7 @@ func (c *Changes) produceChangeSets(blockFrom, blockTo uint64, historyType, bitm
|
||||
// (there are 3 of them, one for "keys", one for values "before" every change, and one for values "after" every change)
|
||||
// and create a B-tree where each key is only represented once, with the value corresponding to the "after" value
|
||||
// of the latest change.
|
||||
func (c *Changes) aggregateToBtree(bt *btree.BTree, prefixLen int, commitments bool) error {
|
||||
func (c *Changes) aggregateToBtree(bt *btree.BTree, prefixLen int, commitMerge commitmentMerger) error {
|
||||
var b bool
|
||||
var e error
|
||||
var key, before, after []byte
|
||||
@ -841,26 +845,27 @@ func (c *Changes) aggregateToBtree(bt *btree.BTree, prefixLen int, commitments b
|
||||
item := &AggregateItem{k: prefix, count: 0}
|
||||
bt.ReplaceOrInsert(item)
|
||||
}
|
||||
|
||||
ai.k = key
|
||||
i := bt.Get(&ai)
|
||||
if i == nil {
|
||||
item := &AggregateItem{k: common.Copy(key), v: common.Copy(after), count: 1}
|
||||
bt.ReplaceOrInsert(item)
|
||||
} else {
|
||||
item := i.(*AggregateItem)
|
||||
if commitments {
|
||||
var err error
|
||||
var mergedVal []byte
|
||||
if mergedVal, err = commitment.MergeBranches(item.v, after, nil); err != nil {
|
||||
return fmt.Errorf("merge branches: %w", err)
|
||||
}
|
||||
//fmt.Printf("aggregateToBtree prefix [%x], [%x]+[%x]=>[%x]\n", commitment.CompactToHex(key), after, item.v, mergedVal)
|
||||
item.v = mergedVal
|
||||
} else {
|
||||
item.v = common.Copy(after)
|
||||
}
|
||||
item.count++
|
||||
continue
|
||||
}
|
||||
|
||||
item := i.(*AggregateItem)
|
||||
if commitMerge != nil {
|
||||
mergedVal, err := commitMerge(item.v, after, nil)
|
||||
if err != nil {
|
||||
return fmt.Errorf("merge branches (%T) : %w", commitMerge, err)
|
||||
}
|
||||
//fmt.Printf("aggregateToBtree prefix [%x], [%x]+[%x]=>[%x]\n", commitment.CompactToHex(key), after, item.v, mergedVal)
|
||||
item.v = mergedVal
|
||||
} else {
|
||||
item.v = common.Copy(after)
|
||||
}
|
||||
item.count++
|
||||
}
|
||||
if e != nil {
|
||||
return fmt.Errorf("aggregateToBtree nextTriple: %w", e)
|
||||
@ -921,7 +926,7 @@ type byEndBlockItem struct {
|
||||
endBlock uint64
|
||||
decompressor *compress.Decompressor
|
||||
getter *compress.Getter // reader for the decompressor
|
||||
getterMerge *compress.Getter // reader for the decomporessor used in the background merge thread
|
||||
getterMerge *compress.Getter // reader for the decompressor used in the background merge thread
|
||||
index *recsplit.Index
|
||||
indexReader *recsplit.IndexReader // reader for the index
|
||||
readerMerge *recsplit.IndexReader // index reader for the background merge thread
|
||||
@ -984,14 +989,14 @@ func (a *Aggregator) scanStateFiles(files []fs.DirEntry) {
|
||||
}
|
||||
}
|
||||
|
||||
func NewAggregator(diffDir string, unwindLimit uint64, aggregationStep uint64, changesets, commitments bool, minArch uint64, tx kv.RwTx) (*Aggregator, error) {
|
||||
func NewAggregator(diffDir string, unwindLimit uint64, aggregationStep uint64, changesets, commitments bool, minArch uint64, trie commitment.Trie, tx kv.RwTx) (*Aggregator, error) {
|
||||
a := &Aggregator{
|
||||
diffDir: diffDir,
|
||||
unwindLimit: unwindLimit,
|
||||
aggregationStep: aggregationStep,
|
||||
tracedKeys: map[string]struct{}{},
|
||||
keccak: sha3.NewLegacyKeccak256(),
|
||||
hph: commitment.NewHexPatriciaHashed(length.Addr, nil, nil, nil),
|
||||
hph: trie,
|
||||
aggChannel: make(chan *AggregationTask, 1024),
|
||||
aggError: make(chan error, 1),
|
||||
mergeChannel: make(chan struct{}, 1),
|
||||
@ -1002,6 +1007,7 @@ func NewAggregator(diffDir string, unwindLimit uint64, aggregationStep uint64, c
|
||||
commitments: commitments,
|
||||
archHasher: murmur3.New128WithSeed(0), // TODO: Randomise salt
|
||||
}
|
||||
a.trace = true
|
||||
for fType := FirstType; fType < NumberOfTypes; fType++ {
|
||||
a.files[fType] = btree.New(32)
|
||||
}
|
||||
@ -1119,6 +1125,7 @@ func (a *Aggregator) rebuildRecentState(tx kv.RwTx) error {
|
||||
t := time.Now()
|
||||
var err error
|
||||
trees := map[FileType]*btree.BTree{}
|
||||
|
||||
a.changesBtree.Ascend(func(i btree.Item) bool {
|
||||
item := i.(*ChangesItem)
|
||||
for fType := FirstType; fType < NumberOfStateTypes; fType++ {
|
||||
@ -1136,7 +1143,20 @@ func (a *Aggregator) rebuildRecentState(tx kv.RwTx) error {
|
||||
if fType == Storage {
|
||||
prefixLen = length.Addr
|
||||
}
|
||||
if err = changes.aggregateToBtree(tree, prefixLen, fType == Commitment); err != nil {
|
||||
|
||||
var commitMerger commitmentMerger
|
||||
if fType == Commitment {
|
||||
switch a.hph.Variant() {
|
||||
case commitment.VariantHexPatriciaTrie, commitment.VariantReducedHexPatriciaTrie:
|
||||
commitMerger = mergeCommitments
|
||||
case commitment.VariantBinPatriciaTrie:
|
||||
commitMerger = mergeBinCommitments
|
||||
default:
|
||||
return false
|
||||
}
|
||||
}
|
||||
|
||||
if err = changes.aggregateToBtree(tree, prefixLen, commitMerger); err != nil {
|
||||
return false
|
||||
}
|
||||
if err = changes.closeFiles(); err != nil {
|
||||
@ -1365,7 +1385,7 @@ func (a *Aggregator) backgroundAggregation() {
|
||||
|
||||
type CommitmentValTransform struct {
|
||||
pre [NumberOfAccountStorageTypes][]*byEndBlockItem // List of state files before the merge
|
||||
post [NumberOfAccountStorageTypes][]*byEndBlockItem // List of state files aftee the merge
|
||||
post [NumberOfAccountStorageTypes][]*byEndBlockItem // List of state files after the merge
|
||||
}
|
||||
|
||||
func decodeU64(from []byte) uint64 {
|
||||
@ -1398,7 +1418,24 @@ func encodeU64(i uint64, to []byte) []byte {
|
||||
}
|
||||
}
|
||||
|
||||
// commitmentValTransform parses the value of of the commitment record to extract references
|
||||
var replaceHistory = make(map[string][]string)
|
||||
|
||||
func addKeyTransition(from, to string) {
|
||||
v, ok := replaceHistory[from]
|
||||
if !ok {
|
||||
v = make([]string, 0)
|
||||
}
|
||||
v = append(v, to)
|
||||
replaceHistory[from] = v
|
||||
}
|
||||
|
||||
var spkNotFound = make(map[string]int)
|
||||
|
||||
func markKeyNotFound(k string) {
|
||||
spkNotFound[k]++
|
||||
}
|
||||
|
||||
// commitmentValTransform parses the value of the commitment record to extract references
|
||||
// to accounts and storage items, then looks them up in the new, merged files, and replaces them with
|
||||
// the updated references
|
||||
func (cvt *CommitmentValTransform) commitmentValTransform(val []byte, transValBuf []byte) ([]byte, error) {
|
||||
@ -1444,6 +1481,122 @@ func (cvt *CommitmentValTransform) commitmentValTransform(val []byte, transValBu
|
||||
}
|
||||
transAccountPks = append(transAccountPks, accountPlainKey)
|
||||
}
|
||||
for _, storagePlainKey := range storagePlainKeys {
|
||||
if len(storagePlainKey) == length.Addr+length.Hash {
|
||||
// Non-optimised key originating from a database record
|
||||
spkBuf = append(spkBuf[:0], storagePlainKey...)
|
||||
} else {
|
||||
// Optimised key referencing a state file record (file number and offset within the file)
|
||||
fileI := int(storagePlainKey[0])
|
||||
offset := decodeU64(storagePlainKey[1:])
|
||||
g := cvt.pre[Storage][fileI].getterMerge
|
||||
g.Reset(offset)
|
||||
spkBuf, _ = g.Next(spkBuf[:0])
|
||||
// fmt.Printf("replacing storage [%x] from [%x]\n", spkBuf, storagePlainKey)
|
||||
}
|
||||
if bytes.Equal(storagePlainKey, wantedOfft) || bytes.Equal(spkBuf, wantedOfft) {
|
||||
fmt.Printf("WantedOffset replacing storage [%x] => [%x]\n", spkBuf, storagePlainKey)
|
||||
}
|
||||
// Lookup spkBuf in the post storage files
|
||||
for j := len(cvt.post[Storage]); j > 0; j-- {
|
||||
item := cvt.post[Storage][j-1]
|
||||
if item.index.Empty() {
|
||||
continue
|
||||
}
|
||||
offset := item.readerMerge.Lookup(spkBuf)
|
||||
g := item.getterMerge
|
||||
g.Reset(offset)
|
||||
if g.HasNext() {
|
||||
if keyMatch, _ := g.Match(spkBuf); keyMatch {
|
||||
storagePlainKey = encodeU64(offset, []byte{byte(j - 1)})
|
||||
addKeyTransition(hex.EncodeToString(spkBuf), hex.EncodeToString(storagePlainKey))
|
||||
// fmt.Printf("replacing storage [%x] => [%x]\n", spkBuf, storagePlainKey)
|
||||
if bytes.Equal(storagePlainKey, wantedOfft) {
|
||||
fmt.Printf("OFF replacing storage [%x] => [%x]\n", spkBuf, storagePlainKey)
|
||||
}
|
||||
break
|
||||
} else {
|
||||
if j == 1 {
|
||||
markKeyNotFound(hex.EncodeToString(spkBuf))
|
||||
hist, _ := replaceHistory[hex.EncodeToString(spkBuf)]
|
||||
var str string
|
||||
str = "{ "
|
||||
for _, v := range hist {
|
||||
str += fmt.Sprintf("%v, ", v)
|
||||
}
|
||||
str += "}"
|
||||
if len(spkBuf) == 0 {
|
||||
fmt.Printf("F[%d|%d] spk mismatch '%x' => %v, times %d\n", j-1, offset, spkBuf, str, spkNotFound[hex.EncodeToString(spkBuf)])
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
transStoragePks = append(transStoragePks, storagePlainKey)
|
||||
}
|
||||
if transValBuf, err = commitment.ReplacePlainKeys(val, transAccountPks, transStoragePks, transValBuf); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return transValBuf, nil
|
||||
}
|
||||
|
||||
//var wanted = []byte{0, 3, 70,113}
|
||||
// var wanted = []byte{138, 1, 88, 39, 36, 194, 18, 220, 117, 172, 221, 139, 208, 27, 186, 172, 217, 9, 154, 251, 240, 124, 16, 228, 140, 98, 195, 47, 222, 155, 131, 231, 90, 114, 61, 225, 14, 230, 104, 165, 113, 52, 4, 143, 167, 207, 154, 237, 244, 218, 83, 204}
|
||||
var Wanted = []byte{87, 13, 60, 125, 6, 210, 211, 78, 26, 212, 11, 71, 211, 176, 73, 96, 60, 95, 127, 73, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1}
|
||||
|
||||
var wantedOfft = encodeU64(6583, []byte{0})
|
||||
|
||||
// var wantedOfft = encodeU64(38437, []byte{0})
|
||||
|
||||
// commitmentValTransform parses the value of the commitment record to extract references
|
||||
// to accounts and storage items, then looks them up in the new, merged files, and replaces them with
|
||||
// the updated references
|
||||
//
|
||||
// this is copy of the above function, but with a different plainkeys extractor
|
||||
func (cvt *CommitmentValTransform) commitmentBinValTransform(val []byte, transValBuf []byte) ([]byte, error) {
|
||||
if len(val) == 0 {
|
||||
return transValBuf, nil
|
||||
}
|
||||
|
||||
accountPlainKeys, storagePlainKeys, err := commitment.ExtractBinPlainKeys(val)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
var transAccountPks [][]byte
|
||||
var transStoragePks [][]byte
|
||||
var apkBuf, spkBuf []byte
|
||||
for _, accountPlainKey := range accountPlainKeys {
|
||||
if len(accountPlainKey) == length.Addr {
|
||||
// Non-optimised key originating from a database record
|
||||
apkBuf = append(apkBuf[:0], accountPlainKey...)
|
||||
} else {
|
||||
// Optimised key referencing a state file record (file number and offset within the file)
|
||||
fileI := int(accountPlainKey[0])
|
||||
offset := decodeU64(accountPlainKey[1:])
|
||||
g := cvt.pre[Account][fileI].getterMerge
|
||||
g.Reset(offset)
|
||||
apkBuf, _ = g.Next(apkBuf[:0])
|
||||
//fmt.Printf("replacing account [%x] from [%x]\n", apkBuf, accountPlainKey)
|
||||
}
|
||||
// Look up apkBuf in the post account files
|
||||
for j := len(cvt.post[Account]); j > 0; j-- {
|
||||
item := cvt.post[Account][j-1]
|
||||
if item.index.Empty() {
|
||||
continue
|
||||
}
|
||||
offset := item.readerMerge.Lookup(apkBuf)
|
||||
g := item.getterMerge
|
||||
g.Reset(offset)
|
||||
if g.HasNext() {
|
||||
if keyMatch, _ := g.Match(apkBuf); keyMatch {
|
||||
accountPlainKey = encodeU64(offset, []byte{byte(j - 1)})
|
||||
//fmt.Printf("replaced account [%x]=>[%x] for file [%d-%d]\n", apkBuf, accountPlainKey, item.startBlock, item.endBlock)
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
transAccountPks = append(transAccountPks, accountPlainKey)
|
||||
}
|
||||
for _, storagePlainKey := range storagePlainKeys {
|
||||
if len(storagePlainKey) == length.Addr+length.Hash {
|
||||
// Non-optimised key originating from a database record
|
||||
@ -1476,9 +1629,10 @@ func (cvt *CommitmentValTransform) commitmentValTransform(val []byte, transValBu
|
||||
}
|
||||
transStoragePks = append(transStoragePks, storagePlainKey)
|
||||
}
|
||||
if transValBuf, err = commitment.ReplacePlainKeys(val, transAccountPks, transStoragePks, transValBuf); err != nil {
|
||||
if transValBuf, err = commitment.ReplaceBinPlainKeys(val, transAccountPks, transStoragePks, transValBuf); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return transValBuf, nil
|
||||
}
|
||||
|
||||
@ -1521,8 +1675,14 @@ func (a *Aggregator) backgroundMerge() {
|
||||
var valTransform func([]byte, []byte) ([]byte, error)
|
||||
var mergeFunc func([]byte, []byte, []byte) ([]byte, error)
|
||||
if fType == Commitment {
|
||||
valTransform = cvt.commitmentValTransform
|
||||
mergeFunc = mergeCommitments
|
||||
switch a.hph.Variant() {
|
||||
case commitment.VariantBinPatriciaTrie:
|
||||
valTransform = cvt.commitmentBinValTransform
|
||||
mergeFunc = mergeBinCommitments
|
||||
case commitment.VariantHexPatriciaTrie, commitment.VariantReducedHexPatriciaTrie:
|
||||
valTransform = cvt.commitmentValTransform
|
||||
mergeFunc = mergeCommitments
|
||||
}
|
||||
} else {
|
||||
mergeFunc = mergeReplace
|
||||
}
|
||||
@ -1649,6 +1809,8 @@ func (a *Aggregator) reduceHistoryFiles(fType FileType, item *byEndBlockItem) er
|
||||
return nil
|
||||
}
|
||||
|
||||
type commitmentMerger func(prev, current, target []byte) ([]byte, error)
|
||||
|
||||
func mergeReplace(preval, val, buf []byte) ([]byte, error) {
|
||||
return append(buf, val...), nil
|
||||
}
|
||||
@ -1671,7 +1833,11 @@ func mergeBitmaps(preval, val, buf []byte) ([]byte, error) {
|
||||
}
|
||||
|
||||
func mergeCommitments(preval, val, buf []byte) ([]byte, error) {
|
||||
return commitment.MergeBranches(preval, val, buf)
|
||||
return commitment.MergeHexBranches(preval, val, buf)
|
||||
}
|
||||
|
||||
func mergeBinCommitments(preval, val, buf []byte) ([]byte, error) {
|
||||
return mergeReplace(preval, val, buf)
|
||||
}
|
||||
|
||||
func (a *Aggregator) backgroundHistoryMerge() {
|
||||
@ -1704,14 +1870,22 @@ func (a *Aggregator) backgroundHistoryMerge() {
|
||||
}
|
||||
if len(toRemove[fType]) > 1 {
|
||||
isBitmap := fType == AccountBitmap || fType == StorageBitmap || fType == CodeBitmap
|
||||
var mergeFunc func([]byte, []byte, []byte) ([]byte, error)
|
||||
if isBitmap {
|
||||
|
||||
var mergeFunc commitmentMerger
|
||||
switch {
|
||||
case isBitmap:
|
||||
mergeFunc = mergeBitmaps
|
||||
} else if fType == Commitment {
|
||||
mergeFunc = mergeCommitments
|
||||
} else {
|
||||
case fType == Commitment:
|
||||
switch a.hph.Variant() {
|
||||
case commitment.VariantBinPatriciaTrie:
|
||||
mergeFunc = mergeBinCommitments
|
||||
case commitment.VariantHexPatriciaTrie, commitment.VariantReducedHexPatriciaTrie:
|
||||
mergeFunc = mergeCommitments
|
||||
}
|
||||
default:
|
||||
mergeFunc = mergeReplace
|
||||
}
|
||||
|
||||
if newItems[fType], err = a.computeAggregation(fType, toRemove[fType], from, to, nil /* valTransform */, mergeFunc,
|
||||
!isBitmap /* valCompressed */, !finalMerge || isBitmap /* withIndex */, 0 /* prefixLen */); err != nil {
|
||||
a.historyError <- fmt.Errorf("computeAggreation %s: %w", fType.String(), err)
|
||||
@ -1928,6 +2102,7 @@ func (a *Aggregator) readFromFiles(fType FileType, lock bool, blockNum uint64, f
|
||||
}
|
||||
val = ai.(*AggregateItem).v
|
||||
startBlock = item.startBlock
|
||||
|
||||
return false
|
||||
}
|
||||
if item.index.Empty() {
|
||||
@ -1950,12 +2125,25 @@ func (a *Aggregator) readFromFiles(fType FileType, lock bool, blockNum uint64, f
|
||||
atomic.AddUint64(&a.fileMisses, 1)
|
||||
return true
|
||||
})
|
||||
|
||||
if fType == Commitment {
|
||||
var plainKeysExtractor func(branchData []byte) (accountPlainKeys [][]byte, storagePlainKeys [][]byte, err error)
|
||||
var plainKeysReplacer func(branchData []byte, accountPlainKeys [][]byte, storagePlainKeys [][]byte, newData []byte) ([]byte, error)
|
||||
|
||||
switch a.hph.Variant() {
|
||||
case commitment.VariantHexPatriciaTrie, commitment.VariantReducedHexPatriciaTrie:
|
||||
plainKeysExtractor = commitment.ExtractPlainKeys
|
||||
plainKeysReplacer = commitment.ReplacePlainKeys
|
||||
case commitment.VariantBinPatriciaTrie:
|
||||
plainKeysExtractor = commitment.ExtractBinPlainKeys
|
||||
plainKeysReplacer = commitment.ReplaceBinPlainKeys
|
||||
}
|
||||
|
||||
// Transform references
|
||||
if len(val) > 0 {
|
||||
accountPlainKeys, storagePlainKeys, err := commitment.ExtractPlainKeys(val)
|
||||
accountPlainKeys, storagePlainKeys, err := plainKeysExtractor(val)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
panic(fmt.Errorf("value %x: %w", val, err))
|
||||
}
|
||||
var transAccountPks [][]byte
|
||||
var transStoragePks [][]byte
|
||||
@ -1985,7 +2173,7 @@ func (a *Aggregator) readFromFiles(fType FileType, lock bool, blockNum uint64, f
|
||||
}
|
||||
transStoragePks = append(transStoragePks, spkBuf)
|
||||
}
|
||||
if val, err = commitment.ReplacePlainKeys(val, transAccountPks, transStoragePks, nil); err != nil {
|
||||
if val, err = plainKeysReplacer(val, transAccountPks, transStoragePks, nil); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
||||
@ -2007,6 +2195,7 @@ func (a *Aggregator) readByOffset(fType FileType, fileI int, offset uint64) ([]b
|
||||
g.Reset(offset)
|
||||
key, _ = g.Next(nil)
|
||||
val, _ = g.Next(nil)
|
||||
|
||||
return false
|
||||
})
|
||||
return key, val
|
||||
@ -2188,7 +2377,7 @@ func (w *Writer) branchFn(prefix []byte) ([]byte, error) {
|
||||
//fmt.Printf("Pre-merge prefix [%x] [%x]+[%x], startBlock %d\n", commitment.CompactToHex(prefix), val, mergedVal, startBlock)
|
||||
if mergedVal == nil {
|
||||
mergedVal = val
|
||||
} else if mergedVal, err = commitment.MergeBranches(val, mergedVal, nil); err != nil {
|
||||
} else if mergedVal, err = commitment.MergeHexBranches(val, mergedVal, nil); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
//fmt.Printf("Post-merge prefix [%x] [%x], startBlock %d\n", commitment.CompactToHex(prefix), mergedVal, startBlock)
|
||||
@ -2394,6 +2583,7 @@ func (w *Writer) computeCommitment(trace bool) ([]byte, error) {
|
||||
if trace {
|
||||
fmt.Printf("computeCommitment w.commTree.Len()=%d\n", w.commTree.Len())
|
||||
}
|
||||
|
||||
plainKeys := make([][]byte, w.commTree.Len())
|
||||
hashedKeys := make([][]byte, w.commTree.Len())
|
||||
updates := make([]commitment.Update, w.commTree.Len())
|
||||
@ -2439,18 +2629,33 @@ func (w *Writer) computeCommitment(trace bool) ([]byte, error) {
|
||||
original = prevV[4:]
|
||||
}
|
||||
if original != nil {
|
||||
var mergedVal []byte
|
||||
if mergedVal, err = commitment.MergeBranches(original, branchNodeUpdate, nil); err == nil {
|
||||
switch w.a.hph.Variant() {
|
||||
case commitment.VariantBinPatriciaTrie:
|
||||
branchNodeUpdate, err = mergeReplace(original, branchNodeUpdate, nil)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
// use current version without merge
|
||||
case commitment.VariantHexPatriciaTrie, commitment.VariantReducedHexPatriciaTrie:
|
||||
// try to merge previous (original) and current (branchNodeUpdate) into one update
|
||||
mergedVal, err := commitment.MergeHexBranches(original, branchNodeUpdate, nil)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
//fmt.Printf("computeCommitment merge [%x] [%x]+[%x]=>[%x]\n", commitment.CompactToHex(prefix), original, branchNodeUpdate, mergedVal)
|
||||
branchNodeUpdate = mergedVal
|
||||
} else {
|
||||
return nil, err
|
||||
default:
|
||||
panic(fmt.Errorf("unknown commitment trie variant %s", w.a.hph.Variant()))
|
||||
}
|
||||
|
||||
//fmt.Printf("bstat: %v\n", w.a.hph.(*commitment.BinPatriciaTrie).StatString())
|
||||
}
|
||||
|
||||
//fmt.Printf("computeCommitment set [%x] [%x]\n", commitment.CompactToHex(prefix), branchNodeUpdate)
|
||||
v := make([]byte, 4+len(branchNodeUpdate))
|
||||
binary.BigEndian.PutUint32(v[:4], prevNum+1)
|
||||
copy(v[4:], branchNodeUpdate)
|
||||
|
||||
if err = w.tx.Put(kv.StateCommitment, prefix, v); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -2916,7 +3121,7 @@ func (a *Aggregator) findLargestMerge(fType FileType, maxTo uint64, maxSpan uint
|
||||
func (a *Aggregator) computeAggregation(fType FileType,
|
||||
toAggregate []*byEndBlockItem, aggFrom uint64, aggTo uint64,
|
||||
valTransform func(val []byte, transValBuf []byte) ([]byte, error),
|
||||
mergeFunc func(preval, val, buf []byte) ([]byte, error),
|
||||
mergeFunc commitmentMerger,
|
||||
valCompressed bool,
|
||||
withIndex bool, prefixLen int) (*byEndBlockItem, error) {
|
||||
var item2 = &byEndBlockItem{startBlock: aggFrom, endBlock: aggTo}
|
||||
@ -3009,7 +3214,20 @@ func (w *Writer) aggregateUpto(blockFrom, blockTo uint64) error {
|
||||
if fType == Storage {
|
||||
prefixLen = length.Addr
|
||||
}
|
||||
if aggTask.bt[fType], err = aggTask.changes[fType].aggregate(blockFrom, blockTo, prefixLen, w.tx, fType.Table(), fType == Commitment); err != nil {
|
||||
|
||||
var commitMerger commitmentMerger
|
||||
if fType == Commitment {
|
||||
switch w.a.hph.Variant() {
|
||||
case commitment.VariantHexPatriciaTrie, commitment.VariantReducedHexPatriciaTrie:
|
||||
commitMerger = mergeCommitments
|
||||
case commitment.VariantBinPatriciaTrie:
|
||||
commitMerger = mergeBinCommitments
|
||||
default:
|
||||
return fmt.Errorf("unknown commitment variant %s: failed to define how to merge commitments", w.a.hph.Variant())
|
||||
}
|
||||
}
|
||||
|
||||
if aggTask.bt[fType], err = aggTask.changes[fType].aggregate(blockFrom, blockTo, prefixLen, w.tx, fType.Table(), commitMerger); err != nil {
|
||||
return fmt.Errorf("aggregate %sChanges: %w", fType.String(), err)
|
||||
}
|
||||
}
|
||||
@ -3094,6 +3312,7 @@ func (a *Aggregator) mergeIntoStateFile(cp *CursorHeap, prefixLen int,
|
||||
} else {
|
||||
ci1.val, _ = ci1.dg.NextUncompressed()
|
||||
}
|
||||
|
||||
heap.Fix(cp, 0)
|
||||
} else {
|
||||
heap.Pop(cp)
|
||||
@ -3139,8 +3358,9 @@ func (a *Aggregator) mergeIntoStateFile(cp *CursorHeap, prefixLen int,
|
||||
count++ // Only counting keys, not values
|
||||
if valTransform != nil {
|
||||
if transValBuf, err = valTransform(valBuf, transValBuf[:0]); err != nil {
|
||||
return nil, 0, fmt.Errorf("mergeIntoStateFile valTransform [%x]: %w", valBuf, err)
|
||||
return nil, 0, fmt.Errorf("mergeIntoStateFile -valTransform [%x]: %w", valBuf, err)
|
||||
}
|
||||
|
||||
if err = comp.AddWord(transValBuf); err != nil {
|
||||
return nil, 0, err
|
||||
}
|
||||
@ -3157,6 +3377,7 @@ func (a *Aggregator) mergeIntoStateFile(cp *CursorHeap, prefixLen int,
|
||||
// fmt.Printf("merge %s.%d-%d [%x]=>[%x]\n", fType.String(), startBlock, endBlock, keyBuf, valBuf)
|
||||
//}
|
||||
}
|
||||
|
||||
keyBuf = append(keyBuf[:0], lastKey...)
|
||||
valBuf = append(valBuf[:0], lastVal...)
|
||||
}
|
||||
|
@ -23,6 +23,8 @@ import (
|
||||
"testing"
|
||||
|
||||
"github.com/holiman/uint256"
|
||||
|
||||
"github.com/ledgerwatch/erigon-lib/commitment"
|
||||
"github.com/ledgerwatch/erigon-lib/kv"
|
||||
"github.com/ledgerwatch/erigon-lib/kv/memdb"
|
||||
)
|
||||
@ -79,7 +81,8 @@ func TestSimpleAggregator(t *testing.T) {
|
||||
}
|
||||
defer rwTx.Rollback()
|
||||
tmpDir := t.TempDir()
|
||||
a, err := NewAggregator(tmpDir, 16, 4, true, true, 1000, rwTx)
|
||||
trie := commitment.InitializeTrie(commitment.VariantHexPatriciaTrie)
|
||||
a, err := NewAggregator(tmpDir, 16, 4, true, true, 1000, trie, rwTx)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
@ -120,7 +123,8 @@ func TestLoopAggregator(t *testing.T) {
|
||||
}
|
||||
defer rwTx.Rollback()
|
||||
tmpDir := t.TempDir()
|
||||
a, err := NewAggregator(tmpDir, 16, 4, true, true, 1000, rwTx)
|
||||
trie := commitment.InitializeTrie(commitment.VariantHexPatriciaTrie)
|
||||
a, err := NewAggregator(tmpDir, 16, 4, true, true, 1000, trie, rwTx)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
@ -166,7 +170,9 @@ func TestRecreateAccountWithStorage(t *testing.T) {
|
||||
}
|
||||
defer rwTx.Rollback()
|
||||
tmpDir := t.TempDir()
|
||||
a, err := NewAggregator(tmpDir, 16, 4, true, true, 1000, rwTx)
|
||||
|
||||
trie := commitment.InitializeTrie(commitment.VariantHexPatriciaTrie)
|
||||
a, err := NewAggregator(tmpDir, 16, 4, true, true, 1000, trie, rwTx)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
@ -274,7 +280,8 @@ func TestChangeCode(t *testing.T) {
|
||||
}
|
||||
defer rwTx.Rollback()
|
||||
tmpDir := t.TempDir()
|
||||
a, err := NewAggregator(tmpDir, 16, 4, true, true, 1000, rwTx)
|
||||
trie := commitment.InitializeTrie(commitment.VariantHexPatriciaTrie)
|
||||
a, err := NewAggregator(tmpDir, 16, 4, true, true, 1000, trie, rwTx)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
1375
commitment/bin_patricia_hashed.go
Normal file
1375
commitment/bin_patricia_hashed.go
Normal file
File diff suppressed because it is too large
Load Diff
908
commitment/bin_patricia_trie.go
Normal file
908
commitment/bin_patricia_trie.go
Normal file
@ -0,0 +1,908 @@
|
||||
package commitment
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/binary"
|
||||
"encoding/hex"
|
||||
"fmt"
|
||||
"strings"
|
||||
|
||||
"github.com/ledgerwatch/erigon-lib/common/length"
|
||||
|
||||
"golang.org/x/crypto/sha3"
|
||||
)
|
||||
|
||||
type BinPatriciaTrie struct {
|
||||
root *RootNode
|
||||
trace bool
|
||||
stat stat
|
||||
keccak keccakState
|
||||
}
|
||||
|
||||
type stat struct {
|
||||
hashesTotal uint64
|
||||
nodesTotal uint64
|
||||
}
|
||||
|
||||
func NewBinaryPatriciaTrie() *BinPatriciaTrie {
|
||||
return &BinPatriciaTrie{
|
||||
keccak: sha3.NewLegacyKeccak256().(keccakState),
|
||||
}
|
||||
}
|
||||
|
||||
func (t *BinPatriciaTrie) Update(key, value []byte) {
|
||||
keyPath := newBitstring(key)
|
||||
if t.root == nil {
|
||||
t.root = &RootNode{
|
||||
Node: &Node{
|
||||
Key: key,
|
||||
Value: value,
|
||||
},
|
||||
CommonPrefix: keyPath,
|
||||
}
|
||||
t.stat.nodesTotal++
|
||||
return
|
||||
}
|
||||
|
||||
edge, keyPathRest, splitAt, latest := t.root.Walk(keyPath, func(_ *Node) {})
|
||||
if len(edge) == 0 && len(keyPathRest) == 0 {
|
||||
latest.Value = value
|
||||
return
|
||||
}
|
||||
pathToLatest := keyPath[:len(keyPath)-len(keyPathRest)]
|
||||
|
||||
t.stat.nodesTotal++
|
||||
newLeaf := &Node{P: latest, Key: key, Value: value}
|
||||
latest.splitEdge(pathToLatest, edge[:splitAt], edge[splitAt:], keyPathRest, newLeaf, t.hash)
|
||||
if latest.P == nil {
|
||||
t.root.CommonPrefix = edge[:splitAt]
|
||||
}
|
||||
}
|
||||
|
||||
// Key describes path in trie to value. When UpdateHashed is used,
|
||||
// hashed key describes path to the leaf node and plainKey is stored in the leaf node Key field.
|
||||
func (t *BinPatriciaTrie) UpdateHashed(plainKey, hashedKey, value []byte, isStorage bool) (updates map[string][]byte) {
|
||||
keyPath := newBitstring(hashedKey)
|
||||
updates = make(map[string][]byte)
|
||||
if t.root == nil {
|
||||
t.root = &RootNode{
|
||||
Node: &Node{
|
||||
Key: plainKey,
|
||||
Value: value,
|
||||
isStorage: isStorage,
|
||||
},
|
||||
CommonPrefix: keyPath,
|
||||
}
|
||||
|
||||
t.stat.nodesTotal++
|
||||
t.hash(t.root.Node, keyPath, 0)
|
||||
|
||||
touchMap := uint16(1 << keyPath[len(keyPath)-1])
|
||||
|
||||
updates[keyPath.String()] = encodeNodeUpdate(t.root.Node, touchMap, touchMap, nil)
|
||||
return updates
|
||||
}
|
||||
|
||||
touchedNodes := make([]*Node, 0)
|
||||
|
||||
edge, keyPathRest, splitAt, latest := t.root.Walk(keyPath, func(fn *Node) { fn.hash = fn.hash[:0]; touchedNodes = append(touchedNodes, fn) })
|
||||
pathToLatest := keyPath[:len(keyPath)-len(keyPathRest)]
|
||||
|
||||
var touchMap uint16
|
||||
if len(edge) == 0 && len(keyPathRest) == 0 { // we found the leaf
|
||||
latest.Value = value
|
||||
t.hash(latest, bitstring{}, 0)
|
||||
|
||||
touchMap = 1 << edge[len(edge)-1]
|
||||
updates[pathToLatest.String()] = encodeNodeUpdate(latest, touchMap, touchMap, nil)
|
||||
return updates
|
||||
}
|
||||
|
||||
// split existing edge and insert new leaf
|
||||
t.stat.nodesTotal++
|
||||
|
||||
newLeaf := &Node{P: latest, Key: plainKey, Value: value, isStorage: isStorage}
|
||||
updates = latest.splitEdge(pathToLatest, edge[:splitAt], edge[splitAt:], keyPathRest, newLeaf, t.hash)
|
||||
if latest.P == nil {
|
||||
t.root.CommonPrefix = edge[:splitAt]
|
||||
}
|
||||
return updates
|
||||
}
|
||||
|
||||
// Get returns value stored by provided key.
|
||||
func (t *BinPatriciaTrie) Get(key []byte) ([]byte, bool) {
|
||||
keyPath := newBitstring(key)
|
||||
if t.root == nil {
|
||||
return nil, false
|
||||
}
|
||||
|
||||
edge, keyPathRest, _, latest := t.root.Walk(keyPath, func(_ *Node) {})
|
||||
if len(edge) == 0 && len(keyPathRest) == 0 {
|
||||
if latest.Value == nil {
|
||||
switch {
|
||||
case len(latest.LPrefix) == 0 && latest.L != nil:
|
||||
return latest.L.Value, true
|
||||
case len(latest.RPrefix) == 0 && latest.R != nil:
|
||||
return latest.R.Value, true
|
||||
}
|
||||
}
|
||||
return latest.Value, true
|
||||
}
|
||||
return nil, false
|
||||
}
|
||||
|
||||
// Get returns value stored by provided key.
|
||||
func (t *BinPatriciaTrie) getNode(key []byte) *Node {
|
||||
keyPath := newBitstring(key)
|
||||
if t.root == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
edge, keyPathRest, _, latest := t.root.Walk(keyPath, func(_ *Node) {})
|
||||
if len(edge) == 0 && len(keyPathRest) == 0 {
|
||||
return latest
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (t *BinPatriciaTrie) RootHash() ([]byte, error) {
|
||||
if t.root == nil {
|
||||
return EmptyRootHash, nil
|
||||
}
|
||||
return t.hash(t.root.Node, t.root.CommonPrefix, 0), nil
|
||||
}
|
||||
|
||||
func (t *BinPatriciaTrie) ProcessUpdates(plainKeys, hashedKeys [][]byte, updates []Update) (branchNodeUpdates map[string][]byte, err error) {
|
||||
branchNodeUpdates = make(map[string][]byte)
|
||||
for i, update := range updates {
|
||||
account := new(Account)
|
||||
node := t.getNode(hashedKeys[i]) // check if key exist
|
||||
if node != nil && !node.isStorage {
|
||||
account.decode(node.Value)
|
||||
}
|
||||
|
||||
// apply supported updates
|
||||
if update.Flags == DELETE_UPDATE {
|
||||
//continue
|
||||
if node != nil {
|
||||
if node.P != nil {
|
||||
meltPrefix := node.P.deleteChild(node)
|
||||
if node.P.P == nil {
|
||||
t.root.CommonPrefix = append(t.root.CommonPrefix, meltPrefix...)
|
||||
}
|
||||
} else { // remove root
|
||||
t.root = nil
|
||||
}
|
||||
t.stat.nodesTotal--
|
||||
branchNodeUpdates[newBitstring(hashedKeys[i]).String()] = []byte{}
|
||||
}
|
||||
continue
|
||||
}
|
||||
if update.Flags&BALANCE_UPDATE != 0 {
|
||||
account.Balance.Set(&update.Balance)
|
||||
}
|
||||
if update.Flags&NONCE_UPDATE != 0 {
|
||||
account.Nonce = update.Nonce
|
||||
}
|
||||
if update.Flags&CODE_UPDATE != 0 {
|
||||
if account.CodeHash == nil {
|
||||
account.CodeHash = make([]byte, len(update.CodeHashOrStorage))
|
||||
}
|
||||
copy(account.CodeHash, update.CodeHashOrStorage[:])
|
||||
}
|
||||
|
||||
aux := make([]byte, 0)
|
||||
isStorage := false
|
||||
if update.Flags&STORAGE_UPDATE != 0 {
|
||||
isStorage = true
|
||||
aux = update.CodeHashOrStorage[:update.ValLength]
|
||||
}
|
||||
|
||||
// aux is not empty only when storage update is there
|
||||
if len(aux) == 0 {
|
||||
aux = account.encode(aux)
|
||||
}
|
||||
|
||||
ukey := t.UpdateHashed(plainKeys[i], hashedKeys[i], aux, isStorage)
|
||||
for pref, val := range ukey {
|
||||
branchNodeUpdates[pref] = val
|
||||
if val != nil && t.trace {
|
||||
fmt.Printf("%q => %s\n", pref, branchToString2(val))
|
||||
}
|
||||
}
|
||||
for pref, upd := range t.rootHashWithUpdates() {
|
||||
v, ex := branchNodeUpdates[pref]
|
||||
if ex {
|
||||
upd = append(v[:4], upd[4:]...)
|
||||
}
|
||||
branchNodeUpdates[pref] = upd
|
||||
}
|
||||
}
|
||||
|
||||
return branchNodeUpdates, nil
|
||||
}
|
||||
|
||||
func DecodeNodeFromUpdate(buf []byte) (touch, after uint16, node Node, err error) {
|
||||
if len(buf) < 5 {
|
||||
return
|
||||
}
|
||||
|
||||
touch = binary.BigEndian.Uint16(buf[:2])
|
||||
after = binary.BigEndian.Uint16(buf[2:4])
|
||||
bits, pos := PartFlags(buf[4]), 5
|
||||
|
||||
if bits&ACCOUNT_PLAIN_PART != 0 {
|
||||
n, aux, err := decodeSizedBuffer(buf[pos:])
|
||||
if err != nil {
|
||||
return touch, after, Node{}, fmt.Errorf("decode account plain key: %w", err)
|
||||
}
|
||||
pos += n
|
||||
node.Key = aux
|
||||
}
|
||||
|
||||
if bits&STORAGE_PLAIN_PART != 0 {
|
||||
n, aux, err := decodeSizedBuffer(buf[pos:])
|
||||
if err != nil {
|
||||
return touch, after, Node{}, fmt.Errorf("decode storage plain key: %w", err)
|
||||
}
|
||||
pos += n
|
||||
node.Key = aux
|
||||
node.isStorage = true
|
||||
}
|
||||
|
||||
if bits&HASH_PART != 0 {
|
||||
n, aux, err := decodeSizedBuffer(buf[pos:])
|
||||
if err != nil {
|
||||
return touch, after, Node{}, fmt.Errorf("decode node hash: %w", err)
|
||||
}
|
||||
pos += n
|
||||
_ = pos
|
||||
node.hash = aux
|
||||
}
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
func encodeNodeUpdate(node *Node, touched, after uint16, branchData []byte) []byte {
|
||||
var numBuf [binary.MaxVarintLen64]byte
|
||||
binary.BigEndian.PutUint16(numBuf[0:], touched)
|
||||
binary.BigEndian.PutUint16(numBuf[2:], after)
|
||||
|
||||
if branchData == nil {
|
||||
branchData = make([]byte, 4, 32)
|
||||
}
|
||||
copy(branchData[:4], numBuf[:])
|
||||
|
||||
var fieldBits PartFlags
|
||||
if node.Value != nil {
|
||||
fieldBits = ACCOUNT_PLAIN_PART
|
||||
if node.isStorage {
|
||||
fieldBits = STORAGE_PLAIN_PART
|
||||
}
|
||||
}
|
||||
if len(node.hash) == length.Hash {
|
||||
fieldBits |= HASH_PART
|
||||
}
|
||||
|
||||
branchData = append(branchData, byte(fieldBits))
|
||||
if fieldBits&(ACCOUNT_PLAIN_PART|STORAGE_PLAIN_PART) != 0 {
|
||||
n := binary.PutUvarint(numBuf[:], uint64(len(node.Key)))
|
||||
branchData = append(branchData, append(numBuf[:n], node.Key...)...)
|
||||
}
|
||||
|
||||
if fieldBits&HASH_PART > 0 {
|
||||
n := binary.PutUvarint(numBuf[:], uint64(len(node.hash)))
|
||||
branchData = append(branchData, append(numBuf[:n], node.hash...)...)
|
||||
}
|
||||
|
||||
return branchData
|
||||
}
|
||||
|
||||
func (t *BinPatriciaTrie) Reset() {
|
||||
t.root = nil
|
||||
fmt.Printf("trie %v\n", t.StatString())
|
||||
}
|
||||
|
||||
func (t *BinPatriciaTrie) ResetFns(
|
||||
branchFn func(prefix []byte) ([]byte, error),
|
||||
accountFn func(plainKey []byte, cell *Cell) error,
|
||||
storageFn func(plainKey []byte, cell *Cell) error,
|
||||
) {
|
||||
}
|
||||
|
||||
func (t *BinPatriciaTrie) Variant() TrieVariant { return VariantBinPatriciaTrie }
|
||||
|
||||
func (t *BinPatriciaTrie) SetTrace(b bool) { t.trace = b }
|
||||
|
||||
type RootNode struct {
|
||||
*Node
|
||||
CommonPrefix bitstring
|
||||
}
|
||||
|
||||
// There are three types of nodes:
|
||||
// - Leaf (with a value and without branches)
|
||||
// - Branch (with left and right child)
|
||||
// - Root - Either leaf or branch. When root is branch, it's Key contains their common prefix as a bitstring.
|
||||
type Node struct {
|
||||
L, R, P *Node // left and right child, parent. For root P is nil
|
||||
LPrefix bitstring // left child prefix, always begins with 0
|
||||
RPrefix bitstring // right child prefix, always begins with 1
|
||||
hash []byte // node hash
|
||||
Key []byte // same as common prefix, useful for debugging, actual key should be reconstructed by path to the node
|
||||
Value []byte // exists only in LEAF node
|
||||
isStorage bool // if true, then Value holds storage value for the Key, otherwise it holds encoded account
|
||||
}
|
||||
|
||||
func (n *Node) splitEdge(pathToNode, commonPath, detachedPath, restKeyPath bitstring, newLeaf *Node, hasher func(n *Node, pref bitstring, offt int) []byte) map[string][]byte {
|
||||
var movedNode *Node
|
||||
switch {
|
||||
case n.Value == nil:
|
||||
movedNode = &Node{ // move existed branch
|
||||
L: n.L,
|
||||
R: n.R,
|
||||
P: n,
|
||||
LPrefix: n.LPrefix,
|
||||
RPrefix: n.RPrefix,
|
||||
hash: n.hash,
|
||||
}
|
||||
movedNode.L.P, movedNode.R.P = movedNode, movedNode
|
||||
default:
|
||||
movedNode = &Node{ // move existed leaf
|
||||
P: n,
|
||||
Key: n.Key,
|
||||
Value: n.Value,
|
||||
isStorage: n.isStorage,
|
||||
hash: n.hash,
|
||||
}
|
||||
}
|
||||
newLeaf.P = n
|
||||
|
||||
switch restKeyPath[0] {
|
||||
case 0:
|
||||
n.LPrefix, n.L = restKeyPath, newLeaf
|
||||
n.RPrefix, n.R = detachedPath, movedNode
|
||||
case 1:
|
||||
n.LPrefix, n.L = detachedPath, movedNode
|
||||
n.RPrefix, n.R = restKeyPath, newLeaf
|
||||
}
|
||||
|
||||
// node become extended, reset key and value
|
||||
n.Key, n.Value, n.hash, n.isStorage = nil, nil, nil, false
|
||||
hasher(n, commonPath, 0)
|
||||
|
||||
nodeTouch := uint16(1 << pathToNode[len(pathToNode)-1])
|
||||
nodeAfter := uint16(3) // both child has been updated
|
||||
updates := make(map[string][]byte, 3)
|
||||
|
||||
hasher(n, bitstring{}, 0)
|
||||
|
||||
updates[pathToNode.String()] = encodeNodeUpdate(n, nodeTouch, nodeAfter, nil)
|
||||
|
||||
rtouch := uint16(1 << restKeyPath[0])
|
||||
updates[append(pathToNode, restKeyPath[0]).String()] = encodeNodeUpdate(newLeaf, rtouch, rtouch, nil)
|
||||
|
||||
if n.P == nil {
|
||||
// commonPath should be set to RootNode.CommonPrefix outside the function
|
||||
return updates
|
||||
}
|
||||
|
||||
if len(commonPath) > 0 {
|
||||
switch commonPath[0] {
|
||||
case 1:
|
||||
//if n.P != nil {
|
||||
n.P.RPrefix = commonPath
|
||||
//}
|
||||
//n.RPrefix = commonPath
|
||||
case 0:
|
||||
//if n.P != nil {
|
||||
n.P.LPrefix = commonPath
|
||||
//}
|
||||
//n.LPrefix = commonPath
|
||||
}
|
||||
}
|
||||
return updates
|
||||
}
|
||||
|
||||
func (n *RootNode) Walk(path bitstring, fn func(cd *Node)) (nodePath, pathRest bitstring, splitAt int, current *Node) {
|
||||
nodePath = n.CommonPrefix
|
||||
|
||||
var bit uint8
|
||||
var equal bool
|
||||
for current = n.Node; current != nil; {
|
||||
fn(current)
|
||||
|
||||
splitAt, bit, equal = nodePath.splitPoint(path)
|
||||
if equal {
|
||||
return bitstring{}, bitstring{}, 0, current
|
||||
}
|
||||
|
||||
if splitAt < len(nodePath) {
|
||||
return nodePath, path[splitAt:], splitAt, current
|
||||
}
|
||||
|
||||
if splitAt == 0 || splitAt == len(nodePath) {
|
||||
path = path[splitAt:]
|
||||
|
||||
switch bit {
|
||||
case 1:
|
||||
if current.R == nil {
|
||||
return nodePath, path, splitAt, current
|
||||
}
|
||||
nodePath = current.RPrefix
|
||||
current = current.R
|
||||
case 0:
|
||||
if current.L == nil {
|
||||
return nodePath, path, splitAt, current
|
||||
}
|
||||
nodePath = current.LPrefix
|
||||
current = current.L
|
||||
}
|
||||
|
||||
continue
|
||||
}
|
||||
break
|
||||
}
|
||||
return nodePath, path, splitAt, current
|
||||
}
|
||||
|
||||
func (n *Node) deleteChild(child *Node) bitstring {
|
||||
var melt *Node
|
||||
var meltPrefix bitstring
|
||||
|
||||
// remove child data
|
||||
switch child {
|
||||
case n.L:
|
||||
n.L, n.LPrefix = nil, nil
|
||||
melt = n.R
|
||||
meltPrefix = n.RPrefix
|
||||
case n.R:
|
||||
n.R, n.RPrefix = nil, nil
|
||||
melt = n.L
|
||||
meltPrefix = n.LPrefix
|
||||
default:
|
||||
panic("could delete only child nodes")
|
||||
}
|
||||
melt.P = n.P
|
||||
|
||||
// merge parent path to skip this half-branch node
|
||||
if n.P != nil {
|
||||
switch {
|
||||
case n.P.L == n:
|
||||
n.P.L, n.P.LPrefix = melt, append(n.P.LPrefix, meltPrefix...)
|
||||
case n.P.R == n:
|
||||
n.P.R, n.P.RPrefix = melt, append(n.P.RPrefix, meltPrefix...)
|
||||
default:
|
||||
panic("failed to merge parent path")
|
||||
}
|
||||
} else { // n is root
|
||||
n.LPrefix, n.RPrefix = melt.LPrefix, melt.RPrefix
|
||||
n.Key = melt.Key
|
||||
n.Value = melt.Value
|
||||
n.L, n.R, n.Value = melt.L, melt.R, melt.Value
|
||||
n.hash = n.hash[:0]
|
||||
}
|
||||
return meltPrefix
|
||||
}
|
||||
|
||||
func (t *BinPatriciaTrie) StatString() string {
|
||||
s := t.stat
|
||||
return fmt.Sprintf("hashes_total %d nodes %d", s.hashesTotal, s.nodesTotal)
|
||||
}
|
||||
|
||||
func (t *BinPatriciaTrie) rootHashWithUpdates() map[string][]byte {
|
||||
//if t.root == nil {
|
||||
// return EmptyRootHash, nil
|
||||
//}
|
||||
//return t.hash(t.root.Node, t.root.CommonPrefix, 0), nil
|
||||
updates := make(map[string][]byte)
|
||||
t.hashWithUpdates(t.root.Node, t.root.CommonPrefix, &updates)
|
||||
return updates
|
||||
}
|
||||
|
||||
func (t *BinPatriciaTrie) hashWithUpdates(n *Node, pref bitstring, updates *map[string][]byte) ([]byte, []byte) {
|
||||
if len(n.hash) == 32 {
|
||||
return n.hash, nil
|
||||
}
|
||||
t.keccak.Reset()
|
||||
|
||||
t.stat.hashesTotal++
|
||||
|
||||
var hash []byte
|
||||
if n.Value == nil {
|
||||
// This is a branch node, so the rule is
|
||||
// branch_hash = hash(left_root_hash || right_root_hash)
|
||||
lkey := bitstring(make([]byte, len(pref)+len(n.LPrefix)))
|
||||
copy(lkey, pref)
|
||||
copy(lkey[len(pref):], n.LPrefix)
|
||||
|
||||
rkey := bitstring(make([]byte, len(pref)+len(n.RPrefix)))
|
||||
copy(rkey, pref)
|
||||
copy(rkey[len(pref):], n.RPrefix)
|
||||
|
||||
lh, lupd := t.hashWithUpdates(n.L, lkey, updates)
|
||||
rh, rupd := t.hashWithUpdates(n.R, rkey, updates)
|
||||
t.keccak.Write(lh)
|
||||
t.keccak.Write(rh)
|
||||
hash = t.keccak.Sum(nil)
|
||||
if len(lupd) > 0 {
|
||||
binary.BigEndian.PutUint16(lupd[0:], 1)
|
||||
(*updates)[lkey.String()] = lupd
|
||||
}
|
||||
if len(rupd) > 0 {
|
||||
binary.BigEndian.PutUint16(rupd[0:], 2)
|
||||
(*updates)[rkey.String()] = rupd
|
||||
}
|
||||
|
||||
if t.trace {
|
||||
fmt.Printf("branch %v (%v|%v)\n", hex.EncodeToString(hash), hex.EncodeToString(lh), hex.EncodeToString(rh))
|
||||
}
|
||||
t.keccak.Reset()
|
||||
} else {
|
||||
// This is a leaf node, so the hashing rule is
|
||||
// leaf_hash = hash(hash(key) || hash(leaf_value))
|
||||
t.keccak.Write(n.Key)
|
||||
kh := t.keccak.Sum(nil)
|
||||
t.keccak.Reset()
|
||||
|
||||
t.keccak.Write(n.Value)
|
||||
hash = t.keccak.Sum(nil)
|
||||
t.keccak.Reset()
|
||||
|
||||
t.keccak.Write(kh)
|
||||
t.keccak.Write(hash)
|
||||
hash = t.keccak.Sum(nil)
|
||||
t.keccak.Reset()
|
||||
|
||||
if t.trace {
|
||||
fmt.Printf("leaf %v\n", hex.EncodeToString(hash))
|
||||
}
|
||||
}
|
||||
|
||||
n.hash = hash
|
||||
upd := encodeNodeUpdate(n, 0, 3, nil)
|
||||
if n.P == nil {
|
||||
binary.BigEndian.PutUint16(upd[0:], 3)
|
||||
(*updates)[pref.String()] = upd
|
||||
}
|
||||
|
||||
return hash, upd
|
||||
}
|
||||
func (t *BinPatriciaTrie) hash(n *Node, pref bitstring, off int) []byte {
|
||||
t.keccak.Reset()
|
||||
t.stat.hashesTotal++
|
||||
|
||||
if len(n.hash) == 32 && n.P != nil {
|
||||
return n.hash
|
||||
}
|
||||
|
||||
var hash []byte
|
||||
if n.Value == nil {
|
||||
// This is a branch node, so the rule is
|
||||
// branch_hash = hash(left_root_hash || right_root_hash)
|
||||
lh := t.hash(n.L, n.LPrefix, off+len(pref))
|
||||
rh := t.hash(n.R, n.RPrefix, off+len(pref))
|
||||
t.keccak.Write(lh)
|
||||
t.keccak.Write(rh)
|
||||
hash = t.keccak.Sum(nil)
|
||||
if t.trace {
|
||||
fmt.Printf("branch %v (%v|%v)\n", hex.EncodeToString(hash), hex.EncodeToString(lh), hex.EncodeToString(rh))
|
||||
}
|
||||
t.keccak.Reset()
|
||||
} else {
|
||||
// This is a leaf node, so the hashing rule is
|
||||
// leaf_hash = hash(hash(key) || hash(leaf_value))
|
||||
t.keccak.Write(n.Key)
|
||||
kh := t.keccak.Sum(nil)
|
||||
t.keccak.Reset()
|
||||
|
||||
t.keccak.Write(n.Value)
|
||||
hash = t.keccak.Sum(nil)
|
||||
t.keccak.Reset()
|
||||
|
||||
t.keccak.Write(kh)
|
||||
t.keccak.Write(hash)
|
||||
hash = t.keccak.Sum(nil)
|
||||
t.keccak.Reset()
|
||||
|
||||
if t.trace {
|
||||
fmt.Printf("leaf %v\n", hex.EncodeToString(hash))
|
||||
}
|
||||
}
|
||||
|
||||
//if len(pref) > 1 {
|
||||
// fpLen := len(pref) + off
|
||||
// t.keccak.Write([]byte{byte(fpLen), byte(fpLen >> 8)})
|
||||
// t.keccak.Write(zero30)
|
||||
// t.keccak.Write(hash)
|
||||
//
|
||||
// hash = t.keccak.Sum(nil)
|
||||
// t.keccak.Reset()
|
||||
//}
|
||||
//if t.trace {
|
||||
// fmt.Printf("hash %v off %d, pref %d\n", hex.EncodeToString(hash), off, len(pref))
|
||||
//}
|
||||
n.hash = hash
|
||||
|
||||
return hash
|
||||
}
|
||||
|
||||
var Zero30 = []byte{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}
|
||||
|
||||
type bitstring []uint8
|
||||
|
||||
func newBitstring(key []byte) bitstring {
|
||||
bits := make([]byte, 8*len(key))
|
||||
for i := range bits {
|
||||
|
||||
if key[i/8]&(1<<(7-i%8)) == 0 {
|
||||
bits[i] = 0
|
||||
} else {
|
||||
bits[i] = 1
|
||||
}
|
||||
}
|
||||
|
||||
return bits
|
||||
}
|
||||
|
||||
func bitstringWithPadding(key []byte, _ int) bitstring {
|
||||
bs := newBitstring(key)
|
||||
if last := key[len(key)-1]; last&0xf0 != 0 {
|
||||
padding := int(0xf0 ^ last)
|
||||
bs = bs[:len(bs)-8-padding]
|
||||
}
|
||||
// bs = bs[:len(bs)-padding-1]
|
||||
return bs
|
||||
}
|
||||
|
||||
func (b bitstring) String() string {
|
||||
var s string
|
||||
for _, bit := range b {
|
||||
switch bit {
|
||||
case 1:
|
||||
s += "1"
|
||||
case 0:
|
||||
s += "0"
|
||||
default:
|
||||
panic(fmt.Errorf("invalid bit %d in bitstring", bit))
|
||||
|
||||
}
|
||||
}
|
||||
return s
|
||||
}
|
||||
|
||||
func (b bitstring) splitPoint(other bitstring) (at int, bit byte, equal bool) {
|
||||
for ; at < len(b) && at < len(other); at++ {
|
||||
if b[at] != other[at] {
|
||||
return at, other[at], false
|
||||
}
|
||||
}
|
||||
|
||||
switch {
|
||||
case len(b) == len(other):
|
||||
return 0, 0, true
|
||||
case at == len(b): // b ends before other
|
||||
return at, other[at], false
|
||||
case at == len(other): // other ends before b
|
||||
return at, b[at], false
|
||||
default:
|
||||
panic("oro")
|
||||
}
|
||||
}
|
||||
|
||||
// Converts b into slice of bytes.
|
||||
// if len of b is not a multiple of 8, we add 1 <= padding <= 7 zeros to the latest byte
|
||||
// and return amount of added zeros
|
||||
func (b bitstring) reconstructHex() (re []byte, padding int) {
|
||||
re = make([]byte, len(b)/8)
|
||||
|
||||
var offt, i int
|
||||
for {
|
||||
bt, ok := b.readByte(offt)
|
||||
if !ok {
|
||||
break
|
||||
}
|
||||
re[i] = bt
|
||||
offt += 8
|
||||
i++
|
||||
}
|
||||
|
||||
if offt >= len(b) {
|
||||
return re, 0
|
||||
}
|
||||
|
||||
padding = offt + 8 - len(b)
|
||||
pd := append(b[offt:], bytes.Repeat([]byte{0}, padding)...)
|
||||
|
||||
last, ok := pd.readByte(0)
|
||||
if !ok {
|
||||
panic(fmt.Errorf("reconstruct failed: padding %d padded size %d", padding, len(pd)))
|
||||
}
|
||||
pad := byte(padding | 0xf0)
|
||||
return append(re, last, pad), padding
|
||||
}
|
||||
|
||||
func (b bitstring) readByte(offsetBits int) (byte, bool) {
|
||||
if len(b) <= offsetBits+7 {
|
||||
return 0, false
|
||||
}
|
||||
return b[offsetBits+7] | b[offsetBits+6]<<1 | b[offsetBits+5]<<2 | b[offsetBits+4]<<3 | b[offsetBits+3]<<4 | b[offsetBits+2]<<5 | b[offsetBits+1]<<6 | b[offsetBits]<<7, true
|
||||
}
|
||||
|
||||
// ExtractPlainKeys parses branchData and extract the plain keys for accounts and storage in the same order
|
||||
// they appear witjin the branchData
|
||||
func ExtractBinPlainKeys(branchData []byte) (accountPlainKeys [][]byte, storagePlainKeys [][]byte, err error) {
|
||||
storagePlainKeys = make([][]byte, 0)
|
||||
accountPlainKeys = make([][]byte, 0)
|
||||
|
||||
touchMap := binary.BigEndian.Uint16(branchData[0:])
|
||||
afterMap := binary.BigEndian.Uint16(branchData[2:])
|
||||
pos := 4
|
||||
|
||||
for bitset, noop := touchMap&afterMap, 0; bitset != 0; noop++ {
|
||||
if pos >= len(branchData) {
|
||||
break
|
||||
}
|
||||
bit := bitset & -bitset
|
||||
|
||||
fieldBits := PartFlags(branchData[pos])
|
||||
pos++
|
||||
|
||||
if fieldBits&ACCOUNT_PLAIN_PART > 0 {
|
||||
n, aux, err := decodeSizedBuffer(branchData[pos:])
|
||||
if err != nil {
|
||||
return nil, nil, fmt.Errorf("extractBinPlainKeys: [%x] account %w", branchData, err)
|
||||
}
|
||||
accountPlainKeys = append(accountPlainKeys, aux)
|
||||
pos += n
|
||||
}
|
||||
|
||||
if fieldBits&STORAGE_PLAIN_PART > 0 {
|
||||
n, aux, err := decodeSizedBuffer(branchData[pos:])
|
||||
if err != nil {
|
||||
return nil, nil, fmt.Errorf("extractBinPlainKeys: storage %w", err)
|
||||
}
|
||||
storagePlainKeys = append(storagePlainKeys, aux)
|
||||
pos += n
|
||||
}
|
||||
if fieldBits&HASH_PART > 0 {
|
||||
n, _, err := decodeSizedBuffer(branchData[pos:])
|
||||
if err != nil {
|
||||
return nil, nil, fmt.Errorf("extractBinPlainKeys: hash %w", err)
|
||||
}
|
||||
pos += n
|
||||
}
|
||||
bitset ^= bit
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
func decodeSizedBuffer(buf []byte) (int, []byte, error) {
|
||||
sz, n := binary.Uvarint(buf)
|
||||
switch {
|
||||
case n == 0:
|
||||
return 0, nil, fmt.Errorf("buffer size too small")
|
||||
case n < 0:
|
||||
return 0, nil, fmt.Errorf("value overflow")
|
||||
default:
|
||||
}
|
||||
size := int(sz)
|
||||
if len(buf) < n+size {
|
||||
return n, []byte{}, fmt.Errorf("encoded size larger than buffer size")
|
||||
}
|
||||
return n + size, buf[n : n+size], nil
|
||||
}
|
||||
|
||||
func ReplaceBinPlainKeys(branchData []byte, accountPlainKeys [][]byte, storagePlainKeys [][]byte, newData []byte) ([]byte, error) {
|
||||
var numBuf [binary.MaxVarintLen64]byte
|
||||
touchMap := binary.BigEndian.Uint16(branchData[0:])
|
||||
afterMap := binary.BigEndian.Uint16(branchData[2:])
|
||||
pos := 4
|
||||
if cap(newData) < 4 {
|
||||
newData = make([]byte, 4)
|
||||
}
|
||||
copy(newData, branchData[:4])
|
||||
|
||||
var accountI, storageI int
|
||||
for bitset, noop := touchMap&afterMap, 0; bitset != 0; noop++ {
|
||||
if pos >= len(branchData) {
|
||||
break
|
||||
}
|
||||
bit := bitset & -bitset
|
||||
|
||||
fieldBits := PartFlags(branchData[pos])
|
||||
newData = append(newData, byte(fieldBits))
|
||||
pos++
|
||||
|
||||
if fieldBits == 0 {
|
||||
continue
|
||||
}
|
||||
|
||||
if fieldBits&ACCOUNT_PLAIN_PART > 0 {
|
||||
ptr, _, err := decodeSizedBuffer(branchData[pos:])
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("replaceBinPlainKeys: account %w", err)
|
||||
}
|
||||
n := binary.PutUvarint(numBuf[:], uint64(len(accountPlainKeys[accountI])))
|
||||
newData = append(newData, numBuf[:n]...)
|
||||
newData = append(newData, accountPlainKeys[accountI]...)
|
||||
accountI++
|
||||
pos += ptr
|
||||
}
|
||||
if fieldBits&STORAGE_PLAIN_PART > 0 {
|
||||
ptr, _, err := decodeSizedBuffer(branchData[pos:])
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("replaceBinPlainKeys: storage %w", err)
|
||||
}
|
||||
n := binary.PutUvarint(numBuf[:], uint64(len(storagePlainKeys[storageI])))
|
||||
newData = append(newData, numBuf[:n]...)
|
||||
newData = append(newData, storagePlainKeys[storageI]...)
|
||||
storageI++
|
||||
pos += ptr
|
||||
}
|
||||
if fieldBits&HASH_PART > 0 {
|
||||
n, _, err := decodeSizedBuffer(branchData[pos:])
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("extractBinPlainKeys: hash %w", err)
|
||||
}
|
||||
newData = append(newData, branchData[pos:pos+n]...)
|
||||
pos += n
|
||||
}
|
||||
bitset ^= bit
|
||||
}
|
||||
return newData, nil
|
||||
}
|
||||
|
||||
func branchToString2(branchData []byte) string {
|
||||
if len(branchData) == 0 {
|
||||
return "{ DELETED }"
|
||||
}
|
||||
touchMap := binary.BigEndian.Uint16(branchData[0:])
|
||||
afterMap := binary.BigEndian.Uint16(branchData[2:])
|
||||
pos := 4
|
||||
|
||||
var sb strings.Builder
|
||||
fmt.Fprintf(&sb, "touchMap %016b, afterMap %016b\n", touchMap, afterMap)
|
||||
|
||||
for bitset, noop := touchMap&afterMap, 0; bitset != 0; noop++ {
|
||||
if pos >= len(branchData) {
|
||||
break
|
||||
}
|
||||
bit := bitset & -bitset
|
||||
if pos >= len(branchData) {
|
||||
break
|
||||
}
|
||||
|
||||
fieldBits := PartFlags(branchData[pos])
|
||||
pos++
|
||||
|
||||
sb.WriteString("{")
|
||||
if fieldBits&ACCOUNT_PLAIN_PART > 0 {
|
||||
n, pk, err := decodeSizedBuffer(branchData[pos:])
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
fmt.Fprintf(&sb, "accountPlainKey=[%x]", pk)
|
||||
pos += n
|
||||
|
||||
}
|
||||
if fieldBits&STORAGE_PLAIN_PART > 0 {
|
||||
n, pk, err := decodeSizedBuffer(branchData[pos:])
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
fmt.Fprintf(&sb, "storagePlainKey=[%x]", pk)
|
||||
pos += n
|
||||
}
|
||||
if fieldBits&HASH_PART > 0 {
|
||||
n, hash, err := decodeSizedBuffer(branchData[pos:])
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
fmt.Fprintf(&sb, " hash=[%x]", hash)
|
||||
pos += n
|
||||
}
|
||||
sb.WriteString(" }\n")
|
||||
bitset ^= bit
|
||||
}
|
||||
return sb.String()
|
||||
}
|
608
commitment/bin_patricia_trie_test.go
Normal file
608
commitment/bin_patricia_trie_test.go
Normal file
@ -0,0 +1,608 @@
|
||||
package commitment
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/hex"
|
||||
"fmt"
|
||||
"math/rand"
|
||||
"testing"
|
||||
|
||||
"github.com/holiman/uint256"
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
||||
func Test_Update(t *testing.T) {
|
||||
|
||||
tests := []struct {
|
||||
key, value []byte
|
||||
}{
|
||||
{key: []byte{12}, value: []byte("notorious")},
|
||||
{key: []byte{14}, value: []byte("2pac")},
|
||||
{key: []byte{15}, value: []byte("eminem")},
|
||||
{key: []byte{11}, value: []byte("big pun")},
|
||||
{key: []byte{20}, value: []byte("method-man")},
|
||||
{key: []byte{18}, value: []byte("fat-joe")},
|
||||
{key: []byte{30}, value: []byte("jay-z")},
|
||||
{key: []byte{5}, value: []byte("redman")},
|
||||
}
|
||||
|
||||
bt := NewBinaryPatriciaTrie()
|
||||
for _, test := range tests {
|
||||
bt.Update(test.key, test.value)
|
||||
}
|
||||
|
||||
require.NotNil(t, bt.root.Node)
|
||||
|
||||
stack := make([]*Node, 0)
|
||||
var stackPtr int
|
||||
|
||||
stack = append(stack, bt.root.Node)
|
||||
stackPtr++
|
||||
visited := make(map[*Node]struct{})
|
||||
|
||||
antipaths := make(map[*Node]string)
|
||||
antipaths[bt.root.Node] = bitstring(bt.root.CommonPrefix).String()
|
||||
|
||||
for len(stack) > 0 {
|
||||
next := stack[stackPtr-1]
|
||||
_, seen := visited[next]
|
||||
if seen {
|
||||
stack = stack[:stackPtr-1]
|
||||
stackPtr--
|
||||
continue
|
||||
}
|
||||
visited[next] = struct{}{}
|
||||
|
||||
if next.Value == nil {
|
||||
//require.Truef(t, next.L != nil || next.R != nil, "if node is not a leaf, at least one child should present")
|
||||
if next.P != nil {
|
||||
require.True(t, next.R != nil && next.L != nil, "bot child should exist L: %p, R: %p", next.L, next.R)
|
||||
}
|
||||
}
|
||||
if next.L != nil || next.R != nil {
|
||||
require.Truef(t, next.Value == nil, "if node has childs, node value should be nil, got %v", next.Value)
|
||||
}
|
||||
if next.L != nil {
|
||||
stack = append(stack, next.L)
|
||||
stackPtr++
|
||||
|
||||
curp := antipaths[next]
|
||||
antipaths[next.L] = curp + bitstring(next.LPrefix).String()
|
||||
|
||||
require.Truef(t, bytes.HasPrefix(next.LPrefix, []byte{0}), "left prefix always begins with 0, got %v", next.LPrefix)
|
||||
}
|
||||
if next.R != nil {
|
||||
stack = append(stack, next.R)
|
||||
stackPtr++
|
||||
|
||||
curp := antipaths[next]
|
||||
antipaths[next.R] = curp + bitstring(next.RPrefix).String()
|
||||
|
||||
require.Truef(t, bytes.HasPrefix(next.RPrefix, []byte{1}), "right prefix always begins with 1, got %v", next.RPrefix)
|
||||
}
|
||||
|
||||
if next.Value != nil {
|
||||
// leaf, go back
|
||||
stack = stack[:stackPtr-1]
|
||||
stackPtr--
|
||||
continue
|
||||
}
|
||||
}
|
||||
|
||||
for node, path := range antipaths {
|
||||
if node.Value == nil {
|
||||
continue
|
||||
}
|
||||
|
||||
if newBitstring(node.Key).String() != path {
|
||||
t.Fatalf("node key %v- %v, path %v", node.Key, newBitstring(node.Key).String(), path)
|
||||
}
|
||||
}
|
||||
|
||||
t.Logf("tree total nodes: %d", len(visited))
|
||||
}
|
||||
|
||||
func Test_Get(t *testing.T) {
|
||||
bt := NewBinaryPatriciaTrie()
|
||||
|
||||
tests := []struct {
|
||||
key, value []byte
|
||||
}{
|
||||
{key: []byte{12}, value: []byte("notorious")},
|
||||
{key: []byte{14}, value: []byte("2pac")},
|
||||
{key: []byte{15}, value: []byte("eminem")},
|
||||
{key: []byte{11}, value: []byte("big pun")},
|
||||
{key: []byte{20}, value: []byte("method-man")},
|
||||
{key: []byte{18}, value: []byte("fat-joe")},
|
||||
{key: []byte{30}, value: []byte("jay-z")},
|
||||
{key: []byte{5}, value: []byte("redman")},
|
||||
}
|
||||
|
||||
for _, test := range tests {
|
||||
bt.Update(test.key, test.value)
|
||||
}
|
||||
|
||||
require.NotNil(t, bt.root.Node)
|
||||
|
||||
for _, test := range tests {
|
||||
buf, ok := bt.Get(test.key)
|
||||
require.Truef(t, ok, "key %v not found", test.key)
|
||||
require.EqualValues(t, test.value, buf)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
func Test_BinaryPatriciaTrie_ProcessUpdates(t *testing.T) {
|
||||
bt := NewBinaryPatriciaTrie()
|
||||
|
||||
builder := NewUpdateBuilder().
|
||||
Balance("9a", 100000).
|
||||
Balance("e8", 200000).
|
||||
Balance("a2", 300000).
|
||||
Balance("f0", 400000).
|
||||
Balance("af", 500000).
|
||||
Balance("33", 600000).
|
||||
Nonce("aa", 184)
|
||||
|
||||
plainKeys, hashedKeys, updates := builder.Build()
|
||||
bt.SetTrace(true)
|
||||
_, err := bt.ProcessUpdates(plainKeys, hashedKeys, updates)
|
||||
require.NoError(t, err)
|
||||
|
||||
require.NotNil(t, bt.root.Node)
|
||||
|
||||
stack := make([]*Node, 0)
|
||||
var stackPtr int
|
||||
|
||||
stack = append(stack, bt.root.Node)
|
||||
stackPtr++
|
||||
visited := make(map[*Node]struct{})
|
||||
|
||||
// validity check
|
||||
for len(stack) > 0 {
|
||||
next := stack[stackPtr-1]
|
||||
_, seen := visited[next]
|
||||
if seen {
|
||||
stack = stack[:stackPtr-1]
|
||||
stackPtr--
|
||||
continue
|
||||
}
|
||||
visited[next] = struct{}{}
|
||||
|
||||
if next.Value == nil {
|
||||
require.Truef(t, next.L != nil || next.R != nil, "if node is not a leaf, at least one child should present")
|
||||
if next.P != nil {
|
||||
require.True(t, next.R != nil && next.L != nil, "bot child should exist L: %p, R: %p", next.L, next.R)
|
||||
}
|
||||
}
|
||||
if next.L != nil || next.R != nil {
|
||||
require.Truef(t, next.Value == nil, "if node has childs, node value should be nil, got %v", next.Value)
|
||||
}
|
||||
if next.L != nil {
|
||||
stack = append(stack, next.L)
|
||||
stackPtr++
|
||||
|
||||
require.Truef(t, bytes.HasPrefix(next.LPrefix, []byte{0}), "left prefix always begins with 0, got %v", next.LPrefix)
|
||||
}
|
||||
if next.R != nil {
|
||||
stack = append(stack, next.R)
|
||||
stackPtr++
|
||||
|
||||
require.Truef(t, bytes.HasPrefix(next.RPrefix, []byte{1}), "right prefix always begins with 1, got %v", next.RPrefix)
|
||||
}
|
||||
|
||||
if next.Value != nil {
|
||||
// leaf, go back
|
||||
stack = stack[:stackPtr-1]
|
||||
stackPtr--
|
||||
continue
|
||||
}
|
||||
}
|
||||
rootHash, _ := bt.RootHash()
|
||||
require.Len(t, rootHash, 32)
|
||||
fmt.Printf("%+v\n", hex.EncodeToString(rootHash))
|
||||
t.Logf("tree total nodes: %d", len(visited))
|
||||
}
|
||||
|
||||
func Test_BinaryPatriciaTrie_UniqueRepresentation(t *testing.T) {
|
||||
trieSequential := NewBinaryPatriciaTrie()
|
||||
|
||||
builder := NewUpdateBuilder().
|
||||
Balance("9a", 100000).
|
||||
Balance("e8", 200000).
|
||||
Balance("a2", 300000).
|
||||
Balance("f0", 400000).
|
||||
Balance("af", 500000).
|
||||
Balance("33", 600000).
|
||||
Nonce("aa", 184)
|
||||
|
||||
plainKeys, hashedKeys, updates := builder.Build()
|
||||
|
||||
emptyHash, _ := trieSequential.RootHash()
|
||||
require.EqualValues(t, EmptyRootHash, emptyHash)
|
||||
|
||||
for i := 0; i < len(plainKeys); i++ {
|
||||
trieSequential.ProcessUpdates(plainKeys[i:i+1], hashedKeys[i:i+1], updates[i:i+1])
|
||||
sequentialHash, _ := trieSequential.RootHash()
|
||||
require.Len(t, sequentialHash, 32)
|
||||
}
|
||||
|
||||
trieBatch := NewBinaryPatriciaTrie()
|
||||
trieBatch.SetTrace(true)
|
||||
trieBatch.ProcessUpdates(plainKeys, hashedKeys, updates)
|
||||
|
||||
sequentialHash, _ := trieSequential.RootHash()
|
||||
batchHash, _ := trieBatch.RootHash()
|
||||
|
||||
require.EqualValues(t, batchHash, sequentialHash)
|
||||
}
|
||||
|
||||
func Test_BinaryPatriciaTrie_BranchEncoding(t *testing.T) {
|
||||
builder := NewUpdateBuilder().
|
||||
Balance("9a", 100000).
|
||||
Balance("e8", 200000).
|
||||
Balance("a2", 300000).
|
||||
Balance("f0", 400000).
|
||||
Balance("af", 500000).
|
||||
Balance("33", 600000).
|
||||
Nonce("aa", 184)
|
||||
|
||||
plainKeys, hashedKeys, updates := builder.Build()
|
||||
|
||||
trie := NewBinaryPatriciaTrie()
|
||||
|
||||
emptyHash, _ := trie.RootHash()
|
||||
require.EqualValues(t, EmptyRootHash, emptyHash)
|
||||
|
||||
trie.SetTrace(true)
|
||||
|
||||
branchUpdates, err := trie.ProcessUpdates(plainKeys, hashedKeys, updates)
|
||||
require.NoError(t, err)
|
||||
require.NotEmpty(t, branchUpdates)
|
||||
|
||||
//sequentialHash, _ := trie.RootHash()
|
||||
//expectedRoot, _ := hex.DecodeString("87809bbb5282c01ac13cac744db5fee083882e93f781d6af2ad028455d5bdaac")
|
||||
//
|
||||
//require.EqualValues(t, expectedRoot, sequentialHash)
|
||||
|
||||
for pref, update := range branchUpdates {
|
||||
account, _, _ := ExtractBinPlainKeys(update)
|
||||
t.Logf("pref %v: accounts:", pref)
|
||||
for _, acc := range account {
|
||||
t.Logf("\t%s\n", hex.EncodeToString(acc))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func Test_ReplaceBinPlainKeys(t *testing.T) {
|
||||
v, err := hex.DecodeString("0000000310ea0300000000000001010000000000000001000100000000010001010000000000010101000000000101000100000000000101000000000000010001000000000100010000000000010000000000000000010000000000000100010000000000000100010000000001000000000000000100000100000000000000010000000000000101000000000100010000000000010001000000000001000000000000000101010000000000010101000000000000000101000000000000000000000000000000000000000001000000000000000001010100000000000100000000000000010001000000000000000000000000010101010000000001010100000000000000000100000000010000010000000001010101000000000100010000000000000101010000000001010000000000000000010100000000010100010000000000000100000000000000010000000000000000000000000001000101000000000101010100000000000001000000000001000001000000000101010000000000010001010000000001010101000000000101010000000000010001000000000000000001000000000001010000000000000001010000000001000100000000000100010000000000010101000000000000010100000000000101010100000000000000000000000001010000ea03010000000000010001000000000000010101000000000000000100000000010001010000000000010101000000000000000000000000010000010000000001010101000000000101010000000000010000000000000001000101000000000000010100000000000101010000000000010100000000000000010100000000010100000000000001010100000000000100010000000000010101010000000001010101000000000001000000000000010000000000000000000001000000000000000100000000000100000000000000000101000000000101000100000000000100000000000000010100000000000001000000000000000001000000000000010101000000000001000100000000000001010000000001010101000000000100000000000000010100010000000001000101000000000101010100000000010001000000000000010100000000000101010100000000000101000000000000000100000000000000000100000000010100010000000000010100000000000000010000000000000001000000000001010100000000000000000000000000010000010000000001010000000000000100000000000000010001010000000000010000000000000100000000000000000100000000000001000100000000000001010000000000010001010214b910f4453d5fa062f828caf3b0e2adff3824407c24e68080a000000000000000000000000000000000000000000000000000000000000000000214d2798468b343c4b104ecc2585e1c1b57b7c1807424e68080a00000000000000000000000000000000000000000000000000000000000000000")
|
||||
require.NoError(t, err)
|
||||
|
||||
accountPlainKeys := make([][]byte, 0)
|
||||
accountPlainKeys = append(accountPlainKeys, []byte("1a"))
|
||||
accountPlainKeys = append(accountPlainKeys, []byte("b1"))
|
||||
|
||||
storageKeys := make([][]byte, 0)
|
||||
buf := make([]byte, 0)
|
||||
fin, err := ReplaceBinPlainKeys(v, accountPlainKeys, storageKeys, buf)
|
||||
require.NoError(t, err)
|
||||
require.NotEmpty(t, fin)
|
||||
}
|
||||
|
||||
func Test_ReplaceBinPlainKeys2(t *testing.T) {
|
||||
//rnd := rand.New(rand.NewSource(time.Now().UnixNano()))
|
||||
key := make([]byte, 52)
|
||||
|
||||
_, err := rand.Read(key)
|
||||
require.NoError(t, err)
|
||||
|
||||
L := &Node{Key: key, Value: []byte("aa"), isStorage: true}
|
||||
buf := encodeNodeUpdate(L, 1, 1, nil)
|
||||
|
||||
accountPlainKeys, storageKeys, err := ExtractBinPlainKeys(buf)
|
||||
require.NoError(t, err)
|
||||
require.Empty(t, accountPlainKeys)
|
||||
require.Contains(t, storageKeys, key)
|
||||
|
||||
newStorageKeys := make([][]byte, 0)
|
||||
newStorageKeys = append(newStorageKeys, []byte("1aa0"))
|
||||
//fin, err := ReplacePlainKeys(v, accountPlainKeys, storageKeys, buf)
|
||||
|
||||
fin := make([]byte, 0)
|
||||
fin, err = ReplaceBinPlainKeys(buf, accountPlainKeys, newStorageKeys, fin)
|
||||
require.NoError(t, err)
|
||||
require.NotEmpty(t, fin)
|
||||
require.NotEqualValues(t, fin, buf)
|
||||
|
||||
accountPlainKeys, storageKeys, err = ExtractBinPlainKeys(fin)
|
||||
require.NoError(t, err)
|
||||
require.Empty(t, accountPlainKeys)
|
||||
require.Contains(t, storageKeys, newStorageKeys[0])
|
||||
}
|
||||
|
||||
func Test_EncodeUpdate_Storage(t *testing.T) {
|
||||
L := &Node{Key: []byte("1aa0"), Value: []byte("aa"), isStorage: true}
|
||||
|
||||
buf := encodeNodeUpdate(L, 1, 1, nil)
|
||||
|
||||
accountPlainKeys, storagePlainKeys, err := ExtractBinPlainKeys(buf)
|
||||
require.NoError(t, err)
|
||||
require.Len(t, storagePlainKeys, 1)
|
||||
require.Len(t, accountPlainKeys, 0)
|
||||
require.Contains(t, storagePlainKeys, L.Key)
|
||||
|
||||
newAccountPlainKeys := make([][]byte, 0)
|
||||
newAccountPlainKeys = append(newAccountPlainKeys, []byte("11a"))
|
||||
newStorageKeys := make([][]byte, 0)
|
||||
newStorageKeys = append(newStorageKeys, []byte("7770"))
|
||||
|
||||
fin, err := ReplaceBinPlainKeys(buf, newAccountPlainKeys, newStorageKeys, []byte{})
|
||||
require.NoError(t, err)
|
||||
require.NotEmpty(t, fin)
|
||||
|
||||
accountPlainKeys, storagePlainKeys, err = ExtractBinPlainKeys(fin)
|
||||
require.NoError(t, err)
|
||||
require.Len(t, storagePlainKeys, 1)
|
||||
require.Len(t, accountPlainKeys, 0)
|
||||
require.Contains(t, storagePlainKeys, newStorageKeys[0])
|
||||
|
||||
// ====================== replace account plain key
|
||||
|
||||
R := &Node{Key: []byte("1a"), Value: []byte("bb")}
|
||||
buf = encodeNodeUpdate(R, 2, 2, nil)
|
||||
|
||||
accountPlainKeys, storagePlainKeys, err = ExtractBinPlainKeys(buf)
|
||||
require.NoError(t, err)
|
||||
require.Len(t, storagePlainKeys, 0)
|
||||
require.Len(t, accountPlainKeys, 1)
|
||||
require.Contains(t, accountPlainKeys, R.Key)
|
||||
|
||||
fin, err = ReplaceBinPlainKeys(buf, newAccountPlainKeys, newStorageKeys, []byte{})
|
||||
require.NoError(t, err)
|
||||
require.NotEmpty(t, fin)
|
||||
|
||||
accountPlainKeys, storagePlainKeys, err = ExtractBinPlainKeys(fin)
|
||||
require.NoError(t, err)
|
||||
require.Len(t, storagePlainKeys, 0)
|
||||
require.Len(t, accountPlainKeys, 1)
|
||||
require.Contains(t, accountPlainKeys, newAccountPlainKeys[0])
|
||||
}
|
||||
|
||||
func Test_bitstring_encode_decode_padding(t *testing.T) {
|
||||
key, err := hex.DecodeString("db3164534fec08b5a86ae5dda0a997a63f2ee408")
|
||||
require.NoError(t, err)
|
||||
|
||||
bs := newBitstring(key)
|
||||
re, padding := bs.reconstructHex()
|
||||
require.Zerof(t, padding, "padding should be zero")
|
||||
require.EqualValues(t, key, re)
|
||||
}
|
||||
|
||||
func Test_bitstring_encode_decode_empty(t *testing.T) {
|
||||
re, pad := bitstring{}.reconstructHex()
|
||||
require.EqualValues(t, bitstring{}, re)
|
||||
require.EqualValues(t, 0, pad)
|
||||
}
|
||||
|
||||
func Test_bitstring_encode_decode_one_padding(t *testing.T) {
|
||||
bs := bitstring{1}
|
||||
re, pad := bs.reconstructHex()
|
||||
require.EqualValues(t, 7, pad)
|
||||
require.EqualValues(t, []byte{1 << 7, byte(0xf0 | pad)}, re)
|
||||
|
||||
bs2 := bitstringWithPadding(re, pad)
|
||||
require.EqualValues(t, bs, bs2)
|
||||
}
|
||||
|
||||
func Test_bitstring_encode_decode_padding_notzero(t *testing.T) {
|
||||
t.Skip("Failing")
|
||||
key, err := hex.DecodeString("db3164534fec08b5a86ae5dda0a997a63f2ee408")
|
||||
require.NoError(t, err)
|
||||
|
||||
bs := newBitstring(key)
|
||||
offt := 3 // last byte is 08 => 1000, chop last three zeros
|
||||
|
||||
chop := bs[len(bs)-offt:]
|
||||
bs = bs[:len(bs)-offt]
|
||||
_ = chop
|
||||
re, padding := bs.reconstructHex() // during reconstruction padding will be applied - add 3 chopped zero
|
||||
require.EqualValues(t, offt, padding)
|
||||
require.EqualValues(t, key, re)
|
||||
}
|
||||
|
||||
func Test_BinaryPatriciaTrie_ProcessUpdatesDelete(t *testing.T) {
|
||||
bt := NewBinaryPatriciaTrie()
|
||||
|
||||
builder := NewUpdateBuilder().
|
||||
Balance("ffff", 200000).
|
||||
Balance("feff", 300000).
|
||||
Balance("ffdf", 400000).
|
||||
Balance("fedf", 400000)
|
||||
|
||||
plainKeys, hashedKeys, updates := builder.Build()
|
||||
|
||||
bt.SetTrace(true)
|
||||
_, err := bt.ProcessUpdates(plainKeys, hashedKeys, updates)
|
||||
require.NotNil(t, bt.root.Node)
|
||||
require.NoError(t, err)
|
||||
t.Logf("trie stat: %s", bt.StatString())
|
||||
|
||||
builder = NewUpdateBuilder().
|
||||
Delete("fedf").
|
||||
Delete("ffff")
|
||||
|
||||
plainKeys, hashedKeys, updates = builder.Build()
|
||||
require.NoError(t, err)
|
||||
|
||||
_, err = bt.ProcessUpdates(plainKeys, hashedKeys, updates)
|
||||
require.NotNil(t, bt.root.Node)
|
||||
require.NoError(t, err)
|
||||
|
||||
for i, key := range hashedKeys {
|
||||
v, ok := bt.Get(key) // keys "af" and "e8" should be deleted
|
||||
if len(v) > 0 {
|
||||
t.Logf("key %x: %v", hashedKeys[i], new(Account).decode(v).String())
|
||||
}
|
||||
require.Emptyf(t, v, "value for key %x should be not empty", plainKeys[i])
|
||||
require.False(t, ok)
|
||||
}
|
||||
|
||||
stack := make([]*Node, 0)
|
||||
var stackPtr int
|
||||
|
||||
stack = append(stack, bt.root.Node)
|
||||
stackPtr++
|
||||
visited := make(map[*Node]struct{})
|
||||
|
||||
// validity check
|
||||
for len(stack) > 0 {
|
||||
next := stack[stackPtr-1]
|
||||
_, seen := visited[next]
|
||||
if seen {
|
||||
stack = stack[:stackPtr-1]
|
||||
stackPtr--
|
||||
continue
|
||||
}
|
||||
visited[next] = struct{}{}
|
||||
|
||||
if next.Value == nil {
|
||||
require.Truef(t, next.L != nil || next.R != nil, "if node is not a leaf, at least one child should present")
|
||||
if next.P != nil {
|
||||
require.True(t, next.R != nil && next.L != nil, "bot child should exist L: %p, R: %p", next.L, next.R)
|
||||
}
|
||||
}
|
||||
if next.L != nil || next.R != nil {
|
||||
require.Truef(t, next.Value == nil, "if node has childs, node value should be nil, got %v", next.Value)
|
||||
}
|
||||
if next.L != nil {
|
||||
stack = append(stack, next.L)
|
||||
stackPtr++
|
||||
|
||||
require.Truef(t, bytes.HasPrefix(next.LPrefix, []byte{0}), "left prefix always begins with 0, got %v", next.LPrefix)
|
||||
}
|
||||
if next.R != nil {
|
||||
stack = append(stack, next.R)
|
||||
stackPtr++
|
||||
|
||||
require.Truef(t, bytes.HasPrefix(next.RPrefix, []byte{1}), "right prefix always begins with 1, got %v", next.RPrefix)
|
||||
}
|
||||
|
||||
if next.Value != nil {
|
||||
// leaf, go back
|
||||
stack = stack[:stackPtr-1]
|
||||
stackPtr--
|
||||
continue
|
||||
}
|
||||
}
|
||||
rootHash, _ := bt.RootHash()
|
||||
require.Len(t, rootHash, 32)
|
||||
fmt.Printf("%+v\n", hex.EncodeToString(rootHash))
|
||||
t.Logf("tree total nodes: %d", len(visited))
|
||||
}
|
||||
|
||||
func Test_BinaryPatriciaTrie_ProcessStorageUpdates(t *testing.T) {
|
||||
bt := NewBinaryPatriciaTrie()
|
||||
|
||||
builder := NewUpdateBuilder().
|
||||
Storage("e8", "02", "98").
|
||||
Balance("e8", 1337)
|
||||
|
||||
plainKeys, hashedKeys, updates := builder.Build()
|
||||
|
||||
bt.SetTrace(true)
|
||||
_, err := bt.ProcessUpdates(plainKeys, hashedKeys, updates)
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, bt.root.Node)
|
||||
|
||||
checkPlainKeys, checkHashedKeys, _ := NewUpdateBuilder().Delete("e8").Build()
|
||||
|
||||
av, exist := bt.Get(checkHashedKeys[0])
|
||||
require.Truef(t, exist, "key %x should exist", checkPlainKeys[0])
|
||||
acc := new(Account).decode(av)
|
||||
require.Truef(t, acc.Balance.Eq(uint256.NewInt(1337)), "balance should be 1337, got %v", acc.Balance)
|
||||
|
||||
accountPlainKey, accountHashedKey, upd := NewUpdateBuilder().DeleteStorage("e8", "02").Build()
|
||||
_, err = bt.ProcessUpdates(accountPlainKey, accountHashedKey, upd)
|
||||
require.NoError(t, err)
|
||||
|
||||
for i, key := range accountHashedKey {
|
||||
v, ok := bt.Get(key)
|
||||
require.Emptyf(t, v, "value for key %x should be empty", accountPlainKey[i])
|
||||
require.False(t, ok)
|
||||
}
|
||||
|
||||
stack := make([]*Node, 0)
|
||||
var stackPtr int
|
||||
|
||||
stack = append(stack, bt.root.Node)
|
||||
stackPtr++
|
||||
visited := make(map[*Node]struct{})
|
||||
|
||||
// validity check
|
||||
for len(stack) > 0 {
|
||||
next := stack[stackPtr-1]
|
||||
_, seen := visited[next]
|
||||
if seen {
|
||||
stack = stack[:stackPtr-1]
|
||||
stackPtr--
|
||||
continue
|
||||
}
|
||||
visited[next] = struct{}{}
|
||||
|
||||
if next.Value == nil {
|
||||
require.Truef(t, next.L != nil || next.R != nil, "if node is not a leaf, at least one child should present")
|
||||
if next.P != nil {
|
||||
require.True(t, next.R != nil && next.L != nil, "bot child should exist L: %p, R: %p", next.L, next.R)
|
||||
}
|
||||
}
|
||||
if next.L != nil || next.R != nil {
|
||||
require.Truef(t, next.Value == nil, "if node has childs, node value should be nil, got %v", next.Value)
|
||||
}
|
||||
if next.L != nil {
|
||||
stack = append(stack, next.L)
|
||||
stackPtr++
|
||||
if len(next.LPrefix) == 0 {
|
||||
require.NotNilf(t, next.L.Value, "if left prefix is empty, left child MUST be leaf and there MUST be another child down on path, got branch")
|
||||
} else {
|
||||
require.Truef(t, bytes.HasPrefix(next.LPrefix, []byte{0}), "left prefix always begins with 0, got %v", next.LPrefix)
|
||||
}
|
||||
}
|
||||
if next.R != nil {
|
||||
stack = append(stack, next.R)
|
||||
stackPtr++
|
||||
if len(next.RPrefix) == 0 {
|
||||
require.NotNilf(t, next.R.Value, "if right prefix is nil, right child MUST be leaf, got branch")
|
||||
} else {
|
||||
require.Truef(t, bytes.HasPrefix(next.RPrefix, []byte{1}), "right prefix always begins with 1, got %v", next.RPrefix)
|
||||
}
|
||||
}
|
||||
|
||||
if next.Value != nil {
|
||||
// leaf, go back
|
||||
stack = stack[:stackPtr-1]
|
||||
stackPtr--
|
||||
continue
|
||||
}
|
||||
}
|
||||
rootHash, _ := bt.RootHash()
|
||||
require.Len(t, rootHash, 32)
|
||||
fmt.Printf("%+v\n", hex.EncodeToString(rootHash))
|
||||
t.Logf("tree total nodes: %d", len(visited))
|
||||
}
|
||||
|
||||
func Test_encodeNode(t *testing.T) {
|
||||
builder := NewUpdateBuilder().
|
||||
Balance("ff", 255).
|
||||
Balance("fd", 253).
|
||||
Balance("fe", 254)
|
||||
|
||||
apk, _, upd := builder.Build()
|
||||
trie := NewBinaryPatriciaTrie()
|
||||
trie.trace = true
|
||||
for i := 0; i < len(upd); i++ {
|
||||
updates, err := trie.ProcessUpdates(apk[i:i+1], apk[i:i+1], upd[i:i+1])
|
||||
require.NoError(t, err)
|
||||
require.NotEmpty(t, updates)
|
||||
fmt.Printf("-----\n")
|
||||
}
|
||||
}
|
126
commitment/commitment.go
Normal file
126
commitment/commitment.go
Normal file
@ -0,0 +1,126 @@
|
||||
package commitment
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/binary"
|
||||
"fmt"
|
||||
|
||||
"github.com/holiman/uint256"
|
||||
|
||||
"github.com/ledgerwatch/erigon-lib/common/length"
|
||||
)
|
||||
|
||||
// Trie represents commitment variant.
|
||||
type Trie interface {
|
||||
ProcessUpdates(plainKeys, hashedKeys [][]byte, updates []Update) (branchNodeUpdates map[string][]byte, err error)
|
||||
|
||||
// RootHash produces root hash of the trie
|
||||
RootHash() (hash []byte, err error)
|
||||
|
||||
// Variant returns commitment trie variant
|
||||
Variant() TrieVariant
|
||||
|
||||
// Reset Drops everything from the trie
|
||||
Reset()
|
||||
|
||||
ResetFns(
|
||||
branchFn func(prefix []byte) ([]byte, error),
|
||||
accountFn func(plainKey []byte, cell *Cell) error,
|
||||
storageFn func(plainKey []byte, cell *Cell) error,
|
||||
)
|
||||
|
||||
// Makes trie more verbose
|
||||
SetTrace(bool)
|
||||
}
|
||||
|
||||
type TrieVariant string
|
||||
|
||||
const (
|
||||
// HexPatriciaHashed used as default commitment approach
|
||||
VariantHexPatriciaTrie TrieVariant = "hex-patricia-hashed"
|
||||
VariantReducedHexPatriciaTrie TrieVariant = "modified-hex-patricia-hashed"
|
||||
// Experimental mode with binary key representation
|
||||
VariantBinPatriciaTrie TrieVariant = "bin-patricia-hashed"
|
||||
)
|
||||
|
||||
func InitializeTrie(tv TrieVariant) Trie {
|
||||
switch tv {
|
||||
case VariantBinPatriciaTrie:
|
||||
return NewBinaryPatriciaTrie()
|
||||
case VariantReducedHexPatriciaTrie:
|
||||
return NewBinHashed(length.Addr, nil, nil, nil)
|
||||
case VariantHexPatriciaTrie:
|
||||
fallthrough
|
||||
default:
|
||||
return NewHexPatriciaHashed(length.Addr, nil, nil, nil)
|
||||
}
|
||||
}
|
||||
|
||||
type Account struct {
|
||||
CodeHash []byte // hash of the bytecode
|
||||
Nonce uint64
|
||||
Balance uint256.Int
|
||||
}
|
||||
|
||||
func (a *Account) String() string {
|
||||
return fmt.Sprintf("Account{Nonce: %d, Balance: %s, CodeHash: %x}", a.Nonce, a.Balance.String(), a.CodeHash)
|
||||
}
|
||||
|
||||
func (a *Account) decode(buffer []byte) *Account {
|
||||
var pos int
|
||||
if buffer[pos] < 128 {
|
||||
a.Nonce = uint64(buffer[pos])
|
||||
pos++
|
||||
} else {
|
||||
var nonce uint64
|
||||
sizeBytes := int(buffer[pos] - 128)
|
||||
pos++
|
||||
nonce, n := binary.Uvarint(buffer[pos : pos+sizeBytes])
|
||||
a.Nonce = nonce
|
||||
pos += n
|
||||
}
|
||||
|
||||
if buffer[pos] < 128 {
|
||||
b := uint256.NewInt(uint64(buffer[pos]))
|
||||
a.Balance = *b
|
||||
pos++
|
||||
} else {
|
||||
bc := int(buffer[pos] - 128)
|
||||
pos++
|
||||
a.Balance.SetBytes(buffer[pos : pos+bc])
|
||||
pos += bc
|
||||
}
|
||||
|
||||
codeSize := int(buffer[pos] - 128)
|
||||
if codeSize > 0 {
|
||||
pos++
|
||||
a.CodeHash = make([]byte, codeSize)
|
||||
copy(a.CodeHash, buffer[pos:pos+codeSize])
|
||||
}
|
||||
return a
|
||||
}
|
||||
|
||||
func (a *Account) encode(bb []byte) []byte {
|
||||
buffer := bytes.NewBuffer(bb)
|
||||
|
||||
// Encoding nonce
|
||||
if a.Nonce < 128 && a.Nonce != 0 {
|
||||
buffer.WriteByte(byte(a.Nonce))
|
||||
} else {
|
||||
aux := [binary.MaxVarintLen64]byte{}
|
||||
n := binary.PutUvarint(aux[:], a.Nonce)
|
||||
buffer.WriteByte(byte(128 + n))
|
||||
buffer.Write(aux[:n])
|
||||
}
|
||||
|
||||
// Encoding balance
|
||||
if a.Balance.LtUint64(128) && !a.Balance.IsZero() {
|
||||
buffer.WriteByte(byte(a.Balance.Uint64()))
|
||||
} else {
|
||||
buffer.WriteByte(byte(128 + a.Balance.ByteLen()))
|
||||
buffer.Write(a.Balance.Bytes())
|
||||
}
|
||||
buffer.WriteByte(byte(128 + len(a.CodeHash)))
|
||||
buffer.Write(a.CodeHash)
|
||||
return buffer.Bytes()
|
||||
}
|
74
commitment/commitment_test.go
Normal file
74
commitment/commitment_test.go
Normal file
@ -0,0 +1,74 @@
|
||||
package commitment
|
||||
|
||||
import (
|
||||
"crypto/rand"
|
||||
"testing"
|
||||
|
||||
"github.com/holiman/uint256"
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
||||
func Test_AccountEncodeDecode(t *testing.T) {
|
||||
balance := uint256.NewInt(1002020020)
|
||||
acc := &Account{
|
||||
Nonce: 1913453,
|
||||
CodeHash: []byte{10, 20, 30, 10},
|
||||
Balance: *balance,
|
||||
}
|
||||
rand.Read(acc.CodeHash[:])
|
||||
|
||||
aux := make([]byte, 0)
|
||||
aux = acc.encode(aux)
|
||||
require.NotEmpty(t, aux)
|
||||
|
||||
bcc := new(Account)
|
||||
bcc.decode(aux)
|
||||
|
||||
c := new(Account) //.decode([]byte{128, 0, 1, 128})
|
||||
ff := c.encode(nil)
|
||||
require.NotEmpty(t, ff)
|
||||
|
||||
_ = c
|
||||
require.EqualValues(t, acc.Nonce, bcc.Nonce)
|
||||
require.True(t, acc.Balance.Eq(&bcc.Balance))
|
||||
require.EqualValues(t, acc.CodeHash, bcc.CodeHash)
|
||||
}
|
||||
|
||||
func Test_BinPatriciaTrie_UniqueRepresentation(t *testing.T) {
|
||||
trie := NewBinaryPatriciaTrie()
|
||||
trieBatch := NewBinaryPatriciaTrie()
|
||||
|
||||
plainKeys, hashedKeys, updates := NewUpdateBuilder().
|
||||
Balance("01", 12).
|
||||
Balance("f1", 120000).
|
||||
Nonce("aa", 152512).
|
||||
Balance("9a", 100000).
|
||||
Balance("e8", 200000).
|
||||
Balance("a2", 300000).
|
||||
Balance("f0", 400000).
|
||||
Balance("af", 500000).
|
||||
Balance("33", 600000).
|
||||
Nonce("aa", 184).
|
||||
Build()
|
||||
|
||||
for i := 0; i < len(updates); i++ {
|
||||
_, err := trie.ProcessUpdates(plainKeys[i:i+1], hashedKeys[i:i+1], updates[i:i+1])
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
trieBatch.ProcessUpdates(plainKeys, hashedKeys, updates)
|
||||
|
||||
hash, _ := trie.RootHash()
|
||||
require.Len(t, hash, 32)
|
||||
|
||||
batchHash, _ := trieBatch.RootHash()
|
||||
require.EqualValues(t, hash, batchHash)
|
||||
|
||||
for i, hkey := range hashedKeys {
|
||||
buf, ok := trie.Get(hkey)
|
||||
require.Truef(t, ok, "key %x should be present, but not found", plainKeys[i])
|
||||
buf2, ok := trieBatch.Get(hkey)
|
||||
require.True(t, ok)
|
||||
require.EqualValues(t, buf2, buf)
|
||||
}
|
||||
}
|
@ -843,7 +843,6 @@ func (hph *HexPatriciaHashed) needUnfolding(hashedKey []byte) int {
|
||||
if hph.trace {
|
||||
fmt.Printf("needUnfolding root, rootChecked = %t\n", hph.rootChecked)
|
||||
}
|
||||
//hph.activeRows++
|
||||
cell = &hph.root
|
||||
if cell.downHashedLen == 0 && cell.hl == 0 && !hph.rootChecked {
|
||||
// Need to attempt to unfold the root
|
||||
@ -885,11 +884,7 @@ func (hph *HexPatriciaHashed) needUnfolding(hashedKey []byte) int {
|
||||
}
|
||||
|
||||
func (hph *HexPatriciaHashed) unfoldBranchNode(row int, deleted bool, depth int) error {
|
||||
key := hph.currentKey[:hph.currentKeyLen]
|
||||
//if len(key) == 0 {
|
||||
// return nil
|
||||
//}
|
||||
branchData, err := hph.branchFn(hexToCompact(key))
|
||||
branchData, err := hph.branchFn(hexToCompact(hph.currentKey[:hph.currentKeyLen]))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -1042,10 +1037,6 @@ func (hph *HexPatriciaHashed) needFolding(hashedKey []byte) bool {
|
||||
// (in other words until the needFolding function returns 0)
|
||||
func (hph *HexPatriciaHashed) fold() ([]byte, []byte, error) {
|
||||
updateKeyLen := hph.currentKeyLen
|
||||
//if updateKeyLen == 0 {
|
||||
// hph.activeRows--
|
||||
// return nil, nil, nil
|
||||
//}
|
||||
if hph.activeRows == 0 {
|
||||
return nil, nil, fmt.Errorf("cannot fold - no active rows")
|
||||
}
|
||||
@ -1200,7 +1191,7 @@ func (hph *HexPatriciaHashed) fold() ([]byte, []byte, error) {
|
||||
if hph.trace {
|
||||
fmt.Printf("%x: computeCellHash(%d,%x,depth=%d)=[%x]\n", nibble, row, nibble, depth, cellHash)
|
||||
}
|
||||
if _, err := hph.keccak2.Write(cellHash); err != nil {
|
||||
if _, err = hph.keccak2.Write(cellHash); err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
if bitmap&bit != 0 {
|
||||
@ -1321,19 +1312,10 @@ func (hph *HexPatriciaHashed) deleteCell(hashedKey []byte) {
|
||||
|
||||
func (hph *HexPatriciaHashed) updateAccount(plainKey, hashedKey []byte) *Cell {
|
||||
var cell *Cell
|
||||
var col int
|
||||
var depth int
|
||||
var col, depth int
|
||||
if hph.activeRows == 0 {
|
||||
//if hph.root.apl == 0 {
|
||||
hph.activeRows++
|
||||
//} else {
|
||||
// //Update the root
|
||||
// cell = &hph.root
|
||||
//hph.rootTouched = true
|
||||
//hph.rootPresent = true
|
||||
//}
|
||||
}
|
||||
//} else {
|
||||
row := hph.activeRows - 1
|
||||
depth = hph.depths[hph.activeRows-1]
|
||||
col = int(hashedKey[hph.currentKeyLen])
|
||||
@ -1343,7 +1325,6 @@ func (hph *HexPatriciaHashed) updateAccount(plainKey, hashedKey []byte) *Cell {
|
||||
if hph.trace {
|
||||
fmt.Printf("updateAccount setting (%d, %x), depth=%d\n", row, col, depth)
|
||||
}
|
||||
//}
|
||||
if cell.downHashedLen == 0 {
|
||||
copy(cell.downHashedKey[:], hashedKey[depth:])
|
||||
cell.downHashedLen = len(hashedKey) - depth
|
||||
@ -1388,16 +1369,10 @@ func (hph *HexPatriciaHashed) updateStorage(plainKey, hashedKey, value []byte) {
|
||||
if hph.trace {
|
||||
fmt.Printf("updateStorage, activeRows = %d\n", hph.activeRows)
|
||||
}
|
||||
var col int
|
||||
var depth int
|
||||
var col, depth int
|
||||
var cell *Cell
|
||||
if hph.activeRows == 0 {
|
||||
// Update the root
|
||||
hph.activeRows++
|
||||
//cell = &hph.root
|
||||
//hph.rootTouched = true
|
||||
//hph.rootPresent = true
|
||||
//} else {
|
||||
}
|
||||
depth = hph.depths[hph.activeRows-1]
|
||||
col = int(hashedKey[hph.currentKeyLen])
|
||||
@ -1407,7 +1382,6 @@ func (hph *HexPatriciaHashed) updateStorage(plainKey, hashedKey, value []byte) {
|
||||
if hph.trace {
|
||||
fmt.Printf("updateStorage setting (%d, %x), touchMap[%d]=%016b, depth=%d\n", hph.activeRows-1, col, hph.activeRows-1, hph.touchMap[hph.activeRows-1], depth)
|
||||
}
|
||||
//}
|
||||
if cell.downHashedLen == 0 {
|
||||
copy(cell.downHashedKey[:], hashedKey[depth:])
|
||||
cell.downHashedLen = len(hashedKey) - depth
|
||||
@ -1428,10 +1402,6 @@ func (hph *HexPatriciaHashed) updateStorage(plainKey, hashedKey, value []byte) {
|
||||
}
|
||||
|
||||
func (hph *HexPatriciaHashed) RootHash() ([]byte, error) {
|
||||
//if hph.root.hl > 0 {
|
||||
// return hph.root.h[:hph.root.hl], nil
|
||||
//}
|
||||
|
||||
hash, err := hph.computeCellHash(&hph.root, 0, nil)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@ -1827,6 +1797,8 @@ func ReplacePlainKeys(branchData []byte, accountPlainKeys [][]byte, storagePlain
|
||||
return newData, nil
|
||||
}
|
||||
|
||||
func (hph *HexPatriciaHashed) Variant() TrieVariant { return VariantHexPatriciaTrie }
|
||||
|
||||
// IsComplete determines whether given branch data is complete, meaning that all information about all the children is present
|
||||
// All of the 16 children of a branch node have two attributes
|
||||
// touch - whether this child has been modified or deleted in this branchData (corresponding bit in touchMap is set)
|
||||
@ -1837,8 +1809,8 @@ func IsComplete(branchData []byte) bool {
|
||||
return ^touchMap&afterMap == 0
|
||||
}
|
||||
|
||||
// MergeBranches combines two branchData, number 2 coming after (and potentially shadowing) number 1
|
||||
func MergeBranches(branchData1, branchData2 []byte, newData []byte) ([]byte, error) {
|
||||
// MergeHexBranches combines two branchData, number 2 coming after (and potentially shadowing) number 1
|
||||
func MergeHexBranches(branchData1, branchData2 []byte, newData []byte) ([]byte, error) {
|
||||
touchMap1 := binary.BigEndian.Uint16(branchData1[0:])
|
||||
afterMap1 := binary.BigEndian.Uint16(branchData1[2:])
|
||||
bitmap1 := touchMap1 & afterMap1
|
||||
@ -1861,14 +1833,14 @@ func MergeBranches(branchData1, branchData2 []byte, newData []byte) ([]byte, err
|
||||
for i := 0; i < bits.OnesCount8(byte(fieldBits)); i++ {
|
||||
l, n := binary.Uvarint(branchData2[pos2:])
|
||||
if n == 0 {
|
||||
return nil, fmt.Errorf("MergeBranches buffer2 too small for field")
|
||||
return nil, fmt.Errorf("MergeHexBranches buffer2 too small for field")
|
||||
} else if n < 0 {
|
||||
return nil, fmt.Errorf("MergeBranches value2 overflow for field")
|
||||
return nil, fmt.Errorf("MergeHexBranches value2 overflow for field")
|
||||
}
|
||||
newData = append(newData, branchData2[pos2:pos2+n]...)
|
||||
pos2 += n
|
||||
if len(branchData2) < pos2+int(l) {
|
||||
return nil, fmt.Errorf("MergeBranches buffer2 too small for field")
|
||||
return nil, fmt.Errorf("MergeHexBranches buffer2 too small for field")
|
||||
}
|
||||
if l > 0 {
|
||||
newData = append(newData, branchData2[pos2:pos2+int(l)]...)
|
||||
@ -1886,16 +1858,16 @@ func MergeBranches(branchData1, branchData2 []byte, newData []byte) ([]byte, err
|
||||
for i := 0; i < bits.OnesCount8(byte(fieldBits)); i++ {
|
||||
l, n := binary.Uvarint(branchData1[pos1:])
|
||||
if n == 0 {
|
||||
return nil, fmt.Errorf("MergeBranches buffer1 too small for field")
|
||||
return nil, fmt.Errorf("MergeHexBranches buffer1 too small for field")
|
||||
} else if n < 0 {
|
||||
return nil, fmt.Errorf("MergeBranches value1 overflow for field")
|
||||
return nil, fmt.Errorf("MergeHexBranches value1 overflow for field")
|
||||
}
|
||||
if add {
|
||||
newData = append(newData, branchData1[pos1:pos1+n]...)
|
||||
}
|
||||
pos1 += n
|
||||
if len(branchData1) < pos1+int(l) {
|
||||
return nil, fmt.Errorf("MergeBranches buffer1 too small for field")
|
||||
return nil, fmt.Errorf("MergeHexBranches buffer1 too small for field")
|
||||
}
|
||||
if l > 0 {
|
||||
if add {
|
||||
|
@ -25,7 +25,6 @@ import (
|
||||
|
||||
"github.com/holiman/uint256"
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
"golang.org/x/crypto/sha3"
|
||||
|
||||
"github.com/ledgerwatch/erigon-lib/common"
|
||||
@ -180,7 +179,7 @@ func (ms *MockState) applyBranchNodeUpdates(updates map[string][]byte) {
|
||||
for key, update := range updates {
|
||||
if pre, ok := ms.cm[key]; ok {
|
||||
// Merge
|
||||
merged, err := MergeBranches(pre, update, nil)
|
||||
merged, err := MergeHexBranches(pre, update, nil)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
@ -382,12 +381,23 @@ func (ub *UpdateBuilder) Build() (plainKeys, hashedKeys [][]byte, updates []Upda
|
||||
u.Flags |= CODE_UPDATE
|
||||
copy(u.CodeHashOrStorage[:], codeHash[:])
|
||||
}
|
||||
if _, del := ub.deletes[string(key)]; del {
|
||||
u.Flags = DELETE_UPDATE
|
||||
continue
|
||||
}
|
||||
} else {
|
||||
if dm, ok1 := ub.deletes2[string(key)]; ok1 {
|
||||
if _, ok2 := dm[string(key2)]; ok2 {
|
||||
u.Flags = DELETE_UPDATE
|
||||
continue
|
||||
}
|
||||
}
|
||||
if sm, ok1 := ub.storages[string(key)]; ok1 {
|
||||
if storage, ok2 := sm[string(key2)]; ok2 {
|
||||
u.Flags |= STORAGE_UPDATE
|
||||
u.CodeHashOrStorage = [32]byte{}
|
||||
copy(u.CodeHashOrStorage[32-len(storage):], storage)
|
||||
u.ValLength = len(storage)
|
||||
copy(u.CodeHashOrStorage[:], storage)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -316,6 +316,10 @@ func (g *Getter) nextPos(clean bool) uint64 {
|
||||
|
||||
func (g *Getter) nextPattern() []byte {
|
||||
table := g.patternDict
|
||||
if table == nil {
|
||||
fmt.Printf("file %q dataP=%d data_size=%d nil table pattern dict\n", g.fName, g.dataP, len(g.data))
|
||||
return []byte{}
|
||||
}
|
||||
if table.bitLen == 0 {
|
||||
return table.patterns[0]
|
||||
}
|
||||
@ -367,7 +371,7 @@ func (g *Getter) Next(buf []byte) ([]byte, uint64) {
|
||||
savePos := g.dataP
|
||||
l := g.nextPos(true)
|
||||
l-- // because when create huffman tree we do ++ , because 0 is terminator
|
||||
if l == 0 {
|
||||
if l == 0 || l == 1<<64-1 {
|
||||
if g.dataBit > 0 {
|
||||
g.dataP++
|
||||
g.dataBit = 0
|
||||
@ -381,13 +385,23 @@ func (g *Getter) Next(buf []byte) ([]byte, uint64) {
|
||||
copy(newBuf, buf)
|
||||
buf = newBuf
|
||||
} else {
|
||||
if len(buf)+int(l) < 0 {
|
||||
buf = make([]byte, 0)
|
||||
} else {
|
||||
buf = buf[:len(buf)+int(l)]
|
||||
}
|
||||
// Expand buffer
|
||||
buf = buf[:len(buf)+int(l)]
|
||||
}
|
||||
// Loop below fills in the patterns
|
||||
for pos := g.nextPos(false /* clean */); pos != 0; pos = g.nextPos(false) {
|
||||
bufPos += int(pos) - 1 // Positions where to insert patterns are encoded relative to one another
|
||||
copy(buf[bufPos:], g.nextPattern())
|
||||
if bufPos > len(buf) {
|
||||
p := g.nextPattern()
|
||||
buf = append(buf, make([]byte, bufPos-len(buf))...)
|
||||
buf = append(buf, p...)
|
||||
} else {
|
||||
copy(buf[bufPos:], g.nextPattern())
|
||||
}
|
||||
}
|
||||
if g.dataBit > 0 {
|
||||
g.dataP++
|
||||
|
Loading…
Reference in New Issue
Block a user