Commit 49ae5385 authored by Jeffrey Wilcke's avatar Jeffrey Wilcke

Merge pull request #1405 from fjl/lean-trie

core, trie: new trie
parents 581c0901 c1a352c1
......@@ -46,9 +46,10 @@ var (
skipPrefixes = []string{
// boring stuff
"Godeps/", "tests/files/", "build/",
// don't relicense vendored packages
// don't relicense vendored sources
"crypto/sha3/", "crypto/ecies/", "logger/glog/",
"crypto/curve.go",
"trie/arc.go",
}
// paths with this prefix are licensed as GPL. all other files are LGPL.
......
......@@ -100,10 +100,8 @@ func (self *BlockProcessor) ApplyTransaction(gp GasPool, statedb *state.StateDB,
}
// Update the state with pending changes
statedb.SyncIntermediate()
usedGas.Add(usedGas, gas)
receipt := types.NewReceipt(statedb.Root().Bytes(), usedGas)
receipt := types.NewReceipt(statedb.IntermediateRoot().Bytes(), usedGas)
receipt.TxHash = tx.Hash()
receipt.GasUsed = new(big.Int).Set(gas)
if MessageCreatesContract(tx) {
......@@ -265,16 +263,16 @@ func (sm *BlockProcessor) processWithParent(block, parent *types.Block) (logs st
// Accumulate static rewards; block reward, uncle's and uncle inclusion.
AccumulateRewards(state, header, uncles)
// Commit state objects/accounts to a temporary trie (does not save)
// used to calculate the state root.
state.SyncObjects()
if header.Root != state.Root() {
err = fmt.Errorf("invalid merkle root. received=%x got=%x", header.Root, state.Root())
return
// Commit state objects/accounts to a database batch and calculate
// the state root. The database is not modified if the root
// doesn't match.
root, batch := state.CommitBatch()
if header.Root != root {
return nil, nil, fmt.Errorf("invalid merkle root: header=%x computed=%x", header.Root, root)
}
// Sync the current block's state to the database
state.Sync()
// Execute the database writes.
batch.Write()
return state.Logs(), receipts, nil
}
......
......@@ -17,6 +17,7 @@
package core
import (
"fmt"
"math/big"
"github.com/ethereum/go-ethereum/common"
......@@ -94,9 +95,9 @@ func (b *BlockGen) AddTx(tx *types.Transaction) {
if err != nil {
panic(err)
}
b.statedb.SyncIntermediate()
root := b.statedb.IntermediateRoot()
b.header.GasUsed.Add(b.header.GasUsed, gas)
receipt := types.NewReceipt(b.statedb.Root().Bytes(), b.header.GasUsed)
receipt := types.NewReceipt(root.Bytes(), b.header.GasUsed)
logs := b.statedb.GetLogs(tx.Hash())
receipt.SetLogs(logs)
receipt.Bloom = types.CreateBloom(types.Receipts{receipt})
......@@ -163,8 +164,11 @@ func GenerateChain(parent *types.Block, db ethdb.Database, n int, gen func(int,
gen(i, b)
}
AccumulateRewards(statedb, h, b.uncles)
statedb.SyncIntermediate()
h.Root = statedb.Root()
root, err := statedb.Commit()
if err != nil {
panic(fmt.Sprintf("state write error: %v", err))
}
h.Root = root
return types.NewBlock(h, b.txs, b.uncles, b.receipts)
}
for i := 0; i < n; i++ {
......@@ -184,7 +188,7 @@ func makeHeader(parent *types.Block, state *state.StateDB) *types.Header {
time = new(big.Int).Add(parent.Time(), big.NewInt(10)) // block time is fixed at 10 seconds
}
return &types.Header{
Root: state.Root(),
Root: state.IntermediateRoot(),
ParentHash: parent.Hash(),
Coinbase: parent.Coinbase(),
Difficulty: CalcDifficulty(time.Uint64(), new(big.Int).Sub(time, big.NewInt(10)).Uint64(), parent.Number(), parent.Difficulty()),
......
......@@ -840,8 +840,8 @@ out:
}
func blockErr(block *types.Block, err error) {
h := block.Header()
glog.V(logger.Error).Infof("Bad block #%v (%x)\n", h.Number, h.Hash().Bytes())
glog.V(logger.Error).Infoln(err)
glog.V(logger.Debug).Infoln(verifyNonces)
if glog.V(logger.Error) {
glog.Errorf("Bad block #%v (%s)\n", block.Number(), block.Hash().Hex())
glog.Errorf(" %v", err)
}
}
......@@ -69,7 +69,7 @@ func WriteGenesisBlock(chainDb ethdb.Database, reader io.Reader) (*types.Block,
statedb.SetState(address, common.HexToHash(key), common.HexToHash(value))
}
}
statedb.SyncObjects()
root, stateBatch := statedb.CommitBatch()
difficulty := common.String2Big(genesis.Difficulty)
block := types.NewBlock(&types.Header{
......@@ -81,7 +81,7 @@ func WriteGenesisBlock(chainDb ethdb.Database, reader io.Reader) (*types.Block,
Difficulty: difficulty,
MixDigest: common.HexToHash(genesis.Mixhash),
Coinbase: common.HexToAddress(genesis.Coinbase),
Root: statedb.Root(),
Root: root,
}, nil, nil, nil)
if block := GetBlock(chainDb, block.Hash()); block != nil {
......@@ -92,8 +92,10 @@ func WriteGenesisBlock(chainDb ethdb.Database, reader io.Reader) (*types.Block,
}
return block, nil
}
statedb.Sync()
if err := stateBatch.Write(); err != nil {
return nil, fmt.Errorf("cannot write state: %v", err)
}
if err := WriteTd(chainDb, block.Hash(), difficulty); err != nil {
return nil, err
}
......@@ -115,12 +117,14 @@ func GenesisBlockForTesting(db ethdb.Database, addr common.Address, balance *big
statedb := state.New(common.Hash{}, db)
obj := statedb.GetOrNewStateObject(addr)
obj.SetBalance(balance)
statedb.SyncObjects()
statedb.Sync()
root, err := statedb.Commit()
if err != nil {
panic(fmt.Sprintf("cannot write state: %v", err))
}
block := types.NewBlock(&types.Header{
Difficulty: params.GenesisDifficulty,
GasLimit: params.GenesisGasLimit,
Root: statedb.Root(),
Root: root,
}, nil, nil, nil)
return block
}
......
......@@ -90,15 +90,13 @@ type StateObject struct {
func NewStateObject(address common.Address, db ethdb.Database) *StateObject {
object := &StateObject{db: db, address: address, balance: new(big.Int), gasPool: new(big.Int), dirty: true}
object.trie = trie.NewSecure((common.Hash{}).Bytes(), db)
object.trie, _ = trie.NewSecure(common.Hash{}, db)
object.storage = make(Storage)
object.gasPool = new(big.Int)
return object
}
func NewStateObjectFromBytes(address common.Address, data []byte, db ethdb.Database) *StateObject {
// TODO clean me up
var extobject struct {
Nonce uint64
Balance *big.Int
......@@ -107,7 +105,13 @@ func NewStateObjectFromBytes(address common.Address, data []byte, db ethdb.Datab
}
err := rlp.Decode(bytes.NewReader(data), &extobject)
if err != nil {
fmt.Println(err)
glog.Errorf("can't decode state object %x: %v", address, err)
return nil
}
trie, err := trie.NewSecure(extobject.Root, db)
if err != nil {
// TODO: bubble this up or panic
glog.Errorf("can't create account trie with root %x: %v", extobject.Root[:], err)
return nil
}
......@@ -115,11 +119,10 @@ func NewStateObjectFromBytes(address common.Address, data []byte, db ethdb.Datab
object.nonce = extobject.Nonce
object.balance = extobject.Balance
object.codeHash = extobject.CodeHash
object.trie = trie.NewSecure(extobject.Root[:], db)
object.trie = trie
object.storage = make(map[string]common.Hash)
object.gasPool = new(big.Int)
object.code, _ = db.Get(extobject.CodeHash)
return object
}
......@@ -215,6 +218,7 @@ func (c *StateObject) ReturnGas(gas, price *big.Int) {}
func (self *StateObject) SetGasLimit(gasLimit *big.Int) {
self.gasPool = new(big.Int).Set(gasLimit)
self.dirty = true
if glog.V(logger.Core) {
glog.Infof("%x: gas (+ %v)", self.Address(), self.gasPool)
......@@ -225,19 +229,14 @@ func (self *StateObject) SubGas(gas, price *big.Int) error {
if self.gasPool.Cmp(gas) < 0 {
return GasLimitError(self.gasPool, gas)
}
self.gasPool.Sub(self.gasPool, gas)
rGas := new(big.Int).Set(gas)
rGas.Mul(rGas, price)
self.dirty = true
return nil
}
func (self *StateObject) AddGas(gas, price *big.Int) {
self.gasPool.Add(self.gasPool, gas)
self.dirty = true
}
func (self *StateObject) Copy() *StateObject {
......
......@@ -89,8 +89,7 @@ func TestNull(t *testing.T) {
//value := common.FromHex("0x823140710bf13990e4500136726d8b55")
var value common.Hash
state.SetState(address, common.Hash{}, value)
state.SyncIntermediate()
state.Sync()
state.Commit()
value = state.GetState(address, common.Hash{})
if !common.EmptyHash(value) {
t.Errorf("expected empty hash. got %x", value)
......
......@@ -35,7 +35,6 @@ import (
type StateDB struct {
db ethdb.Database
trie *trie.SecureTrie
root common.Hash
stateObjects map[string]*StateObject
......@@ -49,12 +48,19 @@ type StateDB struct {
// Create a new state from a given trie
func New(root common.Hash, db ethdb.Database) *StateDB {
trie := trie.NewSecure(root[:], db)
return &StateDB{root: root, db: db, trie: trie, stateObjects: make(map[string]*StateObject), refund: new(big.Int), logs: make(map[common.Hash]Logs)}
}
func (self *StateDB) PrintRoot() {
self.trie.Trie.PrintRoot()
tr, err := trie.NewSecure(root, db)
if err != nil {
// TODO: bubble this up
tr, _ = trie.NewSecure(common.Hash{}, db)
glog.Errorf("can't create state trie with root %x: %v", root[:], err)
}
return &StateDB{
db: db,
trie: tr,
stateObjects: make(map[string]*StateObject),
refund: new(big.Int),
logs: make(map[common.Hash]Logs),
}
}
func (self *StateDB) StartRecord(thash, bhash common.Hash, ti int) {
......@@ -196,7 +202,6 @@ func (self *StateDB) UpdateStateObject(stateObject *StateObject) {
if len(stateObject.CodeHash()) > 0 {
self.db.Put(stateObject.CodeHash(), stateObject.code)
}
addr := stateObject.Address()
self.trie.Update(addr[:], stateObject.RlpEncode())
}
......@@ -207,6 +212,7 @@ func (self *StateDB) DeleteStateObject(stateObject *StateObject) {
addr := stateObject.Address()
self.trie.Delete(addr[:])
//delete(self.stateObjects, addr.Str())
}
// Retrieve a state object given my the address. Nil if not found
......@@ -303,65 +309,67 @@ func (self *StateDB) Set(state *StateDB) {
self.logSize = state.logSize
}
func (s *StateDB) Root() common.Hash {
return common.BytesToHash(s.trie.Root())
}
// Syncs the trie and all siblings
func (s *StateDB) Sync() {
// Sync all nested states
// IntermediateRoot computes the current root hash of the state trie.
// It is called in between transactions to get the root hash that
// goes into transaction receipts.
func (s *StateDB) IntermediateRoot() common.Hash {
s.refund = new(big.Int)
for _, stateObject := range s.stateObjects {
stateObject.trie.Commit()
}
s.trie.Commit()
s.Empty()
}
func (self *StateDB) Empty() {
self.stateObjects = make(map[string]*StateObject)
self.refund = new(big.Int)
}
func (self *StateDB) Refunds() *big.Int {
return self.refund
}
// SyncIntermediate updates the intermediate state and all mid steps
func (self *StateDB) SyncIntermediate() {
self.refund = new(big.Int)
for _, stateObject := range self.stateObjects {
if stateObject.dirty {
if stateObject.remove {
self.DeleteStateObject(stateObject)
s.DeleteStateObject(stateObject)
} else {
stateObject.Update()
self.UpdateStateObject(stateObject)
s.UpdateStateObject(stateObject)
}
stateObject.dirty = false
}
}
return s.trie.Hash()
}
// SyncObjects syncs the changed objects to the trie
func (self *StateDB) SyncObjects() {
self.trie = trie.NewSecure(self.root[:], self.db)
// Commit commits all state changes to the database.
func (s *StateDB) Commit() (root common.Hash, err error) {
return s.commit(s.db)
}
self.refund = new(big.Int)
// CommitBatch commits all state changes to a write batch but does not
// execute the batch. It is used to validate state changes against
// the root hash stored in a block.
func (s *StateDB) CommitBatch() (root common.Hash, batch ethdb.Batch) {
batch = s.db.NewBatch()
root, _ = s.commit(batch)
return root, batch
}
for _, stateObject := range self.stateObjects {
func (s *StateDB) commit(db trie.DatabaseWriter) (common.Hash, error) {
s.refund = new(big.Int)
for _, stateObject := range s.stateObjects {
if stateObject.remove {
self.DeleteStateObject(stateObject)
// If the object has been removed, don't bother syncing it
// and just mark it for deletion in the trie.
s.DeleteStateObject(stateObject)
} else {
// Write any storage changes in the state object to its trie.
stateObject.Update()
self.UpdateStateObject(stateObject)
// Commit the trie of the object to the batch.
// This updates the trie root internally, so
// getting the root hash of the storage trie
// through UpdateStateObject is fast.
if _, err := stateObject.trie.CommitTo(db); err != nil {
return common.Hash{}, err
}
// Update the object in the account trie.
s.UpdateStateObject(stateObject)
}
stateObject.dirty = false
}
return s.trie.CommitTo(db)
}
func (self *StateDB) Refunds() *big.Int {
return self.refund
}
// Debug stuff
......
......@@ -17,8 +17,9 @@
package types
import (
"bytes"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/ethdb"
"github.com/ethereum/go-ethereum/rlp"
"github.com/ethereum/go-ethereum/trie"
)
......@@ -29,12 +30,12 @@ type DerivableList interface {
}
func DeriveSha(list DerivableList) common.Hash {
db, _ := ethdb.NewMemDatabase()
trie := trie.New(nil, db)
keybuf := new(bytes.Buffer)
trie := new(trie.Trie)
for i := 0; i < list.Len(); i++ {
key, _ := rlp.EncodeToBytes(uint(i))
trie.Update(key, list.GetRlp(i))
keybuf.Reset()
rlp.Encode(keybuf, uint(i))
trie.Update(keybuf.Bytes(), list.GetRlp(i))
}
return common.BytesToHash(trie.Root())
return trie.Hash()
}
......@@ -266,7 +266,6 @@ func (self *worker) wait() {
block := result.Block
work := result.Work
work.state.Sync()
if self.fullValidation {
if _, err := self.chain.InsertChain(types.Blocks{block}); err != nil {
glog.V(logger.Error).Infoln("mining err", err)
......@@ -274,6 +273,7 @@ func (self *worker) wait() {
}
go self.mux.Post(core.NewMinedBlockEvent{block})
} else {
work.state.Commit()
parent := self.chain.GetBlock(block.ParentHash())
if parent == nil {
glog.V(logger.Error).Infoln("Invalid block found during mining")
......@@ -528,8 +528,7 @@ func (self *worker) commitNewWork() {
if atomic.LoadInt32(&self.mining) == 1 {
// commit state root after all state transitions.
core.AccumulateRewards(work.state, header, uncles)
work.state.SyncObjects()
header.Root = work.state.Root()
header.Root = work.state.IntermediateRoot()
}
// create the new block whose nonce will be mined.
......
......@@ -253,13 +253,13 @@ func (t *BlockTest) InsertPreState(ethereum *eth.Ethereum) (*state.StateDB, erro
statedb.SetState(common.HexToAddress(addrString), common.HexToHash(k), common.HexToHash(v))
}
}
// sync objects to trie
statedb.SyncObjects()
// sync trie to disk
statedb.Sync()
if !bytes.Equal(t.Genesis.Root().Bytes(), statedb.Root().Bytes()) {
return nil, fmt.Errorf("computed state root does not match genesis block %x %x", t.Genesis.Root().Bytes()[:4], statedb.Root().Bytes()[:4])
root, err := statedb.Commit()
if err != nil {
return nil, fmt.Errorf("error writing state: %v", err)
}
if t.Genesis.Root() != root {
return nil, fmt.Errorf("computed state root does not match genesis block: genesis=%x computed=%x", t.Genesis.Root().Bytes()[:4], root.Bytes()[:4])
}
return statedb, nil
}
......
......@@ -201,9 +201,9 @@ func runStateTest(test VmTest) error {
}
}
statedb.Sync()
if common.HexToHash(test.PostStateRoot) != statedb.Root() {
return fmt.Errorf("Post state root error. Expected %s, got %x", test.PostStateRoot, statedb.Root())
root, _ := statedb.Commit()
if common.HexToHash(test.PostStateRoot) != root {
return fmt.Errorf("Post state root error. Expected %s, got %x", test.PostStateRoot, root)
}
// check logs
......@@ -247,7 +247,7 @@ func RunState(statedb *state.StateDB, env, tx map[string]string) ([]byte, state.
if core.IsNonceErr(err) || core.IsInvalidTxErr(err) || state.IsGasLimitErr(err) {
statedb.Set(snapshot)
}
statedb.SyncObjects()
statedb.Commit()
return ret, vmenv.state.Logs(), vmenv.Gas, err
}
// Copyright (c) 2015 Hans Alexander Gugel <alexander.gugel@gmail.com>
//
// Permission is hereby granted, free of charge, to any person obtaining a copy
// of this software and associated documentation files (the "Software"), to deal
// in the Software without restriction, including without limitation the rights
// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
// copies of the Software, and to permit persons to whom the Software is
// furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in all
// copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
// SOFTWARE.
// This file contains a modified version of package arc from
// https://github.com/alexanderGugel/arc
//
// It implements the ARC (Adaptive Replacement Cache) algorithm as detailed in
// https://www.usenix.org/legacy/event/fast03/tech/full_papers/megiddo/megiddo.pdf
package trie
import (
"container/list"
"sync"
)
type arc struct {
p int
c int
t1 *list.List
b1 *list.List
t2 *list.List
b2 *list.List
cache map[string]*entry
mutex sync.Mutex
}
type entry struct {
key hashNode
value node
ll *list.List
el *list.Element
}
// newARC returns a new Adaptive Replacement Cache with the
// given capacity.
func newARC(c int) *arc {
return &arc{
c: c,
t1: list.New(),
b1: list.New(),
t2: list.New(),
b2: list.New(),
cache: make(map[string]*entry, c),
}
}
// Put inserts a new key-value pair into the cache.
// This optimizes future access to this entry (side effect).
func (a *arc) Put(key hashNode, value node) bool {
a.mutex.Lock()
defer a.mutex.Unlock()
ent, ok := a.cache[string(key)]
if ok != true {
ent = &entry{key: key, value: value}
a.req(ent)
a.cache[string(key)] = ent
} else {
ent.value = value
a.req(ent)
}
return ok
}
// Get retrieves a previously via Set inserted entry.
// This optimizes future access to this entry (side effect).
func (a *arc) Get(key hashNode) (value node, ok bool) {
a.mutex.Lock()
defer a.mutex.Unlock()
ent, ok := a.cache[string(key)]
if ok {
a.req(ent)
return ent.value, ent.value != nil
}
return nil, false
}
func (a *arc) req(ent *entry) {
if ent.ll == a.t1 || ent.ll == a.t2 {
// Case I
ent.setMRU(a.t2)
} else if ent.ll == a.b1 {
// Case II
// Cache Miss in t1 and t2
// Adaptation
var d int
if a.b1.Len() >= a.b2.Len() {
d = 1
} else {
d = a.b2.Len() / a.b1.Len()
}
a.p = a.p + d
if a.p > a.c {
a.p = a.c
}
a.replace(ent)
ent.setMRU(a.t2)
} else if ent.ll == a.b2 {
// Case III
// Cache Miss in t1 and t2
// Adaptation
var d int
if a.b2.Len() >= a.b1.Len() {
d = 1
} else {
d = a.b1.Len() / a.b2.Len()
}
a.p = a.p - d
if a.p < 0 {
a.p = 0
}
a.replace(ent)
ent.setMRU(a.t2)
} else if ent.ll == nil {
// Case IV
if a.t1.Len()+a.b1.Len() == a.c {
// Case A
if a.t1.Len() < a.c {
a.delLRU(a.b1)
a.replace(ent)
} else {
a.delLRU(a.t1)
}
} else if a.t1.Len()+a.b1.Len() < a.c {
// Case B
if a.t1.Len()+a.t2.Len()+a.b1.Len()+a.b2.Len() >= a.c {
if a.t1.Len()+a.t2.Len()+a.b1.Len()+a.b2.Len() == 2*a.c {
a.delLRU(a.b2)
}
a.replace(ent)
}
}
ent.setMRU(a.t1)
}
}
func (a *arc) delLRU(list *list.List) {
lru := list.Back()
list.Remove(lru)
delete(a.cache, string(lru.Value.(*entry).key))
}
func (a *arc) replace(ent *entry) {
if a.t1.Len() > 0 && ((a.t1.Len() > a.p) || (ent.ll == a.b2 && a.t1.Len() == a.p)) {
lru := a.t1.Back().Value.(*entry)
lru.value = nil
lru.setMRU(a.b1)
} else {
lru := a.t2.Back().Value.(*entry)
lru.value = nil
lru.setMRU(a.b2)
}
}
func (e *entry) setLRU(list *list.List) {
e.detach()
e.ll = list
e.el = e.ll.PushBack(e)
}
func (e *entry) setMRU(list *list.List) {
e.detach()
e.ll = list
e.el = e.ll.PushFront(e)
}
func (e *entry) detach() {
if e.ll != nil {
e.ll.Remove(e.el)
}
}
......@@ -16,34 +16,36 @@
package trie
func CompactEncode(hexSlice []byte) []byte {
terminator := 0
func compactEncode(hexSlice []byte) []byte {
terminator := byte(0)
if hexSlice[len(hexSlice)-1] == 16 {
terminator = 1
}
if terminator == 1 {
hexSlice = hexSlice[:len(hexSlice)-1]
}
oddlen := len(hexSlice) % 2
flags := byte(2*terminator + oddlen)
if oddlen != 0 {
hexSlice = append([]byte{flags}, hexSlice...)
} else {
hexSlice = append([]byte{flags, 0}, hexSlice...)
var (
odd = byte(len(hexSlice) % 2)
buflen = len(hexSlice)/2 + 1
bi, hi = 0, 0 // indices
hs = byte(0) // shift: flips between 0 and 4
)
if odd == 0 {
bi = 1
hs = 4
}
l := len(hexSlice) / 2
var buf = make([]byte, l)
for i := 0; i < l; i++ {
buf[i] = 16*hexSlice[2*i] + hexSlice[2*i+1]
buf := make([]byte, buflen)
buf[0] = terminator<<5 | byte(odd)<<4
for bi < len(buf) && hi < len(hexSlice) {
buf[bi] |= hexSlice[hi] << hs
if hs == 0 {
bi++
}
hi, hs = hi+1, hs^(1<<2)
}
return buf
}
func CompactDecode(str []byte) []byte {
base := CompactHexDecode(str)
func compactDecode(str []byte) []byte {
base := compactHexDecode(str)
base = base[:len(base)-1]
if base[0] >= 2 {
base = append(base, 16)
......@@ -53,11 +55,10 @@ func CompactDecode(str []byte) []byte {
} else {
base = base[2:]
}
return base
}
func CompactHexDecode(str []byte) []byte {
func compactHexDecode(str []byte) []byte {
l := len(str)*2 + 1
var nibbles = make([]byte, l)
for i, b := range str {
......@@ -68,7 +69,7 @@ func CompactHexDecode(str []byte) []byte {
return nibbles
}
func DecodeCompact(key []byte) []byte {
func decodeCompact(key []byte) []byte {
l := len(key) / 2
var res = make([]byte, l)
for i := 0; i < l; i++ {
......@@ -77,3 +78,30 @@ func DecodeCompact(key []byte) []byte {
}
return res
}
// prefixLen returns the length of the common prefix of a and b.
func prefixLen(a, b []byte) int {
var i, length = 0, len(a)
if len(b) < length {
length = len(b)
}
for ; i < length; i++ {
if a[i] != b[i] {
break
}
}
return i
}
func hasTerm(s []byte) bool {
return s[len(s)-1] == 16
}
func remTerm(s []byte) []byte {
if hasTerm(s) {
b := make([]byte, len(s)-1)
copy(b, s)
return b
}
return s
}
......@@ -23,7 +23,7 @@ import (
checker "gopkg.in/check.v1"
)
func Test(t *testing.T) { checker.TestingT(t) }
func TestEncoding(t *testing.T) { checker.TestingT(t) }
type TrieEncodingSuite struct{}
......@@ -32,64 +32,64 @@ var _ = checker.Suite(&TrieEncodingSuite{})
func (s *TrieEncodingSuite) TestCompactEncode(c *checker.C) {
// even compact encode
test1 := []byte{1, 2, 3, 4, 5}
res1 := CompactEncode(test1)
res1 := compactEncode(test1)
c.Assert(res1, checker.DeepEquals, []byte("\x11\x23\x45"))
// odd compact encode
test2 := []byte{0, 1, 2, 3, 4, 5}
res2 := CompactEncode(test2)
res2 := compactEncode(test2)
c.Assert(res2, checker.DeepEquals, []byte("\x00\x01\x23\x45"))
//odd terminated compact encode
test3 := []byte{0, 15, 1, 12, 11, 8 /*term*/, 16}
res3 := CompactEncode(test3)
res3 := compactEncode(test3)
c.Assert(res3, checker.DeepEquals, []byte("\x20\x0f\x1c\xb8"))
// even terminated compact encode
test4 := []byte{15, 1, 12, 11, 8 /*term*/, 16}
res4 := CompactEncode(test4)
res4 := compactEncode(test4)
c.Assert(res4, checker.DeepEquals, []byte("\x3f\x1c\xb8"))
}
func (s *TrieEncodingSuite) TestCompactHexDecode(c *checker.C) {
exp := []byte{7, 6, 6, 5, 7, 2, 6, 2, 16}
res := CompactHexDecode([]byte("verb"))
res := compactHexDecode([]byte("verb"))
c.Assert(res, checker.DeepEquals, exp)
}
func (s *TrieEncodingSuite) TestCompactDecode(c *checker.C) {
// odd compact decode
exp := []byte{1, 2, 3, 4, 5}
res := CompactDecode([]byte("\x11\x23\x45"))
res := compactDecode([]byte("\x11\x23\x45"))
c.Assert(res, checker.DeepEquals, exp)
// even compact decode
exp = []byte{0, 1, 2, 3, 4, 5}
res = CompactDecode([]byte("\x00\x01\x23\x45"))
res = compactDecode([]byte("\x00\x01\x23\x45"))
c.Assert(res, checker.DeepEquals, exp)
// even terminated compact decode
exp = []byte{0, 15, 1, 12, 11, 8 /*term*/, 16}
res = CompactDecode([]byte("\x20\x0f\x1c\xb8"))
res = compactDecode([]byte("\x20\x0f\x1c\xb8"))
c.Assert(res, checker.DeepEquals, exp)
// even terminated compact decode
exp = []byte{15, 1, 12, 11, 8 /*term*/, 16}
res = CompactDecode([]byte("\x3f\x1c\xb8"))
res = compactDecode([]byte("\x3f\x1c\xb8"))
c.Assert(res, checker.DeepEquals, exp)
}
func (s *TrieEncodingSuite) TestDecodeCompact(c *checker.C) {
exp, _ := hex.DecodeString("012345")
res := DecodeCompact([]byte{0, 1, 2, 3, 4, 5})
res := decodeCompact([]byte{0, 1, 2, 3, 4, 5})
c.Assert(res, checker.DeepEquals, exp)
exp, _ = hex.DecodeString("012345")
res = DecodeCompact([]byte{0, 1, 2, 3, 4, 5, 16})
res = decodeCompact([]byte{0, 1, 2, 3, 4, 5, 16})
c.Assert(res, checker.DeepEquals, exp)
exp, _ = hex.DecodeString("abcdef")
res = DecodeCompact([]byte{10, 11, 12, 13, 14, 15})
res = decodeCompact([]byte{10, 11, 12, 13, 14, 15})
c.Assert(res, checker.DeepEquals, exp)
}
......@@ -97,29 +97,27 @@ func BenchmarkCompactEncode(b *testing.B) {
testBytes := []byte{0, 15, 1, 12, 11, 8 /*term*/, 16}
for i := 0; i < b.N; i++ {
CompactEncode(testBytes)
compactEncode(testBytes)
}
}
func BenchmarkCompactDecode(b *testing.B) {
testBytes := []byte{0, 15, 1, 12, 11, 8 /*term*/, 16}
for i := 0; i < b.N; i++ {
CompactDecode(testBytes)
compactDecode(testBytes)
}
}
func BenchmarkCompactHexDecode(b *testing.B) {
testBytes := []byte{7, 6, 6, 5, 7, 2, 6, 2, 16}
for i := 0; i < b.N; i++ {
CompactHexDecode(testBytes)
compactHexDecode(testBytes)
}
}
func BenchmarkDecodeCompact(b *testing.B) {
testBytes := []byte{7, 6, 6, 5, 7, 2, 6, 2, 16}
for i := 0; i < b.N; i++ {
DecodeCompact(testBytes)
decodeCompact(testBytes)
}
}
// Copyright 2014 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package trie
type FullNode struct {
trie *Trie
nodes [17]Node
dirty bool
}
func NewFullNode(t *Trie) *FullNode {
return &FullNode{trie: t}
}
func (self *FullNode) Dirty() bool { return self.dirty }
func (self *FullNode) Value() Node {
self.nodes[16] = self.trie.trans(self.nodes[16])
return self.nodes[16]
}
func (self *FullNode) Branches() []Node {
return self.nodes[:16]
}
func (self *FullNode) Copy(t *Trie) Node {
nnode := NewFullNode(t)
for i, node := range self.nodes {
if node != nil {
nnode.nodes[i] = node
}
}
nnode.dirty = true
return nnode
}
// Returns the length of non-nil nodes
func (self *FullNode) Len() (amount int) {
for _, node := range self.nodes {
if node != nil {
amount++
}
}
return
}
func (self *FullNode) Hash() interface{} {
return self.trie.store(self)
}
func (self *FullNode) RlpData() interface{} {
t := make([]interface{}, 17)
for i, node := range self.nodes {
if node != nil {
t[i] = node.Hash()
} else {
t[i] = ""
}
}
return t
}
func (self *FullNode) set(k byte, value Node) {
self.nodes[int(k)] = value
self.dirty = true
}
func (self *FullNode) branch(i byte) Node {
if self.nodes[int(i)] != nil {
self.nodes[int(i)] = self.trie.trans(self.nodes[int(i)])
return self.nodes[int(i)]
}
return nil
}
func (self *FullNode) setDirty(dirty bool) {
self.dirty = dirty
}
// Copyright 2014 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package trie
import "github.com/ethereum/go-ethereum/common"
type HashNode struct {
key []byte
trie *Trie
dirty bool
}
func NewHash(key []byte, trie *Trie) *HashNode {
return &HashNode{key, trie, false}
}
func (self *HashNode) RlpData() interface{} {
return self.key
}
func (self *HashNode) Hash() interface{} {
return self.key
}
func (self *HashNode) setDirty(dirty bool) {
self.dirty = dirty
}
// These methods will never be called but we have to satisfy Node interface
func (self *HashNode) Value() Node { return nil }
func (self *HashNode) Dirty() bool { return true }
func (self *HashNode) Copy(t *Trie) Node { return NewHash(common.CopyBytes(self.key), t) }
......@@ -16,9 +16,7 @@
package trie
import (
"bytes"
)
import "bytes"
type Iterator struct {
trie *Trie
......@@ -32,32 +30,29 @@ func NewIterator(trie *Trie) *Iterator {
}
func (self *Iterator) Next() bool {
self.trie.mu.Lock()
defer self.trie.mu.Unlock()
isIterStart := false
if self.Key == nil {
isIterStart = true
self.Key = make([]byte, 32)
}
key := RemTerm(CompactHexDecode(self.Key))
key := remTerm(compactHexDecode(self.Key))
k := self.next(self.trie.root, key, isIterStart)
self.Key = []byte(DecodeCompact(k))
self.Key = []byte(decodeCompact(k))
return len(k) > 0
}
func (self *Iterator) next(node Node, key []byte, isIterStart bool) []byte {
func (self *Iterator) next(node interface{}, key []byte, isIterStart bool) []byte {
if node == nil {
return nil
}
switch node := node.(type) {
case *FullNode:
case fullNode:
if len(key) > 0 {
k := self.next(node.branch(key[0]), key[1:], isIterStart)
k := self.next(node[key[0]], key[1:], isIterStart)
if k != nil {
return append([]byte{key[0]}, k...)
}
......@@ -69,31 +64,31 @@ func (self *Iterator) next(node Node, key []byte, isIterStart bool) []byte {
}
for i := r; i < 16; i++ {
k := self.key(node.branch(byte(i)))
k := self.key(node[i])
if k != nil {
return append([]byte{i}, k...)
}
}
case *ShortNode:
k := RemTerm(node.Key())
if vnode, ok := node.Value().(*ValueNode); ok {
case shortNode:
k := remTerm(node.Key)
if vnode, ok := node.Val.(valueNode); ok {
switch bytes.Compare([]byte(k), key) {
case 0:
if isIterStart {
self.Value = vnode.Val()
self.Value = vnode
return k
}
case 1:
self.Value = vnode.Val()
self.Value = vnode
return k
}
} else {
cnode := node.Value()
cnode := node.Val
var ret []byte
skey := key[len(k):]
if BeginsWith(key, k) {
if bytes.HasPrefix(key, k) {
ret = self.next(cnode, skey, isIterStart)
} else if bytes.Compare(k, key[:len(k)]) > 0 {
return self.key(node)
......@@ -103,37 +98,36 @@ func (self *Iterator) next(node Node, key []byte, isIterStart bool) []byte {
return append(k, ret...)
}
}
}
case hashNode:
return self.next(self.trie.resolveHash(node), key, isIterStart)
}
return nil
}
func (self *Iterator) key(node Node) []byte {
func (self *Iterator) key(node interface{}) []byte {
switch node := node.(type) {
case *ShortNode:
case shortNode:
// Leaf node
if vnode, ok := node.Value().(*ValueNode); ok {
k := RemTerm(node.Key())
self.Value = vnode.Val()
k := remTerm(node.Key)
if vnode, ok := node.Val.(valueNode); ok {
self.Value = vnode
return k
} else {
k := RemTerm(node.Key())
return append(k, self.key(node.Value())...)
}
case *FullNode:
if node.Value() != nil {
self.Value = node.Value().(*ValueNode).Val()
return append(k, self.key(node.Val)...)
case fullNode:
if node[16] != nil {
self.Value = node[16].(valueNode)
return []byte{16}
}
for i := 0; i < 16; i++ {
k := self.key(node.branch(byte(i)))
k := self.key(node[i])
if k != nil {
return append([]byte{byte(i)}, k...)
}
}
case hashNode:
return self.key(self.trie.resolveHash(node))
}
return nil
......
......@@ -19,7 +19,7 @@ package trie
import "testing"
func TestIterator(t *testing.T) {
trie := NewEmpty()
trie := newEmpty()
vals := []struct{ k, v string }{
{"do", "verb"},
{"ether", "wookiedoo"},
......@@ -32,11 +32,11 @@ func TestIterator(t *testing.T) {
v := make(map[string]bool)
for _, val := range vals {
v[val.k] = false
trie.UpdateString(val.k, val.v)
trie.Update([]byte(val.k), []byte(val.v))
}
trie.Commit()
it := trie.Iterator()
it := NewIterator(trie)
for it.Next() {
v[string(it.Key)] = true
}
......
......@@ -16,46 +16,172 @@
package trie
import "fmt"
import (
"fmt"
"io"
"strings"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/rlp"
)
var indices = []string{"0", "1", "2", "3", "4", "5", "6", "7", "8", "9", "a", "b", "c", "d", "e", "f", "[17]"}
type Node interface {
Value() Node
Copy(*Trie) Node // All nodes, for now, return them self
Dirty() bool
type node interface {
fstring(string) string
Hash() interface{}
RlpData() interface{}
setDirty(dirty bool)
}
// Value node
func (self *ValueNode) String() string { return self.fstring("") }
func (self *FullNode) String() string { return self.fstring("") }
func (self *ShortNode) String() string { return self.fstring("") }
func (self *ValueNode) fstring(ind string) string { return fmt.Sprintf("%x ", self.data) }
type (
fullNode [17]node
shortNode struct {
Key []byte
Val node
}
hashNode []byte
valueNode []byte
)
//func (self *HashNode) fstring(ind string) string { return fmt.Sprintf("< %x > ", self.key) }
func (self *HashNode) fstring(ind string) string {
return fmt.Sprintf("%v", self.trie.trans(self))
}
// Pretty printing.
func (n fullNode) String() string { return n.fstring("") }
func (n shortNode) String() string { return n.fstring("") }
func (n hashNode) String() string { return n.fstring("") }
func (n valueNode) String() string { return n.fstring("") }
// Full node
func (self *FullNode) fstring(ind string) string {
func (n fullNode) fstring(ind string) string {
resp := fmt.Sprintf("[\n%s ", ind)
for i, node := range self.nodes {
for i, node := range n {
if node == nil {
resp += fmt.Sprintf("%s: <nil> ", indices[i])
} else {
resp += fmt.Sprintf("%s: %v", indices[i], node.fstring(ind+" "))
}
}
return resp + fmt.Sprintf("\n%s] ", ind)
}
func (n shortNode) fstring(ind string) string {
return fmt.Sprintf("{%x: %v} ", n.Key, n.Val.fstring(ind+" "))
}
func (n hashNode) fstring(ind string) string {
return fmt.Sprintf("<%x> ", []byte(n))
}
func (n valueNode) fstring(ind string) string {
return fmt.Sprintf("%x ", []byte(n))
}
func mustDecodeNode(dbkey, buf []byte) node {
n, err := decodeNode(buf)
if err != nil {
panic(fmt.Sprintf("node %x: %v", dbkey, err))
}
return n
}
// decodeNode parses the RLP encoding of a trie node.
func decodeNode(buf []byte) (node, error) {
if len(buf) == 0 {
return nil, io.ErrUnexpectedEOF
}
elems, _, err := rlp.SplitList(buf)
if err != nil {
return nil, fmt.Errorf("decode error: %v", err)
}
switch c, _ := rlp.CountValues(elems); c {
case 2:
n, err := decodeShort(elems)
return n, wrapError(err, "short")
case 17:
n, err := decodeFull(elems)
return n, wrapError(err, "full")
default:
return nil, fmt.Errorf("invalid number of list elements: %v", c)
}
}
func decodeShort(buf []byte) (node, error) {
kbuf, rest, err := rlp.SplitString(buf)
if err != nil {
return nil, err
}
key := compactDecode(kbuf)
if key[len(key)-1] == 16 {
// value node
val, _, err := rlp.SplitString(rest)
if err != nil {
return nil, fmt.Errorf("invalid value node: %v", err)
}
return shortNode{key, valueNode(val)}, nil
}
r, _, err := decodeRef(rest)
if err != nil {
return nil, wrapError(err, "val")
}
return shortNode{key, r}, nil
}
func decodeFull(buf []byte) (fullNode, error) {
var n fullNode
for i := 0; i < 16; i++ {
cld, rest, err := decodeRef(buf)
if err != nil {
return n, wrapError(err, fmt.Sprintf("[%d]", i))
}
n[i], buf = cld, rest
}
val, _, err := rlp.SplitString(buf)
if err != nil {
return n, err
}
if len(val) > 0 {
n[16] = valueNode(val)
}
return n, nil
}
const hashLen = len(common.Hash{})
func decodeRef(buf []byte) (node, []byte, error) {
kind, val, rest, err := rlp.Split(buf)
if err != nil {
return nil, buf, err
}
switch {
case kind == rlp.List:
// 'embedded' node reference. The encoding must be smaller
// than a hash in order to be valid.
if size := len(buf) - len(rest); size > hashLen {
err := fmt.Errorf("oversized embedded node (size is %d bytes, want size < %d)", size, hashLen)
return nil, buf, err
}
n, err := decodeNode(buf)
return n, rest, err
case kind == rlp.String && len(val) == 0:
// empty node
return nil, rest, nil
case kind == rlp.String && len(val) == 32:
return hashNode(val), rest, nil
default:
return nil, nil, fmt.Errorf("invalid RLP string size %d (want 0 or 32)", len(val))
}
}
// wraps a decoding error with information about the path to the
// invalid child node (for debugging encoding issues).
type decodeError struct {
what error
stack []string
}
func wrapError(err error, ctx string) error {
if err == nil {
return nil
}
if decErr, ok := err.(*decodeError); ok {
decErr.stack = append(decErr.stack, ctx)
return decErr
}
return &decodeError{err, []string{ctx}}
}
// Short node
func (self *ShortNode) fstring(ind string) string {
return fmt.Sprintf("[ %x: %v ] ", self.key, self.value.fstring(ind+" "))
func (err *decodeError) Error() string {
return fmt.Sprintf("%v (decode path: %s)", err.what, strings.Join(err.stack, "<-"))
}
package trie
import (
"bytes"
"errors"
"fmt"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/crypto/sha3"
"github.com/ethereum/go-ethereum/rlp"
)
// Prove constructs a merkle proof for key. The result contains all
// encoded nodes on the path to the value at key. The value itself is
// also included in the last node and can be retrieved by verifying
// the proof.
//
// The returned proof is nil if the trie does not contain a value for key.
// For existing keys, the proof will have at least one element.
func (t *Trie) Prove(key []byte) []rlp.RawValue {
// Collect all nodes on the path to key.
key = compactHexDecode(key)
nodes := []node{}
tn := t.root
for len(key) > 0 {
switch n := tn.(type) {
case shortNode:
if len(key) < len(n.Key) || !bytes.Equal(n.Key, key[:len(n.Key)]) {
// The trie doesn't contain the key.
return nil
}
tn = n.Val
key = key[len(n.Key):]
nodes = append(nodes, n)
case fullNode:
tn = n[key[0]]
key = key[1:]
nodes = append(nodes, n)
case nil:
return nil
case hashNode:
tn = t.resolveHash(n)
default:
panic(fmt.Sprintf("%T: invalid node: %v", tn, tn))
}
}
if t.hasher == nil {
t.hasher = newHasher()
}
proof := make([]rlp.RawValue, 0, len(nodes))
for i, n := range nodes {
// Don't bother checking for errors here since hasher panics
// if encoding doesn't work and we're not writing to any database.
n, _ = t.hasher.replaceChildren(n, nil)
hn, _ := t.hasher.store(n, nil, false)
if _, ok := hn.(hashNode); ok || i == 0 {
// If the node's database encoding is a hash (or is the
// root node), it becomes a proof element.
enc, _ := rlp.EncodeToBytes(n)
proof = append(proof, enc)
}
}
return proof
}
// VerifyProof checks merkle proofs. The given proof must contain the
// value for key in a trie with the given root hash. VerifyProof
// returns an error if the proof contains invalid trie nodes or the
// wrong value.
func VerifyProof(rootHash common.Hash, key []byte, proof []rlp.RawValue) (value []byte, err error) {
key = compactHexDecode(key)
sha := sha3.NewKeccak256()
wantHash := rootHash.Bytes()
for i, buf := range proof {
sha.Reset()
sha.Write(buf)
if !bytes.Equal(sha.Sum(nil), wantHash) {
return nil, fmt.Errorf("bad proof node %d: hash mismatch", i)
}
n, err := decodeNode(buf)
if err != nil {
return nil, fmt.Errorf("bad proof node %d: %v", i, err)
}
keyrest, cld := get(n, key)
switch cld := cld.(type) {
case nil:
return nil, fmt.Errorf("key mismatch at proof node %d", i)
case hashNode:
key = keyrest
wantHash = cld
case valueNode:
if i != len(proof)-1 {
return nil, errors.New("additional nodes at end of proof")
}
return cld, nil
}
}
return nil, errors.New("unexpected end of proof")
}
func get(tn node, key []byte) ([]byte, node) {
for len(key) > 0 {
switch n := tn.(type) {
case shortNode:
if len(key) < len(n.Key) || !bytes.Equal(n.Key, key[:len(n.Key)]) {
return nil, nil
}
tn = n.Val
key = key[len(n.Key):]
case fullNode:
tn = n[key[0]]
key = key[1:]
case hashNode:
return key, n
case nil:
return key, nil
default:
panic(fmt.Sprintf("%T: invalid node: %v", tn, tn))
}
}
return nil, tn.(valueNode)
}
package trie
import (
"bytes"
crand "crypto/rand"
mrand "math/rand"
"testing"
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/rlp"
)
func init() {
mrand.Seed(time.Now().Unix())
}
func TestProof(t *testing.T) {
trie, vals := randomTrie(500)
root := trie.Hash()
for _, kv := range vals {
proof := trie.Prove(kv.k)
if proof == nil {
t.Fatalf("missing key %x while constructing proof", kv.k)
}
val, err := VerifyProof(root, kv.k, proof)
if err != nil {
t.Fatalf("VerifyProof error for key %x: %v\nraw proof: %x", kv.k, err, proof)
}
if !bytes.Equal(val, kv.v) {
t.Fatalf("VerifyProof returned wrong value for key %x: got %x, want %x", kv.k, val, kv.v)
}
}
}
func TestOneElementProof(t *testing.T) {
trie := new(Trie)
updateString(trie, "k", "v")
proof := trie.Prove([]byte("k"))
if proof == nil {
t.Fatal("nil proof")
}
if len(proof) != 1 {
t.Error("proof should have one element")
}
val, err := VerifyProof(trie.Hash(), []byte("k"), proof)
if err != nil {
t.Fatalf("VerifyProof error: %v\nraw proof: %x", err, proof)
}
if !bytes.Equal(val, []byte("v")) {
t.Fatalf("VerifyProof returned wrong value: got %x, want 'k'", val)
}
}
func TestVerifyBadProof(t *testing.T) {
trie, vals := randomTrie(800)
root := trie.Hash()
for _, kv := range vals {
proof := trie.Prove(kv.k)
if proof == nil {
t.Fatal("nil proof")
}
mutateByte(proof[mrand.Intn(len(proof))])
if _, err := VerifyProof(root, kv.k, proof); err == nil {
t.Fatalf("expected proof to fail for key %x", kv.k)
}
}
}
// mutateByte changes one byte in b.
func mutateByte(b []byte) {
for r := mrand.Intn(len(b)); ; {
new := byte(mrand.Intn(255))
if new != b[r] {
b[r] = new
break
}
}
}
func BenchmarkProve(b *testing.B) {
trie, vals := randomTrie(100)
var keys []string
for k := range vals {
keys = append(keys, k)
}
b.ResetTimer()
for i := 0; i < b.N; i++ {
kv := vals[keys[i%len(keys)]]
if trie.Prove(kv.k) == nil {
b.Fatalf("nil proof for %x", kv.k)
}
}
}
func BenchmarkVerifyProof(b *testing.B) {
trie, vals := randomTrie(100)
root := trie.Hash()
var keys []string
var proofs [][]rlp.RawValue
for k := range vals {
keys = append(keys, k)
proofs = append(proofs, trie.Prove([]byte(k)))
}
b.ResetTimer()
for i := 0; i < b.N; i++ {
im := i % len(keys)
if _, err := VerifyProof(root, []byte(keys[im]), proofs[im]); err != nil {
b.Fatalf("key %x: error", keys[im], err)
}
}
}
func randomTrie(n int) (*Trie, map[string]*kv) {
trie := new(Trie)
vals := make(map[string]*kv)
for i := byte(0); i < 100; i++ {
value := &kv{common.LeftPadBytes([]byte{i}, 32), []byte{i}, false}
value2 := &kv{common.LeftPadBytes([]byte{i + 10}, 32), []byte{i}, false}
trie.Update(value.k, value.v)
trie.Update(value2.k, value2.v)
vals[string(value.k)] = value
vals[string(value2.k)] = value2
}
for i := 0; i < n; i++ {
value := &kv{randBytes(32), randBytes(20), false}
trie.Update(value.k, value.v)
vals[string(value.k)] = value
}
return trie, vals
}
func randBytes(n int) []byte {
r := make([]byte, n)
crand.Read(r)
return r
}
......@@ -16,46 +16,93 @@
package trie
import "github.com/ethereum/go-ethereum/crypto"
import (
"hash"
var keyPrefix = []byte("secure-key-")
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/crypto/sha3"
)
var secureKeyPrefix = []byte("secure-key-")
// SecureTrie wraps a trie with key hashing. In a secure trie, all
// access operations hash the key using keccak256. This prevents
// calling code from creating long chains of nodes that
// increase the access time.
//
// Contrary to a regular trie, a SecureTrie can only be created with
// New and must have an attached database. The database also stores
// the preimage of each key.
//
// SecureTrie is not safe for concurrent use.
type SecureTrie struct {
*Trie
}
func NewSecure(root []byte, backend Backend) *SecureTrie {
return &SecureTrie{New(root, backend)}
hash hash.Hash
secKeyBuf []byte
hashKeyBuf []byte
}
func (self *SecureTrie) Update(key, value []byte) Node {
shaKey := crypto.Sha3(key)
self.Trie.cache.Put(append(keyPrefix, shaKey...), key)
return self.Trie.Update(shaKey, value)
}
func (self *SecureTrie) UpdateString(key, value string) Node {
return self.Update([]byte(key), []byte(value))
// NewSecure creates a trie with an existing root node from db.
//
// If root is the zero hash or the sha3 hash of an empty string, the
// trie is initially empty. Otherwise, New will panics if db is nil
// and returns ErrMissingRoot if the root node cannpt be found.
// Accessing the trie loads nodes from db on demand.
func NewSecure(root common.Hash, db Database) (*SecureTrie, error) {
if db == nil {
panic("NewSecure called with nil database")
}
trie, err := New(root, db)
if err != nil {
return nil, err
}
return &SecureTrie{Trie: trie}, nil
}
func (self *SecureTrie) Get(key []byte) []byte {
return self.Trie.Get(crypto.Sha3(key))
// Get returns the value for key stored in the trie.
// The value bytes must not be modified by the caller.
func (t *SecureTrie) Get(key []byte) []byte {
return t.Trie.Get(t.hashKey(key))
}
func (self *SecureTrie) GetString(key string) []byte {
return self.Get([]byte(key))
// Update associates key with value in the trie. Subsequent calls to
// Get will return value. If value has length zero, any existing value
// is deleted from the trie and calls to Get will return nil.
//
// The value bytes must not be modified by the caller while they are
// stored in the trie.
func (t *SecureTrie) Update(key, value []byte) {
hk := t.hashKey(key)
t.Trie.Update(hk, value)
t.Trie.db.Put(t.secKey(hk), key)
}
func (self *SecureTrie) Delete(key []byte) Node {
return self.Trie.Delete(crypto.Sha3(key))
// Delete removes any existing value for key from the trie.
func (t *SecureTrie) Delete(key []byte) {
t.Trie.Delete(t.hashKey(key))
}
func (self *SecureTrie) DeleteString(key string) Node {
return self.Delete([]byte(key))
// GetKey returns the sha3 preimage of a hashed key that was
// previously used to store a value.
func (t *SecureTrie) GetKey(shaKey []byte) []byte {
key, _ := t.Trie.db.Get(t.secKey(shaKey))
return key
}
func (self *SecureTrie) Copy() *SecureTrie {
return &SecureTrie{self.Trie.Copy()}
func (t *SecureTrie) secKey(key []byte) []byte {
t.secKeyBuf = append(t.secKeyBuf[:0], secureKeyPrefix...)
t.secKeyBuf = append(t.secKeyBuf, key...)
return t.secKeyBuf
}
func (self *SecureTrie) GetKey(shaKey []byte) []byte {
return self.Trie.cache.Get(append(keyPrefix, shaKey...))
func (t *SecureTrie) hashKey(key []byte) []byte {
if t.hash == nil {
t.hash = sha3.NewKeccak256()
t.hashKeyBuf = make([]byte, 32)
}
t.hash.Reset()
t.hash.Write(key)
t.hashKeyBuf = t.hash.Sum(t.hashKeyBuf[:0])
return t.hashKeyBuf
}
// Copyright 2014 The go-ethereum Authors
// Copyright 2015 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
......@@ -17,62 +17,58 @@
package trie
import (
"bytes"
"testing"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/ethdb"
"github.com/ethereum/go-ethereum/logger/glog"
"github.com/syndtr/goleveldb/leveldb"
)
type Backend interface {
Get([]byte) ([]byte, error)
Put([]byte, []byte) error
}
type Cache struct {
batch *leveldb.Batch
store map[string][]byte
backend Backend
}
func NewCache(backend Backend) *Cache {
return &Cache{new(leveldb.Batch), make(map[string][]byte), backend}
func newEmptySecure() *SecureTrie {
db, _ := ethdb.NewMemDatabase()
trie, _ := NewSecure(common.Hash{}, db)
return trie
}
func (self *Cache) Get(key []byte) []byte {
data := self.store[string(key)]
if data == nil {
data, _ = self.backend.Get(key)
func TestSecureDelete(t *testing.T) {
trie := newEmptySecure()
vals := []struct{ k, v string }{
{"do", "verb"},
{"ether", "wookiedoo"},
{"horse", "stallion"},
{"shaman", "horse"},
{"doge", "coin"},
{"ether", ""},
{"dog", "puppy"},
{"shaman", ""},
}
return data
}
func (self *Cache) Put(key []byte, data []byte) {
self.batch.Put(key, data)
self.store[string(key)] = data
}
// Flush flushes the trie to the backing layer. If this is a leveldb instance
// we'll use a batched write, otherwise we'll use regular put.
func (self *Cache) Flush() {
if db, ok := self.backend.(*ethdb.LDBDatabase); ok {
if err := db.LDB().Write(self.batch, nil); err != nil {
glog.Fatal("db write err:", err)
}
} else {
for k, v := range self.store {
self.backend.Put([]byte(k), v)
for _, val := range vals {
if val.v != "" {
trie.Update([]byte(val.k), []byte(val.v))
} else {
trie.Delete([]byte(val.k))
}
}
}
func (self *Cache) Copy() *Cache {
cache := NewCache(self.backend)
for k, v := range self.store {
cache.store[k] = v
hash := trie.Hash()
exp := common.HexToHash("29b235a58c3c25ab83010c327d5932bcf05324b7d6b1185e650798034783ca9d")
if hash != exp {
t.Errorf("expected %x got %x", exp, hash)
}
return cache
}
func (self *Cache) Reset() {
//self.store = make(map[string][]byte)
func TestSecureGetKey(t *testing.T) {
trie := newEmptySecure()
trie.Update([]byte("foo"), []byte("bar"))
key := []byte("foo")
value := []byte("bar")
seckey := crypto.Sha3(key)
if !bytes.Equal(trie.Get(key), value) {
t.Errorf("Get did not return bar")
}
if k := trie.GetKey(seckey); !bytes.Equal(k, key) {
t.Errorf("GetKey returned %q, want %q", k, key)
}
}
// Copyright 2014 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package trie
import "github.com/ethereum/go-ethereum/common"
type ShortNode struct {
trie *Trie
key []byte
value Node
dirty bool
}
func NewShortNode(t *Trie, key []byte, value Node) *ShortNode {
return &ShortNode{t, CompactEncode(key), value, false}
}
func (self *ShortNode) Value() Node {
self.value = self.trie.trans(self.value)
return self.value
}
func (self *ShortNode) Dirty() bool { return self.dirty }
func (self *ShortNode) Copy(t *Trie) Node {
node := &ShortNode{t, nil, self.value.Copy(t), self.dirty}
node.key = common.CopyBytes(self.key)
node.dirty = true
return node
}
func (self *ShortNode) RlpData() interface{} {
return []interface{}{self.key, self.value.Hash()}
}
func (self *ShortNode) Hash() interface{} {
return self.trie.store(self)
}
func (self *ShortNode) Key() []byte {
return CompactDecode(self.key)
}
func (self *ShortNode) setDirty(dirty bool) {
self.dirty = dirty
}
// Copyright 2014 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package trie
import (
"bytes"
"math"
)
// Helper function for comparing slices
func CompareIntSlice(a, b []int) bool {
if len(a) != len(b) {
return false
}
for i, v := range a {
if v != b[i] {
return false
}
}
return true
}
// Returns the amount of nibbles that match each other from 0 ...
func MatchingNibbleLength(a, b []byte) int {
var i, length = 0, int(math.Min(float64(len(a)), float64(len(b))))
for i < length {
if a[i] != b[i] {
break
}
i++
}
return i
}
func HasTerm(s []byte) bool {
return s[len(s)-1] == 16
}
func RemTerm(s []byte) []byte {
if HasTerm(s) {
return s[:len(s)-1]
}
return s
}
func BeginsWith(a, b []byte) bool {
if len(b) > len(a) {
return false
}
return bytes.Equal(a[:len(b)], b)
}
This diff is collapsed.
This diff is collapsed.
// Copyright 2014 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package trie
import "github.com/ethereum/go-ethereum/common"
type ValueNode struct {
trie *Trie
data []byte
dirty bool
}
func NewValueNode(trie *Trie, data []byte) *ValueNode {
return &ValueNode{trie, data, false}
}
func (self *ValueNode) Value() Node { return self } // Best not to call :-)
func (self *ValueNode) Val() []byte { return self.data }
func (self *ValueNode) Dirty() bool { return self.dirty }
func (self *ValueNode) Copy(t *Trie) Node {
return &ValueNode{t, common.CopyBytes(self.data), self.dirty}
}
func (self *ValueNode) RlpData() interface{} { return self.data }
func (self *ValueNode) Hash() interface{} { return self.data }
func (self *ValueNode) setDirty(dirty bool) {
self.dirty = dirty
}
Markdown is supported
0% or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment