Begin of moving objects to types package

* Block(s)
* Transaction(s)
This commit is contained in:
obscuren
2014-11-18 16:58:22 +01:00
parent 62cd9946ee
commit a1b6a9ac29
33 changed files with 189 additions and 151 deletions

View File

@@ -9,6 +9,7 @@ import (
"sync"
"time"
"github.com/ethereum/go-ethereum/chain/types"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/ethutil"
"github.com/ethereum/go-ethereum/event"
@@ -69,7 +70,7 @@ type BlockManager struct {
// The last attempted block is mainly used for debugging purposes
// This does not have to be a valid block and will be set during
// 'Process' & canonical validation.
lastAttemptedBlock *Block
lastAttemptedBlock *types.Block
events event.Subscription
}
@@ -117,11 +118,11 @@ func (sm *BlockManager) ChainManager() *ChainManager {
return sm.bc
}
func (self *BlockManager) ProcessTransactions(coinbase *state.StateObject, state *state.State, block, parent *Block, txs Transactions) (Receipts, Transactions, Transactions, Transactions, error) {
func (self *BlockManager) ProcessTransactions(coinbase *state.StateObject, state *state.State, block, parent *types.Block, txs types.Transactions) (types.Receipts, types.Transactions, types.Transactions, types.Transactions, error) {
var (
receipts Receipts
handled, unhandled Transactions
erroneous Transactions
receipts types.Receipts
handled, unhandled types.Transactions
erroneous types.Transactions
totalUsedGas = big.NewInt(0)
err error
)
@@ -159,8 +160,9 @@ done:
txGas.Sub(txGas, st.gas)
cumulative := new(big.Int).Set(totalUsedGas.Add(totalUsedGas, txGas))
receipt := &Receipt{ethutil.CopyBytes(state.Root()), cumulative, nil /*bloom*/, state.Logs()}
receipt.Bloom = CreateBloom(Receipts{receipt})
receipt := types.NewReceipt(state.Root(), cumulative)
receipt.SetLogs(state.Logs())
receipt.Bloom = types.CreateBloom(types.Receipts{receipt})
// Notify all subscribers
go self.eth.EventMux().Post(TxPostEvent{tx})
@@ -178,7 +180,7 @@ done:
return receipts, handled, unhandled, erroneous, err
}
func (sm *BlockManager) Process(block *Block) (td *big.Int, msgs state.Messages, err error) {
func (sm *BlockManager) Process(block *types.Block) (td *big.Int, msgs state.Messages, err error) {
// Processing a blocks may never happen simultaneously
sm.mutex.Lock()
defer sm.mutex.Unlock()
@@ -195,7 +197,7 @@ func (sm *BlockManager) Process(block *Block) (td *big.Int, msgs state.Messages,
return sm.ProcessWithParent(block, parent)
}
func (sm *BlockManager) ProcessWithParent(block, parent *Block) (td *big.Int, messages state.Messages, err error) {
func (sm *BlockManager) ProcessWithParent(block, parent *types.Block) (td *big.Int, messages state.Messages, err error) {
sm.lastAttemptedBlock = block
state := parent.State().Copy()
@@ -215,13 +217,13 @@ func (sm *BlockManager) ProcessWithParent(block, parent *Block) (td *big.Int, me
return
}
txSha := DeriveSha(block.transactions)
txSha := types.DeriveSha(block.Transactions())
if bytes.Compare(txSha, block.TxSha) != 0 {
err = fmt.Errorf("validating transaction root. received=%x got=%x", block.TxSha, txSha)
return
}
receiptSha := DeriveSha(receipts)
receiptSha := types.DeriveSha(receipts)
if bytes.Compare(receiptSha, block.ReceiptSha) != 0 {
err = fmt.Errorf("validating receipt root. received=%x got=%x", block.ReceiptSha, receiptSha)
return
@@ -238,8 +240,8 @@ func (sm *BlockManager) ProcessWithParent(block, parent *Block) (td *big.Int, me
return
}
block.receipts = receipts // although this isn't necessary it be in the future
rbloom := CreateBloom(receipts)
//block.receipts = receipts // although this isn't necessary it be in the future
rbloom := types.CreateBloom(receipts)
if bytes.Compare(rbloom, block.LogsBloom) != 0 {
err = fmt.Errorf("unable to replicate block's bloom=%x", rbloom)
return
@@ -272,7 +274,7 @@ func (sm *BlockManager) ProcessWithParent(block, parent *Block) (td *big.Int, me
}
}
func (sm *BlockManager) ApplyDiff(state *state.State, parent, block *Block) (receipts Receipts, err error) {
func (sm *BlockManager) ApplyDiff(state *state.State, parent, block *types.Block) (receipts types.Receipts, err error) {
coinbase := state.GetOrNewStateObject(block.Coinbase)
coinbase.SetGasPool(block.CalcGasLimit(parent))
@@ -285,7 +287,7 @@ func (sm *BlockManager) ApplyDiff(state *state.State, parent, block *Block) (rec
return receipts, nil
}
func (sm *BlockManager) CalculateTD(block *Block) (*big.Int, bool) {
func (sm *BlockManager) CalculateTD(block *types.Block) (*big.Int, bool) {
uncleDiff := new(big.Int)
for _, uncle := range block.Uncles {
uncleDiff = uncleDiff.Add(uncleDiff, uncle.Difficulty)
@@ -311,7 +313,7 @@ func (sm *BlockManager) CalculateTD(block *Block) (*big.Int, bool) {
// Validates the current block. Returns an error if the block was invalid,
// an uncle or anything that isn't on the current block chain.
// Validation validates easy over difficult (dagger takes longer time = difficult)
func (sm *BlockManager) ValidateBlock(block, parent *Block) error {
func (sm *BlockManager) ValidateBlock(block, parent *types.Block) error {
expd := CalcDifficulty(block, parent)
if expd.Cmp(block.Difficulty) < 0 {
return fmt.Errorf("Difficulty check failed for block %v, %v", block.Difficulty, expd)
@@ -337,7 +339,7 @@ func (sm *BlockManager) ValidateBlock(block, parent *Block) error {
return nil
}
func (sm *BlockManager) AccumelateRewards(state *state.State, block, parent *Block) error {
func (sm *BlockManager) AccumelateRewards(state *state.State, block, parent *types.Block) error {
reward := new(big.Int).Set(BlockReward)
knownUncles := ethutil.Set(parent.Uncles)
@@ -380,7 +382,7 @@ func (sm *BlockManager) AccumelateRewards(state *state.State, block, parent *Blo
return nil
}
func (sm *BlockManager) GetMessages(block *Block) (messages []*state.Message, err error) {
func (sm *BlockManager) GetMessages(block *types.Block) (messages []*state.Message, err error) {
if !sm.bc.HasBlock(block.PrevHash) {
return nil, ParentError(block.PrevHash)
}

View File

@@ -6,6 +6,7 @@ import (
"fmt"
"math/big"
"github.com/ethereum/go-ethereum/chain/types"
"github.com/ethereum/go-ethereum/ethutil"
"github.com/ethereum/go-ethereum/logger"
"github.com/ethereum/go-ethereum/state"
@@ -13,7 +14,7 @@ import (
var chainlogger = logger.NewLogger("CHAIN")
func AddTestNetFunds(block *Block) {
func AddTestNetFunds(block *types.Block) {
for _, addr := range []string{
"51ba59315b3a95761d0863b05ccc7a7f54703d99",
"e4157b34ea9615cfbde6b4fda419828124b70c78",
@@ -25,13 +26,13 @@ func AddTestNetFunds(block *Block) {
"1a26338f0d905e295fccb71fa9ea849ffa12aaf4",
} {
codedAddr := ethutil.Hex2Bytes(addr)
account := block.state.GetAccount(codedAddr)
account := block.State().GetAccount(codedAddr)
account.SetBalance(ethutil.Big("1606938044258990275541962092341162602522202993782792835301376")) //ethutil.BigPow(2, 200)
block.state.UpdateStateObject(account)
block.State().UpdateStateObject(account)
}
}
func CalcDifficulty(block, parent *Block) *big.Int {
func CalcDifficulty(block, parent *types.Block) *big.Int {
diff := new(big.Int)
adjust := new(big.Int).Rsh(parent.Difficulty, 10)
@@ -45,36 +46,41 @@ func CalcDifficulty(block, parent *Block) *big.Int {
}
type ChainManager struct {
eth EthManager
genesisBlock *Block
//eth EthManager
processor types.BlockProcessor
genesisBlock *types.Block
// Last known total difficulty
TD *big.Int
LastBlockNumber uint64
CurrentBlock *Block
CurrentBlock *types.Block
LastBlockHash []byte
workingChain *BlockChain
}
func NewChainManager(ethereum EthManager) *ChainManager {
func NewChainManager() *ChainManager {
bc := &ChainManager{}
bc.genesisBlock = NewBlockFromBytes(ethutil.Encode(Genesis))
bc.eth = ethereum
bc.genesisBlock = types.NewBlockFromBytes(ethutil.Encode(Genesis))
//bc.eth = ethereum
bc.setLastBlock()
return bc
}
func (self *ChainManager) SetProcessor(proc types.BlockProcessor) {
self.processor = proc
}
func (bc *ChainManager) setLastBlock() {
data, _ := ethutil.Config.Db.Get([]byte("LastBlock"))
if len(data) != 0 {
// Prep genesis
AddTestNetFunds(bc.genesisBlock)
block := NewBlockFromBytes(data)
block := types.NewBlockFromBytes(data)
bc.CurrentBlock = block
bc.LastBlockHash = block.Hash()
bc.LastBlockNumber = block.Number.Uint64()
@@ -89,7 +95,7 @@ func (bc *ChainManager) setLastBlock() {
}
// Block creation & chain handling
func (bc *ChainManager) NewBlock(coinbase []byte) *Block {
func (bc *ChainManager) NewBlock(coinbase []byte) *types.Block {
var root interface{}
hash := ZeroHash256
@@ -98,7 +104,7 @@ func (bc *ChainManager) NewBlock(coinbase []byte) *Block {
hash = bc.LastBlockHash
}
block := CreateBlock(
block := types.CreateBlock(
root,
hash,
coinbase,
@@ -122,7 +128,7 @@ func (bc *ChainManager) NewBlock(coinbase []byte) *Block {
func (bc *ChainManager) Reset() {
AddTestNetFunds(bc.genesisBlock)
bc.genesisBlock.state.Trie.Sync()
bc.genesisBlock.Trie().Sync()
// Prepare the genesis block
bc.add(bc.genesisBlock)
bc.CurrentBlock = bc.genesisBlock
@@ -134,7 +140,7 @@ func (bc *ChainManager) Reset() {
}
// Add a block to the chain and record addition information
func (bc *ChainManager) add(block *Block) {
func (bc *ChainManager) add(block *types.Block) {
bc.writeBlockInfo(block)
bc.CurrentBlock = block
@@ -148,7 +154,7 @@ func (bc *ChainManager) add(block *Block) {
}
// Accessors
func (bc *ChainManager) Genesis() *Block {
func (bc *ChainManager) Genesis() *types.Block {
return bc.genesisBlock
}
@@ -179,7 +185,7 @@ func (self *ChainManager) GetChainHashesFromHash(hash []byte, max uint64) (chain
return
}
func (self *ChainManager) GetBlock(hash []byte) *Block {
func (self *ChainManager) GetBlock(hash []byte) *types.Block {
data, _ := ethutil.Config.Db.Get(hash)
if len(data) == 0 {
if self.workingChain != nil {
@@ -194,10 +200,10 @@ func (self *ChainManager) GetBlock(hash []byte) *Block {
return nil
}
return NewBlockFromBytes(data)
return types.NewBlockFromBytes(data)
}
func (self *ChainManager) GetBlockByNumber(num uint64) *Block {
func (self *ChainManager) GetBlockByNumber(num uint64) *types.Block {
block := self.CurrentBlock
for ; block != nil; block = self.GetBlock(block.PrevHash) {
if block.Number.Uint64() == num {
@@ -217,7 +223,7 @@ func (bc *ChainManager) SetTotalDifficulty(td *big.Int) {
bc.TD = td
}
func (self *ChainManager) CalcTotalDiff(block *Block) (*big.Int, error) {
func (self *ChainManager) CalcTotalDiff(block *types.Block) (*big.Int, error) {
parent := self.GetBlock(block.PrevHash)
if parent == nil {
return nil, fmt.Errorf("Unable to calculate total diff without known parent %x", block.PrevHash)
@@ -237,8 +243,8 @@ func (self *ChainManager) CalcTotalDiff(block *Block) (*big.Int, error) {
return td, nil
}
func (bc *ChainManager) BlockInfo(block *Block) BlockInfo {
bi := BlockInfo{}
func (bc *ChainManager) BlockInfo(block *types.Block) types.BlockInfo {
bi := types.BlockInfo{}
data, _ := ethutil.Config.Db.Get(append(block.Hash(), []byte("Info")...))
bi.RlpDecode(data)
@@ -246,9 +252,9 @@ func (bc *ChainManager) BlockInfo(block *Block) BlockInfo {
}
// Unexported method for writing extra non-essential block info to the db
func (bc *ChainManager) writeBlockInfo(block *Block) {
func (bc *ChainManager) writeBlockInfo(block *types.Block) {
bc.LastBlockNumber++
bi := BlockInfo{Number: bc.LastBlockNumber, Hash: block.Hash(), Parent: block.PrevHash, TD: bc.TD}
bi := types.BlockInfo{Number: bc.LastBlockNumber, Hash: block.Hash(), Parent: block.PrevHash, TD: bc.TD}
// For now we use the block hash with the words "info" appended as key
ethutil.Config.Db.Put(append(block.Hash(), []byte("Info")...), bi.RlpEncode())
@@ -271,8 +277,8 @@ func (self *ChainManager) InsertChain(chain *BlockChain) {
self.add(link.block)
self.SetTotalDifficulty(link.td)
self.eth.EventMux().Post(NewBlockEvent{link.block})
self.eth.EventMux().Post(link.messages)
//self.eth.EventMux().Post(NewBlockEvent{link.block})
//self.eth.EventMux().Post(link.messages)
}
b, e := chain.Front(), chain.Back()
@@ -299,7 +305,7 @@ func (self *ChainManager) TestChain(chain *BlockChain) (td *big.Int, err error)
}
var messages state.Messages
td, messages, err = self.eth.BlockManager().ProcessWithParent(block, parent)
td, messages, err = self.processor.ProcessWithParent(block, parent) //self.eth.BlockManager().ProcessWithParent(block, parent)
if err != nil {
chainlogger.Infoln(err)
chainlogger.Debugf("Block #%v failed (%x...)\n", block.Number, block.Hash()[0:4])
@@ -323,7 +329,7 @@ func (self *ChainManager) TestChain(chain *BlockChain) (td *big.Int, err error)
}
type link struct {
block *Block
block *types.Block
messages state.Messages
td *big.Int
}
@@ -332,7 +338,7 @@ type BlockChain struct {
*list.List
}
func NewChain(blocks Blocks) *BlockChain {
func NewChain(blocks types.Blocks) *BlockChain {
chain := &BlockChain{list.New()}
for _, block := range blocks {
@@ -353,10 +359,10 @@ func (self *BlockChain) RlpEncode() []byte {
type ChainIterator struct {
cm *ChainManager
block *Block // current block in the iterator
block *types.Block // current block in the iterator
}
func (self *ChainIterator) Prev() *Block {
func (self *ChainIterator) Prev() *types.Block {
self.block = self.cm.GetBlock(self.block.PrevHash)
return self.block
}

View File

@@ -6,6 +6,7 @@ import (
"math/rand"
"time"
"github.com/ethereum/go-ethereum/chain/types"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/ethutil"
"github.com/ethereum/go-ethereum/logger"
@@ -15,7 +16,7 @@ import (
var powlogger = logger.NewLogger("POW")
type PoW interface {
Search(block *Block, stop <-chan struct{}) []byte
Search(block *types.Block, stop <-chan struct{}) []byte
Verify(hash []byte, diff *big.Int, nonce []byte) bool
GetHashrate() int64
Turbo(bool)
@@ -35,7 +36,7 @@ func (pow *EasyPow) Turbo(on bool) {
pow.turbo = on
}
func (pow *EasyPow) Search(block *Block, stop <-chan struct{}) []byte {
func (pow *EasyPow) Search(block *types.Block, stop <-chan struct{}) []byte {
r := rand.New(rand.NewSource(time.Now().UnixNano()))
hash := block.HashNoNonce()
diff := block.Difficulty

View File

@@ -1,10 +1,12 @@
package chain
import "github.com/ethereum/go-ethereum/chain/types"
// TxPreEvent is posted when a transaction enters the transaction pool.
type TxPreEvent struct{ Tx *Transaction }
type TxPreEvent struct{ Tx *types.Transaction }
// TxPostEvent is posted when a transaction has been processed.
type TxPostEvent struct{ Tx *Transaction }
type TxPostEvent struct{ Tx *types.Transaction }
// NewBlockEvent is posted when a block has been imported.
type NewBlockEvent struct{ Block *Block }
type NewBlockEvent struct{ Block *types.Block }

View File

@@ -5,6 +5,7 @@ import (
"math"
"math/big"
"github.com/ethereum/go-ethereum/chain/types"
"github.com/ethereum/go-ethereum/ethutil"
"github.com/ethereum/go-ethereum/state"
)
@@ -24,7 +25,7 @@ type Filter struct {
Altered []AccountChange
BlockCallback func(*Block)
BlockCallback func(*types.Block)
MessageCallback func(state.Messages)
}
@@ -171,11 +172,11 @@ func (self *Filter) FilterMessages(msgs []*state.Message) []*state.Message {
return messages
}
func (self *Filter) bloomFilter(block *Block) bool {
func (self *Filter) bloomFilter(block *types.Block) bool {
var fromIncluded, toIncluded bool
if len(self.from) > 0 {
for _, from := range self.from {
if BloomLookup(block.LogsBloom, from) {
if types.BloomLookup(block.LogsBloom, from) {
fromIncluded = true
break
}
@@ -186,7 +187,7 @@ func (self *Filter) bloomFilter(block *Block) bool {
if len(self.to) > 0 {
for _, to := range self.to {
if BloomLookup(block.LogsBloom, ethutil.U256(new(big.Int).Add(ethutil.Big1, ethutil.BigD(to))).Bytes()) {
if types.BloomLookup(block.LogsBloom, ethutil.U256(new(big.Int).Add(ethutil.Big1, ethutil.BigD(to))).Bytes()) {
toIncluded = true
break
}

View File

@@ -4,6 +4,7 @@ import (
"fmt"
"math/big"
"github.com/ethereum/go-ethereum/chain/types"
"github.com/ethereum/go-ethereum/state"
"github.com/ethereum/go-ethereum/vm"
)
@@ -26,17 +27,17 @@ import (
*/
type StateTransition struct {
coinbase, receiver []byte
tx *Transaction
tx *types.Transaction
gas, gasPrice *big.Int
value *big.Int
data []byte
state *state.State
block *Block
block *types.Block
cb, rec, sen *state.StateObject
}
func NewStateTransition(coinbase *state.StateObject, tx *Transaction, state *state.State, block *Block) *StateTransition {
func NewStateTransition(coinbase *state.StateObject, tx *types.Transaction, state *state.State, block *types.Block) *StateTransition {
return &StateTransition{coinbase.Address(), tx.Recipient, tx, new(big.Int), new(big.Int).Set(tx.GasPrice), tx.Value, tx.Data, state, block, coinbase, nil, nil}
}
@@ -203,7 +204,7 @@ func (self *StateTransition) TransitionState() (err error) {
})
// Process the init code and create 'valid' contract
if IsContractAddr(self.receiver) {
if types.IsContractAddr(self.receiver) {
// Evaluate the initialization script
// and use the return value as the
// script section for the state object.
@@ -280,7 +281,7 @@ func (self *StateTransition) Eval(msg *state.Message, script []byte, context *st
}
// Converts an transaction in to a state object
func MakeContract(tx *Transaction, state *state.State) *state.StateObject {
func MakeContract(tx *types.Transaction, state *state.State) *state.StateObject {
addr := tx.CreationAddress(state)
contract := state.GetOrNewStateObject(addr)

View File

@@ -7,6 +7,7 @@ import (
"math/big"
"sync"
"github.com/ethereum/go-ethereum/chain/types"
"github.com/ethereum/go-ethereum/logger"
"github.com/ethereum/go-ethereum/state"
"github.com/ethereum/go-ethereum/wire"
@@ -16,7 +17,7 @@ var txplogger = logger.NewLogger("TXP")
const txPoolQueueSize = 50
type TxPoolHook chan *Transaction
type TxPoolHook chan *types.Transaction
type TxMsgTy byte
const (
@@ -26,21 +27,21 @@ const (
var MinGasPrice = big.NewInt(10000000000000)
type TxMsg struct {
Tx *Transaction
Tx *types.Transaction
Type TxMsgTy
}
func EachTx(pool *list.List, it func(*Transaction, *list.Element) bool) {
func EachTx(pool *list.List, it func(*types.Transaction, *list.Element) bool) {
for e := pool.Front(); e != nil; e = e.Next() {
if it(e.Value.(*Transaction), e) {
if it(e.Value.(*types.Transaction), e) {
break
}
}
}
func FindTx(pool *list.List, finder func(*Transaction, *list.Element) bool) *Transaction {
func FindTx(pool *list.List, finder func(*types.Transaction, *list.Element) bool) *types.Transaction {
for e := pool.Front(); e != nil; e = e.Next() {
if tx, ok := e.Value.(*Transaction); ok {
if tx, ok := e.Value.(*types.Transaction); ok {
if finder(tx, e) {
return tx
}
@@ -51,7 +52,7 @@ func FindTx(pool *list.List, finder func(*Transaction, *list.Element) bool) *Tra
}
type TxProcessor interface {
ProcessTransaction(tx *Transaction)
ProcessTransaction(tx *types.Transaction)
}
// The tx pool a thread safe transaction pool handler. In order to
@@ -65,7 +66,7 @@ type TxPool struct {
mutex sync.Mutex
// Queueing channel for reading and writing incoming
// transactions to
queueChan chan *Transaction
queueChan chan *types.Transaction
// Quiting channel
quit chan bool
// The actual pool
@@ -79,14 +80,14 @@ type TxPool struct {
func NewTxPool(ethereum EthManager) *TxPool {
return &TxPool{
pool: list.New(),
queueChan: make(chan *Transaction, txPoolQueueSize),
queueChan: make(chan *types.Transaction, txPoolQueueSize),
quit: make(chan bool),
Ethereum: ethereum,
}
}
// Blocking function. Don't use directly. Use QueueTransaction instead
func (pool *TxPool) addTransaction(tx *Transaction) {
func (pool *TxPool) addTransaction(tx *types.Transaction) {
pool.mutex.Lock()
defer pool.mutex.Unlock()
@@ -96,7 +97,7 @@ func (pool *TxPool) addTransaction(tx *Transaction) {
pool.Ethereum.Broadcast(wire.MsgTxTy, []interface{}{tx.RlpData()})
}
func (pool *TxPool) ValidateTransaction(tx *Transaction) error {
func (pool *TxPool) ValidateTransaction(tx *types.Transaction) error {
// Get the last block so we can retrieve the sender and receiver from
// the merkle trie
block := pool.Ethereum.ChainManager().CurrentBlock
@@ -142,7 +143,7 @@ out:
select {
case tx := <-pool.queueChan:
hash := tx.Hash()
foundTx := FindTx(pool.pool, func(tx *Transaction, e *list.Element) bool {
foundTx := FindTx(pool.pool, func(tx *types.Transaction, e *list.Element) bool {
return bytes.Compare(tx.Hash(), hash) == 0
})
@@ -172,18 +173,18 @@ out:
}
}
func (pool *TxPool) QueueTransaction(tx *Transaction) {
func (pool *TxPool) QueueTransaction(tx *types.Transaction) {
pool.queueChan <- tx
}
func (pool *TxPool) CurrentTransactions() []*Transaction {
func (pool *TxPool) CurrentTransactions() []*types.Transaction {
pool.mutex.Lock()
defer pool.mutex.Unlock()
txList := make([]*Transaction, pool.pool.Len())
txList := make([]*types.Transaction, pool.pool.Len())
i := 0
for e := pool.pool.Front(); e != nil; e = e.Next() {
tx := e.Value.(*Transaction)
tx := e.Value.(*types.Transaction)
txList[i] = tx
@@ -198,7 +199,7 @@ func (pool *TxPool) RemoveInvalid(state *state.State) {
defer pool.mutex.Unlock()
for e := pool.pool.Front(); e != nil; e = e.Next() {
tx := e.Value.(*Transaction)
tx := e.Value.(*types.Transaction)
sender := state.GetAccount(tx.Sender())
err := pool.ValidateTransaction(tx)
if err != nil || sender.Nonce >= tx.Nonce {
@@ -207,12 +208,12 @@ func (pool *TxPool) RemoveInvalid(state *state.State) {
}
}
func (self *TxPool) RemoveSet(txs Transactions) {
func (self *TxPool) RemoveSet(txs types.Transactions) {
self.mutex.Lock()
defer self.mutex.Unlock()
for _, tx := range txs {
EachTx(self.pool, func(t *Transaction, element *list.Element) bool {
EachTx(self.pool, func(t *types.Transaction, element *list.Element) bool {
if t == tx {
self.pool.Remove(element)
return true // To stop the loop
@@ -222,7 +223,7 @@ func (self *TxPool) RemoveSet(txs Transactions) {
}
}
func (pool *TxPool) Flush() []*Transaction {
func (pool *TxPool) Flush() []*types.Transaction {
txList := pool.CurrentTransactions()
// Recreate a new list all together

View File

@@ -1 +0,0 @@
package chain

View File

@@ -1,4 +1,4 @@
package chain
package types
import (
"bytes"
@@ -156,7 +156,7 @@ func (block *Block) State() *state.State {
return block.state
}
func (block *Block) Transactions() []*Transaction {
func (block *Block) Transactions() Transactions {
return block.transactions
}

View File

@@ -1,4 +1,4 @@
package chain
package types
import (
"math/big"

View File

@@ -1,7 +1,8 @@
package chain
package types
import (
"testing"
"github.com/ethereum/go-ethereum/state"
)

10
chain/types/common.go Normal file
View File

@@ -0,0 +1,10 @@
package types
import (
"math/big"
"github.com/ethereum/go-ethereum/state"
)
type BlockProcessor interface {
ProcessWithParent(*Block, *Block) (*big.Int, state.Messages, error)
}

View File

@@ -1,4 +1,4 @@
package chain
package types
import (
"github.com/ethereum/go-ethereum/ethutil"

View File

@@ -1,4 +1,4 @@
package chain
package types
import (
"bytes"
@@ -16,6 +16,10 @@ type Receipt struct {
logs state.Logs
}
func NewReceipt(root []byte, cumalativeGasUsed *big.Int) *Receipt {
return &Receipt{PostState: ethutil.CopyBytes(root), CumulativeGasUsed: cumalativeGasUsed}
}
func NewRecieptFromValue(val *ethutil.Value) *Receipt {
r := &Receipt{}
r.RlpValueDecode(val)
@@ -23,6 +27,10 @@ func NewRecieptFromValue(val *ethutil.Value) *Receipt {
return r
}
func (self *Receipt) SetLogs(logs state.Logs) {
self.logs = logs
}
func (self *Receipt) RlpValueDecode(decoder *ethutil.Value) {
self.PostState = decoder.Get(0).Bytes()
self.CumulativeGasUsed = decoder.Get(1).BigInt()

View File

@@ -1,4 +1,4 @@
package chain
package types
import (
"fmt"

View File

@@ -0,0 +1 @@
package types

View File

@@ -3,17 +3,18 @@ package chain
import (
"math/big"
"github.com/ethereum/go-ethereum/chain/types"
"github.com/ethereum/go-ethereum/state"
"github.com/ethereum/go-ethereum/vm"
)
type VMEnv struct {
state *state.State
block *Block
tx *Transaction
block *types.Block
tx *types.Transaction
}
func NewEnv(state *state.State, tx *Transaction, block *Block) *VMEnv {
func NewEnv(state *state.State, tx *types.Transaction, block *types.Block) *VMEnv {
return &VMEnv{
state: state,
block: block,